From 1021ea9002dc7250c000512db7a8dec92f462404 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Fri, 13 Jan 2012 19:53:32 +0000 Subject: [PATCH 001/151] KAFKA 244 Improve log4j appender to use kafka.producer.Producer; patched by vtkstef; reviewed by nehanarkhede git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1231276 13f79535-47bb-0310-9956-ffa450edef68 --- .../kafka/producer/KafkaLog4jAppender.scala | 83 +++++++----- .../scala/kafka/producer/ProducerConfig.scala | 4 + .../kafka/log4j/KafkaLog4jAppenderTest.scala | 128 +++++++++++++----- 3 files changed, 148 insertions(+), 67 deletions(-) diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index 414f57941db51..b1955c77e46fb 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -18,66 +18,77 @@ package kafka.producer import async.MissingConfigException -import org.apache.log4j.spi.LoggingEvent +import org.apache.log4j.spi.{LoggingEvent, ErrorCode} import org.apache.log4j.{Logger, AppenderSkeleton} +import org.apache.log4j.helpers.LogLog import kafka.utils.{Utils, Logging} import kafka.serializer.Encoder import java.util.{Properties, Date} import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} +import scala.collection._ class KafkaLog4jAppender extends AppenderSkeleton with Logging { var port:Int = 0 var host:String = null var topic:String = null - var encoderClass:String = null + var serializerClass:String = null + var zkConnect:String = null + var brokerList:String = null - private var producer:SyncProducer = null - private var encoder: Encoder[AnyRef] = null - - def getPort:Int = port - def setPort(port: Int) = { this.port = port } - - def getHost:String = host - def setHost(host: String) = { this.host = host } + private var producer: Producer[String, String] = null def getTopic:String = topic - def setTopic(topic: String) = { this.topic = topic } + def setTopic(topic: String) { this.topic = topic } - def getEncoder:String = encoderClass - def setEncoder(encoder: String) = { this.encoderClass = encoder } + def getZkConnect:String = zkConnect + def setZkConnect(zkConnect: String) { this.zkConnect = zkConnect } - override def activateOptions = { + def getBrokerList:String = brokerList + def setBrokerList(brokerList: String) { this.brokerList = brokerList } + + def getSerializerClass:String = serializerClass + def setSerializerClass(serializerClass:String) { this.serializerClass = serializerClass } + + override def activateOptions() { + val connectDiagnostic : mutable.ListBuffer[String] = mutable.ListBuffer(); // check for config parameter validity - if(host == null) - throw new MissingConfigException("Broker Host must be specified by the Kafka log4j appender") - if(port == 0) - throw new MissingConfigException("Broker Port must be specified by the Kafka log4j appender") + val props = new Properties() + if( zkConnect == null) connectDiagnostic += "zkConnect" + else props.put("zk.connect", zkConnect); + if( brokerList == null) connectDiagnostic += "brokerList" + else if( props.isEmpty) props.put("broker.list", brokerList) + if(props.isEmpty ) + throw new MissingConfigException( + connectDiagnostic mkString ("One of these connection properties must be specified: ", ", ", ".") + ) if(topic == null) throw new MissingConfigException("topic must be specified by the Kafka log4j appender") - if(encoderClass == null) { - info("Using default encoder - kafka.producer.DefaultStringEncoder") - encoder = Utils.getObject("kafka.producer.DefaultStringEncoder") - }else // instantiate the encoder, if present - encoder = Utils.getObject(encoderClass) - val props = new Properties() - props.put("host", host) - props.put("port", port.toString) - producer = new SyncProducer(new SyncProducerConfig(props)) - info("Kafka producer connected to " + host + "," + port) - info("Logging for topic: " + topic) + if(serializerClass == null) { + serializerClass = "kafka.serializer.StringEncoder" + LogLog.warn("Using default encoder - kafka.serializer.StringEncoder") + } + props.put("serializer.class", serializerClass) + val config : ProducerConfig = new ProducerConfig(props) + producer = new Producer[String, String](config) + LogLog.debug("Kafka producer connected to " + (if(config.zkConnect == null) config.brokerList else config.zkConnect)) + LogLog.debug("Logging for topic: " + topic) } - override def append(event: LoggingEvent) = { - debug("[" + new Date(event.getTimeStamp).toString + "]" + event.getRenderedMessage + - " for " + host + "," + port) - val message = encoder.toMessage(event) - producer.send(topic, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message)) + override def append(event: LoggingEvent) { + val message : String = if( this.layout == null) { + event.getRenderedMessage + } + else this.layout.format(event) + LogLog.debug("[" + new Date(event.getTimeStamp).toString + "]" + message) + val messageData : ProducerData[String, String] = + new ProducerData[String, String](topic, message) + producer.send(messageData); } - override def close = { + override def close() { if(!this.closed) { this.closed = true - producer.close + producer.close() } } diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index b62143940e0af..3cdbdb4b546f0 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -32,6 +32,10 @@ class ProducerConfig(val props: Properties) extends ZKConfig(props) if(brokerList != null && Utils.getString(props, "partitioner.class", null) != null) throw new InvalidConfigException("partitioner.class cannot be used when broker.list is set") + /** If both broker.list and zk.connect options are specified, throw an exception */ + if(brokerList != null && zkConnect != null) + throw new InvalidConfigException("only one of broker.list and zk.connect can be specified") + /** the partitioner class for partitioning events amongst sub-topics */ val partitionerClass = Utils.getString(props, "partitioner.class", "kafka.producer.DefaultPartitioner") diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala index d0b9d0cedc6ad..f134b3e8e19f6 100644 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala @@ -24,7 +24,8 @@ import java.io.File import kafka.consumer.SimpleConsumer import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.TestUtils -import kafka.utils.{Utils, Logging} +import kafka.utils.{TestUtils, TestZKUtils,Utils, Logging} +import kafka.zk.EmbeddedZookeeper import junit.framework.Assert._ import kafka.api.FetchRequest import kafka.serializer.Encoder @@ -35,30 +36,58 @@ import org.junit.{After, Before, Test} class KafkaLog4jAppenderTest extends JUnitSuite with Logging { - var logDir: File = null + var logDirZk: File = null + var logDirBl: File = null // var topicLogDir: File = null - var server: KafkaServer = null - val brokerPort: Int = 9092 - var simpleConsumer: SimpleConsumer = null + var serverBl: KafkaServer = null + var serverZk: KafkaServer = null + + var simpleConsumerZk: SimpleConsumer = null + var simpleConsumerBl: SimpleConsumer = null + val tLogger = Logger.getLogger(getClass()) + private val brokerZk = 0 + private val brokerBl = 1 + + private val ports = TestUtils.choosePorts(2) + private val (portZk, portBl) = (ports(0), ports(1)) + + private var zkServer:EmbeddedZookeeper = null + @Before def setUp() { - val config: Properties = createBrokerConfig(1, brokerPort) - val logDirPath = config.getProperty("log.dir") - logDir = new File(logDirPath) + zkServer = new EmbeddedZookeeper(TestZKUtils.zookeeperConnect) + + val propsZk = TestUtils.createBrokerConfig(brokerZk, portZk) + val logDirZkPath = propsZk.getProperty("log.dir") + logDirZk = new File(logDirZkPath) + serverZk = TestUtils.createServer(new KafkaConfig(propsZk)); + + val propsBl: Properties = createBrokerConfig(brokerBl, portBl) + val logDirBlPath = propsBl.getProperty("log.dir") + logDirBl = new File(logDirBlPath) + serverBl = TestUtils.createServer(new KafkaConfig(propsBl)) - server = TestUtils.createServer(new KafkaConfig(config)) Thread.sleep(100) - simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024) + + simpleConsumerZk = new SimpleConsumer("localhost", portZk, 1000000, 64*1024) + simpleConsumerBl = new SimpleConsumer("localhost", portBl, 1000000, 64*1024) } @After def tearDown() { - simpleConsumer.close - server.shutdown - Thread.sleep(100) - Utils.rm(logDir) + simpleConsumerZk.close + simpleConsumerBl.close + + serverZk.shutdown + serverBl.shutdown + Utils.rm(logDirZk) + Utils.rm(logDirBl) + + Thread.sleep(500) + zkServer.shutdown + Thread.sleep(500) } @Test @@ -66,9 +95,10 @@ class KafkaLog4jAppenderTest extends JUnitSuite with Logging { var props = new Properties() props.put("log4j.rootLogger", "INFO") props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.Host", "localhost") + props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") + props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.appender.KAFKA.encoder", "kafka.log4j.AppenderStringEncoder") + props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder") props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") // port missing @@ -82,9 +112,10 @@ class KafkaLog4jAppenderTest extends JUnitSuite with Logging { props = new Properties() props.put("log4j.rootLogger", "INFO") props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") + props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") + props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.appender.KAFKA.Encoder", "kafka.log4j.AppenderStringEncoder") - props.put("log4j.appender.KAFKA.Port", "9092") + props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder") props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") // host missing @@ -98,9 +129,10 @@ class KafkaLog4jAppenderTest extends JUnitSuite with Logging { props = new Properties() props.put("log4j.rootLogger", "INFO") props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.Host", "localhost") - props.put("log4j.appender.KAFKA.Port", "9092") - props.put("log4j.appender.KAFKA.Encoder", "kafka.log4j.AppenderStringEncoder") + props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") + props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") + props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder") + props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString) props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") // topic missing @@ -114,22 +146,44 @@ class KafkaLog4jAppenderTest extends JUnitSuite with Logging { props = new Properties() props.put("log4j.rootLogger", "INFO") props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.Host", "localhost") + props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") + props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") + props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString) props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.appender.KAFKA.Port", "9092") props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") // serializer missing try { PropertyConfigurator.configure(props) }catch { - case e: MissingConfigException => fail("should default to kafka.producer.DefaultStringEncoder") + case e: MissingConfigException => fail("should default to kafka.serializer.StringEncoder") + } + } + + @Test + def testBrokerListLog4jAppends() { + PropertyConfigurator.configure(getLog4jConfigWithBrokerList) + + for(i <- 1 to 5) + info("test") + + Thread.sleep(500) + + var offset = 0L + val messages = simpleConsumerBl.fetch(new FetchRequest("test-topic", 0, offset, 1024*1024)) + + var count = 0 + for(message <- messages) { + count = count + 1 + offset += message.offset } + + assertEquals(5, count) } @Test - def testLog4jAppends() { - PropertyConfigurator.configure(getLog4jConfig) + def testZkConnectLog4jAppends() { + PropertyConfigurator.configure(getLog4jConfigWithZkConnect) for(i <- 1 to 5) info("test") @@ -137,7 +191,7 @@ class KafkaLog4jAppenderTest extends JUnitSuite with Logging { Thread.sleep(500) var offset = 0L - val messages = simpleConsumer.fetch(new FetchRequest("test-topic", 0, offset, 1024*1024)) + val messages = simpleConsumerZk.fetch(new FetchRequest("test-topic", 0, offset, 1024*1024)) var count = 0 for(message <- messages) { @@ -148,15 +202,27 @@ class KafkaLog4jAppenderTest extends JUnitSuite with Logging { assertEquals(5, count) } + private def getLog4jConfigWithBrokerList: Properties = { + var props = new Properties() + props.put("log4j.rootLogger", "INFO") + props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") + props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") + props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") + props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString) + props.put("log4j.appender.KAFKA.Topic", "test-topic") + props.put("log4j.logger.kafka.log4j", "INFO,KAFKA") + props + } - private def getLog4jConfig: Properties = { + private def getLog4jConfigWithZkConnect: Properties = { var props = new Properties() props.put("log4j.rootLogger", "INFO") props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender") - props.put("log4j.appender.KAFKA.Port", "9092") - props.put("log4j.appender.KAFKA.Host", "localhost") + props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout") + props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n") + props.put("log4j.appender.KAFKA.ZkConnect", TestZKUtils.zookeeperConnect) props.put("log4j.appender.KAFKA.Topic", "test-topic") - props.put("log4j.logger.kafka.log4j", "INFO, KAFKA") + props.put("log4j.logger.kafka.log4j", "INFO,KAFKA") props } From d3c58b64287b73ff9613e53e293d4a0989389cb2 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 17 Jan 2012 17:30:38 +0000 Subject: [PATCH 002/151] max.message.size and fetch.size defaults should be consistent; patched by Pierre-Yves Ritschard; reviewed by Jun Rao; KAFKA-247 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1232500 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/consumer/ConsumerConfig.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 31196ff92e569..c4b3e71b94947 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -24,7 +24,7 @@ import kafka.common.InvalidConfigException object ConsumerConfig { val SocketTimeout = 30 * 1000 val SocketBufferSize = 64*1024 - val FetchSize = 300 * 1024 + val FetchSize = 1024 * 1024 val MaxFetchSize = 10*FetchSize val DefaultFetcherBackoffMs = 1000 val AutoCommit = true @@ -62,9 +62,6 @@ class ConsumerConfig(props: Properties) extends ZKConfig(props) { /** the number of byes of messages to attempt to fetch */ val fetchSize = Utils.getInt(props, "fetch.size", FetchSize) - /** the maximum allowable fetch size for a very large message */ - val maxFetchSize: Int = fetchSize * 10 - /** to avoid repeatedly polling a broker node which has no new data we will backoff every time we get an empty set from the broker*/ val fetcherBackoffMs: Long = Utils.getInt(props, "fetcher.backoff.ms", DefaultFetcherBackoffMs) From fe2f0557130b21aca203ced11bd6990322847b97 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 19 Jan 2012 17:53:32 +0000 Subject: [PATCH 003/151] ConsumerIterator throws a IllegalStateException after a ConsumerTimeout occurs; patched by Jun Rao; reviewed by Joel Koshy; KAFKA-241 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1233501 13f79535-47bb-0310-9956-ffa450edef68 --- .../kafka/consumer/ConsumerIterator.scala | 2 ++ .../scala/kafka/utils/IteratorTemplate.scala | 5 ++++- .../ZookeeperConsumerConnectorTest.scala | 21 ++++++++++++------- 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index 35fb06deab786..cb959190cffab 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -58,6 +58,8 @@ class ConsumerIterator[T](private val topic: String, else { currentDataChunk = channel.poll(consumerTimeoutMs, TimeUnit.MILLISECONDS) if (currentDataChunk == null) { + // reset state to make the iterator re-iterable + resetState() throw new ConsumerTimeoutException } } diff --git a/core/src/main/scala/kafka/utils/IteratorTemplate.scala b/core/src/main/scala/kafka/utils/IteratorTemplate.scala index 660e0551b1e6f..3f110c35a4317 100644 --- a/core/src/main/scala/kafka/utils/IteratorTemplate.scala +++ b/core/src/main/scala/kafka/utils/IteratorTemplate.scala @@ -72,6 +72,9 @@ abstract class IteratorTemplate[T] extends Iterator[T] with java.util.Iterator[T def remove = throw new UnsupportedOperationException("Removal not supported") - + + protected def resetState() { + state = NOT_READY + } } diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index f6001ec24d6ac..e255b7d4be8a9 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -63,15 +63,20 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar } val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true) val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2)) - try { - getMessages(nMessages*2, topicMessageStreams0) - fail("should get an exception") - } - catch { - case e: ConsumerTimeoutException => // this is ok - println("This is ok") - case e => throw e + + // no messages to consume, we should hit timeout; + // also the iterator should support re-entrant, so loop it twice + for (i <- 0 until 2) { + try { + getMessages(nMessages*2, topicMessageStreams0) + fail("should get an exception") + } + catch { + case e: ConsumerTimeoutException => // this is ok + case e => throw e + } } + zkConsumerConnector0.shutdown // send some messages to each broker From fcc82dce6aff8fee30ce8a5ada3c6167aef1462f Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 2 Feb 2012 17:33:46 +0000 Subject: [PATCH 004/151] Corrupted request shuts down the broker; patched by Jun Rao; reviewed by Jay Kreps and Neha Narkhede; KAFKA-261 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1239740 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/log/Log.scala | 16 ++++++++++++---- .../kafka/server/KafkaRequestHandlers.scala | 8 -------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index b25a8081abf48..2dd1113dd53b4 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -211,10 +211,18 @@ private[log] class Log(val dir: File, val maxSize: Long, val flushInterval: Int, // they are valid, insert them in the log lock synchronized { - val segment = segments.view.last - segment.messageSet.append(messages) - maybeFlush(numberOfMessages) - maybeRoll(segment) + try { + val segment = segments.view.last + segment.messageSet.append(messages) + maybeFlush(numberOfMessages) + maybeRoll(segment) + } + catch { + case e: IOException => + fatal("Halting due to unrecoverable I/O error while handling producer request", e) + Runtime.getRuntime.halt(1) + case e2 => throw e2 + } } } diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala b/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala index 8d5da7cc2a0d2..2317372140061 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala @@ -25,7 +25,6 @@ import kafka.api._ import kafka.common.ErrorMapping import kafka.utils.SystemTime import kafka.utils.Logging -import java.io.IOException /** * Logic to handle the various Kafka requests @@ -74,15 +73,8 @@ private[kafka] class KafkaRequestHandlers(val logManager: LogManager) extends Lo catch { case e => error("Error processing " + requestHandlerName + " on " + request.topic + ":" + partition, e) - e match { - case _: IOException => - fatal("Halting due to unrecoverable I/O error while handling producer request: " + e.getMessage, e) - Runtime.getRuntime.halt(1) - case _ => - } throw e } - None } def handleFetchRequest(request: Receive): Option[Send] = { From afd4b0d772f35f902a7acf81039abcc47b983d1f Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Thu, 2 Feb 2012 19:07:48 +0000 Subject: [PATCH 005/151] KAFKA 256 Bug in the consumer rebalancing logic leads to the consumer not pulling data from some partitions; patched by nehanarkhede; reviewed by joelkoshy git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1239766 13f79535-47bb-0310-9956-ffa450edef68 --- .../scala/kafka/api/MultiFetchRequest.scala | 2 - .../src/main/scala/kafka/cluster/Broker.scala | 4 - .../main/scala/kafka/cluster/Cluster.scala | 3 +- .../scala/kafka/common/ErrorMapping.scala | 1 - .../kafka/consumer/ConsoleConsumer.scala | 2 - .../main/scala/kafka/consumer/Fetcher.scala | 9 +- .../scala/kafka/consumer/TopicCount.scala | 4 +- .../consumer/ZookeeperConsumerConnector.scala | 209 +++++++++++------- .../consumer/ZookeeperTopicEventWatcher.scala | 2 +- .../main/scala/kafka/javaapi/Implicits.scala | 4 +- .../kafka/javaapi/message/MessageSet.scala | 3 +- .../kafka/javaapi/producer/Producer.scala | 1 - .../message/ByteBufferBackedInputStream.scala | 1 - .../kafka/message/ByteBufferMessageSet.scala | 1 - .../kafka/message/CompressionUtils.scala | 4 +- .../scala/kafka/message/FileMessageSet.scala | 2 - .../main/scala/kafka/message/Message.scala | 3 - .../scala/kafka/network/SocketServer.scala | 1 - .../kafka/network/SocketServerStats.scala | 2 - .../kafka/producer/ConsoleProducer.scala | 4 - .../kafka/producer/KafkaLog4jAppender.scala | 2 +- .../kafka/producer/SyncProducerConfig.scala | 1 - .../main/scala/kafka/server/KafkaServer.scala | 1 - .../kafka/server/MultiMessageSetSend.scala | 3 - core/src/main/scala/kafka/tools/JmxTool.scala | 2 +- .../scala/kafka/tools/ProducerShell.scala | 3 - .../scala/kafka/tools/ReplayLogProducer.scala | 8 +- .../kafka/tools/SimpleConsumerShell.scala | 1 - .../kafka/tools/VerifyConsumerRebalance.scala | 137 ++++++++++++ .../scala/kafka/utils/KafkaScheduler.scala | 1 - core/src/main/scala/kafka/utils/Range.scala | 3 +- .../scala/kafka/utils/UpdateOffsetsInZK.scala | 7 +- core/src/main/scala/kafka/utils/Utils.scala | 1 - core/src/main/scala/kafka/utils/ZkUtils.scala | 39 ++++ core/src/test/resources/log4j.properties | 3 +- system_test/broker_failure/bin/run-test.sh | 18 +- .../broker_failure/config/log4j.properties | 4 +- 37 files changed, 356 insertions(+), 140 deletions(-) create mode 100644 core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala diff --git a/core/src/main/scala/kafka/api/MultiFetchRequest.scala b/core/src/main/scala/kafka/api/MultiFetchRequest.scala index cd8dba423d345..6ecc619f47362 100644 --- a/core/src/main/scala/kafka/api/MultiFetchRequest.scala +++ b/core/src/main/scala/kafka/api/MultiFetchRequest.scala @@ -19,8 +19,6 @@ package kafka.api import java.nio._ import kafka.network._ -import kafka.utils._ -import kafka.api._ object MultiFetchRequest { def readFrom(buffer: ByteBuffer): MultiFetchRequest = { diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 8d12be54ca0e7..be44b48943e5d 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -17,11 +17,7 @@ package kafka.cluster -import java.util.Arrays import kafka.utils._ -import java.net.InetAddress -import kafka.server.KafkaConfig -import util.parsing.json.JSON /** * A Kafka broker diff --git a/core/src/main/scala/kafka/cluster/Cluster.scala b/core/src/main/scala/kafka/cluster/Cluster.scala index a517fa42f5c5f..992c54e0368b3 100644 --- a/core/src/main/scala/kafka/cluster/Cluster.scala +++ b/core/src/main/scala/kafka/cluster/Cluster.scala @@ -17,7 +17,6 @@ package kafka.cluster -import kafka.utils._ import scala.collection._ /** @@ -33,7 +32,7 @@ private[kafka] class Cluster { brokers.put(broker.id, broker) } - def getBroker(id: Int) = brokers.get(id).get + def getBroker(id: Int): Option[Broker] = brokers.get(id) def add(broker: Broker) = brokers.put(broker.id, broker) diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index 3c036b06dba2d..31614585eb018 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -17,7 +17,6 @@ package kafka.common -import kafka.consumer._ import kafka.message.InvalidMessageException import java.nio.ByteBuffer import java.lang.Throwable diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala index 0d4d83605835d..43683d5375b90 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -21,13 +21,11 @@ import scala.collection.mutable._ import scala.collection.JavaConversions._ import org.I0Itec.zkclient._ import joptsimple._ -import java.util.Arrays.asList import java.util.Properties import java.util.Random import java.io.PrintStream import kafka.message._ import kafka.utils.{Utils, Logging} -import kafka.utils.ZkUtils import kafka.utils.ZKStringSerializer /** diff --git a/core/src/main/scala/kafka/consumer/Fetcher.scala b/core/src/main/scala/kafka/consumer/Fetcher.scala index f0f771f661590..d18faca85b03f 100644 --- a/core/src/main/scala/kafka/consumer/Fetcher.scala +++ b/core/src/main/scala/kafka/consumer/Fetcher.scala @@ -22,6 +22,7 @@ import kafka.cluster._ import org.I0Itec.zkclient.ZkClient import java.util.concurrent.BlockingQueue import kafka.utils._ +import java.lang.IllegalStateException /** * The fetcher is a background thread that fetches data from a set of servers @@ -73,7 +74,13 @@ private [consumer] class Fetcher(val config: ConsumerConfig, val zkClient : ZkCl // open a new fetcher thread for each broker val ids = Set() ++ topicInfos.map(_.brokerId) - val brokers = ids.map(cluster.getBroker(_)) + val brokers = ids.map { id => + cluster.getBroker(id) match { + case Some(broker) => broker + case None => throw new IllegalStateException("Broker " + id + " is unavailable, fetchers could not be started") + } + } + fetcherThreads = new Array[FetcherRunnable](brokers.size) var i = 0 for(broker <- brokers) { diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala index b3680c0ebc75b..51bf516ee0590 100644 --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ b/core/src/main/scala/kafka/consumer/TopicCount.scala @@ -21,7 +21,7 @@ import scala.collection._ import scala.util.parsing.json.JSON import kafka.utils.Logging -private[consumer] object TopicCount extends Logging { +private[kafka] object TopicCount extends Logging { val myConversionFunc = {input : String => input.toInt} JSON.globalNumberParser = myConversionFunc @@ -44,7 +44,7 @@ private[consumer] object TopicCount extends Logging { } -private[consumer] class TopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) { +private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) { def getConsumerThreadIdsPerTopic() : Map[String, Set[String]] = { diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 7b9804fce04ee..221d2a56e9a63 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -5,7 +5,7 @@ * 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 @@ -30,6 +30,8 @@ import kafka.api.OffsetRequest import java.util.UUID import kafka.serializer.Decoder import kafka.common.{ConsumerRebalanceFailedException, InvalidConfigException} +import java.lang.IllegalStateException +import kafka.utils.ZkUtils._ /** * This class handles the consumers interaction with zookeeper @@ -157,7 +159,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, var consumerUuid : String = null config.consumerId match { - case Some(consumerId) // for testing only + case Some(consumerId) // for testing only => consumerUuid = consumerId case None // generate unique consumerId automatically => val uuid = UUID.randomUUID() @@ -193,7 +195,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, ret.foreach { topicAndStreams => // register on broker partition path changes - val partitionPath = ZkUtils.BrokerTopicsPath + "/" + topicAndStreams._1 + val partitionPath = BrokerTopicsPath + "/" + topicAndStreams._1 zkClient.subscribeChildChanges(partitionPath, loadBalancerListener) } @@ -204,7 +206,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = { info("begin registering consumer " + consumerIdString + " in ZK") - ZkUtils.createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString) + createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString) info("end registering consumer " + consumerIdString + " in ZK") } @@ -239,7 +241,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, for (info <- infos.values) { val newOffset = info.getConsumeOffset try { - ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partition.name, + updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partition.name, newOffset.toString) } catch { @@ -289,7 +291,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, try { val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) val znode = topicDirs.consumerOffsetDir + "/" + partition.name - val offsetString = ZkUtils.readDataMaybeNull(zkClient, znode) + val offsetString = readDataMaybeNull(zkClient, znode) if (offsetString != null) return offsetString.toLong else @@ -309,8 +311,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, var simpleConsumer: SimpleConsumer = null var producedOffset: Long = -1L try { - val cluster = ZkUtils.getCluster(zkClient) - val broker = cluster.getBroker(brokerId) + val cluster = getCluster(zkClient) + val broker = cluster.getBroker(brokerId) match { + case Some(b) => b + case None => throw new IllegalStateException("Broker " + brokerId + " is unavailable. Cannot issue " + + "getOffsetsBefore request") + } simpleConsumer = new SimpleConsumer(broker.host, broker.port, ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize) val offsets = simpleConsumer.getOffsetsBefore(topic, partitionId, earliestOrLatest, 1) @@ -358,7 +364,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, loadBalancerListener.syncedRebalance // There is no need to resubscribe to child and state changes. - // The child change watchers will be set inside rebalance when we read the children list. + // The child change watchers will be set inside rebalance when we read the children list. } } @@ -376,34 +382,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def releasePartitionOwnership()= { + info("Releasing partition ownership") for ((topic, infos) <- topicRegistry) { val topicDirs = new ZKGroupTopicDirs(group, topic) for(partition <- infos.keys) { val znode = topicDirs.consumerOwnerDir + "/" + partition - ZkUtils.deletePath(zkClient, znode) + deletePath(zkClient, znode) debug("Consumer " + consumerIdString + " releasing " + znode) } } } - private def getConsumersPerTopic(group: String) : mutable.Map[String, List[String]] = { - val consumers = ZkUtils.getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir) - val consumersPerTopicMap = new mutable.HashMap[String, List[String]] - for (consumer <- consumers) { - val topicCount = getTopicCount(consumer) - for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic()) { - for (consumerThreadId <- consumerThreadIdSet) - consumersPerTopicMap.get(topic) match { - case Some(curConsumers) => consumersPerTopicMap.put(topic, consumerThreadId :: curConsumers) - case _ => consumersPerTopicMap.put(topic, List(consumerThreadId)) - } - } - } - for ( (topic, consumerList) <- consumersPerTopicMap ) - consumersPerTopicMap.put(topic, consumerList.sortWith((s,t) => s < t)) - consumersPerTopicMap - } - private def getRelevantTopicMap(myTopicThreadIdsMap: Map[String, Set[String]], newPartMap: Map[String,List[String]], oldPartMap: Map[String,List[String]], @@ -416,11 +405,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, relevantTopicThreadIdsMap } - private def getTopicCount(consumerId: String) : TopicCount = { - val topicCountJson = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId) - TopicCount.constructTopicCount(consumerId, topicCountJson) - } - def resetState() { topicRegistry.clear oldConsumersPerTopicMap.clear @@ -432,19 +416,34 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, for (i <- 0 until config.maxRebalanceRetries) { info("begin rebalancing consumer " + consumerIdString + " try #" + i) var done = false + val cluster = getCluster(zkClient) try { - done = rebalance() + done = rebalance(cluster) } catch { case e => - // occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. - // For example, a ZK node can disappear between the time we get all children and the time we try to get - // the value of a child. Just let this go since another rebalance will be triggered. + /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. + * For example, a ZK node can disappear between the time we get all children and the time we try to get + * the value of a child. Just let this go since another rebalance will be triggered. + **/ info("exception during rebalance ", e) + /* Explicitly make sure another rebalancing attempt will get triggered. */ + done = false } info("end rebalancing consumer " + consumerIdString + " try #" + i) - if (done) + if (done) { return + }else { + /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should + * clear the cache */ + info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") + oldConsumersPerTopicMap.clear() + oldPartitionsPerTopicMap.clear() + } + // commit offsets + commitOffsets() + // stop all fetchers and clear all the queues to avoid data duplication + closeFetchersForQueues(cluster, kafkaMessageStreams, queues.map(q => q._2)) // release all partitions, reset state and retry releasePartitionOwnership() Thread.sleep(config.rebalanceBackoffMs) @@ -454,26 +453,30 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, throw new ConsumerRebalanceFailedException(consumerIdString + " can't rebalance after " + config.maxRebalanceRetries +" retries") } - private def rebalance(): Boolean = { - val myTopicThreadIdsMap = getTopicCount(consumerIdString).getConsumerThreadIdsPerTopic - val cluster = ZkUtils.getCluster(zkClient) - val consumersPerTopicMap = getConsumersPerTopic(group) - val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, myTopicThreadIdsMap.keys.iterator) + private def rebalance(cluster: Cluster): Boolean = { + val myTopicThreadIdsMap = getTopicCount(zkClient, group, consumerIdString).getConsumerThreadIdsPerTopic + val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) + val partitionsPerTopicMap = getPartitionsForTopics(zkClient, myTopicThreadIdsMap.keys.iterator) val relevantTopicThreadIdsMap = getRelevantTopicMap(myTopicThreadIdsMap, partitionsPerTopicMap, oldPartitionsPerTopicMap, consumersPerTopicMap, oldConsumersPerTopicMap) if (relevantTopicThreadIdsMap.size <= 0) { - info("Consumer " + consumerIdString + " with " + consumersPerTopicMap + " doesn't need to rebalance.") + info("Consumer %s with %s and topic partitions %s doesn't need to rebalance.". + format(consumerIdString, consumersPerTopicMap, partitionsPerTopicMap)) + debug("Partitions per topic cache " + oldPartitionsPerTopicMap) + debug("Consumers per topic cache " + oldConsumersPerTopicMap) return true } - // fetchers must be stopped to avoid data duplication, since if the current - // rebalancing attempt fails, the partitions that are released could be owned by another consumer. - // But if we don't stop the fetchers first, this consumer would continue returning data for released - // partitions in parallel. So, not stopping the fetchers leads to duplicate data. + /** + * fetchers must be stopped to avoid data duplication, since if the current + * rebalancing attempt fails, the partitions that are released could be owned by another consumer. + * But if we don't stop the fetchers first, this consumer would continue returning data for released + * partitions in parallel. So, not stopping the fetchers leads to duplicate data. + */ closeFetchers(cluster, kafkaMessageStreams, relevantTopicThreadIdsMap) - info("Releasing partition ownership") releasePartitionOwnership() + var partitionOwnershipDecision = new collection.mutable.HashMap[(String, String), String]() for ((topic, consumerThreadIdSet) <- relevantTopicThreadIdsMap) { topicRegistry.remove(topic) topicRegistry.put(topic, new Pool[Partition, PartitionTopicInfo]) @@ -505,41 +508,60 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val partition = curPartitions(i) info(consumerThreadId + " attempting to claim partition " + partition) val ownPartition = processPartition(topicDirs, partition, topic, consumerThreadId) - if (ownPartition) - info(consumerThreadId + " successfully owned partition " + partition) - else + if (!ownPartition) return false + else // record the partition ownership decision + partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) } } } } - updateFetcher(cluster, kafkaMessageStreams) - oldPartitionsPerTopicMap = partitionsPerTopicMap - oldConsumersPerTopicMap = consumersPerTopicMap - true + + /** + * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt + * A rebalancing attempt is completed successfully only after the fetchers have been started correctly + */ + if(reflectPartitionOwnershipDecision(partitionOwnershipDecision.toMap)) { + info("Updating the cache") + debug("Partitions per topic cache " + partitionsPerTopicMap) + debug("Consumers per topic cache " + consumersPerTopicMap) + oldPartitionsPerTopicMap = partitionsPerTopicMap + oldConsumersPerTopicMap = consumersPerTopicMap + updateFetcher(cluster, kafkaMessageStreams) + true + }else + false } - private def closeFetchers(cluster: Cluster, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]], - relevantTopicThreadIdsMap: Map[String, Set[String]]) { - // only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer - // after this rebalancing attempt - val queuesTobeCleared = queues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) + private def closeFetchersForQueues(cluster: Cluster, + kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]], + queuesToBeCleared: Iterable[BlockingQueue[FetchedDataChunk]]) { var allPartitionInfos = topicRegistry.values.map(p => p.values).flatten fetcher match { case Some(f) => f.stopConnectionsToAllBrokers - f.clearFetcherQueues(allPartitionInfos, cluster, queuesTobeCleared, kafkaMessageStreams) + f.clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, kafkaMessageStreams) info("Committing all offsets after clearing the fetcher queues") - // here, we need to commit offsets before stopping the consumer from returning any more messages - // from the current data chunk. Since partition ownership is not yet released, this commit offsets - // call will ensure that the offsets committed now will be used by the next consumer thread owning the partition - // for the current data chunk. Since the fetchers are already shutdown and this is the last chunk to be iterated - // by the consumer, there will be no more messages returned by this iterator until the rebalancing finishes - // successfully and the fetchers restart to fetch more data chunks + /** + * here, we need to commit offsets before stopping the consumer from returning any more messages + * from the current data chunk. Since partition ownership is not yet released, this commit offsets + * call will ensure that the offsets committed now will be used by the next consumer thread owning the partition + * for the current data chunk. Since the fetchers are already shutdown and this is the last chunk to be iterated + * by the consumer, there will be no more messages returned by this iterator until the rebalancing finishes + * successfully and the fetchers restart to fetch more data chunks + **/ commitOffsets case None => } } + private def closeFetchers(cluster: Cluster, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]], + relevantTopicThreadIdsMap: Map[String, Set[String]]) { + // only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer + // after this rebalancing attempt + val queuesTobeCleared = queues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) + closeFetchersForQueues(cluster, kafkaMessageStreams, queuesTobeCleared) + } + private def updateFetcher[T](cluster: Cluster, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) { // update partitions for fetcher @@ -560,18 +582,47 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def processPartition(topicDirs: ZKGroupTopicDirs, partition: String, topic: String, consumerThreadId: String) : Boolean = { val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition - try { - ZkUtils.createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) + // check if some other consumer owns this partition at this time + val currentPartitionOwner = readDataMaybeNull(zkClient, partitionOwnerPath) + if(currentPartitionOwner != null) { + if(currentPartitionOwner.equals(consumerThreadId)) { + info(partitionOwnerPath + " exists with value " + currentPartitionOwner + " during connection loss; this is ok") + addPartitionTopicInfo(topicDirs, partition, topic, consumerThreadId) + true + } + else { + info(partitionOwnerPath + " exists with value " + currentPartitionOwner) + false + } + } else { + addPartitionTopicInfo(topicDirs, partition, topic, consumerThreadId) + true } - catch { - case e: ZkNodeExistsException => - // The node hasn't been deleted by the original owner. So wait a bit and retry. - info("waiting for the partition ownership to be deleted: " + partition) - return false - case e2 => throw e2 + } + + private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, String), String]): Boolean = { + val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => + val topic = partitionOwner._1._1 + val partition = partitionOwner._1._2 + val consumerThreadId = partitionOwner._2 + val topicDirs = new ZKGroupTopicDirs(group, topic) + val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition + try { + createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) + info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) + true + } + catch { + case e: ZkNodeExistsException => + // The node hasn't been deleted by the original owner. So wait a bit and retry. + info("waiting for the partition ownership to be deleted: " + partition) + false + case e2 => throw e2 + } } - addPartitionTopicInfo(topicDirs, partition, topic, consumerThreadId) - true + val success = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => if(decision) 0 else 1) + if(success > 0) false /* even if one of the partition ownership attempt has failed, return false */ + else true } private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partitionString: String, @@ -580,7 +631,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val partTopicInfoMap = topicRegistry.get(topic) val znode = topicDirs.consumerOffsetDir + "/" + partition.name - val offsetString = ZkUtils.readDataMaybeNull(zkClient, znode) + val offsetString = readDataMaybeNull(zkClient, znode) // If first time starting a consumer, set the initial offset based on the config var offset : Long = 0L if (offsetString == null) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala index 68afef7990e68..eb563e11f7fe2 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala @@ -18,7 +18,7 @@ package kafka.consumer import scala.collection.JavaConversions._ -import kafka.utils.{Utils, ZkUtils, ZKStringSerializer, Logging} +import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} import org.apache.zookeeper.Watcher.Event.KeeperState import kafka.server.KafkaServerStartable diff --git a/core/src/main/scala/kafka/javaapi/Implicits.scala b/core/src/main/scala/kafka/javaapi/Implicits.scala index 7d56f72bb5468..20ca1930e7264 100644 --- a/core/src/main/scala/kafka/javaapi/Implicits.scala +++ b/core/src/main/scala/kafka/javaapi/Implicits.scala @@ -16,10 +16,8 @@ */ package kafka.javaapi -import java.nio.ByteBuffer import kafka.serializer.Encoder -import kafka.producer.{ProducerConfig, ProducerPool} -import kafka.producer.async.{AsyncProducerConfig, QueueItem} +import kafka.producer.async.QueueItem import kafka.utils.Logging private[javaapi] object Implicits extends Logging { diff --git a/core/src/main/scala/kafka/javaapi/message/MessageSet.scala b/core/src/main/scala/kafka/javaapi/message/MessageSet.scala index 4a3f19357ae01..817d9dd0d50fe 100644 --- a/core/src/main/scala/kafka/javaapi/message/MessageSet.scala +++ b/core/src/main/scala/kafka/javaapi/message/MessageSet.scala @@ -17,8 +17,7 @@ package kafka.javaapi.message -import java.nio.channels._ -import kafka.message.{MessageAndOffset, InvalidMessageException, Message} +import kafka.message.{MessageAndOffset, InvalidMessageException} /** * A set of messages. A message set has a fixed serialized form, though the container diff --git a/core/src/main/scala/kafka/javaapi/producer/Producer.scala b/core/src/main/scala/kafka/javaapi/producer/Producer.scala index 364ebf83464c1..faa420dc20ad2 100644 --- a/core/src/main/scala/kafka/javaapi/producer/Producer.scala +++ b/core/src/main/scala/kafka/javaapi/producer/Producer.scala @@ -54,7 +54,6 @@ class Producer[K,V](config: ProducerConfig, * partitioning strategy on the message key (of type K) that is specified through the ProducerData[K, T] * object in the send API */ - import kafka.javaapi.Implicits._ def this(config: ProducerConfig, encoder: Encoder[V], eventHandler: kafka.javaapi.producer.async.EventHandler[V], diff --git a/core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala b/core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala index 9f848273719bb..ce55c163e8d3d 100644 --- a/core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala +++ b/core/src/main/scala/kafka/message/ByteBufferBackedInputStream.scala @@ -19,7 +19,6 @@ package kafka.message import java.io.InputStream import java.nio.ByteBuffer -import scala.Math class ByteBufferBackedInputStream(buffer:ByteBuffer) extends InputStream { override def read():Int = { diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 75da302701529..c1085826e515e 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -17,7 +17,6 @@ package kafka.message -import scala.collection.mutable import kafka.utils.Logging import kafka.common.{InvalidMessageSizeException, ErrorMapping} import java.nio.ByteBuffer diff --git a/core/src/main/scala/kafka/message/CompressionUtils.scala b/core/src/main/scala/kafka/message/CompressionUtils.scala index 494375658d5c6..7d11993151cb7 100644 --- a/core/src/main/scala/kafka/message/CompressionUtils.scala +++ b/core/src/main/scala/kafka/message/CompressionUtils.scala @@ -54,8 +54,8 @@ class GZIPCompression(inputStream: InputStream, outputStream: ByteArrayOutputStr } class SnappyCompression(inputStream: InputStream,outputStream: ByteArrayOutputStream) extends CompressionFacade(inputStream,outputStream) { - import org.xerial.snappy.{SnappyInputStream} - import org.xerial.snappy.{SnappyOutputStream} + import org.xerial.snappy.SnappyInputStream + import org.xerial.snappy.SnappyOutputStream val snappyIn:SnappyInputStream = if (inputStream == null) null else new SnappyInputStream(inputStream) val snappyOut:SnappyOutputStream = if (outputStream == null) null else new SnappyOutputStream(outputStream) diff --git a/core/src/main/scala/kafka/message/FileMessageSet.scala b/core/src/main/scala/kafka/message/FileMessageSet.scala index edfe8efa21fb7..7c9b4f85a60a6 100644 --- a/core/src/main/scala/kafka/message/FileMessageSet.scala +++ b/core/src/main/scala/kafka/message/FileMessageSet.scala @@ -22,8 +22,6 @@ import java.nio._ import java.nio.channels._ import java.util.concurrent.atomic._ -import kafka._ -import kafka.message._ import kafka.utils._ /** diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 2b5145eaa2740..272a0b6b8a468 100644 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -18,9 +18,6 @@ package kafka.message import java.nio._ -import java.nio.channels._ -import java.util.zip.CRC32 -import java.util.UUID import kafka.utils._ import kafka.common.UnknownMagicByteException diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index a6a233013f874..1bc6bc19486ba 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -21,7 +21,6 @@ import java.util.concurrent._ import java.util.concurrent.atomic._ import java.net._ import java.io._ -import java.nio._ import java.nio.channels._ import kafka.utils._ diff --git a/core/src/main/scala/kafka/network/SocketServerStats.scala b/core/src/main/scala/kafka/network/SocketServerStats.scala index 0d6778e9cf0f0..2ec1fa93d6dc8 100644 --- a/core/src/main/scala/kafka/network/SocketServerStats.scala +++ b/core/src/main/scala/kafka/network/SocketServerStats.scala @@ -17,8 +17,6 @@ package kafka.network -import java.util.concurrent.atomic._ -import javax.management._ import kafka.utils._ import kafka.api.RequestKeys diff --git a/core/src/main/scala/kafka/producer/ConsoleProducer.scala b/core/src/main/scala/kafka/producer/ConsoleProducer.scala index 1c3be540244b1..73f88eee0951e 100644 --- a/core/src/main/scala/kafka/producer/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/producer/ConsoleProducer.scala @@ -18,14 +18,10 @@ package kafka.producer import scala.collection.JavaConversions._ -import org.I0Itec.zkclient._ import joptsimple._ -import java.util.Arrays.asList import java.util.Properties -import java.util.Random import java.io._ import kafka.message._ -import kafka.utils._ import kafka.serializer._ object ConsoleProducer { diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index b1955c77e46fb..d971948d799cf 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -19,7 +19,7 @@ package kafka.producer import async.MissingConfigException import org.apache.log4j.spi.{LoggingEvent, ErrorCode} -import org.apache.log4j.{Logger, AppenderSkeleton} +import org.apache.log4j.AppenderSkeleton import org.apache.log4j.helpers.LogLog import kafka.utils.{Utils, Logging} import kafka.serializer.Encoder diff --git a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala index 8e54210843c2a..b11b8c6ef2f0e 100644 --- a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala @@ -19,7 +19,6 @@ package kafka.producer import kafka.utils.Utils import java.util.Properties -import kafka.message.{CompressionUtils, CompressionCodec} class SyncProducerConfig(val props: Properties) extends SyncProducerConfigShared { /** the broker to which the producer sends events */ diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 6199bca7e227c..c6f68ccc88720 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,7 +17,6 @@ package kafka.server -import scala.reflect.BeanProperty import kafka.log.LogManager import java.util.concurrent.CountDownLatch import java.util.concurrent.atomic.AtomicBoolean diff --git a/core/src/main/scala/kafka/server/MultiMessageSetSend.scala b/core/src/main/scala/kafka/server/MultiMessageSetSend.scala index 9fe79df3f5bcf..8926f8b2c346b 100644 --- a/core/src/main/scala/kafka/server/MultiMessageSetSend.scala +++ b/core/src/main/scala/kafka/server/MultiMessageSetSend.scala @@ -17,10 +17,7 @@ package kafka.server -import java.nio._ -import java.nio.channels._ import kafka.network._ -import kafka.message._ import kafka.utils._ /** diff --git a/core/src/main/scala/kafka/tools/JmxTool.scala b/core/src/main/scala/kafka/tools/JmxTool.scala index c0ad6ab745f7c..dbfa1d83afd88 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -22,7 +22,7 @@ import java.util.Date import java.text.SimpleDateFormat import javax.management._ import javax.management.remote._ -import joptsimple.{OptionSet, OptionParser} +import joptsimple.OptionParser import scala.collection.JavaConversions._ import scala.collection.mutable import scala.math._ diff --git a/core/src/main/scala/kafka/tools/ProducerShell.scala b/core/src/main/scala/kafka/tools/ProducerShell.scala index eda58d9be24fa..1d31ffa037db6 100644 --- a/core/src/main/scala/kafka/tools/ProducerShell.scala +++ b/core/src/main/scala/kafka/tools/ProducerShell.scala @@ -17,12 +17,9 @@ package kafka.tools -import java.net.URI import java.io._ import joptsimple._ -import kafka.message._ import kafka.producer._ -import java.util.Properties import kafka.utils.Utils /** diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index 08ba0f5c5c568..79d3998959b7b 100644 --- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala +++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala @@ -17,19 +17,17 @@ package kafka.tools -import java.io.File import joptsimple.OptionParser -import org.apache.log4j.Logger import java.util.concurrent.{Executors, CountDownLatch} import java.util.Properties import kafka.producer.async.DefaultEventHandler -import kafka.serializer.{DefaultEncoder, StringEncoder} +import kafka.serializer.DefaultEncoder import kafka.producer.{ProducerData, DefaultPartitioner, ProducerConfig, Producer} import kafka.consumer._ -import kafka.utils.{ZKStringSerializer, Utils, Logging} +import kafka.utils.{ZKStringSerializer, Logging} import kafka.api.OffsetRequest import org.I0Itec.zkclient._ -import kafka.message.{CompressionCodec, Message, MessageSet, FileMessageSet} +import kafka.message.{CompressionCodec, Message} object ReplayLogProducer extends Logging { diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index a651a991c446e..74218ecb22ba7 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -22,7 +22,6 @@ import joptsimple._ import kafka.api.FetchRequest import kafka.utils._ import kafka.consumer._ -import kafka.server._ /** * Command line program to dump out messages to standard out using the simple consumer diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala new file mode 100644 index 0000000000000..e64a23481b112 --- /dev/null +++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala @@ -0,0 +1,137 @@ +/** + * 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 kafka.tools + +import joptsimple.OptionParser +import org.I0Itec.zkclient.ZkClient +import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, ZKStringSerializer} + +object VerifyConsumerRebalance extends Logging { + def main(args: Array[String]) { + val parser = new OptionParser() + + val zkConnectOpt = parser.accepts("zk.connect", "ZooKeeper connect string."). + withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]); + val groupOpt = parser.accepts("group", "Consumer group."). + withRequiredArg().ofType(classOf[String]) + parser.accepts("help", "Print this message.") + + val options = parser.parse(args : _*) + + if (options.has("help")) { + parser.printHelpOn(System.out) + System.exit(0) + } + + for (opt <- List(groupOpt)) + if (!options.has(opt)) { + System.err.println("Missing required argument: %s".format(opt)) + parser.printHelpOn(System.err) + System.exit(1) + } + + val zkConnect = options.valueOf(zkConnectOpt) + val group = options.valueOf(groupOpt) + + var zkClient: ZkClient = null + try { + zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + + debug("zkConnect = %s; group = %s".format(zkConnect, group)) + + // check if the rebalancing operation succeeded. + try { + if(validateRebalancingOperation(zkClient, group)) + info("Rebalance operation successful !") + else + error("Rebalance operation failed !") + } catch { + case e2: Throwable => error("Error while verifying current rebalancing operation", e2) + } + } + finally { + if (zkClient != null) + zkClient.close() + } + } + + private def validateRebalancingOperation(zkClient: ZkClient, group: String): Boolean = { + info("Verifying rebalancing operation for consumer group " + group) + var rebalanceSucceeded: Boolean = true + /** + * A successful rebalancing operation would select an owner for each available partition + * This means that for each partition registered under /brokers/topics/[topic]/[broker-id], an owner exists + * under /consumers/[consumer_group]/owners/[topic]/[broker_id-partition_id] + */ + val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group) + val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, consumersPerTopicMap.keys.iterator) + + partitionsPerTopicMap.foreach { partitionsForTopic => + val topic = partitionsForTopic._1 + val partitions = partitionsForTopic._2 + val topicDirs = new ZKGroupTopicDirs(group, topic) + info("Alive partitions for topic %s are %s ".format(topic, partitions.toString)) + info("Alive consumers for topic %s => %s ".format(topic, consumersPerTopicMap.get(topic))) + val partitionsWithOwners = ZkUtils.getChildrenParentMayNotExist(zkClient, topicDirs.consumerOwnerDir) + if(partitionsWithOwners.size == 0) { + error("No owners for any partitions for topic " + topic) + rebalanceSucceeded = false + } + debug("Children of " + topicDirs.consumerOwnerDir + " = " + partitionsWithOwners.toString) + val consumerIdsForTopic = consumersPerTopicMap.get(topic) + + // for each available partition for topic, check if an owner exists + partitions.foreach { partition => + // check if there is a node for [partition] + if(!partitionsWithOwners.exists(p => p.equals(partition))) { + error("No owner for topic %s partition %s".format(topic, partition)) + rebalanceSucceeded = false + } + // try reading the partition owner path for see if a valid consumer id exists there + val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition + val partitionOwner = ZkUtils.readDataMaybeNull(zkClient, partitionOwnerPath) + if(partitionOwner == null) { + error("No owner for topic %s partition %s".format(topic, partition)) + rebalanceSucceeded = false + } + else { + // check if the owner is a valid consumer id + consumerIdsForTopic match { + case Some(consumerIds) => + if(!consumerIds.contains(partitionOwner)) { + error("Owner %s for topic %s partition %s is not a valid member of consumer " + + "group %s".format(partitionOwner, topic, partition, group)) + rebalanceSucceeded = false + } + else + info("Owner of topic %s partition %s is %s".format(topic, partition, partitionOwner)) + case None => { + error("No consumer ids registered for topic " + topic) + rebalanceSucceeded = false + } + } + } + } + + } + + rebalanceSucceeded + } + + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala index f8744d35de79a..07b999445d66d 100644 --- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala +++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala @@ -19,7 +19,6 @@ package kafka.utils import java.util.concurrent._ import java.util.concurrent.atomic._ -import kafka.utils._ /** * A scheduler for running jobs in the background diff --git a/core/src/main/scala/kafka/utils/Range.scala b/core/src/main/scala/kafka/utils/Range.scala index 3ec64f708b878..ca7d699ec4139 100644 --- a/core/src/main/scala/kafka/utils/Range.scala +++ b/core/src/main/scala/kafka/utils/Range.scala @@ -17,9 +17,8 @@ package kafka.utils -import scala.math._ -/** +/** * A generic range value with a start and end */ trait Range { diff --git a/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala index 53f8344a3198a..ae0d86e5de0ab 100644 --- a/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala @@ -21,6 +21,7 @@ import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.cluster.Partition import kafka.api.OffsetRequest +import java.lang.IllegalStateException /** * A utility that updates the offset of every broker partition to the offset of latest log segment file, in ZK. @@ -55,7 +56,11 @@ object UpdateOffsetsInZK { var numParts = 0 for (partString <- partitions) { val part = Partition.parse(partString) - val broker = cluster.getBroker(part.brokerId) + val broker = cluster.getBroker(part.brokerId) match { + case Some(b) => b + case None => throw new IllegalStateException("Broker " + part.brokerId + " is unavailable. Cannot issue " + + "getOffsetsBefore request") + } val consumer = new SimpleConsumer(broker.host, broker.port, 10000, 100 * 1024) val offsets = consumer.getOffsetsBefore(topic, part.partId, offsetOption, 1) val topicDirs = new ZKGroupTopicDirs(config.groupId, topic) diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index d3cc66b4bbede..96d04b65ab5fa 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -23,7 +23,6 @@ import java.nio.channels._ import java.util.concurrent.atomic._ import java.lang.management._ import java.util.zip.CRC32 -import org.apache.log4j.Logger import javax.management._ import java.util.Properties import scala.collection._ diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 085b6e8828692..917da0f8cda38 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -23,6 +23,7 @@ import kafka.cluster.{Broker, Cluster} import scala.collection._ import java.util.Properties import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError} +import kafka.consumer.TopicCount object ZkUtils extends Logging { val ConsumersPath = "/consumers" @@ -236,6 +237,44 @@ object ZkUtils extends Logging { val brokerPartTopicPath = BrokerTopicsPath + "/" + topic + "/" + brokerId zkClient.delete(brokerPartTopicPath) } + + def getConsumersInGroup(zkClient: ZkClient, group: String): Seq[String] = { + val dirs = new ZKGroupDirs(group) + getChildren(zkClient, dirs.consumerRegistryDir) + } + + def getTopicCount(zkClient: ZkClient, group: String, consumerId: String) : TopicCount = { + val dirs = new ZKGroupDirs(group) + val topicCountJson = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId) + TopicCount.constructTopicCount(consumerId, topicCountJson) + } + + def getConsumerTopicMaps(zkClient: ZkClient, group: String): Map[String, TopicCount] = { + val dirs = new ZKGroupDirs(group) + val consumersInGroup = getConsumersInGroup(zkClient, group) + val topicCountMaps = consumersInGroup.map(consumerId => TopicCount.constructTopicCount(consumerId, + ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId))) + consumersInGroup.zip(topicCountMaps).toMap + } + + def getConsumersPerTopic(zkClient: ZkClient, group: String) : mutable.Map[String, List[String]] = { + val dirs = new ZKGroupDirs(group) + val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir) + val consumersPerTopicMap = new mutable.HashMap[String, List[String]] + for (consumer <- consumers) { + val topicCount = getTopicCount(zkClient, group, consumer) + for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic()) { + for (consumerThreadId <- consumerThreadIdSet) + consumersPerTopicMap.get(topic) match { + case Some(curConsumers) => consumersPerTopicMap.put(topic, consumerThreadId :: curConsumers) + case _ => consumersPerTopicMap.put(topic, List(consumerThreadId)) + } + } + } + for ( (topic, consumerList) <- consumersPerTopicMap ) + consumersPerTopicMap.put(topic, consumerList.sortWith((s,t) => s < t)) + consumersPerTopicMap + } } object ZKStringSerializer extends ZkSerializer { diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 51c41954d0657..767fe558017e3 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -21,4 +21,5 @@ log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n log4j.logger.kafka=WARN # zkclient can be verbose, during debugging it is common to adjust is separately -log4j.logger.org.I0Itec.zkclient.ZkClient=WARN \ No newline at end of file +log4j.logger.org.I0Itec.zkclient.ZkClient=WARN +log4j.logger.org.apache.zookeeper=WARN diff --git a/system_test/broker_failure/bin/run-test.sh b/system_test/broker_failure/bin/run-test.sh index 2e6cc1875536b..b3284e341fbaf 100755 --- a/system_test/broker_failure/bin/run-test.sh +++ b/system_test/broker_failure/bin/run-test.sh @@ -68,7 +68,7 @@ readonly test_start_time="$(date +%s)" readonly num_msg_per_batch=500 readonly batches_per_iteration=5 -readonly num_iterations=10 +readonly num_iterations=12 readonly zk_source_port=2181 readonly zk_mirror_port=2182 @@ -132,6 +132,8 @@ producer_performance_crc_log=$base_dir/producer_performance_crc.log producer_performance_crc_sorted_log=$base_dir/producer_performance_crc_sorted.log producer_performance_crc_sorted_uniq_log=$base_dir/producer_performance_crc_sorted_uniq.log +consumer_rebalancing_log=$base_dir/consumer_rebalancing_verification.log + consumer_prop_file=$base_dir/config/whitelisttest.consumer.properties checksum_diff_log=$base_dir/checksum_diff.log @@ -173,6 +175,17 @@ get_random_range() { return $(($(($RANDOM % range)) + $lo)) } +verify_consumer_rebalancing() { + + info "Verifying consumer rebalancing operation" + + $base_dir/bin/kafka-run-class.sh \ + kafka.tools.VerifyConsumerRebalance \ + --zk.connect=localhost:2181 \ + --group $consumer_grp \ + 2>&1 >> $consumer_rebalancing_log +} + wait_for_zero_consumer_lags() { # no of times to check for zero lagging @@ -618,6 +631,7 @@ start_test() { sleep $wait_time_after_restarting_broker fi fi + verify_consumer_rebalancing else info "No bouncing performed" fi @@ -662,6 +676,8 @@ start_console_consumer_for_mirror_producer wait_for_zero_source_console_consumer_lags wait_for_zero_mirror_console_consumer_lags +verify_consumer_rebalancing + shutdown_servers cmp_checksum diff --git a/system_test/broker_failure/config/log4j.properties b/system_test/broker_failure/config/log4j.properties index 835d8095fb444..fdf0ee9ba0099 100644 --- a/system_test/broker_failure/config/log4j.properties +++ b/system_test/broker_failure/config/log4j.properties @@ -26,13 +26,15 @@ log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n # Turn on all our debugging info #log4j.logger.kafka=INFO -#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG +log4j.logger.org.I0Itec.zkclient.ZkClient=INFO +log4j.logger.org.apache.zookeeper=INFO log4j.logger.kafka.consumer=DEBUG log4j.logger.kafka.server.EmbeddedConsumer$MirroringThread=TRACE log4j.logger.kafka.server.KafkaRequestHandlers=TRACE #log4j.logger.kafka.producer.async.AsyncProducer=TRACE #log4j.logger.kafka.producer.async.ProducerSendThread=TRACE log4j.logger.kafka.producer.async.DefaultEventHandler=TRACE +log4j.logger.kafka.tools.VerifyConsumerRebalance=DEBUG # to print message checksum from ProducerPerformance log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG From 2552a2488f2674e2fc3abea02272e85ff13a3ee4 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Mon, 6 Feb 2012 23:10:54 +0000 Subject: [PATCH 006/151] KAFKA-257 Hadoop producer should use software load balancer; patched by Sam Shah; reviewed by nehanarkhede git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1241249 13f79535-47bb-0310-9956-ffa450edef68 --- contrib/hadoop-producer/README.md | 35 ++++--- .../bridge/hadoop/KafkaOutputFormat.java | 91 +++++++++++++++---- .../bridge/hadoop/KafkaRecordWriter.java | 21 ++--- 3 files changed, 107 insertions(+), 40 deletions(-) diff --git a/contrib/hadoop-producer/README.md b/contrib/hadoop-producer/README.md index 33862ea1b2aba..6e57fde28b0b6 100644 --- a/contrib/hadoop-producer/README.md +++ b/contrib/hadoop-producer/README.md @@ -1,6 +1,14 @@ Hadoop to Kafka Bridge ====================== +What's new? +----------- + +* Now supports Kafka's software load balancer (Kafka URIs are specified with + kafka+zk as the scheme, as described below) +* Supports Kafka 0.7. Now uses the new Producer API, rather than the legacy + SyncProducer. + What is it? ----------- @@ -17,8 +25,10 @@ multiple times in the same push. How do I use it? ---------------- -With this bridge, Kafka topics are URIs and are specified as -`kafka:///`. +With this bridge, Kafka topics are URIs and are specified in one of two +formats: `kafka+zk://#`, which uses the software load +balancer, or the legacy `kafka:///` to connect to a +specific Kafka broker. ### Pig ### @@ -27,17 +37,19 @@ row. To push data via Kafka, store to the Kafka URI using `AvroKafkaStorage` with the Avro schema as its first argument. You'll need to register the appropriate Kafka JARs. Here is what an example Pig script looks like: - REGISTER hadoop-kafka-bridge-0.5.2.jar; + REGISTER hadoop-producer_2.8.0-0.7.0.jar; REGISTER avro-1.4.0.jar; REGISTER piggybank.jar; - REGISTER kafka-0.5.2.jar; + REGISTER kafka-0.7.0.jar; REGISTER jackson-core-asl-1.5.5.jar; REGISTER jackson-mapper-asl-1.5.5.jar; + REGISTER zkclient-20110412.jar; + REGISTER zookeeper-3.3.4.jar; REGISTER scala-library.jar; member_info = LOAD 'member_info.tsv' as (member_id : int, name : chararray); names = FOREACH member_info GENERATE name; - STORE member_info INTO 'kafka://my-broker:9092/member_info' USING kafka.bridge.AvroKafkaStorage('"string"'); + STORE member_info INTO 'kafka+zk://my-zookeeper:2181/kafka#member_info' USING kafka.bridge.AvroKafkaStorage('"string"'); That's it! The Pig StoreFunc makes use of AvroStorage in Piggybank to convert from Pig's data model to the specified Avro schema. @@ -46,8 +58,8 @@ Further, multi-store is possible with KafkaStorage, so you can easily write to multiple topics and brokers in the same job: SPLIT member_info INTO early_adopters IF member_id < 1000, others IF member_id >= 1000; - STORE early_adopters INTO 'kafka://my-broker:9092/early_adopters' USING AvroKafkaStorage('$schema'); - STORE others INTO 'kafka://my-broker:9092/others' USING AvroKafkaStorage('$schema'); + STORE early_adopters INTO 'kafka+zk://my-zookeeper:2181/kafka#early_adopters' USING AvroKafkaStorage('$schema'); + STORE others INTO 'kafka://my-broker:9092,my-broker2:9092/others' USING AvroKafkaStorage('$schema'); ### KafkaOutputFormat ### @@ -126,9 +138,10 @@ Normally, you needn't change any of these parameters: docs). Default is 64*1024 (64KB). * kafka.output.max_msgsize: Maximum message size in bytes (see Kafka producer docs). Default is 1024*1024 (1MB). +* kafka.output.compression_codec: The compression codec to use (see Kafka producer + docs). Default is 0 (no compression). -For easier debugging, the above values as well as the Kafka URI -(kafka.output.url), the output server (kafka.output.server), the topic -(kafka.output.topic), and the schema (kafka.output.schema) are injected into -the job's configuration. +For easier debugging, the above values as well as the Kafka broker information +(either kafka.zk.connect or kafka.broker.list), the topic (kafka.output.topic), +and the schema (kafka.output.schema) are injected into the job's configuration. diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java index 1ed33b8660961..57332671ea4b4 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java @@ -18,27 +18,42 @@ import java.util.Properties; -import kafka.javaapi.producer.SyncProducer; -import kafka.producer.SyncProducerConfig; +import kafka.javaapi.producer.Producer; +import kafka.message.Message; +import kafka.producer.ProducerConfig; +import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.*; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; +import org.apache.log4j.Logger; import java.io.IOException; import java.net.URI; public class KafkaOutputFormat extends OutputFormat { + private Logger log = Logger.getLogger(KafkaOutputFormat.class); + public static final String KAFKA_URL = "kafka.output.url"; + /** Bytes to buffer before the OutputFormat does a send */ + public static final int KAFKA_QUEUE_SIZE = 10*1024*1024; + + /** Default value for Kafka's connect.timeout.ms */ public static final int KAFKA_PRODUCER_CONNECT_TIMEOUT = 30*1000; + /** Default value for Kafka's reconnect.interval*/ public static final int KAFKA_PRODUCER_RECONNECT_INTERVAL = 1000; + /** Default value for Kafka's buffer.size */ public static final int KAFKA_PRODUCER_BUFFER_SIZE = 64*1024; + /** Default value for Kafka's max.message.size */ public static final int KAFKA_PRODUCER_MAX_MESSAGE_SIZE = 1024*1024; - public static final int KAFKA_QUEUE_SIZE = 10*1024*1024; + /** Default value for Kafka's producer.type */ + public static final String KAFKA_PRODUCER_PRODUCER_TYPE = "sync"; + /** Default value for Kafka's compression.codec */ + public static final int KAFKA_PRODUCER_COMPRESSION_CODEC = 0; public KafkaOutputFormat() { @@ -77,40 +92,80 @@ public RecordWriter getRecordWriter(TaskAttemptContext context) Path outputPath = getOutputPath(context); if (outputPath == null) throw new IllegalArgumentException("no kafka output url specified"); - URI uri = outputPath.toUri(); + URI uri = URI.create(outputPath.toString()); Configuration job = context.getConfiguration(); - final String topic = uri.getPath().substring(1); // ignore the initial '/' in the path + Properties props = new Properties(); + String topic; final int queueSize = job.getInt("kafka.output.queue_size", KAFKA_QUEUE_SIZE); final int timeout = job.getInt("kafka.output.connect_timeout", KAFKA_PRODUCER_CONNECT_TIMEOUT); final int interval = job.getInt("kafka.output.reconnect_interval", KAFKA_PRODUCER_RECONNECT_INTERVAL); final int bufSize = job.getInt("kafka.output.bufsize", KAFKA_PRODUCER_BUFFER_SIZE); final int maxSize = job.getInt("kafka.output.max_msgsize", KAFKA_PRODUCER_MAX_MESSAGE_SIZE); + final String producerType = job.get("kafka.output.producer_type", KAFKA_PRODUCER_PRODUCER_TYPE); + final int compressionCodec = job.getInt("kafka.output.compression_codec", KAFKA_PRODUCER_COMPRESSION_CODEC); - job.set("kafka.output.server", String.format("%s:%d", uri.getHost(), uri.getPort())); - job.set("kafka.output.topic", topic); job.setInt("kafka.output.queue_size", queueSize); job.setInt("kafka.output.connect_timeout", timeout); job.setInt("kafka.output.reconnect_interval", interval); job.setInt("kafka.output.bufsize", bufSize); job.setInt("kafka.output.max_msgsize", maxSize); + job.set("kafka.output.producer_type", producerType); + job.setInt("kafka.output.compression_codec", compressionCodec); - if (uri.getHost().isEmpty()) - throw new IllegalArgumentException("missing kafka server"); - if (uri.getPath().isEmpty()) - throw new IllegalArgumentException("missing kafka topic"); - - Properties props = new Properties(); - props.setProperty("host", uri.getHost()); - props.setProperty("port", Integer.toString(uri.getPort())); + props.setProperty("producer.type", producerType); props.setProperty("buffer.size", Integer.toString(bufSize)); props.setProperty("connect.timeout.ms", Integer.toString(timeout)); props.setProperty("reconnect.interval", Integer.toString(interval)); props.setProperty("max.message.size", Integer.toString(maxSize)); - - SyncProducer producer = new SyncProducer(new SyncProducerConfig(props)); + props.setProperty("compression.codec", Integer.toString(compressionCodec)); + + if (uri.getScheme().equals("kafka+zk")) { + // Software load balancer: + // URL: kafka+zk://# + // e.g. kafka+zk://kafka-zk:2181/kafka#foobar + + String zkConnect = uri.getAuthority() + uri.getPath(); + + props.setProperty("zk.connect", zkConnect); + job.set("kafka.zk.connect", zkConnect); + + topic = uri.getFragment(); + if (topic == null) + throw new IllegalArgumentException("no topic specified in kafka uri fragment"); + + log.info(String.format("using kafka zk.connect %s (topic %s)", zkConnect, topic)); + } else if (uri.getScheme().equals("kafka")) { + // using the legacy direct broker list + // URL: kafka:/// + // e.g. kafka://kafka-server:9000,kafka-server2:9000/foobar + + // Just enumerate broker_ids, as it really doesn't matter what they are as long as they're unique + // (KAFKA-258 will remove the broker_id requirement) + StringBuilder brokerListBuilder = new StringBuilder(); + String delim = ""; + int brokerId = 0; + for (String serverPort : uri.getAuthority().split(",")) { + brokerListBuilder.append(delim).append(String.format("%d:%s", brokerId, serverPort)); + delim = ","; + brokerId++; + } + String brokerList = brokerListBuilder.toString(); + + props.setProperty("broker.list", brokerList); + job.set("kafka.broker.list", brokerList); + + if (uri.getPath() == null || uri.getPath().length() <= 1) + throw new IllegalArgumentException("no topic specified in kafka uri"); + + topic = uri.getPath().substring(1); // ignore the initial '/' in the path + job.set("kafka.output.topic", topic); + log.info(String.format("using kafka broker %s (topic %s)", brokerList, topic)); + } else + throw new IllegalArgumentException("missing scheme from kafka uri (must be kafka:// or kafka+zk://)"); + + Producer producer = new Producer(new ProducerConfig(props)); return new KafkaRecordWriter(producer, topic, queueSize); } } - diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java index 214b30133898a..efb2b5e0231b4 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java @@ -16,30 +16,29 @@ */ package kafka.bridge.hadoop; +import kafka.javaapi.producer.Producer; +import kafka.javaapi.producer.ProducerData; import kafka.message.Message; -import kafka.javaapi.message.ByteBufferMessageSet; -import kafka.javaapi.producer.SyncProducer; -import kafka.message.NoCompressionCodec; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import java.io.IOException; -import java.util.ArrayList; +import java.util.LinkedList; import java.util.List; public class KafkaRecordWriter extends RecordWriter { - protected SyncProducer producer; + protected Producer producer; protected String topic; - protected List msgList = new ArrayList(); + protected List> msgList = new LinkedList>(); protected int totalSize = 0; protected int queueSize; - public KafkaRecordWriter(SyncProducer producer, String topic, int queueSize) + public KafkaRecordWriter(Producer producer, String topic, int queueSize) { this.producer = producer; this.topic = topic; @@ -49,8 +48,7 @@ public KafkaRecordWriter(SyncProducer producer, String topic, int queueSize) protected void sendMsgList() { if (msgList.size() > 0) { - ByteBufferMessageSet msgSet = new ByteBufferMessageSet(kafka.message.NoCompressionCodec$.MODULE$, msgList); - producer.send(topic, msgSet); + producer.send(msgList); msgList.clear(); totalSize = 0; } @@ -60,10 +58,11 @@ protected void sendMsgList() public void write(NullWritable key, BytesWritable value) throws IOException, InterruptedException { Message msg = new Message(value.getBytes()); - msgList.add(msg); + msgList.add(new ProducerData(this.topic, msg)); totalSize += msg.size(); - if (totalSize > queueSize) + // MultiProducerRequest only supports sending up to Short.MAX_VALUE messages in one batch + if (totalSize > queueSize || msgList.size() >= Short.MAX_VALUE) sendMsgList(); } From 345bc7bad594af43df5fc264ad0b5904d1126257 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Tue, 7 Feb 2012 17:38:15 +0000 Subject: [PATCH 007/151] KAFKA-263 Enhance single host broker failure test to have 2 topics with uneven distribution on the source brokers; patched by John Fung; reviewed by Neha Narkhede git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1241528 13f79535-47bb-0310-9956-ffa450edef68 --- .../kafka/consumer/ConsoleConsumer.scala | 12 +- .../kafka/perf/ProducerPerformance.scala | 15 +- system_test/broker_failure/bin/run-test.sh | 205 ++++++++++++++---- .../broker_failure/config/log4j.properties | 72 ++++-- .../config/mirror_producer1.properties | 2 +- .../config/mirror_producer2.properties | 2 +- .../config/mirror_producer3.properties | 2 +- .../config/server_source1.properties | 2 +- .../config/server_source2.properties | 2 +- .../config/server_source3.properties | 2 +- .../config/server_source4.properties | 2 +- .../config/server_target1.properties | 2 +- .../config/server_target2.properties | 2 +- .../config/server_target3.properties | 2 +- .../config/whitelisttest.consumer.properties | 4 +- 15 files changed, 246 insertions(+), 82 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala index 43683d5375b90..bddbb2bc5643f 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -210,9 +210,19 @@ object ConsoleConsumer extends Logging { } class ChecksumMessageFormatter extends MessageFormatter { + private var topicStr: String = _ + + override def init(props: Properties) { + topicStr = props.getProperty("topic") + if (topicStr != null) + topicStr = topicStr + "-" + else + topicStr = "" + } + def writeTo(message: Message, output: PrintStream) { val chksum = message.checksum - output.println("checksum:" + chksum) + output.println(topicStr + "checksum:" + chksum) } } diff --git a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala b/perf/src/main/scala/kafka/perf/ProducerPerformance.scala index 93a3fd69e1d5a..5888f1e30df24 100644 --- a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/ProducerPerformance.scala @@ -20,18 +20,16 @@ package kafka.perf import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong import kafka.producer._ -import async.DefaultEventHandler import org.apache.log4j.Logger -import joptsimple.OptionParser import kafka.message.{CompressionCodec, Message} -import kafka.serializer.DefaultEncoder import java.text.SimpleDateFormat -import java.util.{Date, Random, Properties} +import java.util.{Random, Properties} +import kafka.utils.Logging /** * Load test for the producer */ -object ProducerPerformance { +object ProducerPerformance extends Logging { def main(args: Array[String]) { @@ -141,7 +139,6 @@ object ProducerPerformance { val totalMessagesSent: AtomicLong, val allDone: CountDownLatch, val rand: Random) extends Runnable { - val logger = Logger.getLogger(getClass) val props = new Properties() val brokerInfoList = config.brokerInfo.split("=") if (brokerInfoList(0) == "zk.connect") { @@ -171,7 +168,7 @@ object ProducerPerformance { var lastReportTime = reportTime val messagesPerThread = if(!config.isAsync) config.numMessages / config.numThreads / config.batchSize else config.numMessages / config.numThreads - if(logger.isDebugEnabled) logger.debug("Messages per thread = " + messagesPerThread) + debug("Messages per thread = " + messagesPerThread) var messageSet: List[Message] = Nil if(config.isFixSize) { for(k <- 0 until config.batchSize) { @@ -203,11 +200,11 @@ object ProducerPerformance { rand.nextBytes(messageBytes) val message = new Message(messageBytes) producer.send(new ProducerData[Message,Message](config.topic, message)) - if(logger.isDebugEnabled) println("checksum:" + message.checksum) + debug(config.topic + "-checksum:" + message.checksum) bytesSent += message.payloadSize }else { producer.send(new ProducerData[Message,Message](config.topic, message)) - if(logger.isDebugEnabled) println("checksum:" + message.checksum) + debug(config.topic + "-checksum:" + message.checksum) bytesSent += message.payloadSize } nSends += 1 diff --git a/system_test/broker_failure/bin/run-test.sh b/system_test/broker_failure/bin/run-test.sh index b3284e341fbaf..639b47bcb927b 100755 --- a/system_test/broker_failure/bin/run-test.sh +++ b/system_test/broker_failure/bin/run-test.sh @@ -68,12 +68,14 @@ readonly test_start_time="$(date +%s)" readonly num_msg_per_batch=500 readonly batches_per_iteration=5 -readonly num_iterations=12 +readonly num_iterations=1 readonly zk_source_port=2181 readonly zk_mirror_port=2182 -readonly topic_1=test01 +readonly topic_prefix=test +readonly max_topic_id=2 +readonly unbalanced_start_id=2 readonly consumer_grp=group1 readonly source_console_consumer_grp=source readonly mirror_console_consumer_grp=mirror @@ -96,10 +98,16 @@ readonly num_kafka_target_server=3 readonly wait_time_after_killing_broker=0 readonly wait_time_after_restarting_broker=5 -background_producer_pid= +readonly producer_4_brokerinfo_str="broker.list=1:localhost:9091,2:localhost:9092,3:localhost:9093,4:localhost:9094" +readonly producer_3_brokerinfo_str="broker.list=1:localhost:9091,2:localhost:9092,3:localhost:9093" + +background_producer_pid_1= +background_producer_pid_2= + no_bouncing=$# iter=1 +abort_test=false pid_zk_source= pid_zk_target= @@ -177,17 +185,29 @@ get_random_range() { verify_consumer_rebalancing() { - info "Verifying consumer rebalancing operation" + info "Verifying consumer rebalancing operation" + + CONSUMER_REBALANCING_RESULT=`$base_dir/bin/kafka-run-class.sh \ + kafka.tools.VerifyConsumerRebalance \ + --zk.connect=localhost:2181 \ + --group $consumer_grp` + echo "$CONSUMER_REBALANCING_RESULT" >> $consumer_rebalancing_log + + REBALANCE_STATUS_LINE=`grep "Rebalance operation" $consumer_rebalancing_log | tail -1` + # info "REBALANCE_STATUS_LINE: $REBALANCE_STATUS_LINE" + REBALANCE_STATUS=`echo $REBALANCE_STATUS_LINE | grep "Rebalance operation successful" || echo -n "Rebalance operation failed"` + info "REBALANCE_STATUS: $REBALANCE_STATUS" - $base_dir/bin/kafka-run-class.sh \ - kafka.tools.VerifyConsumerRebalance \ - --zk.connect=localhost:2181 \ - --group $consumer_grp \ - 2>&1 >> $consumer_rebalancing_log + if [ "${REBALANCE_STATUS}_x" == "Rebalance operation failed_x" ]; then + info "setting abort_test to true due to Rebalance operation failed" + abort_test="true" + fi } wait_for_zero_consumer_lags() { + topic_id=$1 + # no of times to check for zero lagging no_of_zero_to_verify=3 @@ -196,7 +216,7 @@ wait_for_zero_consumer_lags() { TOTAL_LAG=0 CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \ --group $consumer_grp --zkconnect localhost:$zk_source_port \ - --topic $topic_1 | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` + --topic ${topic_prefix}_${topic_id} | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` for lag in $CONSUMER_LAGS; do @@ -217,6 +237,8 @@ wait_for_zero_consumer_lags() { wait_for_zero_source_console_consumer_lags() { + topic_id=$1 + # no of times to check for zero lagging no_of_zero_to_verify=3 @@ -225,7 +247,7 @@ wait_for_zero_source_console_consumer_lags() { TOTAL_LAG=0 CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \ --group $source_console_consumer_grp --zkconnect localhost:$zk_source_port \ - --topic $topic_1 | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` + --topic ${topic_prefix}_${topic_id} | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` for lag in $CONSUMER_LAGS; do @@ -246,6 +268,8 @@ wait_for_zero_source_console_consumer_lags() { wait_for_zero_mirror_console_consumer_lags() { + topic_id=$1 + # no of times to check for zero lagging no_of_zero_to_verify=3 @@ -254,7 +278,7 @@ wait_for_zero_mirror_console_consumer_lags() { TOTAL_LAG=0 CONSUMER_LAGS=`$base_dir/bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker \ --group $mirror_console_consumer_grp --zkconnect localhost:$zk_mirror_port \ - --topic $topic_1 | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` + --topic ${topic_prefix}_${topic_id} | grep "Consumer lag" | tr -d ' ' | cut -f2 -d '='` for lag in $CONSUMER_LAGS; do @@ -321,6 +345,8 @@ cleanup() { rm -f $console_consumer_source_crc_sorted_log rm -f $console_consumer_mirror_crc_sorted_uniq_log rm -f $console_consumer_source_crc_sorted_uniq_log + + rm -f $consumer_rebalancing_log } start_zk() { @@ -380,40 +406,65 @@ start_embedded_consumer_server() { } start_console_consumer_for_source_producer() { - info "starting console consumers for source producer" + + topic_id=$1 + + info "starting console consumers for source producer on topic id [$topic_id]" $base_dir/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer \ --zookeeper localhost:$zk_source_port \ - --topic $topic_1 \ + --topic ${topic_prefix}_${topic_id} \ --group $source_console_consumer_grp \ - --from-beginning \ + --from-beginning --consumer-timeout-ms 5000 \ --formatter "kafka.consumer.ConsoleConsumer\$ChecksumMessageFormatter" \ - 2>&1 > ${console_consumer_source_log} & - console_consumer_source_pid=$! - - info " -> console consumer source pid: $console_consumer_source_pid" + --property topic=${topic_prefix}_${topic_id} \ + 2>&1 >> ${console_consumer_source_log} } start_console_consumer_for_mirror_producer() { - info "starting console consumers for mirroring producer" + + topic_id=$1 + + info "starting console consumers for mirroring producer on topic id [$topic_id]" $base_dir/bin/kafka-run-class.sh kafka.consumer.ConsoleConsumer \ --zookeeper localhost:$zk_mirror_port \ - --topic $topic_1 \ + --topic ${topic_prefix}_${topic_id} \ --group $mirror_console_consumer_grp \ - --from-beginning \ + --from-beginning --consumer-timeout-ms 5000 \ --formatter "kafka.consumer.ConsoleConsumer\$ChecksumMessageFormatter" \ - 2>&1 > ${console_consumer_mirror_log} & - console_consumer_mirror_pid=$! + --property topic=${topic_prefix}_${topic_id} \ + 2>&1 >> ${console_consumer_mirror_log} +} - info " -> console consumer mirror pid: $console_consumer_mirror_pid" +consume_source_producer_messages() { + consumer_counter=1 + while [ $consumer_counter -le $max_topic_id ] + do + start_console_consumer_for_source_producer $consumer_counter + consumer_counter=$(( $consumer_counter + 1 )) + done +} + +consume_mirror_producer_messages() { + consumer_counter=1 + while [ $consumer_counter -le $max_topic_id ] + do + start_console_consumer_for_mirror_producer $consumer_counter + consumer_counter=$(( $consumer_counter + 1 )) + done } shutdown_producer() { info "shutting down producer" - if [ "x${background_producer_pid}" != "x" ]; then - # kill_child_processes 0 ${background_producer_pid}; - kill -TERM ${background_producer_pid} 2> /dev/null; + if [ "x${background_producer_pid_1}" != "x" ]; then + # kill_child_processes 0 ${background_producer_pid_1}; + kill -TERM ${background_producer_pid_1} 2> /dev/null; + fi + + if [ "x${background_producer_pid_2}" != "x" ]; then + # kill_child_processes 0 ${background_producer_pid_2}; + kill -TERM ${background_producer_pid_2} 2> /dev/null; fi } @@ -450,13 +501,15 @@ shutdown_servers() { } start_background_producer() { + bkrinfo_str=$1 + start_topic_id=$2 + end_topic_id=$3 batch_no=0 - curr_iter=0 + topic_id=${start_topic_id} - while [ $num_iterations -gt $curr_iter ] + while [ 'x' == 'x' ] do - topic=$1 sleeptime= get_random_range $sleep_min $sleep_max @@ -464,19 +517,24 @@ start_background_producer() { batch_no=$(($batch_no + 1)) + if [ $topic_id -gt $end_topic_id ]; then + topic_id=${start_topic_id} + fi + $base_dir/bin/kafka-run-class.sh \ kafka.perf.ProducerPerformance \ - --brokerinfo zk.connect=localhost:2181 \ - --topic $topic \ + --brokerinfo $bkrinfo_str \ + --topic ${topic_prefix}_${topic_id} \ --messages $num_msg_per_batch \ --message-size $message_size \ --batch-size 50 \ --vary-message-size \ --threads 1 \ - --reporting-interval $num_msg_per_batch \ - --async \ + --reporting-interval $num_msg_per_batch --async \ 2>&1 >> $base_dir/producer_performance.log # appending all producers' msgs + topic_id=$(( $topic_id + 1 )) + sleep $sleeptime done } @@ -485,9 +543,9 @@ cmp_checksum() { cmp_result=0 - grep ^checksum $console_consumer_source_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_source_crc_log - grep ^checksum $console_consumer_mirror_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_mirror_crc_log - grep ^checksum $producer_performance_log | tr -d ' ' | cut -f2 -d ':' > $producer_performance_crc_log + grep checksum $console_consumer_source_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_source_crc_log + grep checksum $console_consumer_mirror_log | tr -d ' ' | cut -f2 -d ':' > $console_consumer_mirror_crc_log + grep checksum $producer_performance_log | tr -d ' ' | cut -f4 -d ':' | cut -f1 -d '(' > $producer_performance_crc_log sort $console_consumer_mirror_crc_log > $console_consumer_mirror_crc_sorted_log sort $console_consumer_source_crc_log > $console_consumer_source_crc_sorted_log @@ -555,6 +613,37 @@ cmp_checksum() { echo "========================================================" >> $checksum_diff_log echo "${duplicate_mirror_crc}" >> $checksum_diff_log + topic_chksum_counter=1 + while [ $topic_chksum_counter -le $max_topic_id ] + do + # get producer topic counts + this_chksum_count=`grep -c ${topic_prefix}_${topic_chksum_counter}\- $producer_performance_log` + echo "PRODUCER topic ${topic_prefix}_${topic_chksum_counter} count: ${this_chksum_count}" + + topic_chksum_counter=$(($topic_chksum_counter + 1)) + done + echo + + topic_chksum_counter=1 + while [ $topic_chksum_counter -le $max_topic_id ] + do + this_chksum_count=`grep -c ${topic_prefix}_${topic_chksum_counter}\- $console_consumer_source_log` + echo "SOURCE consumer topic ${topic_prefix}_${topic_chksum_counter} count: ${this_chksum_count}" + + topic_chksum_counter=$(($topic_chksum_counter + 1)) + done + echo + + topic_chksum_counter=1 + while [ $topic_chksum_counter -le $max_topic_id ] + do + this_chksum_count=`grep -c ${topic_prefix}_${topic_chksum_counter}\- $console_consumer_mirror_log` + echo "MIRROR consumer topic ${topic_prefix}_${topic_chksum_counter} count: ${this_chksum_count}" + + topic_chksum_counter=$(($topic_chksum_counter + 1)) + done + echo + return $cmp_result } @@ -567,15 +656,32 @@ start_test() { start_target_servers_cluster sleep 2 - start_background_producer $topic_1 & - background_producer_pid=$! + start_background_producer $producer_4_brokerinfo_str 1 $(( $unbalanced_start_id - 1 )) & + background_producer_pid_1=$! info "==========================================" - info "Started background producer pid [${background_producer_pid}]" + info "Started background producer pid [${background_producer_pid_1}]" info "==========================================" - sleep 5 - + sleep 10 + + start_background_producer $producer_3_brokerinfo_str $unbalanced_start_id $max_topic_id & + background_producer_pid_2=$! + + info "==========================================" + info "Started background producer pid [${background_producer_pid_2}]" + info "==========================================" + + sleep 10 + + verify_consumer_rebalancing + + info "abort_test: [${abort_test}]" + if [ "${abort_test}_x" == "true_x" ]; then + info "aborting test" + iter=$((${num_iterations} + 1)) + fi + while [ $num_iterations -ge $iter ] do echo @@ -592,7 +698,6 @@ start_test() { # even iterations -> bounce target kafka borker get_random_range 1 $num_kafka_target_server idx=$? - if [ "x${kafka_target_pids[$idx]}" != "x" ]; then echo info "#### Bouncing kafka TARGET broker ####" @@ -631,7 +736,15 @@ start_test() { sleep $wait_time_after_restarting_broker fi fi + verify_consumer_rebalancing + + info "abort_test: [${abort_test}]" + if [ "${abort_test}_x" == "true_x" ]; then + info "aborting test" + iter=$((${num_iterations} + 1)) + fi + else info "No bouncing performed" fi @@ -670,8 +783,8 @@ trap "shutdown_producer; shutdown_servers; cmp_checksum; exit 0" INT start_test -start_console_consumer_for_source_producer -start_console_consumer_for_mirror_producer +consume_source_producer_messages +consume_mirror_producer_messages wait_for_zero_source_console_consumer_lags wait_for_zero_mirror_console_consumer_lags diff --git a/system_test/broker_failure/config/log4j.properties b/system_test/broker_failure/config/log4j.properties index fdf0ee9ba0099..63d2de29cac3c 100644 --- a/system_test/broker_failure/config/log4j.properties +++ b/system_test/broker_failure/config/log4j.properties @@ -12,30 +12,74 @@ # 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. -log4j.rootLogger=INFO, stdout +log4j.rootLogger=INFO, stdout, kafkaAppender + +# ==================================== +# messages going to kafkaAppender +# ==================================== +log4j.logger.kafka=DEBUG, kafkaAppender +log4j.logger.org.I0Itec.zkclient.ZkClient=INFO, kafkaAppender +log4j.logger.org.apache.zookeeper=INFO, kafkaAppender + +# ==================================== +# messages going to zookeeperAppender +# ==================================== +# (comment out this line to redirect ZK-related messages to kafkaAppender +# to allow reading both Kafka and ZK debugging messages in a single file) +#log4j.logger.org.apache.zookeeper=INFO, zookeeperAppender + +# ==================================== +# stdout +# ==================================== log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n -#log4j.appender.fileAppender=org.apache.log4j.FileAppender -#log4j.appender.fileAppender.File=kafka-request.log -#log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout -#log4j.appender.fileAppender.layout.ConversionPattern= %-4r [%t] %-5p %c %x - %m%n +# ==================================== +# fileAppender +# ==================================== +log4j.appender.fileAppender=org.apache.log4j.FileAppender +log4j.appender.fileAppender.File=/tmp/kafka_all_request.log +log4j.appender.fileAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.fileAppender.layout.ConversionPattern=[%d] %p %m (%c)%n +# ==================================== +# kafkaAppender +# ==================================== +log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.kafkaAppender.File=/tmp/kafka.log +log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n +log4j.additivity.kafka=true -# Turn on all our debugging info -#log4j.logger.kafka=INFO -log4j.logger.org.I0Itec.zkclient.ZkClient=INFO -log4j.logger.org.apache.zookeeper=INFO -log4j.logger.kafka.consumer=DEBUG -log4j.logger.kafka.server.EmbeddedConsumer$MirroringThread=TRACE -log4j.logger.kafka.server.KafkaRequestHandlers=TRACE +# ==================================== +# zookeeperAppender +# ==================================== +log4j.appender.zookeeperAppender=org.apache.log4j.DailyRollingFileAppender +log4j.appender.zookeeperAppender.File=/tmp/zookeeper.log +log4j.appender.zookeeperAppender.layout=org.apache.log4j.PatternLayout +log4j.appender.zookeeperAppender.layout.ConversionPattern=[%d] %p %m (%c)%n +log4j.additivity.org.apache.zookeeper=false + +# ==================================== +# other available debugging info +# ==================================== +#log4j.logger.kafka.server.EmbeddedConsumer$MirroringThread=TRACE +#log4j.logger.kafka.server.KafkaRequestHandlers=TRACE #log4j.logger.kafka.producer.async.AsyncProducer=TRACE #log4j.logger.kafka.producer.async.ProducerSendThread=TRACE -log4j.logger.kafka.producer.async.DefaultEventHandler=TRACE +#log4j.logger.kafka.producer.async.DefaultEventHandler=TRACE + +log4j.logger.kafka.consumer=DEBUG log4j.logger.kafka.tools.VerifyConsumerRebalance=DEBUG # to print message checksum from ProducerPerformance -log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG +log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG + +# to print socket buffer size validated by Kafka broker +log4j.logger.kafka.network.Acceptor=DEBUG + +# to print socket buffer size validated by SimpleConsumer +log4j.logger.kafka.consumer.SimpleConsumer=TRACE diff --git a/system_test/broker_failure/config/mirror_producer1.properties b/system_test/broker_failure/config/mirror_producer1.properties index e82bac7413c7d..558d3c943cc61 100644 --- a/system_test/broker_failure/config/mirror_producer1.properties +++ b/system_test/broker_failure/config/mirror_producer1.properties @@ -15,7 +15,7 @@ # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -broker.list=0:localhost:9093 +broker.list=0:localhost:9081 # timeout in ms for connecting to zookeeper zk.connectiontimeout.ms=1000000 diff --git a/system_test/broker_failure/config/mirror_producer2.properties b/system_test/broker_failure/config/mirror_producer2.properties index 30e0fcfc6aa59..d97d6eb01f536 100644 --- a/system_test/broker_failure/config/mirror_producer2.properties +++ b/system_test/broker_failure/config/mirror_producer2.properties @@ -15,7 +15,7 @@ # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -broker.list=0:localhost:9094 +broker.list=0:localhost:9082 # timeout in ms for connecting to zookeeper zk.connectiontimeout.ms=1000000 diff --git a/system_test/broker_failure/config/mirror_producer3.properties b/system_test/broker_failure/config/mirror_producer3.properties index f4cc3d708fdb3..13f9b3539fdfc 100644 --- a/system_test/broker_failure/config/mirror_producer3.properties +++ b/system_test/broker_failure/config/mirror_producer3.properties @@ -15,7 +15,7 @@ # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -broker.list=0:localhost:9095 +broker.list=0:localhost:9083 # timeout in ms for connecting to zookeeper zk.connectiontimeout.ms=1000000 diff --git a/system_test/broker_failure/config/server_source1.properties b/system_test/broker_failure/config/server_source1.properties index f0742304f7fb1..44c319e2af6cf 100644 --- a/system_test/broker_failure/config/server_source1.properties +++ b/system_test/broker_failure/config/server_source1.properties @@ -26,7 +26,7 @@ brokerid=1 num.partitions=1 # the port the socket server runs on -port=9092 +port=9091 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 diff --git a/system_test/broker_failure/config/server_source2.properties b/system_test/broker_failure/config/server_source2.properties index 4b35c2bd79a35..484d2344676b7 100644 --- a/system_test/broker_failure/config/server_source2.properties +++ b/system_test/broker_failure/config/server_source2.properties @@ -26,7 +26,7 @@ brokerid=2 num.partitions=1 # the port the socket server runs on -port=9091 +port=9092 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 diff --git a/system_test/broker_failure/config/server_source3.properties b/system_test/broker_failure/config/server_source3.properties index 5a04a6f0f8478..05fc1636c59c2 100644 --- a/system_test/broker_failure/config/server_source3.properties +++ b/system_test/broker_failure/config/server_source3.properties @@ -26,7 +26,7 @@ brokerid=3 num.partitions=1 # the port the socket server runs on -port=9090 +port=9093 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 diff --git a/system_test/broker_failure/config/server_source4.properties b/system_test/broker_failure/config/server_source4.properties index 3babea4a5a5e6..808c1e662e751 100644 --- a/system_test/broker_failure/config/server_source4.properties +++ b/system_test/broker_failure/config/server_source4.properties @@ -26,7 +26,7 @@ brokerid=4 num.partitions=1 # the port the socket server runs on -port=9096 +port=9094 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 diff --git a/system_test/broker_failure/config/server_target1.properties b/system_test/broker_failure/config/server_target1.properties index b8c778c9d36e2..c7124bdbaec8b 100644 --- a/system_test/broker_failure/config/server_target1.properties +++ b/system_test/broker_failure/config/server_target1.properties @@ -26,7 +26,7 @@ brokerid=1 num.partitions=1 # the port the socket server runs on -port=9093 +port=9081 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 diff --git a/system_test/broker_failure/config/server_target2.properties b/system_test/broker_failure/config/server_target2.properties index f60a1196d3cc4..60e73501e043b 100644 --- a/system_test/broker_failure/config/server_target2.properties +++ b/system_test/broker_failure/config/server_target2.properties @@ -26,7 +26,7 @@ brokerid=2 num.partitions=1 # the port the socket server runs on -port=9094 +port=9082 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 diff --git a/system_test/broker_failure/config/server_target3.properties b/system_test/broker_failure/config/server_target3.properties index eee7a75bbeaac..334bafff60178 100644 --- a/system_test/broker_failure/config/server_target3.properties +++ b/system_test/broker_failure/config/server_target3.properties @@ -26,7 +26,7 @@ brokerid=3 num.partitions=1 # the port the socket server runs on -port=9095 +port=9083 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 diff --git a/system_test/broker_failure/config/whitelisttest.consumer.properties b/system_test/broker_failure/config/whitelisttest.consumer.properties index 5ff54badc1bb0..aaa3f7c07f18a 100644 --- a/system_test/broker_failure/config/whitelisttest.consumer.properties +++ b/system_test/broker_failure/config/whitelisttest.consumer.properties @@ -25,5 +25,5 @@ zk.connectiontimeout.ms=1000000 #consumer group id groupid=group1 -mirror.topics.whitelist=test01 - +mirror.topics.whitelist=test_1,test_2 +autooffset.reset=smallest From a888a9d09e4fc219ab9d8536034ac043093f17df Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Thu, 9 Feb 2012 22:04:32 +0000 Subject: [PATCH 008/151] KAFKA-262 Bug in the consumer rebalancing logic causes one consumer to release partitions that it does not own; patched by Neha Narkhede; reviewed by Jun Rao git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1242552 13f79535-47bb-0310-9956-ffa450edef68 --- .../consumer/ZookeeperConsumerConnector.scala | 83 ++++--------------- .../kafka/message/CompressionUtils.scala | 2 +- .../kafka/producer/KafkaLog4jAppender.scala | 6 +- .../kafka/tools/VerifyConsumerRebalance.scala | 8 +- .../scala/other/kafka/TestKafkaAppender.scala | 2 +- .../other/kafka/TestLinearWriteSpeed.scala | 2 - .../other/kafka/TestLogPerformance.scala | 1 - .../BackwardsCompatibilityTest.scala | 3 +- .../integration/KafkaServerTestHarness.scala | 5 -- .../message/BaseMessageSetTestCases.scala | 1 - .../message/ByteBufferMessageSetTest.scala | 1 - .../kafka/javaapi/producer/ProducerTest.scala | 3 +- .../javaapi/producer/SyncProducerTest.scala | 4 +- .../test/scala/unit/kafka/log/LogTest.scala | 3 +- .../kafka/log4j/KafkaLog4jAppenderTest.scala | 3 +- .../message/BaseMessageSetTestCases.scala | 1 - .../kafka/message/FileMessageSetTest.scala | 3 - .../unit/kafka/message/MessageTest.scala | 3 - .../unit/kafka/network/SocketServerTest.scala | 4 - .../kafka/producer/AsyncProducerTest.scala | 2 +- .../unit/kafka/producer/ProducerTest.scala | 3 +- .../kafka/producer/SyncProducerTest.scala | 4 +- .../kafka/server/ServerShutdownTest.scala | 4 +- 23 files changed, 37 insertions(+), 114 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 221d2a56e9a63..cce2e2d4b0487 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic._ import scala.collection._ import kafka.cluster._ import kafka.utils._ +import mutable.ListBuffer import org.I0Itec.zkclient.exception.ZkNodeExistsException import java.net.InetAddress import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} @@ -91,7 +92,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private val rebalanceLock = new Object private var fetcher: Option[Fetcher] = None private var zkClient: ZkClient = null - private val topicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] + private var topicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] // queues : (topic,consumerThreadId) -> queue private val queues = new Pool[Tuple2[String,String], BlockingQueue[FetchedDataChunk]] private val scheduler = new KafkaScheduler(1, "Kafka-consumer-autocommit-", false) @@ -373,8 +374,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) extends IZkChildListener { private val dirs = new ZKGroupDirs(group) - private var oldPartitionsPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]() - private var oldConsumersPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]() @throws(classOf[Exception]) def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { @@ -390,25 +389,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, deletePath(zkClient, znode) debug("Consumer " + consumerIdString + " releasing " + znode) } + topicRegistry.remove(topic) } } - private def getRelevantTopicMap(myTopicThreadIdsMap: Map[String, Set[String]], - newPartMap: Map[String,List[String]], - oldPartMap: Map[String,List[String]], - newConsumerMap: Map[String,List[String]], - oldConsumerMap: Map[String,List[String]]): Map[String, Set[String]] = { - var relevantTopicThreadIdsMap = new mutable.HashMap[String, Set[String]]() - for ( (topic, consumerThreadIdSet) <- myTopicThreadIdsMap ) - if ( oldPartMap.get(topic) != newPartMap.get(topic) || oldConsumerMap.get(topic) != newConsumerMap.get(topic)) - relevantTopicThreadIdsMap += (topic -> consumerThreadIdSet) - relevantTopicThreadIdsMap - } - def resetState() { topicRegistry.clear - oldConsumersPerTopicMap.clear - oldPartitionsPerTopicMap.clear } def syncedRebalance() { @@ -437,11 +423,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should * clear the cache */ info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") - oldConsumersPerTopicMap.clear() - oldPartitionsPerTopicMap.clear() } - // commit offsets - commitOffsets() // stop all fetchers and clear all the queues to avoid data duplication closeFetchersForQueues(cluster, kafkaMessageStreams, queues.map(q => q._2)) // release all partitions, reset state and retry @@ -457,14 +439,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val myTopicThreadIdsMap = getTopicCount(zkClient, group, consumerIdString).getConsumerThreadIdsPerTopic val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) val partitionsPerTopicMap = getPartitionsForTopics(zkClient, myTopicThreadIdsMap.keys.iterator) - val relevantTopicThreadIdsMap = getRelevantTopicMap(myTopicThreadIdsMap, partitionsPerTopicMap, oldPartitionsPerTopicMap, consumersPerTopicMap, oldConsumersPerTopicMap) - if (relevantTopicThreadIdsMap.size <= 0) { - info("Consumer %s with %s and topic partitions %s doesn't need to rebalance.". - format(consumerIdString, consumersPerTopicMap, partitionsPerTopicMap)) - debug("Partitions per topic cache " + oldPartitionsPerTopicMap) - debug("Consumers per topic cache " + oldConsumersPerTopicMap) - return true - } /** * fetchers must be stopped to avoid data duplication, since if the current @@ -472,14 +446,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * But if we don't stop the fetchers first, this consumer would continue returning data for released * partitions in parallel. So, not stopping the fetchers leads to duplicate data. */ - closeFetchers(cluster, kafkaMessageStreams, relevantTopicThreadIdsMap) + closeFetchers(cluster, kafkaMessageStreams, myTopicThreadIdsMap) releasePartitionOwnership() var partitionOwnershipDecision = new collection.mutable.HashMap[(String, String), String]() - for ((topic, consumerThreadIdSet) <- relevantTopicThreadIdsMap) { - topicRegistry.remove(topic) - topicRegistry.put(topic, new Pool[Partition, PartitionTopicInfo]) + var currentTopicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] + + for ((topic, consumerThreadIdSet) <- myTopicThreadIdsMap) { + currentTopicRegistry.put(topic, new Pool[Partition, PartitionTopicInfo]) val topicDirs = new ZKGroupTopicDirs(group, topic) val curConsumers = consumersPerTopicMap.get(topic).get @@ -507,11 +482,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, for (i <- startPart until startPart + nParts) { val partition = curPartitions(i) info(consumerThreadId + " attempting to claim partition " + partition) - val ownPartition = processPartition(topicDirs, partition, topic, consumerThreadId) - if (!ownPartition) - return false - else // record the partition ownership decision - partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) + addPartitionTopicInfo(currentTopicRegistry, topicDirs, partition, topic, consumerThreadId) + // record the partition ownership decision + partitionOwnershipDecision += ((topic, partition) -> consumerThreadId) } } } @@ -525,8 +498,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, info("Updating the cache") debug("Partitions per topic cache " + partitionsPerTopicMap) debug("Consumers per topic cache " + consumersPerTopicMap) - oldPartitionsPerTopicMap = partitionsPerTopicMap - oldConsumersPerTopicMap = consumersPerTopicMap + topicRegistry = currentTopicRegistry updateFetcher(cluster, kafkaMessageStreams) true }else @@ -579,27 +551,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - private def processPartition(topicDirs: ZKGroupTopicDirs, partition: String, - topic: String, consumerThreadId: String) : Boolean = { - val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition - // check if some other consumer owns this partition at this time - val currentPartitionOwner = readDataMaybeNull(zkClient, partitionOwnerPath) - if(currentPartitionOwner != null) { - if(currentPartitionOwner.equals(consumerThreadId)) { - info(partitionOwnerPath + " exists with value " + currentPartitionOwner + " during connection loss; this is ok") - addPartitionTopicInfo(topicDirs, partition, topic, consumerThreadId) - true - } - else { - info(partitionOwnerPath + " exists with value " + currentPartitionOwner) - false - } - } else { - addPartitionTopicInfo(topicDirs, partition, topic, consumerThreadId) - true - } - } - private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, String), String]): Boolean = { val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => val topic = partitionOwner._1._1 @@ -620,15 +571,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, case e2 => throw e2 } } - val success = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => if(decision) 0 else 1) - if(success > 0) false /* even if one of the partition ownership attempt has failed, return false */ + val hasPartitionOwnershipFailed = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => sum + (if(decision) 0 else 1)) + /* even if one of the partition ownership attempt has failed, return false */ + if(hasPartitionOwnershipFailed > 0) false else true } - private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partitionString: String, + private def addPartitionTopicInfo(currentTopicRegistry: Pool[String, Pool[Partition, PartitionTopicInfo]], + topicDirs: ZKGroupTopicDirs, partitionString: String, topic: String, consumerThreadId: String) { val partition = Partition.parse(partitionString) - val partTopicInfoMap = topicRegistry.get(topic) + val partTopicInfoMap = currentTopicRegistry.get(topic) val znode = topicDirs.consumerOffsetDir + "/" + partition.name val offsetString = readDataMaybeNull(zkClient, znode) diff --git a/core/src/main/scala/kafka/message/CompressionUtils.scala b/core/src/main/scala/kafka/message/CompressionUtils.scala index 7d11993151cb7..607ca7779f593 100644 --- a/core/src/main/scala/kafka/message/CompressionUtils.scala +++ b/core/src/main/scala/kafka/message/CompressionUtils.scala @@ -49,7 +49,7 @@ class GZIPCompression(inputStream: InputStream, outputStream: ByteArrayOutputStr } override def read(a: Array[Byte]): Int = { - gzipIn.read(a) + gzipIn.read(a) } } diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index d971948d799cf..747bbbe498d37 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -18,13 +18,13 @@ package kafka.producer import async.MissingConfigException -import org.apache.log4j.spi.{LoggingEvent, ErrorCode} +import org.apache.log4j.spi.LoggingEvent import org.apache.log4j.AppenderSkeleton import org.apache.log4j.helpers.LogLog -import kafka.utils.{Utils, Logging} +import kafka.utils.Logging import kafka.serializer.Encoder import java.util.{Properties, Date} -import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} +import kafka.message.Message import scala.collection._ class KafkaLog4jAppender extends AppenderSkeleton with Logging { diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala index e64a23481b112..2ad1a20ff2f94 100644 --- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala +++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala @@ -57,9 +57,9 @@ object VerifyConsumerRebalance extends Logging { // check if the rebalancing operation succeeded. try { if(validateRebalancingOperation(zkClient, group)) - info("Rebalance operation successful !") + println("Rebalance operation successful !") else - error("Rebalance operation failed !") + println("Rebalance operation failed !") } catch { case e2: Throwable => error("Error while verifying current rebalancing operation", e2) } @@ -132,6 +132,4 @@ object VerifyConsumerRebalance extends Logging { rebalanceSucceeded } - - -} \ No newline at end of file +} diff --git a/core/src/test/scala/other/kafka/TestKafkaAppender.scala b/core/src/test/scala/other/kafka/TestKafkaAppender.scala index dc200d04d819f..8328e99e24106 100644 --- a/core/src/test/scala/other/kafka/TestKafkaAppender.scala +++ b/core/src/test/scala/other/kafka/TestKafkaAppender.scala @@ -18,7 +18,7 @@ package kafka import message.Message -import org.apache.log4j.{Logger, PropertyConfigurator} +import org.apache.log4j.PropertyConfigurator import kafka.utils.Logging import serializer.Encoder diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 79347d83825c7..d6fc65da179a2 100644 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -17,8 +17,6 @@ package kafka -import java.net.URI -import java.util.Arrays.asList import java.io._ import java.nio._ import java.nio.channels._ diff --git a/core/src/test/scala/other/kafka/TestLogPerformance.scala b/core/src/test/scala/other/kafka/TestLogPerformance.scala index 2a41e2c7cd8de..e46151ff8e7c9 100644 --- a/core/src/test/scala/other/kafka/TestLogPerformance.scala +++ b/core/src/test/scala/other/kafka/TestLogPerformance.scala @@ -17,7 +17,6 @@ package kafka.log -import kafka.log._ import kafka.message._ import kafka.utils.{TestUtils, Utils} diff --git a/core/src/test/scala/unit/kafka/integration/BackwardsCompatibilityTest.scala b/core/src/test/scala/unit/kafka/integration/BackwardsCompatibilityTest.scala index aa4b9903716de..9febfc8bb3e05 100644 --- a/core/src/test/scala/unit/kafka/integration/BackwardsCompatibilityTest.scala +++ b/core/src/test/scala/unit/kafka/integration/BackwardsCompatibilityTest.scala @@ -22,10 +22,9 @@ import org.scalatest.junit.JUnit3Suite import org.apache.log4j.Logger import java.util.Properties import kafka.consumer.SimpleConsumer -import kafka.utils.{Utils, TestUtils} +import kafka.utils.TestUtils import kafka.api.{OffsetRequest, FetchRequest} import junit.framework.Assert._ -import java.io.File class BackwardsCompatibilityTest extends JUnit3Suite { diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 561f609535a1e..6b825f5cf1c01 100644 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -17,11 +17,6 @@ package kafka.integration -import java.util.Properties -import junit.framework.Assert._ -import kafka.producer._ -import kafka.consumer._ -import kafka.message._ import kafka.server._ import kafka.utils.{Utils, TestUtils} import org.scalatest.junit.JUnit3Suite diff --git a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala index d90a9f549a3f5..c48f7dc748766 100644 --- a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala @@ -35,7 +35,6 @@ trait BaseMessageSetTestCases extends JUnitSuite { @Test def testWrittenEqualsRead { - import scala.collection.JavaConversions._ val messageSet = createMessageSet(messages) TestUtils.checkEquals(messages.iterator, toMessageIterator(messageSet)) } diff --git a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala index 67abcfc1bd4cc..86154d946cb6e 100644 --- a/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala @@ -18,7 +18,6 @@ package kafka.javaapi.message import java.nio._ -import junit.framework.TestCase import junit.framework.Assert._ import org.junit.Test import kafka.message.{DefaultCompressionCodec, CompressionCodec, NoCompressionCodec, Message} diff --git a/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala index 3f0fb3a238ee3..296bb072f935a 100644 --- a/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala @@ -24,7 +24,6 @@ import kafka.zk.EmbeddedZookeeper import kafka.utils.{TestZKUtils, TestUtils} import org.junit.{After, Before, Test} import junit.framework.Assert -import collection.mutable.HashMap import org.easymock.EasyMock import kafka.utils.Utils import java.util.concurrent.ConcurrentHashMap @@ -34,7 +33,7 @@ import org.scalatest.junit.JUnitSuite import kafka.producer.{SyncProducerConfig, Partitioner, ProducerConfig, DefaultPartitioner} import kafka.producer.ProducerPool import kafka.javaapi.message.ByteBufferMessageSet -import kafka.producer.async.{AsyncProducer, AsyncProducerConfig} +import kafka.producer.async.AsyncProducer import kafka.javaapi.Implicits._ import kafka.serializer.{StringEncoder, Encoder} import kafka.javaapi.consumer.SimpleConsumer diff --git a/core/src/test/scala/unit/kafka/javaapi/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/javaapi/producer/SyncProducerTest.scala index 526d53941ab6e..1923d240585d3 100644 --- a/core/src/test/scala/unit/kafka/javaapi/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/producer/SyncProducerTest.scala @@ -17,11 +17,11 @@ package kafka.javaapi.producer -import junit.framework.{Assert, TestCase} +import junit.framework.Assert import kafka.utils.SystemTime import kafka.utils.TestUtils import kafka.server.{KafkaServer, KafkaConfig} -import org.apache.log4j.{Logger, Level} +import org.apache.log4j.Logger import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} import java.util.Properties diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 487d79f76d103..32f6f8f05b8d9 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -18,14 +18,13 @@ package kafka.log import java.io._ -import java.nio._ import java.util.ArrayList import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} import kafka.utils.{Utils, TestUtils, Range} import kafka.common.OffsetOutOfRangeException -import kafka.message.{NoCompressionCodec, MessageSet, ByteBufferMessageSet, Message} +import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} class LogTest extends JUnitSuite { diff --git a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala index f134b3e8e19f6..7f67eb3d808a1 100644 --- a/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala +++ b/core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala @@ -23,13 +23,12 @@ import java.util.Properties import java.io.File import kafka.consumer.SimpleConsumer import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.TestUtils import kafka.utils.{TestUtils, TestZKUtils,Utils, Logging} import kafka.zk.EmbeddedZookeeper import junit.framework.Assert._ import kafka.api.FetchRequest import kafka.serializer.Encoder -import kafka.message.{MessageSet, Message} +import kafka.message.Message import kafka.producer.async.MissingConfigException import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} diff --git a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala index 7317fa486c8f0..a6dc642f54a94 100644 --- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala @@ -17,7 +17,6 @@ package kafka.message -import java.util.Arrays import junit.framework.Assert._ import kafka.utils.TestUtils._ import org.scalatest.junit.JUnitSuite diff --git a/core/src/test/scala/unit/kafka/message/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/FileMessageSetTest.scala index 4b8ab1958188e..a683963c9f1c5 100644 --- a/core/src/test/scala/unit/kafka/message/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/FileMessageSetTest.scala @@ -18,12 +18,9 @@ package kafka.message import java.nio._ -import java.util.Arrays -import junit.framework.TestCase import junit.framework.Assert._ import kafka.utils.TestUtils._ import org.junit.Test -import kafka.message._ class FileMessageSetTest extends BaseMessageSetTestCases { diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 0df25da0fe82c..4e3184cf178df 100644 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -19,9 +19,6 @@ package kafka.message import java.util._ import java.nio._ -import java.nio.channels._ -import java.io._ -import junit.framework.TestCase import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{Before, Test} diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 00fc5f534cb0e..cae66516ff79b 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -19,13 +19,9 @@ package kafka.network; import java.net._ import java.io._ -import java.nio._ -import java.nio.channels._ import org.junit._ -import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import kafka.utils.TestUtils -import kafka.network._ import java.util.Random import org.apache.log4j._ diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index bff30f59bac59..b87dc3d87acda 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -17,7 +17,7 @@ package kafka.producer -import junit.framework.{Assert, TestCase} +import junit.framework.Assert import java.util.Properties import org.easymock.EasyMock import kafka.api.ProducerRequest diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index b66a34daa256b..7a6273cc52705 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -17,14 +17,13 @@ package kafka.producer -import async.{AsyncProducerConfig, AsyncProducer} +import async.AsyncProducer import java.util.Properties import org.apache.log4j.{Logger, Level} import kafka.server.{KafkaRequestHandlers, KafkaServer, KafkaConfig} import kafka.zk.EmbeddedZookeeper import org.junit.{After, Before, Test} import junit.framework.Assert -import collection.mutable.HashMap import org.easymock.EasyMock import java.util.concurrent.ConcurrentHashMap import kafka.cluster.Partition diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 29dbe7ea691ea..ac811944c702f 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -17,11 +17,11 @@ package kafka.producer -import junit.framework.{Assert, TestCase} +import junit.framework.Assert import kafka.utils.SystemTime import kafka.utils.TestUtils import kafka.server.{KafkaServer, KafkaConfig} -import org.apache.log4j.{Logger, Level} +import org.apache.log4j.Logger import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} import kafka.common.MessageSizeTooLargeException diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index f0e862b44c346..f6806762a6006 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -20,13 +20,11 @@ import kafka.utils.TestUtils import java.io.File import kafka.utils.Utils import kafka.api.FetchRequest -import kafka.integration.ProducerConsumerTestHarness import kafka.producer.{SyncProducer, SyncProducerConfig} import kafka.consumer.SimpleConsumer import java.util.Properties import org.scalatest.junit.JUnitSuite -import junit.framework.{Assert, TestCase} -import org.junit.{After, Before, Test} +import org.junit.Test import junit.framework.Assert._ import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} From 6a3c2f2032c5e910f634a29bfead555b63433465 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 10 Feb 2012 06:52:37 +0000 Subject: [PATCH 009/151] trivial change to reduce default fetcher queue size git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1242696 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/consumer/ConsumerConfig.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index c4b3e71b94947..c2ac74653368b 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -29,7 +29,7 @@ object ConsumerConfig { val DefaultFetcherBackoffMs = 1000 val AutoCommit = true val AutoCommitInterval = 10 * 1000 - val MaxQueuedChunks = 100 + val MaxQueuedChunks = 10 val MaxRebalanceRetries = 4 val AutoOffsetReset = OffsetRequest.SmallestTimeString val ConsumerTimeoutMs = -1 From 732c27ff3e8bc6d5a9511ee64de275df49adf73f Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 13 Feb 2012 22:31:02 +0000 Subject: [PATCH 010/151] Add a queue of zookeeper notifications in the zookeeper consumer to reduce the number of rebalancing attempts; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-265 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1243721 13f79535-47bb-0310-9956-ffa450edef68 --- .../consumer/ZookeeperConsumerConnector.scala | 36 +++++++++++++++++-- 1 file changed, 33 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index cce2e2d4b0487..aa03bd7a33c4d 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -19,10 +19,10 @@ package kafka.consumer import java.util.concurrent._ import java.util.concurrent.atomic._ +import locks.ReentrantLock import scala.collection._ import kafka.cluster._ import kafka.utils._ -import mutable.ListBuffer import org.I0Itec.zkclient.exception.ZkNodeExistsException import java.net.InetAddress import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} @@ -373,11 +373,41 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, class ZKRebalancerListener[T](val group: String, val consumerIdString: String, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) extends IZkChildListener { - private val dirs = new ZKGroupDirs(group) + private var isWatcherTriggered = false + private val lock = new ReentrantLock + private val cond = lock.newCondition() + private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") { + override def run() { + info("starting watcher executor thread for consumer " + consumerIdString) + while (!isShuttingDown.get) { + try { + lock.lock() + try { + if (!isWatcherTriggered) + cond.wait() + } finally { + isWatcherTriggered = false + lock.unlock() + } + syncedRebalance + } catch { + case t => error("error during syncedRebalance", t) + } + } + info("stopping watcher executor thread for consumer " + consumerIdString) + } + } + watcherExecutorThread.start() @throws(classOf[Exception]) def handleChildChange(parentPath : String, curChilds : java.util.List[String]) { - syncedRebalance + lock.lock() + try { + isWatcherTriggered = true + cond.signalAll() + } finally { + lock.unlock() + } } private def releasePartitionOwnership()= { From 0bd3b9c3f6df76043afbdce817d33d7531f1eecd Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 14 Feb 2012 06:19:34 +0000 Subject: [PATCH 011/151] time-based reconnect in producer; patched by Yang Ye; reviewed by Jun Rao; KAFKA-268 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1243786 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/producer/SyncProducer.scala | 6 +++++- core/src/main/scala/kafka/producer/SyncProducerConfig.scala | 3 +++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index e7372558e298c..a94dc0bb638ab 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -40,6 +40,8 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { private val MaxConnectBackoffMs = 60000 private var channel : SocketChannel = null private var sentOnConnection = 0 + private var lastConnectionTime = System.currentTimeMillis + private val lock = new Object() @volatile private var shutdown: Boolean = false @@ -94,10 +96,12 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { } // TODO: do we still need this? sentOnConnection += 1 - if(sentOnConnection >= config.reconnectInterval) { + + if(sentOnConnection >= config.reconnectInterval || (config.reconnectTimeInterval >= 0 && System.currentTimeMillis - lastConnectionTime >= config.reconnectTimeInterval)) { disconnect() channel = connect() sentOnConnection = 0 + lastConnectionTime = System.currentTimeMillis } val endTime = SystemTime.nanoseconds SyncProducerStats.recordProduceRequest(endTime - startTime) diff --git a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala index b11b8c6ef2f0e..4b78a4b000a59 100644 --- a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala @@ -40,5 +40,8 @@ trait SyncProducerConfigShared { val reconnectInterval = Utils.getInt(props, "reconnect.interval", 30000) + /** negative reconnect time interval means disabling this time-based reconnect feature */ + var reconnectTimeInterval = Utils.getInt(props, "reconnect.time.interval.ms", 1000*1000*10) + val maxMessageSize = Utils.getInt(props, "max.message.size", 1000000) } From 5c4ce616994112a68f8864520c4a2a6c7b545e30 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 15 Feb 2012 22:04:00 +0000 Subject: [PATCH 012/151] add JMX on broker to track bytes/messages per topic; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-272 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1244755 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/log/Log.scala | 3 + .../kafka/server/KafkaRequestHandlers.scala | 56 ++++++++++++++++++- 2 files changed, 56 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 2dd1113dd53b4..4b18c6ba07a9e 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -25,6 +25,7 @@ import kafka.utils._ import kafka.common._ import kafka.api.OffsetRequest import java.util._ +import kafka.server.BrokerTopicStat private[log] object Log { val FileSuffix = ".kafka" @@ -207,6 +208,8 @@ private[log] class Log(val dir: File, val maxSize: Long, val flushInterval: Int, numberOfMessages += 1; } + BrokerTopicStat.getBrokerTopicStat(getTopicName).recordMessagesIn(numberOfMessages) + BrokerTopicStat.getBrokerAllTopicStat.recordMessagesIn(numberOfMessages) logStats.recordAppendedMessages(numberOfMessages) // they are valid, insert them in the log diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala b/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala index 2317372140061..d67d01414bd08 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala @@ -23,8 +23,8 @@ import kafka.network._ import kafka.message._ import kafka.api._ import kafka.common.ErrorMapping -import kafka.utils.SystemTime -import kafka.utils.Logging +import java.util.concurrent.atomic.AtomicLong +import kafka.utils._ /** * Logic to handle the various Kafka requests @@ -69,6 +69,8 @@ private[kafka] class KafkaRequestHandlers(val logManager: LogManager) extends Lo logManager.getOrCreateLog(request.topic, partition).append(request.messages) trace(request.messages.sizeInBytes + " bytes written to logs.") request.messages.foreach(m => trace("wrote message %s to disk".format(m.message.checksum))) + BrokerTopicStat.getBrokerTopicStat(request.topic).recordBytesIn(request.messages.sizeInBytes) + BrokerTopicStat.getBrokerAllTopicStat.recordBytesIn(request.messages.sizeInBytes) } catch { case e => @@ -100,8 +102,11 @@ private[kafka] class KafkaRequestHandlers(val logManager: LogManager) extends Lo try { trace("Fetching log segment for topic, partition, offset, maxSize = " + fetchRequest) val log = logManager.getLog(fetchRequest.topic, fetchRequest.partition) - if (log != null) + if (log != null) { response = new MessageSetSend(log.read(fetchRequest.offset, fetchRequest.maxSize)) + BrokerTopicStat.getBrokerTopicStat(fetchRequest.topic).recordBytesOut(response.messages.sizeInBytes) + BrokerTopicStat.getBrokerAllTopicStat.recordBytesOut(response.messages.sizeInBytes) + } else response = new MessageSetSend() } @@ -122,3 +127,48 @@ private[kafka] class KafkaRequestHandlers(val logManager: LogManager) extends Lo Some(response) } } + +trait BrokerTopicStatMBean { + def getMessagesIn: Long + def getBytesIn: Long + def getBytesOut: Long +} + +@threadsafe +class BrokerTopicStat extends BrokerTopicStatMBean { + private val numCumulatedMessagesIn = new AtomicLong(0) + private val numCumulatedBytesIn = new AtomicLong(0) + private val numCumulatedBytesOut = new AtomicLong(0) + + def getMessagesIn: Long = numCumulatedMessagesIn.get + + def recordMessagesIn(nMessages: Int) = numCumulatedMessagesIn.getAndAdd(nMessages) + + def getBytesIn: Long = numCumulatedBytesIn.get + + def recordBytesIn(nBytes: Long) = numCumulatedBytesIn.getAndAdd(nBytes) + + def getBytesOut: Long = numCumulatedBytesOut.get + + def recordBytesOut(nBytes: Long) = numCumulatedBytesOut.getAndAdd(nBytes) +} + +object BrokerTopicStat extends Logging { + private val stats = new Pool[String, BrokerTopicStat] + private val allTopicStat = new BrokerTopicStat + Utils.registerMBean(allTopicStat, "kafka:type=kafka.BrokerAllTopicStat") + + def getBrokerAllTopicStat(): BrokerTopicStat = allTopicStat + + def getBrokerTopicStat(topic: String): BrokerTopicStat = { + var stat = stats.get(topic) + if (stat == null) { + stat = new BrokerTopicStat + if (stats.putIfNotExists(topic, stat) == null) + Utils.registerMBean(stat, "kafka:type=kafka.BrokerTopicStat." + topic) + else + stat = stats.get(topic) + } + return stat + } +} From 3e79116e7a7c725cafae1b5248a99837641a4ca2 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 15 Feb 2012 22:22:11 +0000 Subject: [PATCH 013/151] fix condition to use await instead of wait; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-265 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1244765 13f79535-47bb-0310-9956-ffa450edef68 --- .../main/scala/kafka/consumer/ZookeeperConsumerConnector.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index aa03bd7a33c4d..a565952406bd5 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -384,7 +384,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, lock.lock() try { if (!isWatcherTriggered) - cond.wait() + cond.await() } finally { isWatcherTriggered = false lock.unlock() From 2ead8a49ea1c7e634c889e9425fd5eb209e7f1ff Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 15 Feb 2012 23:51:27 +0000 Subject: [PATCH 014/151] trivial fix to add trace logging in FetcherRunnable git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1244792 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/consumer/FetcherRunnable.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/kafka/consumer/FetcherRunnable.scala b/core/src/main/scala/kafka/consumer/FetcherRunnable.scala index f573f7f9d5ff7..6f0ea79704d37 100644 --- a/core/src/main/scala/kafka/consumer/FetcherRunnable.scala +++ b/core/src/main/scala/kafka/consumer/FetcherRunnable.scala @@ -58,6 +58,7 @@ class FetcherRunnable(val name: String, trace("fetch request: " + fetches.toString) val response = simpleConsumer.multifetch(fetches : _*) + trace("recevied response from fetch request: " + fetches.toString) var read = 0L From 6e1b2ebe3278c99e1b3ea89a47c60497cb964d44 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 17 Feb 2012 02:14:25 +0000 Subject: [PATCH 015/151] shutdown watch executor thread properly; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-265 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1245295 13f79535-47bb-0310-9956-ffa450edef68 --- .../scala/kafka/consumer/ZookeeperConsumerConnector.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index a565952406bd5..b459da96f62f4 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -379,17 +379,20 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") { override def run() { info("starting watcher executor thread for consumer " + consumerIdString) + var doRebalance = false while (!isShuttingDown.get) { try { lock.lock() try { if (!isWatcherTriggered) - cond.await() + cond.await(1000, TimeUnit.MILLISECONDS) // wake up periodically so that it can check the shutdown flag } finally { + doRebalance = isWatcherTriggered isWatcherTriggered = false lock.unlock() } - syncedRebalance + if (doRebalance) + syncedRebalance } catch { case t => error("error during syncedRebalance", t) } From c28b3a61cce3288d3c0b620d1ef73d936989d3c5 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Fri, 17 Feb 2012 02:33:07 +0000 Subject: [PATCH 016/151] KAFKA-274 Handle corrupted messages cleanly; patched by nehanarkhede; reviewed by junrao git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1245299 13f79535-47bb-0310-9956-ffa450edef68 --- .../kafka/message/ByteBufferMessageSet.scala | 6 ++++++ .../kafka/message/InvalidMessageException.scala | 4 +++- .../main/scala/kafka/producer/SyncProducer.scala | 15 ++++++++++----- 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index c1085826e515e..0da73e5fa7e26 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -38,6 +38,8 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, private val errorCode: Int = ErrorMapping.NoError) extends MessageSet with Logging { private var validByteCount = -1L private var shallowValidByteCount = -1L + if(sizeInBytes > Int.MaxValue) + throw new InvalidMessageSizeException("Message set cannot be larger than " + Int.MaxValue) def this(compressionCodec: CompressionCodec, messages: Message*) { this(MessageSet.createByteBuffer(compressionCodec, messages:_*), 0L, ErrorMapping.NoError) @@ -108,12 +110,16 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, val newMessage = new Message(message) newMessage.compressionCodec match { case NoCompressionCodec => + if(!newMessage.isValid) + throw new InvalidMessageException("Uncompressed essage is invalid") debug("Message is uncompressed. Valid byte count = %d".format(currValidBytes)) innerIter = null currValidBytes += 4 + size trace("currValidBytes = " + currValidBytes) new MessageAndOffset(newMessage, currValidBytes) case _ => + if(!newMessage.isValid) + throw new InvalidMessageException("Compressed message is invalid") debug("Message is compressed. Valid byte count = %d".format(currValidBytes)) innerIter = CompressionUtils.decompress(newMessage).deepIterator if (!innerIter.hasNext) { diff --git a/core/src/main/scala/kafka/message/InvalidMessageException.scala b/core/src/main/scala/kafka/message/InvalidMessageException.scala index 9e33c9fb53103..9f0d6e9c1c8b0 100644 --- a/core/src/main/scala/kafka/message/InvalidMessageException.scala +++ b/core/src/main/scala/kafka/message/InvalidMessageException.scala @@ -20,4 +20,6 @@ package kafka.message /** * Indicates that a message failed its checksum and is corrupt */ -class InvalidMessageException extends RuntimeException +class InvalidMessageException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index a94dc0bb638ab..e0cd2591010c7 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -46,10 +46,15 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { @volatile private var shutdown: Boolean = false - debug("Instantiating Scala Sync Producer") + trace("Instantiating Scala Sync Producer") private def verifySendBuffer(buffer : ByteBuffer) = { - if (logger.isTraceEnabled) { + /** + * This seems a little convoluted, but the idea is to turn on verification simply changing log4j settings + * Also, when verification is turned on, care should be taken to see that the logs don't fill up with unnecessary + * data. So, leaving the rest of the logging at TRACE, while errors should be logged at ERROR level + */ + if (logger.isDebugEnabled) { trace("verifying sendbuffer of size " + buffer.limit) val requestTypeId = buffer.getShort() if (requestTypeId == RequestKeys.MultiProduce) { @@ -59,17 +64,17 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { try { for (messageAndOffset <- produce.messages) if (!messageAndOffset.message.isValid) - trace("topic " + produce.topic + " is invalid") + throw new InvalidMessageException("Message for topic " + produce.topic + " is invalid") } catch { case e: Throwable => - trace("error iterating messages ", e) + error("error iterating messages ", e) } } } catch { case e: Throwable => - trace("error verifying sendbuffer ", e) + error("error verifying sendbuffer ", e) } } } From ff562472ddf834e092cc2a8124dad59e763017db Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Fri, 17 Feb 2012 04:14:47 +0000 Subject: [PATCH 017/151] test access, fixed space git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1245316 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/utils/Logging.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/utils/Logging.scala b/core/src/main/scala/kafka/utils/Logging.scala index e8dbdd917773d..2e664f5c26bc7 100644 --- a/core/src/main/scala/kafka/utils/Logging.scala +++ b/core/src/main/scala/kafka/utils/Logging.scala @@ -72,7 +72,7 @@ trait Logging { logger.warn(msg,e) } - def error(msg: => String):Unit = { + def error(msg: => String): Unit = { logger.error(msg) } def error(e: => Throwable): Any = { @@ -91,4 +91,4 @@ trait Logging { def fatal(msg: => String, e: => Throwable) = { logger.fatal(msg,e) } -} \ No newline at end of file +} From a23d396a5a2f5c8685a404ce8478540c13fac6d4 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 17 Feb 2012 21:54:29 +0000 Subject: [PATCH 018/151] trivial fix for kafka-producer-perf-test.sh git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1245779 13f79535-47bb-0310-9956-ffa450edef68 --- bin/kafka-producer-perf-test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/kafka-producer-perf-test.sh b/bin/kafka-producer-perf-test.sh index 820c8beffa85f..25c4dce5bef9e 100755 --- a/bin/kafka-producer-perf-test.sh +++ b/bin/kafka-producer-perf-test.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -$(dirname $0)/kafka-run-class.sh kafka.tools.ProducerPerformance $@ +$(dirname $0)/kafka-run-class.sh kafka.perf.ProducerPerformance $@ From 95ee38728cb1d8eb9b62a73eb5ad26902142a1fd Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 20 Feb 2012 22:24:52 +0000 Subject: [PATCH 019/151] make time-based reconnect starting at a random time; patched by Yang Ye; reviewed by Jun Rao; KAFKA-268 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1291490 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/producer/SyncProducer.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index e0cd2591010c7..45f2b86f3bfac 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -26,9 +26,11 @@ import kafka.api._ import scala.math._ import kafka.common.MessageSizeTooLargeException import java.nio.ByteBuffer +import java.util.Random object SyncProducer { val RequestKey: Short = 0 + val randomGenerator = new Random } /* @@ -40,7 +42,8 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { private val MaxConnectBackoffMs = 60000 private var channel : SocketChannel = null private var sentOnConnection = 0 - private var lastConnectionTime = System.currentTimeMillis + /** make time-based reconnect starting at a random time **/ + private var lastConnectionTime = System.currentTimeMillis - SyncProducer.randomGenerator.nextDouble() * config.reconnectInterval private val lock = new Object() @volatile From 7066d1c38b66d4de2d02fc1ef2fa8e7908990d07 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 21 Feb 2012 00:25:55 +0000 Subject: [PATCH 020/151] A tool to GET Zookeeper partition-offset and output to files; patched by John Fung; reviewed by Jun Rao; KAFKA-254 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1291535 13f79535-47bb-0310-9956-ffa450edef68 --- .../scala/kafka/tools/ExportZkOffsets.scala | 123 ++++++++++++++++++ 1 file changed, 123 insertions(+) create mode 100644 core/src/main/scala/kafka/tools/ExportZkOffsets.scala diff --git a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala new file mode 100644 index 0000000000000..725a4d310020e --- /dev/null +++ b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala @@ -0,0 +1,123 @@ +/** + * 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 kafka.tools + +import java.io.FileWriter +import joptsimple._ +import kafka.utils.{Logging, ZkUtils, ZKStringSerializer,ZKGroupTopicDirs} +import org.I0Itec.zkclient.ZkClient + + +/** + * A utility that retrieve the offset of broker partitions in ZK and + * prints to an output file in the following format: + * + * /consumers/group1/offsets/topic1/1-0:286894308 + * /consumers/group1/offsets/topic1/2-0:284803985 + * + * This utility expects 3 arguments: + * 1. Zk host:port string + * 2. group name (all groups implied if omitted) + * 3. output filename + * + * To print debug message, add the following line to log4j.properties: + * log4j.logger.kafka.tools.ExportZkOffsets$=DEBUG + * (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin) + */ +object ExportZkOffsets extends Logging { + + def main(args: Array[String]) { + val parser = new OptionParser + + val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.") + .withRequiredArg() + .defaultsTo("localhost:2181") + .ofType(classOf[String]) + val groupOpt = parser.accepts("group", "Consumer group.") + .withRequiredArg() + .ofType(classOf[String]) + val outFileOpt = parser.accepts("output-file", "Output file") + .withRequiredArg() + .ofType(classOf[String]) + parser.accepts("help", "Print this message.") + + val options = parser.parse(args : _*) + + if (options.has("help")) { + parser.printHelpOn(System.out) + System.exit(0) + } + + for (opt <- List(zkConnectOpt, outFileOpt)) { + if (!options.has(opt)) { + System.err.println("Missing required argument: %s".format(opt)) + parser.printHelpOn(System.err) + System.exit(1) + } + } + + val zkConnect = options.valueOf(zkConnectOpt) + val groups = options.valuesOf(groupOpt) + val outfile = options.valueOf(outFileOpt) + + var zkClient : ZkClient = null + val fileWriter : FileWriter = new FileWriter(outfile) + + try { + zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + + var consumerGroups: Seq[String] = null + + if (groups.size == 0) { + consumerGroups = ZkUtils.getChildren(zkClient, ZkUtils.ConsumersPath).toList + } + else { + import scala.collection.JavaConversions._ + consumerGroups = groups + } + + for (consumerGrp <- consumerGroups) { + val topicsList = getTopicsList(zkClient, consumerGrp) + + for (topic <- topicsList) { + val bidPidList = getBrokeridPartition(zkClient, consumerGrp, topic) + + for (bidPid <- bidPidList) { + val zkGrpTpDir = new ZKGroupTopicDirs(consumerGrp,topic) + val offsetPath = zkGrpTpDir.consumerOffsetDir + "/" + bidPid + val offsetVal = ZkUtils.readDataMaybeNull(zkClient, offsetPath) + fileWriter.write(offsetPath + ":" + offsetVal + "\n") + debug(offsetPath + " => " + offsetVal) + } + } + } + } + finally { + fileWriter.flush() + fileWriter.close() + } + } + + private def getBrokeridPartition(zkClient: ZkClient, consumerGroup: String, topic: String): List[String] = { + return ZkUtils.getChildrenParentMayNotExist(zkClient, "/consumers/%s/offsets/%s".format(consumerGroup, topic)).toList + } + + private def getTopicsList(zkClient: ZkClient, consumerGroup: String): List[String] = { + return ZkUtils.getChildren(zkClient, "/consumers/%s/offsets".format(consumerGroup)).toList + } +} From 22d80462b15e24fc63960254bdfd8be74ddc14ba Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 21 Feb 2012 00:26:42 +0000 Subject: [PATCH 021/151] A tool to UPDATE Zookeeper partition-offset with input from a file; patched by John Fung; reviewed by Jun Rao; KAFKA-255 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1291536 13f79535-47bb-0310-9956-ffa450edef68 --- .../scala/kafka/tools/ImportZkOffsets.scala | 112 ++++++++++++++++++ 1 file changed, 112 insertions(+) create mode 100644 core/src/main/scala/kafka/tools/ImportZkOffsets.scala diff --git a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala new file mode 100644 index 0000000000000..63519e12ed504 --- /dev/null +++ b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala @@ -0,0 +1,112 @@ +/** + * 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 kafka.tools + +import java.io.BufferedReader +import java.io.FileReader +import joptsimple._ +import kafka.utils.{Logging, ZkUtils,ZKStringSerializer} +import org.I0Itec.zkclient.ZkClient + + +/** + * A utility that updates the offset of broker partitions in ZK. + * + * This utility expects 2 input files as arguments: + * 1. consumer properties file + * 2. a file contains partition offsets data such as: + * (This output data file can be obtained by running kafka.tools.ExportZkOffsets) + * + * /consumers/group1/offsets/topic1/3-0:285038193 + * /consumers/group1/offsets/topic1/1-0:286894308 + * + * To print debug message, add the following line to log4j.properties: + * log4j.logger.kafka.tools.ImportZkOffsets$=DEBUG + * (for eclipse debugging, copy log4j.properties to the binary directory in "core" such as core/bin) + */ +object ImportZkOffsets extends Logging { + + def main(args: Array[String]) { + val parser = new OptionParser + + val zkConnectOpt = parser.accepts("zkconnect", "ZooKeeper connect string.") + .withRequiredArg() + .defaultsTo("localhost:2181") + .ofType(classOf[String]) + val inFileOpt = parser.accepts("input-file", "Input file") + .withRequiredArg() + .ofType(classOf[String]) + parser.accepts("help", "Print this message.") + + val options = parser.parse(args : _*) + + if (options.has("help")) { + parser.printHelpOn(System.out) + System.exit(0) + } + + for (opt <- List(inFileOpt)) { + if (!options.has(opt)) { + System.err.println("Missing required argument: %s".format(opt)) + parser.printHelpOn(System.err) + System.exit(1) + } + } + + val zkConnect = options.valueOf(zkConnectOpt) + val partitionOffsetFile = options.valueOf(inFileOpt) + + val zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) + val partitionOffsets: Map[String,String] = getPartitionOffsetsFromFile(partitionOffsetFile) + + updateZkOffsets(zkClient, partitionOffsets) + } + + private def getPartitionOffsetsFromFile(filename: String):Map[String,String] = { + val fr = new FileReader(filename) + val br = new BufferedReader(fr) + var partOffsetsMap: Map[String,String] = Map() + + var s: String = br.readLine() + while ( s != null && s.length() >= 1) { + val tokens = s.split(":") + + partOffsetsMap += tokens(0) -> tokens(1) + debug("adding node path [" + s + "]") + + s = br.readLine() + } + + return partOffsetsMap + } + + private def updateZkOffsets(zkClient: ZkClient, partitionOffsets: Map[String,String]): Unit = { + val cluster = ZkUtils.getCluster(zkClient) + var partitions: List[String] = Nil + + for ((partition, offset) <- partitionOffsets) { + debug("updating [" + partition + "] with offset [" + offset + "]") + + try { + ZkUtils.updatePersistentPath(zkClient, partition, offset.toString) + } catch { + case e => e.printStackTrace() + } + } + } +} From 333966f50ea42ba41ff243d1ba5571a2fa8b9a60 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 23 Feb 2012 22:57:40 +0000 Subject: [PATCH 022/151] Add a shallow iterator to the ByteBufferMessageSet; patched by Yang Ye; reviewed by Jun Rao; KAFKA-277 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1293010 13f79535-47bb-0310-9956-ffa450edef68 --- .../kafka/message/ByteBufferMessageSet.scala | 87 ++++++++++++------- .../scala/kafka/producer/SyncProducer.scala | 10 +-- .../kafka/producer/SyncProducerTest.scala | 35 +++++++- 3 files changed, 89 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 0da73e5fa7e26..6f4355f49add0 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -18,10 +18,10 @@ package kafka.message import kafka.utils.Logging -import kafka.common.{InvalidMessageSizeException, ErrorMapping} import java.nio.ByteBuffer import java.nio.channels._ import kafka.utils.IteratorTemplate +import kafka.common.{MessageSizeTooLargeException, InvalidMessageSizeException, ErrorMapping} /** * A sequence of messages stored in a byte buffer @@ -61,7 +61,7 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, private def shallowValidBytes: Long = { if(shallowValidByteCount < 0) { - val iter = deepIterator + val iter = this.internalIterator() while(iter.hasNext) { val messageAndOffset = iter.next shallowValidByteCount = messageAndOffset.offset @@ -75,9 +75,21 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, def writeTo(channel: GatheringByteChannel, offset: Long, size: Long): Long = channel.write(buffer.duplicate) - override def iterator: Iterator[MessageAndOffset] = deepIterator + override def iterator: Iterator[MessageAndOffset] = internalIterator() - private def deepIterator(): Iterator[MessageAndOffset] = { + + def verifyMessageSize(maxMessageSize: Int){ + var shallowIter = internalIterator(true) + while(shallowIter.hasNext){ + var messageAndOffset = shallowIter.next + if (messageAndOffset.message.payloadSize > maxMessageSize) + throw new MessageSizeTooLargeException + } + } + + + /** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. This is used in verifyMessageSize() function **/ + private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = { ErrorMapping.maybeThrowException(errorCode) new IteratorTemplate[MessageAndOffset] { var topIter = buffer.slice() @@ -108,38 +120,51 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, message.limit(size) topIter.position(topIter.position + size) val newMessage = new Message(message) - newMessage.compressionCodec match { - case NoCompressionCodec => - if(!newMessage.isValid) - throw new InvalidMessageException("Uncompressed essage is invalid") - debug("Message is uncompressed. Valid byte count = %d".format(currValidBytes)) - innerIter = null - currValidBytes += 4 + size - trace("currValidBytes = " + currValidBytes) - new MessageAndOffset(newMessage, currValidBytes) - case _ => - if(!newMessage.isValid) - throw new InvalidMessageException("Compressed message is invalid") - debug("Message is compressed. Valid byte count = %d".format(currValidBytes)) - innerIter = CompressionUtils.decompress(newMessage).deepIterator - if (!innerIter.hasNext) { - currValidBytes += 4 + lastMessageSize + + if(isShallow){ + currValidBytes += 4 + size + trace("shallow iterator currValidBytes = " + currValidBytes) + new MessageAndOffset(newMessage, currValidBytes) + } + else{ + newMessage.compressionCodec match { + case NoCompressionCodec => + if(!newMessage.isValid) + throw new InvalidMessageException("Uncompressed essage is invalid") + debug("Message is uncompressed. Valid byte count = %d".format(currValidBytes)) innerIter = null - } - makeNext() + currValidBytes += 4 + size + trace("currValidBytes = " + currValidBytes) + new MessageAndOffset(newMessage, currValidBytes) + case _ => + if(!newMessage.isValid) + throw new InvalidMessageException("Compressed message is invalid") + debug("Message is compressed. Valid byte count = %d".format(currValidBytes)) + innerIter = CompressionUtils.decompress(newMessage).internalIterator() + if (!innerIter.hasNext) { + currValidBytes += 4 + lastMessageSize + innerIter = null + } + makeNext() + } } } override def makeNext(): MessageAndOffset = { - val isInnerDone = innerDone() - debug("makeNext() in deepIterator: innerDone = " + isInnerDone) - isInnerDone match { - case true => makeNextOuter - case false => { - val messageAndOffset = innerIter.next - if (!innerIter.hasNext) - currValidBytes += 4 + lastMessageSize - new MessageAndOffset(messageAndOffset.message, currValidBytes) + if(isShallow){ + makeNextOuter + } + else{ + val isInnerDone = innerDone() + debug("makeNext() in internalIterator: innerDone = " + isInnerDone) + isInnerDone match { + case true => makeNextOuter + case false => { + val messageAndOffset = innerIter.next + if (!innerIter.hasNext) + currValidBytes += 4 + lastMessageSize + new MessageAndOffset(messageAndOffset.message, currValidBytes) + } } } } diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 45f2b86f3bfac..f43685a0703ac 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -24,7 +24,6 @@ import kafka.network._ import kafka.utils._ import kafka.api._ import scala.math._ -import kafka.common.MessageSizeTooLargeException import java.nio.ByteBuffer import java.util.Random @@ -120,7 +119,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { * Send a message */ def send(topic: String, partition: Int, messages: ByteBufferMessageSet) { - verifyMessageSize(messages) + messages.verifyMessageSize(config.maxMessageSize) val setSize = messages.sizeInBytes.asInstanceOf[Int] trace("Got message set with " + setSize + " bytes to send") send(new BoundedByteBufferSend(new ProducerRequest(topic, partition, messages))) @@ -130,7 +129,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { def multiSend(produces: Array[ProducerRequest]) { for (request <- produces) - verifyMessageSize(request.messages) + request.messages.verifyMessageSize(config.maxMessageSize) val setSize = produces.foldLeft(0L)(_ + _.messages.sizeInBytes) trace("Got multi message sets with " + setSize + " bytes to send") send(new BoundedByteBufferSend(new MultiProducerRequest(produces))) @@ -143,11 +142,6 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { } } - private def verifyMessageSize(messages: ByteBufferMessageSet) { - for (messageAndOffset <- messages) - if (messageAndOffset.message.payloadSize > config.maxMessageSize) - throw new MessageSizeTooLargeException - } /** * Disconnect from current channel, closing connection. diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index ac811944c702f..8d65bb67710f7 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -5,7 +5,7 @@ * 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 @@ -27,7 +27,7 @@ import org.junit.{After, Before, Test} import kafka.common.MessageSizeTooLargeException import java.util.Properties import kafka.api.ProducerRequest -import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} +import kafka.message.{NoCompressionCodec, DefaultCompressionCodec, Message, ByteBufferMessageSet} class SyncProducerTest extends JUnitSuite { private var messageBytes = new Array[Byte](2); @@ -86,7 +86,7 @@ class SyncProducerTest extends JUnitSuite { } @Test - def testMessageSizeTooLarge() { + def testSingleMessageSizeTooLarge() { val props = new Properties() props.put("host", "localhost") props.put("port", server.socketServer.port.toString) @@ -104,4 +104,31 @@ class SyncProducerTest extends JUnitSuite { } Assert.assertTrue(failed) } -} + + @Test + def testCompressedMessageSizeTooLarge() { + val props = new Properties() + props.put("host", "localhost") + props.put("port", server.socketServer.port.toString) + props.put("buffer.size", "102400") + props.put("connect.timeout.ms", "300") + props.put("reconnect.interval", "500") + props.put("max.message.size", "100") + val producer = new SyncProducer(new SyncProducerConfig(props)) + val messages = new Array[Message](10) + import Array.fill + var a = 0 + for( a <- 0 to 9){ + val bytes = fill(20){a.asInstanceOf[Byte]} + messages(a) = new Message(bytes) + } + var failed = false + /** After compression, the compressed message has size 118 **/ + try { + producer.send("test", 0, new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = messages: _*)) + }catch { + case e: MessageSizeTooLargeException => failed = true + } + Assert.assertTrue(failed) + } +} \ No newline at end of file From 7e91c167430afcffd5b700de58bbc54b2149be00 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Sat, 25 Feb 2012 23:08:02 +0000 Subject: [PATCH 023/151] add jmx beans in broker to track # of failed requests; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-283 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1293720 13f79535-47bb-0310-9956-ffa450edef68 --- .../kafka/server/KafkaRequestHandlers.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala b/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala index d67d01414bd08..26b1aaa74d614 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandlers.scala @@ -75,6 +75,8 @@ private[kafka] class KafkaRequestHandlers(val logManager: LogManager) extends Lo catch { case e => error("Error processing " + requestHandlerName + " on " + request.topic + ":" + partition, e) + BrokerTopicStat.getBrokerTopicStat(request.topic).recordFailedProduceRequest + BrokerTopicStat.getBrokerAllTopicStat.recordFailedProduceRequest throw e } } @@ -113,6 +115,8 @@ private[kafka] class KafkaRequestHandlers(val logManager: LogManager) extends Lo catch { case e => error("error when processing request " + fetchRequest, e) + BrokerTopicStat.getBrokerTopicStat(fetchRequest.topic).recordFailedFetchRequest + BrokerTopicStat.getBrokerAllTopicStat.recordFailedFetchRequest response=new MessageSetSend(MessageSet.Empty, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } response @@ -132,6 +136,8 @@ trait BrokerTopicStatMBean { def getMessagesIn: Long def getBytesIn: Long def getBytesOut: Long + def getFailedProduceRequest: Long + def getFailedFetchRequest: Long } @threadsafe @@ -139,6 +145,8 @@ class BrokerTopicStat extends BrokerTopicStatMBean { private val numCumulatedMessagesIn = new AtomicLong(0) private val numCumulatedBytesIn = new AtomicLong(0) private val numCumulatedBytesOut = new AtomicLong(0) + private val numCumulatedFailedProduceRequests = new AtomicLong(0) + private val numCumulatedFailedFetchRequests = new AtomicLong(0) def getMessagesIn: Long = numCumulatedMessagesIn.get @@ -151,6 +159,14 @@ class BrokerTopicStat extends BrokerTopicStatMBean { def getBytesOut: Long = numCumulatedBytesOut.get def recordBytesOut(nBytes: Long) = numCumulatedBytesOut.getAndAdd(nBytes) + + def recordFailedProduceRequest = numCumulatedFailedProduceRequests.getAndIncrement + + def getFailedProduceRequest = numCumulatedFailedProduceRequests.get() + + def recordFailedFetchRequest = numCumulatedFailedFetchRequests.getAndIncrement + + def getFailedFetchRequest = numCumulatedFailedFetchRequests.get() } object BrokerTopicStat extends Logging { From caf310c2283731751926ccf8bd902df01dcb04f4 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 27 Feb 2012 19:46:36 +0000 Subject: [PATCH 024/151] consumer sometimes don't release partition ownership properly in ZK during rebalance; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-286 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1294302 13f79535-47bb-0310-9956-ffa450edef68 --- .../consumer/ZookeeperConsumerConnector.scala | 38 +++++++++++-------- 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index b459da96f62f4..28e027b28a487 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -413,16 +413,19 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - private def releasePartitionOwnership()= { + private def deletePartitionOwnershipFromZK(topic: String, partition: String) { + val topicDirs = new ZKGroupTopicDirs(group, topic) + val znode = topicDirs.consumerOwnerDir + "/" + partition + deletePath(zkClient, znode) + debug("Consumer " + consumerIdString + " releasing " + znode) + } + + private def releasePartitionOwnership(localTopicRegistry: Pool[String, Pool[Partition, PartitionTopicInfo]])= { info("Releasing partition ownership") - for ((topic, infos) <- topicRegistry) { - val topicDirs = new ZKGroupTopicDirs(group, topic) - for(partition <- infos.keys) { - val znode = topicDirs.consumerOwnerDir + "/" + partition - deletePath(zkClient, znode) - debug("Consumer " + consumerIdString + " releasing " + znode) - } - topicRegistry.remove(topic) + for ((topic, infos) <- localTopicRegistry) { + for(partition <- infos.keys) + deletePartitionOwnershipFromZK(topic, partition.toString) + localTopicRegistry.remove(topic) } } @@ -446,8 +449,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * the value of a child. Just let this go since another rebalance will be triggered. **/ info("exception during rebalance ", e) - /* Explicitly make sure another rebalancing attempt will get triggered. */ - done = false } info("end rebalancing consumer " + consumerIdString + " try #" + i) if (done) { @@ -459,8 +460,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } // stop all fetchers and clear all the queues to avoid data duplication closeFetchersForQueues(cluster, kafkaMessageStreams, queues.map(q => q._2)) - // release all partitions, reset state and retry - releasePartitionOwnership() Thread.sleep(config.rebalanceBackoffMs) } } @@ -481,7 +480,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, */ closeFetchers(cluster, kafkaMessageStreams, myTopicThreadIdsMap) - releasePartitionOwnership() + releasePartitionOwnership(topicRegistry) var partitionOwnershipDecision = new collection.mutable.HashMap[(String, String), String]() var currentTopicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] @@ -534,8 +533,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, topicRegistry = currentTopicRegistry updateFetcher(cluster, kafkaMessageStreams) true - }else + }else { false + } } private def closeFetchersForQueues(cluster: Cluster, @@ -585,6 +585,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def reflectPartitionOwnershipDecision(partitionOwnershipDecision: Map[(String, String), String]): Boolean = { + var successfullyOwnedPartitions : List[(String, String)] = Nil val partitionOwnershipSuccessful = partitionOwnershipDecision.map { partitionOwner => val topic = partitionOwner._1._1 val partition = partitionOwner._1._2 @@ -594,6 +595,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, try { createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) + successfullyOwnedPartitions ::= (topic, partition) true } catch { @@ -606,7 +608,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } val hasPartitionOwnershipFailed = partitionOwnershipSuccessful.foldLeft(0)((sum, decision) => sum + (if(decision) 0 else 1)) /* even if one of the partition ownership attempt has failed, return false */ - if(hasPartitionOwnershipFailed > 0) false + if(hasPartitionOwnershipFailed > 0) { + // remove all paths that we have owned in ZK + successfullyOwnedPartitions.foreach(topicAndPartition => deletePartitionOwnershipFromZK(topicAndPartition._1, topicAndPartition._2)) + false + } else true } From d8286aad107dbec34faae9e4e29c65bde20eb6f8 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 28 Feb 2012 01:50:38 +0000 Subject: [PATCH 025/151] Increase maximum value of log.retention.size; patched by Elben Shira; reviewed by Jun Rao; KAFKA-285 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1294441 13f79535-47bb-0310-9956-ffa450edef68 --- .../main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/main/scala/kafka/utils/Utils.scala | 42 +++++++++++++++++++ .../scala/unit/kafka/log/LogManagerTest.scala | 2 +- 3 files changed, 44 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 8577f423aaf71..af238d6b0be48 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -65,7 +65,7 @@ class KafkaConfig(props: Properties) extends ZKConfig(props) { val logRetentionHours = Utils.getIntInRange(props, "log.retention.hours", 24 * 7, (1, Int.MaxValue)) /* the maximum size of the log before deleting it */ - val logRetentionSize = Utils.getInt(props, "log.retention.size", -1) + val logRetentionSize = Utils.getLong(props, "log.retention.size", -1) /* the number of hours to keep a log file before deleting it for some specific topic*/ val logRetentionHoursMap = Utils.getTopicRentionHours(Utils.getString(props, "topic.log.retention.hours", "")) diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 96d04b65ab5fa..7b8b5ae370195 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -193,6 +193,48 @@ object Utils extends Logging { v } + /** + * Read a required long property value or throw an exception if no such property is found + */ + def getLong(props: Properties, name: String): Long = { + if(props.containsKey(name)) + return getLong(props, name, -1) + else + throw new IllegalArgumentException("Missing required property '" + name + "'") + } + + /** + * Read an long from the properties instance + * @param props The properties to read from + * @param name The property name + * @param default The default value to use if the property is not found + * @return the long value + */ + def getLong(props: Properties, name: String, default: Long): Long = + getLongInRange(props, name, default, (Long.MinValue, Long.MaxValue)) + + /** + * Read an long from the properties instance. Throw an exception + * if the value is not in the given range (inclusive) + * @param props The properties to read from + * @param name The property name + * @param default The default value to use if the property is not found + * @param range The range in which the value must fall (inclusive) + * @throws IllegalArgumentException If the value is not in the given range + * @return the long value + */ + def getLongInRange(props: Properties, name: String, default: Long, range: (Long, Long)): Long = { + val v = + if(props.containsKey(name)) + props.getProperty(name).toLong + else + default + if(v < range._1 || v > range._2) + throw new IllegalArgumentException(name + " has value " + v + " which is not in the range " + range + ".") + else + v + } + /** * Read a boolean value from the properties instance * @param props The properties to read from diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 8e84f3df0fedf..b6197d97a6dfa 100644 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -107,7 +107,7 @@ class LogManagerTest extends JUnitSuite { config = new KafkaConfig(props) { override val logFileSize = (10 * (setSize - 1)).asInstanceOf[Int] // each segment will be 10 messages override val enableZookeeper = false - override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Int] // keep exactly 6 segments + 1 roll over + override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Long] // keep exactly 6 segments + 1 roll over override val logRetentionHours = retentionHours } logManager = new LogManager(config, null, time, -1, retentionMs, false) From 6c34cf2dcc5961703001ab484a1ca6e662eee362 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 29 Feb 2012 03:22:14 +0000 Subject: [PATCH 026/151] use propertyExists to test if both broker.list and zk.connect are present; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-290 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1294959 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/producer/Producer.scala | 4 ---- core/src/main/scala/kafka/producer/ProducerConfig.scala | 7 +++++-- .../scala/unit/kafka/javaapi/producer/ProducerTest.scala | 4 ++++ .../scala/unit/kafka/producer/AsyncProducerTest.scala | 8 ++++++++ .../src/test/scala/unit/kafka/producer/ProducerTest.scala | 4 ++++ 5 files changed, 21 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala index 8af6faf66c358..7e0a9f56bfca4 100644 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -33,10 +33,6 @@ class Producer[K,V](config: ProducerConfig, /* use the other constructor*/ extends Logging { private val hasShutdown = new AtomicBoolean(false) - if(!Utils.propertyExists(config.zkConnect) && !Utils.propertyExists(config.brokerList)) - throw new InvalidConfigException("At least one of zk.connect or broker.list must be specified") - if (Utils.propertyExists(config.zkConnect) && Utils.propertyExists(config.brokerList)) - warn("Both zk.connect and broker.list provided (zk.connect takes precedence).") private val random = new java.util.Random // check if zookeeper based auto partition discovery is enabled private val zkEnabled = Utils.propertyExists(config.zkConnect) diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index 3cdbdb4b546f0..fa989c88d89d3 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -29,13 +29,16 @@ class ProducerConfig(val props: Properties) extends ZKConfig(props) * to pass in static broker and per-broker partition information. Format- * * brokerid1:host1:port1, brokerid2:host2:port2*/ val brokerList = Utils.getString(props, "broker.list", null) - if(brokerList != null && Utils.getString(props, "partitioner.class", null) != null) + if(Utils.propertyExists(brokerList) && Utils.getString(props, "partitioner.class", null) != null) throw new InvalidConfigException("partitioner.class cannot be used when broker.list is set") /** If both broker.list and zk.connect options are specified, throw an exception */ - if(brokerList != null && zkConnect != null) + if(Utils.propertyExists(brokerList) && Utils.propertyExists(zkConnect)) throw new InvalidConfigException("only one of broker.list and zk.connect can be specified") + if(!Utils.propertyExists(zkConnect) && !Utils.propertyExists(brokerList)) + throw new InvalidConfigException("At least one of zk.connect or broker.list must be specified") + /** the partitioner class for partitioning events amongst sub-topics */ val partitionerClass = Utils.getString(props, "partitioner.class", "kafka.producer.DefaultPartitioner") diff --git a/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala index 296bb072f935a..e749597238445 100644 --- a/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/producer/ProducerTest.scala @@ -232,6 +232,7 @@ class ProducerTest extends JUnitSuite { val props = new Properties() props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, syncProducers, new ConcurrentHashMap[Int, AsyncProducer[String]]()) producerPool.send(producerPool.getProducerPoolData("test-topic", new Partition(brokerId1, 0), Array("test1"))) @@ -264,6 +265,7 @@ class ProducerTest extends JUnitSuite { props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("producer.type", "async") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, new ConcurrentHashMap[Int, kafka.producer.SyncProducer](), asyncProducers) producerPool.send(producerPool.getProducerPoolData(topic, new Partition(brokerId1, 0), Array("test1"))) @@ -289,6 +291,7 @@ class ProducerTest extends JUnitSuite { val props = new Properties() props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, syncProducers, new ConcurrentHashMap[Int, AsyncProducer[String]]()) try { @@ -320,6 +323,7 @@ class ProducerTest extends JUnitSuite { props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("producer.type", "async") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, new ConcurrentHashMap[Int, kafka.producer.SyncProducer](), asyncProducers) try { diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index b87dc3d87acda..5268e12d0c760 100644 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -27,6 +27,7 @@ import org.scalatest.junit.JUnitSuite import kafka.producer.async._ import kafka.serializer.Encoder import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} +import kafka.utils.TestZKUtils class AsyncProducerTest extends JUnitSuite { @@ -54,6 +55,7 @@ class AsyncProducerTest extends JUnitSuite { props.put("port", "9092") props.put("queue.size", "10") props.put("serializer.class", "kafka.producer.StringSerializer") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new AsyncProducerConfig(props) val producer = new AsyncProducer[String](config, basicProducer, new StringSerializer) @@ -92,6 +94,7 @@ class AsyncProducerTest extends JUnitSuite { props.put("port", "9092") props.put("queue.size", "10") props.put("serializer.class", "kafka.producer.StringSerializer") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new AsyncProducerConfig(props) val producer = new AsyncProducer[String](config, basicProducer, new StringSerializer) @@ -130,6 +133,7 @@ class AsyncProducerTest extends JUnitSuite { props.put("queue.size", "10") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("batch.size", "5") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new AsyncProducerConfig(props) @@ -168,6 +172,7 @@ class AsyncProducerTest extends JUnitSuite { props.put("queue.size", "10") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("queue.time", "200") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new AsyncProducerConfig(props) @@ -200,6 +205,7 @@ class AsyncProducerTest extends JUnitSuite { asyncProducerProps.put("queue.size", "10") asyncProducerProps.put("serializer.class", "kafka.producer.StringSerializer") asyncProducerProps.put("queue.time", "100") + asyncProducerProps.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new AsyncProducerConfig(asyncProducerProps) val producer = new AsyncProducer[String](config, basicProducer, new StringSerializer) @@ -226,6 +232,7 @@ class AsyncProducerTest extends JUnitSuite { props.put("queue.size", "50") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("batch.size", "10") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new AsyncProducerConfig(props) @@ -266,6 +273,7 @@ class AsyncProducerTest extends JUnitSuite { props.put("queue.size", "50") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("batch.size", "20") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val config = new AsyncProducerConfig(props) diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 7a6273cc52705..c1e47b91b590e 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -238,6 +238,7 @@ class ProducerTest extends JUnitSuite { val props = new Properties() props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, syncProducers, new ConcurrentHashMap[Int, AsyncProducer[String]]()) producerPool.send(producerPool.getProducerPoolData("test-topic", new Partition(brokerId1, 0), Array("test1"))) @@ -270,6 +271,7 @@ class ProducerTest extends JUnitSuite { props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("producer.type", "async") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, new ConcurrentHashMap[Int, SyncProducer](), asyncProducers) producerPool.send(producerPool.getProducerPoolData(topic, new Partition(brokerId1, 0), Array("test1"))) @@ -295,6 +297,7 @@ class ProducerTest extends JUnitSuite { val props = new Properties() props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, syncProducers, new ConcurrentHashMap[Int, AsyncProducer[String]]()) try { @@ -326,6 +329,7 @@ class ProducerTest extends JUnitSuite { props.put("partitioner.class", "kafka.producer.NegativePartitioner") props.put("serializer.class", "kafka.producer.StringSerializer") props.put("producer.type", "async") + props.put("zk.connect", TestZKUtils.zookeeperConnect) val producerPool = new ProducerPool[String](new ProducerConfig(props), new StringSerializer, new ConcurrentHashMap[Int, SyncProducer](), asyncProducers) try { From ccc464c19a80f9818145c70079e9966fed22d75c Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Thu, 1 Mar 2012 02:47:03 +0000 Subject: [PATCH 027/151] Niek Sanders - KAFKA-284 fixed compilation issue for cpp client git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1295388 13f79535-47bb-0310-9956-ffa450edef68 --- clients/cpp/src/encoder.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/cpp/src/encoder.hpp b/clients/cpp/src/encoder.hpp index 1fec0c43f0725..a4c542c73f756 100644 --- a/clients/cpp/src/encoder.hpp +++ b/clients/cpp/src/encoder.hpp @@ -16,7 +16,7 @@ */ /* * encoder.hpp - * + */ #ifndef KAFKA_ENCODER_HPP_ #define KAFKA_ENCODER_HPP_ From 0443e2dd737cada1fed3964dd1cb2b687944f9c6 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 6 Mar 2012 02:21:54 +0000 Subject: [PATCH 028/151] broker deletes all file segments when cleaning up an empty log segment; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-292 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1297324 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/log/Log.scala | 19 +++++++++++++++---- .../test/scala/unit/kafka/log/LogTest.scala | 5 ++++- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 4b18c6ba07a9e..fe00d2fba0ee3 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -249,10 +249,18 @@ private[log] class Log(val dir: File, val maxSize: Long, val flushInterval: Int, val deletable = view.takeWhile(predicate) for(seg <- deletable) seg.deleted = true - val numToDelete = deletable.size + var numToDelete = deletable.size // if we are deleting everything, create a new empty segment - if(numToDelete == view.size) - roll() + if(numToDelete == view.size) { + if (view(numToDelete - 1).size > 0) + roll() + else { + // If the last segment to be deleted is empty and we roll the log, the new segment will have the same + // file name. So simply reuse the last segment and reset the modified time. + view(numToDelete - 1).file.setLastModified(SystemTime.milliseconds) + numToDelete -=1 + } + } segments.trunc(numToDelete) } } @@ -290,9 +298,12 @@ private[log] class Log(val dir: File, val maxSize: Long, val flushInterval: Int, */ def roll() { lock synchronized { - val last = segments.view.last val newOffset = nextAppendOffset val newFile = new File(dir, Log.nameFromOffset(newOffset)) + if (newFile.exists) { + warn("newly rolled logsegment " + newFile.getName + " already exists; deleting it first") + newFile.delete() + } debug("Rolling log '" + name + "' to " + newFile.getName()) segments.append(new LogSegment(newFile, new FileMessageSet(newFile, true), newOffset)) } diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 32f6f8f05b8d9..90407c00c6540 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -182,7 +182,10 @@ class LogTest extends JUnitSuite { assertEquals(curOffset, log.nextAppendOffset) // time goes by; the log file (which is empty) is deleted again - log.markDeletedWhile(_ => true) + val deletedSegments = log.markDeletedWhile(_ => true) + + // we shouldn't delete the last empty log segment. + assertTrue(deletedSegments.size == 0) // we now have a new log assertEquals(curOffset, log.nextAppendOffset) From 624a361e01431bf87ccda883fb9b1817d67cdbf7 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 7 Mar 2012 17:51:21 +0000 Subject: [PATCH 029/151] trivial fix to include violated payload size in MessageSizeTooLargeException; patched by Jun Rao git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1298030 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/common/ErrorMapping.scala | 1 - core/src/main/scala/kafka/message/ByteBufferMessageSet.scala | 5 +++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index 31614585eb018..ccadd31e70ec1 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -58,5 +58,4 @@ class InvalidTopicException(message: String) extends RuntimeException(message) { } class MessageSizeTooLargeException(message: String) extends RuntimeException(message) { - def this() = this(null) } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 6f4355f49add0..c646e45deaf45 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -82,8 +82,9 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, var shallowIter = internalIterator(true) while(shallowIter.hasNext){ var messageAndOffset = shallowIter.next - if (messageAndOffset.message.payloadSize > maxMessageSize) - throw new MessageSizeTooLargeException + val payloadSize = messageAndOffset.message.payloadSize + if ( payloadSize > maxMessageSize) + throw new MessageSizeTooLargeException("payload size of " + payloadSize + " larger than " + maxMessageSize) } } From 74af0deef17bfed19237367cdbee7027666add08 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 8 Mar 2012 15:18:30 +0000 Subject: [PATCH 030/151] LogManager test fails on linux; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-220 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1298426 13f79535-47bb-0310-9956-ffa450edef68 --- .../test/scala/unit/kafka/log/LogManagerTest.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index b6197d97a6dfa..3eae4b9dc03a8 100644 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -39,6 +39,7 @@ class LogManagerTest extends JUnitSuite { config = new KafkaConfig(props) { override val logFileSize = 1024 override val enableZookeeper = false + override val flushInterval = 100 } logManager = new LogManager(config, null, time, -1, maxLogAge, false) logManager.startup @@ -78,10 +79,13 @@ class LogManagerTest extends JUnitSuite { offset += set.sizeInBytes } log.flush - // Why this sleep is required ? File system takes some time to update the last modified time for a file. - // TODO: What is unknown is why 1 second or couple 100 milliseconds didn't work ? - Thread.sleep(2000) + assertTrue("There should be more than one segment now.", log.numberOfSegments > 1) + + // update the last modified time of all log segments + val logSegments = log.segments.view + logSegments.foreach(s => s.file.setLastModified(time.currentMs)) + time.currentMs += maxLogAge + 3000 logManager.cleanupLogs() assertEquals("Now there should only be only one segment.", 1, log.numberOfSegments) @@ -109,6 +113,7 @@ class LogManagerTest extends JUnitSuite { override val enableZookeeper = false override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Long] // keep exactly 6 segments + 1 roll over override val logRetentionHours = retentionHours + override val flushInterval = 100 } logManager = new LogManager(config, null, time, -1, retentionMs, false) logManager.startup @@ -177,6 +182,7 @@ class LogManagerTest extends JUnitSuite { override val logFileSize = 256 override val enableZookeeper = false override val topicPartitionsMap = Utils.getTopicPartitions("testPartition:2") + override val flushInterval = 100 } logManager = new LogManager(config, null, time, -1, maxLogAge, false) From 66d8073156403ce1a8f91b56f27beed93196ae04 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 14 Mar 2012 22:58:57 +0000 Subject: [PATCH 031/151] Update Go Client to new version of Go; patched by AaronR; KAFKA-296 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1300777 13f79535-47bb-0310-9956-ffa450edef68 --- clients/go/src/consumer.go | 21 +++++++++++---------- clients/go/src/kafka.go | 20 ++++++++++---------- clients/go/src/message.go | 4 ++-- clients/go/src/payload_codec.go | 4 +--- clients/go/src/publisher.go | 8 ++------ clients/go/src/request.go | 2 +- clients/go/src/timing.go | 6 +++--- clients/go/tools/consumer/consumer.go | 23 +++++++++++++++-------- clients/go/tools/offsets/offsets.go | 6 ++---- clients/go/tools/publisher/publisher.go | 4 ++-- 10 files changed, 49 insertions(+), 49 deletions(-) diff --git a/clients/go/src/consumer.go b/clients/go/src/consumer.go index 57a44527cff4d..4420093fd0b0e 100644 --- a/clients/go/src/consumer.go +++ b/clients/go/src/consumer.go @@ -23,11 +23,12 @@ package kafka import ( + "encoding/binary" + "errors" + "io" "log" - "os" "net" "time" - "encoding/binary" ) type BrokerConsumer struct { @@ -66,11 +67,11 @@ func NewBrokerOffsetConsumer(hostname string, topic string, partition int) *Brok func (consumer *BrokerConsumer) AddCodecs(payloadCodecs []PayloadCodec) { // merge to the default map, so one 'could' override the default codecs.. for k, v := range codecsMap(payloadCodecs) { - consumer.codecs[k] = v, true + consumer.codecs[k] = v } } -func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTimeoutMs int64, quit chan bool) (int, os.Error) { +func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTimeoutMs int64, quit chan bool) (int, error) { conn, err := consumer.broker.connect() if err != nil { return -1, err @@ -86,14 +87,14 @@ func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTime }) if err != nil { - if err != os.EOF { + if err != io.EOF { log.Println("Fatal Error: ", err) panic(err) } quit <- true // force quit break } - time.Sleep(pollTimeoutMs * 1000000) + time.Sleep(time.Millisecond * time.Duration(pollTimeoutMs)) } done <- true }() @@ -107,7 +108,7 @@ func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTime type MessageHandlerFunc func(msg *Message) -func (consumer *BrokerConsumer) Consume(handlerFunc MessageHandlerFunc) (int, os.Error) { +func (consumer *BrokerConsumer) Consume(handlerFunc MessageHandlerFunc) (int, error) { conn, err := consumer.broker.connect() if err != nil { return -1, err @@ -123,7 +124,7 @@ func (consumer *BrokerConsumer) Consume(handlerFunc MessageHandlerFunc) (int, os return num, err } -func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc MessageHandlerFunc) (int, os.Error) { +func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc MessageHandlerFunc) (int, error) { _, err := conn.Write(consumer.broker.EncodeConsumeRequest(consumer.offset, consumer.maxSize)) if err != nil { return -1, err @@ -142,7 +143,7 @@ func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc M for currentOffset <= uint64(length-4) { totalLength, msgs := Decode(payload[currentOffset:], consumer.codecs) if msgs == nil { - return num, os.NewError("Error Decoding Message") + return num, errors.New("Error Decoding Message") } msgOffset := consumer.offset + currentOffset for _, msg := range msgs { @@ -164,7 +165,7 @@ func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc M // Get a list of valid offsets (up to maxNumOffsets) before the given time, where // time is in milliseconds (-1, from the latest offset available, -2 from the smallest offset available) // The result is a list of offsets, in descending order. -func (consumer *BrokerConsumer) GetOffsets(time int64, maxNumOffsets uint32) ([]uint64, os.Error) { +func (consumer *BrokerConsumer) GetOffsets(time int64, maxNumOffsets uint32) ([]uint64, error) { offsets := make([]uint64, 0) conn, err := consumer.broker.connect() diff --git a/clients/go/src/kafka.go b/clients/go/src/kafka.go index a87431d85b873..96a19292a9d71 100644 --- a/clients/go/src/kafka.go +++ b/clients/go/src/kafka.go @@ -23,13 +23,13 @@ package kafka import ( - "log" - "net" - "os" - "fmt" + "bufio" "encoding/binary" + "errors" + "fmt" "io" - "bufio" + "log" + "net" ) const ( @@ -48,7 +48,7 @@ func newBroker(hostname string, topic string, partition int) *Broker { hostname: hostname} } -func (b *Broker) connect() (conn *net.TCPConn, error os.Error) { +func (b *Broker) connect() (conn *net.TCPConn, error error) { raddr, err := net.ResolveTCPAddr(NETWORK, b.hostname) if err != nil { log.Println("Fatal Error: ", err) @@ -63,7 +63,7 @@ func (b *Broker) connect() (conn *net.TCPConn, error os.Error) { } // returns length of response & payload & err -func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, os.Error) { +func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, error) { reader := bufio.NewReader(conn) length := make([]byte, 4) lenRead, err := io.ReadFull(reader, length) @@ -71,7 +71,7 @@ func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, os.Error) { return 0, []byte{}, err } if lenRead != 4 || lenRead < 0 { - return 0, []byte{}, os.NewError("invalid length of the packet length field") + return 0, []byte{}, errors.New("invalid length of the packet length field") } expectedLength := binary.BigEndian.Uint32(length) @@ -82,13 +82,13 @@ func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, os.Error) { } if uint32(lenRead) != expectedLength { - return 0, []byte{}, os.NewError(fmt.Sprintf("Fatal Error: Unexpected Length: %d expected: %d", lenRead, expectedLength)) + return 0, []byte{}, errors.New(fmt.Sprintf("Fatal Error: Unexpected Length: %d expected: %d", lenRead, expectedLength)) } errorCode := binary.BigEndian.Uint16(messages[0:2]) if errorCode != 0 { log.Println("errorCode: ", errorCode) - return 0, []byte{}, os.NewError( + return 0, []byte{}, errors.New( fmt.Sprintf("Broker Response Error: %d", errorCode)) } return expectedLength, messages[2:], nil diff --git a/clients/go/src/message.go b/clients/go/src/message.go index aa310486770e7..b214b085fcbbf 100644 --- a/clients/go/src/message.go +++ b/clients/go/src/message.go @@ -23,9 +23,9 @@ package kafka import ( - "hash/crc32" - "encoding/binary" "bytes" + "encoding/binary" + "hash/crc32" "log" ) diff --git a/clients/go/src/payload_codec.go b/clients/go/src/payload_codec.go index 7d6f8b5ec4672..6db6cc72abf45 100644 --- a/clients/go/src/payload_codec.go +++ b/clients/go/src/payload_codec.go @@ -57,7 +57,7 @@ var DefaultCodecsMap = codecsMap(DefaultCodecs) func codecsMap(payloadCodecs []PayloadCodec) map[byte]PayloadCodec { payloadCodecsMap := make(map[byte]PayloadCodec, len(payloadCodecs)) for _, c := range payloadCodecs { - payloadCodecsMap[c.Id()] = c, true + payloadCodecsMap[c.Id()] = c } return payloadCodecsMap } @@ -65,7 +65,6 @@ func codecsMap(payloadCodecs []PayloadCodec) map[byte]PayloadCodec { // No compression codec, noop type NoCompressionPayloadCodec struct { - } func (codec *NoCompressionPayloadCodec) Id() byte { @@ -83,7 +82,6 @@ func (codec *NoCompressionPayloadCodec) Decode(data []byte) []byte { // Gzip Codec type GzipPayloadCodec struct { - } func (codec *GzipPayloadCodec) Id() byte { diff --git a/clients/go/src/publisher.go b/clients/go/src/publisher.go index 5ca309388cf49..0766d1a9a19bb 100644 --- a/clients/go/src/publisher.go +++ b/clients/go/src/publisher.go @@ -22,10 +22,6 @@ package kafka -import ( - "os" -) - type BrokerPublisher struct { broker *Broker } @@ -34,11 +30,11 @@ func NewBrokerPublisher(hostname string, topic string, partition int) *BrokerPub return &BrokerPublisher{broker: newBroker(hostname, topic, partition)} } -func (b *BrokerPublisher) Publish(message *Message) (int, os.Error) { +func (b *BrokerPublisher) Publish(message *Message) (int, error) { return b.BatchPublish(message) } -func (b *BrokerPublisher) BatchPublish(messages ...*Message) (int, os.Error) { +func (b *BrokerPublisher) BatchPublish(messages ...*Message) (int, error) { conn, err := b.broker.connect() if err != nil { return -1, err diff --git a/clients/go/src/request.go b/clients/go/src/request.go index d15db906c1b69..8fbe4d06a5102 100644 --- a/clients/go/src/request.go +++ b/clients/go/src/request.go @@ -23,8 +23,8 @@ package kafka import ( - "encoding/binary" "bytes" + "encoding/binary" ) type RequestType uint16 diff --git a/clients/go/src/timing.go b/clients/go/src/timing.go index 56d01665da939..243b0eaf0f21d 100644 --- a/clients/go/src/timing.go +++ b/clients/go/src/timing.go @@ -34,16 +34,16 @@ type Timing struct { } func StartTiming(label string) *Timing { - return &Timing{label: label, start: time.Nanoseconds(), stop: 0} + return &Timing{label: label, start: time.Now().UnixNano()} } func (t *Timing) Stop() { - t.stop = time.Nanoseconds() + t.stop = time.Now().UnixNano() } func (t *Timing) Print() { if t.stop == 0 { t.Stop() } - log.Printf("%s took: %f ms\n", t.label, float64((time.Nanoseconds()-t.start))/1000000) + log.Printf("%s took: %f ms\n", t.label, float64(t.stop-t.start)/1000000) } diff --git a/clients/go/tools/consumer/consumer.go b/clients/go/tools/consumer/consumer.go index 50f0ebcc23dc4..316321f43f870 100644 --- a/clients/go/tools/consumer/consumer.go +++ b/clients/go/tools/consumer/consumer.go @@ -23,12 +23,12 @@ package main import ( - "kafka" "flag" "fmt" "os" - "strconv" "os/signal" + "strconv" + kafka "svn.apache.org/repos/asf/incubator/kafka.svn/trunk/clients/go/src" "syscall" ) @@ -46,7 +46,7 @@ func init() { flag.StringVar(&topic, "topic", "test", "topic to publish to") flag.IntVar(&partition, "partition", 0, "partition to publish to") flag.Uint64Var(&offset, "offset", 0, "offset to start consuming from") - flag.UintVar(&maxSize, "maxsize", 1048576, "offset to start consuming from") + flag.UintVar(&maxSize, "maxsize", 1048576, "max size in bytes of message set to request") flag.StringVar(&writePayloadsTo, "writeto", "", "write payloads to this file") flag.BoolVar(&consumerForever, "consumeforever", false, "loop forever consuming") flag.BoolVar(&printmessage, "printmessage", true, "print the message details to stdout") @@ -61,7 +61,7 @@ func main() { var payloadFile *os.File = nil if len(writePayloadsTo) > 0 { - var err os.Error + var err error payloadFile, err = os.Create(writePayloadsTo) if err != nil { fmt.Println("Error opening file: ", err) @@ -74,7 +74,7 @@ func main() { msg.Print() } if payloadFile != nil { - payloadFile.Write([]byte("Message at: " + strconv.Uitoa64(msg.Offset()) + "\n")) + payloadFile.Write([]byte("Message at: " + strconv.FormatUint(msg.Offset(), 10) + "\n")) payloadFile.Write(msg.Payload()) payloadFile.Write([]byte("\n-------------------------------\n")) } @@ -83,10 +83,17 @@ func main() { if consumerForever { quit := make(chan bool, 1) go func() { + sigIn := make(chan os.Signal) + signal.Notify(sigIn) for { - sig := <-signal.Incoming - if sig.(os.UnixSignal) == syscall.SIGINT { - quit <- true + + select { + case sig := <-sigIn: + if sig.(os.Signal) == syscall.SIGINT { + quit <- true + } else { + fmt.Println(sig) + } } } }() diff --git a/clients/go/tools/offsets/offsets.go b/clients/go/tools/offsets/offsets.go index 81e60d5c51db7..a748d4044d5cc 100644 --- a/clients/go/tools/offsets/offsets.go +++ b/clients/go/tools/offsets/offsets.go @@ -20,13 +20,12 @@ * of their respective owners. */ - package main import ( - "kafka" "flag" "fmt" + kafka "svn.apache.org/repos/asf/incubator/kafka.svn/trunk/clients/go/src" ) var hostname string @@ -43,7 +42,6 @@ func init() { flag.Int64Var(&time, "time", -1, "timestamp of the offsets before that: time(ms)/-1(latest)/-2(earliest)") } - func main() { flag.Parse() fmt.Println("Offsets :") @@ -56,7 +54,7 @@ func main() { fmt.Println("Error: ", err) } fmt.Printf("Offsets found: %d\n", len(offsets)) - for i := 0 ; i < len(offsets); i++ { + for i := 0; i < len(offsets); i++ { fmt.Printf("Offset[%d] = %d\n", i, offsets[i]) } } diff --git a/clients/go/tools/publisher/publisher.go b/clients/go/tools/publisher/publisher.go index 0a316bf0be025..f98c9b2d03d80 100644 --- a/clients/go/tools/publisher/publisher.go +++ b/clients/go/tools/publisher/publisher.go @@ -23,9 +23,9 @@ package main import ( - "kafka" "flag" "fmt" + kafka "svn.apache.org/repos/asf/incubator/kafka.svn/trunk/clients/go/src" "os" ) @@ -63,7 +63,7 @@ func main() { fmt.Println("Error: ", err) return } - payload := make([]byte, stat.Size) + payload := make([]byte, stat.Size()) file.Read(payload) timing := kafka.StartTiming("Sending") From 5e0e0b17fb5a9549fc9860c91384121def8ae695 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 15 Mar 2012 00:39:42 +0000 Subject: [PATCH 032/151] reverting previous commit for KAFKA-296 because patch didn't apply cleanly git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1300801 13f79535-47bb-0310-9956-ffa450edef68 --- clients/go/src/consumer.go | 21 ++++++++++----------- clients/go/src/kafka.go | 20 ++++++++++---------- clients/go/src/message.go | 4 ++-- clients/go/src/payload_codec.go | 4 +++- clients/go/src/publisher.go | 8 ++++++-- clients/go/src/request.go | 2 +- clients/go/src/timing.go | 6 +++--- clients/go/tools/consumer/consumer.go | 23 ++++++++--------------- clients/go/tools/offsets/offsets.go | 6 ++++-- clients/go/tools/publisher/publisher.go | 4 ++-- 10 files changed, 49 insertions(+), 49 deletions(-) diff --git a/clients/go/src/consumer.go b/clients/go/src/consumer.go index 4420093fd0b0e..57a44527cff4d 100644 --- a/clients/go/src/consumer.go +++ b/clients/go/src/consumer.go @@ -23,12 +23,11 @@ package kafka import ( - "encoding/binary" - "errors" - "io" "log" + "os" "net" "time" + "encoding/binary" ) type BrokerConsumer struct { @@ -67,11 +66,11 @@ func NewBrokerOffsetConsumer(hostname string, topic string, partition int) *Brok func (consumer *BrokerConsumer) AddCodecs(payloadCodecs []PayloadCodec) { // merge to the default map, so one 'could' override the default codecs.. for k, v := range codecsMap(payloadCodecs) { - consumer.codecs[k] = v + consumer.codecs[k] = v, true } } -func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTimeoutMs int64, quit chan bool) (int, error) { +func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTimeoutMs int64, quit chan bool) (int, os.Error) { conn, err := consumer.broker.connect() if err != nil { return -1, err @@ -87,14 +86,14 @@ func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTime }) if err != nil { - if err != io.EOF { + if err != os.EOF { log.Println("Fatal Error: ", err) panic(err) } quit <- true // force quit break } - time.Sleep(time.Millisecond * time.Duration(pollTimeoutMs)) + time.Sleep(pollTimeoutMs * 1000000) } done <- true }() @@ -108,7 +107,7 @@ func (consumer *BrokerConsumer) ConsumeOnChannel(msgChan chan *Message, pollTime type MessageHandlerFunc func(msg *Message) -func (consumer *BrokerConsumer) Consume(handlerFunc MessageHandlerFunc) (int, error) { +func (consumer *BrokerConsumer) Consume(handlerFunc MessageHandlerFunc) (int, os.Error) { conn, err := consumer.broker.connect() if err != nil { return -1, err @@ -124,7 +123,7 @@ func (consumer *BrokerConsumer) Consume(handlerFunc MessageHandlerFunc) (int, er return num, err } -func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc MessageHandlerFunc) (int, error) { +func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc MessageHandlerFunc) (int, os.Error) { _, err := conn.Write(consumer.broker.EncodeConsumeRequest(consumer.offset, consumer.maxSize)) if err != nil { return -1, err @@ -143,7 +142,7 @@ func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc M for currentOffset <= uint64(length-4) { totalLength, msgs := Decode(payload[currentOffset:], consumer.codecs) if msgs == nil { - return num, errors.New("Error Decoding Message") + return num, os.NewError("Error Decoding Message") } msgOffset := consumer.offset + currentOffset for _, msg := range msgs { @@ -165,7 +164,7 @@ func (consumer *BrokerConsumer) consumeWithConn(conn *net.TCPConn, handlerFunc M // Get a list of valid offsets (up to maxNumOffsets) before the given time, where // time is in milliseconds (-1, from the latest offset available, -2 from the smallest offset available) // The result is a list of offsets, in descending order. -func (consumer *BrokerConsumer) GetOffsets(time int64, maxNumOffsets uint32) ([]uint64, error) { +func (consumer *BrokerConsumer) GetOffsets(time int64, maxNumOffsets uint32) ([]uint64, os.Error) { offsets := make([]uint64, 0) conn, err := consumer.broker.connect() diff --git a/clients/go/src/kafka.go b/clients/go/src/kafka.go index 96a19292a9d71..a87431d85b873 100644 --- a/clients/go/src/kafka.go +++ b/clients/go/src/kafka.go @@ -23,13 +23,13 @@ package kafka import ( - "bufio" - "encoding/binary" - "errors" - "fmt" - "io" "log" "net" + "os" + "fmt" + "encoding/binary" + "io" + "bufio" ) const ( @@ -48,7 +48,7 @@ func newBroker(hostname string, topic string, partition int) *Broker { hostname: hostname} } -func (b *Broker) connect() (conn *net.TCPConn, error error) { +func (b *Broker) connect() (conn *net.TCPConn, error os.Error) { raddr, err := net.ResolveTCPAddr(NETWORK, b.hostname) if err != nil { log.Println("Fatal Error: ", err) @@ -63,7 +63,7 @@ func (b *Broker) connect() (conn *net.TCPConn, error error) { } // returns length of response & payload & err -func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, error) { +func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, os.Error) { reader := bufio.NewReader(conn) length := make([]byte, 4) lenRead, err := io.ReadFull(reader, length) @@ -71,7 +71,7 @@ func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, error) { return 0, []byte{}, err } if lenRead != 4 || lenRead < 0 { - return 0, []byte{}, errors.New("invalid length of the packet length field") + return 0, []byte{}, os.NewError("invalid length of the packet length field") } expectedLength := binary.BigEndian.Uint32(length) @@ -82,13 +82,13 @@ func (b *Broker) readResponse(conn *net.TCPConn) (uint32, []byte, error) { } if uint32(lenRead) != expectedLength { - return 0, []byte{}, errors.New(fmt.Sprintf("Fatal Error: Unexpected Length: %d expected: %d", lenRead, expectedLength)) + return 0, []byte{}, os.NewError(fmt.Sprintf("Fatal Error: Unexpected Length: %d expected: %d", lenRead, expectedLength)) } errorCode := binary.BigEndian.Uint16(messages[0:2]) if errorCode != 0 { log.Println("errorCode: ", errorCode) - return 0, []byte{}, errors.New( + return 0, []byte{}, os.NewError( fmt.Sprintf("Broker Response Error: %d", errorCode)) } return expectedLength, messages[2:], nil diff --git a/clients/go/src/message.go b/clients/go/src/message.go index b214b085fcbbf..aa310486770e7 100644 --- a/clients/go/src/message.go +++ b/clients/go/src/message.go @@ -23,9 +23,9 @@ package kafka import ( - "bytes" - "encoding/binary" "hash/crc32" + "encoding/binary" + "bytes" "log" ) diff --git a/clients/go/src/payload_codec.go b/clients/go/src/payload_codec.go index 6db6cc72abf45..7d6f8b5ec4672 100644 --- a/clients/go/src/payload_codec.go +++ b/clients/go/src/payload_codec.go @@ -57,7 +57,7 @@ var DefaultCodecsMap = codecsMap(DefaultCodecs) func codecsMap(payloadCodecs []PayloadCodec) map[byte]PayloadCodec { payloadCodecsMap := make(map[byte]PayloadCodec, len(payloadCodecs)) for _, c := range payloadCodecs { - payloadCodecsMap[c.Id()] = c + payloadCodecsMap[c.Id()] = c, true } return payloadCodecsMap } @@ -65,6 +65,7 @@ func codecsMap(payloadCodecs []PayloadCodec) map[byte]PayloadCodec { // No compression codec, noop type NoCompressionPayloadCodec struct { + } func (codec *NoCompressionPayloadCodec) Id() byte { @@ -82,6 +83,7 @@ func (codec *NoCompressionPayloadCodec) Decode(data []byte) []byte { // Gzip Codec type GzipPayloadCodec struct { + } func (codec *GzipPayloadCodec) Id() byte { diff --git a/clients/go/src/publisher.go b/clients/go/src/publisher.go index 0766d1a9a19bb..5ca309388cf49 100644 --- a/clients/go/src/publisher.go +++ b/clients/go/src/publisher.go @@ -22,6 +22,10 @@ package kafka +import ( + "os" +) + type BrokerPublisher struct { broker *Broker } @@ -30,11 +34,11 @@ func NewBrokerPublisher(hostname string, topic string, partition int) *BrokerPub return &BrokerPublisher{broker: newBroker(hostname, topic, partition)} } -func (b *BrokerPublisher) Publish(message *Message) (int, error) { +func (b *BrokerPublisher) Publish(message *Message) (int, os.Error) { return b.BatchPublish(message) } -func (b *BrokerPublisher) BatchPublish(messages ...*Message) (int, error) { +func (b *BrokerPublisher) BatchPublish(messages ...*Message) (int, os.Error) { conn, err := b.broker.connect() if err != nil { return -1, err diff --git a/clients/go/src/request.go b/clients/go/src/request.go index 8fbe4d06a5102..d15db906c1b69 100644 --- a/clients/go/src/request.go +++ b/clients/go/src/request.go @@ -23,8 +23,8 @@ package kafka import ( - "bytes" "encoding/binary" + "bytes" ) type RequestType uint16 diff --git a/clients/go/src/timing.go b/clients/go/src/timing.go index 243b0eaf0f21d..56d01665da939 100644 --- a/clients/go/src/timing.go +++ b/clients/go/src/timing.go @@ -34,16 +34,16 @@ type Timing struct { } func StartTiming(label string) *Timing { - return &Timing{label: label, start: time.Now().UnixNano()} + return &Timing{label: label, start: time.Nanoseconds(), stop: 0} } func (t *Timing) Stop() { - t.stop = time.Now().UnixNano() + t.stop = time.Nanoseconds() } func (t *Timing) Print() { if t.stop == 0 { t.Stop() } - log.Printf("%s took: %f ms\n", t.label, float64(t.stop-t.start)/1000000) + log.Printf("%s took: %f ms\n", t.label, float64((time.Nanoseconds()-t.start))/1000000) } diff --git a/clients/go/tools/consumer/consumer.go b/clients/go/tools/consumer/consumer.go index 316321f43f870..50f0ebcc23dc4 100644 --- a/clients/go/tools/consumer/consumer.go +++ b/clients/go/tools/consumer/consumer.go @@ -23,12 +23,12 @@ package main import ( + "kafka" "flag" "fmt" "os" - "os/signal" "strconv" - kafka "svn.apache.org/repos/asf/incubator/kafka.svn/trunk/clients/go/src" + "os/signal" "syscall" ) @@ -46,7 +46,7 @@ func init() { flag.StringVar(&topic, "topic", "test", "topic to publish to") flag.IntVar(&partition, "partition", 0, "partition to publish to") flag.Uint64Var(&offset, "offset", 0, "offset to start consuming from") - flag.UintVar(&maxSize, "maxsize", 1048576, "max size in bytes of message set to request") + flag.UintVar(&maxSize, "maxsize", 1048576, "offset to start consuming from") flag.StringVar(&writePayloadsTo, "writeto", "", "write payloads to this file") flag.BoolVar(&consumerForever, "consumeforever", false, "loop forever consuming") flag.BoolVar(&printmessage, "printmessage", true, "print the message details to stdout") @@ -61,7 +61,7 @@ func main() { var payloadFile *os.File = nil if len(writePayloadsTo) > 0 { - var err error + var err os.Error payloadFile, err = os.Create(writePayloadsTo) if err != nil { fmt.Println("Error opening file: ", err) @@ -74,7 +74,7 @@ func main() { msg.Print() } if payloadFile != nil { - payloadFile.Write([]byte("Message at: " + strconv.FormatUint(msg.Offset(), 10) + "\n")) + payloadFile.Write([]byte("Message at: " + strconv.Uitoa64(msg.Offset()) + "\n")) payloadFile.Write(msg.Payload()) payloadFile.Write([]byte("\n-------------------------------\n")) } @@ -83,17 +83,10 @@ func main() { if consumerForever { quit := make(chan bool, 1) go func() { - sigIn := make(chan os.Signal) - signal.Notify(sigIn) for { - - select { - case sig := <-sigIn: - if sig.(os.Signal) == syscall.SIGINT { - quit <- true - } else { - fmt.Println(sig) - } + sig := <-signal.Incoming + if sig.(os.UnixSignal) == syscall.SIGINT { + quit <- true } } }() diff --git a/clients/go/tools/offsets/offsets.go b/clients/go/tools/offsets/offsets.go index a748d4044d5cc..81e60d5c51db7 100644 --- a/clients/go/tools/offsets/offsets.go +++ b/clients/go/tools/offsets/offsets.go @@ -20,12 +20,13 @@ * of their respective owners. */ + package main import ( + "kafka" "flag" "fmt" - kafka "svn.apache.org/repos/asf/incubator/kafka.svn/trunk/clients/go/src" ) var hostname string @@ -42,6 +43,7 @@ func init() { flag.Int64Var(&time, "time", -1, "timestamp of the offsets before that: time(ms)/-1(latest)/-2(earliest)") } + func main() { flag.Parse() fmt.Println("Offsets :") @@ -54,7 +56,7 @@ func main() { fmt.Println("Error: ", err) } fmt.Printf("Offsets found: %d\n", len(offsets)) - for i := 0; i < len(offsets); i++ { + for i := 0 ; i < len(offsets); i++ { fmt.Printf("Offset[%d] = %d\n", i, offsets[i]) } } diff --git a/clients/go/tools/publisher/publisher.go b/clients/go/tools/publisher/publisher.go index f98c9b2d03d80..0a316bf0be025 100644 --- a/clients/go/tools/publisher/publisher.go +++ b/clients/go/tools/publisher/publisher.go @@ -23,9 +23,9 @@ package main import ( + "kafka" "flag" "fmt" - kafka "svn.apache.org/repos/asf/incubator/kafka.svn/trunk/clients/go/src" "os" ) @@ -63,7 +63,7 @@ func main() { fmt.Println("Error: ", err) return } - payload := make([]byte, stat.Size()) + payload := make([]byte, stat.Size) file.Read(payload) timing := kafka.StartTiming("Sending") From 7b39d7eed1449d993cdec6dca6babea6032dbbc3 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 21 Mar 2012 00:25:09 +0000 Subject: [PATCH 033/151] kafka-console-producer does not take in customized values of --batch-size or --timeout; patched by Jun Rao; reviewed by Edward Smith; KAFKA-279 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1303232 13f79535-47bb-0310-9956-ffa450edef68 --- .../main/scala/kafka/producer/ConsoleProducer.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/producer/ConsoleProducer.scala b/core/src/main/scala/kafka/producer/ConsoleProducer.scala index 73f88eee0951e..533fe4600e71b 100644 --- a/core/src/main/scala/kafka/producer/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/producer/ConsoleProducer.scala @@ -36,7 +36,7 @@ object ConsoleProducer { .withRequiredArg .describedAs("connection_string") .ofType(classOf[String]) - val asyncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") + val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") val compressOpt = parser.accepts("compress", "If set, messages batches are sent compressed") val batchSizeOpt = parser.accepts("batch-size", "Number of messages to send in a single batch if they are not being sent synchronously.") .withRequiredArg @@ -78,7 +78,7 @@ object ConsoleProducer { val topic = options.valueOf(topicOpt) val zkConnect = options.valueOf(zkConnectOpt) - val async = options.has(asyncOpt) + val sync = options.has(syncOpt) val compress = options.has(compressOpt) val batchSize = options.valueOf(batchSizeOpt) val sendTimeout = options.valueOf(sendTimeoutOpt) @@ -89,10 +89,10 @@ object ConsoleProducer { val props = new Properties() props.put("zk.connect", zkConnect) props.put("compression.codec", DefaultCompressionCodec.codec.toString) - props.put("producer.type", if(async) "async" else "sync") + props.put("producer.type", if(sync) "sync" else "async") if(options.has(batchSizeOpt)) - props.put("batch.size", batchSize) - props.put("queue.enqueueTimeout.ms", sendTimeout.toString) + props.put("batch.size", batchSize.toString) + props.put("queue.time", sendTimeout.toString) props.put("serializer.class", encoderClass) val reader = Class.forName(readerClass).newInstance().asInstanceOf[MessageReader] From e180e68f94d1d50fb9563d273c03d255eb081998 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 21 Mar 2012 00:38:14 +0000 Subject: [PATCH 034/151] Dead code in the Log4j appender; patched by Jose Quinteiro; reviewed by Jun Rao; KAFKA-303 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1303239 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index 747bbbe498d37..527928460912a 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -94,7 +94,3 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging { override def requiresLayout: Boolean = false } - -class DefaultStringEncoder extends Encoder[LoggingEvent] { - override def toMessage(event: LoggingEvent):Message = new Message(event.getMessage.asInstanceOf[String].getBytes) -} From 2bcc91134316a546322bf59a422bec1934613607 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Thu, 22 Mar 2012 16:03:49 +0000 Subject: [PATCH 035/151] KAFKA-310 Incomplete message set validation checks in Log's append API can corrupt on disk log segment; patched by nehanarkhede; reviewed by junrao git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1303861 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/log/Log.scala | 16 +++++++++++++--- .../kafka/message/ByteBufferMessageSet.scala | 3 +-- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index fe00d2fba0ee3..10c384b0e5188 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -199,7 +199,7 @@ private[log] class Log(val dir: File, val maxSize: Long, val flushInterval: Int, * Append this message set to the active segment of the log, rolling over to a fresh segment if necessary. * Returns the offset at which the messages are written. */ - def append(messages: MessageSet): Unit = { + def append(messages: ByteBufferMessageSet): Unit = { // validate the messages var numberOfMessages = 0 for(messageAndOffset <- messages) { @@ -211,12 +211,22 @@ private[log] class Log(val dir: File, val maxSize: Long, val flushInterval: Int, BrokerTopicStat.getBrokerTopicStat(getTopicName).recordMessagesIn(numberOfMessages) BrokerTopicStat.getBrokerAllTopicStat.recordMessagesIn(numberOfMessages) logStats.recordAppendedMessages(numberOfMessages) - + + // truncate the message set's buffer upto validbytes, before appending it to the on-disk log + val validByteBuffer = messages.getBuffer.duplicate() + val messageSetValidBytes = messages.validBytes + if(messageSetValidBytes > Int.MaxValue || messageSetValidBytes < 0) + throw new InvalidMessageSizeException("Illegal length of message set " + messageSetValidBytes + + " Message set cannot be appended to log. Possible causes are corrupted produce requests") + + validByteBuffer.limit(messageSetValidBytes.asInstanceOf[Int]) + val validMessages = new ByteBufferMessageSet(validByteBuffer) + // they are valid, insert them in the log lock synchronized { try { val segment = segments.view.last - segment.messageSet.append(messages) + segment.messageSet.append(validMessages) maybeFlush(numberOfMessages) maybeRoll(segment) } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index c646e45deaf45..36f33b182e139 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -61,7 +61,7 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, private def shallowValidBytes: Long = { if(shallowValidByteCount < 0) { - val iter = this.internalIterator() + val iter = this.internalIterator(true) while(iter.hasNext) { val messageAndOffset = iter.next shallowValidByteCount = messageAndOffset.offset @@ -88,7 +88,6 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, } } - /** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. This is used in verifyMessageSize() function **/ private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = { ErrorMapping.maybeThrowException(errorCode) From ba8fd9f1ee9fc47a1e62ee6f529f7cb272b3c7f3 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Thu, 22 Mar 2012 16:59:11 +0000 Subject: [PATCH 036/151] KAFKA-309 Bug in FileMessageSet's append API can corrupt on disk log; patched by nehanarkhede; reviewed by junrao git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1303890 13f79535-47bb-0310-9956-ffa450edef68 --- .../main/scala/kafka/message/ByteBufferMessageSet.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 36f33b182e139..01774f5770077 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -72,8 +72,12 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, } /** Write the messages in this set to the given channel */ - def writeTo(channel: GatheringByteChannel, offset: Long, size: Long): Long = - channel.write(buffer.duplicate) + def writeTo(channel: GatheringByteChannel, offset: Long, size: Long): Long = { + buffer.mark() + val written = channel.write(buffer) + buffer.reset() + written + } override def iterator: Iterator[MessageAndOffset] = internalIterator() From f8346239d492b22416d9c5d4639ebd5eafdbae92 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 6 Apr 2012 16:59:44 +0000 Subject: [PATCH 037/151] CallbackHandler.afterDequeuingExistingData is not called during event queue timeout; patched by Jun Rao; reviewed by Neha Narkhede; KAFKA-326 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1310482 13f79535-47bb-0310-9956-ffa450edef68 --- .../producer/async/ProducerSendThread.scala | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala index e3dd841f0bf0d..0c4a4edbc16e9 100644 --- a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala +++ b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala @@ -73,18 +73,21 @@ private[async] class ProducerSendThread[T](val threadName: String, // check if the queue time is reached. This happens when the poll method above returns after a timeout and // returns a null object val expired = currentQueueItem == null - if(currentQueueItem != null) { + if(currentQueueItem != null) trace("Dequeued item for topic %s and partition %d" .format(currentQueueItem.getTopic, currentQueueItem.getPartition)) - // handle the dequeued current item - if(cbkHandler != null) - events = events ++ cbkHandler.afterDequeuingExistingData(currentQueueItem) - else - events += currentQueueItem - // check if the batch size is reached - full = events.size >= batchSize + // handle the dequeued current item + if(cbkHandler != null) + events = events ++ cbkHandler.afterDequeuingExistingData(currentQueueItem) + else { + if (currentQueueItem != null) + events += currentQueueItem } + + // check if the batch size is reached + full = events.size >= batchSize + if(full || expired) { if(expired) debug(elapsed + " ms elapsed. Queue time reached. Sending..") if(full) debug("Batch full. Sending..") From d9441f0066604534a8226ce1809eaf68e5a4ada2 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 6 Apr 2012 21:18:56 +0000 Subject: [PATCH 038/151] enable shallow iterator in ByteBufferMessageSet to allow mirroing data without decompression; patched by Jun Rao; reviewed by Joel Koshy; KAFKA-315 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1310595 13f79535-47bb-0310-9956-ffa450edef68 --- .../scala/kafka/consumer/ConsumerConfig.scala | 7 +++++++ .../kafka/consumer/ConsumerIterator.scala | 6 ++++-- .../kafka/consumer/KafkaMessageStream.scala | 5 +++-- .../consumer/ZookeeperConsumerConnector.scala | 2 +- .../kafka/message/ByteBufferMessageSet.scala | 12 +++++++----- .../message/ByteBufferMessageSetTest.scala | 18 ++++++++++++++++++ 6 files changed, 40 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index c2ac74653368b..9cfda55c409a2 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -105,5 +105,12 @@ class ConsumerConfig(props: Properties) extends ZKConfig(props) { val mirrorConsumerNumThreads = Utils.getInt( props, MirrorConsumerNumThreadsProp, MirrorConsumerNumThreads) + + /** Use shallow iterator over compressed messages directly. This feature should be used very carefully. + * Typically, it's only used for mirroring raw messages from one kafka cluster to another to save the + * overhead of decompression. + * */ + val enableShallowIterator = Utils.getBoolean(props, "shallowiterator.enable", false) + } diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index cb959190cffab..e8bc4f9571416 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -31,7 +31,8 @@ import java.util.concurrent.atomic.AtomicReference class ConsumerIterator[T](private val topic: String, private val channel: BlockingQueue[FetchedDataChunk], consumerTimeoutMs: Int, - private val decoder: Decoder[T]) + private val decoder: Decoder[T], + val enableShallowIterator: Boolean) extends IteratorTemplate[T] with Logging { private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null) @@ -74,7 +75,8 @@ class ConsumerIterator[T](private val topic: String, .format(currentTopicInfo.getConsumeOffset, currentDataChunk.fetchOffset, currentTopicInfo)) currentTopicInfo.resetConsumeOffset(currentDataChunk.fetchOffset) } - localCurrent = currentDataChunk.messages.iterator + localCurrent = if (enableShallowIterator) currentDataChunk.messages.shallowIterator + else currentDataChunk.messages.iterator current.set(localCurrent) } } diff --git a/core/src/main/scala/kafka/consumer/KafkaMessageStream.scala b/core/src/main/scala/kafka/consumer/KafkaMessageStream.scala index 0771c320e2955..288d85948d3d2 100644 --- a/core/src/main/scala/kafka/consumer/KafkaMessageStream.scala +++ b/core/src/main/scala/kafka/consumer/KafkaMessageStream.scala @@ -27,11 +27,12 @@ import kafka.serializer.Decoder class KafkaMessageStream[T](val topic: String, private val queue: BlockingQueue[FetchedDataChunk], consumerTimeoutMs: Int, - private val decoder: Decoder[T]) + private val decoder: Decoder[T], + val enableShallowIterator: Boolean) extends Iterable[T] with java.lang.Iterable[T]{ private val iter: ConsumerIterator[T] = - new ConsumerIterator[T](topic, queue, consumerTimeoutMs, decoder) + new ConsumerIterator[T](topic, queue, consumerTimeoutMs, decoder, enableShallowIterator) /** * Create an iterator over messages in the stream. diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 28e027b28a487..595efeafcba8d 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -178,7 +178,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, for (threadId <- threadIdSet) { val stream = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) queues.put((topic, threadId), stream) - streamList ::= new KafkaMessageStream[T](topic, stream, config.consumerTimeoutMs, decoder) + streamList ::= new KafkaMessageStream[T](topic, stream, config.consumerTimeoutMs, decoder, config.enableShallowIterator) } ret += (topic -> streamList) debug("adding topic " + topic + " and stream to map..") diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 01774f5770077..07c5b1f1bde49 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -78,9 +78,12 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, buffer.reset() written } - + + /** default iterator that iterates over decompressed messages */ override def iterator: Iterator[MessageAndOffset] = internalIterator() + /** iterator over compressed messages without decompressing */ + def shallowIterator: Iterator[MessageAndOffset] = internalIterator(true) def verifyMessageSize(maxMessageSize: Int){ var shallowIter = internalIterator(true) @@ -124,6 +127,9 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, message.limit(size) topIter.position(topIter.position + size) val newMessage = new Message(message) + if(!newMessage.isValid) + throw new InvalidMessageException("message is invalid, compression codec: " + newMessage.compressionCodec + + " size: " + size + " curr offset: " + currValidBytes + " init offset: " + initialOffset) if(isShallow){ currValidBytes += 4 + size @@ -133,16 +139,12 @@ class ByteBufferMessageSet(private val buffer: ByteBuffer, else{ newMessage.compressionCodec match { case NoCompressionCodec => - if(!newMessage.isValid) - throw new InvalidMessageException("Uncompressed essage is invalid") debug("Message is uncompressed. Valid byte count = %d".format(currValidBytes)) innerIter = null currValidBytes += 4 + size trace("currValidBytes = " + currValidBytes) new MessageAndOffset(newMessage, currValidBytes) case _ => - if(!newMessage.isValid) - throw new InvalidMessageException("Compressed message is invalid") debug("Message is compressed. Valid byte count = %d".format(currValidBytes)) innerIter = CompressionUtils.decompress(newMessage).internalIterator() if (!innerIter.hasNext) { diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 962a86dca47b9..c81c35675e217 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -94,6 +94,10 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) //make sure the last offset after iteration is correct assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.serialized.limit) + + //make sure shallow iterator is the same as deep iterator + TestUtils.checkEquals[Message](TestUtils.getMessageIterator(messageSet.shallowIterator), + TestUtils.getMessageIterator(messageSet.iterator)) } // test for compressed regular messages @@ -104,6 +108,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) //make sure the last offset after iteration is correct assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.serialized.limit) + + verifyShallowIterator(messageSet) } // test for mixed empty and non-empty messagesets uncompressed @@ -121,6 +127,10 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) //make sure the last offset after iteration is correct assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.serialized.limit) + + //make sure shallow iterator is the same as deep iterator + TestUtils.checkEquals[Message](TestUtils.getMessageIterator(mixedMessageSet.shallowIterator), + TestUtils.getMessageIterator(mixedMessageSet.iterator)) } // test for mixed empty and non-empty messagesets compressed @@ -138,7 +148,15 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) //make sure the last offset after iteration is correct assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.serialized.limit) + + verifyShallowIterator(mixedMessageSet) } } + def verifyShallowIterator(messageSet: ByteBufferMessageSet) { + //make sure the offsets returned by a shallow iterator is a subset of that of a deep iterator + val shallowOffsets = messageSet.shallowIterator.map(msgAndOff => msgAndOff.offset).toSet + val deepOffsets = messageSet.iterator.map(msgAndOff => msgAndOff.offset).toSet + assertTrue(shallowOffsets.subsetOf(deepOffsets)) + } } From c0e190cdefabc7c59b11bbe0de843baec78efe3f Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Sat, 7 Apr 2012 00:04:51 +0000 Subject: [PATCH 039/151] Separate out Kafka mirroring into a stand-alone app; patched by Joel Koshy; reviewed by Jun Rao and Neha Narkhede; KAFKA-249 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1310645 13f79535-47bb-0310-9956-ffa450edef68 --- .../main/java/kafka/etl/KafkaETLContext.java | 20 +- .../java/kafka/etl/KafkaETLInputFormat.java | 5 +- .../src/main/java/kafka/etl/KafkaETLJob.java | 2 +- .../src/main/java/kafka/etl/KafkaETLKey.java | 2 +- .../main/java/kafka/etl/KafkaETLUtils.java | 2 +- .../java/kafka/etl/impl/DataGenerator.java | 24 +- .../kafka/bridge/examples/TextPublisher.java | 5 +- .../bridge/hadoop/KafkaOutputFormat.java | 16 +- .../bridge/hadoop/KafkaRecordWriter.java | 9 +- .../kafka/bridge/pig/AvroKafkaStorage.java | 9 +- core/src/main/scala/kafka/Kafka.scala | 15 +- .../kafka/consumer/ConsoleConsumer.scala | 54 +-- .../scala/kafka/consumer/ConsumerConfig.scala | 18 - .../kafka/consumer/ConsumerConnector.scala | 22 +- .../kafka/consumer/ConsumerIterator.scala | 25 +- .../main/scala/kafka/consumer/Fetcher.scala | 12 +- ...aMessageStream.scala => KafkaStream.scala} | 23 +- .../scala/kafka/consumer/TopicCount.scala | 149 ++++++-- .../scala/kafka/consumer/TopicFilter.scala | 76 ++++ .../consumer/ZookeeperConsumerConnector.scala | 299 +++++++++++---- .../consumer/ZookeeperTopicEventWatcher.scala | 14 +- .../javaapi/consumer/ConsumerConnector.java | 65 ++-- .../consumer/ZookeeperConsumerConnector.scala | 20 +- .../kafka/javaapi/message/MessageSet.scala | 2 + .../producer/async/CallbackHandler.java | 2 +- .../javaapi/producer/async/EventHandler.java | 6 +- .../kafka/message/ByteBufferMessageSet.scala | 1 - .../kafka/message/MessageAndMetadata.scala | 21 ++ .../kafka/message/MessageAndOffset.scala | 9 +- .../kafka/producer/KafkaLog4jAppender.scala | 2 - .../main/scala/kafka/producer/Producer.scala | 2 +- .../kafka/server/KafkaServerStartable.scala | 170 +-------- .../scala/kafka/tools/ConsumerShell.scala | 6 +- .../main/scala/kafka/tools/MirrorMaker.scala | 162 ++++++++ .../scala/kafka/tools/ReplayLogProducer.scala | 8 +- core/src/main/scala/kafka/utils/Logging.scala | 40 +- core/src/main/scala/kafka/utils/Utils.scala | 13 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 12 +- .../other/kafka/TestZKConsumerOffsets.scala | 6 +- ...cCountTest.scala => TopicFilterTest.scala} | 44 +-- .../ZookeeperConsumerConnectorTest.scala | 6 +- .../unit/kafka/integration/FetcherTest.scala | 2 +- .../ZookeeperConsumerConnectorTest.scala | 7 +- .../scala/unit/kafka/utils/UtilsTest.scala | 3 +- .../main/java/kafka/examples/Consumer.java | 17 +- .../java/kafka/examples/ExampleUtils.java | 2 +- .../main/java/kafka/examples/Producer.java | 3 +- .../kafka/examples/SimpleConsumerDemo.java | 7 +- .../kafka/perf/ConsumerPerformance.scala | 11 +- .../main/scala/kafka/perf/PerfConfig.scala | 2 +- .../perf/SimpleConsumerPerformance.scala | 2 - system_test/embedded_consumer/README | 27 -- system_test/embedded_consumer/bin/run-test.sh | 328 ---------------- system_test/embedded_consumer/expected.out | 11 - system_test/mirror_maker/README | 22 ++ .../bin/expected.out | 0 system_test/mirror_maker/bin/run-test.sh | 357 ++++++++++++++++++ .../config/blacklisttest.consumer.properties | 3 +- .../config/mirror_producer.properties | 5 +- .../config/server_source_1_1.properties} | 4 +- .../config/server_source_1_2.properties} | 2 +- .../config/server_source_2_1.properties} | 8 +- .../config/server_source_2_2.properties | 76 ++++ .../config/server_target_1_1.properties} | 6 +- .../config/server_target_1_2.properties} | 6 +- .../whitelisttest_1.consumer.properties} | 3 +- .../whitelisttest_2.consumer.properties | 28 ++ .../config/zookeeper_source_1.properties} | 2 +- .../config/zookeeper_source_2.properties} | 4 + .../config/zookeeper_target.properties | 2 +- 70 files changed, 1411 insertions(+), 937 deletions(-) rename core/src/main/scala/kafka/consumer/{KafkaMessageStream.scala => KafkaStream.scala} (67%) create mode 100644 core/src/main/scala/kafka/consumer/TopicFilter.scala create mode 100644 core/src/main/scala/kafka/message/MessageAndMetadata.scala create mode 100644 core/src/main/scala/kafka/tools/MirrorMaker.scala rename core/src/test/scala/unit/kafka/consumer/{TopicCountTest.scala => TopicFilterTest.scala} (53%) delete mode 100644 system_test/embedded_consumer/README delete mode 100755 system_test/embedded_consumer/bin/run-test.sh delete mode 100644 system_test/embedded_consumer/expected.out create mode 100644 system_test/mirror_maker/README rename system_test/{embedded_consumer => mirror_maker}/bin/expected.out (100%) create mode 100644 system_test/mirror_maker/bin/run-test.sh rename system_test/{embedded_consumer => mirror_maker}/config/blacklisttest.consumer.properties (96%) rename system_test/{embedded_consumer => mirror_maker}/config/mirror_producer.properties (90%) rename system_test/{embedded_consumer/config/server_source1.properties => mirror_maker/config/server_source_1_1.properties} (98%) rename system_test/{embedded_consumer/config/server_source2.properties => mirror_maker/config/server_source_1_2.properties} (98%) rename system_test/{embedded_consumer/config/server_source3.properties => mirror_maker/config/server_source_2_1.properties} (96%) create mode 100644 system_test/mirror_maker/config/server_source_2_2.properties rename system_test/{embedded_consumer/config/server_target1.properties => mirror_maker/config/server_target_1_1.properties} (97%) rename system_test/{embedded_consumer/config/server_target2.properties => mirror_maker/config/server_target_1_2.properties} (97%) rename system_test/{embedded_consumer/config/whitelisttest.consumer.properties => mirror_maker/config/whitelisttest_1.consumer.properties} (97%) create mode 100644 system_test/mirror_maker/config/whitelisttest_2.consumer.properties rename system_test/{embedded_consumer/config/zookeeper_source.properties => mirror_maker/config/zookeeper_source_1.properties} (96%) rename system_test/{embedded_consumer/config/consumer.properties => mirror_maker/config/zookeeper_source_2.properties} (84%) rename system_test/{embedded_consumer => mirror_maker}/config/zookeeper_target.properties (98%) diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java index 1c05a77ea0ff8..1c18832d8dfa9 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java @@ -16,27 +16,25 @@ */ package kafka.etl; + import java.io.IOException; import java.net.URI; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.zip.CRC32; import kafka.api.FetchRequest; -import kafka.javaapi.MultiFetchResponse; import kafka.api.OffsetRequest; import kafka.common.ErrorMapping; +import kafka.javaapi.MultiFetchResponse; import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; -import kafka.message.Message; import kafka.message.MessageAndOffset; -import kafka.message.MessageSet; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.lib.MultipleOutputs; -import java.nio.ByteBuffer; @SuppressWarnings({ "deprecation"}) public class KafkaETLContext { @@ -139,7 +137,7 @@ public boolean getNext(KafkaETLKey key, BytesWritable value) throws IOException while ( !gotNext && _respIterator.hasNext()) { ByteBufferMessageSet msgSet = _respIterator.next(); if ( hasError(msgSet)) return false; - _messageIt = (Iterator) msgSet.iterator(); + _messageIt = msgSet.iterator(); gotNext = get(key, value); } } @@ -190,17 +188,17 @@ public void close() throws IOException { protected boolean get(KafkaETLKey key, BytesWritable value) throws IOException { if (_messageIt != null && _messageIt.hasNext()) { - MessageAndOffset msgAndOffset = _messageIt.next(); + MessageAndOffset messageAndOffset = _messageIt.next(); - ByteBuffer buf = msgAndOffset.message().payload(); + ByteBuffer buf = messageAndOffset.message().payload(); int origSize = buf.remaining(); byte[] bytes = new byte[origSize]; - buf.get(bytes, buf.position(), origSize); + buf.get(bytes, buf.position(), origSize); value.set(bytes, 0, origSize); - key.set(_index, _offset, msgAndOffset.message().checksum()); + key.set(_index, _offset, messageAndOffset.message().checksum()); - _offset = msgAndOffset.offset(); //increase offset + _offset = messageAndOffset.offset(); //increase offset _count ++; //increase count return true; diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLInputFormat.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLInputFormat.java index 69aafeddbec88..ddd6b724ab619 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLInputFormat.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLInputFormat.java @@ -16,6 +16,7 @@ */ package kafka.etl; + import java.io.IOException; import java.net.URI; import java.util.Map; @@ -23,13 +24,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.lib.MultipleOutputs; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.SequenceFileInputFormat; +import org.apache.hadoop.mapred.lib.MultipleOutputs; @SuppressWarnings("deprecation") diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLJob.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLJob.java index 5b8e77d003a48..1a4bcbae6ec3d 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLJob.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLJob.java @@ -16,13 +16,13 @@ */ package kafka.etl; + import java.net.URI; import org.apache.hadoop.filecache.DistributedCache; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.hadoop.mapred.lib.MultipleOutputs; diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLKey.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLKey.java index e448366527852..aafeceaab13cb 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLKey.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLKey.java @@ -16,11 +16,11 @@ */ package kafka.etl; + import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import org.apache.hadoop.io.WritableComparable; -import kafka.etl.KafkaETLKey; public class KafkaETLKey implements WritableComparable{ diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLUtils.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLUtils.java index 9691b948309da..02d79a1884c48 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLUtils.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLUtils.java @@ -17,6 +17,7 @@ package kafka.etl; + import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -33,7 +34,6 @@ import java.util.Enumeration; import java.util.List; import java.util.Properties; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java index ba9646b96420d..5166358481680 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/impl/DataGenerator.java @@ -17,32 +17,24 @@ package kafka.etl.impl; -import java.io.IOException; + import java.net.URI; -import java.net.URISyntaxException; import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; import java.util.List; -import java.util.Random; -import java.util.Map.Entry; import java.util.Properties; - -import kafka.message.NoCompressionCodec; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.mapred.JobConf; - +import java.util.Random; import kafka.etl.KafkaETLKey; import kafka.etl.KafkaETLRequest; -import kafka.etl.KafkaETLUtils; import kafka.etl.Props; import kafka.javaapi.message.ByteBufferMessageSet; -import kafka.message.Message; import kafka.javaapi.producer.SyncProducer; +import kafka.message.Message; import kafka.producer.SyncProducerConfig; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.mapred.JobConf; /** * Use this class to produce test events to Kafka server. Each event contains a diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/examples/TextPublisher.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/examples/TextPublisher.java index 815c13fb43693..5acbceeeff308 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/examples/TextPublisher.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/examples/TextPublisher.java @@ -16,8 +16,9 @@ */ package kafka.bridge.examples; -import kafka.bridge.hadoop.KafkaOutputFormat; +import java.io.IOException; +import kafka.bridge.hadoop.KafkaOutputFormat; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; @@ -27,8 +28,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; -import java.io.IOException; - public class TextPublisher { public static void main(String[] args) throws Exception diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java index 57332671ea4b4..4b9343f26eeab 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaOutputFormat.java @@ -16,24 +16,26 @@ */ package kafka.bridge.hadoop; -import java.util.Properties; +import java.io.IOException; +import java.net.URI; +import java.util.Properties; import kafka.javaapi.producer.Producer; import kafka.message.Message; import kafka.producer.ProducerConfig; - -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapreduce.*; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; import org.apache.log4j.Logger; -import java.io.IOException; -import java.net.URI; - public class KafkaOutputFormat extends OutputFormat { private Logger log = Logger.getLogger(KafkaOutputFormat.class); diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java index efb2b5e0231b4..af9c6500563b4 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/hadoop/KafkaRecordWriter.java @@ -16,19 +16,18 @@ */ package kafka.bridge.hadoop; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; import kafka.javaapi.producer.Producer; import kafka.javaapi.producer.ProducerData; import kafka.message.Message; - import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; - public class KafkaRecordWriter extends RecordWriter { protected Producer producer; diff --git a/contrib/hadoop-producer/src/main/java/kafka/bridge/pig/AvroKafkaStorage.java b/contrib/hadoop-producer/src/main/java/kafka/bridge/pig/AvroKafkaStorage.java index cac634b4b8625..faa1950b8d595 100644 --- a/contrib/hadoop-producer/src/main/java/kafka/bridge/pig/AvroKafkaStorage.java +++ b/contrib/hadoop-producer/src/main/java/kafka/bridge/pig/AvroKafkaStorage.java @@ -16,9 +16,12 @@ */ package kafka.bridge.pig; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; import kafka.bridge.hadoop.KafkaOutputFormat; import kafka.bridge.hadoop.KafkaRecordWriter; - import org.apache.avro.io.BinaryEncoder; import org.apache.avro.io.Encoder; import org.apache.hadoop.fs.Path; @@ -33,10 +36,6 @@ import org.apache.pig.piggybank.storage.avro.PigAvroDatumWriter; import org.apache.pig.piggybank.storage.avro.PigSchema2Avro; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStream; - public class AvroKafkaStorage extends StoreFunc { protected KafkaRecordWriter writer; diff --git a/core/src/main/scala/kafka/Kafka.scala b/core/src/main/scala/kafka/Kafka.scala index c5b8c0998a92d..3ac2e7bdad365 100644 --- a/core/src/main/scala/kafka/Kafka.scala +++ b/core/src/main/scala/kafka/Kafka.scala @@ -17,8 +17,6 @@ package kafka -import consumer.ConsumerConfig -import producer.ProducerConfig import server.{KafkaConfig, KafkaServerStartable, KafkaServer} import utils.{Utils, Logging} import org.apache.log4j.jmx.LoggerDynamicMBean @@ -30,8 +28,8 @@ object Kafka extends Logging { import org.apache.log4j.Logger Utils.registerMBean(new LoggerDynamicMBean(Logger.getRootLogger()), kafkaLog4jMBeanName) - if (!List(1, 3).contains(args.length)) { - println("USAGE: java [options] %s server.properties [consumer.properties producer.properties]".format(classOf[KafkaServer].getSimpleName())) + if (args.length != 1) { + println("USAGE: java [options] %s server.properties".format(classOf[KafkaServer].getSimpleName())) System.exit(1) } @@ -39,14 +37,7 @@ object Kafka extends Logging { val props = Utils.loadProps(args(0)) val serverConfig = new KafkaConfig(props) - val kafkaServerStartble = args.length match { - case 3 => - val consumerConfig = new ConsumerConfig(Utils.loadProps(args(1))) - val producerConfig = new ProducerConfig(Utils.loadProps(args(2))) - new KafkaServerStartable(serverConfig, consumerConfig, producerConfig) - case 1 => - new KafkaServerStartable(serverConfig) - } + val kafkaServerStartble = new KafkaServerStartable(serverConfig) // attach shutdown handler to catch control-c Runtime.getRuntime().addShutdownHook(new Thread() { diff --git a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala index bddbb2bc5643f..49a6f396f76ce 100644 --- a/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala @@ -36,11 +36,19 @@ object ConsoleConsumer extends Logging { def main(args: Array[String]) { val parser = new OptionParser - val topicIdOpt = parser.accepts("topic", "REQUIRED: The topic id to consume on.") + val topicIdOpt = parser.accepts("topic", "The topic id to consume on.") .withRequiredArg .describedAs("topic") .ofType(classOf[String]) - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + + val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to include for consumption.") + .withRequiredArg + .describedAs("whitelist") + .ofType(classOf[String]) + val blacklistOpt = parser.accepts("blacklist", "Blacklist of topics to exclude from consumption.") + .withRequiredArg + .describedAs("blacklist") + .ofType(classOf[String]) + val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + "Multiple URLS can be given to allow fail-over.") .withRequiredArg .describedAs("urls") @@ -90,8 +98,20 @@ object ConsoleConsumer extends Logging { "skip it instead of halt.") val options: OptionSet = tryParse(parser, args) - checkRequiredArgs(parser, options, topicIdOpt, zkConnectOpt) + Utils.checkRequiredArgs(parser, options, zkConnectOpt) + val topicOrFilterOpt = List(topicIdOpt, whitelistOpt, blacklistOpt).filter(options.has) + if (topicOrFilterOpt.size != 1) { + error("Exactly one of whitelist/blacklist/topic is required.") + parser.printHelpOn(System.err) + System.exit(1) + } + val topicArg = options.valueOf(topicOrFilterOpt.head) + val filterSpec = if (options.has(blacklistOpt)) + new Blacklist(topicArg) + else + new Whitelist(topicArg) + val props = new Properties() props.put("groupid", options.valueOf(groupIdOpt)) props.put("socket.buffersize", options.valueOf(socketBufferSizeOpt).toString) @@ -104,7 +124,6 @@ object ConsoleConsumer extends Logging { val config = new ConsumerConfig(props) val skipMessageOnError = if (options.has(skipMessageOnErrorOpt)) true else false - val topic = options.valueOf(topicIdOpt) val messageFormatterClass = Class.forName(options.valueOf(messageFormatterOpt)) val formatterArgs = tryParseFormatterArgs(options.valuesOf(messageFormatterArgOpt)) @@ -123,21 +142,20 @@ object ConsoleConsumer extends Logging { tryCleanupZookeeper(options.valueOf(zkConnectOpt), options.valueOf(groupIdOpt)) } }) - - var stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).get.get(0) - val iter = - if(maxMessages >= 0) - stream.slice(0, maxMessages) - else - stream + + val stream = connector.createMessageStreamsByFilter(filterSpec).get(0) + val iter = if(maxMessages >= 0) + stream.slice(0, maxMessages) + else + stream val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter] formatter.init(formatterArgs) try { - for(message <- iter) { + for(messageAndTopic <- iter) { try { - formatter.writeTo(message, System.out) + formatter.writeTo(messageAndTopic.message, System.out) } catch { case e => if (skipMessageOnError) @@ -173,16 +191,6 @@ object ConsoleConsumer extends Logging { } } - def checkRequiredArgs(parser: OptionParser, options: OptionSet, required: OptionSpec[_]*) { - for(arg <- required) { - if(!options.has(arg)) { - error("Missing required argument \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } - } - def tryParseFormatterArgs(args: Iterable[String]): Properties = { val splits = args.map(_ split "=").filterNot(_ == null).filterNot(_.length == 0) if(!splits.forall(_.length == 2)) { diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 9cfda55c409a2..c531cd1feea64 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -20,7 +20,6 @@ package kafka.consumer import java.util.Properties import kafka.utils.{ZKConfig, Utils} import kafka.api.OffsetRequest -import kafka.common.InvalidConfigException object ConsumerConfig { val SocketTimeout = 30 * 1000 val SocketBufferSize = 64*1024 @@ -90,27 +89,10 @@ class ConsumerConfig(props: Properties) extends ZKConfig(props) { /** throw a timeout exception to the consumer if no message is available for consumption after the specified interval */ val consumerTimeoutMs = Utils.getInt(props, "consumer.timeout.ms", ConsumerTimeoutMs) - /** Whitelist of topics for this mirror's embedded consumer to consume. At - * most one of whitelist/blacklist may be specified. */ - val mirrorTopicsWhitelist = Utils.getString( - props, MirrorTopicsWhitelistProp, MirrorTopicsWhitelist) - - /** Topics to skip mirroring. At most one of whitelist/blacklist may be - * specified */ - val mirrorTopicsBlackList = Utils.getString( - props, MirrorTopicsBlacklistProp, MirrorTopicsBlacklist) - - if (mirrorTopicsWhitelist.nonEmpty && mirrorTopicsBlackList.nonEmpty) - throw new InvalidConfigException("The embedded consumer's mirror topics configuration can only contain one of blacklist or whitelist") - - val mirrorConsumerNumThreads = Utils.getInt( - props, MirrorConsumerNumThreadsProp, MirrorConsumerNumThreads) - /** Use shallow iterator over compressed messages directly. This feature should be used very carefully. * Typically, it's only used for mirroring raw messages from one kafka cluster to another to save the * overhead of decompression. * */ val enableShallowIterator = Utils.getBoolean(props, "shallowiterator.enable", false) - } diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index 14671d60b5cd8..94cb2f1996b7b 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala @@ -29,12 +29,28 @@ trait ConsumerConnector { * Create a list of MessageStreams for each topic. * * @param topicCountMap a map of (topic, #streams) pair - * @return a map of (topic, list of KafkaMessageStream) pair. The number of items in the - * list is #streams. Each KafkaMessageStream supports an iterator of messages. + * @param decoder Decoder to decode each Message to type T + * @return a map of (topic, list of KafkaStream) pairs. + * The number of items in the list is #streams. Each stream supports + * an iterator over message/metadata pairs. */ def createMessageStreams[T](topicCountMap: Map[String,Int], decoder: Decoder[T] = new DefaultDecoder) - : Map[String,List[KafkaMessageStream[T]]] + : Map[String,List[KafkaStream[T]]] + + /** + * Create a list of message streams for all topics that match a given filter. + * + * @param topicFilter Either a Whitelist or Blacklist TopicFilter object. + * @param numStreams Number of streams to return + * @param decoder Decoder to decode each Message to type T + * @return a list of KafkaStream each of which provides an + * iterator over message/metadata pairs over allowed topics. + */ + def createMessageStreamsByFilter[T](topicFilter: TopicFilter, + numStreams: Int = 1, + decoder: Decoder[T] = new DefaultDecoder) + : Seq[KafkaStream[T]] /** * Commit the offsets of all broker partitions connected by this connector. diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index e8bc4f9571416..9c6828c25d7c5 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -19,37 +19,38 @@ package kafka.consumer import kafka.utils.{IteratorTemplate, Logging} import java.util.concurrent.{TimeUnit, BlockingQueue} -import kafka.message.MessageAndOffset import kafka.serializer.Decoder import java.util.concurrent.atomic.AtomicReference +import kafka.message.{MessageAndOffset, MessageAndMetadata} + /** * An iterator that blocks until a value can be read from the supplied queue. * The iterator takes a shutdownCommand object which can be added to the queue to trigger a shutdown * */ -class ConsumerIterator[T](private val topic: String, - private val channel: BlockingQueue[FetchedDataChunk], +class ConsumerIterator[T](private val channel: BlockingQueue[FetchedDataChunk], consumerTimeoutMs: Int, private val decoder: Decoder[T], val enableShallowIterator: Boolean) - extends IteratorTemplate[T] with Logging { + extends IteratorTemplate[MessageAndMetadata[T]] with Logging { private var current: AtomicReference[Iterator[MessageAndOffset]] = new AtomicReference(null) private var currentTopicInfo:PartitionTopicInfo = null private var consumedOffset: Long = -1L - override def next(): T = { - val decodedMessage = super.next() + override def next(): MessageAndMetadata[T] = { + val item = super.next() if(consumedOffset < 0) throw new IllegalStateException("Offset returned by the message set is invalid %d".format(consumedOffset)) currentTopicInfo.resetConsumeOffset(consumedOffset) - trace("Setting consumed offset to %d".format(consumedOffset)) + val topic = currentTopicInfo.topic + trace("Setting %s consumed offset to %d".format(topic, consumedOffset)) ConsumerTopicStat.getConsumerTopicStat(topic).recordMessagesPerTopic(1) - decodedMessage + item } - protected def makeNext(): T = { + protected def makeNext(): MessageAndMetadata[T] = { var currentDataChunk: FetchedDataChunk = null // if we don't have an iterator, get one var localCurrent = current.get() @@ -82,10 +83,11 @@ class ConsumerIterator[T](private val topic: String, } val item = localCurrent.next() consumedOffset = item.offset - decoder.toEvent(item.message) + + new MessageAndMetadata(decoder.toEvent(item.message), currentTopicInfo.topic) } - def clearCurrentChunk() = { + def clearCurrentChunk() { try { info("Clearing the current data chunk for this consumer iterator") current.set(null) @@ -94,3 +96,4 @@ class ConsumerIterator[T](private val topic: String, } class ConsumerTimeoutException() extends RuntimeException() + diff --git a/core/src/main/scala/kafka/consumer/Fetcher.scala b/core/src/main/scala/kafka/consumer/Fetcher.scala index d18faca85b03f..5e65df9c7637f 100644 --- a/core/src/main/scala/kafka/consumer/Fetcher.scala +++ b/core/src/main/scala/kafka/consumer/Fetcher.scala @@ -42,24 +42,24 @@ private [consumer] class Fetcher(val config: ConsumerConfig, val zkClient : ZkCl fetcherThreads = EMPTY_FETCHER_THREADS } - def clearFetcherQueues[T](topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster, + def clearFetcherQueues(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster, queuesTobeCleared: Iterable[BlockingQueue[FetchedDataChunk]], - kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) { + messageStreams: Map[String,List[KafkaStream[_]]]) { // Clear all but the currently iterated upon chunk in the consumer thread's queue queuesTobeCleared.foreach(_.clear) info("Cleared all relevant queues for this fetcher") // Also clear the currently iterated upon chunk in the consumer threads - if(kafkaMessageStreams != null) - kafkaMessageStreams.foreach(_._2.foreach(s => s.clear())) + if(messageStreams != null) + messageStreams.foreach(_._2.foreach(s => s.clear())) info("Cleared the data chunks in all the consumer message iterators") } - def startConnections[T](topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster, - kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) { + def startConnections(topicInfos: Iterable[PartitionTopicInfo], + cluster: Cluster) { if (topicInfos == null) return diff --git a/core/src/main/scala/kafka/consumer/KafkaMessageStream.scala b/core/src/main/scala/kafka/consumer/KafkaStream.scala similarity index 67% rename from core/src/main/scala/kafka/consumer/KafkaMessageStream.scala rename to core/src/main/scala/kafka/consumer/KafkaStream.scala index 288d85948d3d2..3ef097887092a 100644 --- a/core/src/main/scala/kafka/consumer/KafkaMessageStream.scala +++ b/core/src/main/scala/kafka/consumer/KafkaStream.scala @@ -5,7 +5,7 @@ * 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 @@ -17,23 +17,20 @@ package kafka.consumer + import java.util.concurrent.BlockingQueue import kafka.serializer.Decoder +import kafka.message.MessageAndMetadata -/** - * All calls to elements should produce the same thread-safe iterator? Should have a separate thread - * that feeds messages into a blocking queue for processing. - */ -class KafkaMessageStream[T](val topic: String, - private val queue: BlockingQueue[FetchedDataChunk], - consumerTimeoutMs: Int, - private val decoder: Decoder[T], - val enableShallowIterator: Boolean) - extends Iterable[T] with java.lang.Iterable[T]{ +class KafkaStream[T](private val queue: BlockingQueue[FetchedDataChunk], + consumerTimeoutMs: Int, + private val decoder: Decoder[T], + val enableShallowIterator: Boolean) + extends Iterable[MessageAndMetadata[T]] with java.lang.Iterable[MessageAndMetadata[T]] { private val iter: ConsumerIterator[T] = - new ConsumerIterator[T](topic, queue, consumerTimeoutMs, decoder, enableShallowIterator) - + new ConsumerIterator[T](queue, consumerTimeoutMs, decoder, enableShallowIterator) + /** * Create an iterator over messages in the stream. */ diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala index 51bf516ee0590..c60f5a3124f40 100644 --- a/core/src/main/scala/kafka/consumer/TopicCount.scala +++ b/core/src/main/scala/kafka/consumer/TopicCount.scala @@ -19,35 +19,18 @@ package kafka.consumer import scala.collection._ import scala.util.parsing.json.JSON -import kafka.utils.Logging +import org.I0Itec.zkclient.ZkClient +import java.util.regex.Pattern +import kafka.utils.{ZKGroupDirs, ZkUtils, Logging} -private[kafka] object TopicCount extends Logging { - val myConversionFunc = {input : String => input.toInt} - JSON.globalNumberParser = myConversionFunc - - def constructTopicCount(consumerIdSting: String, jsonString : String) : TopicCount = { - var topMap : Map[String,Int] = null - try { - JSON.parseFull(jsonString) match { - case Some(m) => topMap = m.asInstanceOf[Map[String,Int]] - case None => throw new RuntimeException("error constructing TopicCount : " + jsonString) - } - } - catch { - case e => - error("error parsing consumer json string " + jsonString, e) - throw e - } - - new TopicCount(consumerIdSting, topMap) - } -} - -private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) { +private[kafka] trait TopicCount { + def getConsumerThreadIdsPerTopic: Map[String, Set[String]] - def getConsumerThreadIdsPerTopic() - : Map[String, Set[String]] = { + def dbString: String + + protected def makeConsumerThreadIdsPerTopic(consumerIdString: String, + topicCountMap: Map[String, Int]) = { val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[String]]() for ((topic, nConsumers) <- topicCountMap) { val consumerSet = new mutable.HashSet[String] @@ -58,11 +41,96 @@ private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: } consumerThreadIdsPerTopicMap } +} + +private[kafka] object TopicCount extends Logging { + + /* + * Example of whitelist topic count stored in ZooKeeper: + * Topics with whitetopic as prefix, and four streams: *4*whitetopic.* + * + * Example of blacklist topic count stored in ZooKeeper: + * Topics with blacktopic as prefix, and four streams: !4!blacktopic.* + */ + + val WHITELIST_MARKER = "*" + val BLACKLIST_MARKER = "!" + private val WHITELIST_PATTERN = + Pattern.compile("""\*(\p{Digit}+)\*(.*)""") + private val BLACKLIST_PATTERN = + Pattern.compile("""!(\p{Digit}+)!(.*)""") + + val myConversionFunc = {input : String => input.toInt} + JSON.globalNumberParser = myConversionFunc + + def constructTopicCount(group: String, + consumerId: String, + zkClient: ZkClient) : TopicCount = { + val dirs = new ZKGroupDirs(group) + val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId) + val hasWhitelist = topicCountString.startsWith(WHITELIST_MARKER) + val hasBlacklist = topicCountString.startsWith(BLACKLIST_MARKER) + + if (hasWhitelist || hasBlacklist) + info("Constructing topic count for %s from %s using %s as pattern." + .format(consumerId, topicCountString, + if (hasWhitelist) WHITELIST_PATTERN else BLACKLIST_PATTERN)) + + if (hasWhitelist || hasBlacklist) { + val matcher = if (hasWhitelist) + WHITELIST_PATTERN.matcher(topicCountString) + else + BLACKLIST_PATTERN.matcher(topicCountString) + require(matcher.matches()) + val numStreams = matcher.group(1).toInt + val regex = matcher.group(2) + val filter = if (hasWhitelist) + new Whitelist(regex) + else + new Blacklist(regex) + + new WildcardTopicCount(zkClient, consumerId, filter, numStreams) + } + else { + var topMap : Map[String,Int] = null + try { + JSON.parseFull(topicCountString) match { + case Some(m) => topMap = m.asInstanceOf[Map[String,Int]] + case None => throw new RuntimeException("error constructing TopicCount : " + topicCountString) + } + } + catch { + case e => + error("error parsing consumer json string " + topicCountString, e) + throw e + } + + new StaticTopicCount(consumerId, topMap) + } + } + + def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) = + new StaticTopicCount(consumerIdString, topicCount) + + def constructTopicCount(consumerIdString: String, + filter: TopicFilter, + numStreams: Int, + zkClient: ZkClient) = + new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams) + +} + +private[kafka] class StaticTopicCount(val consumerIdString: String, + val topicCountMap: Map[String, Int]) + extends TopicCount { + + def getConsumerThreadIdsPerTopic = + makeConsumerThreadIdsPerTopic(consumerIdString, topicCountMap) override def equals(obj: Any): Boolean = { obj match { case null => false - case n: TopicCount => consumerIdString == n.consumerIdString && topicCountMap == n.topicCountMap + case n: StaticTopicCount => consumerIdString == n.consumerIdString && topicCountMap == n.topicCountMap case _ => false } } @@ -73,7 +141,7 @@ private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: * "topic2" : 4 * } */ - def toJsonString() : String = { + def dbString = { val builder = new StringBuilder builder.append("{ ") var i = 0 @@ -84,6 +152,29 @@ private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: i += 1 } builder.append(" }") - builder.toString + builder.toString() } } + +private[kafka] class WildcardTopicCount(zkClient: ZkClient, + consumerIdString: String, + topicFilter: TopicFilter, + numStreams: Int) extends TopicCount { + def getConsumerThreadIdsPerTopic = { + val wildcardTopics = ZkUtils.getChildrenParentMayNotExist( + zkClient, ZkUtils.BrokerTopicsPath).filter(topicFilter.isTopicAllowed(_)) + makeConsumerThreadIdsPerTopic(consumerIdString, + Map(wildcardTopics.map((_, numStreams)): _*)) + } + + def dbString = { + val marker = topicFilter match { + case wl: Whitelist => TopicCount.WHITELIST_MARKER + case bl: Blacklist => TopicCount.BLACKLIST_MARKER + } + + "%s%d%s%s".format(marker, numStreams, marker, topicFilter.regex) + } + +} + diff --git a/core/src/main/scala/kafka/consumer/TopicFilter.scala b/core/src/main/scala/kafka/consumer/TopicFilter.scala new file mode 100644 index 0000000000000..cf3853b223095 --- /dev/null +++ b/core/src/main/scala/kafka/consumer/TopicFilter.scala @@ -0,0 +1,76 @@ +/** + * 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 kafka.consumer + + +import kafka.utils.Logging +import java.util.regex.{PatternSyntaxException, Pattern} + + +sealed abstract class TopicFilter(rawRegex: String) extends Logging { + + val regex = rawRegex + .trim + .replace(',', '|') + .replace(" ", "") + .replaceAll("""^["']+""","") + .replaceAll("""["']+$""","") // property files may bring quotes + + try { + Pattern.compile(regex) + } + catch { + case e: PatternSyntaxException => + throw new RuntimeException(regex + " is an invalid regex.") + } + + override def toString = regex + + def requiresTopicEventWatcher: Boolean + + def isTopicAllowed(topic: String): Boolean +} + +case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) { + override def requiresTopicEventWatcher = !regex.matches("""[\p{Alnum}-|]+""") + + override def isTopicAllowed(topic: String) = { + val allowed = topic.matches(regex) + + debug("%s %s".format( + topic, if (allowed) "allowed" else "filtered")) + + allowed + } + + +} + +case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) { + override def requiresTopicEventWatcher = true + + override def isTopicAllowed(topic: String) = { + val allowed = !topic.matches(regex) + + debug("%s %s".format( + topic, if (allowed) "allowed" else "filtered")) + + allowed + } +} + diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 595efeafcba8d..f7782df231b3f 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -34,6 +34,7 @@ import kafka.common.{ConsumerRebalanceFailedException, InvalidConfigException} import java.lang.IllegalStateException import kafka.utils.ZkUtils._ + /** * This class handles the consumers interaction with zookeeper * @@ -86,16 +87,37 @@ trait ZookeeperConsumerConnectorMBean { private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val enableFetcher: Boolean) // for testing only - extends ConsumerConnector with ZookeeperConsumerConnectorMBean with Logging { - + extends ConsumerConnector with ZookeeperConsumerConnectorMBean + with Logging { private val isShuttingDown = new AtomicBoolean(false) private val rebalanceLock = new Object private var fetcher: Option[Fetcher] = None private var zkClient: ZkClient = null private var topicRegistry = new Pool[String, Pool[Partition, PartitionTopicInfo]] - // queues : (topic,consumerThreadId) -> queue - private val queues = new Pool[Tuple2[String,String], BlockingQueue[FetchedDataChunk]] + // topicThreadIdAndQueues : (topic,consumerThreadId) -> queue + private val topicThreadIdAndQueues = new Pool[Tuple2[String,String], BlockingQueue[FetchedDataChunk]] private val scheduler = new KafkaScheduler(1, "Kafka-consumer-autocommit-", false) + private val messageStreamCreated = new AtomicBoolean(false) + + private var sessionExpirationListener: ZKSessionExpireListener = null + private var loadBalancerListener: ZKRebalancerListener = null + + private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null + + val consumerIdString = { + var consumerUuid : String = null + config.consumerId match { + case Some(consumerId) // for testing only + => consumerUuid = consumerId + case None // generate unique consumerId automatically + => val uuid = UUID.randomUUID() + consumerUuid = "%s-%d-%s".format( + InetAddress.getLocalHost.getHostName, System.currentTimeMillis, + uuid.getMostSignificantBits().toHexString.substring(0,8)) + } + config.groupId + "_" + consumerUuid + } + this.logIdent = consumerIdString + " " connectZk() createFetcher() @@ -108,10 +130,18 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def createMessageStreams[T](topicCountMap: Map[String,Int], decoder: Decoder[T]) - : Map[String,List[KafkaMessageStream[T]]] = { + : Map[String,List[KafkaStream[T]]] = { + if (messageStreamCreated.getAndSet(true)) + throw new RuntimeException(this.getClass.getSimpleName + + " can create message streams at most once") consume(topicCountMap, decoder) } + def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) = { + val wildcardStreamsHandler = new WildcardStreamsHandler[T](topicFilter, numStreams, decoder) + wildcardStreamsHandler.streams + } + private def createFetcher() { if (enableFetcher) fetcher = Some(new Fetcher(config, zkClient)) @@ -126,6 +156,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val canShutdown = isShuttingDown.compareAndSet(false, true); if (canShutdown) { info("ZKConsumerConnector shutting down") + + if (wildcardTopicWatcher != null) + wildcardTopicWatcher.shutdown() try { scheduler.shutdownNow() fetcher match { @@ -150,69 +183,42 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def consume[T](topicCountMap: scala.collection.Map[String,Int], decoder: Decoder[T]) - : Map[String,List[KafkaMessageStream[T]]] = { + : Map[String,List[KafkaStream[T]]] = { debug("entering consume ") if (topicCountMap == null) throw new RuntimeException("topicCountMap is null") - val dirs = new ZKGroupDirs(config.groupId) - var ret = new mutable.HashMap[String,List[KafkaMessageStream[T]]] + val topicCount = TopicCount.constructTopicCount(consumerIdString, topicCountMap) - var consumerUuid : String = null - config.consumerId match { - case Some(consumerId) // for testing only - => consumerUuid = consumerId - case None // generate unique consumerId automatically - => val uuid = UUID.randomUUID() - consumerUuid = "%s-%d-%s".format( - InetAddress.getLocalHost.getHostName, System.currentTimeMillis, - uuid.getMostSignificantBits().toHexString.substring(0,8)) - } - val consumerIdString = config.groupId + "_" + consumerUuid - val topicCount = new TopicCount(consumerIdString, topicCountMap) - - // create a queue per topic per consumer thread - val consumerThreadIdsPerTopic = topicCount.getConsumerThreadIdsPerTopic - for ((topic, threadIdSet) <- consumerThreadIdsPerTopic) { - var streamList: List[KafkaMessageStream[T]] = Nil - for (threadId <- threadIdSet) { - val stream = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) - queues.put((topic, threadId), stream) - streamList ::= new KafkaMessageStream[T](topic, stream, config.consumerTimeoutMs, decoder, config.enableShallowIterator) - } - ret += (topic -> streamList) - debug("adding topic " + topic + " and stream to map..") - } - - // listener to consumer and partition changes - val loadBalancerListener = new ZKRebalancerListener[T](config.groupId, consumerIdString, ret) - registerConsumerInZK(dirs, consumerIdString, topicCount) + val topicThreadIds = topicCount.getConsumerThreadIdsPerTopic - // register listener for session expired event - zkClient.subscribeStateChanges( - new ZKSessionExpireListener[T](dirs, consumerIdString, topicCount, loadBalancerListener)) + // make a list of (queue,stream) pairs, one pair for each threadId + val queuesAndStreams = topicThreadIds.values.map(threadIdSet => + threadIdSet.map(_ => { + val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) + val stream = new KafkaStream[T]( + queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator) + (queue, stream) + }) + ).flatten.toList - zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener) - - ret.foreach { topicAndStreams => - // register on broker partition path changes - val partitionPath = BrokerTopicsPath + "/" + topicAndStreams._1 - zkClient.subscribeChildChanges(partitionPath, loadBalancerListener) - } + val dirs = new ZKGroupDirs(config.groupId) + registerConsumerInZK(dirs, consumerIdString, topicCount) + reinitializeConsumer(topicCount, queuesAndStreams) - // explicitly trigger load balancing for this consumer - loadBalancerListener.syncedRebalance() - ret + loadBalancerListener.kafkaMessageAndMetadataStreams.asInstanceOf[Map[String, List[KafkaStream[T]]]] } - private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = { + private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) { info("begin registering consumer " + consumerIdString + " in ZK") - createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString) + createEphemeralPathExpectConflict(zkClient, + dirs.consumerRegistryDir + "/" + consumerIdString, + topicCount.dbString) info("end registering consumer " + consumerIdString + " in ZK") } private def sendShutdownToAllQueues() = { - for (queue <- queues.values) { + for (queue <- topicThreadIdAndQueues.values) { debug("Clearing up queue") queue.clear() queue.put(ZookeeperConsumerConnector.shutdownCommand) @@ -334,10 +340,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, producedOffset } - class ZKSessionExpireListener[T](val dirs: ZKGroupDirs, + class ZKSessionExpireListener(val dirs: ZKGroupDirs, val consumerIdString: String, val topicCount: TopicCount, - val loadBalancerListener: ZKRebalancerListener[T]) + val loadBalancerListener: ZKRebalancerListener) extends IZkStateListener { @throws(classOf[Exception]) def handleStateChanged(state: KeeperState) { @@ -359,10 +365,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * consumer in the consumer registry and trigger a rebalance. */ info("ZK expired; release old broker parition ownership; re-register consumer " + consumerIdString) - loadBalancerListener.resetState + loadBalancerListener.resetState() registerConsumerInZK(dirs, consumerIdString, topicCount) // explicitly trigger load balancing for this consumer - loadBalancerListener.syncedRebalance + loadBalancerListener.syncedRebalance() // There is no need to resubscribe to child and state changes. // The child change watchers will be set inside rebalance when we read the children list. @@ -370,8 +376,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } - class ZKRebalancerListener[T](val group: String, val consumerIdString: String, - kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) + class ZKRebalancerListener(val group: String, val consumerIdString: String, + val kafkaMessageAndMetadataStreams: mutable.Map[String,List[KafkaStream[_]]]) extends IZkChildListener { private var isWatcherTriggered = false private val lock = new ReentrantLock @@ -459,7 +465,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") } // stop all fetchers and clear all the queues to avoid data duplication - closeFetchersForQueues(cluster, kafkaMessageStreams, queues.map(q => q._2)) + closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) Thread.sleep(config.rebalanceBackoffMs) } } @@ -468,7 +474,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def rebalance(cluster: Cluster): Boolean = { - val myTopicThreadIdsMap = getTopicCount(zkClient, group, consumerIdString).getConsumerThreadIdsPerTopic + val myTopicThreadIdsMap = TopicCount.constructTopicCount(group, consumerIdString, zkClient).getConsumerThreadIdsPerTopic val consumersPerTopicMap = getConsumersPerTopic(zkClient, group) val partitionsPerTopicMap = getPartitionsForTopics(zkClient, myTopicThreadIdsMap.keys.iterator) @@ -478,7 +484,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * But if we don't stop the fetchers first, this consumer would continue returning data for released * partitions in parallel. So, not stopping the fetchers leads to duplicate data. */ - closeFetchers(cluster, kafkaMessageStreams, myTopicThreadIdsMap) + closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap) releasePartitionOwnership(topicRegistry) @@ -531,7 +537,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, debug("Partitions per topic cache " + partitionsPerTopicMap) debug("Consumers per topic cache " + consumersPerTopicMap) topicRegistry = currentTopicRegistry - updateFetcher(cluster, kafkaMessageStreams) + updateFetcher(cluster) true }else { false @@ -539,12 +545,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def closeFetchersForQueues(cluster: Cluster, - kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]], + messageStreams: Map[String,List[KafkaStream[_]]], queuesToBeCleared: Iterable[BlockingQueue[FetchedDataChunk]]) { var allPartitionInfos = topicRegistry.values.map(p => p.values).flatten fetcher match { case Some(f) => f.stopConnectionsToAllBrokers - f.clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, kafkaMessageStreams) + f.clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams) info("Committing all offsets after clearing the fetcher queues") /** * here, we need to commit offsets before stopping the consumer from returning any more messages @@ -559,16 +565,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - private def closeFetchers(cluster: Cluster, kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]], + private def closeFetchers(cluster: Cluster, messageStreams: Map[String,List[KafkaStream[_]]], relevantTopicThreadIdsMap: Map[String, Set[String]]) { // only clear the fetcher queues for certain topic partitions that *might* no longer be served by this consumer // after this rebalancing attempt - val queuesTobeCleared = queues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) - closeFetchersForQueues(cluster, kafkaMessageStreams, queuesTobeCleared) + val queuesTobeCleared = topicThreadIdAndQueues.filter(q => relevantTopicThreadIdsMap.contains(q._1._1)).map(q => q._2) + closeFetchersForQueues(cluster, messageStreams, queuesTobeCleared) } - private def updateFetcher[T](cluster: Cluster, - kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]]) { + private def updateFetcher(cluster: Cluster) { // update partitions for fetcher var allPartitionInfos : List[PartitionTopicInfo] = Nil for (partitionInfos <- topicRegistry.values) @@ -579,7 +584,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, fetcher match { case Some(f) => - f.startConnections(allPartitionInfos, cluster, kafkaMessageStreams) + f.startConnections(allPartitionInfos, cluster) case None => } } @@ -637,7 +642,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } else offset = offsetString.toLong - val queue = queues.get((topic, consumerThreadId)) + val queue = topicThreadIdAndQueues.get((topic, consumerThreadId)) val consumedOffset = new AtomicLong(offset) val fetchedOffset = new AtomicLong(offset) val partTopicInfo = new PartitionTopicInfo(topic, @@ -651,5 +656,155 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, debug(partTopicInfo + " selected new offset " + offset) } } + + private def reinitializeConsumer[T]( + topicCount: TopicCount, + queuesAndStreams: List[(LinkedBlockingQueue[FetchedDataChunk],KafkaStream[T])]) { + + val dirs = new ZKGroupDirs(config.groupId) + + // listener to consumer and partition changes + if (loadBalancerListener == null) { + val topicStreamsMap = new mutable.HashMap[String,List[KafkaStream[T]]] + loadBalancerListener = new ZKRebalancerListener( + config.groupId, consumerIdString, topicStreamsMap.asInstanceOf[scala.collection.mutable.Map[String, List[KafkaStream[_]]]]) + } + + // register listener for session expired event + if (sessionExpirationListener == null) + sessionExpirationListener = new ZKSessionExpireListener( + dirs, consumerIdString, topicCount, loadBalancerListener) + + val topicStreamsMap = loadBalancerListener.kafkaMessageAndMetadataStreams + + // map of {topic -> Set(thread-1, thread-2, ...)} + val consumerThreadIdsPerTopic: Map[String, Set[String]] = + topicCount.getConsumerThreadIdsPerTopic + + /* + * This usage of map flatten breaks up consumerThreadIdsPerTopic into + * a set of (topic, thread-id) pairs that we then use to construct + * the updated (topic, thread-id) -> queues map + */ + implicit def getTopicThreadIds(v: (String, Set[String])): Set[(String, String)] = v._2.map((v._1, _)) + + // iterator over (topic, thread-id) tuples + val topicThreadIds: Iterable[(String, String)] = + consumerThreadIdsPerTopic.flatten + + // list of (pairs of pairs): e.g., ((topic, thread-id),(queue, stream)) + val threadQueueStreamPairs = topicCount match { + case wildTopicCount: WildcardTopicCount => + for (tt <- topicThreadIds; qs <- queuesAndStreams) yield (tt -> qs) + case statTopicCount: StaticTopicCount => { + require(topicThreadIds.size == queuesAndStreams.size, + "Mismatch between thread ID count (%d) and queue count (%d)".format( + topicThreadIds.size, queuesAndStreams.size)) + topicThreadIds.zip(queuesAndStreams) + } + } + + threadQueueStreamPairs.foreach(e => { + val topicThreadId = e._1 + val q = e._2._1 + topicThreadIdAndQueues.put(topicThreadId, q) + }) + + val groupedByTopic = threadQueueStreamPairs.groupBy(_._1._1) + groupedByTopic.foreach(e => { + val topic = e._1 + val streams = e._2.map(_._2._2).toList + topicStreamsMap += (topic -> streams) + debug("adding topic %s and %d streams to map.".format(topic, streams.size)) + }) + + // listener to consumer and partition changes + zkClient.subscribeStateChanges(sessionExpirationListener) + + zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener) + + topicStreamsMap.foreach { topicAndStreams => + // register on broker partition path changes + val partitionPath = BrokerTopicsPath + "/" + topicAndStreams._1 + zkClient.subscribeChildChanges(partitionPath, loadBalancerListener) + } + + // explicitly trigger load balancing for this consumer + loadBalancerListener.syncedRebalance() + } + + class WildcardStreamsHandler[T](topicFilter: TopicFilter, + numStreams: Int, + decoder: Decoder[T]) + extends TopicEventHandler[String] { + + if (messageStreamCreated.getAndSet(true)) + throw new RuntimeException("Each consumer connector can create " + + "message streams by filter at most once.") + + private val wildcardQueuesAndStreams = (1 to numStreams) + .map(e => { + val queue = new LinkedBlockingQueue[FetchedDataChunk](config.maxQueuedChunks) + val stream = new KafkaStream[T]( + queue, config.consumerTimeoutMs, decoder, config.enableShallowIterator) + (queue, stream) + }).toList + + // bootstrap with existing topics + private var wildcardTopics = + getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) + .filter(topicFilter.isTopicAllowed) + + private val wildcardTopicCount = TopicCount.constructTopicCount( + consumerIdString, topicFilter, numStreams, zkClient) + + val dirs = new ZKGroupDirs(config.groupId) + registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount) + reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams) + + if (!topicFilter.requiresTopicEventWatcher) { + info("Not creating event watcher for trivial whitelist " + topicFilter) + } + else { + info("Creating topic event watcher for whitelist " + topicFilter) + wildcardTopicWatcher = new ZookeeperTopicEventWatcher(config, this) + + /* + * Topic events will trigger subsequent synced rebalances. Also, the + * consumer will get registered only after an allowed topic becomes + * available. + */ + } + + def handleTopicEvent(allTopics: Seq[String]) { + debug("Handling topic event") + + val updatedTopics = allTopics.filter(topicFilter.isTopicAllowed) + + val addedTopics = updatedTopics filterNot (wildcardTopics contains) + if (addedTopics.nonEmpty) + info("Topic event: added topics = %s" + .format(addedTopics)) + + /* + * TODO: Deleted topics are interesting (and will not be a concern until + * 0.8 release). We may need to remove these topics from the rebalance + * listener's map in reinitializeConsumer. + */ + val deletedTopics = wildcardTopics filterNot (updatedTopics contains) + if (deletedTopics.nonEmpty) + info("Topic event: deleted topics = %s" + .format(deletedTopics)) + + wildcardTopics = updatedTopics + info("Topics to consume = %s".format(wildcardTopics)) + + if (addedTopics.nonEmpty || deletedTopics.nonEmpty) + reinitializeConsumer(wildcardTopicCount, wildcardQueuesAndStreams) + } + + def streams: Seq[KafkaStream[T]] = + wildcardQueuesAndStreams.map(_._2) + } } diff --git a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala index eb563e11f7fe2..df83baa050a02 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala @@ -21,11 +21,9 @@ import scala.collection.JavaConversions._ import kafka.utils.{ZkUtils, ZKStringSerializer, Logging} import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient} import org.apache.zookeeper.Watcher.Event.KeeperState -import kafka.server.KafkaServerStartable -import kafka.common.ConsumerRebalanceFailedException class ZookeeperTopicEventWatcher(val config:ConsumerConfig, - val eventHandler: TopicEventHandler[String], kafkaServerStartable: KafkaServerStartable) extends Logging { + val eventHandler: TopicEventHandler[String]) extends Logging { val lock = new Object() @@ -35,7 +33,7 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig, startWatchingTopicEvents() private def startWatchingTopicEvents() { - val topicEventListener = new ZkTopicEventListener(kafkaServerStartable) + val topicEventListener = new ZkTopicEventListener() ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath) zkClient.subscribeStateChanges( @@ -52,6 +50,7 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig, def shutdown() { lock.synchronized { + info("Shutting down topic event watcher.") if (zkClient != null) { stopWatchingTopicEvents() zkClient.close() @@ -62,7 +61,7 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig, } } - class ZkTopicEventListener(val kafkaServerStartable: KafkaServerStartable) extends IZkChildListener { + class ZkTopicEventListener extends IZkChildListener { @throws(classOf[Exception]) def handleChildChange(parent: String, children: java.util.List[String]) { @@ -76,11 +75,8 @@ class ZookeeperTopicEventWatcher(val config:ConsumerConfig, } } catch { - case e: ConsumerRebalanceFailedException => - fatal("can't rebalance in embedded consumer; proceed to shutdown", e) - kafkaServerStartable.shutdown() case e => - error("error in handling child changes in embedded consumer", e) + error("error in handling child changes", e) } } } diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java index 6f50e974398e2..afb6b0ae540ca 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java @@ -17,34 +17,53 @@ package kafka.javaapi.consumer; -import kafka.consumer.KafkaMessageStream; -import kafka.message.Message; -import kafka.serializer.Decoder; import java.util.List; import java.util.Map; +import kafka.consumer.KafkaStream; +import kafka.consumer.TopicFilter; +import kafka.message.Message; +import kafka.serializer.Decoder; public interface ConsumerConnector { - /** - * Create a list of MessageStreams of type T for each topic. - * - * @param topicCountMap a map of (topic, #streams) pair - * @param decoder a decoder that converts from Message to T - * @return a map of (topic, list of KafkaMessageStream) pair. The number of items in the - * list is #streams. Each KafkaMessageStream supports an iterator of messages. - */ - public Map>> createMessageStreams( - Map topicCountMap, Decoder decoder); - public Map>> createMessageStreams( - Map topicCountMap); + /** + * Create a list of MessageStreams of type T for each topic. + * + * @param topicCountMap a map of (topic, #streams) pair + * @param decoder a decoder that converts from Message to T + * @return a map of (topic, list of KafkaStream) pairs. + * The number of items in the list is #streams. Each stream supports + * an iterator over message/metadata pairs. + */ + public Map>> createMessageStreams( + Map topicCountMap, Decoder decoder); + public Map>> createMessageStreams( + Map topicCountMap); + + /** + * Create a list of MessageAndTopicStreams containing messages of type T. + * + * @param topicFilter a TopicFilter that specifies which topics to + * subscribe to (encapsulates a whitelist or a blacklist). + * @param numStreams the number of message streams to return. + * @param decoder a decoder that converts from Message to T + * @return a list of KafkaStream. Each stream supports an + * iterator over its MessageAndMetadata elements. + */ + public List> createMessageStreamsByFilter( + TopicFilter topicFilter, int numStreams, Decoder decoder); + public List> createMessageStreamsByFilter( + TopicFilter topicFilter, int numStreams); + public List> createMessageStreamsByFilter( + TopicFilter topicFilter); - /** - * Commit the offsets of all broker partitions connected by this connector. - */ - public void commitOffsets(); + /** + * Commit the offsets of all broker partitions connected by this connector. + */ + public void commitOffsets(); - /** - * Shut down the connector - */ - public void shutdown(); + /** + * Shut down the connector + */ + public void shutdown(); } diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala index 0ee7488cfcc01..f1a469b23ca17 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala @@ -16,9 +16,11 @@ */ package kafka.javaapi.consumer -import kafka.consumer.{KafkaMessageStream, ConsumerConfig} import kafka.message.Message import kafka.serializer.{DefaultDecoder, Decoder} +import kafka.consumer._ +import scala.collection.JavaConversions.asList + /** * This class handles the consumers interaction with zookeeper @@ -68,14 +70,14 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def createMessageStreams[T]( topicCountMap: java.util.Map[String,java.lang.Integer], decoder: Decoder[T]) - : java.util.Map[String,java.util.List[KafkaMessageStream[T]]] = { + : java.util.Map[String,java.util.List[KafkaStream[T]]] = { import scala.collection.JavaConversions._ val scalaTopicCountMap: Map[String, Int] = Map.empty[String, Int] ++ asMap(topicCountMap.asInstanceOf[java.util.Map[String, Int]]) val scalaReturn = underlying.consume(scalaTopicCountMap, decoder) - val ret = new java.util.HashMap[String,java.util.List[KafkaMessageStream[T]]] + val ret = new java.util.HashMap[String,java.util.List[KafkaStream[T]]] for ((topic, streams) <- scalaReturn) { - var javaStreamList = new java.util.ArrayList[KafkaMessageStream[T]] + var javaStreamList = new java.util.ArrayList[KafkaStream[T]] for (stream <- streams) javaStreamList.add(stream) ret.put(topic, javaStreamList) @@ -85,9 +87,17 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def createMessageStreams( topicCountMap: java.util.Map[String,java.lang.Integer]) - : java.util.Map[String,java.util.List[KafkaMessageStream[Message]]] = + : java.util.Map[String,java.util.List[KafkaStream[Message]]] = createMessageStreams(topicCountMap, new DefaultDecoder) + def createMessageStreamsByFilter[T](topicFilter: TopicFilter, numStreams: Int, decoder: Decoder[T]) = + asList(underlying.createMessageStreamsByFilter(topicFilter, numStreams, decoder)) + + def createMessageStreamsByFilter(topicFilter: TopicFilter, numStreams: Int) = + createMessageStreamsByFilter(topicFilter, numStreams, new DefaultDecoder) + + def createMessageStreamsByFilter(topicFilter: TopicFilter) = + createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder) def commitOffsets() { underlying.commitOffsets diff --git a/core/src/main/scala/kafka/javaapi/message/MessageSet.scala b/core/src/main/scala/kafka/javaapi/message/MessageSet.scala index 817d9dd0d50fe..9c9c72fb634a4 100644 --- a/core/src/main/scala/kafka/javaapi/message/MessageSet.scala +++ b/core/src/main/scala/kafka/javaapi/message/MessageSet.scala @@ -17,8 +17,10 @@ package kafka.javaapi.message + import kafka.message.{MessageAndOffset, InvalidMessageException} + /** * A set of messages. A message set has a fixed serialized form, though the container * for the bytes could be either in-memory or on disk. A The format of each message is diff --git a/core/src/main/scala/kafka/javaapi/producer/async/CallbackHandler.java b/core/src/main/scala/kafka/javaapi/producer/async/CallbackHandler.java index 0e0df3dba3414..2b9397424ed92 100644 --- a/core/src/main/scala/kafka/javaapi/producer/async/CallbackHandler.java +++ b/core/src/main/scala/kafka/javaapi/producer/async/CallbackHandler.java @@ -16,9 +16,9 @@ */ package kafka.javaapi.producer.async; -import kafka.producer.async.QueueItem; import java.util.Properties; +import kafka.producer.async.QueueItem; /** * Callback handler APIs for use in the async producer. The purpose is to diff --git a/core/src/main/scala/kafka/javaapi/producer/async/EventHandler.java b/core/src/main/scala/kafka/javaapi/producer/async/EventHandler.java index 381f8e22c4ba1..842799d7e94fb 100644 --- a/core/src/main/scala/kafka/javaapi/producer/async/EventHandler.java +++ b/core/src/main/scala/kafka/javaapi/producer/async/EventHandler.java @@ -16,12 +16,12 @@ */ package kafka.javaapi.producer.async; -import kafka.javaapi.producer.SyncProducer; -import kafka.producer.async.QueueItem; -import kafka.serializer.Encoder; import java.util.List; import java.util.Properties; +import kafka.javaapi.producer.SyncProducer; +import kafka.producer.async.QueueItem; +import kafka.serializer.Encoder; /** * Handler that dispatches the batched data from the queue of the diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 07c5b1f1bde49..5afd6e1132327 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -36,7 +36,6 @@ import kafka.common.{MessageSizeTooLargeException, InvalidMessageSizeException, class ByteBufferMessageSet(private val buffer: ByteBuffer, private val initialOffset: Long = 0L, private val errorCode: Int = ErrorMapping.NoError) extends MessageSet with Logging { - private var validByteCount = -1L private var shallowValidByteCount = -1L if(sizeInBytes > Int.MaxValue) throw new InvalidMessageSizeException("Message set cannot be larger than " + Int.MaxValue) diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala new file mode 100644 index 0000000000000..710308e3228d8 --- /dev/null +++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala @@ -0,0 +1,21 @@ +/** + * 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 kafka.message + +case class MessageAndMetadata[T](message: T, topic: String = "") + diff --git a/core/src/main/scala/kafka/message/MessageAndOffset.scala b/core/src/main/scala/kafka/message/MessageAndOffset.scala index 1429bb2149278..d769fc6773992 100644 --- a/core/src/main/scala/kafka/message/MessageAndOffset.scala +++ b/core/src/main/scala/kafka/message/MessageAndOffset.scala @@ -13,11 +13,10 @@ * 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 kafka.message -/** - * Represents message and offset of the next message. This is used in the MessageSet to iterate over it - */ -case class MessageAndOffset(val message: Message, val offset: Long) + +case class MessageAndOffset(message: Message, offset: Long) + diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index 527928460912a..417da27f5c519 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -22,9 +22,7 @@ import org.apache.log4j.spi.LoggingEvent import org.apache.log4j.AppenderSkeleton import org.apache.log4j.helpers.LogLog import kafka.utils.Logging -import kafka.serializer.Encoder import java.util.{Properties, Date} -import kafka.message.Message import scala.collection._ class KafkaLog4jAppender extends AppenderSkeleton with Logging { diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala index 7e0a9f56bfca4..dafa6d2133bec 100644 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -22,7 +22,7 @@ import kafka.utils._ import java.util.Properties import kafka.cluster.{Partition, Broker} import java.util.concurrent.atomic.AtomicBoolean -import kafka.common.{NoBrokersForPartitionException, InvalidConfigException, InvalidPartitionException} +import kafka.common.{NoBrokersForPartitionException, InvalidPartitionException} import kafka.api.ProducerRequest class Producer[K,V](config: ProducerConfig, diff --git a/core/src/main/scala/kafka/server/KafkaServerStartable.scala b/core/src/main/scala/kafka/server/KafkaServerStartable.scala index 3e196e890dee5..370e20d29d9f8 100644 --- a/core/src/main/scala/kafka/server/KafkaServerStartable.scala +++ b/core/src/main/scala/kafka/server/KafkaServerStartable.scala @@ -17,36 +17,21 @@ package kafka.server -import kafka.utils.{Utils, Logging} -import kafka.consumer._ -import kafka.producer.{ProducerData, ProducerConfig, Producer} -import kafka.message.Message -import java.util.concurrent.CountDownLatch +import kafka.utils.Logging -import scala.collection.Map -class KafkaServerStartable(val serverConfig: KafkaConfig, - val consumerConfig: ConsumerConfig, - val producerConfig: ProducerConfig) extends Logging { +class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging { private var server : KafkaServer = null - private var embeddedConsumer : EmbeddedConsumer = null init - def this(serverConfig: KafkaConfig) = this(serverConfig, null, null) - private def init() { server = new KafkaServer(serverConfig) - if (consumerConfig != null) - embeddedConsumer = - new EmbeddedConsumer(consumerConfig, producerConfig, this) } def startup() { try { server.startup() - if (embeddedConsumer != null) - embeddedConsumer.startup() } catch { case e => @@ -57,8 +42,6 @@ class KafkaServerStartable(val serverConfig: KafkaConfig, def shutdown() { try { - if (embeddedConsumer != null) - embeddedConsumer.shutdown() server.shutdown() } catch { @@ -73,153 +56,4 @@ class KafkaServerStartable(val serverConfig: KafkaConfig, } } -class EmbeddedConsumer(private val consumerConfig: ConsumerConfig, - private val producerConfig: ProducerConfig, - private val kafkaServerStartable: KafkaServerStartable) extends TopicEventHandler[String] with Logging { - - private val whiteListTopics = - consumerConfig.mirrorTopicsWhitelist.split(",").toList.map(_.trim) - - private val blackListTopics = - consumerConfig.mirrorTopicsBlackList.split(",").toList.map(_.trim) - - // mirrorTopics should be accessed by handleTopicEvent only - private var mirrorTopics:Seq[String] = List() - - private var consumerConnector: ConsumerConnector = null - private var topicEventWatcher:ZookeeperTopicEventWatcher = null - - private val producer = new Producer[Null, Message](producerConfig) - - var threadList = List[MirroringThread]() - - private def isTopicAllowed(topic: String) = { - if (consumerConfig.mirrorTopicsWhitelist.nonEmpty) - whiteListTopics.contains(topic) - else - !blackListTopics.contains(topic) - } - - // TopicEventHandler call-back only - @Override - def handleTopicEvent(allTopics: Seq[String]) { - val newMirrorTopics = allTopics.filter(isTopicAllowed) - - val addedTopics = newMirrorTopics filterNot (mirrorTopics contains) - if (addedTopics.nonEmpty) - info("topic event: added topics = %s".format(addedTopics)) - - val deletedTopics = mirrorTopics filterNot (newMirrorTopics contains) - if (deletedTopics.nonEmpty) - info("topic event: deleted topics = %s".format(deletedTopics)) - - mirrorTopics = newMirrorTopics - - if (addedTopics.nonEmpty || deletedTopics.nonEmpty) { - info("mirror topics = %s".format(mirrorTopics)) - startNewConsumerThreads(makeTopicMap(mirrorTopics)) - } - } - - private def makeTopicMap(mirrorTopics: Seq[String]) = { - if (mirrorTopics.nonEmpty) - Utils.getConsumerTopicMap(mirrorTopics.mkString( - "", ":%d,".format(consumerConfig.mirrorConsumerNumThreads), - ":%d".format(consumerConfig.mirrorConsumerNumThreads))) - else - Utils.getConsumerTopicMap("") - } - - private def startNewConsumerThreads(topicMap: Map[String, Int]) { - if (topicMap.nonEmpty) { - if (consumerConnector != null) - consumerConnector.shutdown() - - /** - * Before starting new consumer threads for the updated set of topics, - * shutdown the existing mirroring threads. Since the consumer connector - * is already shutdown, the mirroring threads should finish their task almost - * instantaneously. If they don't, this points to an error that needs to be looked - * into, and further mirroring should stop - */ - threadList.foreach(_.shutdown) - - // KAFKA: 212: clear the thread list to remove the older thread references that are already shutdown - threadList = Nil - - consumerConnector = Consumer.create(consumerConfig) - val topicMessageStreams = consumerConnector.createMessageStreams(topicMap) - for ((topic, streamList) <- topicMessageStreams) - for (i <- 0 until streamList.length) - threadList ::= new MirroringThread(streamList(i), topic, i) - - threadList.foreach(_.start) - } - else - info("Not starting mirroring threads (mirror topic list is empty)") - } - - def startup() { - info("staring up embedded consumer") - topicEventWatcher = new ZookeeperTopicEventWatcher(consumerConfig, this, kafkaServerStartable) - /* - * consumer threads are (re-)started upon topic events (which includes an - * initial startup event which lists the current topics) - */ - } - - def shutdown() { - // first shutdown the topic watcher to prevent creating new consumer streams - if (topicEventWatcher != null) - topicEventWatcher.shutdown() - info("Stopped the ZK watcher for new topics, now stopping the Kafka consumers") - // stop pulling more data for mirroring - if (consumerConnector != null) - consumerConnector.shutdown() - info("Stopped the kafka consumer threads for existing topics, now stopping the existing mirroring threads") - // wait for all mirroring threads to stop - threadList.foreach(_.shutdown) - info("Stopped all existing mirroring threads, now stopping the producer") - // only then, shutdown the producer - producer.close() - info("Successfully shutdown this Kafka mirror") - } - - class MirroringThread(val stream: KafkaMessageStream[Message], val topic: String, val threadId: Int) extends Thread with Logging { - val shutdownComplete = new CountDownLatch(1) - val name = "kafka-embedded-consumer-%s-%d".format(topic, threadId) - this.setDaemon(false) - this.setName(name) - - - override def run = { - info("Starting mirroring thread %s for topic %s and stream %d".format(name, topic, threadId)) - - try { - for (message <- stream) { - trace("Mirroring thread received message " + message.checksum) - val pd = new ProducerData[Null, Message](topic, message) - producer.send(pd) - } - } - catch { - case e => - fatal(topic + " stream " + threadId + " unexpectedly exited", e) - }finally { - shutdownComplete.countDown - info("Stopped mirroring thread %s for topic %s and stream %d".format(name, topic, threadId)) - } - } - - def shutdown = { - try { - shutdownComplete.await - }catch { - case e: InterruptedException => fatal("Shutdown of thread " + name + " interrupted. " + - "Mirroring thread might leak data!") - } - } - } -} - diff --git a/core/src/main/scala/kafka/tools/ConsumerShell.scala b/core/src/main/scala/kafka/tools/ConsumerShell.scala index 53f59b7529a7b..5eb526944562e 100644 --- a/core/src/main/scala/kafka/tools/ConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/ConsumerShell.scala @@ -82,15 +82,15 @@ object ConsumerShell { } } -class ZKConsumerThread(stream: KafkaMessageStream[String]) extends Thread with Logging { +class ZKConsumerThread(stream: KafkaStream[String]) extends Thread with Logging { val shutdownLatch = new CountDownLatch(1) override def run() { println("Starting consumer thread..") var count: Int = 0 try { - for (message <- stream) { - println("consumed: " + message) + for (messageAndMetadata <- stream) { + println("consumed: " + messageAndMetadata.message) count += 1 } }catch { diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala new file mode 100644 index 0000000000000..98dd65df5e770 --- /dev/null +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -0,0 +1,162 @@ +/** + * 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 kafka.tools + +import kafka.message.Message +import joptsimple.OptionParser +import kafka.utils.{Utils, Logging} +import kafka.producer.{ProducerData, ProducerConfig, Producer} +import scala.collection.JavaConversions._ +import java.util.concurrent.CountDownLatch +import kafka.consumer._ + + +object MirrorMaker extends Logging { + + def main(args: Array[String]) { + + info ("Starting mirror maker") + val parser = new OptionParser + + val consumerConfigOpt = parser.accepts("consumer-config", + "Consumer config to consume from a source cluster. " + + "You may specify multiple of these.") + .withRequiredArg() + .describedAs("config file") + .ofType(classOf[String]) + + val producerConfigOpt = parser.accepts("producer-config", + "Embedded producer config.") + .withRequiredArg() + .describedAs("config file") + .ofType(classOf[String]) + + val numStreamsOpt = parser.accepts("num-streams", + "Number of consumption streams.") + .withRequiredArg() + .describedAs("Number of threads") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) + + val whitelistOpt = parser.accepts("whitelist", + "Whitelist of topics to mirror.") + .withRequiredArg() + .describedAs("Java regex (String)") + .ofType(classOf[String]) + + val blacklistOpt = parser.accepts("blacklist", + "Blacklist of topics to mirror.") + .withRequiredArg() + .describedAs("Java regex (String)") + .ofType(classOf[String]) + + val helpOpt = parser.accepts("help", "Print this message.") + + val options = parser.parse(args : _*) + + if (options.has(helpOpt)) { + parser.printHelpOn(System.out) + System.exit(0) + } + + Utils.checkRequiredArgs( + parser, options, consumerConfigOpt, producerConfigOpt) + if (List(whitelistOpt, blacklistOpt).count(options.has) != 1) { + println("Exactly one of whitelist or blacklist is required.") + System.exit(1) + } + + val numStreams = options.valueOf(numStreamsOpt) + + val producer = { + val config = new ProducerConfig( + Utils.loadProps(options.valueOf(producerConfigOpt))) + new Producer[Null, Message](config) + } + + val threads = { + val connectors = options.valuesOf(consumerConfigOpt).toList + .map(cfg => new ConsumerConfig(Utils.loadProps(cfg.toString))) + .map(new ZookeeperConsumerConnector(_)) + + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + connectors.foreach(_.shutdown()) + producer.close() + } + }) + + val filterSpec = if (options.has(whitelistOpt)) + new Whitelist(options.valueOf(whitelistOpt)) + else + new Blacklist(options.valueOf(blacklistOpt)) + + val streams = + connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue())) + + streams.flatten.zipWithIndex.map(streamAndIndex => { + new MirrorMakerThread(streamAndIndex._1, producer, streamAndIndex._2) + }) + } + + threads.foreach(_.start()) + + threads.foreach(_.awaitShutdown()) + } + + class MirrorMakerThread(stream: KafkaStream[Message], + producer: Producer[Null, Message], + threadId: Int) + extends Thread with Logging { + + private val shutdownLatch = new CountDownLatch(1) + private val threadName = "mirrormaker-" + threadId + + this.setName(threadName) + + override def run() { + try { + for (msgAndMetadata <- stream) { + val pd = new ProducerData[Null, Message]( + msgAndMetadata.topic, msgAndMetadata.message) + producer.send(pd) + } + } + catch { + case e => + fatal("%s stream unexpectedly exited.", e) + } + finally { + shutdownLatch.countDown() + info("Stopped thread %s.".format(threadName)) + } + } + + def awaitShutdown() { + try { + shutdownLatch.await() + } + catch { + case e: InterruptedException => fatal( + "Shutdown of thread %s interrupted. This might leak data!" + .format(threadName)) + } + } + } +} + diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index 79d3998959b7b..1300cf6d0d9c7 100644 --- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala +++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala @@ -34,8 +34,6 @@ object ReplayLogProducer extends Logging { private val GROUPID: String = "replay-log-producer" def main(args: Array[String]) { - var isNoPrint = false; - val config = new Config(args) val executor = Executors.newFixedThreadPool(config.numThreads) @@ -153,7 +151,7 @@ object ReplayLogProducer extends Logging { } } - class ZKConsumerThread(config: Config, stream: KafkaMessageStream[Message]) extends Thread with Logging { + class ZKConsumerThread(config: Config, stream: KafkaStream[Message]) extends Thread with Logging { val shutdownLatch = new CountDownLatch(1) val props = new Properties() val brokerInfoList = config.brokerInfo.split("=") @@ -184,9 +182,9 @@ object ReplayLogProducer extends Logging { stream.slice(0, config.numMessages) else stream - for (message <- iter) { + for (messageAndMetadata <- iter) { try { - producer.send(new ProducerData[Message, Message](config.outputTopic, message)) + producer.send(new ProducerData[Message, Message](config.outputTopic, messageAndMetadata.message)) if (config.delayedMSBtwSend > 0 && (messageCount + 1) % config.batchSize == 0) Thread.sleep(config.delayedMSBtwSend) messageCount += 1 diff --git a/core/src/main/scala/kafka/utils/Logging.scala b/core/src/main/scala/kafka/utils/Logging.scala index 2e664f5c26bc7..6e05eb461689a 100644 --- a/core/src/main/scala/kafka/utils/Logging.scala +++ b/core/src/main/scala/kafka/utils/Logging.scala @@ -23,72 +23,76 @@ trait Logging { val loggerName = this.getClass.getName lazy val logger = Logger.getLogger(loggerName) + protected var logIdent = "" + + private def msgWithLogIdent(msg: String) = "%s%s".format(logIdent, msg) + def trace(msg: => String): Unit = { if (logger.isTraceEnabled()) - logger.trace(msg) + logger.trace(msgWithLogIdent(msg)) } def trace(e: => Throwable): Any = { if (logger.isTraceEnabled()) - logger.trace("",e) + logger.trace(logIdent,e) } def trace(msg: => String, e: => Throwable) = { if (logger.isTraceEnabled()) - logger.trace(msg,e) + logger.trace(msgWithLogIdent(msg),e) } def debug(msg: => String): Unit = { if (logger.isDebugEnabled()) - logger.debug(msg) + logger.debug(msgWithLogIdent(msg)) } def debug(e: => Throwable): Any = { if (logger.isDebugEnabled()) - logger.debug("",e) + logger.debug(logIdent,e) } def debug(msg: => String, e: => Throwable) = { if (logger.isDebugEnabled()) - logger.debug(msg,e) + logger.debug(msgWithLogIdent(msg),e) } def info(msg: => String): Unit = { if (logger.isInfoEnabled()) - logger.info(msg) + logger.info(msgWithLogIdent(msg)) } def info(e: => Throwable): Any = { if (logger.isInfoEnabled()) - logger.info("",e) + logger.info(logIdent,e) } def info(msg: => String,e: => Throwable) = { if (logger.isInfoEnabled()) - logger.info(msg,e) + logger.info(msgWithLogIdent(msg),e) } def warn(msg: => String): Unit = { - logger.warn(msg) + logger.warn(msgWithLogIdent(msg)) } def warn(e: => Throwable): Any = { - logger.warn("",e) + logger.warn(logIdent,e) } def warn(msg: => String, e: => Throwable) = { - logger.warn(msg,e) + logger.warn(msgWithLogIdent(msg),e) } def error(msg: => String): Unit = { - logger.error(msg) + logger.error(msgWithLogIdent(msg)) } def error(e: => Throwable): Any = { - logger.error("",e) + logger.error(logIdent,e) } def error(msg: => String, e: => Throwable) = { - logger.error(msg,e) + logger.error(msgWithLogIdent(msg),e) } def fatal(msg: => String): Unit = { - logger.fatal(msg) + logger.fatal(msgWithLogIdent(msg)) } def fatal(e: => Throwable): Any = { - logger.fatal("",e) + logger.fatal(logIdent,e) } def fatal(msg: => String, e: => Throwable) = { - logger.fatal(msg,e) + logger.fatal(msgWithLogIdent(msg),e) } } diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 7b8b5ae370195..a3c27010be3c3 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -29,12 +29,13 @@ import scala.collection._ import scala.collection.mutable import kafka.message.{NoCompressionCodec, CompressionCodec} import org.I0Itec.zkclient.ZkClient +import joptsimple.{OptionSpec, OptionSet, OptionParser} + /** * Helper functions! */ object Utils extends Logging { - /** * Wrap the given function in a java.lang.Runnable * @param fun A function @@ -657,6 +658,16 @@ object Utils extends Logging { case _ => // swallow } } + + def checkRequiredArgs(parser: OptionParser, options: OptionSet, required: OptionSpec[_]*) { + for(arg <- required) { + if(!options.has(arg)) { + error("Missing required argument \"" + arg + "\"") + parser.printHelpOn(System.err) + System.exit(1) + } + } + } } class SnapshotStats(private val monitorDurationNs: Long = 600L * 1000L * 1000L * 1000L) { diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 917da0f8cda38..caddb06bea1b4 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -243,17 +243,11 @@ object ZkUtils extends Logging { getChildren(zkClient, dirs.consumerRegistryDir) } - def getTopicCount(zkClient: ZkClient, group: String, consumerId: String) : TopicCount = { - val dirs = new ZKGroupDirs(group) - val topicCountJson = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId) - TopicCount.constructTopicCount(consumerId, topicCountJson) - } - def getConsumerTopicMaps(zkClient: ZkClient, group: String): Map[String, TopicCount] = { val dirs = new ZKGroupDirs(group) val consumersInGroup = getConsumersInGroup(zkClient, group) val topicCountMaps = consumersInGroup.map(consumerId => TopicCount.constructTopicCount(consumerId, - ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId))) + ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId), zkClient)) consumersInGroup.zip(topicCountMaps).toMap } @@ -262,8 +256,8 @@ object ZkUtils extends Logging { val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir) val consumersPerTopicMap = new mutable.HashMap[String, List[String]] for (consumer <- consumers) { - val topicCount = getTopicCount(zkClient, group, consumer) - for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic()) { + val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient) + for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) { for (consumerThreadId <- consumerThreadIdSet) consumersPerTopicMap.get(topic) match { case Some(curConsumers) => consumersPerTopicMap.put(topic, consumerThreadId :: curConsumers) diff --git a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala b/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala index c8e4a3c65de7a..fa709de76145c 100644 --- a/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala +++ b/core/src/test/scala/other/kafka/TestZKConsumerOffsets.scala @@ -56,13 +56,13 @@ object TestZKConsumerOffsets { } } -private class ConsumerThread(stream: KafkaMessageStream[Message]) extends Thread { +private class ConsumerThread(stream: KafkaStream[Message]) extends Thread { val shutdownLatch = new CountDownLatch(1) override def run() { println("Starting consumer thread..") - for (message <- stream) { - println("consumed: " + Utils.toString(message.payload, "UTF-8")) + for (messageAndMetadata <- stream) { + println("consumed: " + Utils.toString(messageAndMetadata.message.payload, "UTF-8")) } shutdownLatch.countDown println("thread shutdown !" ) diff --git a/core/src/test/scala/unit/kafka/consumer/TopicCountTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala similarity index 53% rename from core/src/test/scala/unit/kafka/consumer/TopicCountTest.scala rename to core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala index 18134442ecadc..40a2bf7a9277e 100644 --- a/core/src/test/scala/unit/kafka/consumer/TopicCountTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala @@ -5,7 +5,7 @@ * 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 @@ -17,33 +17,35 @@ package kafka.consumer + import junit.framework.Assert._ -import org.junit.Test import org.scalatest.junit.JUnitSuite -import kafka.cluster.Partition +import org.junit.Test -class TopicCountTest extends JUnitSuite { +class TopicFilterTest extends JUnitSuite { @Test - def testBasic() { - val consumer = "conusmer1" - val json = """{ "topic1" : 2, "topic2" : 3 }""" - val topicCount = TopicCount.constructTopicCount(consumer, json) - val topicCountMap = Map( - "topic1" -> 2, - "topic2" -> 3 - ) - val expectedTopicCount = new TopicCount(consumer, topicCountMap) - assertTrue(expectedTopicCount == topicCount) - - val topicCount2 = TopicCount.constructTopicCount(consumer, expectedTopicCount.toJsonString) - assertTrue(expectedTopicCount == topicCount2) + def testWhitelists() { + + val topicFilter1 = new Whitelist("white1,white2") + assertFalse(topicFilter1.requiresTopicEventWatcher) + assertTrue(topicFilter1.isTopicAllowed("white2")) + assertFalse(topicFilter1.isTopicAllowed("black1")) + + val topicFilter2 = new Whitelist(".+") + assertTrue(topicFilter2.requiresTopicEventWatcher) + assertTrue(topicFilter2.isTopicAllowed("alltopics")) + + val topicFilter3 = new Whitelist("white_listed-topic.+") + assertTrue(topicFilter3.requiresTopicEventWatcher) + assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1")) + assertFalse(topicFilter3.isTopicAllowed("black1")) } @Test - def testPartition() { - assertTrue(new Partition(10, 0) == new Partition(10, 0)) - assertTrue(new Partition(10, 1) != new Partition(10, 0)) + def testBlacklists() { + val topicFilter1 = new Blacklist("black1") + assertTrue(topicFilter1.requiresTopicEventWatcher) } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index e255b7d4be8a9..0df05d3b82d53 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -234,7 +234,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val iterator = messageStream.iterator for (i <- 0 until nMessages * 2) { assertTrue(iterator.hasNext()) - val message = iterator.next() + val message = iterator.next().message receivedMessages ::= message debug("received message: " + message) } @@ -270,14 +270,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar messages.sortWith((s,t) => s.checksum < t.checksum) } - def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= { + def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaStream[Message]]]): List[Message]= { var messages: List[Message] = Nil for ((topic, messageStreams) <- topicMessageStreams) { for (messageStream <- messageStreams) { val iterator = messageStream.iterator for (i <- 0 until nMessagesPerThread) { assertTrue(iterator.hasNext) - val message = iterator.next + val message = iterator.next.message messages ::= message debug("received message: " + Utils.toString(message.payload, "UTF-8")) } diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 40b7ff4143014..915af85e78774 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -56,7 +56,7 @@ class FetcherTest extends JUnit3Suite with KafkaServerTestHarness { super.setUp fetcher = new Fetcher(new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), null) fetcher.stopConnectionsToAllBrokers - fetcher.startConnections(topicInfos, cluster, null) + fetcher.startConnections(topicInfos, cluster) } override def tearDown() { diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala index c7653e58271e9..f7a4b154761c9 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -26,9 +26,10 @@ import kafka.utils.{TestZKUtils, TestUtils} import org.scalatest.junit.JUnit3Suite import scala.collection.JavaConversions._ import kafka.javaapi.message.ByteBufferMessageSet -import kafka.consumer.{ConsumerConfig, KafkaMessageStream} import org.apache.log4j.{Level, Logger} import kafka.message.{NoCompressionCodec, CompressionCodec, Message} +import kafka.consumer.{KafkaStream, ConsumerConfig} + class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with ZooKeeperTestHarness with Logging { @@ -91,7 +92,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar messages.sortWith((s,t) => s.checksum < t.checksum) } - def getMessages(nMessagesPerThread: Int, jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaMessageStream[Message]]]) + def getMessages(nMessagesPerThread: Int, jTopicMessageStreams: java.util.Map[String, java.util.List[KafkaStream[Message]]]) : List[Message]= { var messages: List[Message] = Nil val topicMessageStreams = asMap(jTopicMessageStreams) @@ -100,7 +101,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val iterator = messageStream.iterator for (i <- 0 until nMessagesPerThread) { assertTrue(iterator.hasNext) - val message = iterator.next + val message = iterator.next.message messages ::= message debug("received message: " + Utils.toString(message.payload, "UTF-8")) } diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala index 7e75f8f2b2d43..218e2298536c4 100644 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -21,6 +21,7 @@ import org.apache.log4j.Logger import org.scalatest.junit.JUnitSuite import org.junit.Test + class UtilsTest extends JUnitSuite { private val logger = Logger.getLogger(classOf[UtilsTest]) @@ -29,5 +30,5 @@ class UtilsTest extends JUnitSuite { def testSwallow() { Utils.swallow(logger.info, throw new IllegalStateException("test")) } - + } diff --git a/examples/src/main/java/kafka/examples/Consumer.java b/examples/src/main/java/kafka/examples/Consumer.java index 18b7348c56473..cb01577f68276 100644 --- a/examples/src/main/java/kafka/examples/Consumer.java +++ b/examples/src/main/java/kafka/examples/Consumer.java @@ -16,16 +16,17 @@ */ package kafka.examples; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.ConsumerIterator; -import kafka.consumer.KafkaMessageStream; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.Message; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.ConsumerIterator; +import kafka.consumer.KafkaStream; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.Message; + public class Consumer extends Thread { @@ -55,10 +56,10 @@ private static ConsumerConfig createConsumerConfig() public void run() { Map topicCountMap = new HashMap(); topicCountMap.put(topic, new Integer(1)); - Map>> consumerMap = consumer.createMessageStreams(topicCountMap); - KafkaMessageStream stream = consumerMap.get(topic).get(0); + Map>> consumerMap = consumer.createMessageStreams(topicCountMap); + KafkaStream stream = consumerMap.get(topic).get(0); ConsumerIterator it = stream.iterator(); while(it.hasNext()) - System.out.println(ExampleUtils.getMessage(it.next())); + System.out.println(ExampleUtils.getMessage(it.next().message())); } } diff --git a/examples/src/main/java/kafka/examples/ExampleUtils.java b/examples/src/main/java/kafka/examples/ExampleUtils.java index c301a526e75af..34fd1c00ea123 100644 --- a/examples/src/main/java/kafka/examples/ExampleUtils.java +++ b/examples/src/main/java/kafka/examples/ExampleUtils.java @@ -16,8 +16,8 @@ */ package kafka.examples; -import java.nio.ByteBuffer; +import java.nio.ByteBuffer; import kafka.message.Message; public class ExampleUtils diff --git a/examples/src/main/java/kafka/examples/Producer.java b/examples/src/main/java/kafka/examples/Producer.java index c663a5449a160..353a7eb194957 100644 --- a/examples/src/main/java/kafka/examples/Producer.java +++ b/examples/src/main/java/kafka/examples/Producer.java @@ -16,9 +16,10 @@ */ package kafka.examples; + +import java.util.Properties; import kafka.javaapi.producer.ProducerData; import kafka.producer.ProducerConfig; -import java.util.Properties; public class Producer extends Thread { diff --git a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java index 1cb8a832ab505..c2b88dae11732 100644 --- a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java +++ b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java @@ -16,17 +16,14 @@ */ package kafka.examples; + import java.util.ArrayList; import java.util.List; - +import kafka.api.FetchRequest; import kafka.javaapi.MultiFetchResponse; import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; import kafka.message.MessageAndOffset; -import scala.collection.Iterator; - -import kafka.api.FetchRequest; -import kafka.message.Message; public class SimpleConsumerDemo diff --git a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala b/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala index 541bf42f276ac..414c9650d6c22 100644 --- a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala @@ -17,15 +17,12 @@ package kafka.perf -import java.net.URI import java.util.concurrent.CountDownLatch import java.util.concurrent.atomic.AtomicLong import java.nio.channels.ClosedByInterruptException -import joptsimple._ import org.apache.log4j.Logger import kafka.message.Message -import org.I0Itec.zkclient.ZkClient -import kafka.utils.{ZKStringSerializer, Utils} +import kafka.utils.Utils import java.util.{Random, Properties} import kafka.consumer._ import java.text.SimpleDateFormat @@ -139,7 +136,7 @@ object ConsumerPerformance { val hideHeader = options.has(hideHeaderOpt) } - class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaMessageStream[Message], + class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Message], config:ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) extends Thread(name) { private val shutdownLatch = new CountDownLatch(1) @@ -157,9 +154,9 @@ object ConsumerPerformance { var lastMessagesRead = 0L try { - for (message <- stream if messagesRead < config.numMessages) { + for (messageAndMetadata <- stream if messagesRead < config.numMessages) { messagesRead += 1 - bytesRead += message.payloadSize + bytesRead += messageAndMetadata.message.payloadSize if (messagesRead % config.reportingInterval == 0) { if(config.showDetailedStats) diff --git a/perf/src/main/scala/kafka/perf/PerfConfig.scala b/perf/src/main/scala/kafka/perf/PerfConfig.scala index 265caa8a59bea..db2c1a193860e 100644 --- a/perf/src/main/scala/kafka/perf/PerfConfig.scala +++ b/perf/src/main/scala/kafka/perf/PerfConfig.scala @@ -18,7 +18,7 @@ package kafka.perf import joptsimple.OptionParser -import java.text.SimpleDateFormat + class PerfConfig(args: Array[String]) { val parser = new OptionParser diff --git a/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala b/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala index 02c30083172d0..ca8df590e20d8 100644 --- a/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/SimpleConsumerPerformance.scala @@ -18,9 +18,7 @@ package kafka.perf import java.net.URI -import joptsimple._ import kafka.utils._ -import kafka.server._ import kafka.consumer.SimpleConsumer import org.apache.log4j.Logger import kafka.api.{OffsetRequest, FetchRequest} diff --git a/system_test/embedded_consumer/README b/system_test/embedded_consumer/README deleted file mode 100644 index 8cd6b88f8c79b..0000000000000 --- a/system_test/embedded_consumer/README +++ /dev/null @@ -1,27 +0,0 @@ -This test replicates messages from 3 kafka brokers to 2 other kafka brokers -using the embedded consumer. At the end, the messages produced at the source -brokers should match that at the target brokers. - -To run this test, do -bin/run-test.sh - -The expected output is given in bin/expected.out. There is only 1 thing that's -important. -1. The output should have a line "test passed". - -In the event of failure, by default the brokers and zookeepers remain running -to make it easier to debug the issue - hit Ctrl-C to shut them down. You can -change this behavior by setting the action_on_fail flag in the script to "exit" -or "proceed", in which case a snapshot of all the logs and directories is -placed in the test's base directory. - -If you are making any changes that may affect the embedded consumer, it is a -good idea to run the test in a loop. E.g.: - -:>/tmp/embeddedconsumer_test.log -for i in {1..10}; do echo "run $i"; ./bin/run-test.sh 2>1 >> /tmp/embeddedconsumer_test.log; done -tail -F /tmp/embeddedconsumer_test.log - -grep -ic passed /tmp/embeddedconsumer_test.log -grep -ic failed /tmp/embeddedconsumer_test.log - diff --git a/system_test/embedded_consumer/bin/run-test.sh b/system_test/embedded_consumer/bin/run-test.sh deleted file mode 100755 index e11fe27d89fe5..0000000000000 --- a/system_test/embedded_consumer/bin/run-test.sh +++ /dev/null @@ -1,328 +0,0 @@ -#!/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. - -readonly num_messages=400000 -readonly message_size=400 -readonly action_on_fail="proceed" - -readonly test_start_time="$(date +%s)" - -readonly base_dir=$(dirname $0)/.. - -info() { - echo -e "$(date +"%Y-%m-%d %H:%M:%S") $*" -} - -kill_child_processes() { - isTopmost=$1 - curPid=$2 - childPids=$(ps a -o pid= -o ppid= | grep "${curPid}$" | awk '{print $1;}') - for childPid in $childPids - do - kill_child_processes 0 $childPid - done - if [ $isTopmost -eq 0 ]; then - kill -15 $curPid 2> /dev/null - fi -} - -cleanup() { - info "cleaning up" - - pid_zk_source= - pid_zk_target= - pid_kafka_source1= - pid_kafka_source2= - pid_kafka_source3= - pid_kafka_target1= - pid_kafka_target2= - pid_producer= - - rm -rf /tmp/zookeeper_source - rm -rf /tmp/zookeeper_target - - rm -rf /tmp/kafka-source{1..3}-logs - # mkdir -p /tmp/kafka-source{1..3}-logs/test0{1..3}-0 - # touch /tmp/kafka-source{1..3}-logs/test0{1..3}-0/00000000000000000000.kafka - - rm -rf /tmp/kafka-target{1..2}-logs -} - -begin_timer() { - t_begin=$(date +%s) -} - -end_timer() { - t_end=$(date +%s) -} - -start_zk() { - info "starting zookeepers" - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source.properties 2>&1 > $base_dir/zookeeper_source.log & - pid_zk_source=$! - $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_target.properties 2>&1 > $base_dir/zookeeper_target.log & - pid_zk_target=$! -} - -start_source_servers() { - info "starting source cluster" - $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source1.properties 2>&1 > $base_dir/kafka_source1.log & - pid_kafka_source1=$! - $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source2.properties 2>&1 > $base_dir/kafka_source2.log & - pid_kafka_source2=$! - $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source3.properties 2>&1 > $base_dir/kafka_source3.log & - pid_kafka_source3=$! -} - -start_target_servers_for_whitelist_test() { - echo "starting mirror cluster" - $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target1.properties $base_dir/config/whitelisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target1.log & - pid_kafka_target1=$! - $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target2.properties $base_dir/config/whitelisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target2.log & - pid_kafka_target2=$! -} - -start_target_servers_for_blacklist_test() { - echo "starting mirror cluster" - $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target1.properties $base_dir/config/blacklisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target1.log & - pid_kafka_target1=$! - $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target2.properties $base_dir/config/blacklisttest.consumer.properties $base_dir/config/mirror_producer.properties 2>&1 > $base_dir/kafka_target2.log & - pid_kafka_target2=$! -} - -shutdown_servers() { - info "stopping producer" - if [ "x${pid_producer}" != "x" ]; then kill_child_processes 0 ${pid_producer}; fi - - info "shutting down target servers" - if [ "x${pid_kafka_target1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target1}; fi - if [ "x${pid_kafka_target2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target2}; fi - sleep 2 - - info "shutting down source servers" - if [ "x${pid_kafka_source1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source1}; fi - if [ "x${pid_kafka_source2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source2}; fi - if [ "x${pid_kafka_source3}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source3}; fi - - info "shutting down zookeeper servers" - if [ "x${pid_zk_target}" != "x" ]; then kill_child_processes 0 ${pid_zk_target}; fi - if [ "x${pid_zk_source}" != "x" ]; then kill_child_processes 0 ${pid_zk_source}; fi -} - -start_producer() { - topic=$1 - info "start producing messages for topic $topic ..." - $base_dir/../../bin/kafka-run-class.sh kafka.tools.ProducerPerformance --brokerinfo zk.connect=localhost:2181 --topic $topic --messages $num_messages --message-size $message_size --batch-size 200 --vary-message-size --threads 1 --reporting-interval $num_messages --async 2>&1 > $base_dir/producer_performance.log & - pid_producer=$! -} - -# In case the consumer does not consume, the test may exit prematurely (i.e., -# shut down the kafka brokers, and ProducerPerformance will start throwing ugly -# exceptions. So, wait for the producer to finish before shutting down. If it -# takes too long, the user can just hit Ctrl-c which is trapped to kill child -# processes. -# Usage: wait_partition_done ([kafka-server] [topic] [partition-id])+ -wait_partition_done() { - n_tuples=$(($# / 3)) - - i=1 - while (($#)); do - kafka_server[i]=$1 - topic[i]=$2 - partitionid[i]=$3 - prev_offset[i]=0 - info "\twaiting for partition on server ${kafka_server[i]}, topic ${topic[i]}, partition ${partitionid[i]}" - i=$((i+1)) - shift 3 - done - - all_done=0 - - # set -x - while [[ $all_done != 1 ]]; do - sleep 4 - i=$n_tuples - all_done=1 - for ((i=1; i <= $n_tuples; i++)); do - cur_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server ${kafka_server[i]} --topic ${topic[i]} --partition ${partitionid[i]} --time -1 --offsets 1 | tail -1) - if [ "x$cur_size" != "x${prev_offset[i]}" ]; then - all_done=0 - prev_offset[i]=$cur_size - fi - done - done - -} - -cmp_logs() { - topic=$1 - info "comparing source and target logs for topic $topic" - source_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9092 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9091 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - source_part2_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9090 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - target_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9093 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - target_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9094 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) - if [ "x$target_part0_size" == "x" ]; then target_part0_size=0; fi - if [ "x$target_part1_size" == "x" ]; then target_part1_size=0; fi - expected_size=$(($source_part0_size + $source_part1_size + $source_part2_size)) - actual_size=$(($target_part0_size + $target_part1_size)) - if [ "x$expected_size" != "x$actual_size" ] - then - info "source size: $expected_size target size: $actual_size" - return 1 - else - return 0 - fi -} - -take_fail_snapshot() { - snapshot_dir="$base_dir/failed-${snapshot_prefix}-${test_start_time}" - mkdir $snapshot_dir - for dir in /tmp/zookeeper_source /tmp/zookeeper_target /tmp/kafka-source{1..3}-logs /tmp/kafka-target{1..2}-logs; do - if [ -d $dir ]; then - cp -r $dir $snapshot_dir - fi - done -} - -# Usage: process_test_result -# result: last test result -# action_on_fail: (exit|wait|proceed) -# ("wait" is useful if you want to troubleshoot using zookeeper) -process_test_result() { - result=$1 - if [ $1 -eq 0 ]; then - info "test passed" - else - info "test failed" - case "$2" in - "wait") info "waiting: hit Ctrl-c to quit" - wait - ;; - "exit") shutdown_servers - take_fail_snapshot - exit $result - ;; - *) shutdown_servers - take_fail_snapshot - info "proceeding" - ;; - esac - fi -} - -test_whitelists() { - info "### Testing whitelists" - snapshot_prefix="whitelist-test" - - cleanup - start_zk - start_source_servers - start_target_servers_for_whitelist_test - sleep 4 - - begin_timer - - start_producer test01 - info "waiting for producer to finish producing ..." - wait_partition_done kafka://localhost:9090 test01 0 kafka://localhost:9091 test01 0 kafka://localhost:9092 test01 0 - - info "waiting for consumer to finish consuming ..." - wait_partition_done kafka://localhost:9093 test01 0 kafka://localhost:9094 test01 0 - - end_timer - info "embedded consumer took $((t_end - t_begin)) seconds" - - sleep 2 - - cmp_logs test01 - result=$? - - return $result -} - -test_blacklists() { - info "### Testing blacklists" - snapshot_prefix="blacklist-test" - cleanup - start_zk - start_source_servers - start_target_servers_for_blacklist_test - sleep 4 - - start_producer test02 - info "waiting for producer to finish producing test02 ..." - wait_partition_done kafka://localhost:9090 test02 0 kafka://localhost:9091 test02 0 kafka://localhost:9092 test02 0 - - # start_producer test03 - # info "waiting for producer to finish producing test03 ..." - # wait_partition_done kafka://localhost:9090 test03 0 kafka://localhost:9091 test03 0 kafka://localhost:9092 test03 0 - - begin_timer - - start_producer test01 - info "waiting for producer to finish producing ..." - wait_partition_done kafka://localhost:9090 test01 0 kafka://localhost:9091 test01 0 kafka://localhost:9092 test01 0 - - info "waiting for consumer to finish consuming ..." - wait_partition_done kafka://localhost:9093 test01 0 kafka://localhost:9094 test01 0 - - end_timer - - info "embedded consumer took $((t_end - t_begin)) seconds" - - sleep 2 - - cmp_logs test02 - result1=$? - # cmp_logs test03 - # result2=$? - # if [[ "x$result1" == "x0" || "x$result2" == "x0" ]]; then - if [[ "x$result1" == "x0" ]]; then - result=1 - else - cmp_logs test01 - result=$? - fi - - return $result -} - -# main test begins - -echo "Test-$test_start_time" - -# Ctrl-c trap. Catches INT signal -trap "shutdown_servers; exit 0" INT - -test_whitelists -result=$? - -process_test_result $result $action_on_fail - -shutdown_servers - -sleep 2 - -test_blacklists -result=$? - -process_test_result $result $action_on_fail - -shutdown_servers - -exit $result - diff --git a/system_test/embedded_consumer/expected.out b/system_test/embedded_consumer/expected.out deleted file mode 100644 index 2d64ec9ba1a0b..0000000000000 --- a/system_test/embedded_consumer/expected.out +++ /dev/null @@ -1,11 +0,0 @@ -start the servers ... -start producing messages ... -Total Num Messages: 10000000 bytes: 1994374785 in 106.076 secs -Messages/sec: 94272.0314 -MB/sec: 17.9304 -[2011-05-02 11:50:29,022] INFO Disconnecting from localhost:9092 (kafka.producer.SyncProducer) -wait for consumer to finish consuming ... -test passed -bin/../../../bin/kafka-server-start.sh: line 11: 359 Terminated $(dirname $0)/kafka-run-class.sh kafka.Kafka $@ -bin/../../../bin/zookeeper-server-start.sh: line 9: 357 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ -bin/../../../bin/zookeeper-server-start.sh: line 9: 358 Terminated $(dirname $0)/kafka-run-class.sh org.apache.zookeeper.server.quorum.QuorumPeerMain $@ diff --git a/system_test/mirror_maker/README b/system_test/mirror_maker/README new file mode 100644 index 0000000000000..da53c14446162 --- /dev/null +++ b/system_test/mirror_maker/README @@ -0,0 +1,22 @@ +This test replicates messages from two source kafka clusters into one target +kafka cluster using the mirror-maker tool. At the end, the messages produced +at the source brokers should match that at the target brokers. + +To run this test, do +bin/run-test.sh + +In the event of failure, by default the brokers and zookeepers remain running +to make it easier to debug the issue - hit Ctrl-C to shut them down. You can +change this behavior by setting the action_on_fail flag in the script to "exit" +or "proceed", in which case a snapshot of all the logs and directories is +placed in the test's base directory. + +It is a good idea to run the test in a loop. E.g.: + +:>/tmp/mirrormaker_test.log +for i in {1..10}; do echo "run $i"; ./bin/run-test.sh 2>1 >> /tmp/mirrormaker_test.log; done +tail -F /tmp/mirrormaker_test.log + +grep -ic passed /tmp/mirrormaker_test.log +grep -ic failed /tmp/mirrormaker_test.log + diff --git a/system_test/embedded_consumer/bin/expected.out b/system_test/mirror_maker/bin/expected.out similarity index 100% rename from system_test/embedded_consumer/bin/expected.out rename to system_test/mirror_maker/bin/expected.out diff --git a/system_test/mirror_maker/bin/run-test.sh b/system_test/mirror_maker/bin/run-test.sh new file mode 100644 index 0000000000000..bdc3f37d8e836 --- /dev/null +++ b/system_test/mirror_maker/bin/run-test.sh @@ -0,0 +1,357 @@ +#!/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. + +readonly num_messages=10000 +readonly message_size=100 +readonly action_on_fail="proceed" +# readonly action_on_fail="wait" + +readonly test_start_time="$(date +%s)" + +readonly base_dir=$(dirname $0)/.. + +info() { + echo -e "$(date +"%Y-%m-%d %H:%M:%S") $*" +} + +kill_child_processes() { + isTopmost=$1 + curPid=$2 + childPids=$(ps a -o pid= -o ppid= | grep "${curPid}$" | awk '{print $1;}') + for childPid in $childPids + do + kill_child_processes 0 $childPid + done + if [ $isTopmost -eq 0 ]; then + kill -15 $curPid 2> /dev/null + fi +} + +cleanup() { + info "cleaning up" + + pid_zk_source1= + pid_zk_source2= + pid_zk_target= + pid_kafka_source_1_1= + pid_kafka_source_1_2= + pid_kafka_source_2_1= + pid_kafka_source_2_2= + pid_kafka_target_1_1= + pid_kafka_target_1_2= + pid_producer= + pid_mirrormaker_1= + pid_mirrormaker_2= + + rm -rf /tmp/zookeeper* + + rm -rf /tmp/kafka* +} + +begin_timer() { + t_begin=$(date +%s) +} + +end_timer() { + t_end=$(date +%s) +} + +start_zk() { + info "starting zookeepers" + $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_1.properties 2>&1 > $base_dir/zookeeper_source-1.log & + pid_zk_source1=$! + $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_source_2.properties 2>&1 > $base_dir/zookeeper_source-2.log & + pid_zk_source2=$! + $base_dir/../../bin/zookeeper-server-start.sh $base_dir/config/zookeeper_target.properties 2>&1 > $base_dir/zookeeper_target.log & + pid_zk_target=$! +} + +start_source_servers() { + info "starting source cluster" + + JMX_PORT=1111 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_1.properties 2>&1 > $base_dir/kafka_source-1-1.log & + pid_kafka_source_1_1=$! + JMX_PORT=2222 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_1_2.properties 2>&1 > $base_dir/kafka_source-1-2.log & + pid_kafka_source_1_2=$! + JMX_PORT=3333 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_1.properties 2>&1 > $base_dir/kafka_source-2-1.log & + pid_kafka_source_2_1=$! + JMX_PORT=4444 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_source_2_2.properties 2>&1 > $base_dir/kafka_source-2-2.log & + pid_kafka_source_2_2=$! +} + +start_target_servers() { + info "starting mirror cluster" + JMX_PORT=5555 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_1.properties 2>&1 > $base_dir/kafka_target-1-1.log & + pid_kafka_target_1_1=$! + JMX_PORT=6666 $base_dir/../../bin/kafka-run-class.sh kafka.Kafka $base_dir/config/server_target_1_2.properties 2>&1 > $base_dir/kafka_target-1-2.log & + pid_kafka_target_1_2=$! +} + +shutdown_servers() { + info "stopping mirror-maker" + if [ "x${pid_mirrormaker_1}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_1}; fi + # sleep to avoid rebalancing during shutdown + sleep 2 + if [ "x${pid_mirrormaker_2}" != "x" ]; then kill_child_processes 0 ${pid_mirrormaker_2}; fi + + info "stopping producer" + if [ "x${pid_producer}" != "x" ]; then kill_child_processes 0 ${pid_producer}; fi + + info "shutting down target servers" + if [ "x${pid_kafka_target_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_1}; fi + if [ "x${pid_kafka_target_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_target_1_2}; fi + sleep 2 + + info "shutting down source servers" + if [ "x${pid_kafka_source_1_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_1}; fi + if [ "x${pid_kafka_source_1_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_1_2}; fi + if [ "x${pid_kafka_source_2_1}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_1}; fi + if [ "x${pid_kafka_source_2_2}" != "x" ]; then kill_child_processes 0 ${pid_kafka_source_2_2}; fi + + info "shutting down zookeeper servers" + if [ "x${pid_zk_target}" != "x" ]; then kill_child_processes 0 ${pid_zk_target}; fi + if [ "x${pid_zk_source1}" != "x" ]; then kill_child_processes 0 ${pid_zk_source1}; fi + if [ "x${pid_zk_source2}" != "x" ]; then kill_child_processes 0 ${pid_zk_source2}; fi +} + +start_producer() { + topic=$1 + zk=$2 + info "start producing messages for topic $topic to zookeeper $zk ..." + $base_dir/../../bin/kafka-run-class.sh kafka.perf.ProducerPerformance --brokerinfo zk.connect=$zk --topic $topic --messages $num_messages --message-size $message_size --batch-size 200 --vary-message-size --threads 1 --reporting-interval $num_messages --async 2>&1 > $base_dir/producer_performance.log & + pid_producer=$! +} + +# Usage: wait_partition_done ([kafka-server] [topic] [partition-id])+ +wait_partition_done() { + n_tuples=$(($# / 3)) + + i=1 + while (($#)); do + kafka_server[i]=$1 + topic[i]=$2 + partitionid[i]=$3 + prev_offset[i]=0 + info "\twaiting for partition on server ${kafka_server[i]}, topic ${topic[i]}, partition ${partitionid[i]}" + i=$((i+1)) + shift 3 + done + + all_done=0 + + # set -x + while [[ $all_done != 1 ]]; do + sleep 4 + i=$n_tuples + all_done=1 + for ((i=1; i <= $n_tuples; i++)); do + cur_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server ${kafka_server[i]} --topic ${topic[i]} --partition ${partitionid[i]} --time -1 --offsets 1 | tail -1) + if [ "x$cur_size" != "x${prev_offset[i]}" ]; then + all_done=0 + prev_offset[i]=$cur_size + fi + done + done + +} + +cmp_logs() { + topic=$1 + info "comparing source and target logs for topic $topic" + source_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9090 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) + source_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9091 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) + source_part2_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9092 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) + source_part3_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9093 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) + target_part0_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9094 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) + target_part1_size=$($base_dir/../../bin/kafka-run-class.sh kafka.tools.GetOffsetShell --server kafka://localhost:9095 --topic $topic --partition 0 --time -1 --offsets 1 | tail -1) + if [ "x$source_part0_size" == "x" ]; then source_part0_size=0; fi + if [ "x$source_part1_size" == "x" ]; then source_part1_size=0; fi + if [ "x$source_part2_size" == "x" ]; then source_part2_size=0; fi + if [ "x$source_part3_size" == "x" ]; then source_part3_size=0; fi + if [ "x$target_part0_size" == "x" ]; then target_part0_size=0; fi + if [ "x$target_part1_size" == "x" ]; then target_part1_size=0; fi + expected_size=$(($source_part0_size + $source_part1_size + $source_part2_size + $source_part3_size)) + actual_size=$(($target_part0_size + $target_part1_size)) + if [ "x$expected_size" != "x$actual_size" ] + then + info "source size: $expected_size target size: $actual_size" + return 1 + else + return 0 + fi +} + +take_fail_snapshot() { + snapshot_dir="$base_dir/failed-${snapshot_prefix}-${test_start_time}" + mkdir $snapshot_dir + for dir in /tmp/zookeeper_source{1..2} /tmp/zookeeper_target /tmp/kafka-source-{1..2}-{1..2}-logs /tmp/kafka-target{1..2}-logs; do + if [ -d $dir ]; then + cp -r $dir $snapshot_dir + fi + done +} + +# Usage: process_test_result +# result: last test result +# action_on_fail: (exit|wait|proceed) +# ("wait" is useful if you want to troubleshoot using zookeeper) +process_test_result() { + result=$1 + if [ $1 -eq 0 ]; then + info "test passed" + else + info "test failed" + case "$2" in + "wait") info "waiting: hit Ctrl-c to quit" + wait + ;; + "exit") shutdown_servers + take_fail_snapshot + exit $result + ;; + *) shutdown_servers + take_fail_snapshot + info "proceeding" + ;; + esac + fi +} + +test_whitelists() { + info "### Testing whitelists" + snapshot_prefix="whitelist-test" + + cleanup + start_zk + start_source_servers + start_target_servers + sleep 4 + + info "starting mirror makers" + JMX_PORT=7777 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer-config $base_dir/config/whitelisttest_1.consumer.properties --consumer-config $base_dir/config/whitelisttest_2.consumer.properties --producer-config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num-streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & + pid_mirrormaker_1=$! + JMX_PORT=8888 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer-config $base_dir/config/whitelisttest_1.consumer.properties --consumer-config $base_dir/config/whitelisttest_2.consumer.properties --producer-config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num-streams 2 2>&1 > $base_dir/kafka_mirrormaker_2.log & + pid_mirrormaker_2=$! + + begin_timer + + start_producer whitetopic01 localhost:2181 + start_producer whitetopic01 localhost:2182 + info "waiting for whitetopic01 producers to finish producing ..." + wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 kafka://localhost:9092 whitetopic01 0 kafka://localhost:9093 whitetopic01 0 + + start_producer whitetopic02 localhost:2181 + start_producer whitetopic03 localhost:2181 + start_producer whitetopic04 localhost:2182 + info "waiting for whitetopic02,whitetopic03,whitetopic04 producers to finish producing ..." + wait_partition_done kafka://localhost:9090 whitetopic02 0 kafka://localhost:9091 whitetopic02 0 kafka://localhost:9090 whitetopic03 0 kafka://localhost:9091 whitetopic03 0 kafka://localhost:9092 whitetopic04 0 kafka://localhost:9093 whitetopic04 0 + + start_producer blacktopic01 localhost:2182 + info "waiting for blacktopic01 producer to finish producing ..." + wait_partition_done kafka://localhost:9092 blacktopic01 0 kafka://localhost:9093 blacktopic01 0 + + info "waiting for consumer to finish consuming ..." + + wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 kafka://localhost:9094 whitetopic02 0 kafka://localhost:9095 whitetopic02 0 kafka://localhost:9094 whitetopic03 0 kafka://localhost:9095 whitetopic03 0 kafka://localhost:9094 whitetopic04 0 kafka://localhost:9095 whitetopic04 0 + + end_timer + info "embedded consumer took $((t_end - t_begin)) seconds" + + sleep 2 + + # if [[ -d /tmp/kafka-target-1-1-logs/blacktopic01 || /tmp/kafka-target-1-2-logs/blacktopic01 ]]; then + # echo "blacktopic01 found on target cluster" + # result=1 + # else + # cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04 + # result=$? + # fi + + cmp_logs blacktopic01 + + cmp_logs whitetopic01 && cmp_logs whitetopic02 && cmp_logs whitetopic03 && cmp_logs whitetopic04 + result=$? + + return $result +} + +test_blacklists() { + info "### Testing blacklists" + snapshot_prefix="blacklist-test" + cleanup + start_zk + start_source_servers + start_target_servers + sleep 4 + + info "starting mirror maker" + $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer-config $base_dir/config/blacklisttest.consumer.properties --producer-config $base_dir/config/mirror_producer.properties --blacklist="black.*" --num-streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & + pid_mirrormaker_1=$! + + start_producer blacktopic01 localhost:2181 + start_producer blacktopic02 localhost:2181 + info "waiting for producer to finish producing blacktopic01,blacktopic02 ..." + wait_partition_done kafka://localhost:9090 blacktopic01 0 kafka://localhost:9091 blacktopic01 0 kafka://localhost:9090 blacktopic02 0 kafka://localhost:9091 blacktopic02 0 + + begin_timer + + start_producer whitetopic01 localhost:2181 + info "waiting for producer to finish producing whitetopic01 ..." + wait_partition_done kafka://localhost:9090 whitetopic01 0 kafka://localhost:9091 whitetopic01 0 + + info "waiting for consumer to finish consuming ..." + wait_partition_done kafka://localhost:9094 whitetopic01 0 kafka://localhost:9095 whitetopic01 0 + + end_timer + + info "embedded consumer took $((t_end - t_begin)) seconds" + + sleep 2 + + cmp_logs blacktopic01 || cmp_logs blacktopic02 + if [ $? -eq 0 ]; then + return 1 + fi + + cmp_logs whitetopic01 + return $? +} + +# main test begins + +echo "Test-$test_start_time" + +# Ctrl-c trap. Catches INT signal +trap "shutdown_servers; exit 0" INT + +test_whitelists +result=$? + +process_test_result $result $action_on_fail + +shutdown_servers + +sleep 2 + +test_blacklists +result=$? + +process_test_result $result $action_on_fail + +shutdown_servers + +exit $result + diff --git a/system_test/embedded_consumer/config/blacklisttest.consumer.properties b/system_test/mirror_maker/config/blacklisttest.consumer.properties similarity index 96% rename from system_test/embedded_consumer/config/blacklisttest.consumer.properties rename to system_test/mirror_maker/config/blacklisttest.consumer.properties index 2d51badf9ea98..6ea85ecb9a112 100644 --- a/system_test/embedded_consumer/config/blacklisttest.consumer.properties +++ b/system_test/mirror_maker/config/blacklisttest.consumer.properties @@ -24,6 +24,5 @@ zk.connectiontimeout.ms=1000000 #consumer group id groupid=group1 - -mirror.topics.blacklist=test02,test03 +shallowiterator.enable=true diff --git a/system_test/embedded_consumer/config/mirror_producer.properties b/system_test/mirror_maker/config/mirror_producer.properties similarity index 90% rename from system_test/embedded_consumer/config/mirror_producer.properties rename to system_test/mirror_maker/config/mirror_producer.properties index 9ea68d059c42c..5940c242d3f72 100644 --- a/system_test/embedded_consumer/config/mirror_producer.properties +++ b/system_test/mirror_maker/config/mirror_producer.properties @@ -15,10 +15,11 @@ # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 +zk.connect=localhost:2183 +# broker.list=1:localhost:9094,2:localhost:9095 # timeout in ms for connecting to zookeeper -zk.connectiontimeout.ms=1000000 +# zk.connectiontimeout.ms=1000000 producer.type=async diff --git a/system_test/embedded_consumer/config/server_source1.properties b/system_test/mirror_maker/config/server_source_1_1.properties similarity index 98% rename from system_test/embedded_consumer/config/server_source1.properties rename to system_test/mirror_maker/config/server_source_1_1.properties index 11bfe1d78dd1b..d89c4fbe0e819 100644 --- a/system_test/embedded_consumer/config/server_source1.properties +++ b/system_test/mirror_maker/config/server_source_1_1.properties @@ -26,13 +26,13 @@ brokerid=1 num.partitions=1 # the port the socket server runs on -port=9092 +port=9090 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 # the directory in which to store log files -log.dir=/tmp/kafka-source1-logs +log.dir=/tmp/kafka-source-1-1-logs # the send buffer used by the socket server socket.send.buffer=1048576 diff --git a/system_test/embedded_consumer/config/server_source2.properties b/system_test/mirror_maker/config/server_source_1_2.properties similarity index 98% rename from system_test/embedded_consumer/config/server_source2.properties rename to system_test/mirror_maker/config/server_source_1_2.properties index 24991bb674031..063d68b0113c9 100644 --- a/system_test/embedded_consumer/config/server_source2.properties +++ b/system_test/mirror_maker/config/server_source_1_2.properties @@ -32,7 +32,7 @@ port=9091 num.threads=8 # the directory in which to store log files -log.dir=/tmp/kafka-source2-logs +log.dir=/tmp/kafka-source-1-2-logs # the send buffer used by the socket server socket.send.buffer=1048576 diff --git a/system_test/embedded_consumer/config/server_source3.properties b/system_test/mirror_maker/config/server_source_2_1.properties similarity index 96% rename from system_test/embedded_consumer/config/server_source3.properties rename to system_test/mirror_maker/config/server_source_2_1.properties index 02c2cde885458..998b460b944af 100644 --- a/system_test/embedded_consumer/config/server_source3.properties +++ b/system_test/mirror_maker/config/server_source_2_1.properties @@ -15,7 +15,7 @@ # see kafka.server.KafkaConfig for additional details and defaults # the id of the broker -brokerid=3 +brokerid=1 # hostname of broker. If not set, will pick up from the value returned # from getLocalHost. If there are multiple interfaces getLocalHost @@ -26,13 +26,13 @@ brokerid=3 num.partitions=1 # the port the socket server runs on -port=9090 +port=9092 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 # the directory in which to store log files -log.dir=/tmp/kafka-source3-logs +log.dir=/tmp/kafka-source-2-1-logs # the send buffer used by the socket server socket.send.buffer=1048576 @@ -60,7 +60,7 @@ enable.zookeeper=true # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2181 +zk.connect=localhost:2182 # timeout in ms for connecting to zookeeper zk.connectiontimeout.ms=1000000 diff --git a/system_test/mirror_maker/config/server_source_2_2.properties b/system_test/mirror_maker/config/server_source_2_2.properties new file mode 100644 index 0000000000000..81427aee9264a --- /dev/null +++ b/system_test/mirror_maker/config/server_source_2_2.properties @@ -0,0 +1,76 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults + +# the id of the broker +brokerid=2 + +# hostname of broker. If not set, will pick up from the value returned +# from getLocalHost. If there are multiple interfaces getLocalHost +# may not be what you want. +# hostname= + +# number of logical partitions on this broker +num.partitions=1 + +# the port the socket server runs on +port=9093 + +# the number of processor threads the socket server uses. Defaults to the number of cores on the machine +num.threads=8 + +# the directory in which to store log files +log.dir=/tmp/kafka-source-2-2-logs + +# the send buffer used by the socket server +socket.send.buffer=1048576 + +# the receive buffer used by the socket server +socket.receive.buffer=1048576 + +# the maximum size of a log segment +log.file.size=536870912 + +# the interval between running cleanup on the logs +log.cleanup.interval.mins=1 + +# the minimum age of a log file to eligible for deletion +log.retention.hours=168 + +#the number of messages to accept without flushing the log to disk +log.flush.interval=600 + +#set the following properties to use zookeeper + +# enable connecting to zookeeper +enable.zookeeper=true + +# zk connection string +# comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" +zk.connect=localhost:2182 + +# timeout in ms for connecting to zookeeper +zk.connectiontimeout.ms=1000000 + +# time based topic flush intervals in ms +#topic.flush.intervals.ms=topic:1000 + +# default time based flush interval in ms +log.default.flush.interval.ms=1000 + +# time based topic flasher time rate in ms +log.default.flush.scheduler.interval.ms=1000 + diff --git a/system_test/embedded_consumer/config/server_target1.properties b/system_test/mirror_maker/config/server_target_1_1.properties similarity index 97% rename from system_test/embedded_consumer/config/server_target1.properties rename to system_test/mirror_maker/config/server_target_1_1.properties index 72da00205077d..0265f4efc4086 100644 --- a/system_test/embedded_consumer/config/server_target1.properties +++ b/system_test/mirror_maker/config/server_target_1_1.properties @@ -26,13 +26,13 @@ brokerid=1 num.partitions=1 # the port the socket server runs on -port=9093 +port=9094 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 # the directory in which to store log files -log.dir=/tmp/kafka-target1-logs +log.dir=/tmp/kafka-target-1-1-logs # the send buffer used by the socket server socket.send.buffer=1048576 @@ -60,7 +60,7 @@ enable.zookeeper=true # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 +zk.connect=localhost:2183 # timeout in ms for connecting to zookeeper zk.connectiontimeout.ms=1000000 diff --git a/system_test/embedded_consumer/config/server_target2.properties b/system_test/mirror_maker/config/server_target_1_2.properties similarity index 97% rename from system_test/embedded_consumer/config/server_target2.properties rename to system_test/mirror_maker/config/server_target_1_2.properties index 96c52955781b5..a31e9ca1762b5 100644 --- a/system_test/embedded_consumer/config/server_target2.properties +++ b/system_test/mirror_maker/config/server_target_1_2.properties @@ -26,13 +26,13 @@ brokerid=2 num.partitions=1 # the port the socket server runs on -port=9094 +port=9095 # the number of processor threads the socket server uses. Defaults to the number of cores on the machine num.threads=8 # the directory in which to store log files -log.dir=/tmp/kafka-target2-logs +log.dir=/tmp/kafka-target-1-2-logs # the send buffer used by the socket server socket.send.buffer=1048576 @@ -60,7 +60,7 @@ enable.zookeeper=true # zk connection string # comma separated host:port pairs, each corresponding to a zk # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" -zk.connect=localhost:2182 +zk.connect=localhost:2183 # timeout in ms for connecting to zookeeper zk.connectiontimeout.ms=1000000 diff --git a/system_test/embedded_consumer/config/whitelisttest.consumer.properties b/system_test/mirror_maker/config/whitelisttest_1.consumer.properties similarity index 97% rename from system_test/embedded_consumer/config/whitelisttest.consumer.properties rename to system_test/mirror_maker/config/whitelisttest_1.consumer.properties index 5ff54badc1bb0..6ea85ecb9a112 100644 --- a/system_test/embedded_consumer/config/whitelisttest.consumer.properties +++ b/system_test/mirror_maker/config/whitelisttest_1.consumer.properties @@ -24,6 +24,5 @@ zk.connectiontimeout.ms=1000000 #consumer group id groupid=group1 - -mirror.topics.whitelist=test01 +shallowiterator.enable=true diff --git a/system_test/mirror_maker/config/whitelisttest_2.consumer.properties b/system_test/mirror_maker/config/whitelisttest_2.consumer.properties new file mode 100644 index 0000000000000..e11112fdc006c --- /dev/null +++ b/system_test/mirror_maker/config/whitelisttest_2.consumer.properties @@ -0,0 +1,28 @@ +# 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. +# see kafka.consumer.ConsumerConfig for more details + +# zk connection string +# comma separated host:port pairs, each corresponding to a zk +# server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" +zk.connect=localhost:2182 + +# timeout in ms for connecting to zookeeper +zk.connectiontimeout.ms=1000000 + +#consumer group id +groupid=group1 +shallowiterator.enable=true + diff --git a/system_test/embedded_consumer/config/zookeeper_source.properties b/system_test/mirror_maker/config/zookeeper_source_1.properties similarity index 96% rename from system_test/embedded_consumer/config/zookeeper_source.properties rename to system_test/mirror_maker/config/zookeeper_source_1.properties index 76b02a2682761..f85179616ebba 100644 --- a/system_test/embedded_consumer/config/zookeeper_source.properties +++ b/system_test/mirror_maker/config/zookeeper_source_1.properties @@ -13,6 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # the directory where the snapshot is stored. -dataDir=/tmp/zookeeper_source +dataDir=/tmp/zookeeper_source-1 # the port at which the clients will connect clientPort=2181 diff --git a/system_test/embedded_consumer/config/consumer.properties b/system_test/mirror_maker/config/zookeeper_source_2.properties similarity index 84% rename from system_test/embedded_consumer/config/consumer.properties rename to system_test/mirror_maker/config/zookeeper_source_2.properties index 8bcc48e82d1fa..d534d180daf88 100644 --- a/system_test/embedded_consumer/config/consumer.properties +++ b/system_test/mirror_maker/config/zookeeper_source_2.properties @@ -12,3 +12,7 @@ # 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. +# the directory where the snapshot is stored. +dataDir=/tmp/zookeeper_source-2 +# the port at which the clients will connect +clientPort=2182 diff --git a/system_test/embedded_consumer/config/zookeeper_target.properties b/system_test/mirror_maker/config/zookeeper_target.properties similarity index 98% rename from system_test/embedded_consumer/config/zookeeper_target.properties rename to system_test/mirror_maker/config/zookeeper_target.properties index 28561d95898e5..55a7eb189d642 100644 --- a/system_test/embedded_consumer/config/zookeeper_target.properties +++ b/system_test/mirror_maker/config/zookeeper_target.properties @@ -15,4 +15,4 @@ # the directory where the snapshot is stored. dataDir=/tmp/zookeeper_target # the port at which the clients will connect -clientPort=2182 +clientPort=2183 From d97c557202e7f20001d2597d09bcbf0bfc123fa9 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Sun, 8 Apr 2012 02:00:19 +0000 Subject: [PATCH 040/151] KAFKA-320 testZKSendWithDeadBroker fails intermittently due to ZKNodeExistsException; patched by nehanarkhede; reviewed by junrao and prashanth menon git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1310937 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/server/KafkaServer.scala | 8 +++++--- core/src/test/resources/log4j.properties | 4 ++-- .../test/scala/unit/kafka/log/LogOffsetTest.scala | 2 -- .../scala/unit/kafka/producer/ProducerTest.scala | 13 +++++++++++-- .../unit/kafka/server/ServerShutdownTest.scala | 2 ++ .../scala/unit/kafka/zk/EmbeddedZookeeper.scala | 5 +---- .../scala/unit/kafka/zk/ZKLoadBalanceTest.scala | 10 +++++----- .../scala/unit/kafka/zk/ZooKeeperTestHarness.scala | 6 ++++++ 8 files changed, 32 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index c6f68ccc88720..b2abb08d19c5e 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -30,9 +30,9 @@ import java.io.File */ class KafkaServer(val config: KafkaConfig) extends Logging { val CLEAN_SHUTDOWN_FILE = ".kafka_cleanshutdown" - private val isShuttingDown = new AtomicBoolean(false) - - private val shutdownLatch = new CountDownLatch(1) + private var isShuttingDown = new AtomicBoolean(false) + private var shutdownLatch = new CountDownLatch(1) + private val statsMBeanName = "kafka:type=kafka.SocketServerStats" var socketServer: SocketServer = null @@ -47,6 +47,8 @@ class KafkaServer(val config: KafkaConfig) extends Logging { */ def startup() { info("Starting Kafka server...") + isShuttingDown = new AtomicBoolean(false) + shutdownLatch = new CountDownLatch(1) var needRecovery = true val cleanShutDownFile = new File(new File(config.logDir), CLEAN_SHUTDOWN_FILE) if (cleanShutDownFile.exists) { diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 767fe558017e3..fd6697792c7e9 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -4,9 +4,9 @@ # 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. diff --git a/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala index e173c1fb1e5ed..3ba26d349812c 100644 --- a/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogOffsetTest.scala @@ -42,8 +42,6 @@ class LogOffsetTest extends JUnitSuite { val brokerPort: Int = 9099 var simpleConsumer: SimpleConsumer = null - private val logger = Logger.getLogger(classOf[LogOffsetTest]) - @Before def setUp() { val config: Properties = createBrokerConfig(1, brokerPort) diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index c1e47b91b590e..53e920c910e67 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -187,6 +187,8 @@ class ProducerTest extends JUnitSuite { Assert.fail("Should fail with InvalidPartitionException") }catch { case e: InvalidPartitionException => // expected, do nothing + }finally { + richProducer.close() } } @@ -202,17 +204,22 @@ class ProducerTest extends JUnitSuite { fail("Should fail with ClassCastException due to incompatible Encoder") } catch { case e: ClassCastException => + }finally { + stringProducer1.close() } props.put("serializer.class", "kafka.serializer.StringEncoder") val stringProducer2 = new Producer[String, String](new ProducerConfig(props)) stringProducer2.send(new ProducerData[String, String](topic, "test", Array("test"))) + stringProducer2.close() val messageProducer1 = new Producer[String, Message](config) try { messageProducer1.send(new ProducerData[String, Message](topic, "test", Array(new Message("test".getBytes)))) } catch { case e: ClassCastException => fail("Should not fail with ClassCastException due to default Encoder") + }finally { + messageProducer1.close() } } @@ -423,8 +430,9 @@ class ProducerTest extends JUnitSuite { Assert.assertEquals(new Message("test1".getBytes), messageSet2.next.message) } catch { case e: Exception => fail("Not expected", e) + }finally { + producer.close } - producer.close } @Test @@ -459,8 +467,9 @@ class ProducerTest extends JUnitSuite { Assert.assertEquals(new Message("test1".getBytes), messageSet1.next.message) } catch { case e: Exception => fail("Not expected") + }finally { + producer.close } - producer.close } @Test diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index f6806762a6006..5b0aefed65672 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -66,6 +66,7 @@ class ServerShutdownTest extends JUnitSuite { server.shutdown() val cleanShutDownFile = new File(new File(config.logDir), server.CLEAN_SHUTDOWN_FILE) assertTrue(cleanShutDownFile.exists) + producer.close() } @@ -103,6 +104,7 @@ class ServerShutdownTest extends JUnitSuite { server.shutdown() Utils.rm(server.config.logDir) + producer.close() } } diff --git a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala index 82bd1ef04d4c5..44eb4929f8fa7 100644 --- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala +++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala @@ -20,9 +20,8 @@ package kafka.zk import org.apache.zookeeper.server.ZooKeeperServer import org.apache.zookeeper.server.NIOServerCnxn import kafka.utils.TestUtils -import org.I0Itec.zkclient.ZkClient import java.net.InetSocketAddress -import kafka.utils.{Utils, ZKStringSerializer} +import kafka.utils.Utils class EmbeddedZookeeper(val connectString: String) { val snapshotDir = TestUtils.tempDir() @@ -31,8 +30,6 @@ class EmbeddedZookeeper(val connectString: String) { val port = connectString.split(":")(1).toInt val factory = new NIOServerCnxn.Factory(new InetSocketAddress("127.0.0.1", port)) factory.startup(zookeeper) - val client = new ZkClient(connectString) - client.setZkSerializer(ZKStringSerializer) def shutdown() { factory.shutdown() diff --git a/core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala b/core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala index 46295054f6785..a2303dadbe228 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala @@ -40,7 +40,7 @@ class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness { def testLoadBalance() { // create the first partition - ZkUtils.setupPartition(zookeeper.client, 400, "broker1", 1111, "topic1", 1) + ZkUtils.setupPartition(zkClient, 400, "broker1", 1111, "topic1", 1) // add the first consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, firstConsumer)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, false) @@ -74,7 +74,7 @@ class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness { (300, "broker3", 1111, "topic1", 2) ) for ((brokerID, host, port, topic, nParts) <- brokers) - ZkUtils.setupPartition(zookeeper.client, brokerID, host, port, topic, nParts) + ZkUtils.setupPartition(zkClient, brokerID, host, port, topic, nParts) // wait a bit to make sure rebalancing logic is triggered @@ -91,7 +91,7 @@ class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness { { // now delete a partition - ZkUtils.deletePartition(zookeeper.client, 400, "topic1") + ZkUtils.deletePartition(zkClient, 400, "topic1") // wait a bit to make sure rebalancing logic is triggered Thread.sleep(500) @@ -110,11 +110,11 @@ class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness { private def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { import scala.collection.JavaConversions - val children = zookeeper.client.getChildren(path) + val children = zkClient.getChildren(path) Collections.sort(children) val childrenAsSeq : Seq[java.lang.String] = JavaConversions.asBuffer(children) childrenAsSeq.map(partition => - (partition, zookeeper.client.readData(path + "/" + partition).asInstanceOf[String])) + (partition, zkClient.readData(path + "/" + partition).asInstanceOf[String])) } private def checkSetEqual(actual : Seq[Tuple2[String,String]], expected : Seq[Tuple2[String,String]]) { diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index 71ead0049dc46..c85640d844d47 100644 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -18,18 +18,24 @@ package kafka.zk import org.scalatest.junit.JUnit3Suite +import org.I0Itec.zkclient.ZkClient +import kafka.utils.ZKStringSerializer trait ZooKeeperTestHarness extends JUnit3Suite { val zkConnect: String var zookeeper: EmbeddedZookeeper = null + var zkClient: ZkClient = null override def setUp() { zookeeper = new EmbeddedZookeeper(zkConnect) + zkClient = new ZkClient(zookeeper.connectString) + zkClient.setZkSerializer(ZKStringSerializer) super.setUp } override def tearDown() { super.tearDown + zkClient.close() zookeeper.shutdown() } From d6b1de35f6b9cd5370c7812790fea8e61618f461 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 25 Apr 2012 02:24:47 +0000 Subject: [PATCH 041/151] Mirroring should use multiple producers; add producer retries to DefaultEventHandler; patched by Joel Koshy; reviewed by Jun Rao; KAFKA-332 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1330083 13f79535-47bb-0310-9956-ffa450edef68 --- .../scala/kafka/producer/ProducerConfig.scala | 11 ++++++++ .../producer/async/DefaultEventHandler.scala | 22 +++++++++++++--- .../producer/async/ProducerSendThread.scala | 2 +- .../main/scala/kafka/tools/MirrorMaker.scala | 25 +++++++++++++------ core/src/main/scala/kafka/utils/Utils.scala | 11 ++++++++ .../scala/unit/kafka/utils/UtilsTest.scala | 20 +++++++++++++++ system_test/mirror_maker/bin/run-test.sh | 6 ++--- .../config/mirror_producer.properties | 2 ++ 8 files changed, 84 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index fa989c88d89d3..8a5b53cb0ac25 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -32,6 +32,17 @@ class ProducerConfig(val props: Properties) extends ZKConfig(props) if(Utils.propertyExists(brokerList) && Utils.getString(props, "partitioner.class", null) != null) throw new InvalidConfigException("partitioner.class cannot be used when broker.list is set") + /** + * If DefaultEventHandler is used, this specifies the number of times to + * retry if an error is encountered during send. Currently, it is only + * appropriate when broker.list points to a VIP. If the zk.connect option + * is used instead, this will not have any effect because with the zk-based + * producer, brokers are not re-selected upon retry. So retries would go to + * the same (potentially still down) broker. (KAFKA-253 will help address + * this.) + */ + val numRetries = Utils.getInt(props, "num.retries", 0) + /** If both broker.list and zk.connect options are specified, throw an exception */ if(Utils.propertyExists(brokerList) && Utils.propertyExists(zkConnect)) throw new InvalidConfigException("only one of broker.list and zk.connect can be specified") diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index 0bf3c3c1e30d5..f72eed1814763 100644 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -47,9 +47,25 @@ private[kafka] class DefaultEventHandler[T](val config: ProducerConfig, private def send(messagesPerTopic: Map[(String, Int), ByteBufferMessageSet], syncProducer: SyncProducer) { if(messagesPerTopic.size > 0) { val requests = messagesPerTopic.map(f => new ProducerRequest(f._1._1, f._1._2, f._2)).toArray - syncProducer.multiSend(requests) - trace("kafka producer sent messages for topics %s to broker %s:%d" - .format(messagesPerTopic, syncProducer.config.host, syncProducer.config.port)) + + val maxAttempts = config.numRetries + 1 + var attemptsRemaining = maxAttempts + var sent = false + + while (attemptsRemaining > 0 && !sent) { + attemptsRemaining -= 1 + try { + syncProducer.multiSend(requests) + trace("kafka producer sent messages for topics %s to broker %s:%d (on attempt %d)" + .format(messagesPerTopic, syncProducer.config.host, syncProducer.config.port, maxAttempts - attemptsRemaining)) + sent = true + } + catch { + case e => warn("Error sending messages, %d attempts remaining".format(attemptsRemaining)) + if (attemptsRemaining == 0) + throw e + } + } } } diff --git a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala index 0c4a4edbc16e9..91c2fad2bcb53 100644 --- a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala +++ b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala @@ -115,7 +115,7 @@ private[async] class ProducerSendThread[T](val threadName: String, if(events.size > 0) handler.handle(events, underlyingProducer, serializer) }catch { - case e: Exception => error("Error in handling batch of " + events.size + " events", e) + case e => error("Error in handling batch of " + events.size + " events", e) } } diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 98dd65df5e770..3438f2c910b36 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -33,20 +33,27 @@ object MirrorMaker extends Logging { info ("Starting mirror maker") val parser = new OptionParser - val consumerConfigOpt = parser.accepts("consumer-config", + val consumerConfigOpt = parser.accepts("consumer.config", "Consumer config to consume from a source cluster. " + "You may specify multiple of these.") .withRequiredArg() .describedAs("config file") .ofType(classOf[String]) - val producerConfigOpt = parser.accepts("producer-config", + val producerConfigOpt = parser.accepts("producer.config", "Embedded producer config.") .withRequiredArg() .describedAs("config file") .ofType(classOf[String]) + + val numProducersOpt = parser.accepts("num.producers", + "Number of producer instances") + .withRequiredArg() + .describedAs("Number of producers") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) - val numStreamsOpt = parser.accepts("num-streams", + val numStreamsOpt = parser.accepts("num.streams", "Number of consumption streams.") .withRequiredArg() .describedAs("Number of threads") @@ -83,11 +90,11 @@ object MirrorMaker extends Logging { val numStreams = options.valueOf(numStreamsOpt) - val producer = { + val producers = (1 to options.valueOf(numProducersOpt).intValue()).map(_ => { val config = new ProducerConfig( Utils.loadProps(options.valueOf(producerConfigOpt))) new Producer[Null, Message](config) - } + }) val threads = { val connectors = options.valuesOf(consumerConfigOpt).toList @@ -97,7 +104,7 @@ object MirrorMaker extends Logging { Runtime.getRuntime.addShutdownHook(new Thread() { override def run() { connectors.foreach(_.shutdown()) - producer.close() + producers.foreach(_.close()) } }) @@ -110,7 +117,7 @@ object MirrorMaker extends Logging { connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams.intValue())) streams.flatten.zipWithIndex.map(streamAndIndex => { - new MirrorMakerThread(streamAndIndex._1, producer, streamAndIndex._2) + new MirrorMakerThread(streamAndIndex._1, producers, streamAndIndex._2) }) } @@ -120,18 +127,20 @@ object MirrorMaker extends Logging { } class MirrorMakerThread(stream: KafkaStream[Message], - producer: Producer[Null, Message], + producers: Seq[Producer[Null, Message]], threadId: Int) extends Thread with Logging { private val shutdownLatch = new CountDownLatch(1) private val threadName = "mirrormaker-" + threadId + private val producerSelector = Utils.circularIterator(producers) this.setName(threadName) override def run() { try { for (msgAndMetadata <- stream) { + val producer = producerSelector.next() val pd = new ProducerData[Null, Message]( msgAndMetadata.topic, msgAndMetadata.message) producer.send(pd) diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index a3c27010be3c3..eb72b0fb1dd1d 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -668,6 +668,17 @@ object Utils extends Logging { } } } + + /** + * Create a circular (looping) iterator over a collection. + * @param coll An iterable over the underlying collection. + * @return A circular iterator over the collection. + */ + def circularIterator[T](coll: Iterable[T]) = { + val stream: Stream[T] = + for (forever <- Stream.continually(1); t <- coll) yield t + stream.iterator + } } class SnapshotStats(private val monitorDurationNs: Long = 600L * 1000L * 1000L * 1000L) { diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala index 218e2298536c4..771432e1990a8 100644 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -20,6 +20,7 @@ package kafka.utils import org.apache.log4j.Logger import org.scalatest.junit.JUnitSuite import org.junit.Test +import org.junit.Assert._ class UtilsTest extends JUnitSuite { @@ -31,4 +32,23 @@ class UtilsTest extends JUnitSuite { Utils.swallow(logger.info, throw new IllegalStateException("test")) } + @Test + def testCircularIterator() { + val l = List(1, 2) + val itl = Utils.circularIterator(l) + assertEquals(1, itl.next()) + assertEquals(2, itl.next()) + assertEquals(1, itl.next()) + assertEquals(2, itl.next()) + assertFalse(itl.hasDefiniteSize) + + val s = Set(1, 2) + val its = Utils.circularIterator(s) + assertEquals(1, its.next()) + assertEquals(2, its.next()) + assertEquals(1, its.next()) + assertEquals(2, its.next()) + assertEquals(1, its.next()) + } + } diff --git a/system_test/mirror_maker/bin/run-test.sh b/system_test/mirror_maker/bin/run-test.sh index bdc3f37d8e836..e4bbd81619290 100644 --- a/system_test/mirror_maker/bin/run-test.sh +++ b/system_test/mirror_maker/bin/run-test.sh @@ -241,9 +241,9 @@ test_whitelists() { sleep 4 info "starting mirror makers" - JMX_PORT=7777 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer-config $base_dir/config/whitelisttest_1.consumer.properties --consumer-config $base_dir/config/whitelisttest_2.consumer.properties --producer-config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num-streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & + JMX_PORT=7777 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & pid_mirrormaker_1=$! - JMX_PORT=8888 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer-config $base_dir/config/whitelisttest_1.consumer.properties --consumer-config $base_dir/config/whitelisttest_2.consumer.properties --producer-config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num-streams 2 2>&1 > $base_dir/kafka_mirrormaker_2.log & + JMX_PORT=8888 $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/whitelisttest_1.consumer.properties --consumer.config $base_dir/config/whitelisttest_2.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --whitelist="white.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_2.log & pid_mirrormaker_2=$! begin_timer @@ -298,7 +298,7 @@ test_blacklists() { sleep 4 info "starting mirror maker" - $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer-config $base_dir/config/blacklisttest.consumer.properties --producer-config $base_dir/config/mirror_producer.properties --blacklist="black.*" --num-streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & + $base_dir/../../bin/kafka-run-class.sh kafka.tools.MirrorMaker --consumer.config $base_dir/config/blacklisttest.consumer.properties --producer.config $base_dir/config/mirror_producer.properties --blacklist="black.*" --num.streams 2 2>&1 > $base_dir/kafka_mirrormaker_1.log & pid_mirrormaker_1=$! start_producer blacktopic01 localhost:2181 diff --git a/system_test/mirror_maker/config/mirror_producer.properties b/system_test/mirror_maker/config/mirror_producer.properties index 5940c242d3f72..b74c631d1d28e 100644 --- a/system_test/mirror_maker/config/mirror_producer.properties +++ b/system_test/mirror_maker/config/mirror_producer.properties @@ -26,3 +26,5 @@ producer.type=async # to avoid dropping events if the queue is full, wait indefinitely queue.enqueueTimeout.ms=-1 +num.producers.per.broker=2 + From 2a59ad76c657e4aad8ee6ca67078f49d2f6017c9 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Fri, 18 May 2012 01:22:11 +0000 Subject: [PATCH 042/151] Minor fix: Turning on TCP NODELAY in the Simple Consumer. This fix has a significant impact on single fetch request performance from a latency standpoint git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1339944 13f79535-47bb-0310-9956-ffa450edef68 --- core/src/main/scala/kafka/consumer/SimpleConsumer.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index b9b0ff416042c..3064fae0fa3a2 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -45,6 +45,7 @@ class SimpleConsumer(val host: String, channel.socket.setReceiveBufferSize(bufferSize) channel.socket.setSoTimeout(soTimeout) channel.socket.setKeepAlive(true) + channel.socket.setTcpNoDelay(true) channel.connect(address) trace("requested receive buffer size=" + bufferSize + " actual receive buffer size= " + channel.socket.getReceiveBufferSize) trace("soTimeout=" + soTimeout + " actual soTimeout= " + channel.socket.getSoTimeout) From f8ce071f3fa0efdad555a82aba9e024e3a7f4f56 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Thu, 14 Jun 2012 17:47:20 +0000 Subject: [PATCH 043/151] KAFKA-365 change copyright in NOTICE to current year, reviewed by Jun Rao git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1350349 13f79535-47bb-0310-9956-ffa450edef68 --- NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/NOTICE b/NOTICE index c1eb9b0bb8e47..9cf7aa102f89c 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Kafka -Copyright 2011 The Apache Software Foundation. +Copyright 2012 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 16b600d4fd972be697af95398f22f23aa8ad2608 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 19 Jun 2012 00:27:00 +0000 Subject: [PATCH 044/151] add jmx beans in broker to track # bytes in consumer; patched by Jun Rao; reviewed by Joel Koshy; KAFKA-336 git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1351544 13f79535-47bb-0310-9956-ffa450edef68 --- .../main/scala/kafka/consumer/ConsumerIterator.scala | 1 + .../main/scala/kafka/consumer/ConsumerTopicStat.scala | 10 ++++++++++ .../main/scala/kafka/consumer/PartitionTopicInfo.scala | 2 ++ 3 files changed, 13 insertions(+) diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index 9c6828c25d7c5..73e27943ca9bb 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -47,6 +47,7 @@ class ConsumerIterator[T](private val channel: BlockingQueue[FetchedDataChunk], val topic = currentTopicInfo.topic trace("Setting %s consumed offset to %d".format(topic, consumedOffset)) ConsumerTopicStat.getConsumerTopicStat(topic).recordMessagesPerTopic(1) + ConsumerTopicStat.getConsumerAllTopicStat().recordMessagesPerTopic(1) item } diff --git a/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala b/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala index a001a5e83ad8e..3a9de2a9915df 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerTopicStat.scala @@ -22,19 +22,29 @@ import kafka.utils.{Pool, Utils, threadsafe, Logging} trait ConsumerTopicStatMBean { def getMessagesPerTopic: Long + def getBytesPerTopic: Long } @threadsafe class ConsumerTopicStat extends ConsumerTopicStatMBean { private val numCumulatedMessagesPerTopic = new AtomicLong(0) + private val numCumulatedBytesPerTopic = new AtomicLong(0) def getMessagesPerTopic: Long = numCumulatedMessagesPerTopic.get def recordMessagesPerTopic(nMessages: Int) = numCumulatedMessagesPerTopic.getAndAdd(nMessages) + + def getBytesPerTopic: Long = numCumulatedBytesPerTopic.get + + def recordBytesPerTopic(nBytes: Long) = numCumulatedBytesPerTopic.getAndAdd(nBytes) } object ConsumerTopicStat extends Logging { private val stats = new Pool[String, ConsumerTopicStat] + private val allTopicStat = new ConsumerTopicStat + Utils.registerMBean(allTopicStat, "kafka:type=kafka.ConsumerAllTopicStat") + + def getConsumerAllTopicStat(): ConsumerTopicStat = allTopicStat def getConsumerTopicStat(topic: String): ConsumerTopicStat = { var stat = stats.get(topic) diff --git a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala b/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala index f64f2b0ea7208..2a4caa727932b 100644 --- a/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala +++ b/core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala @@ -61,6 +61,8 @@ private[consumer] class PartitionTopicInfo(val topic: String, chunkQueue.put(new FetchedDataChunk(messages, this, fetchOffset)) val newOffset = fetchedOffset.addAndGet(size) debug("updated fetch offset of ( %s ) to %d".format(this, newOffset)) + ConsumerTopicStat.getConsumerTopicStat(topic).recordBytesPerTopic(size) + ConsumerTopicStat.getConsumerAllTopicStat().recordBytesPerTopic(size) } size } From 1af23bb2633c162659ab6fa99020307960f41465 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Wed, 20 Jun 2012 15:04:04 +0000 Subject: [PATCH 045/151] KAFKA-368 use the pig core jar from maven instead of distributing it patch by Joe Stein reviewed by Jun Rao and Neha Narkhede git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/trunk@1352145 13f79535-47bb-0310-9956-ffa450edef68 --- .../hadoop-consumer/lib/pig-0.8.0-core.jar | Bin 2234753 -> 0 bytes .../hadoop-producer/lib/pig-0.8.0-core.jar | Bin 2234753 -> 0 bytes project/build/KafkaProject.scala | 6 ++++++ 3 files changed, 6 insertions(+) delete mode 100644 contrib/hadoop-consumer/lib/pig-0.8.0-core.jar delete mode 100644 contrib/hadoop-producer/lib/pig-0.8.0-core.jar diff --git a/contrib/hadoop-consumer/lib/pig-0.8.0-core.jar b/contrib/hadoop-consumer/lib/pig-0.8.0-core.jar deleted file mode 100644 index 34418e5e302190e3e6c6985adb4e0d97b918aebe..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2234753 zcmcG$WmMit@&*b72<{TxU4py2yZei~y9OtCaCZq1+}+(JxJ!`W?zv=UcV{M>&Fuf~ zz4^j9IbG8AR9973*YA~+d<6;x1Ox#D1gsX%4fLmfkU*e7(!xsoG~zNMZ=Z*NfaLzM z^kwbK(&NJ~x*IRcg@J*9P+tE2YpFE9jJSxfq7tpNh*dPdEC@YHz=`xdp8HHsLR}bn zD~;>BZbG~Fs){an6NXDxO@VvuH#DaDIdeCy$sZ8Xk56k?*w|TUldPN`>n*$J9o@xBi6?$&4RTZ9@4#; zE%dKImqlFR4-Y|-V43X(lPR`?!&J(E@uy3>+ZPVyvHFf zXWgmO;XWZ(Pt%&qnG#GdjSAxC@2=6;pmghxW4du*?3F%nSbuBltcrdc;?)J)x2^k6 zlK=sEs{Yk_pf4t|u{U}9$BOSp`;&Qrp#QM~U<)uXHTuOu_`fc+HTxA;^nYKh4=}JW zvNrsM1fu^;WC(BsI5^ta15AFUBH4dtVB~6K;N)m#V{K$@VrKnkCSE_74EH}XF$Eaf z*x3GQ2S3tM{w?i)w-MUE<@4X~=R4ZpUEm)8f5+{gQ2b**Kk~8y*eVzqIvE&A0o;u2 zf3aPO|CO~Zz~0g9H&pB${v_u|9~Jzchn9XuRY#q(4 z%-sKy3Vzh%CtmcM@_*6eCEPpw?_3;S>J+O#gZ85?)c?(k6o0S8e`E9iLY+TS$uHC)`FnNjoh*(1oW_5|;wOjtpO*P=c7pux z8rT~B%3pt?@^^xN);K@Qn?I`j=I1j1#_4|Is}2VCW-syPm&peI--Z1_X71)_`pYbU z`V(qX8v~19VEPjhM;jZZ z+%H-0m)Q#OuY3MGkNDw;{JvcOuk{YZZ#Q@Y+cGepmxjjMS3p2N!Ia;?z{$$#rK+{D zr~R$=Nmp2qctwxGZMQ$)bn*aBIE|kB)*3ZCM1Wt;@{BveF1%-iE;i&j1>XA|9?7bi za^8Iwc-JU*iOg+*LYC&m4-js-t~o^BBx&og znQxRSbQP}7{FUBMSB`>=&|eAmQSSEddv-bAkhRZuKU6G|sUc|na3FeO*Yp9MaYWbk>t%ZJf=Bv( zfxNPX0of|2jvjg1L&0PWAj`N^A}V}H27N~L5bM4J0+S>jtpo#fgZh|1FwJLy*a97ilnGRKkhk^v7sDj=We z4j-$1z@PgSF<8F2tN8k6L*MNkXRrC=4tY=v8~8>8dDT!NO$8$DJiReJs{6nXQ@cqg zLY`pC%x{966NR&RR@`{kOGNkBg6YHPNmEXbJrg`FEegW@22oTUuJeuFq#7N&B zU}o?S=;WkddQ%8qL<3!ZAt9k>5+OQTt+$9WJplm$LPYIZnnN6Uj)T^TJ?&pQ@z$dl zoI6c7#Z4K!(w$dRS)Xn_Jzm>JD8j$SO#$Uqa5kNop}!Ae`tD>YOLF%)!lW zDz2nVE`iKGtO8DLS;gyDC;a@bmT=~QLjgdjrt<|epn7*iEu$n`-F_%r#mhjSin^cl zYJxB>qe`xtcqXXt<2HX7lQPAIH42$LKAJ$=N4-vY{0sDqDy4G~mesQje%TOp&eYFU28P z4ka2EYgFI!&GWZG2@3!VcWfN9CfGaHXDqqMiS%Zl$auhE(VEky(B9aEnAiJpL{_Yo zdIJB|;IN;^_@rJ8UjAb6pOm43HkOt~295xIOCu>8fT4(!^*>FoFevegAHi!UMqGb+ zvkV;((%tToP(p?u1stpw_#|D%=y2jXV1e$%^6;m*J(%_Wu5>IYmKoX6I%w%4n7r%xJo;3^@Y<=PsczTq-pku6*t#_c3GUvFYg)v?&mw z#xj;n#VK|TDyIpxVN!aHKTim?(V|YX-nY)fYxRlinjLR*cUN55?bvXf15$CiY|L-VO8)K>N!Qb`{9lOzjO1O8;RDO9-u24gAa zmx|Ckl_S`{B9hdm1Tpx6hAKD^&`)Y%K^v==Twvy4WBmsh`O2EI$oweJK?$@p%6ZE| zLVQ#;J5{b*F_=hx1ZE1Az6wZd1xwjD%g5H&0*5~5y|3xdI!_Oy7%~=EP$ikb@SMMl zjNEwbj4a_$Q}o_YAl=}bssw7P@C7MNlEy>147_$39*??SAbA_yWFHs{6>;DxiiH43&X7Fn&|1D*=a{=oq=1rpx zTpI2Px9HGX1=(Y_wX(!q+;a(aG17Y?PFvW@x5_#F&m((_7`%lKq^l79sNgrk5rBKT z3eN=;)P3=(`C96w-60}W)2fdUO@pOPjyN&C@Ivv-g~SZvglRBc=xItLwF+bEgTIiLIE^{@d#hxcKvlm8gSdhKG4w)%E>u80hKD) z9`5iUKZX*RJOenXi3Lasa!;#J@| zLxW0thdf8M=||BqP}B*U&wgq)4Ms4VY)YAyxlIR^r_bSIcmZh`Trf8#obuc`Ze_U_ z0bhjgaijsytD;=QV|Nj+(Q321Es2N zdMKYUzO{}Tn^bdZL4qa>gP{l#s?UU`2${coHwi@l9)B@3cGQruiL-j;RY)D~XN|9= zt)QjcG~AZG{`i}~b;1^!M_y(pvvuZ2MOIcNPp!!dM9o+*n|Pen7p*txCpS1JgLAl^ zpdFR^b$(G`__o2Gp370P12<%^Zc%t}dQ5B!Jv~`MaBkE5#^?kdx zb|jt}HQLtur2|WK;y@ik>cJhyM?GwBd5f=-;1QiyYIeSYx(20(hNqmTa>I)ez`Nc; z1d9TvgMUCdyxF23d_?7KD_*zYr|$3titK9xy5TKbhvaReeJ^`|27C8hfVT#Fuc+dY=*Z?Fs*-V!1!d#e96j`+?J&a&R2OPD93I$cd)U;VCZ&^K*Qg3MzUxS+Yj07{iz!r^ZkmgK9CGf((PD%-tMLP*d zxvE9Ye6CC-7I8dJRk-gAffDHIlEa>Wvm3nWaV!`MNz^sV#q>Nu`MS}R)WUL zR9uE(|1m}lKv z6~2Gh{bgFQh}n$L#k}aTPGnzPu_&po@*S~@EK!{M`A5z8Dx_9%Zc1BE!Jx1+jFk_w+t1k>5-IE() zD79RaiFDW-l;BRD$`vC0s`8P{!Z*VNDyYq#Nn2yrSsqMa4=UNf76V1Sup=Y76#|#lbnFJjPNeZ>oxh@KqMl zQzM1qxK-^v_Vi%9Oo2J^C?v0YKyY6?wZDR3wS(VoR9&OJftBf33kunIBh>1`xuJrb zdrCo-3&_TU6>1l#5qtW1GizGGY`@*4)|G|lhRzygPU&fy2@YU)g`ykzMtztm!9nbO zCde#)n&Nts*Dx#L2cjH)uAh9eloC{%o71>-=wmuVu#z{3a8x*0T5BX|#xc);x%2gj z#^ldLxyGxS#CxG#Z`?>x|& zrWca$nyEg!sEaHvjXAI&I|sRN+Ia(=Ezuu|AQ{di=+ahV327|)ZCXEH10%8wy!yf| z-~58XJ+)=hes~!yQ*}8CQ#0KVx_O50#|>f@m431e!?&@T-LraXsK71c zbYzx!-94Ga;*yu8BVWTUgdE(G9E!?A@pY8wLH(7TMw~lz<_5(BiGZ()6xKkobRfMt z^qffS{D|*WFi7jk^>Zjm-w#qssex3(&!#Lib-zkDp3f-Y?_0N3W9;VWM#HVKtB{wH zpN!lI{iyr_)T7g{R)vi9vYMs-fD4F(=_w?3z{e+J9HdP3x9rHq_p_XVn{*rgL$10HQ_lDHLJ?|^BH&NC+O(QaURh(Jb<%coZ zjg&49J{vJ-(1mtnlaEqY_rF@ze*{mTVL)r+zNt6Mu-BEI5*LzNI|8N3Sjl}_v z3BGs;x9XG&x-D{&3%44nVkVDCKjLrOl*A2^D1jj=n@Vj`P=kHs`pG^c0WKa||3E4~ z=W`24Ubty_3#t&#*P~(W$$Hl0c!%Atpz|!H@L2skPNYE??ji{d@eE(C5mVBVUZcjH zi5Tfq0m05{Otf^)YoXR#*vt4n9}SXudvdvttBv^*N{#u$$v_TH!|YtmdX?bHAXK~I zs0_*^Tm{2ZrIm8ki-S|@4b{}RYw09-MScFdi1%#Q)`PO1CM|(PhHhTFL9k=60f>9Y z>h(>9x%DoJvZ6Ol8Eugn(iU@9R)kJoyF+l!$LjgkXbtS2LdBL>t6DjUon|3_iMmH{n+V z>B{DYuDxqR*Wa>sO%ah({<>oT`q?51!`B9$ZEV?>)WwTF5u1T5nR}hB65dxoBggB8 zaF5$(%L^isFJ3atML`~Q_FP@opP#W{Dw+6lf?>Kstv(w>3;Tc83_fkrZPKLQYMy2z0FriHPEp;?X-7l)#!?* zR!55k95WGm2hKB$K-06Q@@ut8-ZE?D>QU{h{Yg@-3{D~+_Oa5 zFjG;%rHFz|a8wKYbkoIcl$gCdyRv03>Eh57+0;bCxWC(`z~%HcWlb0(9bZbKGnb^S zJB2JE^9ZwO-+~*4f82&`U&FzNi>lNV)!593$%#v#$Q80Px8*Tr;cTmzXtz}41q0`; z0MR2Z2>g((6$sgyyZTsrtm2T%Dzz6h zbC%&~NQ)-OCpF4!BcIFGVROk9mQmO(nm3uwhilnU>MwyvU&^~Q@(TZC1&^s#HaAcj zun{)gmC>B~Se|9YJZqWcCU!Tcv}tXprIjSw!i?EhALGmW3Jk8ZMkOOgAJfua9aH31 zTVUWx%T(N~AK}I=UgBFU>WJ)Sd&U)!;4SS46)Scp5cyLR=~g(n@eHM20N8tEEerPoQZ)NyKVz6M$nx)W}GCUNjWLZ007PVV;5SQ8scmj^|)`b%6u4pT=9W)vAp z<5qJYcbmoCHx4ab0+iBxDAEW<6xQ`ViEL9E$iO6Db3{xc6#KD{*rBPm4qygkiI|)o zT7B*-H?X+z5eUXcX-?pMd*7U z;$`sWXZ8Ne6pE6YtPZ!YIfYBs(v|m|b86s7T1@CH3v8jWqm_=C zmW*vgLa)KMc*0WQrP_yPm|;el#qk;PW*7DX07Tp#Yxh zaL4o+jc*(lU=tzv3L>7qkG(IZMIO@@n1I?KaK2=Fgr!8Q zlg!R*WW0O#NMsYCN*A#qpaa=c5t5?nJR}H9?Pk-_%uuzJcSIHs-?&}@9U@~?9NjL} zf-FAG0Ue2DATmyE22wZw1mPfbul=24Wb|$T*!X+bNbJ)z)6>^(&jhfV+HVMR=3WU3 zT{%=#`@mA^3_1zzcMu;N#D22zyyZS;w|@RpEA0Ez_0A(V7R!tKeSC30y1#a(zt8;$ z|7)0pSisC0VDI+FfBd5xspw-dEr-HgE}hEKSNrwAIDMLZSE{Vf80kdPFW%Z&Kl^4i&E=T;Wc6k>DpR-p)qIx$n8Q5z zCc`mQE~H{&Hq`(i_nq_x7F3EUg&JQ8kRUz{#*LvRy2yN7=h0gX58|NWq1r? z%{oy`k~m8{jTBxWs%AeMQLXn0jzsMV_rO;M0)}-UYzj>+nfN_Uh^YmfsAwM$ied9q4TGSnc zDM612DT^hAW3;*Faq)nWKC&D(DU7zx43f{FK5~j`K;1yozGHWX!ML9W#rM9bOyo;^ z6=s;YH%t&D)K;A7w?B%=aQ_8Dy_MeYl6|aQ!udnS>j{#AKtgw=YlmR4n~2!Y>yP}-Mpu;ry4EA;Ys@sn4fKU|LZr8DbfW&OM7MJQ^^&hVjd zD-JojP<)~$q|P}vs`~UkVjNSTsA1$M8QjMg55DcL;2VELK)`FC zcf3QcCK@U^A$j{p>5L9z%MLd8Tf2iUKsUC{;E{KfDQX|}vFD`KE^WKJrNLFhd<{(< zbwd&fLjvIZSGbfpTE;u@$V<;*gBE7jmcDuj9rujC*E*r3sSoz|O1|##Hr&~43%M35 z9%%wydE_(7TYR6p!N+(TRg+*whNMmksk`52_9jH=65cTSBi%fQltQu*<*Wr*phWSu zogHuY=h=HVVUO&Q`cksDoTKK=XLLuy!9#KK-%35^PQ8@uhPz{;P(R(PF?I?D0``5b z=$O8_1pq!7X*l{0$V_>}%ke)l6$S~5~k^RY*ybkfHweo@d7 zj@Se(%|JMwaB1>_nSF-Zg)lhc6t!(f*y|{q$A>yi*~H3e=|yenCMKAJA}rVo9324* z&j2@;xa%nMJ-cRY$P|rP_B$yJ*Tp6`GIdTQ@?lHl;$*0k$K3L1C{n*9vg1j&>lCGR z-S5iA=}snUFL)hkWr*`Kd8@W+`6K>w-G-|qFEI$|B?kTEVfiokTRS-Z4u7EHlJztn z0(WXE@?4(TUSOPLs)Fg}6h~Z%QUHSdhiCyJF%(Fh=-5vRs(p2qg?_Rd(5~x&2`s|+ z1ZP8{1ja|8(bEaRY3b?Q9$!X2U0lq~od8*HdV}v)kQ0UJgG)A0)(s<-i8s&^1?po` z>C-@xL(3U`5{S1I?7|eZmt}oOBhX#?eB(GeQ<|Y-b8;TpJ{Wy$*N&PYn>a*9kEGlz zSs2UQy}9(nYIV$VRfn;+fU{z`Uq*1L&&sD5mOEYdFm!W~?Qc`g2mNwO!a&*nK zGReuQq zZ4_XG_lO=v#)z&*GN-k7n)yKHB3+OsU&MEL+5PO$skUv!lz}>(e)-fdsf>GS0w-8g zJE(T*7NWn9!_t$n3y*sk<54Rvg3zm7{k{t0X^#eC3q_Vd)-}T!vviZX2-nXm2S3QH zODXy>1Vey?SPw0b&lDFuI588MQ{-!$dw7!jyIVx0g9F6*XS{4Kq=1Kj*GU~F_O@?A z#)8m;a|n@+1JK>qvLC57L#9wSj$X%{$Ug?n3>%2uT>@czz~O`21SaL-gJYDaiFo_1 zMjk^Yd_&?LbOS;|IG96<|LwGiyqnVPpB@>$#~I&gMTXoLAA5Qk1pjrMk+uD!=88~S zQc+YzN$+1~A?P?GSDIPG&qC-4(V9ULzz3DWg0O$b@;L!lKVZT(=8M4R)sxcJ_G@I% zsrgwr&c%)D%n8{&QAfv8FK1L5oS0zlT`iBJ#%rS;uG@!IJRtB>!B@)-hoW$*o=I1L zl_ELK;(TXr&Br_I?$%;LG?jYlXZ}j z5E{lVt$f9OU6R_U5YSzt_8&kLLGb9F(_eCAy9VSw|=xlFK!KUAW z#elFTdGB=3VS4W0#ox`u6m_Q3(&$j_FoDL@rBn4otK`R&n6jl%Cd_rI3=Wm;cI7{6 zu-VlrjVV*!<*e*bV|>ORFlJ%z=aE8>t%fvXZ)b?tavrdBm+%i3+w}8yfP8I&hDw>P zz!7hYCNokLxDRHpD@8q0)-z23>n*%1rLkVDG>jQO6I5#y2%XT1#!|A=(kkL{;Y1^w zm*C2qj_%@sc9Nj{@occ!!`-E}1Lv(yfro|o&e5`#l>@V?PK^pQ%9(iKB`Pct_qP@G z6YG`;C3*|ggi`wqn)vdiIu^*54=w5CBFvYE>lw@0_^koWO&Nu`jYEwwLEwz-#+sis zl1DUL771ufmj+YGw79uK%_kwv=hvp2r?0}q>awocYpi$xb1X!%?Y`vfx1;){X`|A+ zcZW1l_A*tPKorkpW8kN-ZC4wdu$_g;;1%yqUa9qwz(TBpwTKl$UW2h;Dgc?#4>e^v z_GA;}7{I|IM{v}n?y)JpOh<*gj%?tJkfp_uPl}Ua@7OLwiiXIrZ$Jbik8tx3w@K+Q zCX*+;AE!g_bgh(Isu9mLIU=)CEl$wR7mFbpjnbuZGlCZ8yxES;Q)RVZpxmkdb^>em zX7zdXoQSST@hgvx9TN+mZXDxf#4?t}E;J|v5mY~l2x=r%I7ONqTM=EEtpq?|qoxZZ za2J&nqcRu|X#(C}F#0+N*g;INo2w`{;PGNE`^k^{W}``1q8luE(1CwQ_g&F>@L*JK z#KSt=6V$j@gUuk}5<0ctgJ=QZeb_WtEkcB4B@ntclwQC++*%;3;jMQN#MQLzLv}0y z<5@nL(Tx~xbx_^V5jo4Pcx!<-ZD2B{6(C3 zJjKwtiU+hs%~#-CNi+TV$CFPYoFl2FplFw!*&5b>su<00cA_a{_3MUXjLY)xgD#Eg|Bey_j zG6?+oIPM`qoYDwQiw&Z)8Y2-e;A)oOv0RWiBGOIy(Id04p4dd-J<~WygrXLMVbeOC zLRUM)AH`(dkqhnM{axP?Jc`piqhFo2a~lDa^%FRvpGlQEBbd{|YQMeuYdJM?8TC`mbl|XOrx6hxmV%^8oMw( zoE^E0l2+ey^a0gw7pDEA7(*XmeVJf>FQBz;mrB+m|D%6{%S*k zh+0NQWs*|TguW^PU@BXXcAUq!6)~J^5WlgrGgaJe?CP~I#W0UPW2DynF{Mr_N+DNN z;wb%d_$P|0q5!7x$pbI~BI{wgm`g6T{yFv&(yw|ty zi%$B1Deu1Ye`8CO8&`Ls*M(Ri(<;cGc9qF6+EmihOPR@+ zAyYGd^X*UHNBwpqoIvfld2yfmm#6rjWJwV-OCxb>TPMfAjz*OJ<2Am3hMGng1p!nD zny_vetp#6A2(gnd6v2Xy3vS5_qx#Ugt{N2STS$9{s+X@2(%bT9@~vc=8gMz^+bWaY zw7WKkmG6)7&mel(Z;`|Z4fG~D4==J+lLL9Gjh zT|kb-VMusC#m2LUNjV};>vKGFz{bcn%o$-|T6ss`H>m2JAPQ!tk#mU5q_xU`!o06> zuw2A#E!6BrOxuw4byK;5E6hSC`Wyi>m)Tg{B^-0#VQre;T%oRL##1)sOUqn)3-5 zstn-0R9l~~s6UVqPcCaFLD^&d4UsLihlGhSY1a}`FmY_Prcsdr9$L-{QSi%05sJz! zNXhpClxfLvno>pDvU$Zqi{F?cZYhwxEmLA$!!cgT#<+$q5N7~n3^Ez>p0Awf9I=}f zs+ZY})z6^)rpmNUVK4G1)U}F8P4zMWoBAt^6&-X=8$tA1(!1vQ<5apX^5tm$XdRTQ zsW9C8)8MUYEVaoZg%?pGm{dC|Y#56nUjoRQve6?p+n`()k-RVpL2L4e7{=8Jf+{a& zNoK=DG=^cMYjTN*%~JN=l4VH6MXIcA@eUPD+XTk%Cqh|blr(ClRp!=R z@*-lejfMr+5Z$8svTZ`BefXwr=Ri#$@qsD zgg6rI6=VinrxerSxqr67X%JZ@dLu&CB<38;B9iiT4-&%$Vbl0FLEx?UQ(>bt1CB9> zenb|0Yy&ewKM?N57~doMUn8$2zHaC!7!Z&*%unZvz8@V$TT3&?-^W?L&lmk81glsm zBCEdfW(lxFQUyaf6`)c^5S!4mYDK63i^GJ{kJzmJOtL^6Hf+6YDd*9?aLsz#dW6nX z!-$&fT(DKhG2K4-ZF1j1CILZUF(-N9B7=LkmFtB0VRxA?rb|trQ8G@-0#DO3NyCqLxmm7xJb_&_qP0EHHu-X=>*Jb@)9{0@X$O{AtUm zWLg}0MQTcb5?hp-IGDMaQuc0|C>o1NamQ1?d$? zny}@^iSkI(MT%w&K?sWZX};@^2uc;-6c(=EB*(QHCP-yOE!1u8dl##LW{2I2`e zQY^77LrT7lhY31T;%*CXpm#79fg)3=GihMwHVBWeE&!N>XN2bDL5c8sS`Stx7=p)Y zD#sZINZp@Sm`Jk~ZP48_VfPTg8d00=@s|gI9$UD+3&!N>5vAmrJ%!{kYMz~Y?}vkh zrPgh7o2INj&Zfgs`e`r3D%RTq)@Aw-{Wiw;_-jV|g zmXQ9D!l_~C# zdpKO%YgL3$)sRXkry7gKv0I8JTTGfa1-^z6XRF0$I(}Gr6tEtUXAMlx;H$)_ebafs zQ)6nLs2e6B5TaB|d*He*VzmbIL3Ra(0x1U{LsX=lHbG?+o+y%$>X$XP22lqOp~$O(avS*+l8%nHv~^Rf>Q=( zcoW!Kk-VY~)Ap8NM|M1GVTreljw4F%%>9XdveL1aVI+Feqw)hlsgM0FZaVqjd%nEA zmHB7tZvbO}3<(5wx(@(2&*%O9ZrzP7Xe+{wtD>l9ZwVER7}?=SO@;?o+_@FlY<@Zbn)i#<_!spPw%-(BIvt6#W}oR;VacGRIU+7Sz9_o zPI|kWLoix7fBDGvHTf{?K)r}*u}Gz^gaPhOnEqUF3^MG935)nZYd>3j%q<$b>WY+; zI?Xy%Ko{klAbWAcSFvp%pwTJbo=$-Up>9d{vlxj=BY#z5!7TaQ@&}4q84bh5fo;8$ z_6`+ObZD}`3r}MW>mK&7M(tgNa;`$!s51pEY;^A76geAqlPa&0^W~5mS=ETW(=M-B z5_U6o!~-fT=P*SUnbg{|EaPxhhT5(E)LYLbdf$$Or}|p@T6NI?imt~9D&moqijp_w zka)+tynPHCTinvfF`UInt1HS>8l3@;MH{I=fi&Wu#1bH4)@M@jvQBm!s4ErF66t6v zd93+Rnfbf?eYWe3BER;0mK18j7l{cj3wBl7gu#WL-M&_`l^H{3haEJj4C!-69q;M5 zxH|aPNAtfA>VZCJHU>hzAp7#E$4?&azGJFrYxK{_VO0xTB@+zqv18Ka1#AH^d_D)W zcS)Acua*=*B?K0kz_i60z`F8L*cZ6P;jG7Kn@y-DN>D57^7HFv6>9MpMQA85k*pIW zzfMe^Wp3f6Pr4^hF4{fr;KldtK8y`oS1pVnynCWOx$!>PaeKSAp_-}F0<$jTxh@J6 z6_^-K7)*Y*V8~O$=akGStFsM9!lfqLQMmEgfHrpOLY2J;obfE)w9ANiGtQkaX`>nkNT8ujjLSSSi>MyYOd)d`fO zPX-M6jjFzdjEK3DjLk5LYta}7(~6q4Z4Hh@{M@aWcI&GI?w|PmX-CFct*rXOuZge7 ztxh>IC?Qbja^9VEh2pUGb7%_%?6HFhK{t(8T*irLqK?$DGbA9wi(*r9%fmbqZXdd| z(6T5e0M58yj2Tq7*Ayx6s z3{0%*gL4lDR;*^A0BJcV?UC`#0MqkNPUb@@BMQ)n(Xf9g}yc~ z_Zf4m)%=e1y!{r8AXIxxGzbS0f@u>*NI_Zosq+u`&WlAvU@mV-$N^m(Eq!QnNXIY)G(0r~6#K0{I zS^MCSUa=mzu?DmA+> z`2^uk398ijsoOXC5rR#7G^2rQ<)MR0Jma5gVef$@C5+tMr5evLb1JP3fw+A`$U9>k z3*H_%78LUlD^xI!L826(_d7XsU~Us~e9Y&d)-)$`GOv77q`@#qTCR5{&CT2h*I?)l zE8)v?l)<9?tlbP6x?%POto^tk6;`doW(G*hR0IKcIS4^REiSD0HF<)qA$oLXt|h3w z@!NGZ@r)g{PCF78*{;!cK73a=2{|`(TKb0Gb;zKe*$doI&>ftH6&h}n{L5x`+5p*u z(=14mn%r<_C50`ngOb8N(gRK5NOw&hjY-l{2(f0AMvc*U$;Ij8c>}w*w_git3ImtY zP@PWgEk~*T8)eXGZSTQW(kVC0j`A8^(LqKxIqr%b zW$S`mSZD}-@lpNGT&3+`0G-OWP1C$-Zt{}iX0sip(yuKeI1xpr?Sg#Yx)G?|Z?1yU zwEQruVX2WyxnhWM%>lUBZ!767h_c6gDZ~&An5z5?BiB9phOef3MO@uGZ68AYRJn*z zwE&utY}|9JeAcCj2pyrbcbE1UR!77MI1N!Qnr_v|HH>f;7#MBKVUWwkR`g83noLH6Hgd0@de{OZTV@WC~F^lM56+(4I4Z2a_F%k zs)y)K({s}UWn3!OCR>MZDiM|QCUqq(`6wV_AHr^gln-jR@HSp~Au%87v<3!ikJBT^ zl^X|j2?y1ozla=G#p60sZ#FqJ)RW{j#TSHluJ8p{$F!J6b&?%*l2l?YfkD_+)OzGZ zxyB3ZZ<(7q+Q`_QX^Qh(Bk`Rn-Lyua$~)3za$H>Z;?f?aUJ7`hoCcJy&2Pj}80Bwm zTrc6_6y3G^tvs4sClSs`emDk0LvrY?358UIiY>;E-X^ z%RQciXP~8;%hqR#F&>}w&qoQ{(9kg1y;cNH1kvHDS^aGl1=DU#+p7 zC00m1I%T()oDIS3VHN1~XWHy>VI*e(O2vyXGG*jU=AS^925IEpCbW_y+Ifq6rMs-Y zK~LSZtV87_DbqtAHmD*v_7q&DHXX3$QX4>UX(KRk+byMlLBtUuUyT*su9uO;YY1?Y zin-G!03L;~UZSOLjy#qVuAP6sOt5iIK@F`u0}-m{I7b~XnX2#Ls!yFLCtR9E*Bm73 zgHil3R*R^Cd-uS8WZl&lxH&}KQvSjgqkap@-#gw^J9qG;8qu?(u>dFR6AIInw#k*QEysv9#V<&|j8cb-}NxIo} z)V*6M+m#be*s$7yPj=Xl+QMG-?^=Yr+SZj(8fBrec>~&oWvUPPK3slf0GCp(p$J#DXwzsGN`UD+8vdm}hSipZWIy!{3oASQH+dYsj) z;Oxv|#S$k>WEx#?!kvzu>k}0wug)U!=<0&GvJ~z&uyG}k7$Kb-*KJZxl~{+GjmZh{ zFjVl)ftcZ*TX1HNSZ7>GlxtT66=>OxXk;=olm&b4Uh!$X$4i}u^`;Jw@qR@E#hZb` zs=`=fKsRDc8nv_I-miv^S6y3mnQ&T0p_}%yC54zB97m(}EeTpx(TTXs5@_gca}^jw zG4W@n_OQc8u8upva~m-p5<0^z0j@dqhQql5wmJ13?_ZlWAo^uV-Y=Co#?Pk?#H|gD zTz~cnn6xa)%O_yH{cz1We&pU+0j~&bE6?gQ&Zh#?8`D5 zy-Hs%9ZRsg_`)oUI81k|iv(Abqn49d>;?~@2;luK)Jl5(u|%3ApTe^0i~TTnf>&8!Sp(3kha1WW4eS)F1_d|WEphN_0&Q(uo^mJYx`T!|QiOOv z$RP!NTiT;u_~0szSMT#7IG_~coRcf#e(uN@}fF3L+q_hxq^+Bm3Svi zm7lb0QENF<*1BFTukKqsLRfG_%^qCp= z7Wk38Cy_~$3Y~xhaj??CbIfIYTzAa}i0z%#MqnUPx0Xqk@C+QL@A(^*n=yE*2&&cH zy771uvX9+zT+d%ZGZK75sap(_#)ZhdJD9suvLyg9G=?>~_{XrZjnrM6ORk1b9*@R{L^r-D3@fE&UY?`n?+WEWSb%HF<>%k%CYs)5KdB zc?&Y7qbw*GX0BIUGcd!mKQMjpac9dxSC{szVGy61%NOZrX2vx` zJSO`Ln78he7`BjC^}-N0w^uIWyRh`5glR0Fao=a~WWs3_x7rSLRCHdvJN0YM!F0)& zD zu%UOLpy}Fwx?wF`Q6X#KvD<>-{ESKK&uydyb5a;V+TTIvzdbn4u-r}UwN_YeIF6`K zV1o~a0&56&0bcb^_?3DW!(F5B={p%lS20$qdAf$E3ie1jB~1zstSGa$Q~MN0%{8-T zR+CSZLu$DL&K1FAJ0KYw%o&N$?SC4H`|eg_U^IA|P(VP}FF)n-lZ)M7-#yY*p&hZ6 zF`liRtE-!nN$vIvv=z)dCak9d^1%d}DL%*4#iZ~^X-H5gkv7MU#*L0KaB5bK_`pOuqu z-mHAz_w=Hg)@YbT&Y$zL=Ksf2M)>H5w=Sj%sh9>K` zEnE=zwF=ko@r!r*@UI8vU{P;btmDd0$Q$@Nb--A`=#f=#&$RDa7os$`Z#>S0F&HbT zLJZEne@CVHG*4!Bm z#e4Z{znA@!H!p59%j#agzrsWm%A;DREU0H#L^$9r;;9gp4EkwZvbc6uareMp$%WTS zQ?1NbnOLL)I%JJ|$5DoD&$zu>kBBtByzUwWdYTdv_batov0e@-nnuD*@fxTZHFu5b zh|+rw2Q!y3sa-3tQsWya>Ii4c$vX4OngsFQCd=W0JdLoOOLOt*kv?~O`oXnDx@66c z%mVp3fN`D|$*kA=mfDdDQ1ilQY6Pt|nrxH&v^2k9P)hrz6?)Rfol}^j>8gZ`5wgp9 ziRxzMzAA?NrK4oaRULqaR|*Z?+eNyaK0;#+W*U;xi5M%5vDy7@`7Iap%|lvA@6bVW zv{x%d7UIC19Qjo;+yD*n!J!poFc*vW4p&lD6_RTOWakW+?xezcVif6-ad4%&_4*bO zN~$bY^O18v}n2Uq>=4G(l!;od@+%UG9%5qYqSPhxpH(hXZ+k1T>HY((xLR#8#$l=u- z2m@1GtaPoqlO(6n=!zDs0B;Ic5~;pGb4sUox|NV%8;OA5c>6-`tW&CFJT zDr(i7(cjiUV}GD~xUZpoFE=_Ht3VBQKP43R2^`qiyvaveg$y4=U9`{`Q4qDLWZEPAierZ0`LFzdVH^W<{T34)_U6UKt_@P(;B^UD=amlbh{qVwc1oBI5E zZP}DgI|B*vA=KPKyHOfTg2)yzsUxXd386{iJ9jnt(m!Gp@l|$=R@2)d(bzBx1t0D% zmZc&6>>~dD#?(W|qq-a7tD>Y08y6Z&{dcp9oQ91mjpWkZaw^;bgFmx&OkMv>)97HJ za2M}TyrpfW4G{qK@^;W&wfpI=>q7LlF;TUyr~%45$|`SIoeI}l+=W{%l-`ktDsLb@ zMqbIk8mALw24_q{HVW5c+InJ4GbAE_tiG7B<18~08asddS+Oc(o!^7miyXNMrCbI2 zGp}?0bQV2PWdMTUX+LB*uVRQr;gZ5}3>RN2G4zBS2lfzP1TwD5m18I)w zJOdtx6U@M-^f0b0mUS(4dg#MnN!1T5hSf?}q?I6-@QqL#NbJvEe^h=gDA%V0dxvN) zM@OJ~PO(FCH!-YngiY(RolK!+3Tjl1(-L@iG}j>3lI%ifO83_?E@o@WrrUIw<4Xtv zZQ{LGohXojVJcFbQ?#~L?J4zfA#KwS0jwHPH#63zhB9^S?M|6kdRG)uQlwvueg!?F zD3nzv#V{JSmf{~$kSYAP`Y)$>bHX2g_9&i4t!K><=KYUwtXM6s(*XLrly^~3t7r6+ zrf%~NX~i}9xK@;*k~-S(e|O&Cx2htGjrX8 z#pW^(s8V?IJ-v)Id)0#*hN3fTk12s0xzBpVs5fdL050%NDfZCoQQ6-&1Aw4acJki9 z=Le3y*lWZg+HLw@6teqjS+Ju+By6LVVh)p*E0Hx`$j+5c&Wp`Bxvl$V9IEW6yq+k9 z_F`Ovx{9jqKY0V-B0?xit6GwJOko!yPV~Ae^g=+aMQ$NbsygOCeYEq|xNQ&= zSexN##nrZ7c1&>xNt%}oG5}doIw}u*R?~}mzJWT!VpWBbIns=8o;OV$Q4WBmZU3<6 zVC57j6NH}VBgIl5>>M`Twi;O*yT?-0{*b#mbx0RVIrveOYE$jo41u6Uu`2f><5XZ# zeTXuNBp2%PV;<%Rd}lucO>ar96?}Q1$`&CTjihXCCy8IjE}~%=049C=ex;tZnn7%s^$ zY*!7lqY#u$c!v(kqUWs?^@$C|-0i$Z7>o23e}bu>*&@GB=ckED^6Rc^n=t@K6vk`6 z#y<#gZs7DTKbXGMS)`K$NSqD`_1vi|Q`8+nMY&)M9uW9#k-aS{GJo0Kfwv%Iv>Wzg zuA)2VLZ!p%_>ZD(;8}j2_IOv-*r`MtQ)z&)@I>^8)x{?K=_VwFosXc;jfCHYBp7f3 zpdMH1efV&+L2AO3VABYdQLRL%LS|v5i18+Ug(3y81{>cRM$KoxH4HybUyq3I3qgod zn*Zj?z%R~Y83zqQTlK&>kC(2`#?Uh=!mZbq(5A$rWnG@tLCtQw zh>L`>eTB^@-5i8LlXiO~?=SPkJKmXYj1|!8;%qCfvM99uMA6nB#yt6a3)c! zGO`XQ(4;^lhC%yz8YzP^bcm8D~DH~iBGT&cc|WUhquPOfC)PN7F|xV-<6Cy zbg9Ts{{mN=xNQZhzu+pu7pn{KzY8_~=}Kjco%H{4C;SgmL3xaf?H55hU{gY9%@6IR z+a=4csN|t10s8y1ayG$tfpRRrDE9J2Ef*@&#hZ%Uhf4b`@T*(|L$x?^Z8N|8`HKkC z4bzQkpSQOM+%9PDP&>XJP(ij?{W95mVI1Pr5bYJ|Fd=g*v`Fn_NDXmW#6m`3 z1p=~09A|kP+NBQC<+uq7De~Y-Yl-rUA`9sq*r6nPH~T3OH*wj*5W(N;ZgRP&8=q>m zkuY7uxoGOY#mD$}KW8^+gkN7=5cPW}i#l1{rbzTHT{>#JIi%bsIj%QM@dlfYV{cjX zz4nY=6u^^52r?I81V5WM%4eu7vTtMT)uQGpmV$_I_l3``vT@24}IKLnxgXKj!Z zrVd`iUj{q^i4YSG33cuc2-&I`vGwCscpL4Y8{ICi&0`-u+E`QOd$92Iy$VCQT*F$q z*8gR_G`%wM9rFv^@_ogd=HJI#9_VN$Z0lk}{9kr!O2+?5j*nNm{6{Co6X}d1xsK69 zVfc}9M4npOcZS^b3n4bygql$zEEBIxFmWbkX2NIG`ag7HVwjhwPv4en!WQ>B+FUL< z9@7>dKR#Zc!TB-wiH*2b6uOvxn*ThdR7Me*LKZKMqYe&%{h11L@)3COD*FDbv3W4c z0*)pVJY%!wJDBw%kf!tEYI+EZ;i%y75r%hfy~7yhJVW38&5jS}e$!oVz2`?wPZr9G z?O0?un$YnrOw|*Ei{P|nZPDbP*wc;d{;FRq74_(FoC!W%TyW>)i2AqCy{9>DeUZ)E zc8#WW-#48-;x%xLr7K6%Ss$&txeF3?c&mgjpjh0BS1NGN=&AK)-PCN6IgeKNSe9!% zXyFAjlk#%JiA!`$VJf75VcW15ZD`)WnagYG9=VjeG$6FHYqXGJh7TGy?V~AJ`rZRPngkOw*KUKSCFJ zs9G4etWdW9Ri`0S202F6k99U8t`361sHxM;)quH2IpZ=wkLs8XsZ$IS<48YAtm3ZN z6s2B~F$N@h|GM~I^R2Jo+atL}qH$BCc3k0Yc2amofiQkn9INUiB40B|#V-}J!M{ZZ z03=u8YsFzI2SU*g^gTID5l*ok7>Q@L@;s#$S<_!+YGvYvC4;U=gz`QUFui2KKV6@IPAoCzGV#7_ z;qSX%tXr?xC>8*inmr!Z9K8?UCZ5kPIzQi^*?!M{p^x>Z7y&erX6^VS0KiV6!^3zp$+h9VGqqzD>qigp5*6ZZs3{|Bt>$MI0?g9HW*aefZ5Z++` z^Q0a-Oox3I!`>;GUW+5tNkr9L@xY#vTMs{ZgP|sGpsro5>;#W*?IolV9rPm6qO=Gz zb?+^%>@eE1jvjkk#T#Y~$>`2RnRyd!;SqH7(QwT^2WLP=H@Efk%3dsR`-AdZ?%zcT)D?;c7BGDp~4K+PxK&y&bw}AvGfJ-O90b+kvCg z!0th1zB3=K;rW{O&&*%(9y@nersNFR(T>!RW+GJ+T8wDXY2dl2hGAtsoc_^b;l&X~ zV_C~fp|xE)2q-CbmCp(>RraT7d+lBA?x)u63#N9TP88vbFutbF7~OGe>$0O>3B}Y) z0$$KhVIK>v#f*w+Nb5`YS_e#nPKN0ca!oCkTDFPy9;=Y*pQ7F)A?po+i}nt}m}XYU zVDaj+f6k?63t*m}uH{$^t`6t)qc{f&^9k>`_grrwWA$gb3?*zU7vk9e0b7C%FpcW_ z2A3Q7Aa<0BAQbVD_p_diqlk*)&jk3~2m8AOP92{xYYbdD-SqmY6NwIihAYfNtAkha z0&UCp#;7i97@-qUj&0PiAvpw)G-7zjtKStjv&;k_P=={Nr~#~qbYZrH3P(~_gcJ(9 zP{7|E9$iq`tWn*R(+KP#ER?rQ@%z%hQB0z_B!Y|6iBXlfB&aH!W8}X-n|R%vv8Vrx z6B}0TQWsaYMS2IV7L&=LFm78-*ek`bMaKE0LGrA$zWVXAU-J+0^q5`)B-{v}NW-iv z%@2*N(!9(V8rictW{f#Cl-GK}6Zp0i8fkyZC38hd7IQ6hhY!;$QS6Li@=4p38}3lH z-mPMj;J@2*E|$kyA#PQ2!G~rMjMr{$^Kevn{Vx$!9h=~#;+)ayu}xEdNaiv7Uwo^) z*TAXd@ndaA5G><8i9aDJ{Fa1Z-u&mr2d5m4X^ir$p>?#_@5DpS@blsq9FEhoAF2HL zgt7P#4>Kud>AoJviV0c2=iR2F99n(OGGIi7?W~Wu1hMQ#C`R6)8Q5qN?(h~PZ`Ygn z!M~KK2=rkWhOZK({Kd5RH&)pHv_1dtGx$GBl(hMODaF!NwQR9PP(B8nYZ_{slZp9} z&9DXS&ib^n;?T&<)AK+YZLoKkvuPUZd7KMYE&|$r5Lz1q*`Igg>RK7*-< z3$?D^E`4Zh%T`rdX7?4%F~&0W@YY)aD5T-)1L8DWqy|YVb_!S#ozkO~o-&5Jb3v>Y zjoA~$8N)xC8A$q$gG@oihbeU?Z5HvXNJ^FRKaFB&nh4V&mQ@0rpi?n+L`G{zJ$1!< zJm5=A@?2p4hGraxaBSnTS<~3I ztW|ss?r$moq#E-oZYHZ-;UT)(uQM)qeXkY!N!j8qfzh*$vAmw9V*edkdGD2De7T_m zfLyxchLh{y!F?K+78iouB=J`Z%BI*AAHV1c0bk;HkQld6GzNwlwJ@8CoI$!y`OFFS z47*dcWqg$QXvZ_o znTqS;wvd=w2+@VZC-XEWIVWsu!`RcCJX%ocCxla@Me~s7`LiHHnzT&-}+-VMDPEK7o@V z_I4$LM1JvVodMn-A1e-j#IW)|iz`#2MX;C5p(_YTpv3bxhNBGK6<}az! zp%44-Rpc#3Hw@6WObM+MHTk9&x+>#rv1;IqQQ=_&Tc1Q@HM3P~k(; z`#r$?ErRL^REoJXm@j*mjxPjx@0fmK2jyKTI=BezTKKrd4Yu<2E@J>&z{oL=s9yP4 z(;b`#P}^0@dxO83DLr3s{~r<~Z55rm5UaxcZ%fO3|Ze*9jKUJQ#oDx}^89rYQF zMlZQZU18|JEUHkp^=5YVFBK>-xZxA^s}XnmszPl4H%*eFv7?=>jpKiDyp{eJ9P`2^ zbB+aD`R>pEMCli`_q#X>K}9~l1ObsEMc*y?kx`SLNR)IqfKZah$Im6#CW> zEQ9k*kuplOuCt5DW}BPs`SZ)^E&8`oGnAf;1&3u00 z1Ao;NMHs8dZg0wX=2MvE6~b$J{F8z`R0pI?!joOe3BdmH>sLdUJR0XFSmR~_uqCA7 zE@q^x2iy7GVdP|2u8{sZ=($s@SW^-8cwGdCH_`du2pelJJAH{%FP_E0iHbb~4nC32 zF*ag-nQf`W`o{@zltYKQo$WI9fd`4k>e6AYAZi`0^IQlS_VR|Vo~({=Coq<#53$}- zFYhCtSKV6^Md91qWg%5x1`hdd21CS^Qe>eMa0h1NN^?1$0S|(^$m+Roh{=yA#`0PB z-1@$@%Ae^~eDreD`wUnj)ZdON^z*lS2@^=uY_LF^2f&RsWRB06pPBM0B0!L_um+2> z?t=+l{gMP@@^f$u2mO|m0IakHxYXK`mI;sy6)lpl_j7P|;TJS(Jsc;%pDywx2NY@o z%_*P%O`rPD;whRZ^5Xal;&*&y+W!y4|F5_1KO1o6^M5qZX*47@ikc~czDQY~e#`>j zXFvF}zUBN84M8bZxcPzpdZG(#XF7DusvLmF`yD7)1}gF%Rg%{r>$sb`em|l4asN6f z_f5+caZCTV7zitbCFnDYn(P2B36MKV!$O7>lC5Px{81y6I#%QNpx+6mv79k)7jn-` z@*qeC656BUv0tMNIZv(dO9Tq93nNh(t5&-Z1DgItJX|QXwV+!O5|aV~uQo#4rADG3 zA?ma*j8eQG57mx$2CirIru4gV`GeR*ecFx0=CTl`n=vfFMiLcenzxR64wFjbXt3ny%MTusAcaE(F4M zvQu6;Oys=OEC;T2K&p|BusL>6GDUphQ?vop3JJ}5P&G1M0^-~p8MYmRqWsPNoj8fB zOveCP-Q@gWXJxI0UpI}k5rW-LK*_~VT#y0i8L126A(pKnc~q57HG6Lwuw1j=Hx zQhDxEqKGL%WCKK(Nb~5$YQy`=dw=Xc(ljdd=DQ9!=c5&PEYs0-xl$RKA@d8V88wkC zKrK~f*a&4*B%5$shz|@%_d<7=a2+{1kAf4R>G78c7!eikYpPXMOCI1A61S7&J0{bI z>%$?}Kas-Qg6gI}zI-cPAknF|P!s)L*2vg;2HqG_D32k_`RGR5OtKIk7~vF-mF0U8ow?#| z|36&(j|A@ARYwrhT4*1`Aeaq;Z+YmGS#})KfoVJ;zN8yp3EuiuBkFd#ySyIoU z5f}CV1LPHppqi6So6~tXD3#tDosC!zegv8F3qoXjeZ%)}f(z<^iApy~5EgfL^&oO?@g za2+}tc1)Pwwa_cz=2lvQCXo&U>U;w7uTRqFns`21@BC_Y)u3k?xssI3z&vE*%=M;( zoHqSgp6Oc}S;(n5BXh0_>#0OGsg7m1*MK~3a{RckT+5#ju0Ey4jEAEW>16-8Szq;u zG2$-W-{1fPZWsjtmOi37!MbUEvk$LOgve=wqS)2F#8tmF(&klAFnT|)a(^CW&uRqd z0JCx+GcWRTjQ(wi0i$#LUW@NO3w63!bC2|6QNG~=LmrPy z6=?0KDV3~oU+Gf${@e2ZpU^Y%`tY<%2sobN2IwU=kirciA7Y1S`MmZ)iBlv;u2Cb8G0_z7g%Up_j%6b|xGSu5nh~ z4$9JW_T$=K__+HDEK_5X1Oz=77-7Ji*z8*yzk2y^96T(9ug472-)7@W-1)WAOxv)= zo~M>siDwH1>@LSIOn01LUU?;U6vY_7GW9wPu_mUm7EuXX&q9sM5Nv)*e<6=$YUU1|Lo4*4Z!e+#HG7Z{7E8vpEJV{o)giU1U*JS;FRYl z0nHS_5YR1a?=+pD^;mSSPU<1@^ze~w%~HH;mKXs;r3NoPp_;w=6Zk+`Ml+K~YQ9Ef zi7bf2p9DpZyVcaSm$$@eOc|t)#3rCcb_tiQe$BeX=0}V0A&J*ZQ=@?SD*im11bL%| z5Tl2jEAOz0*!-IiN)Y#FZ70YS|SI0 z7C`7hF<6QV(ejWi?j#6{H_$mC$!RCV32A1UXa+NC;+Ha~&nGV#{=Wv^nk7^UV46YB-LEF)G{QP9~yo9h-AaWFw!PFV}3GonvWLq zB`Vn}rSkc{m# zqRkCAwr|pNCMiqgV5wK(7zQEPJv06E$jl7d=-|m)s3dPpg6KptR|;$ll5r^pbr`*o zhRxP#v?J0-Bw(jGYknKr5p3Dw()3rRybKD`JAuPsyo2ycF!oixiL?$rl>FFQ+&}p*>T6{`=`rv#lV(P;VaGgc4#oVzLYBLOSfxXTEY_Roh z!e09Vc$xe7V6NqaJdJ`PVXuP$yexgD1y9Cu@%eFVO)JWRCEev@*yay#`N_MYG1f6z?^G zv>qu}xRKAY4~)jyXVydfzSM;2(OxhbRykr1EJl1n4{SzLQJ^p!Q&E3mIF_P>U^wQY z5&(gi4ov`fMx$VW7=}Y0Kn$~HE-Dw6V<5^9mSZByIDndM&s9j4eot6PmT}KWNS0x* zJD`ky4-mjT7Zr%fXf#wDkY+M;9FV3zlpMgl7F7$wu^QzF!?7533&XJ)_4TK|04mt# zsVFh5nvtkbnC7LZ`2cN$pI-GR*}nVlr9`-M`g!M==PoGGH9)LJF~B z$25EdGY<1nC>0oo(z2a6Ab%%#qT0O{caWt{lN3>GSqu9$wu)1%6;~8VFr=P?*j*w? zj6E2b+6z1-+&x8-kqLIS;I3yD-3{L@MG99!94k2?DxlWpDs;d0^y8m@xf-~7Qcp@r z%#WQ5b-zmz%AMJEeL1o#^OyH9p({@lOXqPs9M`q~@yGkFEx)!jaa`z1Hgq8O7?auK z_$U{#jw2*aS7qAO$X-gxnz)%;8$0xhIk?5{?8uIBWKuRiK^I*HfMOcEhGUJ>Nsbea zjH-zo58)-Hb}5ycLY-!nw=Lq95}aWK9j+5?YqfYh6D&;Kk3hqbj@3!L(nOqWCM~L1 z+#l>z2hc>96+^x*F{8u*?HPB@PPG^Mq{zTsCkG!-9xA~$3ZIuiSh=-xY%QB0Zs{>T7J*CWqmHYyt zx`MH3s+sMz$_n-k<35T|iZ%T{UhQ0+rG);b3`vWs2i!*!hu?hE75L3_PA099SU&s_HE{&$H1S58;Z4$T*8x_R zVqmy#u2} z>BO8;6lVI+;aUqg#Y8$Gb86SL{VH`5W-TqSqbD)b2Tnk4(HB6f#^$T3u*EHdV&-;K zA+O@9U~l{VFpXrV8|-$0vglk$n8vV?D|BgTPCOSqe6iBxkMg|WcnkB;67mtOdV(~~ zMrrdT(e)yG?GG=-MP*y5oZ<`@w*V^5*p(Nd8|wq4MM?WkFcNtpL6y-)W@L;rPNR%j z;dYOfb*phgJVg7xPL8%ZGooQ;+~u?jg9k~-*eej? zV0)`#*X(#Xz2-|G1RU-oZRk>Rd*w2L)j#+&Ll2H2)`0giB-Th3X9z_lI`A{B7U$vN zP&H^z)w|A0LacUkj}=Oh&WdG3iD27KYw0Opxpk9` zH$W?bC&KhW`{pE}wE zoVG~~ywZv%i6Y=CVR{&{jGfE8(ftvu!ivXXO-$)%95iD?cAVn?(aJK~(4O%#>n0LI zMaMt286xM-6QC87&GCgZ4X4Ovak%tl&P=AKD=HXPtedy$qyq<6z8aAcFyO^@9oJG-)_Y(=wQmfQpJ2P{aCC8kdHtGmcSCI4iHu*s3Wys7nFz_~sBw+UoJvEB7r4Mh-S{7;~9( zzfTXzkYyZuu7%UIMaMGSjWX`{@;k(IF8B>Kb;cw(#*Bj#jM8hRyl_n8n7;e@N)2B| zEm7`LOFB^1?pzCNx>1d!$Zq*7VZ=p5353}we|%n5yf#NTmW4&;ZlyRo2sP0%mge(f z1uu$7Ixti*vvD>n0u3ygb4HPep`sHNWY4QWMJ7RWs?FS`f_a=u@U`r6@qNMyk8X%N zxivm8viJDEYh#hiWp^Eb`!#Uyk$Jn7aECWo@_OWs&c5HkxE0V&-i+4n%e~*=xK-$V zNN0#;8{tU_YBomx6~%De(kH3>5Z1}KOyo;Mf7utgG3b2rZ;QI6i%6GPWaqZhhqO2) ztTyRv^V8A@KPR7s^I2Sh=V9EwToDFs*7j0j$5b1T_?c^t8upNrm+NLt*wFyM38&ay zCCX4=6_&)~=^%;Oqn@~LDRcF9qqtbuH}-DEU43&1Jlap}jO(1)-}iR&Z;N3^u*r)m zj&{o9W=bCb`K|Y%scSsAx4(RZ3bG&U>K5=#PKbTrT6wd;j!9$rMp!E4Cc^>BG?fG6 zcC>WgcBl_nOxm0vwo@jI;t4y>7x&7ZeD-^y{qQU!j7gCF>(H-r%gjb3MP|@2S*S9q z=FsGUHVuX>#4)ODxlFZ@l&15o0WhL2JWg)qdLOC{FZ-8fTtlaO_ug)x-Uo?nJ2std z!9F$#jZgl23+Xqi4ivAX`9@KEi`dK>*YKF?`2W0(*DRs$iLHTgM5##`}lkBEuLL; zpF!A&0p;>7{G;{C8INHaT{l7N=BoVStlYGTB2E@!nX@{&O4-rq? zeT6)ARl)M{9Z39wHQP@ChDUed#K@QZ%)LwrI1RMFg2cS(RpDToEuLd^_=$2yHxiwm zk{|tanF*6m`?WWf-kw;Q*M`VQLg1>)gPN*-$2BS9`a_uoX2F1-t28}cI{b%G^u0=P zLh8J+2Rb>2m(o%6_ZyODM(gH;mjDMs)@bQON8Y_epV&%ylvx%3w;PoisOJ<aI^ENl4fLS!mmiu7MRkl}g zEEQo@ZsDQY!u!!jH?H^e$4T%TUdf_Px_V%Jt zpO;Lw&7W^OQ_c->u}PO2`HibBK6BeeGf2(Q-Je!Oe)>W=lrSRjOH|%ExZt_8DH&>ZkCcrHR?? zY>bfQ)tgw}{7F<7f4q*n~zU($mvMFa;PUg9;EmllXEr zGaxAZX9CU*h1Ro0+#=k7 z)=hf9oe?OsHas^f7+^W#^zM+CDib9Fpo$qWyg!ryVR8RNI&IKW}Nt;3E) z7F?DGTs2HDHl+MLjMLhbNauwQ8qE??BsAc~EcUSSRU|{NBO9ch3l$>F$yF_e`y}5c z{@}`wIKUH9b0za5>#u|u*G43vE0U|N%F@&iXzDZ!Kxv=|YbiAJqb$gy@Y~R0HOBQ; zDWbRx3^mBspM5$6CW@%8a*ShzIuKg2e8wdvwz3~rGqa?K%1v^)12uXra%{)3xQa%@ z0J9<$*%`s|1wlr(9Nz3A@xbbY7l{5JwHSuWarUYj2dIeWQEIFkDO1ajwFeIfKjI5EE@+H#c!)PpLzKwZMtBh0l-;zrczq zV?2?;l#7*~YmCxx`dR#ERM~kejiC%h-t@a!#7`XG#aXNbsy=hUG{+?Vr^H6$-2^3b zd=^A|*!=|u5nbwprM@@$JSBBmed)q61X4|nPa4Yzk>|CrWqA`m2Ck=QXLYYR4uT2H zT5>^IS|(AReQc#@9uo;Ei}jF$F#>Ri!ZLAo2~H7yN5cGx$r_Xy+#xaFTM8_qR8GEu zjYzjPdV$qor)0rNjQsPkKP!aIMWn{LDX~HI%>C&8sS*`Z;__t(QbaOFYUF?N*-JM{ z;0t>p0*&AmO>O6CKbQIKMcCpd$M85xJ2HYY#Dr(b-K*Bfj8D?dccaC5WV8##M<(27n}$~%kozm*LY9?GFs-DRC-KZF=tr@hEM#X`C+_M;SO_^qjwZ1M;ZPN7WZ z!rC();eU9Gyg`oWQs}_vjg4C~hZ<)6aE)yJVZU5JsU$_ERe&%-=BE?!sV-w-o>)*L zKl>R!9!$!C1yP#i%Xo&vRWziWT405!Kh7cOq?V}?OGAm58cKE1Qb1w_Q7emITUMxb zjxnLoY(h2~?!?05EheP#4KUbs?Gq7Hc&o9`3jCfd6j!2uDr#q9GboZj6mdgJy=s;} zG*O*uhnK0J z;R59qXV7&1Z&8D?j!&S~pgv#RGaUp)oWB zygbw|m1cw|hrw9uJSa>!+f2L!*Thjvh3ch7cqp#cmpqc}#L6+_4G&kWXpA(B?wdlD z5w&r^e?oV0jW%Ov$N0TuS*9oK@$xkPt1`=~^Fhh|%32eTnYwvi$u4-h}1)_hXS zHIP}QL6dYed%K#BhN^(?X* zY?Sk5OVz>82suZIP%`Bu(?BqbOrz?uwP46oPOV!BCwn%}P*$e^tYXp)YUzUAJL~aX zk?4Uyim6Dp8mO;thK5D&WSW&k%q-e*$Cyb(H*+kcci;ETGI;}2Z#tB2U)L_HyHc6Z zo|ddS{7%p1S#gvC&t?mYcO|J%%@0Zeh`bHb6e?Q_nERL@3JpT6F(<q9upcF zECPv*-`0s5xs0!v3Ry{W(gfzL351OM%~8`-w5Sdy3L!1lAzRv(U-6*3uVD&B<|N`J zy~~z4t^-3_%wmRX8OwsLF-`qZSTylEzTF44K)fP@VcZbI;I!P^}mReP6fV_zn=)_bELBWnHa> z=OVG)`Q_=N9c4ea!#u7e_K)hQa+yM zovhpoQwiF4w8kJKf}LN&kuY3>Z(=XhA44#GG;+bUu>QTN`n(|&x>#ifB1l0YF5{5U zn&rozaJn2*(iX(yE&bfEq4mF=&rOTSW6RHlzmvTx@=QB-fHwHaQ?h$!_pa?RYofOf z>FlTyYM#3Cv(=GiGCzn($D2)enrAu;bZ_?P?e1!QJApYr*l1CzRXf$; zxk-6_gLGo$F)36^?6BR!=g3#@h+F|5Wz2Cy8gO@XV?~|rG8ZzEe|w;4bq`vBf0(o1 z+EI~SdC=-5C3%Q*r1lN`PsZ}Do_4$Ec^QY}0gc8)AJ)87f|@E_hKheyrMl~Y+S!G`+pssD zO3P%)4J>ib%M{dM5vbJa7W<-}j9^oeJNY~T>m5@$+{yKkE#dC!-~)g)Z}6lk>J7Zd2BHS7}_WPwLR9RN?`h$E^`>oU6D_ z3wo+OmGaCXKC*Kivhb{xgUErm$iZ}^XW!FhX8u8C`sGk5M7V5g4O8Ey;rH4rpCb~3 zWK!HJXV+}!lq%mV!_KlQ#h-!}@Ubt#pD)8OS-yO%(3Rp&bE(;6?&&l#I` zmj|6v?#ySz%;$VKR0_pSuje^)Mr+I%wcwV8?nDQCPBwzhPoTy5d%|NuwfiJ5VaTb+$xwy0a8YJhJnx7M#C>%6%|vQwSBaB!eZvm>@j6dd~IkD+6gO0GZ%e4@FS}ooL_Mv+W<` z;hJ2_+8aRfU$fbNO)?mJHH^dMwG^kNKCAY0^HkMo(IS3^7u_}^`+%gMgVpfuR4P)^I(t}hBE5mzz%JDB-%#29N1P{ z54ExD8+AWU20{>ZteB3QLu!Y7S3)(E06a@zK0 z3Tm;_woj<7A2-5Pk}X3#7{zL^gOcLnJ96L!r+mee+W@9~H?V49)Wu&O+qT%CE}nTu zTbM8nk)paC(eL}PRwZ4$TH%WF^x*~Zx$N|#{{YVj0!Tk)AL`M=baab;yXyMZ(I7kZ zP5KuT0iImdZv%H(6Q0MyL*<2Aes@_@p2#13-84}L2_vr976N8st`Zt%`KORIJ608=9W_1m zY#{|bx6&pKJ2@yBC<$iTaaWM^XSi>4@{5eG0;a2D7W1sB_b;acZ-rmdV7f>cmXFIn zcLhZ-PHLe?1p5Vu(kd=3ZEj*)KCFOr9VoU2vOvOpB zfBm|XTmFKgYNIC!UjdX?!2)dpukY;C3mk(?&(P1rKp(p1Qo8D#lf2>QCm|ko@o?#D zVd3DxR(`3p`crV5_{HD&%q_`7H@dsa?{5-|~$a`gFV1w@X_%!P?j&vBQm&LS~#G*5U-SIqFI+OqW_1SI!VPDI>l-ayY z={_sullojPI7c>itv1d}ABSzKH0e#>XqIvDFL(iscdtT1rJICr{eTakF1!7g`TcxYk5>0em?7}B3LvRejwEp zYk2}HDAz9uaGZHCsZj6@aU#xNn7|`N+et=xc00N|SUBaDWhVA0tFfD?!uW@_L9lwL z*dQ&$J8lTkts~W|A3Utd?|=NEM6k>9$TDZt^V7?(fK_Uy$ZtZWN6S!`U>;n+LHTv) zIj_9-U)G{j+$9gcnO1Z=SBT5)EX|@6cU8It&Z}PVU|?vG(TM7HrJHm^<*0aU(8~EN z_&(CO+0d&F(vm>?eFz}kTF7P%w39fklTbD`M;8x^lB&#riv8}QRA0TiKArVAO0X2R zLLCza2}jc^`bUMfTZ;ex!`C}UXA*Vmz8%|k$9`kmwr$%^I<}LJZQIrx+g8VR(n&fu z`+noxGtR#I-ZAE%YgN@4HLGgYT3EiHg4f*=HYIWgDrhSCqTC&k-kTNgd3c zX1r(tL0?#BtVszmWZz6lAteCq;a0$Efzh>>sW?yA-fE8Z;f~a-+>lS;zlykeR|&S- zHh>U_C-R#%IE#DF{9q2y9-84+RD?PdB=XWfgFEtIH2mP+-h$wM6}ZhdP;lj_W67Cp zS9kK9xp(VsSv0~K{gW#pz`ftVKYy!bjFFIePyrPc}mfS`Na?!g8;d`zl(T-2m3 zlb1D~VzK@Usngai$+iJNd(dh9&?~(W>%Dkd3oEZZP<1ArHhNt(Im7*6-Bd$gJk4bq zz6OSIz8Jj2zP1R`H{0AYczxO0<+2Gt@Cx2}-^;yX1f2F@T{nQu``96R!}ctGw{LNW zjy?%)!_`lXs6>$z%^HB1OL&jS0Z>r?)i^^laSXa% z;a-_vsh7|HtMDuRjGlOJd1e8dENulyMF;7aSgm^#v>@!i}5gvLw%uaE&slnjoN zA08&%DOKElhaMywEK%zJinN&j8Y%<)9#N#c{~hn>gsQLQ|0~)}eN}0Eb(#4AIRE#Y zm^wpfF@uo*O!z)zMbfR)_^Ybaf5rD$-OSO9>B=qh+^UJmDG${*E1_SLLBtT4X+OD& z7Ha#Iz$wRO*;K6EdClacCK?y!ptSpgVi}_f4`J#d5W{L1(OL(5IdN$A(aVDt^Ks?F zOB^A|-7E$h95Go3*F%o*IYDMhGV=z+90_${PKFR1L7InV_mLcldF5Le21jlYKS23I zW%hX5(c$)(FQGQ6A$J20_tQ8r{6i)BkJ{mD`&({ps>7?OOnPulZt-jbZK!~~n6qKK z`(loK`iLAxv|GZF$AB%cwtb=kg1j)q54^3woBo_vkjnvvJ%ID!3mn6T68eC?7Zop= z#o&-LfR`jbX!OY1QJB<)Z9K86=}H_dDi$Di3f zIWMMea)wZkBemPyZb<$Fj{b0G$`6qpn6H$V!O?q&SJ7VByh)BdxVKiS#G6Qzd$d<& ze~7Qd-6&Lq5$NfVo8}$p4P;R}R%tsD?`G=LWz76qg2+;LR!mE_HJPd*;Tn+W=EH{g z-@yD<7${2}r+Rm8#QB*)qm58b-7ajT1;9w+h8m}I7v!ilhe?M!ZQP=}NrpR;{L1xc zaLZ`Dun!p0{LW~_hJaJ&GtSJ?zfhKDhKdhdq~+d`A>JXUN()x`my|=#44S^I(ueeL}R;i~7qUp9P5`%bwW z4KoiQD?YtI@j9$kuV}{}4`g<)aN|K9{Hz`qrZc?|>Kj;DpVb3#2i!K#fGC|Fj(wON zz4qyM?A%j7pfSu%F7^%6+JHD|MxvHWk54tb#Vo^1v? zyaBMtdkvyddM|GEcI{qsZ}>mD92orfd|(Cc{etr{xc!Zu2LduaZ)IGN|BdoO_c2=> ztRCxrtni}tvBaZVeosG^W*X#`_ngw+XCoEnScMs~E&YqW?0`Z#Z8vCE2m=rqWhaU8 za=26G^bN&O82S0&!nF3$hOPX=*fLN;69Pm@S&>d09Uh$&E)3ZhdEii#I`2M0jWPDw z895+mdQM>z!Jil7+kJV%gb`X{w8^mknOy8lmqYU%Q`&ZnWxWEgavRBt7cT%dm^b_z z0|zx5D&$;Re`^s^*SGa6Z{G_X;t0VLFIQpi$3{l<9cU2bIb{%5KpfVf;=+oTbt~~- zKSorB!RUd7O(VE}pbQ0_UBr=T2jh%O*zjW3A!fyUs@?EyD%(=^I3CFSlkND{w`t%A zEZ0Fp5tRl5jt`O+h|$IR701}Hep58v_;o)RG>`W^$^T{2J%GD_>W69sgFH-wyy9g} zL#2}K+uUk5uB(yV?4qjvST=XC9oY>e`kl#l$6a{=LOTW8otvNQSWSHk*^S~YRCFWL z9hom|`Ro_MaY#RhCx6=$$}^q`S-#=XIEmw3$?o^8N*Kq0;-l6PI1-4F1p9#{mN;EPJ%0Fc)#Lk2Qmmg!8tdwHh}~EIcgW(;^K2Qf$#xvK1uwb zAmyP3Jol)>8Y65-q^yYS3+5A54^Gt(+tDta^T&BQ4U!WKWY1x^6<;5?F zSKdYb#w8}xm`ykdms|c?0zw2FAwJ9IfxIA~K4*4;K-*veu>q|t;u~j{SN4InJ+0FQ zq?1lx!nhk>Hn&3gPs^R5ffF>Iy_I}I&!nQevL&M}1FDP)7%!sYc4Z`90#6|Sh~~1C zmb}OEi5Apfn>{WlZ~vHyubeV0PzLTfnUM{5MoofrxBOMmI-WVa3>1fJ zoh~ zX4y~P(33PnNXP)Qh2_Vfl>!j*YO}4y9{rM_NFaLnu%(4+u9kg}*v zhL!0EEW`Ne^v6P}=}42q#t{(^b5xkZL3;)fMSIQ-=(Sjrd+rU{g{i5xluq!x4Ay%H zjlYPZmTyyEQM`w7;iC?3f1&`Hm7>EVxOtUX_fY2}s)o6F$KXUY8ci+>8kzmrMez9=cdFAPl#d z3gkf{;fK)PL0CTxs%A$rpkfl3@3H#A7?g6w2ZOwFR6d~m*0@!R643gBtKgW1HRz%| z+i7Hf{nb>!vnLXAr8qGsfP5W_CIZCf&-p@uS;Tv+=)TT7AHM1;d+C>83r+dwhriI$ z)n4}Z3B#uqjJ!2Z(fHj;PSGZOc4OZ~+11Z>4u2KWKo_J3*f98Yq5sNprBA2gmJN90 zrrq3|l(m#W6X3;#v>zn0UHIC|dBtO?>nxiCiNgWcjrXmT^6P``W_{eYd zE8P4I{Y$<^e*)h95bBk9T1n37O<;@yFNL7_Kww?N%(f=>O=23n+>zXT!P)sKyZeF1u?n?D6Hp=a0)C+ip(34S{%qx#ngrY`}#FL3_= z7n`;=l=`5JQLi~k{v(e4+08MgopA5~Qj@`+?6V!FH8N}8)d?<}(tWVxK>f944PrHN zyIy7s(r5VU_Sa7Z`i*_9d62pYp0OWWFhp>TvIj3zBYLM;9wrU#P~j0iPATn3lo9WD zjCWAk^n^dt?j&>p&j+}7KhHv3-9QyIvQYQ%#J{KE8O~@vI2Qt5%)JOOO$}n zaew|C0$q+`}A-WIujV9$Y?r*1doY~<=b+Z|*#!%m38h|X<` zXI3G!oTqK#7+MX@0GTF)yBb{9krodo=>q%`aG*N)c_^7+5cEB%Nr-NRw$o zM?~E_szw-|U~MRX-XB+7i9&`!vp;FTI}{o~oz*CYou?~aE#08WrBn3i<%o&b1QRQH zmU5;mPqU*|69LV(iHdji48B9!x=8$1c&VC~6>bwt*Q~WQUQzXIe5c}0m`$rl56eRV zK#uh;fMdQPBXY(33Ac!XJ5Hk$i{#59`K}UMi_=5P#2wR!9%2!HGEW2Tk&3X3_sJBS z`&ejhQc~6gAINR$<$O}^g1ICG_7>C27?CuN+YaqMXzyIk#~2Y@z7=J$x31~{RUZ%X zjXL2&KDgP#AQEJ_BZg=o8u)pK1CK#8#3-#|9;sjy_nJMJz#!Gr7e;@8$Eik=O$wn) z#xecgJH0PiYqK3p*@ zLiDt_v~p;gXp$H?_@|0txZ(fnRtrJ^_Z4lW&jMn>a$f??!$u}=KcE2TY#7>rQ~l%a3!~5XRXb9= z)lnJ2^~lEJ(s(3=l=eW#=+CH&60GpMk5PASMFF*2EzSnpx2V@*K_dj%W}IX#rHO}I zq|1QeRqH%>65eFW4P)k3Sf)KQz}u=}Pg8u3*`O|7`KePV|BEAJ+23kWIV)uDO`|YE z&e}h0^0UwI`PCn!doQV8=?z}R=+)M_wwTg(R_DWU=x)0W+*9VYIlJVC-Qd{*0D2$I z4Q5n9%?25oO*Q&F1Y@bg8>p7^jsTv$TtD%7%xkmHHH!HS_HXm-%O1sDsSn43lC#}o zV@Jy5_&?QJahH~PO5b8a?J&;PetOVP?g#KFz}8&K`?AJJB*ijl&`x0DbG7J@G5 zR25tPWD9&-wCxVYxVo5yOblKPwGpYf5z)q)3F1Q!ExE&X{ov%Y-0Y<%5rWyQZoH@G zX3OW=%dbr!!0!{p5bBMQtXd*LNFA6R6ANA1Fg`trY>WbGa2Hd9Y~DT_ny}}u14JCk zbW&Y$b2C`Y#HdlOVavpLQskSnmIGmxSxz1@32WZD+u(Xwu+7oxkm<_dY1}N4-@Ve( zZ|hIsu2frIaO$Xy{mu5W40b3~Uz!S?vU*5_D1^!}4$E{ysUUvmMtqkrUPM@13^uud}J1}@$1RKN4I@1BKqfD99+~2rU{Zma6Vj;{_)^JMZTe16- z4YlzLvj8td3^`GhksOr#XOQJBkJvwuSS1l$!wE}p$Ov2x!_@$!Y~&OvQ7N~m$J)hO zhc#b>+0cMHy#Jk!|HT(je5PNy`!82dl>d=0@STogo+f6Fu2v5A|H;SH|B!R|(F{x+ z@F}f?(aIDj=(Wn&ibI8UlpW7o0UgaVCC$177EY~X7fTn-NB*?vu)G6ks6UsWz#l=# zP)P-Oe+`47rdaGj{RsF~_3LKJo*mC3c0rYo$K`nH#n0k+DyHx4WRW4_l){@_7-j*l zB3i!QD}9vYV~+%)^sNT{Y`>h{#L)SV2mSr)*N z`S2MnAQY?Jp-Y4GyOgcp0z%9zpt@NR(7&j;LZisdHd!NwE4arVuM?i#*p|P zGxEw*ZW0!>Gee&W)_Q&#rdk`wS&}9L%r^hgEyv-qp<7W{b8r7kyoML5G=>KwK%tcb z!K%M?82sflOdqG;`1w)ZxSP2sD~KH-bz(%-pWh&TUirbB$xV~Nd5s$HNsV?`l)_A@kai1Yc@ivQa!a0aOR`n1m=Oma^U0^4phNoDW7undR zA^`gQ0ltHG_>egOP$pshdkiFnSYPHB@r_tK_mO3x=yR=ywcb%VIC_<_b?TP%_PbAU zy{1xCH4PPbQLeFhY!+>9fvi54nOBmUb9z$Ct)4tUwzeY$O@BSO13ZPjD&avtFRFpHE24)jITeR!GZR39Zyc}Z z8l#vrkD1o9MGQaX3Y}SaAodn2aA7T(i)IZ~E?3LZpR*`yX%uFd_Yy+rVlSHda7Q9qw7ACe-2b%B`B@B(gyqN3TS?&Y*B-H4`@>^b3NXp~L$V1w(cpFilWN^pZ1y~nf zHG{F(i{D^Qp}f-Q1yb(jC*0`WzcWY|>lg+*+fx1`9*io|A-5*oJ`O~wj81$WUOO_p zY-Y;rJlOC2IRr?cfHG~vlI#YIZRh^?U~kB=&%R8TYDJX0LRa_w(;y^@&i3_L=D zyvM>2z$*Cmjvbc2y|uZhwfUP`B>Y6L9E__?M%4CEsaf3rjhn)1B>(WfO9nYd9kI;a zI3=sRiHR;oE4IoqoukMA6$T?MAE$v>9lB?Q=tl*SVbNs&Rg#>WNJoH~G(A}3S1v^S z#;osK@O0brXTYNuao3;Q@PVND?RZS$!-Tqk;=K}GA8FsF9#TvQ8_I|Wy6e`^B6X6|d|nFGA)c!yJ!OU144eC_^%{{#*;UdK}nruL(_beY0wkN;pLSF8I= z*sj1Lw-+ByZNx3dROAGzkvrV|i_tm6%C0jYzxl;Bx*yBa?G(;&eG9bHw?9J0N`M90 zLgbNBggEqNUc#?%{c3bIQ`4$au2LGALHmTn2eZ*B#KfOO#GeHJjAMC#(w`Q%L;V+u zb>mR+l|OQNdgJl|t;?A^8DPnl>sM)@k=b>rSvV{Ul-EhoF?iNhPkZ3#A_ zcTOH_+I_$7@HuYp8HNMPFjn<4_VE{RCZA!Xd%C2$v;ERdWP@CM@N1K4Byhu1(X-z7 zYtzClJ#o80yz{CCYw5#L@unaC+sa&u5uxSI9>$8iU)fPl~J`fn)4NQ+$yXLeMbvm3oA%IMG_ z_AjFO&)LFQxzqF#THEpWKq20;TV5xfeW>BmLAc>)SOydZ+HT?Q%3It!kmsKa#)Y!`<^R&`^pgh|J?Wg6iF?rItpk)=>Nh1dabI0w?QL#ttDF`Xj1S@$C?k>n{Rl|8lhu({wrj@b$l!DNh6$edKA81>z5YcI zL_lIm2iCybo3I7*jBQe_wc8YpC9Gp|M=qJSQ(`wFl!!Lt+;Uo1@$}k-UG6J6AtAXsGlYk&?Yk58lP`?2DAn^A7>2VQwm`CRlDj7PT&$S;eX>75Jg&+(gh!`LtL4w z6{HJWV59X39^?XX8s-9itH+NqH!h|!80PtmYW=ks3(}{8^U}@NYP|{Jno4$)G{b`W zF+0Ba%AI-+Q!1bEmla&EnyO>h>?L*~Ov-_wj%@q`z9oupud-KJZt+A+lk}6Z3&p#G#o78GW^T+Byo736KBC z4JW6&byjf=9*5dud&N~|s+qP)S?xBrA22d&O$sfu^r=W{gTiis0)fxMy;-|P{b$F& zr}&PSnH-;Q;s;N>&rJ8fDlC2Q`WSi40x*ZSx#%`G-_3C0ZhDs8#N(Tix7xUyw>>$9 zyyRn|;$kK7Cq3XfW8ptAUgv~D zNB$7Uo+TY5Dprx`QP%m=E{va)7!z5K-_Enr7zwM==uD>LrcQ3aT9v{j;*78+n@g!w zEZSnqjk&9&rX5Qf)GX=}vCs2jCIS-YeRDqeJ5w+{tqPEIDu5Ev?MC;}Zk@8=<*DY8 ztRUgR@^1Ia406HpGJb1fvD|EL_2IvzTsrD=_J_PX2PG!0LgQPSmD>YE#j8YDmKRtr znt?Jlp{uFFwmI?_IBABgsIr+-!d9CDdx?gJzX&4PTcOZwpvATmcQoe+>tzS`8w^%k zM{0I}h9ubMEO=;%J3z?*k`3(G7jfYMOP$+xE4J1W18%fZjON3YOsSjGC`c2V6{Y~b z+%*(lWV7qUaYu#QULcD5YWgVUEE8$b(6Uf5i_lue*qHytB}2-y%C;}B`O|Lz1pi`qu=?8N>6Cd9L}N{IB} z+!<#krFoZ-A)9G*qDx26a!>aT+UExU7(az6it|*W!ow!C?w4pPhh`*yJyR>`&}K!z zM6cDtjvBpo0iCua-+kFh-UXeDDRH~dDk&SnW2jwALc4J=<)$$gHP*#*hH?#lWlfuY zYO>u1G%j66MWDz*-pfnSh;0glczDWu4ogf>((f|Wos&GC`RogUMWHnr45y$R3`b{R zj*uK%MV-uA7@S zF}LhD7Y8ZLAZ;Xwmq=!o-VowBOOJ$4uRWeZ24-9h>-c8HhfakMM2;qL>y1!eA!&!R zlW~;2g%iB3&G|_!?yw)Mu-J?eP2Pp2KXj1wxG#67KW17;)QbKmA459OgfBA^lY@sZt-IzlK#db zoDnvY-w2gerUijVb1*0fPKH@5k5q6cX~pU-m^my9&$kOU8+Lod#AiaFs0$mB@mE$z zb~m)JWaTns!tBX*a}ea&Je1oOn*Kyl*w8!6ipI$ty?@=aTtiC$W>b)KfWJ7kc_e{8 ze~ue`+1fhipivZ6KC01c8yn#PdWHTR(e=9nMsY}mW(M*f`h{3b>*lS%g4YO$>V=On zPvFUM_~=rXA+Lui;kQC5-wtp?-E0`wN1ZkViAW#|7lx6EWr_3}2O5J$M>-M69M=Su zF%iZJ@ww>=0*g4!_sqA9_YT{a`|aA*%3>i~<|uO5@tQ+xDO4LR?SyzxZFWqb!J!IC z1LewUASv-S;kP2tLjG2)bq8)bh0CvEFNt4qyJ0U$=|SqrY$9xhph3DVA|7Yk@Zv#{ z%-65!Dv47ByDtEO-4_XipQXi@Sk=lX=r(YYq4(Pk|K&>3T_OQLoL-9h!igwX-nf+Y zQywvC37e)hIZbM-MK&z<#Il#I`h)-=bse@j$Pz*uDb&V*5e2OMxin>T)JY|%}G2NFNG)sPj2~JuX z;a|e#8KfKXr=(yn;|U-;i$@SG@&_{P94d(0vwq4!t~ZA}j;P9xEO(6Y8_uC{ThB2a zjCI`yi;3jevz~%)57*nZ(qRI4fBrs78`)woTeC0azvL1uK9!crI4H}**f+n^t;K&?x^Tm(W6{?2^#gXxN#faKa5xgfObIJKGE3{ zlF>5w4;XzMmuuGqQo30q@1NM*!<%@E^>wTj;07D~!^^QXC;`+>C5;#*mFLy1&ZikjdS{-H+Ah$<3YUDQ8K?e1_$W z{(k%cH(s<4N5OdH5A--$zRX94oY^Y)F^b1F)eǣUxoKh>1wnM%sXS@;eba=_e; z`+t+zzrCi|Gr##nssx))NcBchx+@HBzE{xbGdDC2R)aW;LF<55oy9O@11;V@j(UtGyhsoY*-kF59RluU# ziQr4Jo5DSe(HfB(dDQ}}{|y>Pq@43KmM<&BXTRcq<1-ff3j_BsnS(K$9klc`G&zEl zrq4)r>;yX9Aq|5hM}z$5!@T3x#%AU{#imT=p+=!gtJ{O}<$hG2#8#b++&+1?6`&kE z{*~+wWLlfFR5dW9H}cbu?!6BB5yy7F03OiB1q?;Jst+PQZl~+3r4*GYx0E1}syKyr z8pVE>>)g}i4D54pCV}fjGmn_%`67{7NL$5xS+)PmEIV4tnpUXToo6??g_5_&0lmv6 zrxIaUJJSK)_Rdap$T%@!mlJY${1wmU9x1A~E$7WIT3H>`>08PexUnw*=?DSC= zpZn9!es%;;QJq4JrZ4Sb2!>ImGhE_kC|LOV26`z65+Y0ynKO8>N8~m;jGbGxNF*m* zpAa)A%)zzQ)7A3U5f@2DzKE6ag>?cnv`h&gCzgNhC97a0J~9=hMXj0(wAI%cu{p^W ze`1NNe{tTNauj~YjcPt&6p=CNedYz-fYU7K38`wX+rz!WxLnxj=fApW7kY=3y^_OU z?v31RaoFeI5r{s!O>`6>1kX+sBz4Ifl)66Ji-rH;R0ok8{OlUmk|cFy7ao(Szn3I+ z_z$@)Z$?J+2=#3yM3nIOIk8*xQI}*lCt=K@{Ro!=3B>@A=z@mo6{7=))HNBSP<~cL zOuRWE2K8lc@?xp`?XcY4kxgWR?E*8DF5wB{AsnCs@N#=V0=FiHslHTPUmll-j@fkbXW;{frX|k;hMz~oh21}#G^E^}QN)j+ zE>TVT!`~lJV<5QbpCKk=W=Z`~D+2dQ;ylj|9o^7F1Xd<|eoSfy%`?;> zGm}p(H?85)sdzY9-pC|B5KQ-H=%feI!Nat+*S-(!GBoZaP9%*2EUmI;#-Ic^`ZkDI zUPUuanA!5WFont;crDgY<$ZUES z<;tg(pcvNN$5l9*#WeHzmG!#Q1J-O<7fZsniVt&zfB|(jA|p5IBx@A;@{x7}y)X$? zdie@+Y;}hs$$>*1_Qm|YcG{1S4-n3{5zlu~5ziXv75ddoqgpy{BTiX^|YN!}N- z*%HR#3;BlP*$1{tBa|7=iKmDt<_W&hu7zW7LN`^lrxA ziu4GBsiRWP(5k#^OHaa@>D>E44|QwO*81s&(A7PYP<nB{9THH0V{!RBF@dwJY_tD+S_qf@Ri5Q0pOj)-2Bz^zCYTS9`sA z1Xbu}h}WU`5%5qRNG5Yr>JQBE&kb-U`I zOC0T(J9;^TaY9qoL@X+^aUPWOdvB{3;q@-ay@+A%98$K#h(7ZV?M$M(l7-=VLb0E{oIU84rOiEb!&u#wW&ne{Q*Vw0I{kfs5!L`U@l9V_2| z#)?WM%CBqG)kz(AxWx#$!jTW^%RQ$`FT@3egY@T%?HYVxP)nqqihhtt zv@_02MXy2La0be|GvLwTI6O>~Br}21_)U*YajCCxALKlj{>}G2WThLyPp>wVDXuQn zUw9w%q+aAj0FdEe$ba{2-2;}h`&;tIe|mi;mXik{KiqtWdL9Wx0Gq>MAII(0w4u3M z)fel$FpJO`eTS*Lt}6j>d9Jm3t+w5hiWnaZ|9chizX-RfUT~?9-<>TSxc|{u>VLNy zR2-aLrR>cezFnpM9}ng46+uNjHSB+w&TI8_ZEK(fC55f*f4#sP=Z(A*TPRmaX#e2nWZfk z)dV`>h`_3fhK$!UauFXE2XTv<#ZcWEs@furMT~QnnGhchC~f@ z7HsyQKEDl?rovlV(esakS@Ge_l`U2XPRO1+(PONbcVIwc>4&Lg&M#sn@11j?=?dFv za=a?KduQvAZg@ti^BN^z8fTV7zm$lj@l{xmEs4raBI2N#r);vfk(CnXw^Q|`D?TjZj7ASSmOVcBGeVU89v zip$-kv*tm;v*AVUWH^#Y1SxV*L7vmHwaBLFV?nvrtt#jYqVaL^4xDsSY%?f8G`K)W zw0Lq#+OCr(H1B?9pSN3)x$1Dl8kL2kLhyP9qzg^5+S-%F<;FiW-?~R(ASBJ{d$W+M za?1~4Do8q7{D!0*@L>SnPmPOFLy~7wDY4j$yyXMR)l~jSrdi@J85hZdYk`9C%AL88 z^5Zq2ZgH2C%s9)4!Re#KYnWNm08!Es6$hAzNh^#|xt~0%l2hs}1QnWbOSicT@FLY~ z*yNV)IcF6+YS11%yywMIc3C&m_bRL~;IO#lrEXK_PV4#UvD=L(*)77Nnj)$sjkH5S znwk|qSyAp~57}UzXtS?arqNfaEZ{O0eieqQG_|3t9E8H+X>ptCSk#;z#zeVrGMKpg z^-q(9bLtCQ%ZQL3#vYAiTr$f<8RKpi{%}B~Z-wi2|0QGXyAM7oWsmhIf3ryfYz{l1CvOOW>lbn#b7xGmAr>D|ahnjM zHi-2GPpt^Fh*ZK_NU`Li~Gxgh@|V_v#^HXKj)5)P;58Nx@b^*hh5vM1fTM>uS3$ z5URQC*m0om2ixg==Y2UMGs9%jF|IZBBl+o!&OD{&evPK69wKEo23+}M5kHmBk(j45 z=Rqpg-uNNNJ@?Q2*2q!m_WaEY{o!YPTu}=BmyJuT`2-P`qxl4h!#T4hfCdxNO>Eo@ zZ)Tk$gZ%-feUVi~u!?0VVlM)c4-~?s9&6J}eRNN}`!n&!B`Llk(INIbILb4%___t> z5dSFpJE+U0O!tU$`&jQF`7>K>qr)xjJ7z$`jbpN3yw0V>mU)^}_~);_b%odP&r4a$ zI6%a-tBI`y)P&Ux&Btl+n=Uz|EAbBwfboxQSGLq~_OQ=p^+2WWI5190cMC~>q$yjwp~GhJCOY{KKyH7sYDu^?e3Aps{#PI&eyOFxd~ftKjaAF){v67syQV1RuLL5Y zqp$z-g_@7zBsRzQ?V^_T|FvB3pWf|%y0+7P9=@t-&#xUv(^(naA?Tp+Y(L41L6ynS zLw}&kf}&~*ju2;%fhA{fvJ|wpZnxCgIF#EAsHxJep)x+V*s84Vtn9e2>aVy(U%X_$ z?{rrX3vKV7yh!Y9z3lXy_cc^{bW_?#PnBnq z!b;v|l#q3b0^2sOOg~0l9D%1H5RJ9q7E8s;Cb4>_QI4iq+Oj>Tay9YK*hgdfWo?PsxS#0~$_x@p0T(clMKD5j0QPpHWXnR2 z=|@pLU;B9y?IzKz+eNb?0Q(uq9{QDigt1;Xcn*nZAA!6R{3kcG;2J#%$vJ9{aTA_A zrcSmo?Wes0%)8c&~R)~)Rsne zhe`8nDU#PNkJN2DrnP4RcwC)w@H)>*buJwvFGl3c+5^K}NsR)~nBD+w?OIz0=xqyo z_C=HS96H4It0M&Vo*@*sC{ATMTjcoiPHif#Tw)wfT9>C_tUpe z@K2CVUP(RPKj=t{(@LyJeNNF55g0@m@mwQ^OUn*TYrc0pogcY?i|hj=`8x@YZid{S zhGWYLor!k%tc?_Uyd}{B9V3eeIM(lUPIE5ucf6n97xdtI7o^fHS!YmaZ=k+)hFKS( zakid!cAu%zEngVdznlNgAw{xNxUk9Um=fZVeV|w78Bf@w5YO)z)pvSOXHPM_dwNf- z$L|~!zkPsqR{`L6iJEo03(Yc%WH`G|)%no#-NQ6EgfRLVqVGldB6IzBg#J}Z`Q;Sl zaQ4!tb-)87Kc1b$Q!-cL+GI5RHdHKLhp$x~vU4GSwp21X4V2{In<$J9L^fqtn;cI9 zGX>};V3-kFkx2ZW!ish6P)#e(r%?G| z`qI&E+cspUkW)@l7Z=pJO@O&^El;0EMm&iQdvnR9EpFR(WEV^ZyK*f~t547bG0CJg zxp9R@xwv`N>kyLs*8`PRlf0~I2O=M%D_J52zNgr!S3QY~W%FAybZDzuY-OP&ZfEJj znfen@G`X*8!Q~ugo4<)6-;pHV69uSp>i|xZgcytZmQRXvY|1{a%W~Lk+u~XMZ93vE z;}uRT?~O~7m^bSI#Je{om#<54G8ZRkq~UE{ny!ncnPUa0{jG1Ybuy_)m9~>V^O%U0 z?~qCZ(Q9RyLqzc5*}%qT8^~wSCAbwQ>4gr?4pZdRMze)sdo5HlNDqV#(Z)ev4jZgV z-9dCv$*OClUfLmq6{oqAs}fk#*21&S8R?&w{(&N@e$a|su(COMNCjr>?Tbj2hIbX_T|-PW4u5b2KjI{CX5bEw@Weg z@XUtwU)lk48oTSg16ca|+sKNXp_$!*YCH9L*EQiJsA5Y`hE!WS`O>|PYv4aONF(xn z_E#tjB$nb?!WXwf1m+zrRGJ6ykNXcIGazY-?EI_wn>CUahi9LmKIc2&VOe4wFx-!u zfTsK=jNR?4N-BP3vlJN?M1L|Ccv8R-bufWY=09;Ws|we5Lp@)XOORb z3>z}RL9d?Vv;r|eHgFcucdhQ#cM^B6V(6q3q64Cr-RXB4o=*27Op@R?UL z--+>>90JSt12-(rBm)<5;32f2wr5$(jSP zS(k9GPKSxpN?YT@gae^dNt7H|l{-|IZStAf@zQ@uAe=)uu!>&Ja2)WKOnbnRU2;N- zt0?@n7bY=dOXS=@e-I^sAoH>UbFF@P7ml zrAVoU=#8tdgSnDt367Hep#l`zR$1iKMN`bwZ=q$zMjawB`rCa4-HMCQ%MqamxRgkf zjubVPqg4r{l8vEwFxT(DiKXhXbYrv%C?LLo6c$BZDlL{xdnft!0zx}mV8WTM@kW0X zqtyV^kz2*(>*FM;JNaA4L^InL3taSEQNc8_;0u-qqiH%C)i>q5t}RV*o|x~F$s9i( z6}xi~yIib~KSjti88_93Jo4Ez1Qv*C{y|CraRqXV{t1GWAYB<_{a|??an8s{knjz$ zpKuk;9Q1D%RaHyHq`4E;ZM@=W(PzE`2vY?iph&yJ(@rwrrhw3!@iv=G!2(h`9aFXl z@!2wK&VRr(xA{1aoQYGK4p~hv5r$c9s)v3?Q`46oXXpq+Js1r)BnxUg9CM!uD=n_Q%25*bF z=9jZ}y3~}nbSn;`9gmnrbmfFuSIs6X&Bt~aK_p%7rV`cmnAK*)(ckE32R$l0*E3$Bv7M_yCUi)z+ zqZ>aRpqmN*oYt~?7Vk-=<>r}7pJf9JMLv?~rJR2Us;~1eDp|hw<_^5%CnD+DO4U+& zt5`?;aUuFNA)f0hyM})4rkCNd{H@5+r2a5&zJT?Av2~8YnZ!@K4mQ@twr$(CZQHhO zYhxQ5+qP|6Pm)be-t(_hNAHK3srlN~HGNP2y01G)&+u&vQ-r?}v|^$9H?gG3PPi^! zV*j$lQO@wc%+ns%uqeCtDBE`ku~*w`NRNMl(AHkW6dO6F#r(4N8&6I$_}q-Q2^uB6 zI8AQ0BPaj-CE6hUAvKrQZ|qw{cxXo;-&^+xf*GwytQNSIq0P-x`HiHy<65TIbLP== z>goS%kOdcKForPi-0-iU>+Mv0mIZ~ZYhA@Qv%ZamI;}5b;B2XzyKN8kq>bPu9k~!Z z(rwP>h>LHwO5_Na3!^|^YECUQhYVcnr9k28p1=`0JvD81@5??C!Ay^3H1^=8#9k<< zXbq^yDaFRG zj(N*kmbDw$1Z)2iTFf0* znBeRQ)kO8(0%u+JWKNM>ov4pt84fNX-;4{2HvghxYP4T=8`P(L;$87vFjKb=O0{RR z|C?f~CLRB4Bmn`gi(z9};H%ukFC)zCuaj~rk}9O*tKc7|iCdM=qVPjn6KPM9Fnc01 zlQDxH0yZDShSZQnZZcXf5a?SqIXA*or94cEx%J*(-uODZ{hB5T#KwRV?G7B4Vp4zG z^o2XCAQ0S^P}@|WmYy)m3@p}eHz3K=*uU%HbnDfT7OEy}S;PmcWL{Xow1u~Y$};T0 zIxy_9g&PtXA7oiRY+Ew#o{~ZX`Ki0n|B(CP&jx#$Bq_L4qE>yhwSC{X>0DVjkW8%KD{vxV)F3LTLF5`A>eWKgg0hW-+qMBjj;>9jMu!K{Ci&RIgOGDm76}Y zDc7RNEK49bkEuu)V9*(*9KbRh#x%KP?5e$+&tgbCQxR_B83L3c5&tk=Plj7mSl_BLDhKSOwGF1S}-1 zxuFaog(OU#EY9h9grm6wkVm_20Zx=beIDzY{QkjzpB$jIW3vbGz(?MG?yJCO$?299poC#eC*d1yvlRgBx75 zvW&Cgi7GVMRiPOH~ysJ7&-$|2V7ST89ffd?o_;V$^-44--{o7U_fo z`Od?$^6~NHIN6I{!_anQsy-d|&y%UF=mRG&Qp>$HcDpxPh$A7ZnevUu&5q1V>9u`c z-UBkJVBA_~7fc~pn?j@8Ig7luUkChcKz>L0XLaS)*jf-fSq6i&mYDKTGqxR!PVkdW zmQXJl(sm_YZi{$+H6)YCD#a<+Ve7s(Xb0KnqCjwD0@ZFn^x+>B-Pa@^>dF?>QoTfa zhmB2ec2XK#Z#lA;W6Z(gI$yb=E=Bd%ALk09@afAsH1YqetKUt-fHTf0YDcDu0DztwBw;tsJ8X+nH{U{kq%E<`6#leplILF4y=f(uKg)2 z*Q!9zL9g-y3$FOs-yxq~XlIFGD6H5}Gtkz}Gaq|A2J7gWOF$owhVzW1%i$l%7y2lk0b z@o@ol6s=9fN+`CEV$mCx1F3uf+)6*?rcXTSrYP7K{FfTKZ-X)Z3369J3gP!qZkSJ& zg__1SoLmGEjeS!R1oUP3HS>d}E}!|>=`l>0k75ss@FMi3LK-qwR;&x4kx)Oe4(nm| zfmOICPWaT5XE(wAPEdk4O4gA@v>=VB@k^9eihvrHFC;!zvc{;)bHCm_dtBBY>w(s< zhHh7`(g4CV%qJW&BU_451$lJXVZGuz$x8xaxC%NYrkPx zL7{=NMrUXTPL9NM!26*4_3WXCkUznn@ksx8*9$qP`5MHMBC8woK^Txo$e)&v@t_RA z3`YFeQZg8uRRljmFIMPD(MM$Rmsbz_w_y@+GxBB2ICs1n_6Z@5-+#=?Trd>5>!9NQ zeYA?i|1F*V&35E3agsAlRnyULe#icn4c?pWj`^Vd!V@g{fo+TR(EGrx3;LuxeOC+_ z97!}(SiZ=zR!%!w3!C&P@It)es7RuxxgQM?$zOOaT5u;+ZD0{JD_qoZ zOD9oYRSsc!2S%a|b5DeyvdV;kZohsYjvBat+BlLHky~o{1G9IXwzQy|js=P)$Lh+% zx3kd9GPGEKAZx~T)MyUVJBlv_HcHS}WZ6}z7431r(BBiW?3r&BT&w)BvRR#On_pdVQS96r5arHjq(g3Tc$?3>}nfOa7+~afa+6Ha6BJ^ zPzmn}o+WD=l5@LxMZJNspznFE~Hg4up8GjVd|BI0j&Mw?G9Jx|?sk4fenGBy{SNm~v!@2An5mZ4D%G@Dge z6T{jTvq3L=Z%E-$6_S8IhyIhuHue=|iy;C74d*z8sIfvS<&fUK64b@6u{MUi2C}WL zjU`@)VQ+>yf`pGOGO97%l+mk}HBrFS)~A661Ko@w%EiSkV7MoCbYwuGQQz@Pyhg&% zid3t#HVspo@ef^;YKHRut1{H5V#A;g9X_nLx5ZM{jhlOBqLpJoY-D>s6UppnOR;8U zf$CFLplXGY3rDcm%-%-Sy2ZZa*9eTbqmtPC&Q8#O)v|vEYum^+JO?i_DwHj?$P?>p zX0h0U+2NK@Fuhdiz$RP%MP(6VYEs zEXA$emF+`2QIw!UqAAj!V{Y*{J}L7I%xpOiwwBkpQ_|eY#aD7ClzSaN&I@QQHiyBH4RBO?HsG}@K{SYNzJ~ntEYmFtz1SuF>^<$M-Jfk^G&k_ zo9}kvPwEeARni{*W8=?60UPx(AFO4`*t{%K#wt>#@)Ai)T5hq!1U$Yp)x0M_x2bN? zgr~>r2a6!I|abNHlx`Gu;op?17kWB-ZNUr zP|)Pf%?`B{gt9IzvGQvTq$CLJ?t9iO&aqi*!`fE0PS3?H`x#k`T8LZ3 zIDk?2j?~2IG3M%afh4x%k7=1;pB>@E4B_{D-16{k*N?m+Qj?gioXsKgoXV8M>4-a&}1OY>1p{EuHPwMjs!=$x!D zC(R_sG1RPDbEvSkic(fs##km5twgh>YLzCLqiXvs@<8N|=eF`Y^^b~6~vX&q0B z9Ghuv0o$!Xui3TXaJMS)(xdFJGCP^9Xo&-zD}G+u0@ea;o7h|pNI2RV7Au&0;OPo; zdGVWUutc1TJ1crGV^mwbixtjP$ky*Nej_Z6& zF7MZ|x4}`~>nC~nUnzk*C$URa!M#l7eo68^o5DriR%Y+Z@D^9Tj2BmS!ev@6R_CzA z;nluvbH?d3xxGDx!(FJ<^I&e|J8hNeE3{R*ZASY+u${gi2+ISz8z+MXx?HjTL%bq0 zYkBP8`DGPt{{%YfLWjTCf4}d5u5E%8ZA?MpdFG3B}a^ z^B~ArS;P!$MuMv-uGuP9vpHo(m<(!wOHX?#d|0e9i}9?{pUBm3YnxzkFPgnErPI?u zp{6G{#M8*p^mlGo^?7M#ge*uLmi*S8SPGs(=9$bs;bEh^wx}TW$6WdZjXZa?CNs6` zkCO3nwQQB7xGnApIhqBz)dRJw1rylx)Tx$is&4Rp4!Z<*vH7#PzG(|{Iut4*3`eCbxCXLCr3CfT}95`5cP|^EuNSx+2OE6xiWgX-0^7K z%F&=VYG=-YPYN-XVl>Y)gOv{0soeccq_scYVu#jj;tn1-eh(lrpGOg{Urd7|_ptFx zWfDXz`6eii;B3l9m1|qF!Z$IlGwwq@Y)S^(IxR#&Sk1>`JW zUJkqlJaIOF1%J>k$K_fO<>fe=1Z*Yg2$5HhKn7M1HSuV$@H1!jb=yDdRmpV>oG4A1 z^10qD@;oD4lP)e)#^zR9324+l*SH1s>|M5*@1KJ?J-ztJ8vRU1w9OW1r1Z0kmy>#} z0ThI#0*ORI17+R8ArT;-^;+jF@v3sy5PFzKc&wXNuHj*=z2-oR|CpO6&U(eCu(f57 zoOaZfDi5tMdCIR-M|>!7ZwRwK{Mf{+jgqw}?h&XZJe8r6b=hRdks|V9HI5c#(dBmoOnsZ_r1Q(L|w==DwXI+ztUo)j1L=A zQT+LfeB;mGkbq~+oaOJLE9n8kKGBOq$ioFolirqeiRxzMU`!(1x#-JWAA#tn6{3$O z@gr(N_GX2Ev~d{G(aKPhlHuJA%Q9gywAVVa2K5Um^LK)Tsw@yb`+DS`ZkJjC9}HJF zfX)U6`+d1a*2=sLqU+S$~y7f9Rt5{uRSTIv$ zx{a>-%G0-eU47B!HlAT#wvF}@ELSF$mW3kLx@Y{8wuV$BmD#_!hyn`cBGa1GN&DpY z__3{B9(si;{x(KFW4Sh}5EJti{EZ;0^O zwC}O6zy}}g?|d9%Wt7(5e4g)74(YT|3N85r_1ZOv-D{Wz{J8Or9<<@*A0L$DQ=9?> z3E59YxrJi`gh!A-4q&gu*ndO<_G@J~!spif67M_LeC~mL7H(|Z5I{Kz^5hxyL+4{i z3~rHB`ceLy>!y`HoSh*4+b$te+xo+`YQ3xVxEHTl&z+DuF3r%lb)%hSf}oMTUrBTc zEhPXgbdJ_wKZvtV>Brm#Z1kNrWcx-VcJLjOQTvA{`;#XDIb(L%vDxb&o{+3nFu*gf z!kh-5&TUvA0)>j(fh$kjofjFA1KQCby&*~toT&r+RERU;_@g0|4v=_%=oNUisu9t+ zor>_npUmwLQrHs%l+Dn=V-4o$?U>nLonY1yKI)}&VvY*bXovGYeMv72cIFGb=uAm!FyS-!%BnwBMI`WM4!{G-UN|#$tVK4 zZ2+PdzJU6iraTYak1a9XW+=oKD;4qwZ`{`e>mIAN6`o}*s9Ml-Lui${DI2{wHLwQq zlB~E2Q>?)43Mlep1oqv&5ioxx!Y?cI&j))UhWTeK-BD2fYu)+@55t)qC!h#3dXekK z4nsevj%*9AT4V=q*dA>7%TNJBro8<&NC`OB zb@=1O>f>d3-58f@nlgX@8uJ2C8O*GOPoAx5op8%ZOW;ygHWH#z17qt zGBh%IfmV65kRICNhqdu_oyE^RG)9@HUPTfnB(yZB$?4v2pVuw z(@0}x;rl!*1HaWEj5T;?j+Ic>P3yyO;WyWyoA;~sq*}3Rjji{)8|601lkq z(cRI?jKAoNZVk1^cmA%eS3d)%O}YE`yE8UvBRJEDU(~(5vcD7z6Ps)BI1x15n*O;B zOxq@l?wmhq)?49TZ5~#glE8tke;#AiJBl(p!oZI)pj8;sQiQN;ZZ{8^<`sq7ay9^Y z)hH+W_CaX^xl@K2p5VJ@Iv)NeNFkBjSNH`(6O=3N_`+3?U@NxNx_^ahU6f{Dq?Qq& zWePs>h0$FIy8#aHG_Gs~a9U+2d(X4wQ~Zr?m zf53}iJ8^d#G+UtJAk|j$Hu+L}L{FV=U2ez{=3TInLui<3P%%GszfCB89iw}TRC+mf zWc}kq;Kp1TsGTrtPaFy(oQMi<;7&-q4Y#O)9;LxfW8gjSwhJ12M!?qe(GVTJlu>qo zL3MZSzwSO!DSkaKo|wSm1t*l_Xhey-(Y}Y|?YFCRcfw8EuPj6?xGP!jaY|bqC=(Yq zZ(LvdOLf!Hwfx?irWXpz{ZrQgq+%19K6s%En7ec`!`KC>L%1YOx;T-+vsozflEic1 zg8w=we;E-aXD)w~T%}#)igSs6_AsS80`?UN`jeU=ts5FyF?!mxPLTGZ3iS$?BPdgD zPe;UoD9JkA$@;`7J3IA*8N|#Xm)kLEyFRDWVU2`JeB+2SsWUkT56c6M9cjx%g|EI8 zLe6f;F<%H7AHNh0`=kPD(52jv3!2qAqg)w0?K*IA>?lPm?*2SYA|Dao8R`Jy${FW2 zysQ>C#A@PhS~ zVr>HoNZK2a^hfo6JIbkH?{kz8QzZCL-?LD#@GLbI{x*nNStQ;(Fd92kG#-6OBp#vo zjj1C3r&D%u$U*Hx$CUa_ZzVlm;6Rs~?61a~s65;yjZYDa4OZd`0CITXVfekOqXz`$ zLlAQmd?mrYy3m0GZKO#1Y-o5ILg>Q~_BrmHZ_ey%bhHfCD}(z4ytik}EUtI*C%F&4 zp&t)~42DZu3KRNyy~1I#-q#aErwnd&ocG)TpsjhDL3PBlII_Z`F#WO^ll^-mD2-rzAuW>rz=x{ z!`Dvo^2^kVUS|UM`E>_xZ6|}f1e=c7)Qyv^2V~#iUOkrAf%HpPRO;ZuuB{Z?%>(Di zUT<^NRf(DP!DuB`{cjVK3OA<(5icH2Rj{Ypq(MZCnZ zNnCvQ2rWKt%Y*to*SgAND(7kk=CAWvV5L|`XA$;|55d>NO>w&JIQ=Wh4*;0nUS0VK z$_M`MJiVBhd&REOSu2?o5 zJC!Us&5)Jx#rWqMOfvFSB@$zaCALIFPNt;N6!;gqOL^Mv2XZb+p))PHiaA?MYg6mPi&_0WKl68jHTjN?MKex=J(3ubos_{ zU~)coJuZy=)kaAe@g#?l%h0;3g~s^p?iTv6?do|@=WNXCmv6va(wONKLWaRr>hB>C z`=G;tAfA>$d_ew3jsp?=E}0RFvzhCFeS))&0%>IIfBP<{O^p1It$B!&$)*l%C zvB?1ay9Ad=Hc*H!7(YaY8ON8(h%z#TD|q%uRpOY3BXahj$uWygOx!qqBKax^c)~`4 z13wCY!>4X}8os;T^i*GR;PAiKh|l>I2+w_rb{_eQMFGMecDnYRYd zi3cFy5v;vV?o5<%+&!qkApH(WZ&V=%rCfpgVS@J?OJF`+OO#J0D5Mwhc7cpM>JuX9 z!z}_>>w|6;erd*(%Z58m`fC`%rL2{X5|J*L*r!uhrEwP)A?iO8Vg=oPd#p9b%d*{BqhMrdvBP|W8BJ)&m*(+HZ_BwOH028 z{cCCup5wyH!5F{_-f0}4RU1yK;g{QQ%rHx%GR~ukbQ;cX^m#|7Nwu_LUveKJ?l!oI zkf0XN>J?tB>N*d0oAune-wkmwfmbfP=d5f<4%ZqJK>2#*@&-@(0Nb@dzrzfFyV(7H z3^lS3l`=k?15j9)E~?2{xrcfFP!TOsq$)d&-~LhrJH?E>WfWhc2`i+7d>z9 zb}%2HzsLQB2>QhsMxD>p&u`52odb8Ozcyk>5z|W2j07t!R2P0;zGq$E3`3@E^Dkt2 z#nU1rLYLEKL@)hrFaa?l5RWL+j4Ot5zhNKao619`&jT!8@7L9mmwEQesMVCzsOZ)!bdWVJ{Rn zf&pSTC^y-;UuRbSz5G((7sz%R4MV~+gdMEj;l?`+wvO__sArC?}O9fxkI1j!6o^753d zv^EN{d^9G>c;E!d_t%4qBV_n=MA1W{H%pyX3#7U9k0~i)A3Lm_-Hc+F3}!c06b~Fn z=7QT0%cl#ND{i?XMz%zJ#z-4X`|U#_s@|YChIlo#oB7C%_^geyNhyDmTCx`=DEq7!?UET#{HiY!gY_30I}v`_pp}L&`q8v5ZNLuz zE}zQ~Zbl$Z5NSG(F{T@=E(7XtZ$qM}_e81TZu6!QZ#L_$|9X&FuW>4dB71enReeA2 zNhHk_&0n&r4k$TLwa%wXMl}JzcTx&}#hpx} zk@wWN;MPmBEI-)2p_^KN$>J=6eYeAgDouW)=`zL>fA>BWo%)CKtEd}nZzj@qAQnb1 zte6saU^FrDfW=gW@^lcmgL!3G6jwG*Tsyim#?r-A(x{oP!6j^yT^r6oWL8bAK@nFq zKIS-=ouV+fF9VsXgQJ|B<=L2)YN!>FPwq2zWGbd!kKY)r=g~BMyR`mj!dIrvraDgsMP> z7h-upl|ft^CVGIyp>!3%=1AKdtuL&igP=VG=aAM0H-p%@$K%MXLt`g6?MA&tVi(SP z0N*CR1r|@pn=jOR50|p#u88jw!O)9>(7##;(S!Ip@b`dEjo`-Icqq79iKn{l^pzq3 zeNbTdjMJfg_zZL||HWYKnc-rcjU(#so!U_+1=wOLycVFf?q&yznJH{=KV717OF@THsnlk5hr(0{!zEBLvgw zgH9b#orPFVyD<&MTuMc)Crn#)SLEICFk)9lD#BHXj~~@SC|TjP(J=O8$KDwho0>ZV zcP;QNLs$Sm@6omiQ&^Y2CozdGGl=j2j~a4j5bOd{9*$}Y^&faxtOTBKZ%^(}%~1Yn58FT=?t!x)PnRgefa?(3HyLdtBWRg=IV zPKP=yMiISNhRu}meM*7$1FEELTuLR>6>}2UO^Ia!9Rg7^SqOnsE-EE2sv$S-SOZ>k zJPLiM{Cb=F!58>MG~OX%1;;zSA@G(H13Pfer$YpC(ZfFxjazMo0(284re~nXWC3A7UH&@gu^k`4%JSGr(Icw zaH|>ei7Z5HqyAJMHDafxX{|LQNp)GARLqe~23M9cj3YIn%%eq`DXs^Rrxf43kwOhu zpDv3gvdW(_a$de-c}!G%SfD_MJ{Z^tZZ<($ud*`Wq-kEKj0-uKs+*tyZ>*w1rr?ZI z%;VICi(seu$GL+!6Wf`hZ|P=jGh?U_4OcruQQ8wY z+i+wGuOip3>{U`k>Tc+fC8ii#vAhz(UW0VCHu`H?3& z2P>z92q~`hTZLKJ4924cm3Zh+jXaWr+_z!h0j5(zgt=MK;*8l!DrLIv?qH=`I%z5F z!rW@g2i+>VS)< z??B$%`b&nHwy{&6i(j@;q9Vfp%3`rBN)BJBb%SKt#0;p;-|%VLYa@t7pgyJ5_A?#a zh2fKB=BMgvu*Cl36^pgc?&_=*4k?hh{u#|`S*^bne^+f+5V2hwz|#9+wFouq-oj~% zCkyG^ED*G1sL88KvuW2S;gRc+!G~H8T$?r^-2B|D#FK17_jTHUDrwNN;{71TbpI3) zPNw}O`q&C9-iTG(bIYK;4e`W~WF<7pPIw&{n>j6HZ5Mdy6MfKYliESxJ_>eA0}we- zw>ul*A9v*y{mfNw+NAP3{Atj$q%|XGuu0M%y**O3BAzLk7(E%IGOuU1zyyi(ugnas zZ}|hbsSP>L5cYI>C#f8kXi8#><;$rYMEbixMyE|i-Xxo#i#E1vq^^UD#Px3lZQ_`!WETMT;iO68E}cM~4+8Mui9w?; zaL@}`@^BoYYR8;F2w)t9!+q#4ijXnMv5GgS#3hkG-fq4}{>bTjE%#{wrJBkSTzGqV z@SeJlM7pBH<^+76f-egc_|^%e@hBHAS67jXE)pN{+Yr9SLBM1Lai(Da7B9=qQ$Hm{jRi+Kr&VkogH3!;+r%n3R95-OM__$GPRe!K7?%x#hx)OLG z^r8b^&@_tJ*iHKf@1mZo!&oVLL__wq&X<>h4+mzbne~v<~GLjY$w`p+Hc_$DS!vch{vkcsj zn03&!G??ZKks=!;&hP%F&3l3*7IrVbKG#D=+N6lIHig^T(Xi@P&`#vk*YMh4dv%Yh zSvqwT6)zQ`a&@fBhuMcKD=lb)U2%{)T~|xZri?8_0&-HuoD_*BysgS^D9wkKQbly3R7GTq3?wIPBdBq3S+_3*}xlH8W?1-Aw;zp zWtPZ^e zxLZ6Hh-+KW{}G|FbL>~+4n=Nm8;*Qyt`td#OXA3ac)Pt@mrWS;pZ)$*{ptHsylpPO z;hQ$%+OPW={|R*zYFHo4KTbKzfJk@be;{AwD|LErUY|F}W@e_Jov4jW4$%C{8CDI~ zp0}j|)ri_SIdwIy@M-xasha=W2EJYq$Dkl1)*#hZ_TtR*D`;4jnFbx4T>qs$c{)=B zjZf5eOUTskLe4=^(>9M{2R_u)y?(J=!fy@F?Ol#yb8zMl)~mdG#JLw?WeG0BFLJH? zpq0LA*OPIsO~5t;9sq-bN9mjz7_Zqz5@uTq6w;RT$hxY(2p4Lwg_S{x8)U@dG7%;& zOmhQVqlC3z@olrZg$TWw5IqCikXwP|VLq*QOn(44_Sd-niE~3Ubk3*JSJZ?`ylMR1 z>fO2!MhW~c4u)#nTKHdQQ9z6fEt^f7wI}ga3(Qo$6M{P4^VL*g%Z{U9$e)Y960Gg2A{t$$&(q_-sR$uWr~Ik|p4H+0)5_W5NZs$5uc z0sz3)*qh21q&NC8m5s6 z;Cz>v;AJE<<@!cHw!oW$?IFD-4+WkWbW;k5cU&%Aky}u|rd=QY0=WNi`a3Ee0ARl8 zT*G5`4{x=VR7`2}xh*&Y%Ye1O!8<$*t-EQ55gd0{0I2iL^J$@oAk@zrU9f9Ur0Y^wCJ5YQ2w*xFfXkdQ#<{q z=n;zmkMQW;iPn35{Db}cG5D`@5A<)HFaY|Yt`S%^{VmTBF_)+aA8_qA=>OgY|1ZQo z@A@E+&<`>=?*|`C@ZX|iRqT~roJcX;2R!;bhK8F=60z#>*mGPLDH39n%dmy^Z$!t&$`#1rsnFa9SkNByz|!=r zss!1uGBh6M&lBP~_wN{&#bS%gMyVA22(q=p!@eP}em`6LhW0v zM2_jH=tn|jdTxyU_(<54@{+bBv{Wbz_UEZyW89dh)qYbaBdYwFvbL47{awkt#D8wH zLW2*}hF71)pEB^9Dw!IUsqXnh8vX?eJzCV!L7+`%jz4g^J=KBjww5QbKIBRv?uXej zOpRvBYu{jeN?JY8(rr-Yx+&cn)jENOULV`i5e;``)O6$bf5iX4;3i_a*+U8tKtKxc zKtTBaZT!^@o$M^_%>NU9-!yd>lvT{pZ(=AHQWdenN`m>sK7wBnMv+WcaYGsNQ(ork zMwWw8G)972fzaI>k%um2?&OY0Wt>p3k&X^B67Eh9IpI8~gtMMw+D;F?yLy{Kzl)u} zdLKJ&Z+4hE{QI8afcc~qXZ@J`k>tnKkBdux@=9Kv(lpyr;%Q(IxR~#dn~@A{ ztd;<+?AJDX!94e-;b_yF+MFawZESFFoa)FgYmBAwq#;Z1o3WIPcj%$gb=P3D`ud)? zl%sYaIasy;#I52NocjzE4usA429&{gG&rI+p`Wy?J7W2 zH#)R&&v4<@GDfs9^hpWbi=5+Zb3mGAk87Iy^n-1a&j-ye<2Y`%0FckJReG%YoECkovk>^7xn_CuadT&67p$6F_`6LtZ9YZf z(1xecopzqq6zqsnaj{C0oG=IYE&miv^XBsAUt>(8iE#y0K;5i;EeT$3-|t_Y3Aho> z8lMACLa#UAtn#v9O#daef$bB=e!18^1e{XE1w~Vt-&z^$6^()T3`;8=Mk0GWPS3}kZDRz-j?U$h>zRo{+ry>55F)Q^3o00nZ$LdO};eJ|>=~)(DIG_L~nDls>VB;6dN?*P(0*kI1<#6DWQ` z4z;Uv`M=4xa=YD7B+6WNeACOFft+xPLwM47mp2Sk)dozyIC%+PT)&C*BXoVrf1t;g zE9|?2Tio*JdU5iPTlZ1T09X`0X-a0we+a%v@6;{3Vw$BrT~ByRx+)PWX^~UN6O_yCXc@z9FvDo)VbQ~B8N+FCMzLn08p#?>nv|_7)x$aDD^#sh z)kxZOtpn9)+LWym)ri`(ts~W_+SDs%Sw`1Jms4B~&qlKu&qmloY*~-PTtjS`Fhg9! zY}qiwtr51Y=#j3Gw#?|!w&iK~BDP#Uu3vGr*NVix2}q8jq$@%4cW{6Y|hf+2DP1CZK~ z(L^qYXyR*f8c2rZ1%yKE1&jl9Ldqe$At#V%hL#~11*Ac;7O`{U8fg85H1Q4r^+ZDa zhNi#r>!}7FLzWRi2C^V62R0ys`*ktgcY2@digFu)mo19F$*M(}Ta74bn)`&QyIs=f zqa8TAbJXTobs}etQS}C`wnngc{;c?mVt>gQeD49CjyE+-0DjQpv+8UDl zPcO6ol;v1uTl)nC6hAsz>J+;}(-}r~*Em@Nsa&a;KpL7DOeBy<)udth^x8(c5*o0V zP8{i+d5f2Id{dskY>17f1+p{ao}PKmGu@|OUyuLL19VY}5*lg_C>x=bpdYAMt|0Ge zf;>}$W<|vagF@yyg3w{-eT%fL@s_f0F$QRF*B&sMJ1({KmiL1LSq;{#p1NyiX4^cC z&duG_{q5o2iD#Q(ot`{K#8WYrf;ar>NEdhPeX22GH}lPgW^V0JCqMeQ;XP6EtDiIXZCf2DX4`KZ{#IcN&$@M%Y{fi{i|roCWqY?8z?GY( zztSLf3Q9Al*skK;Fw*QQwd7eNdwCcw#wZTnbog*QL`Y6V2^8CK0l!WEYCVbbI?mkK zMs_!N`x4e>$;y(WhE#%+Y`~_BMLQ`8QisiEcCgmyw}!j)z=-f1VkP-)u90Dv+B)X) zPN%9raY8m?;VMV#EP-3Qs~K0o;py-NpJvzDdtO;GL1)`6HY-qBEb0t&9x`{Y%E%~c zBGM4z{il=g63X(v!nnQ;Qdjt02yM_e$=R=320$$V0T2!Gv8Ms!{D#}N9s~KE1yY@~ z^i{GCyz4egBX-iOMA)P2~H-xLV>+==MoB#|6ZcL(~QsFYay zyIo|7U1go!Xg}XX(pamJ*fS=>5z5Wcn2XY2S7c;WTLa}(P!W@pMrFYt`2SA2ekRUO zf*%EV^Fz~T|Nj->|C4n8>s8Fs)W+mLiTIxsEYbL9kL`~6opZPr=RJ*7cCOv=lI1q& zsCU(@8`x^CUUr>DYr9G|Ex`qj{&AqwaNE`Cyb%`**ZR=dKk}jBsR=HD3K>b5RZ7E< zkPs9ygz(vyE$~TBz_;7kbjYzjA_;khFZIP~BB2I?d5!x_~>Iq-mFrnLy& z8K|LCOCP>aW<(j*K_VB!xn?*#6W|OxW$O#V{V|#)ota#xsEzojQq(~vnT2VUhuNo( zxQ;e@&ZN+b+CS!K-EUI#p^L-KKpdoI?7|v#_0faX|9zy7GO8bS&_baXe=v!(VKcq3 ze9&gL-eIYO!$&CSB{VQXxLA+7>@c!yBx)V@N-hbBZ6%qUfYDw(Q+Zx!g|bi-SArcn z*yUWC>Pc=&YBOf-HnT7ZA1W2G){a?IEVagJq!rq%rPeN={w(w!WR{IeT^8H%D+Q6W z5WB-!MM$~PJclTJibs_1XM%HMUCL6bX=JIQ5Tm%IITWf-ATLK@aR3Bu|=k3K9C$=<`C&IVd}k?ov%l`R8W(7QL)F zdKOs<@h^PJAr{^E3Qcq?sj+HB3NTTaEaw&1+fozxgy15#)V=$oiH|&2O%)!llM%Qd z0Q}C~3q9`gO!}~J|m!4 zXsXFdakeT>&G)bOvim65+(Zg>x}oddLPvlLCv%#o>6*jttDN=R8QyebS58V8=@8DM z^Uv5>wUcn=wF&}Pa&n*`3cd3_4*dlcqg3(~u(#AeD{Aq2l-`te+Ma?l=XC4q&7=uQ zj@~eApcWW0l_SL^P85@Iq7c?>LimaP>`yY(GV{~R+w=J5AS+P~IZQHhOyJFk6 zZQC}V*mhE}lS)qBv%h`!Is1I)-ur8{w*IWv=9+W#G5hFabgUj)0v7AKH_iyB;Y&Kp zMoooV6crq$a>E-i;afP&Z|7fP98KTqyms9D!B!I3Dw&8~59NWt=jN_o9#Z7(awFs; z#$FJ1lWtfL#Ftu+?Mf*Y>>2U3FwLVM7|K-@G~CjmNfgbywaJRZzmqUsRF$$M+sh~_ zs!wC8+ej3EItUCVc?hLOTyJ^S59LvIQg1=Jh>U(-PrMNL5E~&~1_fHhFtaWmt<1>i zEIahw(N-UKu*wC->TEqWgXxb$f6~aySwXBpJt2iI+c)9 z=)N|0%I>bocDFxoRh(Tg4Ynk)(;GO@8~d$%6RDF_8k zamY^p8rNEUtao4p0S&+z@5aALo%iRGaGmT8J?=(h$N!Qxio^O!OOUWREGE1M%sNBt zL}o}4>BI!=ry`RIGinTjT{IjJz;Z;7a`Dh~h5feU9bDcZ$ z5*6&TZXlKJuZ75(uxUIEIHkqMR<~D&te^ZypBf+u7yjZQw?ojxIs?hk;@iI#GN1Vw zcY8mK^JEO8S>34+#>MMo45{L#;?K4`e~rJeas)R15a+~>{~$&f=inBh?Dtw1BP{hm zuhZx|s|s8=4;iN$caNMqzW6oHB5q4mJaw-f%AvQ*(^*0Hg*qcB>dPX7ccmlfdTnFP zvG6S5_A~P>FU))PH$00L>s zKN7<=|8XVnvDR;yaJ>uu5BU<3sfAVS--jE*78&9eyI9)1i&nU8b;4_WOH){<4~%CiO(tz?(K0^HRT|4+=i&uvL42A@SVBZKujnIX}kBkyE!Vi^4{HV^F zd1K#*?aC(JRISpl8EmOl4Q3_4eUk@GPT(f*X<{u7`$+AuE_}#>zQwA@4iT*nKc7uVpV&4~f%T7K>Kv&a;tZVM37&3_E5&Rwo@6BG3uX|j| zOpSh4YT_nJK09%qR>ezJdE%88H5<8!Zv0X(*Hg4N$(t3i#mIf#55wp?99ZjdQ$X)M z9a!sm0|(!8*5At9bvW31iwn22bi)%|umn&p;KB^2WhtO$ zM3tdvQzhLfkO1?g%wQgW>x0V%40NABgz#I-ad5LS8<&OJeH#1)C-3S zY8Vl&Aws(bmvf^?IR8%I8Y06L?IK!CJ1tYOZ=F2jwotg>Kc+1tN~)h_tmckwPkJsw z8h;1|;8~KJ+vJHnZ8v2OaS)ZWV3vz-Sig#nmW_0;hGCx-I}^*-%l5ZZ1JG13R?LSA zWwx@h3&vUY;20E1nMnzQNTluCTEk}^L<7a-_&&$E8#&`94V`}vgAxR4Vh{2FjLA(H zL#;6xL@}Xt8O;P)eVZgt#Z|r9;d*-=l-!)y7qeLn^F@i0Nf(Oxdmf!lKB#b#7-TpS zBE>9ZU;F%MkJ_UPMM+Y7(>dl4?Bvqy+mTeEXL$j_VZ?@nRq_dP*Nhhwm^z|eG#$b9 z;>6KEl{5#RQBCp0C_Tf{Tb3!5LYj3A*3Ylp)=#5YhuE+w0(V!255k?pyLDz>!;2BV=-7xd=SHGw6>*QMCCboEmGCv?)&tVB z1hO#A%QqY{eo~Ym&6I^#&f2`VWSSIEYEz1kt2bvzq(U?8HLeET$VL4zuqWfaGnwC0?xiJ+c!q#=^(0O<7Ck-|$9 zkM2)qQ&)-%4?|Mp)qf#@X-ygU!@`)9k^YJ#V(5H*$@sIGRH4PZI^e#a7J|gSPJfXuH->ZZ zLLGdY-ZYz^@PZwF(xs~J70aiBC?u7otKh(l-uVNkSH6tn)s|THLoq}J%cuBY#ZAND z{AO5Z)b4O`XI}-kS4cfX6}L7&nanlM|mHRk!i1y^g1 zsFH1{TvnsI{D96)YEG5&PKo@`n={h9ck@ZHAl(>mFoWi2+NE4LbRRbDSsp?f@er4v2%X+x`vu}PxEV| z*K(7C@y>*6LW5`KgrfWloG~vxMZ5~MpaDV2cLvnmL%f%$lAIi^ zG0y5HbYAFSl`#eK{ris8G;q97EA`#viR}*->?>-?c)_}UC-ldVBdm5VW!~34gr=O* zOegtq2N&*BULcLPq-YI|)csoOgY95;nI#b8-H=WrBM^9 zTdKS>Ua!W+yHXG9hg^P*Cfs$U48_+4vmulF|I#OYY)socRaZ15b8{$er_@-Dd}g+z z(LdC!yjVc77kg)St`>_tNYcPfBR{CR%!S&0!3hUsU9;-CR#04A>$@;u%OTZ!PipEu$rZ!m_k)5FAR9%kV z_}j2RgsRp2sKies!zSMJ*&wFey)rS}ZE@{uVhLHltn%y>Eo4_yTK?KhP&ZW51tg`+ zrc8~^Ah-jwa*@Vdp>?0L8Dyn$!@jfw*ERlkY{wv|8RC|$DPKZS@^KwDev`p^^hKK@ z_nD$sSfO*Bls%8yPeu2w)`Nx!N8p>GQit<5B1mRcaNHSQa$u!+?6mBgG98T4D?Eg> z@Zb8}QGZpf3srNy7T5gKO(0Cg#%W0pKm)YgZ+VlFUs^BYy_1`tQ(q1|cVT4Hs63IK z@c0tz)WA#x=%wE2;H&d%Xj!$svex3LQH(b*lq1WXL{V~9}GQoIa z1byB^ap9AC11VF6-;;i%pHYj|kM>B?Gl3_l!V^=*qxsw zu~G!Eb?w~xP#gpvL^4H0FtI9wJ&Z$`WjT1=4`Z5rfOy@@{PphKU?#>+ug@NLopGIc z{+5~dY^;5L_t&BS(2iYD0L>Gj?*d{l4vK;V1mmDFVhv-!5{L;u5D=_EN^8^yNeerm zXrvzntqvR$kqZMrtA8dk;vwJJSe&ph@s*jtctBp|7W`s7lT@ry6DZWc*s)l3 zADn7SV5ND&1&}s7t1^i#c&Bi51WO&HWM8+8jM6x%R#=otThHDg@+6k(I~D88$?{eF zqF*e`^HEg2+O=3Vm&H~?xMGwh>Qxb-2k@5h|7j6*28(=$RocLIqVi;;WI@Lz0loifo_l5HacC4_wPzc-7s!LzF?8zS;=0q*1EAhYjiBMuo0Y$ zIy-|hV&)+0TF*GO$zAy|&8_e1LcJgUbZ*YP(bMoZ zToUjqU`bf?S<%)?yzWi3WBe|$lYE1igI}eJuWp-C9yG#j^(e3pZ7l`+J2W#A1Ekm9m=Ju*JI#t8rzzm1Y{-?vnpaJh@x_%4M2TIPNDI z->Xqnt0%8IaA@WQsU&f5y-NM)&e7_vhjWmf^ttqEq<#-xU>~L-o!IXx`G$@A1^0Cdb z-^jX{6WksvX+3dVcPLS2Hy~w2WC+f#8O!J<_RkWDsOhMB zw!-`nLVarkZ(T!Bt{h`vV<^ZbhEgS?TKf?Iy~oh?w#hv3Xmn9zsg4$B!53vP(8}ZuI5bb!6Yt!025kl z*S;d(%-Fi4Xi6%Rt^JF~jCyntTf)Z0Do6_1tbhbK-7orReMUE4Znz~QAw5nqYr<$D zlJ0P4^!Uwbx@ZE&-OYOc%=%%Da+p@Prv)4nx}*>2Jth(%1GC$H#v^$Ncd8cmi4Ahx zT?`4@Oi_0CmLfct6eeXX2We`DVQyqbWWBNP0h^uh31e%}#6~^hzwX612^oV)dwv?`f<~Rxn9zoDZl*%KS$`a$I*UC#l<@Mu8Iegn- zRfV8b0~2w84}Lzp^(j&r@SMX~pqlT;-JJrbYDt@x3&Q_{bAb)!}UJ$BJm zeNMVjDT7^@40r_Jt6BA}@AmIvA8POlVFGYUjz*Lb0mQx{)ZEeXH`3>xLvJJVOP9XK@_Y$;9z?4Q$&!mH|rkQBOxf=7mX zLOLd6s*C`CxiRmE;fHBQuaMtfGu?FR0cMk39fLxcyKg+Y@pr?{^%TChAqFv(y&?-g zUbBUqx{AX{91RhxZ>iYn49_^U*HR%A<>!cb9c+j3!-M6NMaUSD!^m2Q)B5Sus0M6opZ0xP{Pn_kymDUN<^uu_pE358v zkztgkU2eSQTpASwrEO-`=Q8&R?*xIsO2zH1h6QGof4pf9^ZMc+-CbV3YvljHDGZ@< z4Zy0|%q8*cO}_lA+wd$ljyHHg(r>x^=dYVzKl}|t+&hk7>4M5DJ{H%xDfJ8b{|;w@ z%h(xw-*8sEX)`T)Dd6W>O++v7Pf1JeRlTdTRKW? zl^%Cf6?6i63}2y2YDT)0ol#L5Vm-V}ttF)i?3im%W?EaLo+7!SDaFDf!k3qklEU!x zY=e%_2};p( za398)^P5d*y^#_sh8~G?)v<`rs03M3RU)P$yedUj+K8slIvb1^6UqhvJAH+A(<%lJ z3F^T_F|WH&i}^kDXtmDrN-4ypVDZ^3;V@Nj8?a3TeyI@stE!C!xIsBv^^b+!#YW-L zr>YiMh|8bi6Tk)|5&@BC#GvW)=E}VL?3B3Tgro%A(J}{Xiqn|ShEqTH`$S4LhpE5+j zGI4n57R6mp!(C`9>Km1hN?c*Bhov~TA$x~ZXAns${uwW3UJ|dUt}#vxN~NfHZx3W& zsiAgmq#&#t%&1vIu3i|>VdR{ns5`3(q$3Jxi(;)x0?Y`ZP?2;I9;kGY9-zodx#^5x zyD5!mxoL^Qw?I*M7IT<-iH*p1O%4vC+-e|E(60zA4m2zur_g&oPkngASsc1ktZ@}fLFvHOSIEE|K6O?gn$Sf-RN2*MPF77mug{Z1epKhN)^!UwUJeYK3 zGKiWAv}6w`*KJ5J`xjS5!qUE|UTFJ#w-u7}9B4n7X~lr72#i2f7i52foxWRD7e4uL z?@~z3O_}wtoje#oHKIi7@&8gq_!ivTiJxkTi6l-Qm+zu- zO{K6Y3nNjB+zs`z`aovxx6;3iO{~>^9xSM-L+U-TS(Tns8z3UTN z-T&bm=bRKer-wDLLm6UEg?ASO0m+(a9;PQ^Lygz#1+}DxP_iD8{PsH#_<`&EMhvkb zgu4N})Djbl2Q}voW@s>BRKi`y3);!>Av55EBH**}mMZMhbPV^QGpOCHE-=swF8-(4 zI(;}NsWW?gEk_{h&mK?d1>k4))ji{&nBqQS;Lk?)xkpg+m&D^aG zsvnC2;Y%)f?oGbvOT8j!oeF4V!TN#!y+Q$=KrXV12L_M$%D6wm^jm5fpYo19;{6-u z@>R^_*5{-DNMFM2+$I1+y^{mOj`Sc}$(+-T@)zk>C!$$+WSU;DzGX*2W@%j<=ABsN zW}!X+ar#7ZZdO2jOr41E6x}HvjpgT??Lc#sWgC8HcxA{s0IvcHzgV9FRr9-gHDq{M zj9M!C+y!fd5vy>sCgrgd?rW-|H?*Hnfd+?K;lcO!1swku69n$(sKZ#Nq8$5krW3s7 zp!O|dfO7Pp+Yhnzid$mc0VSKI|6W+QJBj*1^DQ9}_?D26{+}51k42r3o%KI1b^l&J zf+E>^%XrdP?r8um-G6oiW4tzyMnG_rFWK-??u>h(xL%y}mv$7%j5)*mNepwgS|-Sv z(>Bq`@rq;CDaZ5rY*tUt7id+K1RQbJm|IL&P*YGLB!I?NSD1lJ>5in9TqORmhk6fH z$Xc(OZ5?9zi7Ui@l!sLtyx3wRM60!FZw`a8suibIZ+V90?;?3_=HG_%Hr=iIjHmF# zdjb{k*Sy!I4pozId2vtXtL0*8ypu`YRl4SQwfG*7ZVxS^FI;p2*+C zx#dm+H>^9H3$#rQ#cM-*?Il~aXr_Cz%{V%Z9T`@R6Mkw<&HnvPt?Uxs(@ipVsRm~; zDbs?fbKB(-rIF>PJIh`1QqqDsWCbi+(0yFUt06pCi&rZhlh-;fqBY+QbJ`|`1tmjg zC`B*CMW|S@D@fb)RokN#`j}kMcn1>t$8qv;!T|<8ngnoBwj8XFI#cMRi>@AKAy@G& zm%W}0%A|CYxy|P~@T^-69j!8nm2+({o|rYTzj6#X7 zvj2+#T%>inpv8iI2#vLghQ`Kn*`wo-b5nDcG037#UnQ#6A{|($$bic>?qxIJPgQAi zN6AW*bG8z~xAWdZMfT0Hyq9Z?)@JvNJ&U{j>p-O%1{Kqod#B*b8(NzQN@k{2NMliT zI-a3cX|-jO!8wHNTD8MI!Dm7*rG&gOqBx`)i|CBra`~-2LbFyQ=8O`lklVKM^d|q-bto^)*iJE1ZPqyxAqJVEdP(G^i2U$CGKuX_ z3o@R=JTMCMi-eFK8?g6bm-E`6`w-$eXaQP&&qz zvGtIQ=x8m&Vx(MFAi^UX>kod&31xbaZAz67nF(2*gR9(M&}Deyq-0DZzT$c;;Jnw~ zluhj-n(Vz@!}L)BqsRkpy9+TScN13ZVffPBOVwMQ(JdD-m3(dAx6WsuwQnK%u~U_K zRx&yLz}Q{Z5xWrfS+>eNjaj*`qYHZiU;Z*U%dbb}LCa1vPBdH-v1?ncOJ;#V3g4da zGv_0Z_|$DUF*p^jR8I*pB(jzYMD!Gf5vT*_mk~ngSw4|8x;y&*!(9H2H7B~Rlvto< z@G?fm0Ti!&UOi5^(;<&;MxckJb8S{%l0huq0~Zs_zK0hAfg`A3c^2s zC{8B$iag87AKHvD5#H~;!MCpO300V3BI%du?oa~;KW}$i1HWKJP;WIW2ZX;)kIIFkyXi( zsOEoRsXPEgT4Tu$aoQ55Aw3VSO>*9oQh4*};EdDKrFPk8vF8t+ z3F?;Sth1-vg-)|?*-p-XpZR*fe(X{%D~;)wpwf&@9+iZwp&+FtUTRTzg#?6bIDTmh zF$ANSYSPm^gSBs3^y{?y5~O-!Nn%}5nLpFWr9CAay$-mb!LXr{x1u&3>5Rfllc_WQ z=|~oSP$H-M&}z|Qwec)H^zd&MeMJSc$fa{%p=H0YRe1RU7MXe6VzCHffStfplh;hl zio;aqnGyg9IS4Pfu)#2$u2QBcBxQ}7&+osbpk$MS6a!U)-kxC9#M4%c8|$=U&ZLDN z)hx41rn&CXXc-?z*@C`jN+;PdG9~;TVn6xEg_oxIthR(~8cRB401 zs6bJ^yWib`Oy3jGNaH0SUWz*kH|6)|#Ojy&x{jQ6<`0#}fCF{iZC39c!zrU^apl8s zb@XKU4RUt>qv}jltBxL0>QckU(UcJuZvD;3hT=i2nBIzFB-9|9fb382BL`d_^<_Xf z3@2^Dw?%?_t;H;pt1+NxE+U2i*2=ByfXb_S_v4E>Gsft$y44T zk{loj+XMPhtwFhKl4VdzX4@6&a4zE60SiAbHY-LP?yno2b3)7(Y}|6YSk@KDXsUz> z14d`J=!K%FZ_JS66i%u zoSF*O2TaBrB!$FPfxAoY&I5AO5E1d>(s<1%#80kYc1so+=>*B8nHm^rFd(PVQKLV~ zFoq%Vef%-~1S-Ov$!doBF*Z&5B+f1zghXM)7$rx*-9r(V#N9)Wim0|r;C23}QI)nWd=w^M>N+ka#MmZbU8Vgw4*_y~3 zI69d){zt58R(sXMF~j`&Nj=2~%dF$5rKOkv3A);pF99m-Pm0)y1C^ZCgg1DVX_By& zK0RfP#jRQ0vMt6_aa+T>zXk~)L9yW}Tz>y#`hbtxF-x&x%N+9IeH-iLlE`GTf3~$AO5ht&$KAU30S5Qo;@%=bCuZcc7uTJdjt_5x`%Utj*)JXtr zZtOmcf|tT*Dy$c#9^h65ZV8jEI-g_D8F@(rV@IrxiF+e6 zCrl2?ObQuwrisM+glYDz;5HCA?3Hw!pGjt5HTpG;j4elzc{obpYK~Sj$H{#g+8)|W z?a4%3x=`jjkM%v=V}G5qtNmK9zYb+SaA}(V@`V$ur%qAtKo|{7drG*hrc~Kb)}GQj zh~~em%qU4v$y2~>#2jV!%~VKiw5DJD7?ZX9h}j-N7mqK&>>b|y z7*tek8C$qzLAL8#86Ie4)HW{OInB9v2?^KmN)%~MX>^#Krl6}!raRwZytRBtdtvPv ze7mq)zSTkA8B7a5u@8T)9GGTy0D^Gw+8M$*meEFQA6x6fO?&y_Ge)&#Sl|bSTxfV4 zQDw1 zy4(kgDURH0)Rws;$WR6<;WvZcAvZ!uXi8MGr?oJ@!1u{sJ6`;cz&;(g(s~4hI4O8m z8!<%!OXpeP=eRVjfkZXIK!zuWFItdhJ7fX!g8DC#H48z|6T=desPHBQPj|?4Jl)lf zl@U5}OL=lypJkl*L3%g=J5X%J6ne72Tl7&9REeq14zGTO=FO_Kqw~vi8Wbce@hP<} zO|qKeR7xp6dhIRqNam_{A8@5}vAM)(Fi)S=2`bWEV zNc_z9k;HJFwz`p_)x?cFN>iHZneWT*(7jq78@7*N;kY03TeGnjl-8-EIhAEiP&0_; z^PTKHA)V}IJy&HGdVZ|vln+i-aN%6Hnq!mr(7P@eyV{`wunLk%Um})4_vzo1nY!R( zxVtc#e52p`5R&s%fm9h+J&@6l$5exvLV{)WJ{wQyReHwAk#99ICpthmxILzFbxEOb zWVHmCRgeDe6Xq6*a!fIhbh(7sfk2*>3NkXb*rSM=rt5AzV4Y(2K(U`d z9LqlN3>>XN1M%EZ7f-aMmYH}b^Q|mHcJyFA}1+A&_ct#$Y?IKa;6uQG^C>1-Q z!_ezm{X9=ihK&o`x>JKR7AGBZscU&yB11qIdP>uN57_lFxuyJ22_4570*?uGKNcIq z91#Wgfe^AYIEBD`_!Wp}uo`QGBX>alwTe^tiEUv3jGn%ILXSz>oNIwKG$?(}poy1% z2-=RAk|}m3QBnsMyU)HY42rIyl;Bm~ASL~XI79YoRbmPl5z_jwV6|PR{>{q}fV3b_?_Z&##!%><=a+yVpYpz&Q+sTZj79J zsAt0(Lb^K9iuN1$eQ~owb(A5|boOwTRI95S$T za7M*VQ&IY^a`NqED;k2*LMKGcFCB-O?VahDB$7zcoXyd+Lm_q9u7-5&KSBn)hZBjr-a(=T;SIU?wGpTtXhfw2H_(t z9@(~m`TiG8R`WFijfq5)EJM`<)mV;&AytcUnMv=!{WO|1d%tI@mYZBBAeTB*DmTf3`#FP!TkAOA?jjl^q)6D zD>ZBp44;r9>$MUZ#e6m!NfB6tj>q{-i%LYS%nV>8Xf(PLX;vYO&C9DbDfmae7m$yF zo$jo2^oLW9Paxlk-0LJDVSx5Rtz1-O15 zoAlX#0-1u}U$xM94I^W{1ZB-30Ss|o9fVC=-G}6ZC6~;RrWj_*y7Sr9Z|}m8MP(eC^+X$$uCl*=&V7%m9^x_i_(t3j%mVl zR9Qjo&l((*k2}Y?% z$&)UgSBw*L5aeHs!#iyLGd13fz0(drentT@$+1|UFep@orT(qE=bkXY&1K43t7VW! z>2%PwwuZ{QGxOp*)A7WVwMcuYI^VZ#$9f5%yg1@E+m#BrpgW@tS)eeUKA1lAsqmCv zxzA!plHzQ?3pFNt4Wd^@#%@4`$PQ6B-Eg!_hnzU6Os1k z5j#ZryMNbLsEv9$^;7T(5$tg+aDVD2xT`+(6q>9316Nw(@a4~t<&KU~ajF}l+YjM4 zAJSvJu~E5Z+y$u2<)7V(>u1dcs~Vg>L_-e2pV=lJ)Dye7)1S247n^^;r%B#)N_QZ= z^+e!SNJH(vTTgr)Px<~UwEV-6^E#M8p?sHQX1>GAzmWwfIe!DrKcPj!_8W6d4UA0w z6=M|tC+>9P6>4RXQvC=)=UptdlrJwUP*U1mbFkBjU5vpo!}5)W@B!k3 zc1RRtn}@QAJY0Q#$#J@_|9ijkj1P2!l;c)+&`%p^6KSx+9QY#N<}j(ggBgiS6s#o5 zWWLNPwy%C8e~S2=t>UqGiovx{h~yaMMGZb8?1gckv{N=~#zfxD7mrcwQ4ax+Srt9! zf{acQb1oBPbkn5DC+18@7#S!`$d@S-aD<^#GpmMvaj$$A2!-`H3!7r0=JR;QhY0&* zWhN1ib&EGbT8~tUiD{C|OlVMV*Q0(E*WbAAEA=Rk#Wm(=L%JKE9_uDJ;(B}z7lhoE zD@VBq#d^kg zgV-WeaQ|L8tY4K5AS+W>9jj6mfRL$)P4=n1%NsagfD|OjsBU2u{qzgPA&Mc8&7FyH zOF@XGn}nBYQ5qb&d1ykpY)>Fq*_$k&sj(G3Spteu`AW+~vbeN9wQwf0D3ODw*492K z;c8D6oMLceYt86!m!Z3|lH)FKV)qf%hJH7-YGQZT#0nLIUfu6OOVijM4vnHmR)G^8 z@>ZWB6eT}ND8+yzbklDMrvdy3#}Rp%B2VZ`E!Lw=<^pWhkBd3g+$HxIg82%dUh0PY zv}@Z8sdWR{QaSv;lwSTJ27Yh6VLN?iGWsAuK!pDn%#ty2HZV4DHu%?`QrM6LVL;$f zYPP!g>*I|AKrf^#xJ(Qa6j7jt<~3cY7i}4D$fS9*;{Co#C8g;oRQ@stFf+@AS?#yIm>wyY*$^aE+^QFM*3xz~y;puT4laFepc0{$ z(PGxAkm0%n+Zs=44nG|uK2ohw&srtl>>^!^#X}*J)k*|QBZ}&3^pqAus_`+SWqLEU zNfC*0nF=~H6>uMsZ7E9B5)m1gFtN?LvLAGIDJe{YkE#z|ZvcP^J`%Uh;&$Uo?rGIhhY`_ZBMpr%H64i$mJ^sM` z@3CM=j=Ejz`HeEu`(NZC%_;$QQ2vgIdbzsDW%peM(NMqEQ+u?_8xoR5fU z=L3R*K#4B3lO1uTMt-KBVLUEq|MSc014W^1lSrq}yt6#7&NkjBsr*2!J=7oSgvp&b zj>RYW`IrnkJ)BxLAc^$GdX!zM-`3G}EQ-y^{`$#5y+(=+dshk}d#aO8y;x)CU*=4% zIts5=eC~fGDfv)I6Aq_x+B+7fM5}MN4pRNOemo2$PO3#VdgNJ|4vcXj+s$CePu2^l z47yaAhiW23xfro`#v5ZlPP6BxtelVNS+e9ub2&h3OOCLTP-|4se#LP1$1Z**znHnxdu|YD&z|#r^Be%A zy(16tsX)DdElnUnViu7xNu>1^MV4wO87`Y}OT(LKi_!iO!2=6*f6j1f+r7F3Vj?#L zW8&gSstb9|-uzz*0YK`>JRGkCyvX0@NE#9di01!1ONItURwlN_bb|l!6mc^$v3Itx zv;EhxO4?9dkVgqwq2)o86c_+t;auwf(aui?`zME4Q;}2x_4)zq z0qOy_Ya$Ar*AM=vZF%N*Eb=^}wb^HP;>X3;(>lk;$K&@9B@vDR+@nwmt_$1`Ic)xepH(c9Hk9)t3n6-u+H9SWBSj-38X6n%;nrD8y7I)YW zd?N%sHkt(*rqhf`T>3EE=JPB8y``p?g{ND@*5}7U(j3gP7E6puBX+#WM1@uZPFTmK z6dROBvf%=zHE)`}mFSrrCMeI^8%Y=7 z4$eYxGZ+}KUpg&2bz0ZvUL*ZG7$2g9QktmKmJM zr@AH7_R~7dR_L7*`@PEk7?y_`idkQ+Uo$+=*EvusuMV(+(*^J3=5V^YG>%`$;RN=N zxJz->=x$gw|H0Z}IOCWA#ZMBUl|MM1TuPt~1vOl6hf8O^Jgk#2gkCzFb^eRe(3_U$u$adNsRb zX`?kNHM!)yaz+^?9-(zZfIq%6Tw;pPeD-)|-LKaeu-1$o0V9P*FQkz^qmeFn3>QC+1WZ9SlF643Y$0? zIa=6%zy8-@lasOp7C`VeiNfXk;gGCUgh-r;$j0FBw>yBm*d(&Sd4{>pN_NXk>j(r- zg1JQw2$n`&3@kYJehS^9^W)7y&48u}ZV?Qf%i_4;unjA% z(yr48qs>~P$V`4|&v80*nF7%Q;HP^<#u{m4?nXGKYu|W@KxLUSR*f)ALRM?^K#B6| zsCsz0&w3l5kYv^B#ySSIa-rCIA$NQkA%m-IB6+OzV}lA-UqUQ8RAJ%CHU4lJDE)6a zBGQkU)z)uJK>O|qX8!-TK*HaP^-u2jy;%RTFx{$JN(=G`+aTiY;j7RADqU9RB#3rp z>@!2GQ_2-4;{5`+T%ivL@FT>DS$SQYpJ@0WP`uywQiYhzU)gDgKZK5ov;Y2n-jMoz zcl1B$3jLpWX!Lh2M?o)$FU3{oIL}()$`RH(|N420%-SCv08) zcT*ZJIcV=e&3J#SLvQ{>rH-d3G8t7{hQ0wB9YE^nQ`fT}B(wjZ`#f0`**D0}WJUE4 zrI{i78@n(WK`)<<8`o_Jy?Y9dG8|UGIL^2rX%2D)Q*5F1+?NlYKLkVdAeX zIw)!5&kIyhfSWSW^{dIWRNrKGB?yZ5Mkmn}?afi$EPZFR-Pa?6=ZE~r08=Bi*OAEe zc5xM#XZG`aBrh^0qg~@F1>IpU4@yo!y1=VAXCJtzGuaJn%NPwxWH{EyMdz(!8x-&z zZ<>+DSUd_cioJ9amvtg|N*vFz=4RNC>~;}&Nj`n(5C2)tyO9|EGD{_bT|jIDV#=|h z*{s60lQ2NSj-p~3w|M)Kl1Xy6H+pUwhnPPPzE=cXAN=hWJs7{57elxcp)qt0FJ_I% zJa*)gfgCQ+agbdRkv-(Att1Tn5j&nRiEtJwEy}$FfdgtGZ6{h}JMJ3AW5S*jQ7eRr z!k9HUONQVCY0SMXj-Pg*7{!qzSX+WXO(`sipVIz+Q1*?{l{MJ56`LovZQHhOR_s)q zN>Z_HRcza~ZQEugFWIR*h2h_r>{s=ok$gFdVBCqxO2NMxmO0dH>VqOr5q0So?zf7 zEw@%5NoTT~rDv!^Oj-}XHY7eY*4P~)e8vV0w?Ec=`k1z6qD;)|N7vXM3xsiD9sgoz zVZy_>_3Ubzt8$G{LB78jnq#~}bgQ4C!AO-Ea(GC)d}JKPluyP2Y@ z2U)Qk0l$&a_7!7Qe%nLzgM$6G@{4S^8x=Dw;0;LBIgUMPtZf=qCZ#W{Tn6J-#|Kz7 zkAZB*&Bsu4qo>?-TXf-t2)Oq&2#ih5%Vvwg6p6J)emlDZX+ZTY(%{OxE* z9*b%~qqE6tbsgsx=MneP`4vCEFHkpA`$W$e4$YB4V0UEnf|kWucjBK zzzb`d?b2xCgs)m#q7AcRN6$5UyeU-)skb1mn1Lv@iuVo%OU@$_Gw>w_oMkpcl?BTT zoy>Ve0zzSO16Uq%hYRfpU)(zDX)i)bOh({<5U|r$u_#jr3EJRy7~Z?Xk_q6Ens%IW zq?9F(G&5^{4q}C+%DC*4TC_Wp)WF7VApe0ez#9EIlEzhh?L~$lU3B-XFLjmMiRIUI zzO-Bh!}2GScA;7{WAgf)+VJ@^b?nQ7Nln*LVN7IoCjqvnj8>Zxe z@PG`8s6DpbLVQPZWEQ=!d7)w(0tQqt3_nVEn2yz;_q-iV3e7Vf^aRE%(iz5t^}$gx zI?%O3ATi9E<~>DBsBuTx1v}@@a?1Qg4zos@PVpqW4SQ(Hmx`ZZ%?H?j9+==`o8Bds zt>fELPQF$q-~|&`uOjfviOFVn-tL}LfUI6}JCxOS`k&jeg z_{BpCFYpGIFM3Ib5y2gXs3;ohKH2)IkS+BvZgUyI z({`sn&(C1!TSz?WVSrmH0oK<-O7v|^*W5h6ql?P+?szX+zs=6buD=t_U*<`z4XW$H zS2{}jMK*l@Gq_9I*tj?w8d;mlyExmsIRC?1VkKSqc>xsOIoh=iB|$VqQ3^1yQDhr0 zW+Sg@iP{k{WI_x)^EFA#ZV z2tt(%A6){ zi>jhnHyJ}kM5Cry4{ly`wIou_hr$)mSz*P@7zwKhxCvbJtPdz&0>rZ36!~l#hC{I3 z9?o2_muJy6!*HmU+jyn5Ap@N(vdMNL9@fWhDLqLL2u&D=)ILh0nE{L_eY0KV`bo9f zA^t>H<2Zta&I~knI;dJCextw$YmmI>IX2Vgm{}k6nqL#d%zrK_1B2W%k2rT2cv?^F z$aIzh=3vMy)V+WsH7kci0$iYxzT!d{?aBmZR^_|Tm#+9sekg~BR$32q3KC<{)Yne-Tjz+ga= z^dlTr>WJr_B~|YI7?J4z&f@h~*L?>ph~INh!w4rY?^Mf_libxs)z+2|@bI204wfU7 zjIr2INYoG%tdSblShs?IoIDAU9RcYdWe|&#>=U<1gXM$UwrE%P0K!%s%8mxianvpJ zp12`x+8HB_Viy0jf%NqiRJFMZP1VbD7~i4Q9YVXn$ginYQ<$2mvMi08^owbDke)MT z%a&fuf!V{?6=U}YxB$Ke)l%!E(5r{wuM_GJ>sHbg1=f{E1N8-8X(&c%bEk&QQW?>5 zjDk|R#v&hMxjD7x`%i7WCo8nXHlK=^vqW}H3y$@*T|a6hUAc)P7ec4x=i;;9XKg#c zeZ#1ZYL40}xR-fgDJytK|l*z-% zNP~)2@{E|Yluv#}Sf`_|ATH6sGS@Lt&cFd4+z~;psZ}E}DTv{sRzBK4*C{wDA_P&` z7+}IdzuWc?5|4j+3vNLXWm6hq5MwL1X#c_=5<%w==6h)52eWyl{!Kn@BDfXkpKdat zj3F9NB`-{xuGp0Ux$%>am)Rj=d53@0WmbIELEf;?V4H76nbe;vR{ey#{vscbGsxjr zGc1G2#Ta7LaV0bk4MyD_cd{!ZYvZQ<~tU@{TJIKZ;8c1XPx4?Zl-UrkDqt? zws!(?3`Pagu*q7d7N-<5?XONAM2;yOQ;_~nn%dLohkYqjVd69&UPSPK6{E(04Ejx+ z*qkZ%O4NGfot+W~a9h8+kvRQGXNij?iDV`Z31UNNVMlhQ2zJf(G|mNo{7KFu=O2O) zcmd70sEms$2Vf5c&1R7xBum>V-ONrLuAK-E_4ObXT zzN++A`A+D3-;M7MM23bTcXZdQut?Mhd#-PN$3J5xdgX(R^)@^Wah3=+S81W2fB7p0 z=@4GOJF8^4`}YPBsH>p?g0u>1Cmj+U2zVcKSbsUPIbtP}vXFaH?#qLU4akd1^k#DF zy66Mo@A2j@CDi$V(1iU(PIg~P=-=dHrf#OjE`M2(rncr^b_u=c*Z=>Q3^G5M2L=*U z2@~bySS$&~jDcxqtoNM#E}2TA+CGauO+FobO~tJD{EaVRPm6nv7i(7W`+spKpIjcfD3+( z6p(dMc;OLrfr}A|7A<^j42z7NWaCZ>o>}Jv0c*52fn`3eU1R-`@8Z>;>AhF%XZ}k4 zrGl_9qK0lw#>=$n0ve>|7+v=Iix*pbj8b1?6}=}e(kA8I8xD53nr)17I>W!tz^nQQ z9fiTx)+MSBVhh`0G?rYsN13LJYKh*av{f2HpJRdqY$zNMFFf)sF^85R#t=H#afzxc{q>_ zOjl~5OqH%<>>=-U1~u#tHc7D$`GtN&Fuv^#2k8`djeLdR-Ooz2Z^W113kzt?p~EQq z%0vD`1I5FpDe^N4j)_0bp_*|nM|(#3Kr_(4X@;DVhyVF*IOutP)ED?ls5L$HJD>G> zL~=SEZQ9WRzqPMDdKxYpvfi}{;Jgei+v>j zy9n^FJ~=5nBN8cjA$?^PK^4`1=6C9|#Y7-%B7)8wZ0nq|;x~gvnsLs!*j^d7Bt-s{ zHb>lnO5zQ%cTM0>jL4v=4~o4OQ$@7U(8{m9%q>R;?vp1ie?Io^*uR|(2jv=Q_CYIA z?NX_+9&5z!8bSiFIPph9WFY61u&MBvLD|Dm>)zu8*GfCxl*`c1+E~Fmc3XR?IosDz zb%QciZnVJpcNhH_qRx|M_7}N#F0pqzs$Y5A_Mn?g+j=SDI~FX+e3nKTe@QP~sGGIW zDb9=JeZNeCBKq{m|J`GJc~9CTM8&H`S}?zY4jWG@_co9^m?w-iM}Qk=q$GhqLMy2Z365LxqZuf|RKV-^ioIkQZx7?@q84M+rfi0n0K&cD1kCx`uCk=l@^D`JJ2VFsZ z32h3A=S8|TtSyfF@`X_v5^1XdC?5AzgfEp|;nB!)8z-0fm$$I87ctXEVtF@Z#bK6X zn8XF_`M&xO#q542S&Kl#coW^Oo7@qdHY|K#sX7J&~hbVFvt z<(0J5hV3P*T@jOxjdf4Tb0QycXO_ zWW^ka2r}UvuOVIF5cC2^-Sp`_MugVhaMR<9n{5FRTzQ;O0tY2=B4|^RcC2@sj;7ai z#O-ih+p<>%wcb$pyc?w#0LM}kFj7$gn>@l!o}u)z2&c1-$WR1%bz;-OuN^sT0ohlk zi0)BHG5L?Qd#@DBVD8HXfOAzls1T4hTf+V^(HRuH-_JXC~jK8x(6Y7Do&9rnyy~Fk~T7<&CKlUt0O;dE3`(!M?Kp{Pnat;ywRhu1)cWP{_E799W(5ieVI2qM&xo zKOSPH6`pZxRI+tmkJ!P%eWzyL z@QoT3g%Xcu(F05J%(@;2VoH$VLff4$kO!oO3V{vlF6PaHO=cQo0?JEVLj;bPGyhO|j^Cy+Z2L z4yh$3&rDlb)%ARB;n+3eSZ?B2)z zPS0$oJ2N-8-C*06uh$=N1HcbX+vnolLIm_Cjztbz=5b!xyLz|5?fO*^@TR<=xE&+@ ziG%PJ2bzbh5auwNM^&O)zv+^ju^jD)Im&=7#UVhhXfBKJ^q5l#cWR0p>4nWhBA%!b zQCv*~*$G9M=&jgc-*=iIDcRqZX^JJ@nP`RkgV|DRusk(*PzzeY+#Ok(7kI*?nF#M9c_fY17;x7b1a#5JUMZMXF3Iw?G;xqGpp9}O?U{1{e|N}Cv4d)F)2 zB$~cMa#@g@?B@NZFKAUpj_q?B!uBMT&eMI;{xHX&;^8Olk0U zgONv6qtq1R71LE0+BTt@Qd3zYvr(7wZK@Tn9ab&8OudHFC5Ik4N%|DEZ&&312peYa zwAlVH$UJ_HzcBsxn)_eE=0C$*V&aC~nhM5Xc02;NgG>{vuJ{YhG_8eFT9gvF^imD! zSyC)*j;gRdkRu?z*~Ha8qbIxD_gh-?mMm!s=s5^fr1l3zzmxTl%dFfgoi+ZO_tX{7 z*ZfQT``cE07f{B4IhM1vN$xgCWRnx8S4gxYM|5eJEF5|@sU5niS2*5AYx>wHPLUO;vDP3PPR??1QT}% zQ{{9r+3?&@;dV+7V22ZJlxdx<$zfAL`|qMv|)TC zlI>9h8>el}C|(tZK3N;mbRFlQcg?WBp zeu?Nc@&hio**PV~aO|Ly9mDGd0fj0K%HlBD&^Xz2egcHxaDw-iGEjs*5QjrzY+8!+ zkQp5FG=_Cxj^9@dKfxS*o+6ZuYJbb)`d06;;I?_L+y)7f>nDpXM%SOTuM-Dh->a7L{Fw@=;J03@TnRoo-~X1RkvMtDx@ zmfSSQ0NE4_%b<}n&a(4}iBxt=m6z0ll%zEqdpAZZQF#W8fUMI93J~_AhU!1Y-bFZR zg(MJDCKN$cZL1EFw2Re`k0y>r$0Dapm%QYn@@+)|IUz9wFI{sv*x6_7-pT5Pqc8^2 zCHsi)GR{iMEPUz9<>^1&ny2kcrYHefl{@udUJZ2RaJ>R&)E~j9_`nr!?}1GOu*|6FPGB zw<7u!;q~=jr^Mn`KFr8e8p4PM6<8qx7q+qIs+VlvG;*(d ze{E*_ejTQYZfX7?c_@Ov_OnpejI?6h2$c9iIR7;jXS`v>!lCilq40xjI`hRw%9WXn z8^FYz!OEvd>dTsy#got#^JkZ^UVNDMn*B%4C3V(F6{ptI$yMEn=`UYM9aHd?&womK z*5^I#etvanO9B3Er_SGEg2Z2Caus*`ztzYS)vcUW7EnLrm_{;NS_ndO;K5;`zdRCw z>qG$>LQr5r1wb_q)%y6l40sUt+z2dULubwmFq*z<0&tj`SAo%QOadayvPR1WgjKCI zEt?XXm)r|W-{w{w-9L|+>EyY7nrG)ulYHh^bx(YrzDJX5pS-uwKTn?9q6WS%IFMh? zUfaN}Ihz=_1>s^{F6QoVb2-*1_wiy~F2n}EIUhIc_kmz_{?(5&e;pH~%LRR19p%OI z+psC<>VyNq(>bD75AKbHr_`7QZ*FJE2QJ#mO=w8NOMeK1r)1}6P?2*)-<%%Lm*)HrUPpAmpVh1O#BRCKTX_j=~lu$hIYmHq>jCx>u;f@O&zsfaR zztPMYAU40ZK23uNhhfjGdIY+kct^lQmSB6)XbA44ggk~+>*i`k0suRYk=Z?nT+QX`f{zhJ+94qp9+p&hrEUdv ziKx-@HX+xf?4t9?Bd^AmUpgqR+@Y!UHz#3PO|6jOd&~@L-+I>A#Y0JRZ-MY2GwM`7 z$qI7Qsd2+0xb#j&n#-XWbBvVOZ@&REon2jYs5LXD#O>3FkFq+Zxnhf85XUA{??ac# zg`~uA$vPre63xmOT^@%Pe{)-!{S1_viq-ibaB4;RLkg7net$FPmedfAL(`Nj6XkqO z!(9P8tnqD+F?g`FaRnMQ1e#)&cr#wy9rwIGHTmO?B<^8{I~)gidbXb4oQ&B!u@_+ zS5kbI^pcwY`~^)6PL?6jCH|9>TIH;Wyv}6X`&}mCkJoqpHz1nuR`ujMqiyu(f?}>p zhLdrmI{o{Rre3w)?y74m#Ru%b5p$qinQA@VT7zwncQq7m2?bt2gvvF7tM_Eu@k?f} z=vbONkZ6BDQC&+2tGdZhoajV}8&8siR&EJ2vwVR5q8W8coKpQc0}vxa^j+31%0(CZ zHAK6O0j&lW{!TRX5kmWOm^A_j9pdK-t!+xFvq=9V_t&yrzI#eQkt`w^zoaCRGKAm zsPtXf9xn^MRcM{^d@vs9XrsMCni$;8;;H1?Y#IC^?h8>nT6KvD_on52NxEBijm+$t zY6Ady_id*Mw~S=bZH(~)WewZf#D_9!KSsO`MIFuAV*5%SDiA(+l%NOhv}}rijU$7q z6n4xpoJxk;XO;NT=Ukn>Pq6KwM)kNP;;Q|MIcAs4tkARjaib!0cy8Xg<&)GKxv2TMUERIPbM7*dMhFG>ZfI-#rk&KjZ$F?<8{2|x)$af-ECTzqWu1CId@q{*tj6awy zeUy$H`NG3#y3geyM{)Z#q2pb+oZYTCz+5YnhMoa7Bx^uy{A+ZLV=J8_uBdD9jfiVQ zmFEtP6V`CL!+MM-s&2ExoXdeam{ytBg*$sH$rgI#5iCItTsnnE`HcR1UYO|4yX!1s zj$BfOM=rZxd%3pf>E~uLn;9a7I=pUR7d!f6` zW#q7u@!DeK%62M!cdg>NeKsB*n$OMzmzFvztJvYMA7GZ28h>t9Nvq1Nx0%kA{ zMU0mz`k;n@#_fh!?RtJd7T`9M>A%&s!@z{$Oez^jyR1x8*!g8bt@|C`;G!c|H{+8SbbveZ=S4SW!#S5xwinQE~Jylo$ZWPidyy6vXuF%I2%pTF$w1Z7^ zY^~J^0VU)g>}n^@EJzrv*2F;MGF&iWn19?P7M^TKoSytRIY8zg5WOg8u2$hMr%nHR z9BY53G!+x(d%Bh1JQMRBSvTg!c-X#hTmwdveL|r22-Dt%f%+6O9vY;0NA&((xSD}t)E;TpB!+SYG6HWsF%T38{f>u?j#FD-|7T$`Hm z!Zso=Ql=_faAB53*AyovEr*0$+3Ssw+Rjod>R<+!q@og0xKg}}`;?-W~aG3aj`THio7+~Y3s4H&Y`G_nM#vy3%X z?b)-6+8}Pet(Fu7VCnVp6^{Dkw|IaC&jjYlJODU?rf$;WB@RGOy?+_B4ANYVJ5@5K zztR10C--n`Cpf|yIGl3ezT6gl)GJPIbnb#`YZ3Ob(iuMyCfhg4>Iq3#Q zH|1j38i<2eX~UTH3#5y2F{lhD!qv#fctFok7wTmn8HuILCmoL}!5A|y!I(SlcOECI zAOfc`ZLI7^O;|o{zOMM^RBnhqidpfioEiu7wXp2}DewNNJJig=+~vy@_^N!UI$K!( ztr(@Y;`p^(iq}RGkJcEr7ZL;r#36#lkUE8?pAEiWnqb(BF-2gpW1g8dmOZHq*kdys00d-IreZR3Lt(tROljc`M;i}4_+h3a#x=q`2gVZig$EP4W7XjY06p<%~Om}n8Hw9G-j)@f;FysSed5f7_@#iuN(gssr4ui7pu06J(CP+Nj;2G`Ulmwu6s-R~N>P)<)ny+A zz8P&*_ft9gu2Qu9hKjNJeY^TZQry5Zen_*dm23%4sX}P8oTY!)QP-s^&FW z;j!4$IkPM*#d6R$c>X905v?rY5u&=D^6Yw=6PaYy&FRm^LeD!f)9I$&hAg48z6aKo z%zG5DwcHhvqOQXZ*>#O31K zQe*zC#TxWCjFVW3A#88MMF(rL%WW|ZDt%L2t7?{IE~@6jJ|IS$kFHfN9u*gzAD@1LPqd;{hYYSp-$N(t0r4d;=Y8pQciH-S;; z*F@=~Vce@6wGph)0Nq@^y%rt0q2%W<$rn&v>Aoed&YlLeQ9<`whaaAuVi30Znn1YEJhv+U}l5}1#8j<&Bpr9 zq~3VP2Ae%VC!!M`luQR&6l@FLwchFi#4TXAIYLSN5A{53ZFEo>N`JUV2jUbfZ*PPW zsc56~*PxoNoYe*>(557p=ux1zBvIp}nkQCqS5h!Cdms2#@6stmsic;iO&mIQ(RlmH zT9=1>*FoMp2-u#=PWMZ(g;1%71bq%TU?yY7bwjvyoOe=Nk*XntaUQzALBt&5^f53! zj3GSd5%;=ZiaT!z*&QN~ToOB<(Di-|^6e`f)3x5EV%5I{JRzb!<9RlRK)pvj3z_8y z^}BI!*AnXl_`L1#uizh3?u+9ap&(kv3H(4h)f2-$95Kv_01P=0rS3!tJ>ekX43e2R z#54`%8vZJ)77A3H1>rThq48yadd2L;sAzMe-bOv=naWI|I_Ik5Kaq6jaKPs^siXWV zH`IHeR!oE~^3gR?XpID(5A#~1ZZDH7^fiVz;eTxOTl4?M5$A|9BArA$d(A2QHJEwT&w9=5@H-U3GJg}pN^f2@=sDoyFU!^(e2 zNrs7dWQ%OKQ*gQL`;T@UCgxC2>6aZ>_GQQYo1Uq^N$>v}uO+_P`TriU{}!xOZIl7( zU+pg|)iE?-kSec6Jxasj)mphkq2EQzzkj2}D)UZl`L$-kdTz#sUG$#t0o>I%XMy7~ zctXzqjLkLYtMs-mn;*J@5V3*ql;t_ua>O&$aF_E zeUAuTD95cwb4oX3c}jD>H`>FlBeMQZ7@nG}&O#ciO`F-S6<`T0=xp?dNkj2_>t4%` zr7GwUY+&KCENz5*Y!uUm+}-cqLOJ|b&2rU5h|&e&z=NetWuQ@LIO;ZRkFE*6;+HBV zn3Qg$L5Z-FF4`8Ls6}+uI@j;Dv{tn$imnrsYuvT>5-n}6qL&_e6Pt;C%&xbwq^{qd7A27Roept&=wu~`ek9jtTmSVlR~xcql*&* z*OqIqBk>v?#0_2<)@j#8XZm?zwoXBxY6g@#wZUKzt}_F#ZUVYz-}Jt@9{3j9z39ol z_~m)JcUv@ldy6DTTWKNg&;qI>I!ix2B50pMSbn!{#_-0@uiN)RZ?zl*nD(vD7x4)Y zdQoE(EgJkRP`UchlQlsO@D`B*=<>9#nb6F32Zva*@#t5mF5uDg=aH~xKgLn_e)TWr zm@9R3q(xb8weB9SKt6?h6P3A1#htk#Bm2})PgBGF!GBRx(w@wB{ zBUYs;vv-GhZiJnYWLA9CCg`5gnB@D~%`c2w#VOZB`7Rg!(Usxp*YhSHOJR~GVF{g7 zXdl-bQ)vGSC5@xi+T@5Vf-ESz{e}DOj2m>)slA(plWdatM3k&u1S?r`M@quLuY}Cu z*ZpCO6B?Pdl$aX|EpXaLYD_m`sEm`jzQh36w3nbP{+|a6T8274`VZ9Vh?0*U-+waT zL4a|{#6wC%Z&TSnH4Q4HvP4r+5g?VzcXXP!<1 ze0jC)@aMQ7`ZJBds)QYjm$wBbFImeM7r4&~Vt;d%_vk1!9%5Q|rAja1K6aP#zNUTQ z2`_DAAMs8fb?BEYFE9PcKLSxc6!`qJ*T%TG9NIzI!&}*-L-ANXg?TXi$tYdZ*V1rZ zvopl1fArt8qi~-;q2IBlZV3tSg#e_czY~~51ptDwzgXl9cP2MGv+@GO!4dB7@h{fa zHe7OkT>TWV+2TFyFk>+!6VD59CCkFuAW~S#lrpnnw!misa7yQxkWg3urccC-OtWBS zGZP&e-%+8$#T^m*F$h@Pc3W(W8jGw^aU|hO8%uHJNkq}Os1uJ9Y3p!aFD_y(sZXrg zLnCG7;5s;WBWk7RmoJ^wA)`|`pdqK__OSP*FA~PRFRQdfq^WiysM}T4Su{=r&qwILB7! zN@X0SHKWV@%yitYFfe#Hc1+Yz!k_77ELyTI5*KmF*G(}^p$BjMe(!xSp6ULT+S2sBFp+`_KaSgUYimm(@`Hyr(7xYSI+&d634X(P;G8y-Y5ek_>~A=GQqB-hX1l_iYglw4UzVi%Z|&Kki<^tRMp*G0?e6XS zo^sZ6IWTvw^YVynqnd*ji&AZ9b#5~}R^Pi(h`l_?xk*#V@j|1y!^wg1N0RbAwYZvw zHuuBIhC0zY_6S{g?s97yfV@*&1PBu>Slq;&F@s%88msB#;7e`u7(}pDhzx~|S%QDh zvIY+$&2=27VSj*0x{L@WLY-CdLb=766c=Q>!^i8Vk8xh__+bO$ak^deTV6~mJ9Jmz z^zCvz?+^X9>IZb!tt)Q1KMQViS1jw5H361&NgJqjdrw+f!)r{R3WhgTxDSRaoA25b znn+Js;T~O1H#n)9%$P-l$Q|VwLv?vL4tz0e7mR)dADLTxbV|PnqxFb-YV%fqX8JH) zwx>;w9AM)(GiNNQZ&^;Z`mK2VyP$HWSqMT8PmBm6s0SQ2I^7=?+Nm>rtTt)a7Ha$#1GP$c8ap>qKC1 z_#a_31AbsuGxX}2ldK)OX4=$(OpBxvjt1I>KpP@n{<6McQM3r5s_E`FT+a=)6rK;! z0)cRlh&o-?d97~*mM)}S&z4@j*-J8aSdohlgJ)kUDbyG=GGErOb9P0faulQ4;Bh}zR7P)%T#*+@JEWa}kjE7HC*i>M&JCY7Ct!_~+! z*~G}K{A?UpADlO|4M{&Zoh?x%Z43cL*mD8Kt~}gCp3*?Ba0os3@Hd=Z}8y=z!TBcybG^%S>?QFz03O{D(SDWb>6RDwYJX+Q&v36ykq(`8l z8!TV0A;U~`HZEEo>~x1e>H~j`Mn!)hIj7wk%r{^Jlb(E9Cq(i`D&@LT_!*d@DC!loBD8xR1&m3|-4l-X3TiZpI zOrb`Csh{^EpgnZLuvm|eKNc}1X&cpz8o0`!N}lCscDbTs#Uz)tWE+E~d2@f;!f11# zb5HKnfQO6N25qjW%R zP3%u2ZFYs9b9jI5OS-$ei_oH^TH|>l%8?)BK%HDVCexgS6x#Pp<;5RmJdxjTa<4Y{ ztb}Gwah+d1{A`j{?dZ@0#`0a&gLnkx9;(JmQPTK7R0Es9vXGtJnkiTcCx5lunC{UN zD5Pa7bp(vV|9Dk+9XAAfG{Oh{_$Elx;{)sH^-Tb3OWNeAV$gbIfgaoQVQe$2D_r7D zXMN&Gu`@M;{=iQ68Aqh~fJUk}wvGZ$p5VH8^`3_hdl;=56R2FAlPw3jDF+R^1@pUO zqcJ3-Kx95fu9%X2C^Gwb81}X&ly6x=a|nt$K}izR&k3(hDaniCel*g8o|VL8r2@CR zUGgg`x+{G9d!|+p$1cL!xnqI1M^%bl~<0_0~yNC`@3Qe zm_3#U$4#n*HUAGu8V)7x4M%uZ^cDk|9M`g4y{2;>G`SofV7OrXg7?H`+_sWL;{mZ# zIO~9@HHBz88ZmZC55z$^X9IMLH;E2x&*v+|*OLgK{9^?RixjnUK`=4KLTkA;wzGQX zN*r%?#a4(Rl-l;G>!;#4N|aO6sI2fp#s$=D!SAq0ls)#xI-Nr==-V==pK}DHE~e$J)DH&0Pn;SYUF7a1hd2*gIaL^oWMXDA}6N)ESm+}(WXm?Ozl<3YO-p7r; z9Lj3tyV?XCW=3aF0u#zcn<~g2q!&A2eVv>LjRI$wcxewh2g*_ z7|^QxgXFgojrGnbp2pG>m=|GwC+fyHQEs;S45*ar+bQq(m3M;SUIHEX#&K9NW)sk_nsqen;iE9 zt-h(o9-JWUJ_EI`f|FhKz(702Oglt9GwWvgn@MgIe#`TeQ+ zox12*+*{;5KwQLB@2?-JXnUzdf55Yo`j-V}>%BT5`i_6IeZ^{6sEkjN0l!3-T!zh~ zvL1ZX<$>Xe`YB!DM!47Ii22kSRbh`YSJQ-2=N%3a9mca3)-Fx$Mc|9*djWmy1g3Jp z`V{Z5NZhAASjbRyn|Q-MFvCeJOvoZUzF$7$?#st@MUQ;f6v4S87QOe2E46W+Q$dMM z&Oq6Lvt_u?b7T+R4rIGoc$@G&bF5hx3ZyjgLvD553%&`h>{CzPv}95}Dfos9J=k1= zd~V5awnB>cl#QUJCaOMLsDXR^7SZ3Yq6o_C3Q*VBYxf6(_;5bb@CkJ~l=J~%WJCU3x< zWn}axVb&Af#cF`+omXBXhI-*y{FOyywVLXv79Mwa&$*iyfpqD`F zI_c#ZJZlrH`ZGeJ1&sEedX1q^VErfN*_JjP-}E3y4uKyNrdzztU#-wG&ohLowgU77 za?v&PXtnb$&{aWc3IkQYt0bwl6qIEWZ##+X!juo_@1G`z?#_j5myHwa8*|^g3_i}$ z@NR8a+SUH{4R7>K7d_&Jo@q;I;Kr=C>$f2CAYR-AR)%s5v_L4ss)Vo~>-+7qGA5hK89w@xqW2Fi$KD()<^bVaU+RQ{3wq{bF?Q{R zZ3Fa(F}FgyXT=d@;4O5}^&)1;t&Qc;nlH#ddOuvMO}su|qX_L7|8^YVU#dO}M^odk z?mhSa^nE(i-+WQjQ9n0L*y~nc;bh%y7!B9Hx@LxffdfGW$MQgL1^jm?InL$c2uChm z&Co*BDp0jc2DFORaw;0Mlq(Pv)>FXUpGy}l8y26=rmpX9f)}qcS4i5@M-KfS;-@w_ zosKf@c&j`wN6PFvyimKL->E|w_=u9DM-PBnykzL`lgG%VYVZ)iZzOq0O60*Dkh9gU z?GdB5nFSMeRASbmsh-RIhx}6_=YYKA_`aS#IuL4f%%BoR#e>_1df_29olM%1^b(}g zSGi8`ryrp|Ldi(3M$i8!8oP$gq7^3Xq)9&7KT}8H!S7ZFc}|;u^;fyi(CNhM{^`|; z(anzH#T;_LL2Khp8Dh7y=Ga1z`5g6rtt+XkVlP?xtcPe-02p&n>v;l`TEuqf+a$2} zb#dxF5Bn}jyILV0C#Jq1f_5>pS0mAcCQT9-n_^~_l*!OQ_chVTtiQUEJZRklYgdH2 zQnIoU$VYpDcx@xq@9cMLMqJrc_!@P5XnP?+|AQ(Gpjgo)%nbLEwQ|3*5ky*TrsaxN zh<70Eng!ZUhp*8fW7JfV@by~;|EL}?Fx=0ZngRCZ=o5qZgQ^}Ej ztS3>xh>xEx;5Ff5yijC)(d}b=q7{`jlB5Z{L-A31D9kuXcN;~vu%-KE0&Qcq?CGZX z0Fr7Zf$|9qRJg^K%Ch#B1q;6d!8wMzHl8eIn6_VH)YdS45~hRNRyl_fQy`( zLbg%Pb*R+6C9{;DbG9Ol_jze*{0XlAh7qpbb|oKm9%YIy=VbxleWP7Dmr)W1HrV7uIYdQQeP#t^zt)`(opYOfEHOR%>C1 za@rW19OW^p3h;Ortu@%(wRrj0MZ~1nA}cxIm^EoCva&XR-5PijeJzJO;k{a_v8vYl zUF=5BHe4V{$GtyGlZ@6GOS3xuJK8pAlibsRz~f}5#?9Danez~f%YM+wj|3nV^7z_SGkdHr$S5} z%p|s>LG9|bqVkEPhW5$Zb$M44h43lbh0{^L@ocBGpMqT@Dl;h?vLppx_qP?Zf*>i2t)vwJ_TT9RPq{&+M zu~Ia?TES2sp|6imSylHt!%hbv?V9;5F%u(|_AtUZYMs*Fw&6)fX*UaGb8i9pSK0b} z+_<*&x$8(QR#PHpOr-m}jd&E(CfEzQiaf`ON`#$ndvP0UnJRzJ<7GH^w2mEf2yhdF zET_9RX9G6Hk52XCfk?Xpk(J4a=eRjsg43d4B85WDygvfl`?68rTSC)rWE!2M%p$rh zpvOZ=88=HD4`~<5BXYWhKka^W91dee6HTqIY}k-Rjh&5U?s>ZFtiK2v822!9Cj#XQ z1PmKI!-YqN@8umSW;g_eH)zmQQAcE`H~L*LgWZMM#RTVxHqrPD;c7%PEMI&Z-;H;I zSc>#ogTw#)DIjaJQ7afCogaS9S1HKn*OqI*xG7&cc+l6e49bCW1h9;qS6e@_hp3KJ zsR@|gk5ovAu1@x=z|ohWA~@WHv~gclN$?;rD!lg zOncTFFB1AC_Qg)dnRg7M46+gHxINqkl{p$tz$k81w zrid+=oTNHfsfqnrsFRQ;0a;SALwcKw3j3Y#6HX(!Cx7Rz(BLesp78s( zMU6sFKV4HQW?EO)%62bu34cg43SrF#W&zoYdL5;d1vH>3Yizym_chVC4sP!+Tw&Z$ zJGjt0_5yZ9ycxweaHj(i+@n$a5leB!7MUxu?V#ZAxOW;{43SgUc^9POq--(ocQtl4 zc~J{Kit=Y53ch0W#0#OKw3_(G1WsY-J^9e|Q=-r}dgiw}47$1IKUcBw%FZ8&T%4{A z6Zt6G>HC@^Z0fu|5Jfgs!;)DR-oGomKMMaHJ(tnR&SM2{Wbqs>PcWv#mL`cwk|CZ6 zKuS21GKsIf!9zHA_WgT>$jxf#!@YyW!cCvt1tell`VKQfPN$l?j_FgEUZYKMjCdJK z-Gz*|SI!t3Y9Ea=Z#yEuT9O7nG)IW>MK76Gtugt5zrH_@ekKkX>|Wu3rfo4sHNBtD ztFeWJSW-U^NB$-1mE;um+&r%_jRr1ldj~r{fJ5J-{s}zm;St%GiF?L!(A87snL5>S z;MN*C4L;Syg|o`6AY$vIuN*uA*yfVM}Q)4A``G`oID_8Gw}U&X?flyu?&GSsqBL(BX2^+}_x z>(0@Q@7D`#FOkkE<`56MZljt({p3DxWU@fiu76CZn(-A5-p+o!?7HFdvYO^fUEgp? z4EgH3J@f{)3$qvb?02t=m9O&diP8E$m=fvO}pe=;YN%5XlYpP_lu-Z0vQu?H3ynlZ^obo!o-Q^3CEmrEL-QH)@;t{(}5glo;w9)bIC`qJo*=@zLc z$oBp9nsI^oMWxjipEvE3@W)bP%y$dIE$cGyl6_lZObE1IcpX6KNT zzzcul)sQ2z)FNdv>^L$0V6Ch!v6O1PvRV0dR3R5m;VwMYndX=+)FJ<`$hIS1i?R_QB9xJ49{X`3OS)aTWbxF z#H*<{+fxpO3z;F+&E$*HRw!->mo_dsE_<~srPCNIC!uqgMP7;2voOn=g9WPX3Vb|? zo9NyHN4*w9q}xNRO)!XX-s&t6Ho6q)E2v4A-+bvGvXqg~Z;=GxzO?+ew~4sKmIU^1 zAfQYwCMPk3KIC19s^aI5U?q8bQeJ}V(p!kR`;9=Ress|Jbx+_m;M<_BvKrKSn6W0O zuuLT3gw7Bl;ba5CfqSCNLe!J=QgDtu*?V{cM$AIolXd)Xkw3q*lS{DSHjGA2%YO|p zAnONkosMkz>2+Q&lgws-ZT=zCTg{BG`_x}04bbLAbKxwD%9AZV9jm)IoVpuevdv#A zqqb$htQ7dJ4&NadpPkK=QE~B%~Ma) z80~wDbxfwA!{x%PrKUHccbP?d00MPArjM1iWvxEua*r~g<;*&hMyBD=#40SJpn!;o zoj)WZpwsEkLOcQ`9h$sX&(6)W=HN5vYqqCtVkUFAX1s5=XU~n#&TIAqSL_Xm9MAji z08k2L-*1(D7qlD10ti)P11#qK(7w55Ja#$0bC{f+6Y8&2GADBJN?bleR(A3 z)~(ps(TBo@haUuB;gq&P=FXn-4L?C5P7K`i*4 zy_aCfK*L_54>!IrPEMRZmE1mnl9wj2!`Mx7+E;YQH1HHjAMYiLAQwYH_x2$2{v)C6 z(HN_n2*P_Qwif)(*=r!yc6?tz{UZe~)cB%3s-q|K2=GW45DfO=f`Sgo(X&5uGMpjk`)PenV2h9c>dUxP%;zL;uC} z35kK6+ja|gN?1A->ZMI)O&sN$^6OtW92%``Tt?Kz=ON@_@U?mY;w}+_D97S_I;^FG zjYlWxS>ot1n8~c*-Hc<&5 z;U(}}hiB8Q8);Qok)iJkV@dI>hN2~|^<=!xFRO=Ti1~QwNhWxp-xG^$nqpY-S7I$TwlZVNo79y0C*Nr)|9A9ubMJIfs;N3{q z;$HSFhtk)538u%~uJjj+rz(_s{Hk~Wkoh=ArP6a@3S(`xLGWq1iPD;coXZt1M?U+0 z>0cz?Yb|nU7l-0#ku!EC!!C|Gm=6<&1{DG}bE&)P6DAkaf9%>ztE|bOV}A}Vk!0KKmW9>FYh6Ce!_*pD5bhK4`s#jDQB2JlBDshjPS|Pz!@dG zL32PuYrL*jywd6=K15lqde5sW{w&tmtsYKO`9pZXxm8Et1=AA?yi|H9im{_#1gm_% z6q~kxMh!{ovz~GSLOjh(t8$?#UxjJyCvNS+j+0|~p)j#V_Q?4?AiSm#6 zB};+2KNB+Xq}PzpV

&hjle z_rM!JboM;0lSXnS)4+W3{tWh4K##=>&R30|<L@M56V#{GA;9cYn`XXnQNROo>lCG`PZEe&-9jdEQAfHuse%o)p4R2wt!3j}mMqS$+Dgc!@;Nr3K*r zS(SuWxUmeCmU-PQqV^!q+7a8#+X1y$xUp>m)_y4u=z?wbZxk_)OXz%+Y$myCny27t z3h5?E*TmFVdeGonst$SlO|Fr*(*!rxM!QI>=4fvOHSU&?5w7ffeMHqn{5n@tjLD&p zk_~y1WMgvvep41a4fxJ)3qYxCQ}~A4^wMU51RkW zK{?-7-+f2V^%bK%6O;C#qLS)%JvFIPYO8Rhakxa^L}UY^MedMk&(z2*dp#OH^RaP) zy`x_))w2K@$fA)|+SVNL9BS=9pjqGAbb z>);2u*C@-{6iU5l|8gi{x5K^&Vl$gihZ?PR+;2I4z;bk#5w>D`WkaOX6WFrW=b0X; zx`TgS2I53^K6?aGF$21E;(z`Svee;6I|=Ge?oS46%+s8^07mlNvX^dnvv<@Ihk&4mv7E64)MpBpHf+_`FOH zxTKS#l6o7}YQP1OY_inrp_w4$xx3RYx2bg*xoi!`${$}u1HJ}Mjj%e-M&u14r;y$F zm(t`9a6N2-aP|P2CnE2G7BdGww`>1QA(LmS+r+j}gUcU_Yd9aTZ<5xx@EfSz^gY!D zoJ@XX=`BQ^W71|O-#<2l16Pm(cSH;!nx_w%`N|A|6=KnT(z~cvhHDH15**Vf+DQ!- znm1U*6)=a^nntY^@avTRSCh1~!r2nsePkQ{Zfd$4xLjnERn7?YGopr)k-NpHgAF=F z$e1gTeGWD$VKg_^L1tr9@y=$QUSFZ{-(-HQzi&c+`L+ZJ|H7IZKzWfsNE1={%|N$BATq|sT$iIjpY z_;t}dF!_4Ctz|%CxgS2J~9+WWs=r#NaxaVaC?e{ zF4r5yOn()3M^$(Rnj@>V!O2)4m3(n_-2;sL=(Ye?b{pS|u`KzrI zhbAiril`+`lbjJJ&`M(EXJ@X>V0kNa=|MKN3EM+V`%v-MF?Z#~*f~!5sg}xhrI`)< z_N8#>=M{~JfNPV;oF*aK1$OAsLe}A36BS36j>e6Rvq)yG^k40u77hnicCgnCq47J8 zECuWi*V%S1ZPu3u+S`+fupLVLA=*}STRK1*PjSRHQxpBeC+EC5fzC1R6PFJak8kZo zC!g;Z-lXaKhe~KkoFAV*z`7@QZ8mv89deeQgC}z#GvDJnptSk#7p{Xq&^^+-eIV!_ zts#DhlPdxb$770xN0BrZ;4rm{*aX%UcJDzjme37odp%I3*wS5qro(|pc!Yb><1?+& zVQPZQ6d&}~B$DfE+!;xQ>qp`IA#651+o$qBAHCB~Y(#t<(jMSSYA8^im6uPaH!1qG zaGlK9qr&_JWI8&s*DCc2rc(NMQUZ6b)L(foe{JOaHyr@a$DR=5hdDv|!<-QPpS_s> zUl>v(4FBZ?1#H;-ob>Z#W+v*HLA4a2{#j&}pkNUx;Fede4pP+*&*t}Ox-sHC_<^nIqu2O zx+?W0<2}bWS8n+Sh(nwQ+;2u~TqB?1u$Y9md#k#FCY%XH5>wwn<%Rv)i^caeBJ^cM+$ybFwiiYLX zkbhNLfZ5`I`mAdt-LH9Mo-}i-Dh#NI;jTF?)0Z7VnPigI(NP=$G+6^X!c{U8J=}yz zW5;t~$FCjbB=J0K7EMNCGofSk=CKXP3;*+belM7jXRqwq+QzV%tYg%{gC`rfC&fNV z{I<_J`(+a2MlWw&qipH<#Gc{gqCV2Vy2G)I@VVcrkrBnsci4AZ!Xo$g!geRCRpo+Z zVE$W~LV96iY);c4IH8?tyf7@LGH4~tTR4Gx_sF&GAosOa+)F|C9!IhKN>1Dw!92w5 zEy_ZeN$4$o-mz7Jjv*S^gkDOm&$08jv|d0ElhCU>>Zt6(HS(S_>X?dbS22Jt9LETy z$SsmSc*BTbo>GyzQjJim32w3`WUxfHaFRX+%WIpWxqg5(-vA=J6ICRH_Z_}3Fgs8>6!yR)c8I&QF%0{xp;YK11!+BSU2`=V z{s9L#-G?{$L>4kR<|^{pNL%$wibPZCZo_*Xl)y0vlthW}aHEFu{HpwdK4d??99P`4 zsr8T>Dc!sg2aA9Fzncy{j-nKpCL<=5q7p8a88dxMx?g&sqPQneLFKTQs`zWqIQ^Xh zNY9c2BmDkA3E!y4ph-h(ed^`)pH}c2Ez?N&8%m`R6D(EY%xf~RTJ1m5w3X=B{tS92 z=uF5;)c5Y~3&K;BrmS#|nw-Tg$C5^4h*~p$CMFFO#;upPv4jT6n?qg$ zQeHxCgttIyDH0lS;L3RDFcHOLK&gH*iZvK>voSXBB{3E0aTqrcIJ2>NOSs2&mx+&% z1eXt}V}wD5sHXH8rZOg_d76)F=(WYv>Sv;mTW8`*Mk`hm6{#pf-q({iwaF z#h0td^HeWTt6kEfi<42!iCYk9xlVW3HXsi4M6-!vDmMgs;N%^Si_zDb)i10V=qrLy z6@zyLhiGTYsnm3l@BsOAKWWp~EY`32uB%jJ=BPS=bd&bN13m|7c`6U~qL|{mkiP(o zm>i#O%Cc}hX^s5RIekG*lx~nNahkkk2vx^(ynF{icSd!4=13`f47D`%5C2}_WR?hR z%<3exyP>zHvQtbRcVK!wav=goPZ+{b(Nn?4ZX($u&wP>SmutRF#CsWg-VI$YPh?r4 zQjZZpu{#@}CC`{>DMV$lU}+$?qiOPt6LYQC$WFOR(A{25FuB!vRoi7%2PswT0GBY( zLqxpiU~yoMtZd~+9LMr777V*}o2BhX5v-K$`K48K!6(td5_AS@NBxOS`<#Tm;l)$Rv2RDj)V#*eXet`3lJsHi!KYGTH>; zG3=u#ISL{fT0c!O;&&nBO2lWt`pG2-FB}TTyHx^cZIj6jicoYHUL>@4OEvli1+e{ z@LVN|ksBDVmnD?^_@zbM^~d{)bnD^~3BY~z0_rhKSEdFc*d_5xdst|bNGHh3?#-O_ z%r&%%#gVbb0c=ns{%ih;sp7?Pym>pxeCA=cwR}14gnnTKTH3Kk38r^-V3N}?I*t0|Xp@IiEji5r$u4HAHqlli z>9wd0ZHrihBm=h~#$IcagY+rH4TT%4V4BsJeAf4W?VJBW5HiE#tX9#KM=?eHLLjk%nh=Q)?_5-C zf)Lo>!bYNEiw=o|P`HdHeT9bA|LLK>*RkH0&osS{su8pA)>7$H(qR@tryuiYydUOb z4jgsQojks@p#m6@tW6RwbH1}~_)fS^zFmC3pHFmwvU=HIc;N%=D8hJ7VG3|~lut!; z(P$4M0^ER9(X?O-kfFgpVITtJ888QtYNS$eU=jot_0{^OexEZCrFx=nD%x>{iAIf& zav?u4f$RQxNsyn*$7x@X2opvwNe;K*pkbjX;NU@F8QjuRVJZmZ&^M1Uhrg5Ye1Pg( zu`U76XU)ek_|3;eTcvN!J(2Pjls?};oKZaSj#fC6zGy8dGygZMl|i2@_y8(0Zx~)k zZA*Qs8N3$JX46MYN(Pn~FMDymBt0^MBYvE#-5iQ|;9a!#u?E{idm#Vc%T;InWR9Ji zrt`0WE=P*^M8B+PUzHZ;P&*fZN`r~$i9un?s0Cvf9ZH}?2nwxOL>+Z3Sp>?s>19-r zsbuC-0d96U8J-jyV`ufwwDFNFWV0mTZQ5e3Qw@=sotIQGhI)ljX1Ltp4&p(UcE^ zJ+OC-p}CJtI!2o15gLO;xphh?RRj&}X{-d5AFWpjAb}(@Kdw$XT{uF$*<`o1ZU7Hj zmPt7K#WAgmS?(FAqM_FE_wsL@(&Z5E>qBw?0~+=)T9Hx_A*yuQs9r*yZzRL6|MlF0 z{(w_IN_#wlrG5!JjiPj;c`*6QMarfqyE^9-BU`pz0cr@!TU6GL;|;qRWeI)t?@9xR z&Yp1Ge8cy^<*AXSIXF}6c_lA+6Z4VvUuTR{RkLn=Apiq=i5ok7I*Jf6aS{B+gsBe7 zV>`J%Z^7?xO1q#5Qj()|8&n!lYMH07MRpE_mFCjVOpY-T+g|N6s%X6%c`B?uH%dp6 z`A(TW-d1=vA4!SBCY1hlABzc#jZ7YY<0lTWtrQEj#_j%$>E0^Xc(;d*MK{$$nfh2) z36Bbn7?xnoQ_|Hg_dZgni=pQrJ>|pBgm#80xFxY|LBE;h8^_Xd;wk$X_d!og7U6A4 zKhEpyb~HMKjsZmSHZcXS=T7xk%lnN-&cK1h6nE8bxmFYv;~s>@=V%{vg(An3cRViK zUK<~*#9T%nCPUzpZ^YLcM4xUDi$`lYU8o6CMp;r{8BzH=q054>K%G?DDaeeM|>H^ zOnE&+_zAN^X^zxk5EpoE!7B?ISTm1xhbq5WLz6?SoC9#CJ$uffE+FT8|Modyo>i$X zC;)*5MGR|u1d|gywB&DzPt4*A%t9RcKjDzfQ8%Y#`14Rc1)6F_8!ySCcjVD~SkVU@ z(4BdxI*Qo70Eq9|qK(G)IBcEJLNfslO{Ks4n68%brFrlQV7ExBc3C@{XxzPcSw@=R zqIWd1d(7|A^)I=7>eok7IiZd(LDriK`*5B8?t`u45Ff!VF*JsYe6#+rog3>~N%;%e z6_M z)(e`KmY3JpWvnf&o2r{>o0^(BcD^^+n9|dw#s91$`n272d){z7Z#L$-o~(Uyf3Xl{ z6kj{dM{knJbR??YS0JCC{LoRXZ?VbI95Xw!D5uQlvuLNVWO)-;IOpDjn0?%}Rr~R9 z=~o>T(|=PR$U?euv}Fb&@J{UhLqOm#M}Ee{b(u%CakLZ8>4M0;DiQ)K$}MC>cC_2< zgNC%VyA>thnml25OAKU_de$P}8am;4@p9MsJrxY0dsw95UwagZ=M~P~!v3NNsjD)c z`hW)EJ1^n*DoL(8l;~|HxJUS@aHuukndtQ)9CKR zC^HkUb7|LFR%5(H+_93m1uL#mzih^%U4tjAV z`vd-~q)(2^oQaPZwwtue7U#nX+BdZ4q6qedL*R+W-CKvho6e9gZW|WiD_{JPm;5`- zmztQ5;iXizJwn&%RtmvqO=9I?2KU1;(3fyaHS0slXK7cL<8|dDqpz0hmF+__(AV@9 zr~iaCe&hh6M_sbz_NMqX5RQ*@O0)QbwTF1B1LAwG?`CW_H>iP+i(>eO>kRe-Bk=qD z5b?!!{zF^zi;(=SwC~&DzUP%s_A9OhGa@S@*H-pmU&wxBaNxgNv zStr9LmU-j8J=M4oYsi4gU*5ymp(@M;;0XaKha&}OHz{-NsXiIrxS zW;Xid#g-Z~@h4cpcv>+(E91w zz*s`Fj_NX&yg+XSYprqQgh!T6?b`aPslW#bvdXzolZC)j=JZaeU%>ZI>l#c|ok6%f zF7ps?f;NC!+F_7$BIUPE{5oJk(aVb-H)_U|v%rC*Tt;_DNsRznt)K6= zrNm=ZW5LyzGojQYJ;-&<{ehaB90K}qMJH zFOgoxgMB;PZQH%vVa#Xj3m>|$Ng4aX<>f&1G&PJ!82yH1e0<|x7DA;e>yY|ZGS-L& zezf1C+iZlnKIgs)wN0aw27ZW(-Z0fdUk%tatd~J69$|?DmaG_8SxFe5o(O}s#XwCo z_yPNbny2<^gr{na88X#Hm~i!BQ6rlSyiszlB&L_xpI-}Kv%j96)o3(6hnK4XLF~ ztIZahoos?M{tY6*!vy^5Ox<7xBiW5dU2xasQFYqUP7u!Iut(Ea*0r#?@xS0}DYQN7S-M>||Xbi@D)hfv# zpTY0(MZa;Z*K1R}j7U!skz00gBL$ptbl#CF2y%7+E}iz@Nc*qP;LJ1Q!rkLt7p$O` z5{Q4$4vLY>pH8D_Yi$uPCvK<~s0~LwMmuiRfsUTY+bKG;=sV}1^Os!(DmW0y>NsX4 zj2KTA+2vbo7pQb8XkcvrN)G4GsXOts?lL+9Qr9}Fz-|>15}!{A zEjsS#1`KMUAZv_pOg$oM^K}fc}C@V9;#RO)A91nLBla5au zK94E-m;5^~9XSUG> z=@tQ;K*djH0-+1h>m6ZuMlnNfbCw&cSid&pR7A<5ig9Uq;*d4d(gtw$qBTK=WSS%=cXfU5HF$Hy0c)}RM| z0}Gf$Qxc3mr{ArLiB(>f4rOdh{^!3-&hypUL9-cd421p6?C@{_{8VdXyQd89_%o?j zi0A`xiI7#YzC@@kM|%y=GhAwsw_1^x_{aes=w}b<=Sa?fz_zg}W&DG#r1McU-#T?_ z%el|#0eCerejShL6@oG>OU%wBo$@u#bdq2~z*H6$k}P=FTdt3j>5Zlt79kb=F5QNA z{mf`q&;t$iTEcNjT~^28XP_P*_PHbSOC=QwP126fHYe!>iYX>u62B5XmgMt%N-~C+ zc|28b3&|=PF%aGMCGLf9tSZ*d|8Vr->fg_azd*{hG2%)+J{9OSfVf!vHsLN?tV#LV zeJeK26F`{6s<}~zM?^wH&Zik2w2^fQ-Sq1Nm7se>v{OMFVK4{Zne`*j;Tl#LF4;>* zKF~gb8R7>4^2LEHcxaYIc$W(vL>Xo5LH}TWp?xQLM0~5_W!4qwitSsJg1 zNIyJP3SyA_E76dyrDyRi3T`XMemRXF+ju|0B(x1Lmqoot_eBlvDo>ezmRTxGVW@s{ zgZW5OYozkDD}U{J2quqjKFX%cKouh6bGdJ~UCEdlVBD6T41E>eVOSXZ!|D*>`gWZ4$6 zN;m;51NAZOM6631|DXY{%E{%+<<0py_Nc)<`5-0dO4(3` zJ;K6mh`Ly!zrcKL!MRhG6Zr)}lE|a~F3We3vDr6{={U1i1PPF8#zPHcRa3jdU8f8i zY&B_rwKmQ5jPbfw%Xn4)(Wt^{5}_GSAtVheH=wl9^(>Ogu;)mQDo3^m%xBFw#|=Ka zYcsFQXU#mm zaZW$IJ}T%qa17)FKv&UJEIlvLBt0SO>Vhp@qIEA%1l>q;wSmNL%S z?K*1T=H@nfoK8U>8}e)!=S=5~A??uR64U#nb&8&IC(_Q1>y+$>2_~84HVQ2n=d8|L z3*pgFg9>SW)y!cj;tGI1Aq>JCz0zq4(k8KMtU8k3H=Z}cB=Q7q@Xm>kPscCuozj}bch`V zj!W0{zKz;&cXSrXJ>7FeFbMnPlL1oK+T=G_33KWxF7fkK?#NJ5(nir{Ir=cO^;`1M zFM9!ds5gw8<(BB^Sq>#Vdtdo$Y{W^$6dT{aQ=DA-!CY1CXQlm5cLDV4tHI|J70)9f zToftj@1CCdFVZZ9M3m|Ri(}CyBuA%`Gz4w0K{`zR{aP`yGtVMah=l!_60eV*@spPwDF_vr z)vN>JVB?II1au1jQdzxw!UBLMh3pdmMO$DCU*xC6f`dLCylaXJ8Iv26 z5(~9w=Hs>}xk;sY2iTg>W0Ct%A5`KlMC|zr-RsG{05#?PCucv zv3i0^Ly9V$e+pLz!dfw`$uE=^ZQ)xkNCz*4&!m96NVOtJU&fIjUcZ!!6@xo6%8^Sm zY(N&UDxm9*nKG;^d{iWTibg4#eC+++g)z?UIY98mVY7Ckx}qj2oUc#(G`bsfTKB2A z-KL`u>4&}Ylw`uRlyY)Xt5-86GAiQ=@tMNfdn*(4Nn2g+uJa}xXEHsOlJK^CxGxEL z3K??u8Da3K%6sn8ch*DQC5C=BJa`!OV9mk~AEDw`47+VH(l<<`mlrPS(K&Qn6tZGm zs9>zEnN(BJXKQ1;;lv{9=m#`E7qKlJCIk)vT@M{eR2y7ke)5z;=lEY3+pZj@pKHVphIXlVj~ z@EejjIf!Ui#LNUDkE6obhE=tYj9{HR%i$E;;-@}j1L+H2>p1dJ=RMVIZs1jRUxE+F zCUi(ETp!vd!>(NDp0+pcYd$#}yKh5#1EX1n?6Cw@;Zsr5HYx3E-7D`vYkrj(B}BW+ zmGIo}ih)a<+>7-P_JGI?OTQD|nz)Tw-K)*^9X<_O z0>vG1LgeC@Oogb%+V{=_P7Kb;(aCS%onL@0MVCtdm!&Y9T>86wN_7J6;b>G=+wjb( z!T>b~2Z_|~6V_1Xjm&NRk5Q4o#WRiIy#Asb8?pjL+nGJ~?~>YG83k~e{92Pvg7D{B z<4yvC=w0@~FFVr9Y~3rx@ejjXu0WJM%NP3yo^ecwN^WZoae!pPtP4 zpC@PUX1;#$6JOb}&mGJIIE@rHGpjDZBirOd@# zEaMon__7Fp39VePqYLqIiKRmogG^@E4vX2R^1(ilsL0kc=VDJsgxkqO^*+9#Kba*G zRC`dGeUl_zi(39H2WB#3FX+Sx4?Gfe1u3`nlD74poB%hk-7^mRi1N({`7>dR4Y^ai zfOjPn^@f&CrZWL)FN&EEL}M8x0ho3fPlt}M+$KqKbS+-)CzzUj z0Lf?^WN5KoOMM=0Gw}m}-12HiwGTPVViEB3eroiTf@xDk`SWm1*3n2rfDf-E6*q>| z4cA}HK_t1dpQS0bBwO)_b6(lpmGaHZgK7KDtlTFfruw=Nt7W0eLRFbS*_K$YiGR_m zPy$pba;q~f_bem0B-RA9ra)tM5vXn$qDlgI`4DIsYFB|(50;V2Ztr8dm$y?|RETx|M$Fn!qaC(t0OIys$Nx_RNY?e9XZv=Gf+i@JDcqQphtnk1&u& zDCx3XgTIliy_Q;Fo;@H>J~Z6Jrh8~Mq2#Lwf$mUb513Y8&}tvBb+0@9-8=rt-`U+G z#gE(sl6+QiQQ5SN%*&F=(l&6y`nfZ)VguuT{4a1KAdoBS$89M z8A`N)(w5||@A0XCjJ)`DwC1XW@0=9^Ak3PEJ0Pw{=EO>2=EcsKt(!{?MYrG2;9iOn zcU5rkjO*X2yVpf}?=hzZa{yslYAg_? zEpnkPV6mD#(-Q3B&M~l33>cX;!I#3gWp585pRLxC%{>`=1nMv+ClCjv?=xKk1ytpw z>POVtlXmqj2#BzF_clEedbatQHUyeFLK6%bx+8SFm{tFpA#&kFD&hl@0neKZQb+Wg zpJEn>Ptakj|F=&hzcJ$_*~~hFwhU^&%+wo(=W0F7nhj9(~=@Yp&l1sp81C z*%izxOS36P(~5G$dIX{YAtmkoVEl1iw#Fe}Duv`ofU_4%-Dr-|vNNo;Kv|OCVNRlk zv6g5DhK|&eU@(%&3fikH4qRz#ud)>H!Sb(y@%A>O{lk622W>qD)PFyIwp-m*56amA5S^rFH$X3fcSBOn!q%_U3i z5KpoXq+VC!FSYu%bCpqUss@~_+B+t81^z6R5TMV_@K`r0ylSc&PhPX`UA|N}H{r4A z)E=O;|D|ud;cwtrFF(=mr4X8(AlGX#G1XH z*c?V6JeSD$n=H9GBsW{cbKm^?wOLOt{cff$5Js49hysf?BLZRXL9A1@JAXsxMI!RH3dGQu*JrW(~IA@6uK zs(%lcn42z5_f8+V6|WSgaakajld$M%Kj2Z-UeZ|?TeB&9Fxa9EzOyWsyWR+Zghj*y z4qwkdM9NBeVWr&T8Xa;?2))TCexc16cTG&aS;EzcV4>IcP9eHoDwk;yf07d$NH1R~ zWQ#PtP~%99ombHIaO~|7%rc}UD)J6_F6Tc`i-*IMQLd>ZJrhDcO0u%WLA!R+Zj7w4 zF1OcKy;W}Sj^)*^i?rc?9_D?UM0r%#&F`?o*!r72uF<@ZhR$IU?JXHBZ4SJ&aPs&t z_-JlAkdEa0E8!J6`r!SH-zN^?AT+NLaRI~!mDZD-X81waCn)7U^~`AT4+R3QXy`qI z7j;=p;A6}W>5CX$q0bpp69z$L6#-sg;5Z^Mh14~K)m`{%vi zzWkXl;Fue+XGCAn+S7c+7q23?=cMc*sb>OTSXaBcX2Z52y%pvlxxKLY)TV-Zk-ftP zB)KT9_&lHrWzw-&^w-EdWc4?)?B#3%2f~BZ5NS??V2ki8&O~)uvP^Ci9tQke3oxAk zjGijbzOIVJ;~)7Pd{kJbS6;>SY%>wp-YryvY+LEmBaWA$pVez@vY120h^f z?e7Sh^$-*Egc}{va3=4R$lY??M&8VO_{4g=$e|wd34!mF!hc}Uq%GvP;jcMk4L5+&jGJf)i%tdpiJ&lvAfiS@Fj;C&tq!TYCQDs~34kcx@q7pF}LFkC0pb zvIV~yc7(C367_C`?bzv;yARk${)#!5Uj}*p?;vBm;dQ0MpEJVdA28!2k^`)DQ=Sn*6dn+1!x` zw{USuz1J5@*+cLF`gSU%bML2&997Opm!k4oNG*qm(jS^H*=*6_d7Z`kgU7Ue{`j1+ z?Ez&E9a1?|SRBwq=VT~RwY}Ku42E?Z85Dsn#IzmgNTa{Y0f$YM*0SkCgJmV7A*#(t zV`Q-2_XZWU%4;bqh@i6Quas;#R^4m}3+`Ag1HUiu&k8H;XN)Is=%Nhx%J5J0bWKY7 zBgZBf`*PfRzVa>xhx#0l66`7$V%$PGF1e)jvxO49tdtKy<4d}3y0B2cA$S(z%b~Zk z+@%Mji?XsC<9AU^F7}T>-gAsjHpZffIPy!WNw^oTBO%h|D;=&V^I2Q01gr1;d7FE>04rQg2=sBehwNM^I%8jLvF4T>y6Ij{8qAEnQqMf72sl8YyYiM ztKR=$lx^w|RO)ahUuvO>=wV?j670+-k4E%&28Q^@`5llNx(IH_0rA9^XS0REbrrcJ zzO^z!@7`oNA&48F=<;~ScyV}lDVX9)tO@-#(hYL&zGXf(*<8pn;!7us;vG^6JDL~n zDAu>;r+~T6Us4xdl50M`OFd!&;vaj9EXJkoI=e+R+akcHBV+#cTO#D)s-JH3%TH@T zh}OY3zpIzy0cr|o)LLBwW)0qmAg^c8^g#75d+c0>K;JCX=Q(s0*EHFBjztB5K60kD zhd5ug8yhnx`@q;r~NU=xP4_7IyZcW zLiP<1eaBkA6Kdwu*AV0oK7IjF+EjaQ93?C;2I>x9KjeV@b z6MI`(RbA+oXgn+Hgq^VS-;NPe!ZLMoe)@(a+K)r)f9}ru=^g*6^>3+Ky5gv!Z4pmk zyED48+8YWYm)a{z)f2P9lSKm-2J3Mk$z6yyhA3vEitD+O2km-jxc+#4nRo8=|_ z1!2L&+by_uoBntO>GtXFBG#=wPpd62)Wd%ELvC?7-f+BMJ5KYQM19@e@O^{#3b}Q_ zBl2UiM+M;W8uyAQ)VMqR8Qa4GoiPyL-AWoLJ-WpL%^f;{^TOAiy%WLHO^DTHy+!j< zzV@2xb>S{OaL3bStAh;w(@l6{JHDss@X{P}gSi@dbN-U!|LG<$@(hnFEKtx>b^yug zQ+6PRtmlEDuhenAW7ocIm~bPq$j0a1z#;OrKVcHBz@>XMI%fwdoTDkQ-s{u zc%&+`In(Z{ij=MT(yh;hMi3nZ;maUPZE+9?3< zTMJkh{&Riw_B?g$p*vE-5)>+&QM1h}S~pR=S#>3=(WtIDEQ;i^?tu4-PaV?b9xBV# zCXs+~3c7PbS@ZqIcef*28}sO>IsBABImg1ij`v>cu$I*lAr#=Uk&|swd={f^9FU3< zMUKgR6`@USe1v9uVjscf-x#hi!zOJ~Wrvp|co3Y7PjQrfiq|x#(qd$=UURX7Zk<$m z98qPP;YBsuS(G@oz=^dTij(77g3e*>u42$2Oa%c!ilt}*cakMN*7R{--vVRSj}{ev zvdW0j0&(pWk&{>uedm$NDtGvlbiPUt1!{=BVQIIFgb}VZy(c+&84T zdK4tyT{?X-&T)`$E@YG0>^(cY1cvhkym~vJI9GVJlel!WU2_Utc!+}*z2U(l)~M4b z)1|R^4{C&tW+M2VY32lB&&I(^sTGAQC|N%dX7(N zKvngzi=2)nFv%@H3k}+i$udlN1#yq8TrLL<*gYMYREwQYuVks)I_H4U*d-EEt!st> z&#d@R{~FP3LrX3RzcmF+gt)OcQz%6XgnCaLT;QtA-y z_lY1Zj8V8oh6DLk6lz+gz>*b`xlX(^H|G5+C0M3Znvcv4vI@aNP)x^&H1&J zJ7{a>CT+;Z(L!@%d2)N(1lQ=&*73;AjGhLs%1DDRd0Lh2YmP^Tp@?_w#S@PTa9AInvCA zeWBo#o>oJ#?$IqpRg6koawug$!YxZU@{p1|&i%!O)~j0dDiUHwIfttjVE%Yu7i`u- z8SSSe46%<5-Ap&o9aZBu(ke}}#=kgJ@(0CrYede^Nqv&}8pjK@C(Tu?3sSw_+zjig z>so(M&#%uInt&5eZ=g3L7U2ev6UTyLCu|6D^?IC3I1n$iQ4ONC`fjpW$zu9l=Q+p0JoTdXSZhF zPxY+ro%#43eI&^_i`NxGJ*dh)u&EVc;R1fDycHDeqoYCB4{lgs&A{|+iAE8DSQn+Tv4DLoHUu>cr+2x;!$U;4iPPA|f z#$iqy+ObK&B@aJsCOv;jYB2=Hd6iiKh}I%LwhAt6D3n87i;JL)czdB zNhwf4q-yqg3zV2qj+Y6+IVc1@)o!RjBmfGud$x6^$$^Ra_GM(>u^y&qMg(Z;?o9<=Tnv<4Oarb6Q-;~5QGk(`7}*!0&_cxU;u)XaRQ zp$V#^6*s9QLTn2St*JFw@LAqzca%hPT!sg0!Q@;-);AxLPQ9DtNAkL9&?%wNugrL^ zoV_b}G#OSy;2W(j*4`f}QxrZd#JFF7uTS?Awo1C1sEizUsu%ACo7}>&t4nK@mt(HrH*6 z$T2X}nxX@`liiP&nQ3SWNi|YW#kd`fXdcZKd~TqPA6qDn@SEl>2P%<3eg|GK_Xc2D zPe8Wvj~z|_%s4Tnyzh?3$eq-vB>1nkBlSVaRmtzP2y^ToKh*z^56k9%9F~i*sf?kA zsUw4+v9XJdi}ioNY~}5pEp044O&$O93|rJazR`qOU)v^$93CE|))CR@aFk>j2BlJh z@Mt5ySX~?AC8{a+CXbjrm|5)H?Z`>Bw6$wo6}zl>+G*Id8l*-73`CJeR!UZ`Q(HGy z=r^3s=bxI(Jhk4s?93YD2aZiXjIXzjyLR8YcHeu;zS_(g{F81o1JhwZq({!etJdJO zLxM!G{&<5PC3UiQ8X|D&Sw`h+a zd=s$6pn%&j_>)a=5&`m|9siu=ClL*Hi;7dia=R<4Kxi#hNu-w^TuIcG|nXY(+0d*F4U z(-uTNY56yl3WP@i9o7}o<^4l0lv^oeO+5bDij8t@Q&z4>-Wim$tXu-1K!G_k`V@N7 zHjZMd+@r#9Q6Wx6A-|_z4JP?9D>fru`-oqXqV;cTV&8?}{LrpeW!>ge#7|L-36J4$ z2YRh`_G7ax6brAs#6zE;*l|n~FgFu*rP>GFE0pFkV7p&V0As>W2lJMuWLO|x|nHIzGt z*0{z^X3F%!NjJ3uAAVcVEv_pAztBYEV5l)y%QaPbIPXA$(2^=B5edB=7fdgWg3#Xr z*`HOr42`=Z`P>H@b9Sm&MU?!wn8Vsx?i9k4L862lM(lb*b z7UNr!K!>~otC_u)Kk!;Bres&~^VSpguXm;q5aY7>*HFUK7Q6?YH1&z+k8D}3Nu2s| z*38!MlR$r5e3g3Pp=V%xU=B9En3ga}D)m~4A&1&;%u#&sIExp4{N;L#IOOw?hthuV zBtvtY2T(+DEELQuybQ=fLxQK@?zoVwya5z!xWmc>;oWAFdZNDAzW#jJzTstn|Da(~ zJ#&48L)X?@tO9wTfQquARZK{2@MHV8bS;SRu4Frt=76wgr?9ZI zIj{T{nprEu6cJ+|TRhemC?q0}ei8bX?#Pf}gCVJnkaGSK!@?D7i#F|NrjwtaHa$%OraT=9SZY-J0nVC18Jd~-}lgj0W8Gg!@ z8SXJ8*-xbpwzSdB^)&nBX|td93~z;Rye2Bjw|+xkmfKT$zV_Mc2&wAUv|7I|9J*>R z4#`O+lWsiYf*nIY1RT0)Sn-Ges-8Ev**QnLr>PrSO{?vgXOc#88@MXOaU~gVHB!?n z&!UC$a97P9#?e1=H@Mg3W%uROnb8hdjwo*b!KSUOZJZr*-$&)sy(xh zjo)(5P~FxOAI{{mmGtI-x@F|}!D+|S&S>$9}Zb4PAM zElG_@RAVY9xA$JmKz*lBuQJz$CPkB);2bGsljUc~dG#5J)Ap>MmXMJqV#yg8+iVwG z?L4dfrkmyu=BHpeOGd2-{`di%>_jerrt1*o@vl9C`WRvSB)tfCesn8qx|!lIh-D#v zYCNm}fkj3Z#a5wA$&F^gF4~Z-&y~2$Te9FToQ|0#2k;=mNxnI6wizkaLrxypk{-2a z&;6ryt5JM^YUq{OJNnSEb|pWU!hcr(3u+kru}h&DcEr~YJ9nGX1G9w3T_c!NR{Q8e92zZmSPid|?jc)N?ipgDAeA{hf3 zULJIGL}5CkO~oBol#&_$)KN4K`vrck8i^Pg8XP`;dEN?VHDP-X={gl8zzblLR!ZmwQ)HPrL>lx-al0IZ4fjH7WCGSkKFl74rXZ$kK z#-2s1CwQZ90i(?N>x^bn2(pf9WbPH6ijNhRxGWRJTk5?8O{A#$ul!R|Luo=4<`D=`;^2%rq3M89N#TQeig7vu1)m`!MBdJ$NNXlfI}?c!jS5NHJcAt+KzSq zI@I1FoO!(BX?~fyF&_82r8CiRN5uX9-cOydmZJs)4nw}{K?rM-5hN|ju|&2YOYkJAgA)x_4F|GZ0S=l zrYvl5kA+y{`H2jnPpA1&7p{^V89sI45qzaKrB&vLAY3iZi*Mscq`C@f`1bI)&PSSB z^CGG&XRtglHH>{*ysDGxid;u>bA!qTO52{u#gIe}mj0k3?G1#jJ7d2!cZ->0_1-Mh z3bcZLh94y7I9(Phsy8O?6C#;C1`nMHt!>{%D&Ub+0i$;}s=V8&4+e$cz4g)m0Twj*#$HK$97 zR~U#UE#wCib%d;M)#K$Xk&8#zMH|0dtU-!cVmv*<>{Vz^+7 zq+0P+M4I3v45YXR>cu}(pT3cKC^T_KWy0-=#dF|eMp66>;?HA;i5@W~$wq96q96{% zjU^D3iPeW-I(*5=RJPKBIaxASGX72O<7JzaKp?rdY`kcgB``!8M=v5tzxoK2{@E?& z{1LMo+jsx3SI)Ux()k1QG5UsTN^bHNon>z`dcv^2e;81Rxyq<~g`#(9+Wia-{~dAX zuBWmmI=Ua99?LNT*G*;y6Vs5A@XP`;dtc<1kY&1KQhRRS1V_SxG9P)1JiTB?*ke$d ztUqx2zh6!4A)xo4eVad}zll9c|F_+%u${GwjjgbSp{4DAI6(iiajj9){f&+N?Lkm2hnOeuZT zA*(8DVcV99_0mR+u`eq%9Adph2LIH!$qwvyXBy+%T+qp=sHQzzVY8{|mk{qvUxsnY zG!3`oz^qS$NBP70bN;2#jI$m}s!x3$W~!U#Z#;vA89!Hxv-s?Wa1SgDT9CX{K;0Tc z{_^y`L@J4FsOAct_K{{ez(Zz%ZZmU4MFz{!N(sGsMT)d6n#zLh5mGylN`g(s3p{5N zaGYCPDSiRdJHlYP;i=bVa(3WSyu+)^9K)?tECU%A7%nb*dKtZSNEHU^RVZ|?C^A_fw8>M8vV>#s^MS2<;Ip2^_d74Nje>)`c=JoheYcH75^a= zg&K(5dS}VHsPx+A!eq)%8?27@%w_4HhFrm3gR!H+y7;1=T`pPFFbfSTcA!@ndMgu? zGnR2vXf$eSd5ppDuFFf8!TI?INC-4*9irw-jm>O|U;w-QAM0= zge%mZ$QUYemF0xB2IAWeiVphL??0-jcL4V*K!p)-9TlbkX=?tV6C&?|18H8R+m!H! z!{kU1T!mXUxvKP0G`j~aH)|;h;ln}E8ASml)M>P<45i=k zCRRWmUq0U_oYV)I_xQHi{dBi|mS(rC=9)WPCFNrvgI})S9-FsXj0zIOw6Cdk*O|!R^y6V19 z9^h~gfPyjJAp9Xyl2Ix!O!9_u93E*#^~r24k7CZ)(bvDnmvl}%9qbGRsppVW5Jpjk zcsxDMXCuW<@SH~q3+-pOh?r(noieVXJ*E9i`o zT@(GjLxn*aV`tTs7w2>f^%og;P!rf&Z&_11+aFE2$52e1r~H3ITe9Bf2RI4@##1#r zgawbUx>VX6NbpC#qFJD7iKj3U=z`S zndFcTNfYpr@U0GBsIp1&evGM(3jt@`yIJsWqQ`g+@6fGzupAuWq!3Nd-Q)JUFPfyI z!yjz|e+hY7glUIcP39U4ijnrqnRh~#!V}bweMvD!GO14$e?Syd3wTsyn%Fl_yb5x} z1X*V@?O4z^`f^BWkgop{x5M*{}VF0&r(lra9sg1!L$N64ds6l~=drRCncA|!LOHjr#y*ODHuliUoind}Uo z`={AH5ZZk;Xz_a1Xk^+8U%w zx;>nLKpf`-ygmxX9e47+Bsp9Uyt4hrLe+qMy1L^@N`#dD@7XmSmRs?zBl|yz7M1qS z8MY2RQelu=iF~w5y{(ArzDviRa@eC)-rcAbaPrAGi1qUnNS$tERxY>d{nvB3?xvc- zcgL=w2@w)|(~|%=K0djqj`AzT7c&wl?bu7HwwmnPRmawncb!++GP*3mIhUp92{QRT zn!b_Vnb^x6<>@t5_*bQnnNTK!ML%qd#{^`r4`c2XC8>9R;E)ev7nRt68k(oC^qxk? z-9UG-RHyer{LZdnxhL|Zoc!h^h_~B&hGWq(4Ezk(Bmsjg6ES^)k@nL@MA5pVX%b{3 zV7Q>l*4>lF_=cmmoJS*lv<*G>FgOGnw*^R5YVpVTeLS*1LeS%J?SfU+!Vv$2ZmD4a zRgY>fGvuP3$o}4s8wJ)>1*YB75G1O*C`g-)r)b$@{i?CAU+6VI39?3hd9hp>R49u% zafU7QXhln&udiZ}$5H$&sGPtv_ArQT>Xw6N@QZCS%{EZoUlSrJx*7AWU?7^yXpvL$ z5=~s-HtW!B!WZ7q{5ifs{#;>GcqMmT_?lzUIn*KZi#`3g-5s-xDUyXW({z(7_6LvU z@OgDX7W43+iI&o+iIaB|=j7JL7N6wMJuD7GKb7my8x z*5sY?iHTRy==3n@E1o(`#L2H;D&V}X^fB7ODBJ-bTq)T;U|m#aKOv7jy+G=C)+q@i z{4Vp1CIRLxz^Qp?`MeMF6-#)1`&<;{` zP2vj|MaSr#jKqN5vgbxWC00$6cVd}sg?Y+@Mwx0-3|pc39rMEYn{%?W#E_L6F*ed% zMp%^Va)$R`CzppeZe%Cl<@t{M${eTmu_S=yTZ|6@y1L2GkUBS%9^V+Cu&@21ey z`1k*uW%j5`+kK;5KSx*NhGkH&B?tqWNfqpIQJ)ijWzGHiQv?=`DghPz?CyOiQ_nXP zEu18^-TYOcw3ZC+9HIhHBraD8Sdv{&7S*oUR<&9d<+vg!_q@j2BtlROGFnYMak%c; zBgGoo=$S&b_2y{WPxVh zEK-~KgGu1%H5=9aq@+O125x*@DjskXx{t{PPD@%2gD`!ug#x8QQmh+eO-EU_H!%v< z=%Ixgqvg4D(3;9(6KUT3gJI+@JvHzpv>T0?_9dUTs>z$5gTL&ec>V8MwX1N{!izJj zSS>ho>U7eQ%5M{9UP}wE+#Br_+EU+!#Ehgt%UW3)3mwfwShWJ$x>OOL@@}+8MJPck z*Jx_RVr}RWvexAFG}mnz)hSq6L)|HY0#Y~JMSKUN#DiJ()OG6fiKrF1D$^EA*>uM{ zRK-~Bh^%7HAu(=>)B@=gg)#87yO*;2*ctU1L|KjuR4f@U30trstCn))ab` zIp_$ABy0|w%4^t^KjOc--ZM#Zva6C2wU9@rxwKD6G3Qs>t=i7fq!R;YP`<=Qr6#Az zx&P3?1e-sGDDsS7HECtGvH+$fB4O3%!)>TrqokACyJLH(O9fAY+&wtB0W6F^(^Vbd zEx}E58d)XTe~z74-aFu0`Re(zAhd;@G^CP+GQvlcdv|l-u87V%=NDZc8V|9vQx5OPx2}zI5PrXTz*K8*3AF#94Y5sDC z+>K*8di;LB*CkZCN+t5df~T7Zq2aOMX<58*BL$tCZ~+P&z97Ndd{9zU9JSD~+zYQI zcAhA*%ZskSwJxY@2ZQs8TLtlsms{vWK9J6KZ$_E{_Dj`eO;n!&{y}#5RQ@;euw7TT z*KnlBGxZyXF{BUORWSuTzZHnqAfCQ(#<@nIIW+B5Z=kx4_rbxJ5`tYqJiq4$7F#ZX z!p$r4Ca-{D*$GBB^(HBjo^Y1)X5jXkuq|2xj8c|8Mp+lJ5DWtmQZM2c@%icBu87L| zG;eyZ%CIzizmU~?9FMmg-hV_r<<~FWX+96niDV*QuEDwt6F!aL<;NNRdLk{bvJ9$m zivNBHJ|SiCy2Cx196?*5J5cPRdH8R{Vy z@!?W+qh6w%-|P8=>`&HcgT>d4tN(yudyNG3`jGXFXbS;;&=|m6GLFxf5|y0DgYo(a zK7Pbtw}RJHgAlDm)3<{9SW`e~z(TCkXB*X|o7%jDUyY7gBk~)1IV4>T;h_)mt;CAc zM%!sZ_8KGennD8kcqlXY#c+2>N_mh12smybo{EwlZU)Q37f^?&XyaRyDMQRfxAW#9 zU$u+HX?puNSyzNfeua$RVQz~?UlWUQ81mK~gWyi7%Mfgf>b^>!lTcPyL-Mb@>pZ?j zDZbny?2r|ks>H1!+?o1i?UW}XYpVG=3BGEtwfboyCw3n?y>z>+Zxg>y2P3RNrr?Q@ z@C352$WIZ7ro)eGbV7B-kY^gm*`rd40TQAq@6O_n#K_EclB`{it2SGpY|$n@ft~K0 za=h6ZL184P`U63MMVRi$m3?R~*$NaZzwm|?pwAh1^wq0Rpj@ormy9u~<}cfEeF7^b zqs#0Fb;x`JzkdE#waL)R?wW+F;siT;St?_@( zAbS4a7(Y2EYYhw$LF@TejfUV~n$<0;g4Ch@OA<;2oZLALlYdjKC0*TIIphvbJ|o^} zmKsbqW;kco9IX4euO3#NJ+I9)BjJjc<`6qNp0l%@a!P0Oul$}o&g}an?6Q&ee`i#%hVG@3qIYw$RLYv82pFymV zIFDRh4Qjkv*+>ZT{v&Ec%SF|Fa?~n>EbJa(SDT4fI+w(dnNoY|*`k$p>XF>K;$;D2 zM`xNH`dCfe2tyt>z1T#AvXz@U_=zXg4m`NEIXXVGDr+zW-5Z=GxK^P{^z|w0STr>q z=%a?AvyT;U$doHIrA#cbY-Fx~RmNa;y(*K0x|Q^`(r$ceT~T;KCt#rZ_2jvC5N&mE zk#;TDgFHN~V*N8a+ibVeFf-cA!pJBP=v;UEIm^c07-y4H&H`{5uka{MZ!*DsciSRJ z3s|;C>IuG0%hynCLs6^ar_mWDwQf=bN)2`ys;q>R6(FV^CoVHa>A~L|@WFFUA){+= zGGPC85FH2%opn$Z)Ei>cS*{P)0|y`E_>!-6M`#XCHEAY(-@Jw5c%u3joS@r;#T93c zK-5&U(`IYuD#<%@LHCukw%wt^QAOvx&6SiltJ}-p6o1U6L>D^b@xoScucC`OphRcw zlq(h7R5y@`<2;srzC%w$nVMJCDM*5Q(gxM`WasZFoaJE3Trx6vcjX6e8W;0i={QK( zXVCdtFf=4G1GPiuu^UFx{d9i@+uQ`fnTYidJR++I$71sr^F7j)fIogcaX?YGYf+31 zuP6qeAi=-LHjS0t52Bq>Cc-VeF&^vGBV5BARzGrH(5no1XX}h4JGwb1(zCX?nv=KZE$sVa9&|Pdw|5DClCQVC73?V`2h>^nWEJ*0SSJ(ni*NR3?nrqKa{r{6Dwj zEL|mt_;$n`Xq{90!}hsn_DK7|`Y0I==1M^1Rz=}{{rg`cj(pTp_=M3uk}}o_I*J!yQw;6x=)(TBl z&>*aph;(jXjY=VhNB`*lLbwoIO+RgJ$8ZJdSC(Hy0=OV z!ODocXo0F^tL_j60dTAB*L{hJ6ny<%i7C~ul(4J-S!;;dfo+qTppXa5rmDTT~G-Rimt>V!4bD0J7 zcd$8tatv+QZ8H|{QjszD+)eUq1L#Tw`=cMxHp=Y-|DZt{d~f;I@0!uSw~=sE@^9* z2Ij#*Jy$Y?lyI8#;G3-Bq)!GWyqI-f`}3x+0gu`oPA3W4X-@3WNN@Mqdp0ISUx@(t zBF}J>XjB^R#tC|8IIU-QKmFRXeS=5rAOf?fXHkCJMgob(=pc}?-AqzZrT*zoLDoIR z-lrZ7q$}-MaO4R;4wl7!g!d>7CywvPKsGPOX)2$xy|DUA&hgQUfb=!>tr8!bO7+=H z(Spb7-F1#;>u?JOZL5Vh$9oHP>-nr51*b8;jFwNO&XGupP3TOeAI4X1CG_xbc?OY} zV@B_<`Ib8H@@yRB#i zd$Cn}8_0RE6%oypCYRl(y1CvQG1^;6*x^1dtx7>)vyM?BT`DArG6Gc#Onxn&0x7Q{a<<>B1Wv23>g0FNGJkrwHg=@i9ebp@a?Lx&|rV zwlJH5kcX%bS4s`W3V(C}6j)e*Go(SoXV3uo^A}LLi?y(xhdFw(!x*7-RAv&gU1gXR zIwcmB55=)XcdANel<0N{M+vL$8UApndp#;Sq7k@lt$$lMu^3RU5)Z_wMM#I zSlK>y_8MG8&W%>kP~6j3zPc-0$R<^U#zMxW)ZAqwx4`i+_6^O}NUEfQq)vL;ZZyZM zL>yGTCMbTyWUrdEOe|B%ujyuZ9eHE>tETFXgN(ohZ87ne^-ix%Rv=r&gAj|<;m?7@ zQASTjC%MUaxy7$$Eo%R;cKs9^vAZWza=x5(_2&>vcfnazgJq1r3Nyc zGOBA!Hn+WbDRXEKMg*;jVCN9&dDH2mWly%2bezRtDTD_k14ThREIouRT*mG_7K&>e z24Jr#Gd#6ZWZ^*t$=0CrL?;8|9MG?T;r3%kdBUAgpxX6-A2EfFe+mFafN9?F{18{o z4VJg%tW53FeAb3GWG({7mW4LCTqa0|)FGI^g)tUl5d>H|u)$S%``n3C>OA#HM|uRc zG|73WrWiFO7Km(abLv+7{+RO;;>!1|YrpQ%3YEii5V=ZhWpf*et=28d-E1Pf#O&}~ z!#n|7{(NM1#H&K0@Hx8Hp0Kyi`Rww<4!YZNnFZv4d z2K*wp93jTvhDUSeLpPq*SkEC68%E2yLsD*1$yySNv5>D6QM%9RmZWz=1E{KPnU=QXO|a(Th)*RS3Jyg>Dv{DL)d$empa@ca51 zkjcEQ97vu}cG!cn`z**TdI&g(Vw1kZ1@_H^u4Nk^)lWr!<{7)VbHp;#Gj~@VC|8dO zZV9jb8} zZ89uBZ4P!$2+QD>qC1_Xwoc*aasEn9mUyU=|x=TJ*1EPA=L)$5<2G)LPwaj2T|@OX<0 zF=JnwXANTzJQ2!OuEc+p5!MU9hR?A0CH}H(BR(c}`TM zSZq$O@>QAn99=KvNkK#Jr(8xoEViV`ND0Z)>#tcHo;O7+{y7TfG_~>}d*`;1aDT52 zvJ&d5-aw-3iyPt56qn;N&2-XFC3cL_is%AYt6YdZfc@hauMlP0wno2LHC1~WdZOh@ zhJ2G>JWy@ic+TxaS5QnQ-l`b`yNgA|T2*YSy^|Px$|c2e>oW1gPM2xAR!_}-k4@i0 zg@lU0W~rt58*`+A$T0KIxfXkE}`upxA)q>uW++EFL-d+&5k_kQzr~zWya~4ku zpCn<`9$S%$)n6ZqY*;uLZInK9cJ?X(m8nWiX2^+dm!;d=JR%!6{P(j?13Ky|Z?z8< z<>X&RnJIbd$!Y0X*#&hwFZtU~C~+-JoBi_sPX{LZ$YkVEPI{q1u!i5kMxvSV{6!a- z0rJD7I>b14KIkLz2aYtp$hx1l&qzmS3{d#O=+L@lD7Ep+a)b6MQcwc2xtGCyO%SIb z$!8f6B*6%u3AKBnbFo>zQ*`Cw34DqNiWEkGUO{>U>Hw1RFsOG@;`icB3avQ`KK?WK z*JAlsIoi2mVddnkQ})OKa?fa^@(BJzq=kW9tcN!xy2{QF@WEfw(zuKKGX@eeafPO* zm^%1h@c*?;pezWkd%nwL_Zx(4{NGXyS)mIc2?iKWT*f9vYl4dmPZvt`wXR{ zttk-|>F1%gVEE(d6$JbtWmG+<5F}V=)`_$Sv~!=RL=8WoZ~sy|*4~f7YvvG*Ew{(K%aMZ? z?m4vl1jwuD(k!tgzhJekgkY=`p2)hJVG1k&LJNY}Hk`Fx-n`NL&y2x0U>xQF_=vbS z3h86An{eQmbblJBgEA>Fab77xUT>0$u_=0qe6Uyr5JS_(CX>s%a)CZ5ZCm6;QL+!C zbM_Gi8hnB*a1CKD%I)W!b;g0Gfy(zXObn_)hb^DVwOC&VVsxE zJK=_SCd(tU#o0Q~=l(Hy?pJH%Io$|o70sp}wv3^wc*6JDfP6saTbe8VIzyrh0Ldnz z4dIet4(X6$XWe*|aW#TWsF^$EU!(VI?m@C{E?9wB%rZE4W@lxSIdRWo5z2Nd_AImm$@RMC+I5=AnS9>; zdMf(Cs9oJWbfxi~x&#VF@7j;2an+B_I~H(aMAt^QKj!gKemJHCet{Ps0FUry(Y)UX zXFX`f(`rx!DrL`=re=Tk0cSmw!rt8K^GGd+H{Rl_H9(9bj3wU}-E!l5-Zlv9vM&n6 zU|Ib-^yhMie(p>rNOM@7>{tbSv?mhe?T5}%w~vFE8-hA{q=1qioS=1YgyyLOu_0CV zTX^l8@McCBDBW`cSHJDmFJ!*n>w+m9_V+O#FOi@#16l3PENQleg>BUBA?6zs+1sOW zo39RGf%-1}Fx^Q6o0ktS*aEFI?ifuEE|ArOU9Szo1A36v19qJG&^L-eJ;)azKlqKC z|7a20)@YpF9WEl}OFrm|7E?UwUT`s=wLSlf$wAh;yykkd3_wk`lRub!%!RG zvTc8;`%|15sOTL{@DAYJ^OsKFAZwov-8DqOH&^sagY=CfkO!V5!SEmZSJe-M%HO}g zf4l_1=j}fudOQ(0>=N8N=9=Ex{vC_?c*Pa)9h~||rn31vHf#OT4eA%$SHI7oncAlW z1pnkO->Unal*J(!) zN7q3)TV5&F?XF@1Ln}MkcDZYF38hu{4Y~x5Y!EGKnCP??QSYc;w869=LnWF`jYWk2 z=<955DQW93H+H%>OrKYFM*jW#7Fn_k<>B(X2g5@;!c5)LiT51QWm=pOd{7q(+-2WF zy@KO_3_;3g-+&P}GDS%m!-cvA)=At8$ZKO9qYvv$Xh0`)&3)L!gnK{skM)P3PJN=r zek?bQMnpX7k`a?#ni>EBOyY;3mZZmIy21q+|nPdUZ&p8L38)2HulIR!clizUtm~snD@j<7}Zc zG#uTOXqJFJGh-5YN_8JWOUxB7q5KBA_4hn_DsL3?e@U;1M-5A3 zVOdTs*@rbxAKoy&CDi0ni4AkxXV-a}p|UDPlIhSr98mH^&k2@z>x>Ko|EFRfxW9=j zE5#l~ib~{MDRE^JaF!P?0~eFc^TikEP;$^N&c{U&KTC9m#LzBo6HL(Pu$QMd(4vta zh*GjGE46wC%0$z_u?w9;i?Hc}CETn~TCtA0iK^|GhCd(yd;~~eg zXpt2Ags1tc9KFvi$Am11v>g2m`!RrNbg@>1`3}u~FLt&mV6_A+z!3yuN}}qZ3IS2K zaloWG=LYWo#^2cDPE9zCmAQ?P@5+02FzKfU={bpTQ*?1)jgVxkaW3J(|I6jt(kBvh z*i7I5c}tTLs*6*@TI(>6Qbz6>cRxD*I(z_6^gVq7NDTP6QtA*Olu%dPG#e_g?@7}P z?faYk%I#U-1>B8#6_OVAA*PIB`d>0Nu&G=cM-9cr@NXdKt6YgMzsD7_{DmrM7{kG~ z5Q)R(V?$)8%OJcb?X~xRY#fuYM^=j&@E(8AlsmmBkf`!%26qb%&M`wnjwE6^+?T2m z>F!d9c**Z04*uX*HsOwVLC z^FJWy(?zbBs-3UIf}@RxL(#~fxLyQxh z9rODs6#B~MX(|DTz5cg{VMSb9sE?LxzWm6O1Q4iA?#(c6LkPhc;&3$R z$_jJ3hjF6^G~FODk1A^+40nfaFrRuOY2O}^Fja4?uY4f}Z{ZNHJv~4#G-5fB@KPlG z-C<@sGn*#d_W`>95Doi(cMQr7d z#LiztNHYYWXt$6DA1r5>)k{ApRnvj+s_ADSc7xhbO_iNPhN)}}?3m;OWi#F|(fSDO zBQI3sl`ycqr7$!f$OBD-IIz><7#2#-5D{{B(cn9CqwXB)?w}$C#>N2xDo8ggN-A<{ zA%i?x0-Y8WFj@6ggG9_KQv*7x_EN;fNN-HwBSDxZ-WJLe(Lfn(hhtisn;fOqsEt{ zoIoaAMivH75*vDqy0No8Rw62zo^YviJRCv_@gu{S!P89yCeTYQ&p*zf=(56)6E*(~`T@K>JXb{7i=cjp8l z)IlhGnvxzn%|-gdRkx1#Eyx2>!RIgMJK2MVDiC=|^&n?p#&!sgE;=m_lQ&xXyq|Gi^qCVFWv10$zt$=P!ZCz1~=#TmAKqpMJmS2z!da1CoIcKvJ5H^02yu9KYAnp%23Qlq*I zm2`n~uFj#RlB4b**P__b*-%^P>!K_y3@bz)BiBdFdV@LTTdcp35z9-&HuLl^c9**XPQ#!<<+i>lD3s|dbEi)aU_wibbU*R?jyeRg zY#n>2xYM#*uU#RD#!1Homq$A>2}_sQVV;FWA0StEwlde$mhfQf&4Q#~=89@%rXCZa z;a_!HYsnk60mJ{D58-LqjB%N;k6NZlEf)Kr6WAmY1;KAtud)w6rV=ek>W%0LwQ>sm zL(w}13&p%z%6t+fcC8^(7>OR8LRV`*_ivK%-d)1EQr8>?F?7mpMIiCBgEX!g-3^yM=RFfNa&3`+<{(pigay%eh2< zD10>*#{$9g)CDm1cqa=28%|a@&hdnDx^Ex^rg`3r2csQ*gQ7S9>1a2I_n=~KDyfD2WXA%Usa;#Im8>%29ZjR)E8SZs1g$~gGp?JWKzM5*KjUM-`jwo&DTBtd{{gyJ z|6leikS?5O|DkO}EE}JEC5JT9HzYoJu#j9FLe$PZFsOdC@BykhP!0W_G}fuH0tzO@3{D*>Lup8bZG8gKgvqEv zrQkx_hnZOvuZDo(16ou)9cd%~kT_jSOPePqW~CM?cLoS!fRwyqJr?yP$#1gxL~R}K zgh^189-8lj+P~(7_%S z<Nj0ZLJ#Z zz?ZL>De{w_@+?z5m><cePbZz(mZ4tIHN1?tj(W>%;aM$ zcvmxiFWH1)(JyDvV6@=KZGu$NjAi_`IxDd(H?Un#j_M)!SV|chmZAH{ST?_? zaW-kP%Kn%F@)pkA|^}xEId>K%^#-$F#oCOW8 z0YTVDaRE>6EqCGR2>N;rVddAo+jvcC<=bq7`TF&D$tzTc|F*yL&G9w92l7_G@~)=` ztTgcQA++UCDAyM{jT1DoT`>6&vJiGQVd|lHAy662<9)c6UZZ;8c*#~exrByf!-n<#&PBTl{1u^XiNC%puR0U-}mcsLYAd>i+uK82ks6fFi?V$C7i{M|j__ww636t7I40sb9< zwS;2bb-9Fc`&Mb7vD$c6cuz<4^H0& zNF>7^=pQ`3ZqaWHD={;1W~~mmJqXIxH{eg1o(0>lhdm0=Fmx9nPURPIyr{21K0J^B zOvL!~M}5ji^+8k+zgv6@O3y({pXXXTN9(UY#8Y4cXL?SS$Km$B-?JWQ3mvnL`<|9U zqQSp!vG74E5=#gj;8|0#1+e;_tiItk*nh-LzYMT)R11eI&^V;EQ-hyPMk@}JmNCa? zc27V_&fAZfKUZ)P9N&!Ty;}O3lj7cM(U!My-e5C%+a8%VK1P#X0aoqIF{$2j)L;P) zdzmnvL@yKzMnV5FTCfziTR}t%BIm=uR*kfvi>1Z@q8rhd2ykW@+7&sE)icc-TxFGDWCLC5J>=A738fj3f9^~`brCJ%B2DQQ4}Xk*@`!N zJ!wokLYSQjF)M7Mq&5qVJTt+MX7_l}J)wkN?Bw+@8n% z4y>V-8nJhb#tk)@F6MA4llBhDyUPq#W9;1>{n&IdM|9zy#S zqgLPCjvt4vB;Ht>FL}HPv(4E~&@5SyKGeVtTjj>CZh}geu=fbC0mpO$OCO!n1kw!$ z{6RSNFz&8E2l&$7Ma}2t{zDhc&-0ttpDum4!I&VgwOk;sf{ky|uO&TVG=^pDu-kWJ zQ+%37xjVw$p#!>TScpNeZ3yJhqR?~|Lt5Yh!5T54QKd|Z+M1#3s7(x+a;*i;XClI) zpa6BbnaI8p2y?VS#g7dBB`La;uLw#L#FeGNU`FClhHs5kq-|~=f(yLxw@-N? zRs~1NC8Bty(j6iG6hW<~c;?W&=!&hArcBJnoWEKoWYUp)8_s>etw4nH{ooS&OuE}w z48N8z*u$%y@kIujx!H=~WG;Pm@q0jq>6JQdNL!yc4MnZ{ z(NhuZ&@Ca)qf?F2)YGHuJEMuO2ORcj%HQwmDg1NHwn%Y*b$O!)ui+xAls@(je(9My zp8NPV-m8BVENN2eX=4gPtDhF7u|9Dd(J{_lR#3{pgqG}Y`pe74hhW7}TCZl@YM20TC|HgFfzN61;{pC43h20lY#KzsA7)Dp`BzyEKFOS=FC> zBVtQ=i7|gIzdHn~$H7Buw4;c&!A)eeXDv+8MLTxQ&7i&_oH~MGAD}pa9v>i@L3tD| zP|KGPQh#Ee%O;vZlN5?GD?&n-*d`PY6=sBtO2c@b4iyjm5Xl$OheCuzG?2pwcSrsy z0=JUArPpMV1b?t{|KzvpO62~mc%5BOAqpy^Ej;_3hd?s*7>0KMs)+?e;WeE+k~_$A zXs~HP_>rSe{7fu&3Vb%>(MXSR1UtZEW2eWr9h!-3c4`HhfXbFpCKuT|eL?wMrXZqZGyWBMQ2#I6r}u zQ@oT#VV%|e+BkUl1?f}&Gq>Cc!`LXcbwdJCE|Sm7k9`JJL^(KDPO1=(+XjUCv}=iW z{w!!tJ}SZ>?MS4dviz|s@P>Nu%rJBsX*!aC~L58W&ObIkN zF_q~NP~_c>;Z(VD7Jdph>?xA2iJ~5OlmjdYkd{F|ZopgkH0TbUtESW?Aj%6C-0)# z$D(kT$z!AZl^br}!cL5FP&OB)kL(XZ9U$h*CK zMPR$|46beIDl_1)F|dc}B*h1`bKgp3tz@AuWJ0^x`vy1#F-9PnJ6PS0lq@8Aqyl#;^tB>&u%PJ{GUuhA?bYl!I1kDIS8(p)gkA;UWDD~Ox}pI34R>-O(*MvdFcY%M0ykbTw zLhpd++u-wkJ3Y9DI2X7|!~?6>%KIi2I@5m$RbGd-P`Bh2!(yAf3R<^{&GH$3w#i=>q|*-5H~izM){w4xo4JE&<1 zYfVjnR&{v^r_587WBy?6A+6sZ5#h<(4y#FpD0<|x9R$AT^w%01)4L*V-@$tjHYrGg z_4~7i{gW@5{am%}d zJd}|p*r!i82%kQw|DQf0|9fKipKAq%cEtbs_~%>lrn03Xt`eFTt*ffo5{5!p-b$YI ze9R|F$wnBg!dw$(a2T`-YsdOlTFLm$%*mp@X}cl6N+ucJ`}qBckCB^=bP_NSW~m$h z$>2`sEAC_VX?FLiR#xDc@HyhA5Vb zYC2#q8a8L?1nj!!7!YyjuO?*^{Wy(f39{Luun)Qu7EmcsmfTQBhk*;t(?3aM*{mOM zufh7oT7lyq=Uhgjpd$y{G}-APT)oI3c&Xx4bC(5r;g-bKX0cT%gRE9Qvd7?et6@#VKGBIWU~MZ}mBjm%BAMeDiR%4M3$ z3Yne%|zqi?uYQVL{xMl~!6jRxGU!h*`8wo550di0o#e>~5DQ7aU9iIR+nA zbS}9Y_%5aE%&@B)|1dDxl9q@^?90q&imE((OeL;?XqXW%=pes3n5b;5xXb|Yz)<)g zspDP+hA;+jjQ+Qeg8<`}IsQI+1KHRLI81}6IMKktque2z(n2V)nY*7=)8jNCoxzIf zx$6gA9fwFJZUrQdhlGY>i}7^McWwNh(Q2LvK4E(tY^hd)*sjm;3=+%VBtV1i<%er5 z>MF5`&RAx(?hj(n@AcYN&eN>{=T`N`XMW*_2-JfLc@+f^JeiZRX6p#ow7VBQ z2RxqaZHhWS(U$R*@rU3m!E*QO?V5A!5l+%fUSuE;}j*b0Y#QV8IXo% zn~{V0GKZ=Qir4kKctjA?87a>}LqL+eK7MH6!R^!{SISGcRWlpIw$Yc?Z&v#!n^penM&zjZB#v~%%VzX8r3#WEH#X6r6)Hi|``c+uPkU_zs*qPa2lX$_@Q}4b_hH z1|xG4m>cVo!gp~%mLly;EBz8$GRS29ekz~&9wEj9=+ai=cuburIw==6giN_X5u@VG zVvW)Syt@==@}NSY#I$%3X;SXov;h&ja;$x6f%2T8ZUzZ%eYHl}eO8`=V$K$MyfP_W z#Ny!3(A8F-kDW28`NmR(UmvhCm_LOdc60@IoGNEQ{KwQu$A;{1XvHu=WX&V7DnB92 z?2sslULQ`|TrlD;e=*7S;Ok&A=>a+hND-4=r1S_kZ^BFUMx*vBPX{xas~}a}P3^zp zk^3wxV+0)Xy8#~Mw*5cF19+la(Avtu&e{SfFE*wYK%J4WHnsXAK6a*hjt+(fg1W$k zl^>?|rogk1|3s@(<--|m5&Z+sHGNeo$U2-#sDzeb5+3D#`M%L2&l0Db^4!+#_ z7UypogJe3Tz0uWG*2hift2d8J&&STm)DJFakRH7bf_z!ufEvHT5P%M}gV*^FBw=rG zxO4d$Oa^RQaCCY@6)j?wdsFNtKU4ZnvD@bUz63wdm8(G?KM6R>Cpez+9s4J)Xaw>n zkgkz3=WG9lt^v~Si8ANgecGmLbZqDPr*g(ic^kURc-wAF#1R*{UIDIx9rLF)%)Y{uEf2|FqNgUzZQV$JEv~Vv?T0-E?Ctm)PiTq0kS4XFd_*=Bkj$j7@TSl-8`Z%k zga!kswQ{mr`NDos>PJ@4%aP0H%Y;dZP&ZN`%+9#836<izo;XWH?vz zeeRA-aq3lwC%b6XbCV^yaS|Uob5Q2BGL~}9f}tr!q=MY4s)(J98q_P2u(eLGzGjVt zZ2_m52hCLISMwK>YT6ORT{7Vvma`WA+w}sw_qgKF_MFuH8s@}b*H0wb-3k)FS~A21 z>93+JIL$9yETtq#{Ve$M!P`6BI3U6u2Xlsf|MdrceqE(XIl3$s!)8^hkFyNw9ZBx7 zojq-?9X_Y&`p7%U`IC)m<3mb1IL%V#3&9Yi8D3li5$maRya((6Q zl1XHPgD~kRZYda2```vyx6AqLZ_g(bekvxBmgfGM3AzdZY0r-#zZcUJ2H~Wy3NxP_&N&0~i{ghEVw%D>IPXt*B3m0>f}msikO(Q7w7$zVo?F`0GvEkK82CL|*7f2CeFH7&5B>UdD-vkosjFI_lixo#z#3l8i8XrEkO`NQW?f0unW8LyMeP{L+#}7+$05@ECA;OSpzDDT$iUZ?$mn!G8Tn zXKr?%n2``{$p&KCeDraCYFU7<4w zYmL42QNavyUwrI)Bm~r=l5rO_ojM&F`n`e0h>Wu-wXX>z`f3Kmf~&(fn&a zc>O4d$jjri0(XLot9(TxnQ@#fEAoz&$jO)rjKNrzzg~;9IQX$RTR7AupPl{6gQGQM2+)Yl_ToEZSRTm zF0a8Jw#vRseiV^M`FQkLV5@{^N2oIcXQOq`qHdQorL>_VR^zh~y(Jr{>^<%yj{K^?Kpv z7@lT5b<$OrSObfXp*(|SD-$;9{4b{PU2XCU&Wo%?v-AO~nq5OiyC|eM_8yb`paKEJ zidl|wBfH{zXW)BhXh$LQD&$&(U0T9o83a>WLnt>eR!G&Y9u=emuoM_a`4LUhh2wtT zqKew76KoS)F|3F&n%b$E^OEn2GW@T;pO5UhbB9-VLTwWcu_h8c2@siZsD5f=gS>ac zJQ)u6nJVV)^Z=RRMV;wJnqf8lcuVYggO?oSt<`wVgxRO5K>6r#yQ$`yDdL(@a?PO{ z`fNE3YB?;kaS5XN1Iz)&sdbaDALr%B`<%A7Qz-J5_phcW+P)&A8E8UBflK^We}n0f zGc^{_HCDDWb@;Qu|HljI+Wk3%{V&I(C~ZB@hvbDYZx4}@B6eVuk{AN1dQG$TT48!ZY&$gVBOD}qpQ3wJt6hE5egWR5<6dpdA zpiF09#*>UpOkdcEH!5k4$&Yr-o)>ETk3;yu@3fJE9R8jV0$!iABYDuk^jmWaceNAJ(UF#h)20U5k0KYmSR zvvxcSY>VwaaO)cVP_^v9V_l+sIU08h!E4e#CvPdX6!gW<-P!(W@#eq%rSO8&RI+Lz z{EEvW_~^jba2K60b=HYumHQrCKXfaX2(QMW=Ch;!RO8cEYG{N3LL8zfnHuVwYvrtK zs_=|O@G5A;if~M5p7@EX6&mX8&woWefEk)V2Z%i8-$x!o0{s64b4gGj=BG_nOD}J( z=v*Zsz2TU`GhsQ@lxk<}n+WSBY1F4xC)2mYF4vGx`2(wEn2oF~zR9V>#8)J#$;-vY zpVC#}7*vI^6$y+4CSc$$Ov>vueRT{QDdei!ze7t2t<8yVS;ktIB!3MfG!wHFIio2Q9uE#{%@;E#JD384z8@q*id(}1=rk_%ITivJ z1Fzg!s}?I!R6g{?ofU~1gvb%MV_*Z_bNsbl8rfr;B|#g2*ZH|EJ>8lZXMD}8yMH*Wp0=mi652ZultTK(v|nr=q)@-BuGk+#%^V*9Bq%>138B1uk?L&ucMoF zW4v>IB~ERyKAoGXaA7X@4S3uaUgViadACwi_ve$+K;*Lr+1BwGu9M4y0hZKt9?3W5 zH3R}JBDp$1pSB3f5|*`H17xGv6p7ZqeKAouh|TU$Kses$6_Ascb>j03iG0vZ7-mK#k?(<()Z@N-548h#3p}5I;U2-az+!PXe7-+|lRh(eokgKd54)+}6M5I7^1}GC>`k2}y_@ z9;_kT3_LLvz#znh^`yw4ymOwJ{%tl;V^=l@YANh>2+*nua7uwDvpd8389XG0X~uvf zzV~yaEg@WLz;l!-@ir@@x=`Fjn|R4cV{5EsB#~vV004>~b}mIKOig4<2E@8^CT&Ai z(+r69+Vq}g&{`{v1~no;)#~@zWu=o`oU86;R4Y zG-Z0>CK0?X9uPXA#M>9#=<8GlV!!GafEyw#u0X#enEnR2+Q@f$pD2l<+_DdDRS-~r173( z6*Prz<#S`^<_V*%DC{}6sbgh;hZ7<8=M_@TKaMa{5Od7(y)vL3WY%826>#yh?Y{8N zd4nf!8EsE3;;aQ(ADfEchkk`Fw@64NQaZ@tN0s`{5gob=c;@#06ojqH<35R2}`9XYPBu0?pyr$+ksf2 zmX6=u_j*hfoex=l!8+qw`zQ}3-r+az4J*7OaId!qvL{4_UbfL$Pp}TqwPqgv483zS z{xc#q{1hC)0j!&_ov%iIk12a>?*zwv7*QL* z#9fZH_7tU9Mz%K9b_ z?18_3@=jn==MHhZ{+7}McXboz8b3OIHqYb(KLMUnk{2=h>EY8nh~WC#PYcnUhB`M| zIRs5f=m^oZGLbD)N^cSA{t3W=9^>(Wqc87>-O`e#Z=rtXxqEW)8mVsjYt^HJ`>x}U z5`ks$QN^PUzKb5ULqWmKsy+M8k!K+V30w7109l>*O=BI+YEhDXDS#XOIMwOLYQWnF zLLpgM;xJ{N`r%=&dC-*6R%%|*_c6NnsgWeUhlN9)AMR}?`j@|@&y*bmL14Ave#)2? zIMv1C&U-^Hj+O3p%M(bJJk{wP+031l+SA~8-);g+Ayq!c>>f@2T z8x5j#Kr#$4=roLW>}aT#3nt(*T=(dF>i<%nA>Y<-bBc#P)aXobm+v{L!W+e;ZSp*< z2SZfS@gL4`;LUaBPYBEdL*Q>FmH!V8@elZFmDYXHnBB{?~jb)ut#hZ|I7*uv+A@HPdC9$P0&*F+IJ z(Td?9Dd`OEIn7cc$gU)UYitevbV^tdRRP+P*g=!+aGxhdR=upB-Z^j2)3{zJ)y3KP zcVFp3_p${ivP{7eLjpoXY*mqPG0|x-Y-M<>n9xETmpp@Y$cb7SsdZ>kBXn{zi6Z;SciYoTEVo3!Mpf^SP%~ zrjwu_HPiqAdcQv0zbZ=K15~Ub5d2Lb_^y9%{{MH&`0wO5EniwV9RWZypHl93^c#a94>Cq}6KM>0*O>h#nB@~Uj=b2lM9X3YX zN~sb_OX;}~&JnFn5@Cnoo<3z=G*pgeEOrIW&C$FH2#4poS=eKaS{~x2yk!|Z2xveD zCDQ!_#F{o%m-@I>W-e^|IP34Cl(i;ysD4F}0)Hf3L zP9p2|=k38Kktu`h?g{j1aIUGTPn+!!G~W<1 z)uvB zjYw;U79&+r0d9Tz-6G!;Y{< zV#~97jIsCqflGrL#DqBFStTimLJm}uG0_7vYXRf#*oLKbB55fJ*R^6(hxFkFwzfw5xNALGEYbVoeaCAqB_Zjy{WW(KZwNDUupjQBg^p*=8Hhgo&Vlc zm$J6{pL@vva(Jgrn@vmAFOC>s<$ixMecUgPS_y#mFOg5X411@R^@H^G;g!f(z!L(e z(*t4Z7zT1YX|?q<-KVA+G`%8UMkB$Az!c+>OUnFP?x37br(U@z3t2{~;js}Jh7$>A z+P1Wm9~P*&bODNZ(W7=-%7Q_MsEK%N>IP#4wJfKFimjMJPBH%|7}Nd~jHV1PocUC8k;$$6+5EBB{m#w!CFV^{Npzs zFw<|#gTDq1rP62+J|?nu-*=82HMXF_;c9o3@)xz%btsx#n0r-i3xUHDU|?H)coGIJ z%5o2TjvUL51v;Gv1Jyktda1_#5Yy{*xT-e7=RKJ>!U|j|j3C!?5Bu=t<%`NQbfG8+ z$pZOzaw7T?a16)3NkoF1c%sP#av9XC#(SmNKCqwv0s5JiWv?+1^iv?{f3vrj)3tLj zbuhKI`k%T4|CaaDre{(FTvT>fj<|j@S1bTJ;&-R}&K&TlLekIDEeirQNnl&P4woqoA2pAhNTDlT*Cg4bm zc?LFxngbQP@caWCiXq_4Y0&Z6(@Dk#Xm%A6Da~m!CAiCVe}~!-Hbw!JIl=*+1_D(4 z%Ic58qx?~Lz#oN={inkB9acs#ZcC=6yk2*SO&PEPbN&*T^G)+Lz?^r7FDz7l%Z%j? z8f%@@AVy5Y*MD{|t~6&rgIm{zEAz?&!k=n-;qHZ6nirzxLlU(6F)>v0F784t66LM8 z^fMPf-dtCY=OwX|svE4>+u|$L$o$lE{|SbC9+q6ABwN15mc}P4E`mofVFXk(s)X3? zVWIcWhGpgWLVd*|8K?3Y1i+;CWQZ0|_|Kki+-41f!M~XPZx(ofqs@P72mVvw|H1W( zFSb9x$|3#95W|vzX-}y}=`wy9WXUX@#&rfB%C+*~?W!yk>e{iE?*C^w1B z&{yFDYZ2qP*qE(Ax=-l1_8pNiR2X1HYo5Ci#C5C~&0N$HG(G*{#acTl$6Fs5ewcOp z3$y#9&X%GIynpQ98c*4P_izP{gVpch^GD-5=_|ZY-jE1`c^Qt^bFIVwXnb1FuR#1? ze>5It>>qTmk5vqWUKj|yy0xzop1fO5UI67Fhhf5%9L;vVh~s{y*waA>6HAzMA9wN+ zT73%N#eZ~wMSr(pYXXr+2R4fSrl-RHs_nHh{q#tlR%&xe0=^0-VFRDwRHzL<)d~qv zhQWXUj942{{LMwJ{C4V7Vz8b*@xa(pz(Jirtjx8y7vFW(d(u_QPJj9}b5jct#UAHT z-QQ<9A=bGF7ebLg-QUrFc7GQ> zjCMBR!7?P+m%=>_|;q}I31$f>nWs?I^eJ;0{7i{X{$Ek0ZsJ68`@>a0A^#o-jE zQH}p@T@s#@J;W{1nH=U2yw!fxiTTs(mvM*aZbQdF9Bia#$bLWIXP_wV-oW_2GpmCU zK3k;|HB(#?j^fJaSPcFvWDfM11OR=$09W3xZ>v*;VSAUe8c z@jQK`Q5-=g@*B0XA1rTAG|CkKb5FmR)aJ62jP-GY>mNm%h5qWW0K72ui^+=#LxTHQ zi9>epS(P?D+HWAzN1%oW-@=1|);*WB2zf>B1a=pI*Bw9TP(cbuMMeoz5enq;yTGnU z)LkL7uYzwgSf?*)QlxPT)u3mDKrW9Wkp&C{6LusS4GilUSOU2mt{woa)`V!TRfvo2 z5&{{Koab(cIM3DNIm+$-koj+U_vBnBBPPr-&_vet=Qh`~Kh^rAEO#jzuv+ICX_zwi zs?ZjNmgM1p`?)RC_w9r4j5G5WqDMLIfie(k)2ivz2HJ5bB+h|r08@f0@w&(Z`d^_o z1cRTZ0Z!Kc0M0%BO>eh~hWgf42D)~xl7_DTXaxVih5(c{tbx zlIjx8oPYO13CwE7>?zdLV9Y1sgwoa5FJJ7OPZ&5gHSpvxblqU~pz@bSj+nv zK@)GfoGm86&!19D&Xzo?AKF1jBQ1*fnouils3R<&GKzs*lx7%4%V=kHgG;dC~Dx@`3EV$bGX^vyKm zCy|{OxKfH?WUtxpKOOm9st;b2Hm~9A`A@Qx7RQ60qiiT&{N!ylWiciDEWnb^po%3N zPgwfXkrRU`B!=p)ADFCKPdI$9))<2gVZE=jVae%BjL79DkLxd1)v;-mz5AO%R+Yvh zTl$iGTPYw!YKOBIR$ih=#j`XR1IZi#c=adQe&k&xtmi&p{shg3uK{#SLbS2-Cod=! zX~vSQZF#3bmXfb(0e<%cqK30IxP38#T_)5xZI*0GZyL{e^!|Nmi8`UG&AaJ)2_&N1 z^|mm-3z_FR#1R&Deas!%bz` z8pBCeo&Onzp+3kjtva~!1NR^o5j&sC;Y>Jz0BP9KBRp8>YM);$J-IuC+o6^h^#zld zsGIQXGkojF726OysSBELm0r-W{E;}+mM^7j&XnSlk99x#R^S;|r0f&z#y)q8>?`P2 zc(4a6o-KT|D}L9p`!(wJI0{$)%k!u4Yfp<1#327)F+zz$o1!cwPf)?L(Vc+R{3O~a zD*70LOv>ObHKCUNG5x!1 zkvh&TF8BJ;AZ{}^|FIavx**{c4OA*0pi=!!@09Q{fE%i|4){f6-R$qP-d!^Z3czNYZm#VemR?dB0* zyZf_?&nB8OLxd60Ah-o-OOlRc?#GG>Y8JV}@b6)m&QK~9T~9M=UULklU!u(AVAwL6 zGK&nZsCU~sbQ6;by1xS{qSM;;mbU_)7;VGTxWJFm>jceLWhvEYRa+Jpo)r|OpEDdS zCAjlnzpe{Rm)cY{(n_h2a$Gbvt)G5e42S zntMz^k~XzBs(F3{Y$^KTVmH|`_0w!j%?xLPal(*&R4anNL+W|YVQ2bZWzEu&$5^#< z=V)LR!2NooTRtYg>uyszx>o=oP~ zelj{8EjoHZgzPR~7knZHRO@K&ac!%t-+O}s^jyebo4qISwnEyRC=6|Y8I~D#l*T#k*jk=bQreN|+ zi9*8o=JOfLEpDv0!}wC&=!EnptHI{Wb`AO@AChVbAZQc~AekOczj0@3 ze}DKrEc@yA%IUWrjP#4|R*kmm-#pA<6!Bpyu7>az$Vlv!4D1rh%Q#jzQJ_Se^1y%S zDV`+Xspf5RCLRP-41QO7j3)mj1Psn${benK)Lu#^HO}Lf&w9khWI0 z(k4}=P^d^F89ypii0{Y{tH@H@l<-n7bKF=PNXr zMBnQ4NV~v|gjCA+vEc=BvEV#=Fy21gf0MG^S<6SXN@mYRtSC>6HHIybML$4FHt%YF zOuIM@chzOo#|gXfA!*!eSu~P z+gIk1d0iibtUAFl+=G5de`_Rd&o`7+g#5|RYLme#d9OcBF?YGAQT7;rgb3=Ozkw?Y zlmFh*F?VMQQImiA8cAoIKplFdFZ=k~a1&7#%%ZW&A$%6np%N`bgp-~RUk1U`CZaMf znvNE@;JQh*!M7Yua}u7eGBo!eQ*Oqpr>TQS!{bLf;W%rnZdhnZh&rt3;U_dSaW}@tLC+JaDnlwH&ky zSos7-0**N%wXb{_yX5y-*><4%UPZi6ciroUtY@Dp!H&NFJh}j!BSbcvzRF~7ANq*D zT2Xo+a$gn>djihi0q>uFp?|i>TZUwLj{&88>Ib|dhjEwPl}xRF?eMoI)xfjArg^n> zY^Hy8ouUzp;JcM$+1tckhdEgpThuOL6+G30RO^(jvL(Upd4h< z*P`%Uz_Csq-A%aI@&G{^7Js0~&pa{AK|qxdYQBSQ2`R$-z;1%FNaIk^HQfMm8ym{1 zQcG!}Z2}iwC>5z(X=oc;SQ&n0xOiJiMK}e5i-MG6h^Zb&;q=Ua6s2-isA^h@^el|@ zM6OL0EoG@#qTm;zqg=3XGcz5YLzad(hGr zi-X^TT@<(qLyNywC~yfze{OS0GP%wWw7{HJJO_8{C<#p#L$z8M7;k&(Bz0n2L9yi} zcn+i0A{p|EY&k+~dm>u&$W-|auh~%=-c0^>fp!z+qKtW!dmZ-pbx(E=G|W?s`CSav z+bJY^kSy&ff&W95Ut6b)?hEvD5{qbuqas)1cqgoP1+R-?~4l19IIukqnTGF5-~hO0BBFa*W(BT2PtX&**a(Bt^S*VuD*c zb=5B_;URu<IRF)`Gke z>bnkqWuD#8TBtV={s>70(Hhxdht z_T|^j4pRk`5T{CqOyo@sZeE8Sri&^eC`Z!iZ8~EOE^0r+cIwOuN}Z#ML-~VNG3F+; z!=P8>BpRy;c0!qhYht6@kD_D+hzo2+)Rn{dVl`srX0$n|S}Zga+Q{jZ@U%@lKvrND zIQJjz0O?810iKO`-Rva<)!Oh;!?OJn=P0z5WdO}DthO2`31C7X=F3e

|jG+2NtUOyfECWxKZ)U>oNlvv}N0AVczk1;fi ze$Fr`>ab6M!b07rGl1v+tvW;_SnD4NvUYN+vdb}-q`lUwkDj|wyjCt*giRRzMJn#BX%)vMR;UiTE7`g(S&* z1pdFj(R+O?dFEP%L={-q-Z6*@2EIHpN^Y`T`}Tt}o18Ji4kF1%D9Jr+`nzNy65u*6 ztM;MkYpQy~7d9y#(&A??gO`3=*=J&Drb#!ZF(mw!y<795i5(1bG3ba(B)UrnuGnkr ziiuE7 z7d_@_Py&lSw`$g9933j}#Jbpq)yHcU3fmL@Uc?et21AO-Wy?JgMQhUxp*-du<^pSw zl1ef}7?aOP)x3Aqz*}smLE-aOB{eSB@A#Wi+$|k1V*UCCq{!;e08c>Lk@!{_nG9pO5 zIJhce=3WN9m+?rN~N8k68-SA`v5 zwB2rLEoPx9B$25D)|t>b8>iaQaZ@a02KsKvJD3#`>=Aw$;&5=|_zf#G7r}OzT1Bop zEIxx~4nG|KA?(4i4OwD`Me7VVDE%_J^i_mTB~vCa+$Z40=xx4dGR8KAm6MX+MJdGg z)|2_C78bm!aK?f@K!XQ?jtfb7sd?}>9JTP@NyHX=fh)rraY=)Bd8OJC%oeu&`zYD{ z;IGTp@la;+SqxY#nsBNRo(4*__z!CWSo$Y>YR zG#ky55m%Zz4-WuR0X5oX8!7W{Yr0S+lQ=nx0m`PINg49k6FMZ@Ot2{sahqZ@>k6xD ziAIA5j7SNEl&`d@1>yTInxBsLghCpn21f-wnfX6%zA{zHNwFW72{iIBRC)sPL7V#ye)>elNZ9}l+_>}{}FHuYpbxgoeBaCA_be*-p|uuoc#@|?`e zr*X4l($+z8%CoXNuH*`+g>W_CYXyw9`+91cj>*Ndg{=f@jwqO@zz+k&Jx>3>xL>vuF4e15}m>r%cf$3 z^q09Sb-_?(*2Okv;sm_d>2Fuh`%&nGb;#7JNE)TF=b^mL&MHyLcVwNJ1y^0z6t?P$ z=FBT-#=NCb-Ad8a!EIW4<5*O#bfb$7D{3A90|OGVIAO(FRRrPlrfmF(=X6d~7_%cQ zuQh&Pl5X|b2wV%$mkTaS-J!@#o(P6^*V?{$eYK`fxIh#4$UX)$%mVDSF=ST$y(20d zNZpxQ=2b}dSc)FDLCt+eE{0A$AjhEL(9Y~Duu@Ptn=*5i{l-Jom%fL2)eq<|?`s@p$Y=1L*(!;us7Vtjl_?Blbt*MSw}d zPLegQ4_{4|YRmhiK2vZUO;A}^5=_OJj|(fIaFZ`pk`E_VMNxOB?u2L8J{adQ9K9&q z2T&})XELzDKeD`a6SEE}98L`{Ibinpp=z^agJQ8r9+Q}`%ov?C!C|EUw{%c=h{ovT zdLC zIN<{4`(F<31p+Vty_QOo-+f_&?n@LhUN8vr!jK-ac%6OD5yam&I|HwFECj(Tf{y7B z+FkH0cWW~6M_B>^DS|dxbmB>UG`aCnZ0a-ELm#Rd%xU%F9Hd_PTGbe83V&n{rwl>m z*xbEy1~F2)sZ$;^ak^2UD;`wxk*W^UW|0xv@>pt zDp$dn!tJ}Cz_TlflV`K zhGyd@R9J~X11r$8yw$o3JbxOPQbe?vF^MHKVO_ z5HaSsczHq1;I#M;TH>Ar0&+k#A>-QrA z@*(ElT8UbAE+NaT$HiS@0=9&Vj66ZI)3JZHz)g-S$b13%zid zA0J{+lB&3?U0EnyF=|%t09*d_aAcXSKmARFc3>aSZ{F%m2%%li24*rrpuOLyZLA2P zT}8pD{uIP(6mUNwAqshDCJ%|QMixe)iOoVb;6o3^&uez;R4s|p9xVK2<0wYGb4;PB zrR~7b5+gofHEO{E-IG*>&@NG!o3iq~zcCV(#=kh+NHbxRQwgGRGFlNqQS>yAZ11u8 zd6_}a$9yP#YbKgU5)clrkWi0FZbcwDtI2#=;rrU7P+g8>br+WKTl<>F zD~niBvMOfg@FeON#yqqcJC%QK(38959v&phWQoQ7RFuhwu zje)$$?I?I!1o}r~Gan8OYNXd=!h2Es6sWf7y6{LFWUv9vAqmH#;k~$ZEl1sMVMPt3 z4+=|hnySZQHBmT^-J=+9j7M!)JDZ&B2CIKHqbf_t$`N<#)UF(|n>8ESj%oyBug-*L z0xn)R3yC?EzWCl+2?==x&&3m^?W!wTP3vlY=M`l`b2mWYRzMASYYXOW0H3PRlwC!Z z5>}MgeAykCt(L49$|3MY4Hz9~4`Tsc(%W{Ux`#E8`Ga%)-%z*{_r*pkG0RDQ!0rscpV11C||D=Z9tP@yYiqJUT zzg%jk(NAYoitX%jq9*dzbc|O`hvKX(^Dif3gizv@j{=Ua7_9HiL7e_haLuBQq7Ccz z*UOKN&(5_b3nvpVqn44YDi?^^M3@&F{fEjQ5VhO8@h$~=)LxM@&T} zZ#p0x^Pxtj@u>Z+^zA+D-2Cd2Fs#l9S?3+KH{X;I)t(R?vMw8RdU-CLbDya+z*j=R zc5O#LnnVZStKh)-Y!z4QcYIizZG>j^89;EfmRgSqkT9>UI&r|_d9sr3qB3MToVa}g zYzoI~9#+A}WuJ>%3-MOi%y&s+U@Xw+aTSeFi*5pXNO;O(8;-z8=<&@cih9T#JRlaI z69}b`rK@6>k|tR+nl=hp27K)jRE(MNe53GkKeu^;t_D0o~;6WQXLX`)aL zzuailO!Xoy!kzSbgH_cB{Ri3XZ5Wi*rts+TbmFt(KH{3+S?Bo&cQ)i_g2$Q25b!os zpKBunPJc?inJJiq5oV$DEl6g(nB<&V`XHGfVGP%@{vE{Xx%kNoiw(bbtT>&*59;^W zwgXEIPvF*h%@hXI?S6lRl*;^xhts?voih7ie(TKy z!%;?rm(+K-5ET6ni#FPrJj(UDo;&XdV}QPH5#Ak)ixl;hNz@zM9P0)sL7bE-{zR#j1{y_orMg!46pQfKB98 zIC_kfcSNbKcxe7gr2b zOr||BJ@*+k$aQB$&)R-H^cby4A;?8p2Vs@qwJC8q^pozc*_PR2wUfl|%O_8dU;cJT zmmlQZrn4o`x$!!-c|Qo7X(uDWTBV#ox9BEf5{dR4GTm4jc>VUFGJvj^fv2i^^^}r% z)dOs7{)t9*^h?6LV@u00vqkQa7kPTq8xYpj9FRYOuRXa_n0YeUAw!)gRhND^DIjcP z5CMnmJX=A*3GkO99k$}(;ir6dqde$o_eSrwX2yNBodbn*h{`L#`*SKo%YEjb>udf?+%tl&MF z(%A)>A%3DcwKRVHzKi}4n{@mI-bRd}S$X~v4#bwnbZ?S!mp-3nTgWU`g~?M|#r2Vu zd8u%npEuRlt6482)h_*v#Gh>Jfo@74VdxNHLmD9YooVowhfx|fHZHj!lzwO9W`n*_ zAH%HQo_(pZ9fG476PNzx5y>y`HzMV2`x~a<0(_-|hGbd@TYKypsoS#ZMHy*M=(#w* ztAw+ZE6Uh=p;$ksecW1`^69SvdB0|o8(#vGD#Rr36Cd2{$ zO!hU3)c3Uzi{`M+jRX3E2c|Z!+X&yBu~|ID-i6-^kJ%a1BoKm4Xuj!iNn1i!kN)@l zd07ojzg$cz9s$0mg!#ov&?5#|=7_DfbA(cNl(mY!;8orzxZ#FD<#>k%CrO`7X@qyJShTv^{j) z$8Rjtxwi!lRY*jc zsInAz|5}Yk(QKIc2Vpw9<(h2kqHD=!3P>}e49b^b7tM)y#`$=^i^BU$G2d0thP_6R zC8Hr9QFs2r78uQwMA(TJ595-JJ9T^aj}FhRK1b&&l;@iYuOUXf`N(;WDh!hmsHk9gn}qST zJUx^yUEx(3lH?(o-5M5^b{sHrm+%vsqp^MNXMOK8ClGXl2w%gjy=H$UITPiuK$WK-ncWk0$s4^1 zTk5cgQIoQ1&SvdsDI=1H4IeubdoK+Sjvb?H99rI7l9n1A<+HZf{zcB>)-&=lDn_xg z$hI{?Ma~fqkROU_=?aRdkV?XlKX#@USUhrFu*#kHI*sy;*s6RQDz#_D&edULPstbK zOHsI5XaarhQmJW7^5BgmJMffM)!;1jt#VOqju(f<+7vGvc2+AG6?uOdKt=Thqc z2I3Ts;;>V>&#XIBy!+RcNzf>d2|GhYi*zc|S`$WP*s%_1WwQurq`#8>+|pK&PFH@d~)r)-(MfX<89u zMyRDiZY{8WJb{HZLApbo6luNrV!lS*x+&4=)-4i6`Y++G)%8M|;*^^ai6+EWl;lU$ z;)H!^2z#;r8ETu}E1CY^qBw+8fHA&J>%78)#VoUCR93S1xqqAfXx>xVg4{|2&toE?bXD4YB=xx{q;^YR>Cbsz8!7p)2SG+_T@oYIWGy2c``9dx zyu{T!p7RBe%G3r3{Wg}z9CXC9Yc|^}vq=ZLhwVio?agL!I~EbtXx=pLTFQKn(x|(}KKvekG zISf1aT%dC+B338Z0p8`{6D`^{;mA`J&rT-zPMw0M;?Rz_J3^0P)wb?U8}XhW>7nsE zk30XcV{ZyO(KV8hr!j9te`0(v9i2D2U6^jK!2;rX(Qgk4RRT342Qst1xS1-U{-p%4 zb7K5jP)a?xf#0cHq1+5;RGpbyBKm_^Zb=nbZrM$`%nZy&lwv0w?ww)N+^gc1+=bUD zvyu*wP~nusZVGemqYU0ZVlEP7`jQh`t>KDEHL$bm>y@a?Nx1@XQBt#!Q?rekHnRf9 zBe-ZtSMjS6a23KofeP-J9KM{TI5#OiOhd&>-4W~G|5|)eZI+ETPwY~2;p73h zui@zcmVIy=#LzPX*4Lf(6^_2-fUGLO)r48u5-^;E+^tt+hMIA#P} zMN51;)>G&=xgk(gU7fN&B1ZnWJ~8&f9AaR@UdN!1Ts~ltayd!yUKUe6i^!_m8k5ni z&FoYg*H6ek^LTn3PSfPk|U3msH%t1B0hm6R>yw(Fy8C_QT zN9aN5RX79B#`L;7fQS3f4=>DJdMs$iBE7%h;@1q5p-)BvT}TtV;94+*{XEa#)1}Pp zT{Q7eZ{VQ}mAN8ug=+n>%_G~eYv>K{BfQVgv8n=qXPmvHGnNyEPhNN6KuXYI0K93j zszV&whs7hwvN_EoY9&{#37ad4u#eL#PZ)XO6j$KApw0crG(uK%F?TtYk1IN&i|kN$ zBR=94dZlX#roafSq0mp?idn(UY68iEzase+tB7I22)AprK=tZu2*gL6Qeqq+5-@(5 zYbQ2$w&A^{Sa!rNs1XM*y1}Kb7ljK^a{dpg!{roH+yF_L8rWGYw|!cKLSaVI+>HZz zX_=7-?dmE;3SAQCyD=t32fGTg!j$z6yd;d4np{!9ePL^^?z$0lOZ^mmQ*-=;-I}VD z9pw^N;t1Q>$D;jEHu8>ZMyz)Ls+y2k*u6ijoW>P;mI@c9^YshqnKEZ~tK&C7`4slh5YMim?cr5zsS*$Lq8un}pnfs8NO%vB|FI{j z*Y@DRttLx1GwMS+R+C3>RKSRkq&g#wQ}-F#EGLp;5Bf9H&n|YCrdm8^6`)3d7G{km z$IjEXic>TYWoolXza%6jn;7S@03G|Q&>)8(deUm+1r=jG20wI4DG$U_4lF0}Tc;$c zj5CWX6?z{GW6c2$*AfX1xx9s|ne70itLMwXy)~vfhYM$>jIsPM!a0wxK@urB74gCY zJ83L%!p4QvtPe3J<^54M7W@ZD0J#Q4!$ZYId_ZQGLoK9MNg(yOpb4p-A z){Q|X;j-C?z+z<}lGoZYuxYP=9VR2A)C%&e6MN1Ch5x6tu>+KXq59g0m~kHOQl?De znZ_tE@Wo-kI))5~u@@DN+7v}1E>_&8us$QoE}X4w4per&uA3mIT(>D+^)*?>j4)n& z!$>Q5m_O*gT6-<$L~SoIzIPsxYcBU#ZUd2%Yk?yqybr+%V@L|4n0JBxkt@XHoA2Jv z>SQq$A!3hy_7mlY_e}?@so#XNg+utcjPE9(Z>FXSQg2yc%#)`?8CFu+tj9=qbk zG?G$)84BOxC-hW0w72-ibaIoX#T)PMMC2$DBe);7IsYzZL(9#K8q|mIk^8s9s5woN zD%Py80+%d&V^yO^R(KvfyGpnS_$$B_I1C#V{{`;wBY`U$=4biopeyVkYAav|ME7IU z_?M2iGZDo1YhTL!aBW2!xz3@=sR9^<@!G=W7$*A(^Xi>WUj{~Rau?EZGOlDcJ^aMV zN%sLS=4onTjO!C(#~WYBkYXCO^78-cuH!ReA?QDNYOiXW$Korvr2aQ|bX z+hD>UHksN8UMu7IS7*0V^|<$yO3xJWx#dzWrdfq7de=b;t7Ps*c|D>iMWiUSD0t?4 zvwd1GN{>)nLpar`2tV4T>!Vt8hsfn%PM;_-pQr<^06diNgqh4@)6XUn?Ho>qGbAkisLePkbQXI z9V?VWnB&N@{nN7?;Ju89B)NxjUbEy&i>EC|bF6If$O5Ze$hN;o5QbPLhH58~5T7|9 z+v}vWGe3UWKRA-QZ)Lh-dW@#^3tW!bRCmZ?Rl?>&+5@o!nOFmPvaIl$O~V&}dH{A? z%{mg()(7(AQ;+sgdF?_gjq1r)F2C6wG>3^o>lq@<4zUX>ES>4#2$u*P0Xc@~)t;&2 z8Rhu{K}P-};+H_wfRapHKW)@S!-HDcROT{hU6Q}2eJ8|T3*xgV`e#Ncq|u;tVB(5= zPn+`&lgm!bQP0t2+|!eN|7$Ly=L+4c0WF?F`Cq~?LFH%A7QkM(@^`-!l-x6|et`M; z!6s2w@H1kj@S3|Zh72q~k4vQ_>EkX;{YJL>ehJda=sI1)GjChR7#fjl_%rLH4bezz zBkjXus6y?;)((&NR-RUmZrlo}QPG}*HES|~Mpe3ctLu@+id1nhJrfkbbysvfqmdyb z)YAlc;DZCRMGQROv@H;k@3v-FgS;%0Wm-{V3w~fO}Uvn6wYiZ)+)tYBv9Z zyZN??YzDLiZvY1Z8Utb<_Ws@oyj!3+l|@2^P?@4Tlhn^k=ZbKYS0xl6kuR#YUcApP z3i@cY{s~!=GPCENWGIp=iinG(uH4|B6z=e8c}O~&WHD43K8}lF0jEt8m@>AO!cWT^ zU@4?)!gxWNus~_%z_R{1I($5DyUD$wY<=WfpVQE8pYU&5l6lP7D_dJ2{CzW}%AxlK zx60z=L_(X;b>-KvrC*zn|J2XpYUQ)e+=r&)Mfc^MTt$OM)&RKT<=;AP%}t=C-4~NP zVs}p*9yr=-|E$a{%PWXU8TDOC6H{X$lY|I4&?xSLx+2p1y}Tm@Bm{$x`Vl{5K4_Y` zsO@6c)(d}hYZ<9t+w&ARIFFjK&LU^SfUnxI9UTjyWW$+ik;|E>Glk%TH?8_=_5SQt zX+=I;CAn@QpPh^o&3^mQ+?HP0?MJdX$mhgZ6^3ue-8H1qdtyhPOxMez74cwd%#k8K zdu?mMx@yxJ(NbRmBx?=M4d(tkkixoZ4Wx7D_|zYsHmW@L_&ai6H5{}j$h*!s^H{w1Q03`(SwgxF1vb`WW!&d5bz3_)M>o;B? zfQUVLTLV`{Pd8K>gKdrBFLywcPn_u6I&I#sM7L|hHH*Yy0<=uvFXhQRWYbj*<~(~e zm(}FLD1Op7^O;DrX2{MfBG^+~R1{*`7~|1}!D z8gP^f)PLdQU2wAoK#n-pJ|F%rqWd+ieKt+0q_) zwUT=FTv%<7a3nu9so#FKlKTeQV-s5OCKH-zU(IYilK3}&NC}Mjv*p{>?W$hf zy}-Y*2uxfaG+o#}vT&e#rY*f0!DW ze$g7*eZ?_0eD%jSeT~PrT^-7@Pb)66UCybzu+Yc*wDfct@MhF4MztH_j)1SL5iHVp zCUfN1&Bq-O*TmH=Go08qkGx2)v%Z^MJiVh|;QMENvnTZ5;_@p#MrY=mp4VQ5{NoJz zXBI}T;T}kzUplf4e=8nfo@2bD`Vs%t6AY?bP~6u%J%3hzRrl@g$ulf!yS06Y?Na|6 z>G*v_7R~KS|Ej zuX2?WYeGbkeYD5?OX%qsm&-6!_C&Q|Sa42y?C1d0E65LN%@9SJBox)&adsrv@g z_$+kq93b(=1Kiy&oYWXX-Oel^6Q6j^wg_Z8z zxh0ip(@wLKG0$C|PToV5#wVVa$dC}`W0h15b9=t^R;}0thZmuJWY;=jnqk3?FV|8C zGGMzk#PXJ|c*nNY(u-rb@erFO{KQDGv4wC?B5&f3NK+8z^~Tqg_3q(e*+wYKKe+`b zp?1fj2GZ;lw`M7j?TAT7fogbE+Z_GC&uUJ)WKN_Rq`-0CuN@74wcz#0MtU7m{MDslh8cOYuQHxe$A#x62oa=$k~JXJW<_e%#4Ui&9S_#iWNgeeP2oHQ}dK`t#8k6+rj z4hTVzbu*-!yfGqf#ss8zrY))jlw~Aik)2c~9o||WAg;xYr;0mf*+gHNgD)IErhzX9 z-f+w~+=z{ka%_+Jihal3*t_Pu-w2BKPqSRImkt!Ja_LcplyRm;MmMQqbYo&gb64k+!cMHl z11SKC5_k^(z7q!&qqC$Z=*Kwthv!EROU5fO(uIsnW za}It^1*)8qQUeVyA_y_@*!tatNWN0?9No0Fk@v;g52OGu}mpTqlg5CLw_MN^9)pAG{% zI`3PCIRnGL1(zUWv1nBrbYf!F8*pOq3rte6cvXf`}zXv!az&g-$SW49I0vpehV$wP!NPlCwQ-AFNrl z!>T*zFyhb5`%^UyEq&pseS!0=>bYRh8}o|HQPuwXjIG2U)F&F;n$_F{4;l1R3T#zs z@C4a3L6bv68lby@)}t(Rs`I%8U!H?R$WJWBJ^O?4Jhs-Jd1ZB{EhU;5Q-;K^&M>-9H~a+4@uo;{fV02WO0MR zEB+pqOa3`NjbYd%JID#7xuNfr@EvhL-f7wxLINt>6A>VPnJC}Dc~k!_l)N>3?0Syv zCd2kocYXw*15K?Z>qJaOf7vt$!8xTN-S6mW>rMky+H z5<@U0D2m6Lg&^%R*`6GrQ89Z6tX5fdSFq9f6t>_7gJCs!Gg)Z!`@-iBmnC2JH}lRe z3s*r=wqaWxjXkSbwLYMToHvtm`Tbp*H>s+(UpK|>3RABfYAoD$*T66z(YmT>c$)ST zZE2v$O^5_k$s(%Y)3F5&K0!f_Bn)C8LEu%)!G<~?_<;Dv^&XM-4C@Uod}W_SGDz5Zo4qxsC4N} zr)LTYtD)1BKg6l=Kby>L45P0spx@}+SZlaU;h=KkZF~)#P&sI9;WCz8wYM{twaJk% zuqoZ1j@#g<$7n_QBjRGdq^Df+Pz!3t$K_T=Iv!`KW~(f3XS=OOjh{83zy#VRue+^B`>37fZ^Kz0@0f4b~M?z}=tTuSE7)Ky+lx zq(p!?aEL{fWX_~P0OP=oXfZC$q#GB`?F=@VjMhgP4`6r3$)PJ75O=1~qn_B~(FWxl zZ@&)aLorB8trykkJHN&9l${PCJoJ4QX9zSrA{jnxkQ*PO6DOF_bEc+EByj(uH#|J9 z!y!yCTsjIlq|I31kRQ&<5NSp8JbOWm!fu2(UAyEfF#V%$0JR4Mf)#%|*i^k8{y?G7)-TL8OZ1Wj*X!=%GSO=sqAG!s3F_hY>CeAS6*4$~qEOO3QU`H*R+9#Ep@Uk-Rxk>rbvr=;3yo&J zp65=U(df8d^p8RLZ3RDwt`-FcaU8;jH0Dgyt++9+jCv4TNj|sxG~Ai~vYSg)c`8%d z%COoh>`A@+@YN-as+`k4R#iUaehYEMa~V82a2ORtRjG9G^8Bc)=jh>Lm6?ihF-iEWn)T5tal~b#FcPfHqd` zuoDHqmMJH0PY1rtWHort3|_JS+6rEA_!=EPAbigbUU8@%Z(xBbCur{qQKXaa9@%+!Zx;@z&z}e9oFEzDZ`h-KBtxh}Zc(`CoYW4=o26Q% zS_N7qe?%M70G-xl+Nq~sIY{&9wM9q_-3@1!s>w?G zSHobJf9IODTrnf#E5IPCA8-XR_$AkWH@8f zHcrT=`@8S>N8=!^;KD1;8k@Fx@V>!G-(i$Jm)dCd&1b`ZsbObc$J|oo1OfBywgCe^ zih0<@c#C<6*V(WCSFoxXFTBhz^Ai!D{mPYp+2|@yc=!e9@TH~`3=;e)l>5*#+~TLk zhr8%%X+!w%{uluP>oolb7V>HY9grcN0UO?zXRJQ%19L;I(u#V-m&TbT%XE4GK5@b! zq9N!Zsv)c)t|4^sL=%8Xnn_&W!q?{yR;>u8sRM8Msx?3)qsnaC#cEnO&e9Lk**Hhm zo=hhQp5JIAvd7;H6M&@#tYqL2r;X6j?TW2#^)6$`3kxp!i_ahOIUrcg9oZ_BKE? zunH6xZjP)zh&e|K=kB*sImnKdtR>th8}kh_?TYf)aET2>!(mZe_do(FX0K7NkUm65 z6ABWp3&_uyeq&V5H=wK0NJ?h2vlV@i71v<>&0j8`===DCxvB_LI`A<$ctIKe zHYSFDJYBuV?zAWq z;3jRI2RB9OsbY=i@^>;U1!=AP4ee9-g>DNe*yd=O+C!n;Wjt5)Qtz}Bdus;$aZ+}o zU-7@=%k{9^A)VipE!*#^{U2xBzu6mVUQXuU`Y}^ScXMa2|9Hp9l&rjz?2H}$v;5yS zGId==G-345(yc~&{UeH9h^z8u@t<9^!%)JgEzrWjbChKp4f?B>wN3W+0kHwSm}rS& zXy16XaZd}KbWl#2(}Qh+9eeyVI@L(e9a;AwiW$Xy>K7;^4SA_ zv$_^HTQd54md8nokOnMt>a#vRZNWNqSP_ToV9BgRU!ja_y$%@%+E|;lSgc_xG7Nd% zfAXBpRYAond^di?nEV9d!INDB5F3v@hY|`(n2l?I%vVY-_I{-!6RmT%F>Hx*STRg9 zeJAR6!$OmY0d`H;_$v9!3Ja(4AScZLsJZXY7PWPeuLi97c-%PXRAV!fII~MGlBm+- zV<~9FtjmJ+%mQ0<+j1;}AMF1k%r&QW4h0k}+B|c0>zz`F?<7+k$nRZsFUZGhx`5;5 zuSq@|B05|EuNJ+wgkcmkPP)xOdbBqB%|3m!C#=>_!)dk-{!G22_5*L zCTeDwWxl9akla~F1VCat;-3Y=AFxr3-C*w{6k3KSaY+_5CpAV^f}sBB7jVZzvMQ;< z*P#d3n?*GJQ@$dLMVeLGO;)?dmGl?nzYEEw*vRk?1PBNT5(tRS|Fr%6KMLtTIvdRm zRWwb!Ped}i#%e06`SO~Id70lhXEiuNy69v$L|`Tj%kI*MVgLoFsqLBpdTRz$C11!<{mo&mx`Er2met$RGPBvVw>I?(GulafsokSo- z<6|x&WVopJQV885#;yxB+58IsJsnUFlINr;$AZFyuq+#w<5)83#Ws0rvyq}sN7mFu z9CyWN{t_Ox_3Xx>kq}S5CZ-OWSPf>zrY`oIC5w^GFe;BPySs_0%-?@0=o$-W*L8u9 z7LGjQX^S&g+c@SBRqrgdokjr!m1Y>1^jB6;%=6jVR9u&5W$b#hh*dniqf?> zrhJU6fsDk3oP?bdR_CH%ZX&-jEfxQQ++ zMoW<}&~{szKKX_alQgKg_OLG)2Nkmw~UxIP8v&IopauAMz9UOQN+;S}UX%8_x3JrTx~SV4*NETd=xAAZ$_YsXh^t7U9m%RtBy``1crstQPr=0Nir|^+ z_F%?q@&Qm`4OvarK8BlD@!+lf3up@$Y)MvDz6>>dcB(iCUM>qQ;YRG7-IoAUMdl@P zov4VvNlw~*D;XACU7Neu`dvoCBhVDwQ)bl75LL3Vi2R7938-lm_l4Kwv&R^X^$8@( z#J|o`&F)OvTb}z`hg|ns7_1P8hxAip1PkFTcpPnAH5eX=4De{j51-<3mIF-EbZxpX z`(~8Z$h;e~q!LcqT3cILuxzWxiXt5&%to{6)l61IS;7|?hjO{b@G%{5e^U1bh5y#q zwJKp-ro$(eUyPcU%UVU((QiE0h(j%!uiP3bgOx5R=j+&5V?)}PG5b2@QiSeoJSwe5 z{v$_C2jeoO5`>X2ZbR{nSi$1pa08^BU%eZ5Y6XRz&&|?|Pu4p`W+m5I#O3fUHqVcw zl2$?PH?(lJ#DYhkod#2@aU}~6RX6ZHKgwD!o!P9%j9pem1 z^ru)~`cL3{*7mw|Hy-@#BBke45`R|Nxy-)4$P4?TvB-85>WMu|%l;O82c9t0ZuNI0 zVlRqyfzIG9vx-%Msk^N9Izi~~SETK6oRK66!Ao)0D+@{Zf)WNfz&%4hsz7P4yUr3` zFTYtTUu6y*bF-=K0~G^iessBb29Mv6g0=LZK~5FKJ4%uD-stR;@~JaBlzL zpUWc9xjSrR|7H_AxDO-$eq(jt1!TTG^bmSqZFl&W0O&SH06B?tuEx=HJ0WH3~>&)HNV7jnjK4&R>F|M<>obYOhdmX-uhrf#<!EJ!AYANmhGD2d}=)M{Qhl3YL!hc6a zo~1-uyGKPYSV0EmcO+g6%U+S`662AwM_$$N6RrlACCpv@6C*b=A(}9UH!Rg@sqB#X zzwdOokHkV0Z*RtYqtZz?8AQY3b9rAXu8egjtY)jb2dpo2g!xlnqyy=00+Lo6RL!~P2X)t8 zEOyR>3XcIKB~xI(dipS-4RyQN(*M#l!6EdF?nZa}!WhyFm>cxu)Ftf6gw!R)8Yb`A zXtzsPtl;?O_UR(@Z13X!IEghlJE@Blcw0_{Y^cX zxW>nJSa@;YyF|>D4NZYF|Ff{$+4Zj9v$5*~-=nF}HM9GM;4dV0_Zv^Z>Kc*Tp=|nl z8To4Y#h5~pZhHz{2TReFQ~Z?_pI<{&va5}AJKe57{nEmcLjR}`p*uxy1g$YS{WUPMqNPMx>T)`ZF{0!mi9PTGS-sKP+S{Ib+v0+{e~(hk=gm}}3%_4R+p^eV2@y(a=%Ar^l4wFrMJRpZ z2c2hARGDw_LhbW^SJ9evcP<^XcJ6TleK3>ngB^aaCUIQI`Ve0_%JodbLm+=^(Z426 zJ^V?wZE*oV!ue}T+!_0{n~dzHd}Lz=SWR%Z?>u}GLwObS6C+$fqG=(Kng17S=M+tJp`6wsZ{|OGIJr@EC`KA+cegV#^WWIz%{rxGzSlH~B!-EF7P- z#4Xj&r75nqy@DZqCcAbe3YyBFvMpD`hTEeN<(3pKQVyh+l0FkZ2~zr{t&=LEK=;%m za8klMJpp7|&&jFO!rUR3-v67!=z}hJTGlticyxpbaJD$2$HF3}wqC4q` zF-wF^2!cK74QYn&xYn3J@_{%LYzoZXjotjAPQN?gVd+FCa*SqQ46i%!s$9sBF?Xi&BE zhBoeIJPqU#s*@7?J=lE0=Qke#1-oHIn<2#6vfl810>>eljZkjk|jB88j|? zulU+v(Z9b!QkjzZ$;1o%9O<`b(IItWD(w@K%lZcT-7`)LW%h;W!UbR6K#@dBzr>Dz z7#epbFz)}nW5xh!xbg;0`V;31vN!mFX)ljjfA(pZI8!Li(Q9-s`)ZuHQwpEg{9AnZ zkfSD4>7@Wd?Zn<2X;`q_ja=WkOoB4$saOsT8Np(9fhcomzE)|`>Gy3A4eop26cc$5X46S=^`uP~upUdw<3~D5 z(lY{gnVCyrXr`?fav`Q&@Vx@6#l~o+7d{6jn6{T-rVTB0rbGRj zjqp(-T=^LnU@U@eEmmr3{~kE;I?$#EmWUsM8xFZnGM5 zO5xs?Sj#_YPAi5qdWnYMp*N zTSt^jFPHQ`hU~7#!9i=!7Xwy^x9hmJSFcg*jF~Wog=yiR*M88p9o~>Yf$)3v6HKYF zwlmygFbYrCOeKR&nv+X*yo??!G4mq zM%f9w#9DqR&X7hNBBf1S&8#P(^_i59<@&|ELxyS6+EAx%~< zg2@qUfA4^aOyUk3YgQKG!VUA@xw1Xn}x%Oku1xx$y`_0YjRMlkqFBL!_gFJrc%zPRaj~~qnuD#q9V6M-}_Xr5tU{ZXcKh3hSEWVw)16j+@ z)-D8}yT^OPCCf1?%cTV{Wm=rkSroK0 zMZ=X+0U2+c%qu@KbE$i%m}{IbbACI7ZCmV(bwyj*Ix%PquI^uPRG|RdP-)BlF^Ada zBf7m3dG~nF1OmIVmFWoYiRY;ReZ_H0k=ha;PUWyA-EPm$is1~!@rvPSL$drOtw`ON zd6xNlU1a?T@B$Y2PSe7VJAHfSr-baJO9|P=UJ0x*HBauPK851|AHSi+D~k9auS=x7 z2)O32fV$W;7g>{|?XN4HdeLiiWsTPA4itH-1UGk@3)GK;gZgk;Ok3vH?W?@Wn=|D9 zst?Jtj2oFy8RWHT(&KHjc>+Eh`M{5k<+Fz}5Li=0dwhkbfT+)|7z7imZqJe6PwsU= zTQ+aX-r_%~_v}{#l@8euk%QkBcNGk1G3&@v3Ak(5_gN^W_J33E8?w-|OC+c>r)$HY zs~^l;Bf`%*sGkhD^~kIDIlZJlJrX!QlrTHGm|M+ug09;Yd)c=p6Z&8pddK$G4FBfP z8l&YQ?Ld07OOtI5nM0ymeVct+gtU);(&CYW=>I-6+8>ZAXC zKvT8h0=iz_SmJiP9GcC4QaDW`+Q|7AuB~OBpWn2Po4G19;2nQ?a!yb9*xKRYd3G0O zJGr?_0$U7UW01bEPY#eHttUEUj|_(k>bNTnZ+Kk5f#3oTak~KGfgE7ZPknGoKeZ(w zl(I+h$o3bVE%Vet zKYK8waNQ919JT|mI~=zah*Z_Rc1Oa-w9%AR8fa^&Zg$=ZOsdbS5G$}Jk5f-(TX>PF z>^s&s0x2tB|}5JQA$jec43=fo7r#d~mpbzYb^=$ZJHs2ZYzW zFZV|dhQ*!{oFZ$7kftfp#Jp)dsA*I9xv{o%oc{o$0xkE!cn+JC$F#_MMv2E1%xmK0 zcTTla^Tt_MFvq{9E%!m<0)aiKbD&-PY4@ySxV=?GCY+dTr&m%uh)%LHPUNYu z(aS)al7cOT^!bI$hgmF<6;d?YAFI7?IU8&qXQIpb{HaqHbnyO??z&@#HAiMEcbF=7 zY-;!4R&BVXPUKUjs8hu#$Mql~_DE6tN@>DbIl=K=zqoe6XHxy1gokGE1`vEfbUh$> z9vHfVxo_ra&uSm3-qE^8iXS;*OIqErzhW-VaCnk-j+!5FbA#X7AAxj*@a|VniF(4n z`*sUr-8sMF@$x=i3Em?au5<@)YyED*Xwmj&%e{KgPg3R-Ew02swoCOlNKdTwX@qNas;!`Sdc(T+|LN;^!3mv{my}m%(B$##oO}7MLMeS9dfxlYJ zvGpKf>xQ8-_|pRK@`e!ObOCQgX7;Ny%mlB?COM8@8ec9iQR7I{H5QIjoe3E%ek;z| z`>6C{h5TlNbr5sPB|__gNjnM?rss&mbecJOeWf!EqgBXbI3CT+Sz zN2jri@%n|R_P8T%N~80eyuQ`X*H&H_rN5U>iv0HpSMb5Wnd=iX)`deGGd0vyq13_J zOz6It1d9FJR2iKMW)MQSVKHd5fqF%zl|#lfd+DVIL5>Js&p z67H}bO)}<7??|I&lDFF1^cV2EaOxk_8LqYtDcd!7w1(X-3ku~Yrkutv(bLa}w1^&K zpfu2D>&qq(;o>5x^g)%G0z~J;-~>8}%&Aosxv8DO@RDH5pT=};{gu!f!c%=B(`gQk z(laZA`YcQsPXz2ASh~i6I5R|g3tjsJV%SSu_pn?<%zigrIxQ1C9T#*}(Eg=v9~epI zzV?2{*H)hTofX&6gWHJ+j!Ii$X?K}N5_xytm18kcF=OuXo*JS}N!<2(W??~`^Hhx< zTFibO9{}HRQ-9G8KGvK}THa7oWkKXpz4lTg4#ECnG@cFr^yhPYP+Dn1u`Hr=z^7C` ztaQ;Rho7-M-DL3O~~#%G3M353#T{PWWureHPv?- zKf5kDii51ETyJ%v^^Yw9@6OVa^8|i2G7F12_EPR%DTDda0WbR{tj7_0?hmpNsA7rm zci16QddlnOVDa*_u|I<_nlrff)4yXh@2RTOR9T}c$G{Q4Y@OD9VN~{Bgr0nz06uWP zUw{%wL0vUmicwq=G&xVQ6{R5g2K-oQjecyAbIXUOa6}kvP39%|JC{3_K zKBsapnO~5UjKf;wi+is^FWCwm5gV~836>u&76xn?x0Xxw!1<7I%#6rYp~-#8g<--g zaJtZCv157cVFZqYSqrCXN1UGE!2TR63bSbE(TeN|R>6yN`9x3^jLLCN>;S_~+dt*R ze?zaV*pye`>+ck;x!U-3(wgIZCQ|(3Hoi#6Z?%qWu0(x>K?QtT~5%gCseW>_<${$fg)@432w%ie8qt? zQ?hKZC<#JD;%&*|p76!^K4W=kuT*zeyv)hNsnm|p$-T+O)XCzo7m)Uh=Bb9!I@T*v zM>uWq)tJW19m*rZ{hFc|N_W`h2SdZAr59|&WAh`AtuRYZ&i4%K+=&-a=c~f=9ShB| zt)pM-gIQ1X=>3(}pzAGiPo(L+>zkdAAKC+)&IqSF^Uc1?&NM3TRQD%D=_}+;pX|N+ zCzI{o$*YAAuA1&&&9@IP-rljm&p%$?7*w^lvUfkVZzrFmy%WCstFM22w5j%Y&47Fa zv8HV;m_J)q#%|0OU$w8$ePlUvc$Z`jW1Ap;bO<4KBQ}mHl<0j$uH<5}G{Y&4pbcIv z$F6(}dli5A4nM+29gPi6p5>;!5wM0vP!_)0NfbD&2_$z#5k5kiD0gA19{4Vnj-gyR zQG18r@VA)JBORfhnnF7Sb-!3}goR_w--vgOS0#)t5Q!CO=1pQ^3UPcTQ6doC2q=I8 zHp~v+Z}16~`D{7kjdX>YIQ}F!8HS>9bU{PDDE3)}48ba=Yr$~4)v16~?+ocEgKJN& zHo#kw%-t$m8B`;C^khZyDPu69DxIy*=FTyl)&)&8o5#h?UCzgduX5>Bwd5;o1hla! zWDXklofO<{I6Vg{uu3nL*s=GVw}G)VY@NYe4LKbhE)kkj(}d@wLsUHx%=X|z!Zb?s z5!_kQA8u(9T&XUMOW#FfUyetPQ}dE-(yMo>ZdZk!{A)4d5;uS#wEx$$hekxvt^`3* znK}*z$b*nc0#NrpyWH)0@-iHw~IIX)wEf(c&POsp#degaTHTAilRjKrbxu!srZ zBePI;ia;&GwI&{JmTy8d=~U+ot63eHbGg#%(BurlowGRAuTIHX!8|3gM(VC$Ua+P2 z#WN{1XKa(QJkE0=SG8nxk3v1Z+n}ylHhDm{j*Vj#e&qiBA)5S;`qLRO;x>tP#Q>JKO1Zl$)~luXlfyOLEE`=$de2i1p|1{2P+U zRz!1cX@`Rm$(${tl#0y$g6xq9uM!;$eu{)!i+!lywg)2Izv#q*H}FTEPvrngF0h$U z4`;MFuY#@DDAo5SiRM{GqBWVa;VGj`DsDnR280Y?1TDM-t;XY~JBj)`wP_v^)juctHp*n-=wr5)-HAdPF&F8tvnB#sfz<0j7fg&?T9y)NZvYJsS#~sT9EtsBT?BW`~4p&L_zlqEG+_GewYNv`yc7*gyWca@6N0^a68# ztS%*OBxFE_?+ypP&_$l|S+l$Tc|QFc?;#+K?P0248$qy$s4oZt%$oIBA=JaSeSVgu zHpJxEag6E#FKlQxLPFkb(X}tf7Tm{*?kRALl(Uk3l-J66%j*1=dt-`e

I}N$zzA|U!L$~jpe@$PAW+(1T5*&_?vsVSqvF$kn&aURBh2sn;7#yHcO2CoUjDC6G0pBw?K% zcP=E)bBCm!b^mjR>7d%ISKK$8yrrn2xNs-rP;EsxK`LlN7f)yZ*8Ssu#JL4qy?(Vu zJ$(=UJ?if_#UzE{zFdH;B?bscE*^4tRs)E7jG$YBT9o^kO z?UA0u@YVcBD zHHBYSo|v9waM9ON8y~xd$H8<24X89ri1p7&3}+k!QLO&m7+aZ)ewFZ7e+9q`6q;F$ ziPqz(4^;w=*l*h*7I23$6SpqQQ=zfqJ)w}D#E%ZJGgSv6joE9!9Faz+lWyu?vKt;! z)>^dN9)jIsvWISE-pY5tEv6KM4J8CC4$MyKtsP=9Dy@TA*Bxqz{9eygGpcy1{5g>e z8(%&V=`=VGCQ85(loS>k4k-&+Natk(e8Bt7QBdZrwhM|%xOU_Xna}G`u1J}SugUW~ z1hEa`%(?NL{2dBY;1Sci)@!> zvL=kjQ5R+4?1)*qa2vMA8D{2i5a2l{U)_7QW};M`v%0Oy=1 zYXpN(JRjQB@Z1quTA?WS)1Pe@zhk_Dvjv4HXwmn7E&7D}Dtr0R|LktK|J)`2!AD3+ z-_c3j#?IME(di!!Zf$O3D*KNGw>J0qZ`NU19oh|f8Rc8H!N|pyv=$bqrAI|L18uz` ztpy7N76}tZ90CINuVDj5g8z25vkO{Sb8}0Ja=c2tupo-pXe(jbEjfCn!LmwZt;R*; za!ZTGdqYc$!EE+AF%}XzOaAWa&EyWd<8}9m_YAk|{bA2l=dUa=fqqp`2sCJ^(N(Up zfmQA{kX;1ySX+rxO@}t69-Wlv$$AYqRdTh$zne&JIH0N%C(2MOP{@BR*u7L~-gQ`3 z_AuGCD)SWkJ^UeGd-|N!0Qnc7_jUaJBL0PTfkRtme@R(wRO}-A?Aw&uxk~g{`~O(Tilojm(#%2bbQo_y0T5OSto_{~C&OBl5gSS{ z<%#H{=QOlTHLQUECBYj{d{bu7O=kH=9!<=1HO(0&)OocFg5yE@B2Q+Sh|9o;fR&;ZKG?@rd=vc>Dd;j%iJ^Yx^*-YKvg$B3 zH}1(9&%_;{VPKLPJPC248w0YzpSpT1(V?+Bqg+f1T@&;_kcQbxf2;TStP6Cxjmyoa zjdX~X=N1Mtvasv2m+D3cX1ul(8aRk@@DEGH3F%vqyEyY5iBH5GS*fpLJy;_8F@@NM z&qe+<92JOF$*ii3SxwWe-%1QwFiZ<+b!0)};RZ4>Kvql3wr9D!4|->;u#wJbsvikw z`Dw6m&4vvbfnu_@Q9BR4ODInIlw@&i^fN}2aO*=P`WI+eXXtY`Uw6r)n21qIpce3+ z96;fcN!2-_aum@rAzekoHcAs^Fwo5-<~EsqiE3hx*s(>GvwlLd4Y4sSSQP^+tUfMq zkm52JVq2!m2Ua?r`LnJ#^4D5t4-;&uYBi)5yMvtIf&mt_Y$7L&lM*GS7!y^*YUzz) z4q>rNP!>>w5`Zi0z$~|Q4At(21OQ!Htx8G|Ft>C|xD{&S%-Ix1_DWg_^)3Wn=uCzfgLIYX7;o_pLF6dq*CE0tk{-hHZ-+dWSiK zD1M|Hhw7OY8>A+8Gm@73W=o598)k)l$7KF&E}{6OM8u|r?S{QK%ZW)$M+%y-VT8Y@ zYM2b+ISxH%QYcflxndNTwCbv*d-N*C4?&^yj&@S2j1zmw4&&9O+CA%6lLdu?SN-wU zoLS;j`b6|iaQIe-^Oe6zowmpkq6S9%|UVPnUP^r zdlwrM7bSwL=L_UZ37q=cWy+5gFQ88lDlx4$obvnnVx)6b^X`2?I@X0jQBbyz3z}gT zYsNLr{dWI}FMjDO6}Ax107$|+)I1MIZA*?Zx*e72y1<`1oCr0t&*FBAXgp7AOqEAR zAm(C(lW?@DlGE&=s<1f^r%xT9FAOYYvC3xVoL=;F50ePmg0u(HH|VzX;?Wtq8y;Q- z73^;$G(DH(lQ?HOJXlpE&+6Oh_*Njuu!Ddr!xF_58ID#X=bjxte0Vi#$Xae}ixk{6 z;j)SqHf^T3lBZ0s)C^xdDJzVYeW7b|RcQpc&(uNFLDTslL@oj(FT}z%*kCs3+Zxvi zlyal$c%;Ov?cq{DHQ35IBUsMvjMxZjFYjc1juG=qUdMyUa)hU^fDp^fJVO6N*Owb<-z zr7yWIWK!VGzA2BslGwafWg86Y9z$CF2{SnB%XVYaOWy^_9pOkUF$_=hsm?3p*~dYD z7Rc?rV{f~;5Lh_3f0?X2tOWTl>?-x?asqfQkZ zdy!?9rxQSL3!xT)5Pn`!M-54TD(D00P3e}R=vhkYnykRf+N;1ZxW}jgz6dqaQ8Rr~ zxQ<6i@&VZLP_^WOum_(rEK>tkeIYYK+0!L+CQ1Fb*h>$I>uo9c(85wQtya(@PDGW7 z=&J}}kd{R*dtmH-)zkxr-p+}u*EoxHf%0X-Jb<*5nm+_jMU zAiWw|SF8w#Y^SK8fAax7T8*|=^7;D@&wpP-z1$t#zipx<`M(yN`-S;yXFIW;cx*G$O5dQo&aJz4 zCKurB_S40s1~z4A>1^9f%(jGXUwdsHKnWU$o_uZP^qlUMHoXB9g@YpT=Q zATgL?Cd>#vmG~Wx5(^`UVjS4{2ucVwvm)&GnO=v_*i@#3!pr){4NyzMG^Ef>3q+Rm zQ_O@4(;|lpASH*?&CsCrB7(!hO{x$g?d#P7sD{?nh-L&!=j^i0`b7{2tOZ854JcS5 zS**~m)+k^eBOr9pNS`Cnc@R)}knO#Q6AqCo$#tFN^4t7I!Xr=J>@iRxC(EU(N}%dP zWRO%9y455dhnJajt4f0naF`=4PqFO*&0#Vex>~ZT;jm3ITYOkU(oThK={AVkP0Cv` zoI^NGl3T*I5uT?2d)jWI>q6JzuO?M4z7HW+5xCH-skT+NiJ84i3qB&Ka!~iQNq3f% z40<5xG;rGIeWIyR^)WKE-2^l}0;p3;>Jc=Ili`b`!Ly#Q7jFJU^vLVPCG@04F=yv0R zc$fTadzz{|OMN{Iw2Vk|*c3(@=+*dOIzqX+cqt(ahe4wRMYm)k@haVf2X&;-z+r6c z4%9{GGIc4y45v1=Xu#9oj|N4@ve0tVhpFQRo2)v}!~krET)3EZ|NUbea`WeQHYDwB z@+*FI5p{R?31j=TE|kPhd3g<*$l!fN7e@|qZB6TyaBbCR#FwtngfcE#B^?y>&tWsj|gcptrnet>gJYhYkrMf=4M-JnIN5APGs*c+}M#~D`HuPen^AT^N-Njv79!o6;S*E-rD zp37A`QwSjJxa^nYyx%wQ*t6f`=QQhGnjxOZ*TdcnTeW+@4KI8Ue_r!o;3>Tz@K$c) z5i0EQ!(u013>mz*!eqx@@Y>>!w5(J**$1mJNK z*AE!D5DmmdL7rw=p;AkBcCevpJ)*MU1Gehl#4o zljwKM+0Rgd5_~M#^ovFnNcGJ~vy!u2EZ7W)qw^6N6PiMZHly;hO$bpj=nY!3qtZO5 z2ukYym>8@{q{YhNOTXFtaw?iv+QtDE}!)FbCNEn+q=WdgNIYObObN1gt$nri!qpF@1*b zOpI)`$;rj#Ze0nLn7OBkrB9WPPLxG|5ViYf+aZ?9G=>!!;{$17CiPr`s0fpl-i)$o0HEAI^-AlvQ*DeAtkxR_%4A*MdHaQjD>&a#r+(9OzhF{{2 zVRn1tRn_xGTn6j$&e-$;18c=m+t*gu($9>{nfP_vg@9Pu;lvY5-7LpGW>e>lBn(y~ z$MY_kqEw=#u<`ap8wgr5n3)iSJpd1K?`+7gRUZR6Q!BGECu~lVoa4Wfwd@ETEyyel zC)sK4BZ=*oj$h)4F9woB&_e0R;nk)T`=AwZAt%hA8Fap#&84iyH*}#vEzX>~EUT%Y zWvFCg)J}xZVY}-`{G9Fc*A1@e2^tD4z--73s+4UG>D0W}}Je!%5cU)zq+f zXPAIU+b^kAbp|IRKxu2dv&w`xJo;U>iNq-Cr%1@I1AT73VO+I5&evMmf$M8`3~%4c zo#`(DQ?;RP=Pv8%U6Y>lL$9Gh(*=uD?FXYF8*G0I~y*wi4`(ae4$5g~< zkqJx7)m~N-iy<>bs-pBp4aVM-#sQtmR#mtop|(Qf46fvmh10zvUx)!q?WNssoq|LW zegRv53+tY&I*f9mQUr<`_24eOE%Ygc5JIi^){?dAk)52z3ZaCwm9QI@k?ag>6X_IE zQ_X0nY>9HILWgWh@-jsdODOEKz7D4Y3nTCx02{23Lab z0ez83rw*|&i!);mKuPLsM8`U{yyu2wqM%PPdyOK_^2HKgG$!-el9c37Ga^RHc~ex% zg%f=)rtRnzhc;n{wgiAfTO{?u2~^lwgP<;%j@ctvrt(xGyQV-TTV)Y5d&OZj6i92( zx%5`!nE~1`ZQEmJ>B)SW`Nq0DX=s5uOxv_{BQ00Px+$!#hYwA%)cN(#o^s>DpFi(V zOHh(TUE&#f^w(#J4yYu#0EhB4th(nM^SR;DBm9680+uX=TN&8nPfOz$b(K*?j-SQJ z6zke+w`AJuZLagbBJ9RHbS4dNhTS3)VWZP+M^O$WOa@z0Dyz~L4W2FH;kxp(&NODl zj^=hc+Fw)EF3aU|G^9i!i_=%Fo9m23z607*)AZ>yVUjheFCN>j3ODLUwzF#O)H&Yd z&S)t_m8R_0h2xKGz5V}K-fMe#=Ya5re@!EQ2>w~*JLa9P7HHt}j1pu8%xXBFvK4)y z$Th>#uL3%re=IrETDa%NS!Q|jk7x1gyqkrj5i5xa=6MQhDP=gaC9#T z8_}S%b*Rm%(o?BWbNQ?74g~)BsqH;85pPaR$CLDF5eL67i|;+RE_s$6ICLpFdS8mm zA08cB#@K5CMsu@q<&j9JrFS8VqVlacjGKlR(?UI@tJxGaso}*6o>*kfzuN_t2%dhF z+(Oq1zG3z=u{>98u0iiSkbJWxwFkaLD-XVYsito_`)c0W_|pXYn{rdU0G-!X?haf9 zU1$A*U9w*1+6JVCYhL@BpA>A>DtXA9WC>@HWv|=}J3O+wwWqwfHh8 z;y%s;Q!Rw&L;QI$Bzgr5G2*c(XA#kgQmnW5E-!p^@C|gtl`je|P4wVh60G`SZ|`R7k;k`x%H|bRo*_^&mTiRY zhad1hp*nVC;tgx{bh+*&NL;m}{^%<;5*#Z}&;G@>r3$oLZtt4-+hz8b4izQ@c-hAq z2^!Cq#sm4}Bkc)P-gnIxg!!YsO5wv;5`XkifE4D>jL5nb$~{-0s!`pi*4W#pUH+v zzsok^0QWGydR#PC0MPP4uAZvupw@a{>o!BII2yPor+f%I!l=SZgM$JGT^YF~i}7{Y zuf^jz-Qj}VFtrB-zoN`4tr5t!7*=S;ly3%8t!Q4W*r-#i9<^x(Vl8*ss^G9|Js%QP zW+hdIk14CXOFAE9RpQN&$GZFTM9EdlvX_S0m%Gf2EEiZ*IgPARTOw>h}ArmHQx zT5*M}NYtdL?i9Ge&srhhAxvUc+i(V#T*SEy>l7<7A0)9zz$}tzkrh+1I=8xHd1n5^ z!BL`fL-YiuTi&tmTGO%Y>hF#d~#S$wPS?NDW@v+mdR%s|8$x`5c7a&;cwUqUvq#@rl}4f+wP2f(1g z<^HzAE@9QJp4z$Ph|Q&b=XHrGeGHcR2r;+c>fh$6Hwc`*3$|L60}2l`omo2ASw$#T z`c(Xw_+Z~YEI9!jCUDjR)Sy#TuT}=X$#r}IO>D&{9_6~Qs&Ph^UJhsE0R1wj^j%XQc-YYp3o8jnt`Bwvl?!oM#*Z1Q3Kf{qGbi?Z6za9<}v|qn; z|AXO3!C2qHLEpyI_sq^WK$HM9V1s$R}+B*SjcTuBf-VExHwpp0&N7yprs}0 z22wt$%AoP}2RR8U z+}Ki$?whIW1lqWSr_q^@9@lH{Yww%Yt*>d#Qk~y*P`%IjC@+w&abZYaYQz&Rcq#f~ zLvZAu^wb0)ZNHR*;5oa+Zk>a8nP^4cIwbiNgg_Ot7y*1}ad-PZh+tBPs14XAt!SQI zYgBqsR9<|2qC(XDqTR1)Xdhi}a($!iJE3ehWI|j~vL1I7c)0zg-fqaT;cnQnx30N+ zv9tH1Cm>!sk!0=1v=FZaezf#%RfyA7md2+Mry_(!MoDX7m=k82V#Z(Z&F2*~+t^6cs@VlEu=IMtbIgsB)3APTRzs2{Z9!HOah-rY4hWQ?@lZrlmI^ z;{=Hf;>xle2$C%Uo9D}k; zMU+9g1;}P@o(yKdvR+*0g8A}Vl9H4nNyKQqkeZdGJHlQC&qu4wYMMA#CUz*kCs@gE}*@entY@>PZOB5S8G%}#e) zscE~bEM8&6FyuuoFk8w$^2ys=X(%z##91)HOpE(x*1g-~a}O-dCL$wJ(F~LFnA6#9 zGQ$h)I<6yf2c)s|)lN)}_2|6!$qP}m+n|%gCu^k_s3+p*P~l5bvfK4iJl?KTZKECg z8$CHyyF9WetFTb2_^A6fP_GQxA5T??ik!?o{r8p5ivw42_GCNbb8gc@=lEsuGShU` zKqT#w%tlosKHskqQMLONK(do;et`@yBnL5%t>_snH`8VG zlF{I>o6mX0;G*O@{E2Fb7Ifm_i%6GCjc0ChBdvEZzSt#E2e#x@0EV1l#1ON14Y7X1 zt)4=Lp1>Q0W@N@k7~ltg#-*YGT^&#+!TB8@4T=(1l&3cY>c``d?GGYkVwjJaQ5|N?l;fX%-nB@uJ3RbjemN za@r_A_b!lV8og15Y%_D(OD?nrqgy!MGtFV_H}Kr1qIR(+(IBQ9NJ>-q-bd7#0Jf}V z?3O0n!)~#o90NDo{!z!h8HYw9_UZ9^WkT0G?qN$J2ARJU7__$$xT*C1Xc#Lfmn54m z7DoQ*I4sK_jTRdXwR*epzP{sTJI0w;nWN2a9_=K5i=q}^!~Jq6+#pRsvXbrLcoXj& zKLti;-wwD)G0tr-14rseEl!u}6;0$M6JM}r6DyDJOQr|;HQd~PBJ^*6Sls0rkST!Q zyj$<&17Bh+TTiejk!P#gW|ZP;a|i5NO>_QCRRNv7{|SG9tY?mti7R+cL(NvLb&eWy z-P+1pQPdK`oMv)#q%@44S6i8LqlFECT%Lf(_zoFa2c8Q!L*~gdgD-1wOZ_6IbxV>8 zsWMPor}Lb!%o3jP7KZe)7hHYP@dRuW&{JEbO^abi9?PvY$8Rc0pc$6s)a1FkB~|1e zQkpD?VL+}Katkgq8MDM2O_;`^Ply%eo`h@^7QLzooSY`ff}y;^7Ts;IcU{Y{?`il1 zD#tYz&xupaB^|(l^{Uy}b#;mX0+Dp5j}+%9d+Sj$zA&Pln` zi`Lc;9z!zYD<=_OFQo*Zn-b~r(9JntSay=673;@)`e35UQpm~GHs+V7k%};=x`?oVH?G_RWcO>a@vAST0ADp!KOR<%*2!J%C7%{pB!W?2XKU zW{Z3UZyV+vp#6rtA=QY6RSNE`gR67B>Gvp@2ML93LTal7!A%ejuRH_UI=JPM&YMv= zwc&xJZT=Q)YgBlBcwfAQmEn7oJc!F?Js^Le&0A!f7_3<=Lx*Wgf%sF#wW#HR^o^7y ztUNp6sOI*oI(Z8j&;&cuVZlaoL;I>>xZTIYS8@%-PC9`bDAjOGzp5OW zrn&ur4Jcuz`=Dei1NdQ0FUO)47S#?V3bGUU+Oh5G7k>?#|FCI&kvaAagy-)ZaS+Pj z)eVrJg}T+0XBFs}Cew)XNoxTe=eWveWTn+(rCH>eN*z?x-vf|aGCHuHj-4HKw12r{ zm4Rvt_c4~hy-wt@ZjWY}fog#fZrLvy%LoS7BijabeX%EF3-z)8VXL-pu)^1!zl3dr ztAlScva-isrbX+CsgTgSgs?9#2R)tjk7iXek#uz?K|FO^+@s3u&OMmN=@?tuW#w=EX{`NFxCl zi53fA7CBLIk^oE+7Zjc~DaL$gmm5@zJzV%TMrb5j8z&WTItjPs6(|9%j6bkj0UAvC zI9O+-Sons2IQ|O2z_LsSdy_Ys+q}cKFO;<3Im8_6+bC@?`M)_Z(N@mhK~^>LbxJW0 zn=^Cp9T$8QtEm(bz z2Oa-Y{pnc+!vhna5pME`F$B7Zwxd~^KP}dvz!OPb5V>KOvVeR;|Ihu@Tk?eIw#~0! z?zlj|r2ljKssG!;blDB=rlh>V_MPK6%g&g-g^+-t$A?eI42pnALM}m;W+H$nXD>#o zk3T+?9%N!$Sz~;KspT2op_25=z;#$hb+y4jz4PD6n##}Jmn=>|x*YM{htKTK2mdG`nl=) zH_lN}zoGci&QK$Kw4LlrBarv(!Ra~dg0d+Wgy1`hI)!8}L&t8zq~ecIySWH8 zA4`xqK}{lf$_r?Zo+Er{=68XQ4B#bj{SFX;5mO8hxR2+IWmC~F{of2|J7Dsw1k20y zl~(OKu=w9Z7qn@V(?oAKBv6hzh5bdNkP5jCqSiKvMk2gV8Q?mpo{RSnxD9ygc~Nrz zA_e1?{rPjo5TpQS7Y@ulCq@yyWL20Zp4P!gW7d zXonE!esI}PII<(TagF5ELm#LC&oW&qO(>$$@q#gsv6>iwrl50hdmqB$QVjFTpo23a z!P`k)AeFL1NfF0~Qt1G-nWi53P6~vJTC`2uAYGIi&3UQQAx5`^0hOJA61*|jN0 zEES?%D9t8xkvvnYfE{IH2&$us7{AIPd>OY*waPx+kjpNYErmw#)B!sgGZq~XQ#y&z zk;TWHE2hl}xo7Nsg+$glOcA!ZujJvF0j?6kJ9p&d!8J%UgX~+>*{HQ|1>c1~V;}Cg zy)R2+p98LgdeS)z`uHr6+7@CXzGXB#GY>Og^#A(=X^HfeDz}2 z{+3FWEURdDYR{rkoja1C^g_|Dd9KqOW~Q;P2IoOrod<*;64Ew@m|VN<%rzk~z!!v~B8V0N12d-JvBW-I7i7gJ7*|Vti{!6KQR$ zBrKjo-X%j{Hg~_6-|?d#2;ey-xunKp<;d`)5q(i&5$Rgzwhxk?LAZIuQ%kw_&F^5e zbBZS&lfax_x+J^&vIM%)2z8Kr3(I}iiF6QsU+VJrl~diK=L&OZYCVUr_VJRQq*M9U zdXDr5hc=V0a5}m!kba7iUQX|OLXRTgx%LI`5R>26hL_atT#4iSEki?jVmmSWYH3nNF7( zDVR2sUd-~yBF*RIOqZOE1S1cjOELAj#KT^dr5@~cnOo|?9!i%As`Yv*T zET+pMqFlFI9_5lgms~>quEIgm>{Pp?-^Gveu*YQ$Uc@DCSt};Bv>lDI>~W1`E{XK;GND_8O9p9FjV^h#hrKT5x@8D|Wtbi=p;nt{beB-tK1eyTIR!5( zJaVO6<&mr98n--#274?a_a9yIIFCGDp5T%vGUzoU&`@336b$)Gef9o%fp7rt8Q4V) z4gS#7I$t>K4>QQ5T!;8q&sp6V%ac|ds1Gzvu3pw0Zt~-EMup!O3i%pp{gE`6Jc&X7 zVqdtasG+gBsj?~L_tgg)YRej%u;>~p7T;-3iyriP6(I0ZEc2zZp zD1kxo?$uoqd8Ep*;vQAO`o=(=zgoi=C#qyo*EZk@H`WE3!j*nM0w%q9nQw(}G#0O; zjQBdGGEm##YibVp8Jv>%_JE{wNx&IRb^8#75TPb^Wp^iGkF173ErZ<78!ru3`-`e) z`Ws9iu!sy-1sibynPvc(sdhA?D&i@=WTEH_J+ca4Lv^s8hTdFVLsUUXT-p^FGk4ky zQy*5|v)&&L`)d7DgLQTODot(UJ0H?ijRZbwprl9=F`S4MOO!U(FY$-w_?FaZ{0?HidA?A9o}EO|v^0Qx zqi8oLuEOA;;Th9cRrwn!Ggch#@-gm6Mi2~K{!qw(!Kx18 zgbE5c2Qo*s%iG12LfD<{tE~RN#FJ&PyN4cAV3>NIxctU=|aAG6EX)4o-gcf+dCL{1% zNEoJbIGo^EOjzo3#EFFo*q^W-`mQt49l)1?o8U(?hiz8{8)_H~i*LfImfj!-ZKw&< zHiz_(2!)Y%ED1C?Bofx$J#shkVTZi#_?;1;@sru*%r>Nj8!Behgr*Clq@~pQn%26foP{4Q%ELA%H5mv<>Y+#dfRT&(*YS}WC) zq?=1eHqe06TN$irLczHlslFx@tVf^{s}zkPe>I_g*Rm*L@(l4up!$gj$pQ>gv85F> z1*p)I>W1z5s}KRP!-h-@S1sLLB5becr(xBl4$e{ot4DNHh8J~A+Ra`#Ew<~6tPjoh zY17abLYow^567OBq;&(4;-C-bX5nr!j^r3Zb!*DP77pva`rSATjrzzUx@8;@xy(rz zrZxl{rql(imLm-Dsoo!|^;Z+CL@>LTL|@1nBezqDG;o5-G?FeJ(O!k>N<<`zaMw8U z4OwTnDMX5cRulwsS(M?oweLU|%i*pbju#78B0%f>+D0ch&kr;$t)!6!J1~?EGQNxH z+>JI@J)sDF6?FLxzt@PE0V zU~%H`yTMjFV)t6u;Ru#=slt;)4{qN`#Ek6nk=)k3#s7gRIu{>8PyuuR(G`J7*5oh-4L}J2s;*bUu#hu zF^Y2c;E2|fV^Bn8K`(7}3lvS|y0z{An^S#cC}#1kn}%X4@3x^h)iW62nCZyG=L8+L zLT(Cwb!9MwjN|fM9o%+>Lsy#u!{YeP)H*Cz8f>a;ZfwM^`>W#_ksIxO$I`dLbs>`5 z-N7(0L)4F?tz0s;!i~#ocimI9GYytw6+YdPrQ7HmCMgxsN8_fHL0Om1Bf-AI@>!(9 z_BBn%=lHqce)3{BhL%+}kyB##0OMcP7(&^KuM+?h3NAwk5Mcb|E0h6Re8;92Eo6{k ze4~Me8NtwWY_+VMXt+C^1Wp5FI$$^UCJz8++gtZ)Z|wwIhCjT<-#zFNYkp`4jW0$w zQ-Z-dzoX3>d7v3UH^~ECw^>nY)c0e%M@um6FvxPWgm}ti;O`!NA%Xm9jHY$R(e1CE z845Nxc2nz#jp){uKNKpV#kz{Fa$(1lv7_?^f}@Md;?pJemiVl@4)v!Q%?(uyyzSdB zc3w^F9!A^_77Tjmd5BU3gCn~|{cz}wVBbr(II}LeBmwbix})88t?2fveus8YS2b`o zVyK&-8WD0dD#M@VHt3cAZsAY5ZrxUQa3xcFR2Qz89Sk&dmoBU;H+-O}zELkwyO#wc z)^bY7-&p4?jt}l2e3H2KQX*x;-K1Qb{xWDvE~~oJjYFcZ*HT}2PB2*FYgnz_$_{Lr zyG{S?e6@Teg|ra&B$dTl7F^OU4#6pjEroQ|qh|+~kg5})75S0d&1r@f4AuKcI4Tp1 zhO!uSsOCDlla!Mjs*S#;r4(w|v9=n7AHF0_t|pwkYnVl&&gyWJpR5d&1yxSGPD?}-BlqC69gFzjy7*kA&8qnGbH8-|X_mG|+v9eAliyWN1EELjLrrK}S2P>JDz`G*KJVjVP3b=*vxr zS6)UMizghY_7^*JXKVE(y8hv=L3PF4cF<;KIIku;N>7bedfZEVVQ1|(m3(rm{GrHG z^{QCKqCQ0S*>*gpgzO0f&=@3OHBvxTH$w`^H8O6B!HBFQ;(e;&PZX_+PkB4y(NPc) zfehJ>HR3EpQ??#RM@0lUOdukHr~lXbB0Bi7cS4}La5fAd%fz~|mf{K>BJRl8RE<6n zR_CFo2x?{qLxEcJ5W}Mgj!h2LVrd(33}F^2;R-kDtc(r#e+7o!Am(>wR(ixZl2Oi% z>Na*1)z30;I+r$;2O9l#Sb>>vTc<_mKUF_QrqiRJ9Jj6QJeUav*K)suR&M7`M^81^ zM*`Z@bbfqt-t{4Ds)$t|Ln&Ejs_a+oukoRJpox(p_)JEFX(@I7nn+ht&ArnTo1V(a z7(B$kx?QMX=i%#-4#o!x^P^^jLydoLG;_MM9R}&fm%ZG-(qC;b-q$Vu>}UgKkqPxZr=kJOnxt7d>6SG~s#+6J%F)ffh9*NUswUvCBcm-Rxgj4cc9h2J3LNV><-Sjok70WlbPN zaXBjNN6WjK!OgoH{kAiTM;DFX&S&{$F-L;6>QHBWR$xDUNCS=nr(v(C|iHZ2N#XPXID zG?pw+Y`e+d;IC$E&Hv&3E-;NQCJwU652ZN_)EURti2s|7PPkA9(Zf*F?cE&qhlm^a z^k8R1ol34iL!;3fT(n_9lZeX~t}>!d;2rr(X}63vyV&jj2lw5fg5O`k?M<>fs&rRI zY#YXUbhEkjzid8tOedE1%kp}pij+WPDTchW8fK2lYe)kbQWIKC;hJEm-f?HB<{C5s zX@6LA&F&h{$%9MgA_<$tYu0hQ!T+DrE9;8DcXYB_=>7NL;Ul-VPrTj!4v(*j_D&2LGiD5f zlXiFUDt4&GZ{DfSSB;7FYH#XQU3d2?afK?<_E>8B(r#>f_bkOoH}|1#UiUxO&0W|u z9O?2n>hk+t*X94ME3V{6=Bp?IdDYn1==Y`3U-rLq^g0I6>F0Rr=PrlNZ4a@MY=@6D zwjVWi{r_$(j%-PHA5c~2oQLG83^s?V9GfXU*8V6Os?Ghp#^#(`Hs8$ev-w?ooy~oG ziOm=CMK)i}*Vx?6Q*7Rw4@9orDZ`xRZ){pRgZ%3AeRa$IAuJ$IrpqZd`w#os=9xUp z=Dm2f&Aw&d+3aif4TFrBfSlzEFU5JLc81yPzw8$VCA$Ua_6MTa?0fcuOP*@W)8rp) z4xBOQ+5QAF>hS3{7hJmJ8MZuA{*l4%&3C&~$ZW2-%a&)!vu$~fJeR@#I>HTSF~=VY z1w*59rUj~V8iGwZHJCp~AFz}YXvk?=>d%SHe-70-ia~i7A!5|0Or!skEzgtZ@A9a( zzs_dM3*?2iyhvWmVBBw!xiik%u}p&(Ew7==wG7g7V(lNK-&|+Q>*d8Rd4nx)l&uW1VmLWD6!NVm&C!-O z$(vpB7F+&B-fGL+IbCI8Bn_ah^*IQx#l;a#~R)Bdzh zB8CTS`5>|*`H+0rmXF9sZTVM32l<%XVq3~WI(}R}Vaq4wQ?`6sp3Ptanu&VPK@Od& zkrS-RG4Y|AVsn~;Ilj7;zSZHJCH@>|h@*051e=l2UGf=QJ}aMd$>(kP0^!4pwtPwc z&6Y3Y1-tT!E&ncGb;;LkYn6Q6=F|8b1`E0h038lKq9)(4c{v}(U~X5nl5qBsE&oBF zeN#Tl;IOVopvX?$dEKKaG8h$|YhP1NeK6dVgO%jO9W6xDPXUsna%LhsXE1E%f;$;Z zF7H@L4)T)bx@y84UyeQxD+gaj1CVoqPXA0}#8__$!k@j2NL!rbTZ22}3#5<%V zH1idX9GV@XE4Q-V!P3Ai~vlEDrMFZG4};hd$u6@Hure?v|+ zwm2$h&e8x1=x|PRgFk`~E0-E^Kv<7rVwBB~;m6wYTY0|C>v+8_zoRz4r^^p?`H?O^ z5sP_T{)fTT3Y>fFG9p$^J%zpGXa~6AAuE?+%cdo)8d0upH6esP3CGDt-C{6k2S%hV zf0qB%fqzM3bY1cnTmCAyA*dA4g>}%@D(R4z*}Dg-8bo3oC56~&FkOVQeTO&_(c9ug zv8}kW5@cueh!tB2dAqG7UT_MK7KZMW+W~u{6oZN`Q2yE-%Rnk_A6vOl1ypXjOTjxH z=1sQp$gd)UNIXV?fp*QhW~0PO8n9@;SD$<;Av2Q$1{cI6uOr(rtb)pKS9# z^F9oY>JCmAzuVDPJyix0`Oa*|X0VSlAk0F;Xzxjw<<1ZQ^E7I9DI# zNqSA#-;|?c&vSytR%!6BM9x$d^81L)+fVK&TlJwX^hHfZc~Q4f{ZxNj?Ll_~s9OWo zpl+FKXLlx%OpdMgRD*46jde`7$VaX0*lI6%I)kFhV7OrAF9lq{_8bp33iPiM6M#3aIK)#G*#?Y&6BA$_^Xfbs(Ms#d#af3lYBCyAY7FtX zz13K>=i1GhV=hLb(@B=uY9BR@!IJrUgv}%%O|W;7UK~c{K#u=qBbIy)kN34y{K8s6W-AU}GcI(P2zTToXv5EF*zXNWhxNEn6L+ zCb`r>wmMi%w$&6=z138@n}*7nnof^1_%N56X{%W@>qV42TODGnVlr%cB(7xGszjB# zRGF>H)uFbkKzl`1+G>uPYpZ!`K1!>&nMvF+Xsg520+(87t3~Q?TOFZ}?5KpqBKrq* ztS4c|BLRJ}t&UPYTP;yliAr{|0?u*PwyIXh@oQ8q)v(m10=8PFmfNaM4R)z|TQ#U4 zgS~2e^c4WY#9Cx<4f@z`(#VaXrQHX%YE(xv7`Xi{i#W!lr3=w67*x~{_J?rlby7j1 z8$oaU4C_4zInIWxoRz`Qa?NK~2DK4Dh!WvJ;Zbvfh!UEe6cDBu^e$Mkno2l&VkC3E z&KXY*M>qTG0yP1D2x+u%3bKzV`Zq+I*EA9(horQ?AgA4wCTtrqb337ux@YRwKCY_d z4Rk1q2p%GgL)bTw0BNSn3i&#!&QqI1AuNNU$DGd>t8&Hnr!ZInljzntrL^Nr)`% zfTLp_{nath9H~JcnI^&+JsiZ1naLp3t~|mL$5!L3(#+nU6Gm1`lUf~_Oa|E~3;f}R z;qkRlAZD)!R9jxJPP5e?3c+KhhxXl21&u+}J4r3nwkc`M=0GAGV_kE7197b3xx-EN<^I)SW6~Na zhJq%?7uK+eY<*v#ccn#4+Xx{ni{irWP{G(_mf(`+0NUKNppk69o2@RO(O*ac^CA-U zqjsJNn4MMQraGpDe;10`>S7`W1o@?8B6hwE(>_d`LBIH5vKXeIZio=*7bTsdrihD4 zQ`gv0E_Io0on@VETYl9NG5+J1@olx1_~GTKv|H7xg^2dg490h26ztF-C{|av)Rne8 zPhDlJtJMh%s(+7dR<^oEeu1>q^=-8~DaBUTs_SfZy}H4rZnRaay2(~I6E59?`j)lC zsUIVLwi~Qrt2^Yawz`w%VxYQ{I zX>)x=aAla_a6c+cv?3MOT#PvXD5KAhs`sM^(5{<->VeL4`BpVGH`3xa8^d7q_S%lJ z2g5NDKf^mJ5e$UG!G?GUB3F%(P(?G&L%lzaACHaUcO)?Z$2Er}pRTSlWGsLliGg~y zdXO08lJ1XoWMH95YO9B2tF0a;Mf(wj>fc{^mPnq@gt>jEglk$#^dp*pjRxQt6ZZL4S0v%9>8(20AF zJaf?x^;LD*ve@>%8~PSS1Qa$tTIO=J7GSI)#xur6`={A zYrNmexYDM-;`!50x!(j-RA(^|M)KV{ax#%)fT&X3MSfyqdo7vvVP)N&4zWITrRK1B}f`LYX4k zHAp^)aJvi+EcKySbWRFw+AT;{Lx&_MNP|96mpDBgUoYI=Yj;!7Bqc&T$=C1@YXS{5 z!A?d%NEc{|;C}CfxrqA&S!Tyn^O3JoJ*S9}3n}nhKO`bX6zwcuH9~3P_89_sJ!v@^ zrD);-WOK1;2I)<~gj5ETMkGg`WJiuA{Rr5QoyljmIXn6%fvMwEFC+-K%t7jOqDjJc z1QMS0M;eADn~Y>&(-i0IHZr2htjD+l;pz2_L;$@)h@<{6+A(qa=NVl5`!L^K3PC4y zllnXYI{r91upv9~g_L>b3m|c^2ngkUcS0H^`%;G+r?|-%pdBBc>Oj~zZ<;~T&eXRv zO)k`*59{h$iM69#ZpZT+RM|;qI)6rCpL?=#|>< zVYa>;vO8#J=oqV?<*#due3edzCSglUyZ%#}3AoX)++Xcrh#mK-EZXdf%$iPULmoD{ z4jGX*p6xg;=YVreVs^4O8lK5MEwS5&nRiuGbB8bXIRmEfC{#A|F%RamX^Avgumeuk zUUCiFDVm*UK1rpCR4XYO%P7XjIoF90wb-x^(g$L@miu?qX5tRR6XuH-yLnJWG7Vh! znZQ($lpWiFaQuNr`crHuE{#IjiT*J=xv0?!=g6G^Bk63Ew{$1-?%dblh=(KZF4tdR zN?LJw;&)opFn3XSjy~{}rcR)0L&wDg!YB%-A&Bvw2rFEyzx{=fGI(iT(dUbT6gt-v zt8ogLPi>i+)P4~LOq>NUZFk|T3xOWKRs~A z5Kp4{#EwTzs-A0(L>d;U`^POmgx0-8|@3gtP^Si&a3FN4Gu5jd=ru30Q!&_3r4yV!|70FL( zrNyPqSsDtiBpu2~V#1zEFcUNvHg#os98yPfQHjpgNGxL>m}eF@4(HsnZvi z7f&u-Tv|3~@${1NISUrgpG6;YFPU6!Ud%0>HoanTc}3aO>6Ml34vtQ&aiZ4Obw3!? z?gV4n^#R3yj;f*A_C?=t zI><;TPaAPZ!eXkgA=p6FU>J6)^WjwE=1!s*#eMUjXCOQ|91hg#<+2(2tGK;JbkP4v zzfGo+Xze)1-w zg_c+NYnxG;8ogM~tSFmXZlr9K$)-{^J-t&_J5NsZPxr_^=Nkwq5#(`vjx$2OT3Xtt zQzJ->L_bzAwBxrBlR7ZSsGJ@9WC7JlF{}poNyszua$?7rv{*S=TsC!bagrmz<4Wt^ zN6)oy9}E?z1BL>|(dG$+XHA>oxMz|oa0??8qQ7#BNjRY4$ucI@SkjyhnbY{&yGXY4 z8G_`>ZNkC+NrLnevXE1E6$(h~@W5ovMwDe5VxNFnfeQGV{KP1UF&Z>8i>6 zE+cn#g!7CBSseX3!T24BqhxV~QC%|b;|dJ&M(oO$7flbAy|Xmbab0O8szG{jZ+mXVL`%iiZa>r)nwrU| zKeFrjaQw0V#{92zK88S~7LFxFm@GNj!hYw>@Mzz{=JOZvN7<7Co)ukxiHAfF(&wvJ z#oS4IY=_S0nJ#F1A;w-aH4-%GSY1Z}G|sfWph>3L+H{rErDi4l0kIhV#;g| zYb~KVb}}J@McssCj=aa#>!5QG-Z;-NvO{CXz!N%~z*BR&92p`dlOspS+DnYr z%b{h>USkJclLk4tF5q*#Y|d?Jm7@u1PWn(+Im>o*7-IJz0YR+tHxFTZdP(+(8A)Yx z$`YJW2i^qLcA$4iCDXA?^$^Wji43OfGHE0^Z!z5oz>BbI29tKLPU9*tYIzNX5_E2B zILdp?#6%M~X&$39QrM@~`{($=%Oj2m6I>Ze+_{}3@MeaOe1NOXYJ?8N*(ZrZt$6#D zOoTnz$Lv!8$ONEMitrBT$RhpwJ^Mlb{>XmPzyD!B>)-#fUx2Y+@s_h~c)=gKr*oC` zui#SuD(=$1Zl0olJ+X2;6};TW{EVmRdLDs(bnpz`dwI`TI*h^RjJW4Vq={kDx zZ2Ub4-}UBw@VT#kvM?8bD>sjAfRwv*MlW7HI{B}l;4$70?|PeTyg%QA1_drXY{m!R zy#SP8y4r!JTJ{D=ED6lc&D#u~1$i4FbpzPBY(1pW#hu%;dOh^Wwbny=Zr*z6nVY{J zGIBRV=7J57wH|umZ8qNaUJre8H$va_;Kh`F3wlBS4KQFm49vX;25y8wn3#jv_FNBx z^V(pq(t<5u7i@wd46XoghFgvonL`WZW*D}>E3<`b0}OvKx4_y2BRD(+BikUi4X|~N z)#PI}1=O)Z5!G75Nb+G?tn1DqJGbgse z0obt~ZE#@bBy*qM1_!ml!JA+*2ZF7oVOs-qeG<~(8bl-3DHsk~{!Ygnp{2k7R*WdIks{o-;)o<{_cetYN<3nIE>@A^P2T`V7WUNXJt=)L4aD1X6C7nBbfFh@Jzb}zIBCDW-TLdjF-$xj z2I2JsaKneN7kmVx;8U0Y|3a+$0xIAuXoP>m1@JXo3*W#k@GaZ}-@zmBJv;_KAg=z1 z*!C0R<Huz}2CqnKplm|_!|i!Ed+tcH15h^4aQ5WCJ~Y3yv)gI&Qg*mW#R zW9UPWg>Ad}aAcbn^n`(Y1Rsekt}h(Ta}h7o*g#m$^LRe=VB=seFW{pforRzeAB`n? zVxKRax|6b!`k=@zY^Pl-Xn3e(;uv9*d?~7wSmJQKuxS)}Z(d{Z& z$cl6tQO5NSWw_a&`FNxP1{2tFz8~KoDWixj;uElD3o2LzFGLnCG>s5{H2EiDe*KOg zfbWP}6uw)-6w=FqoT!Ev?*vREF_21pZ3Fm_o3&%UtT%XA9}ObgWOcg5_#}RiQ_X06 zL+>OZ!JW|xl&0o^afM~H@WFg?lEOXP7apFp@Dx5ZNnv;U!g)yxPs46&ZozvYoJrVy zO)a;%g)tefPsWP~r>#QnTF?g7ZQ#c{WU%d+eCxJNRPjcr zDKWpbrNoxFtRU2S=>$dkd zL*0U0g8T-kr_628fFdQh85$$WM-zi;gAiT9kvTH7HxE!8v;7fk3Slst2qW16Fp3?B z7NMn$&m%35aJYU^lmXaGLtGc*nO;Cwz4 z?>Ne#Gx;o%4sZg5c@f?Tlvu;~Y<>uqeGk%kF)zVdi7s;SovC9;w2snf9c9rvhB&n1 z1`8==4O@ZuQXY{mYz*cje7V-(OB3OXo|g;rBGaO08U=Lo7C0tyt7oyaz{y)(5S>Dy zWrB$Ed_v+;`O!z^m0ootsMc*;;&^iJ7*Cd(C$B}7fRd!=$`3`JB%u~(t3q?9 zSy0II*QQNg<-V*o@rvu>}Iei(Xy77HJ_B>6)3^k_} zrFjp-$`%|j`=nqCJI0X6l`MqGt6Jbq^DM4{(HZWN7#Xf6GSsh7<#DKRO@s=Mc`&AF z6$*yrAv7hY(l55vyZscj!l(>)>39{X{aCEsD>RiKN0qyD<*sa{E6+vAsn)~sSj+J! zXK@fG&|J`C8=R=eMTJkAZ)V7(FQzv0=-!CKy9$ulbIlO$hnc9I^6|t75@|mWPy1f= zFr3^1ldzIrH0`;Ya2haE0m1WB!LkK%hlk+gHaImdiy7kvIIXgE+bxED#PVZzF!{n( zs82Geye%-%%)+?5hhSA3{9y~^I(Kbw`W6^qGM^DkYlAa4!5@*AB?2U2;CaZW&cgc7 zHl^oY4}DF}a|}`+Pw&>FT7!ytT1;Pf0NFmUvp}-*P#T{Pec8n@lwAq=>}nX#u7Qc{ zTA0hOgC*=n#PL?d@tfczb~9YW{(@TBt+0;W1`o2k;BmGNUSfB{Rx}1aW^M2t+rR|d zh?>J@)EMq#z1jWD%N}6+u?JZ>dx))M53@DwQFbExD?6WUVHdHd*q_9p2yye^2UiU+~JLLqr9;NIT)oW zSm&gh_(lAhM5aC!?_D^f;j$r>9-C3v;ofB0Oz-t z!SBb(;DJt`I34}v)OsY=N(N}oX1Jh}TJuoJy|8_~AM1K^OxdVCn8S}Gvs2$K=0}m0 zOD6?8ZB=7iWIvAB468aF!9~S+#vHgfFVek$i|<8;iSyV3x(!$Vt`?D zNPow-!ak%Lp9t5jQAsO+K6xR3x&lA(nPU7ed-fH;1d z!*em^npT*c4A>DMrxswQYtgKu->bF{`XkY@2R<7?e+UAs7(rix04aqbyd1{zLt#H& z2?ydJr}DW_%;&+Od_FAVhd~uz2sQk02qLYn;fqng90g|~EuF{xDCuk9DqaiMBgi7t zu^;5~ee*Bjj|-{@iOvA+qs-;DOt3h2jIB8XNauC74^ z_!ubX$HF{*92~)qN3frO%GHU`#7~0b_{ne*KLx>lDqPG@gUk3IG`MSU;N>u#Hz2rO zsKOtlS&tj4a6*F!_!O9i)7Yq$wo#6%Mwr_Y1$T>{45shQ1aTBHM{A`#70%VlcFd$S z`N^a-?}j9rl#X{Wg#TeZN}8UL#`y-T4Ygbk*%_{n4$Qovlo(6F9)_+f;kQ4 zB{3e3_?@a5kA+hb@op`PdADG76yU2hZ7{Tb<~sl%4i$Q`V!kZG?`FQjaf-Hpsbe(Z zqXrKlqaq|X;L)lkF%isn@yHPlyF3Q(~7p2Z%z5$24Q4eRF zX(HOq(M~t%Zl&&~L~DH#<+a zY=FOPfLq&%a{h3fijWg$IA9B!10RFbp-Em`g1za1Z;y1`dxSZvG~aRL5fl(CCvJw@ z@a63!!gMY|40EIM$UFDIc(*k^B|9bCeJxy#s(i}WLJxVq5c3dy$OlKYd6n0LaE?_? zSr!$hl2dN9H`QiGpdV1prr$G+6VpRI)Ya?0>SA{nrecXRXzcfp*_j@BG6Q{q zikEa=e55X3l2E)Pu6ShHJY*$og7pkcU8$5u*Hu7wx~?_@`3(%bDK|o<*F#Smsf4NQ zd1^JW@Stenk%hLl`p8MI-I3e28mDF(Y-;`WrTmz4c>Dxz<)6WW{9o`i{{mj%U&71$ zD|i>ne8j(j|M2ga#lL51{0BCe|HvlupI8b151Yq-W~=xw><|1`cCp3S28*-DEQ`Ht zDYn&eam#Y^bSs7LY1#Z>D~*?1J^2DFgCA~X5*@LAum@a)^nM&tY8spgdui@(!!fWo zKOVVyD!kyZs#JK{VO2I9iejDEk_U~FTlooSCb-~WXyqqjJ~!%D*JCYNkiu4>>^ccc zq_QQ*GO1<{+kj~&qvS|M)5}T29R2wzD6wo*n8)d~RDLkFMnc}^xd=g>#cf92qrZj?^nz4Ci7VU!@g^G=3;QQ!5g~`2PHlSc||}8O_ha zw-V=O7(W~D6tmbOehxWWwLBvRZeTGH`(*lgF80&(zx>__xxzR z7ht_ne&|TT3pLle$Z##VQFBp%Vfl%kpuQ1L(7lC1y9qYaT0~51&{lvo2>M$&Fw`0x z<=6eO#fwpV37n&WjvEGRkMi(g{8D_&^xb6yDLw8S?X01aEo30-TrJUDYaK-^eiWDE z|DU64bXS9=+nJHK1*UC=`)E1seo}oOFvdIGKDY@UGTf3@!XBpIHh9Ew4Vb4#O_rF0 zYB0^SH57z39F>6)kYSC4!B#%xS_QDTH43I%W1^;B8bbdH6wIWN%!rP6M&cr%cG_Kq zqTA>p7D^d{egJ!PYDq5Y*?%o1t@$x>j9}UphYRFt<=N4}M8`9Kom3q zB)Dtaf#y1Xy)&ho9nehA#kqW{ONQMGjg7|>X0{TCJqLPO3!slR7y4TBU=M3P>}MSY z)2xM2oZcfkr2V2p>Kiwt>Cqug$01d>8`2H@MrTN0Izy_;#lAj`|IciIXG^qx z!u*I6F5j7ZW3>I20gr`!qXX8yR?CNIyackXD%jJih62kEW36SduT=x%ty(Cwmcm>s z086ap(Us9OH0xWnvQ-!b!`vts#>au7Dhh@w1j9}3z;H9a#R0=-4jAf_%`6V;&@C{8 zezfndU&ME35~7Z~-v*Ea&@PVvB$o^{s}V8aXvBaJadhMhsX1i>;MV zXRVH6!n7y|7DqvFa2yEgqadh95R7UEg1_)v9S}V3fS|;H06YHN26(UlGEhHLeUBX3T1}7YP`z)8@b|#8CSel(kSvsM;697uV{n4BIRE$l#CWr z+TfK!MK!oG+|}dVnSU=#(KVnUfZDWELk0QDPoNvLEndaS?!qzjf_^XnUP7Wg9I-M2 zqY6w1>k^b$mm!SSLO<&&7+_tF)Bk6j`YT|BbtS^;8pQPLpu)NV$-Wg1w{As@zX^QS z%}`_A0?Vwwz-sF@IN7=#&av)@x`u~9mg5E<0qKq#co6hOnBI*LuW{bG(bzO^Q^2R+ zUZWSN&W%F+TrDz~zPm@Wm<%|MuSYA;!rrgp8}LrxNDe@Ia1+)%7bfs~@lM6r_cTkJ z`F+l!DJ_{&WYABW;Wb3A*Go3T8=bl~|4~}dv!M;%B!Rm%pKR8MDP|Tu$p+dCFLYY| zE%Mh`ZSXcto62x|#=A1#Ep%&B$(`+TkxBH=2fdOSc@J$jZ03FQ^nu0(ob#5=@N%d1 ze@LDm%Knjd0DUa(6ALVB4q8`=96p~UM+gSBP!Empj-#k8L-YXaZg5-oz%Xk)tQ&?dIbJx{T0r)9)ruQEpV0f1l(jj z1&>=#!*kX%@UrzRY_*<)_pRsQbL&O;(RxWENF&Obl@1EHP?WrT%eIbB6^QCC;(ew5B>;$)KQ$tKsWi% zZ|4H|D}T)RyV+#y54C=AB>pC1s5{6~x?N-;q|oT%6$rttUV&5cslI+zKvR1{BxEOF zY=nPBu;=qmD&(|o@C6}I&sA;krB}x6xn)gPy$wfN??AwM7tXTYgA1(pqe2@88C{${hi}o|w4QJpe_Y!peUZwZKwuM3 zJd!_&z!q>8+C@+Cr$NR-HFU}Iqa~lAa18z|f6n2EH)sVA;O&T`8JYN3>*3$VO;#Im z=aVDsn*2z}&m_`ut$cw7{bu;OQ)qvqT}ofI!M72AS}Pn#Q52cqt%L8^!4GZlV=Lr! zwoZ_#v;L&B{s*(ht5d&C{M-(AMOmx%SzDih%la3jTc1O3>nj*&{Tq4uH>gp4hZ@!Q zIAK2^OZ^d!uzmu+^&ixle}Oe<*_*B=O+fmAT8^zfR?IiSz{3U1dMmQ2Wq$M#S>O@9pohqYk)k(@6@6f` z*aN1CzA#;Qp-A+DYSAB#76YQ*&XFh#h;InUh5?Sb>4E)>X$eJ9XjMm{H9Zbmp(wOM z2(7j_55KA7{FvdX;rBQGvJ)e)%ESmTXL0zyYuQ|9MH*#$Nq!_4pihSTm`Rzx#6Uoy zH~BHYf5F4BiXwO1$HqbievL-%Y)cxsQ$VZO3<^aSV`lN6p@p2C;eK+w&X~olY`KAn z2fgAU#x^m@pcN(%t9cOB6W7Azci}uA12@1;C`0J? zHm2Y$HWYFF@pzY+7>rmp1hU0Y7$k57b9RFkq47RK4Mw{%od}-FGfR1>;uP$ zec?P_N$|Wl2&s1}yeFo?Ct^B$A!e{N zF_ZNcv)E8k!bXT9mMdnn(c%y`TNJYeqLdvj%2l(pxh%Gw<; z8sEMOlBGHH=rY?_XMTiAPq&MY|ln2{D(t@B!Wy&&X2Y=q#ZrJ$+vdG z&zR?bnx8F3es&aMnh*Ke669x9$j|)9&z2!St3fQQMSivv`B?z@+46XPrX5CI!Ox!F zAwPSIzwKCTPdV@_NBFT+6Wzk>^(?JRpfd%e-fW9c_Snhx<{ zJzMd$PRqEFoDnQjTpf9rh5Cg~xgI9?5*-kv$jV0Ek0r?MH3;hRRYVWH7O>j^C09tCVJk?^Y+de=Yll95f0Kj6D zNek~tyxMv=9&6C6$k#GbWzz9?$t~UkqOA2`Rm^DEs ztesPgs6m%@+qP}n#%bHOZM*xlZQHhO+qP}{Klx`e$#*?-Q^}ju?Nh1Rd%tTH)uOVc zcQK`?)y|0Sww{@4?O}X~=x|9-rz#SXTlO}-P#a$Bb!Wejr~TW+UhmT_eKXUQm4#;& zwJ1c-a)9#X**zZoXFLq_J3oQHU&7A+=fu{AW%Okr*dD683Hn5yJWOG(pg08%d>%I( ze8moZIeW$C)eSWK&QcdUo~W!-TC}JeaA7FAtg=ZkrVaA_Fj}6#R-ncn`(bR;AioVb zC5$^?U@KVB7`F<=Z4lC2D?7O08VY9$_CQxX7S;yswSVxy*IkeYfIb|2pYz?9^>SrRzJ?a@Lwvq4vvZdBNK?X&xe*D0KO^RobYN+vw;U1?FZMLy+7`jTL zYb*wxzFui~cTMErwo0UHq6VS1UTf(3O7@x}-!b(q3?3E#QTrZ}9x3;s{lH0=w&$ez0J=xnd;D@p?aTSz&Nu!|lpgyf=w}Qc z{Nhaj+GElEhY>%R=Ewk}a1~#1&&L?HX|;clOmwH9`?kwudA^l)=Aj@{XP+SvMSZ~3 z;!?6tvL!agGtM3jyU2~1qAj5Kd>e+2BZ-!cS6@%~iSfmSq%9JiGhuW1HQZd$n*!xd zPCW78oI$7*(%OaZH^~pIsd;+zx0o3Bl87cI7tg@K+ahZCKC%!x8l5=ulc$7`lQu8p zXz4|S#!#qpq~}Nv^A>zVIpIq61@k_ac2230aK#UI^H4NyrTSmh8kClcwHuUH%UJY4 z^+7SYfU+SqTu0K$Fm*-I*Q%>&q;ibz?^;X2{rzTy`R5#!=4@%17TaId9rgx)+A8pG zxPoUpO)5Wq?+n%+G1u6uCaZm|+U#v38k^VtD64SX-uCokn!6G^eJ`|^rx#&co;HTT z%>C>2krzwA+;S}0-LVj!p%64vThl?bBRl;;=*F(}9cMdLZ0!AMYa?Q4FApyp({^YI z(pb8*^1SG1s;L)KFGRS)NA?EB;O@ra*4Fh;2Hlo`*fwQ6P*r13=GHzGyOR3Lnvn0V z^!Yt-YpTgM_7y?7DggYGi-Ys_oqz2$Ry?tKF=}H(>s27y+t<8R%F>rVfQcI)mno_k zif5YOpJw&lh@RF}!C+dZok?qC9W-ok8saT1fXAM@+AWszKqSOkSgtljvLB>!x z)Jprx>&P<+nD&8FL7T>CTa;e?rs=S;6FYZRSHM~({BPnfp|wYNcY!#43p*T~b zW8LaGI0G-qLhH_)Q7U8mR}h>5w2AFBG>>ag8hx@!Mr-&-J6F=_{XZn5jnGHJuJv27 zI#jP_tA{?7S$(!C!BsMNV?Si%BAA1!96`U>GBMI`5C-?GzhDh>Y3rGOlf8kuG`L+~ zUL+0HE#L_k@b0UbDt!pL;UKxgFTtE&V$zesB@Z~i;lJRX1BG?Lzf-@^9Fp$2zXKM; zW4Rc2$ZrZ|l6oNgmu-P8Z79A=4&c0T8W2C!Ju<-v*m;NX2%0pJc85h>^kzadbD0$< z4ZzbSRIZc>oS6D_uFTsSQ9OrN|L|T<8bi!6Zl+4h=B~NJ&9k<8D<#u(S%j|DCKgV$ zX*$DdgozQh6pV*0!p2)ELk^kqvP1J3Tp{|nTZ$=@3Lmm=0UaSH32qNf>I|uf8>{9) zTWUty-W2H9Rv?_nTaIYPV?<}f@c#V`By>Sm=E@@2Sg`w87eV&(*)c3a0I&?+949;A zOQ*E0$%b4M=tfk*>e+>;bRM)Gyg7Ufk#A>m)rxzZk-Wd+hSE<=0)OAKRISVu`eu5`v z#OdT|Q|~C&!Z+tY*5R=BL=zy*Y8dDWB^&yS3gt!Ydv5Af)tE1hD^l`P}Z{r(lEFIFNlQ4(Uj$L{u-(YNwPI7B4vRex3 z1TKf}$-}0@x(X^HWgQX}2kGKIw`aSI<_8P6J$DH|*-_@wy@W>tqRp4lqTFMkaYX2X z8QMX@P9RUIVgfZ!{UGfi|J=xFa^H<9GF4;69`qw25qWr0+rUH zGoAlvkCpqkj?G4=ikda$>aKVAw=+#^I98 z8ZjSmj)qPfh9Bgt%e5!4jzGtv>5VIoM6TRDC_9C9LvUJyx9L_JrXM6<>bfGj32IGi zC#}}_k6o^qJ$OG=cluh!Rc$A=+)KRwnoVkX;Co=>6W{I`9cO#%wa46&-<~8Md-AEY z+-o=j*Ct?li0@2h9+~!N?u^E!zCJb{2WAw`3T-l=4P*e~U2in#y9hf39^$yEK^=4` zWGcp)VYjA*YQ#X(L`R2j^&6%bdN_AEz#e8qcF2c&%t$V4zu(&Xn^@&^qs}xeJh836 z?kiV@Cdc4>Gjq&bgf9$>;&H22`Xy}NELE+M+E*p#w|4QsgleWQfq4OzsB55nPJI%< zmyptrj|P}L6S?8+!iyEHO1xKvIYq09WLbGrajw!RFTY)kVNvtr<=;p6ZWg7^>5`^Q z%!XG4ovL?N^NlGwc_c5Zb%vAOai*x*{p)KiR{74GTSO=>v{(JaYp>wNYrB&lsEB<8 zRE8rS$`K{S{bL9=l?W*k#vE5OWpe%MvpM9%NGl<2I-1q5ha4gzZ*U7GiM1P`aR;pt zB7He%RB>Q(UzOj4c823>CS6WR+C@;fhb3DZ7rRU0{6#&8wk^NHQ;m7GwIb$jN~oG; zQ>_PO%I=h@N{DJfQ9YYyGsn&}TC>6#m3{W?`oMn_Q(T>|;Yeg#%_Nf0qOD$ve=TcJ z1YnFwhRs{sr(vs&o_;lKh2I2dZTZ(5F=0`#6!|4&9q zsTBHNGp|Yfb@zAZOp4a}ai}%vVfFF@>z;sz}`~_o;z_=`) zJ1Ce1Nx17%Z5$ONG^+&t54u5dudd1Yvhp<`E4l0QtLccUe(}FZV0Dx$y5d|=CO5(| zf9b}I4x7Z*VPWs=tRN~hDDN<;Sg(oN@oiy;8_LbEVV6F+tl5x1GzaTan#x`mn>>4r zu-DfX7#3*b2z!kOAglMw%%bWA z2+}i&X%zA5J3Cf~%7?!>u}D$~43Q>fxGWCYBYFBSyxcW^xC4K85RF?kEKPaQetbF8 zx9BCW38)MxFVhcpdv=W(iZyAu7;aQ1H^m2Bvn%FdpBv~ZB}uE80n@Mj^c_+)IE!@D zGDYHQl&QefPNbXwBzo<=(eAxOj8aZIB$+rQ(8PqoHCE-*{G;FqPS9I2(_4_P)wK$n z%mq*6X`$zv!|Mo>)}bjA7)|7H3t&2vQeZ12D_%OHY#X$twMRNIHetb&tuT)=Y=d+w zc|QAr)0HfqWK?)pGhl}bNm;&U5K=alkAWo7M77;(H?cTo?8INSohY9iU$dc~A_nRN zkW#SdrMb+eNy~dIO%JfCiuB5#5oIr#3<*Ggh3;hdv{h}ilNZn8uQNADbb>jgL<{jv z8zxuBUAr*ciC(+OAw!@EN~%60bcu2|Js%C(kznib48PP8%4*kc4Ce}5EP&Hxe`em!#9M82_UBv&=!#b!l29H) zPrdvp>J;CDE1P2OE-Tb6GNNZmD8hmnUsP$T{}}7=;x4@#ILmM_K#Fy#S?_aqv8`bb zzA@UwDnxZY2q*hhV`Gd80hx8-uC3L8gK8N-%;xZGC3X2p$+KXjjQq!Ke;lu&7`IO~ zxR}S~8Ag*Vxwiz8Wca8SpcL&LEojeg2G@l`H#iwjfL~&HDbJYcQD$EAQ(;7}2r8{myiqan&Z9aFT+a7y${2)Kbb6O& zjYY3~_b%s(VlNl@!f^rVmB$@krNZwOq1}%yqxvbTIH51Y`eAQQ`W2!*y{+K-iLW@- zD+BwfhI-IbDI8v!Pmz~Vjj(K4Sc0m|)wDS-V_jiz$zm12wUbdGR~NYuw(8*;uTUh{ z5^(8gmdrNDE|_VWQSfV`Q!>*~D|f20S{`2Ey+FDIbSZBZ(J}SO(*#ko3Kr9$TUgE= zR(B~*tD!>-ExQyhw?uVXmMM_jrb(&L8nU=xbLi<%(k{6{QY&{k5Os-FFI|ZQSI%_T zwn3mTi$0KPmguTwlghtIEWa>F&&P}RS<*WOcuw1rvWaiMaNv~XC4^pHe9(W6qEVfl zXuse*SDKyVdJeEwrJrD4l6vrdZqTmOOIKjL~W>5{WajxB3FLcXwh@A4Gusn|)z zFY}wIUGjRse$M|=@FDrF@hc@Ozg-4; z4y;h%FV}p=Z`S^VqvR$h2dt~voz1H^yX1!x8Na` zunRO8r4(H{V5EvyLg|Z;Wp`|p0lj_!LcZG>XEbji9^nV_U8P{0qCuk=nS9Vx`O~{n z!^0Or^x3Oc+gWLuAZ^N&9wa;969&1zR3?Dn0xBo`CvGrM=nhOlSeO$9F`gQVDCe)X z0wRhyf-u5AdZ_|BYL|kx-IC6V(em?aMTZ$g-g|xS>`%P2jN9t-PG!X{cSS~n;wdJ_ z`rO9Or0YhAO*khI*V?pz!UvY82C^=i4-~p0H63Z9YaeWX2;m0MVHxWyq2MyZBvoXF zYCn#p(Z{}ais0PLH6_S(R%|Kg*-Cq?ic+L4rtX)ZjSS_C;hNla+cahk@aqC&)`aur z04>_7X?wIm(r~*%EAmsZ5vN^CC#0rg`n_-6$y`E zj2L~kaQ47j6_x1ih$O{8-_S2x*J2PbIAQiVZ$^)nC&_(Z*F=W<9)aa+VxM|5``dGQ z`2H}WfW2|ccGC4aXej}Np)`m59@Dl({Hcga(aJ1hhC|5gzu5~fS$*6Z_cBz zGz<+eJvMNrrG={g(_t`{*bjYI#T(a+^yrEDXs9wlafmghe+mTdI9rQ(rNYgdNuIQ_hM6s;@n(SgV=CoyM>lzyO3Rs2%xG1V%SEFJmhI$3*_2& ze=fBwLk(;#ER{qp>QaB-76@|kmSRL6+q^y;ESY;J*XA16801k#*CZ!&(9Sa_x(*T+ z(5jfdmN`aXW>wrDMZ1cjMpHaUCBSwLx1Q&U;Za)|Vn;$xai1ZdGw_Zlc<5G#V|*sn zmRd$F$37l8N0*rU&)ECvf;^8gP0a8aJ{iqJfllA;vmBiJ0Nx+L^ce+Y6UY5Q`WLkIgOxzEGPFXV`F;Z+V z=5R4oWL;^NspSgJo}SCQuXyvISUDga}5U!%Ffx%`wB9=Z61fzG|{-ff1XSk-PKn-3Lc8QH@0Xa*fh z=~m*9pQ-A1GAeWbkG`N*F{{CJLpOVW=nF@WrF{AmSW-rlh>N5WL80GAHdFtSH(~RO&4aRv|*L*yi)WWZ;^)nef)(>R;H+r9V zUJge8&vQH^q6-)nX-qBiLQg)eJuSU_seei~p_Hq?(UFFI01r0TxKm!^6_nYelZXEW z*!QsqgSzW!a0Pf&Jnuf+IyBkj4ugs=fRhQoyYNe>$ODnPrY;QjQSn_t7tG7V_Sj{8Jx+|wakDy&m_D@!u4@we-nk=+Ycl%*=>r)K&FCJ32lEc;=>E1N z&rCA1eNGoF`eb;U=1$D)q{Lf7R#fK^%9rd;F#lNjTV)oRxGcH@rx40SdV%r`lFo5= zL2DNA$9TKpUvG7ol-EeY2SA5nZ+Vzh_;3fuoU*vbVIN0-cVRv%v`LWz$aeLrM5!Uw zcIB(Mi(MF1?!kp9f4Viq!=ZM_ky%*#+_T z9I}aM78yRd*9r7<#P;BniRva5+rrf&size0^qth&IoG`?(HM)nc)XyvpnxrOp;xvkcro9tz+;RnD=~UCEuf|1h1^)2cuJs z_wr{G+7QDrs8|JL22PL>I%(Kc8Td%T)t$*9)qHpg=*$q^LKv1!aakjX+~{6G?2D;7 zKC&7Zr=j6H2C*0nq?53D)iY@w_x736V*BD?5Qz+Eb`2F0cv<_5;LT=k&ga4FjcnZA zj$K^$Ahw0`uccs-rj-32-jnE*`E+MA`hG79nC$4@v_L8glI%gsXwWS#U>YKKu45%N zCtqOK;w9U z`G5qR3@|(OyM#QiebS5Ja_v)&9It7(VjeSdsJ|SoB>-W}6vclz8W-eJOLJ$M?v8AL z<+7eB-3Q7S%iJL3xJtad0D&v>_+=Kp&l7LtKx-KUE~yr(H)nXtA$30#dl}B!63&a? zBwPT5uYPy)jHd=YvD+DRsor851>y=sVUTj-H(L?M=8C+$E#?_AZ>4Bv_e060km@0v5oE zq&oNd7%^9#Ipe^BPu*os+@?Ynib=KfRV5HuqaJ4wai!p*@w)j?d7(OiF24Y$lyAbB zEM$kh&FGd`#q@fVf}u~CpmY&`+?%B706!DFC{`3{$&glbLLWa#jSTZz0yG8T~mFp0K^SHfKM=+FeM?pgQxC*`5QOIGX zC{;vlDvi7bif%?D)as{-@-2$@ z{(BG0zB(VhhPiRSF-4N{WmqxXJ>r40S&46-{OV;psMwq6Zyu-bfIOe+01qulH2%D- z?hFFEIP>Z(Y+Q+rZq4?V&Lo@~vPVW~g}&dN??Mm$QvjnCY{=?Fop)pb?cfDQ$a{zZ zXo>--1ll-Piotv6Bh~IKbJqmCyGdr8UB=z-2lTb1<|DcUwwr;Zi+=DlU9$xno+z#^B+Fu)CzoU(03093}Gp$O_~udGbN* z3T*E@&wgJQz^|$gg0jdsWj$vV{;0B_v}(E`PfGw-`M^767fe^xz^ju|E5Iz^q(&86qJ8mw@yC5kV}=#lRWvM-rv#GDQd z7M0d8Bv#cXg~OqH%PtNH83eO3t>O0zHb%{Dx{*o=!-xzTs^x6M;0&Ubit+^6!k0>~;n)k?`^FBjFR^JN#YRf8>HLxlKFjPZQkQ11?p^D z%j89^YFDr_Ga42>KvH`=mc~j&lnz0wjx2vOGNDH$TI-mB!?_{BbL%zubZ1Z_?MIrMX+8<=T=kS42(1X)E3*4;`;&Uvl5z^aH zW^<0(NX9Q%?)m1d2x$1&Ld2~KoOzVXIhU=9APm^dbI*{$_5vi+dGUKJl6G+Rd04U` zjjKaePZTMKp2g_LOr0Z%Mz)gtTm6oC0-{VTT;UV@3k)y83JK7+$s(XH6GgxuMOnYd zMG_Vl*@8@SnA)FFuIegSMxaq{+A~v8kSO#lQ|HLwU)(c_N;2VceyaMFrv$>M&6F8N zPoNcbu2Bo{DKaMrZ8#s8)&&mW-)x*S+v=G1^22$-59Ols#_Uh@r_O+SojKXStA7`5 z`^X;q3H+Y*tCMvgi|;tL38z#Ckkvi1t%(Oi`XD+ImEy5R2XCNAiQN6T z%Q8ksgS2x8{M_c3{$45%@W?T|lB;W#r?zBo{7Db2Dxtbo;!K;B&|3sFO5VRM%?Ghz z3^8N@gftf>PoN?goMmVaB>f%BVg%zUX<)Wz9hC-?GKBIuSmD>fZVVuZw4niOL)2sH z$9rqg*o)@9OGbf;2HU~yN!x{P51)4qYFi&l>$`^Zc7gqAhT2FOU0`=3xnp%A*R^#0 zK0^77NAnex1kyXDl0lKee=y?x;PCAd!(W{}kMCm_jF&c8vHaQuQyuf}!V@b&okQ#c zDtZYh{h6VfHk%Xofg`nPcbET$fL@yIiu=hF zc~R39_6xjSzP&@Xa_zMemV=&Jf&0)jkizYk+{5j*UY6A6I{jFFW!Xq z=nzGHn16!63m~7`N5yX%cjIKjshlECmcR+E&sV7w-g4Y7X3jGE=oC#mhV^%d2f3c3 zg@)hXE_t>7B!g@Gi{Pv7#0yOAbX|0#lihBpwbEWq-*OgOGn1#Q85x;JMW+d5LqUTE z^WRmbvBcNaf(h;Bg@Z;as%|MI(vV-%F_11Uk#|U@G8l-nClm>ew zu3GSIXi!Z)L=lB=cjNa&lvV|Ld|)Ut^c)_*H(v^RQK+atCRVVkIm$-9 zg8X2Xg1Xma#1|1aOGMQYD(E%!QzXo^cP|4g&cC|sfN{QFpT3EH#ZkeARL$k83|fZl zYO~0y$#zTKwE&aO(JE{OB^0;XShiQ)&}-#gAkqr;z0mH}5sn$f@M>^*1hgf3pJ4A? zn+?CDr-{gTLFknj>VZmIX_u{o=jRpCnIT@fPxigao5P(US}Qnz`9OES+$<%vCQ$Um z(KaD*s=8%MvT>(&w)a(=0X!;|2kWOPXx5Y%-0(4Ugq9I~4n# zc+`*(-$E=K+bi}L>|oe(&@7e~=)X{g@|YRTuhs)Cy1HIVP)ww3^ zs?m{Zcb0s3C4DSwR3l-KhD6@N@N-0(P+Hk;S-U#%11yG&%0OqXd*_FF^0@%8E}Ti1 zaP}2^kTXl!dxkztFw1`a`yVVA)pDU)PnZn4xu8nUYH202A($UHX_cM*gC9KV|4ch* z+sG}KkNdFO(5{y|ckw*{Jxe}&;M?fFE8h(HpGNUL2%DFH?j?9(rxyZiO7J6qo)Ef~ zp-d<3Ft>B5ht58rJEgTl!C9w_KCpC)gZ9~YF|}t__cpS3aPAvpvcms%ozyb4kCi9;YhscB`m2#HU|4yip#Q%g!i#GV!ERi)u3o*nAN zW?|2sC2JMbL%UDBhOIfYz6-AryU#d|&ps&2PL$=hr5_&#mNlnJ=-q#3=3^D*y5l}B zL|FdN7gIZv!Y;CRi)F|F{T{+j80qfpeuiACcUB2bIJ$ucDe+Hj9{SZ3E{*Hev z+&TQJ;CJ<1&21O-JK*xHt$Z%ERsP+sF736)zNp<+{lI<&`L5oS@H=&#|F!Qt_v`R^ z_I0cK>F?J3J&Rf4Z*RTy>rDPYcPsu`-bNsF(;9N%k8Hc?2|K-s6@N_)d&v?XiWFt) zeu*!0lO);Xrbc@86m{&1B!aXvCt2;JOL#gFe&~LW=h|ry_itrVJgbggdS(;5@Te8R za<>z~^1Kzn^6(;xwjns9;(#>`dSio=+S87tJgfuN)MDiWpM@3rdZ;%$ODPW&Vc@(I zHS}uR>PZcSEJNr%rx^?0)lbJ5ozTGnrD^W0beanmadmS68ZoL@`8I3=lBE5K3GWm% zlFUiN#8P?~E}BbAC_rwj?8%#@EB$IDY8utBy+nf77*#vbbIrErp^N?;vGcNPtFpj@i~~jX zIYr}88V%-kWK+L}BI-r--&f0@Q!Oj$@Nn%@UhF+o^y{o=mRvN3*$!@aa6UE05ljnM zw)O+^kMR%FM3my#YCE>rj72ujq{kB_^opWj>1HlxP?-XetW`JDNZ+QkTr`f#q41_4 zDLQAiFt(c9feU)GW)BX@9;alvKy0RwOR$ z;`|?%@J-?QX2}At*qRx6qSY53O@~&F5YT(Wbs-|@TlGi|%v@tn@l)~Aa=M()b>%+* zorqK0(sjRpUBihluhBL>z>5Zm$6z8_iuoI`z0DvA68dIW_t2^L-52qz6b86brc$fC zd!qCnpHXsRJr7UcP=)O|^?LW)%{la59b#(_E?WuixO;qOZ&f&Z(#|YOku=?9xZh_? z*Unwej(+j~)1%iWMmEL|3;$doe^uH!4p2-0KL66woy%g|B+4dG@tmX2$>re>luo_vf!_&n<}`-S zL!MK_Fg)&(@_v3^uag{azS8Rf>SN-O5W-TURHCp`*eY%nw8=Px8b%vt8wMWGi7>*w z9@ys!72TxMFf2<4HU2zT4e1U%X6YrGeuZu zq6!eOlUN*1R6P_<6Z zcBxTI7TR2Ty5PEI*g;4cu5^?-g~EXO?zt*6#j-Q6bTm;uriETL5v0aA5{j}h045yn zbo3iw*%mVHgn=?JvB?iKSk8QsfGH0xiI|fCd(1aMFT|TfW(>B;A$;Pnx93w%3C4x- zMe+6PpdL7JXQuX2P44gR)!A@;9TzGzsW`|Nh2N4NYitmwR@&F3X6%eGM z%}fpnY?~~Ww+;~{$%$u!QOP`}DY)W@y}}{wUL_IR44wT3jWCOTf*S^6VAW=>GeEY* zgpc;hCdxeEDt@)(aTNwUPvDAO&+Rv)8*GK=x-p2iFCk_xP2k$^uO9b_qINB1yHowx z4w;*I*T#aK>dQ10gdvI>1`a!F?(PdXNVB818uti8fA=%T?@xl+fC!ExAcoqJM}}Kz zTYM}25mR(4?g^Knz>jD#8Tz`2>YTn7>*DVV#c=!Fh_E6D_9+2|YgeD^_PYeXzU7O? ztA(4Jh0$$MLyPT_$9?7nzkMh05n)?>EA`%!^#7OB!t?+a1qGd?= z^1M%C-+b?c8!_AFTiMT>rvF~NY%}J+qxLgQEZTp7|1+{_F9TvO|02r^2>{^ozmBYd zy`qV+i;;=6fs3t?xrrl*q@AIVovpK@owc&Lqltm>|3q2bz}DE>#PPop-K}clv?+|> zE9+3b2650Pvw&s}+Sc#7I3WQXT-4QME2MYDGDMhUCWr6En~5US>)e_lue z{?SvN!C`vH5Q}mu`Vjg1NBOeW>YN!lx0KPJ6nT=&Htj8e5R>)WwPvVJHNIm14$Uod zz~UXYbY?qO_>5ELMq=G@2zh@bo?Zx%?Mt46yQ`V5YYi3kN138@BX@*Qpv+{TXFt?e zSitgKArPGn6UpQpG9(znJwz}Wc*hp{;mu=c%ZZ#ZZ$c)4G8TLK%se_jt=9Tc=LUqw znjz2oMv+3rhTWO=gREKn-5_uebJ^%3IJl5vG8;HQADp*yA45@reu_Yl zpG&BN%;-1*RS#R{m;ASKV{!mfS|e>_mCRPNEw~|gv(3l=bXUDC(hKe85ODCEkOb2- z=G2+`1z7HO`??_tcoxpYn0h{?zoxDnNpW2vQLM#zTOD%NcMByS*ZD23QHEP|C8MyL z_EM@P7^(tI^?q&utH*?6H?|coG`^QOX6y`S1@j1-j~FJMuQc@8W?z)d5+?Z2E9LX% zO@d31M7R*g&-uCoFYRNt)ri&7$7j}^DPpo)(Q(xUDSGcoKPG7*p zesqr*TqNf^H zhfuQmy$bijF1$`SRq+t^QlqIJWH`YH@jCTx`j>oUxq*BpaO>+`&OnKs7uwQ&9~0&_ z?s=l!9CI}0LJp}#{E22Fo$e2ARU7=F#^?!7v_yly!mvjIEiG}hn}h!j3_9c|!oD4e zeZ&qwnjbNM=LsBx>)#<&yP90!tPPTp9B9Lb6!Yy<>lqmEpBEzCJzz=!4wJg0W!k4Adx_rF+36kAQBJ| zbR1LN1po0V2I$Pr1I^|eC{>&K4Ap8EnY?=2>b8|?RjJDA>gpCcm-7ObL;ln5)NzvZ zpgVFD{+%ti-G4gV?;NKcpSP`6elHY&k_oMaJ%paKyh*7o&vpN>TKh|to-Zz3y}@0I zw~hci4gy%c{n6g$fFJN*lhHu9FWjG<7(1>z+OI(1-!MDoDt?F8T--OfpJ9K$VK>}2 zydRnVUZ>elxUo@ql&?9!;J0|-Uw?e+w{T!lrYE_3=1rr21Lvq;rNAQ(RKG2Ue(wIW z9{%+1JANK#spoH~zSF683-?+)!svhze?%Zc;|2!MD}hTOGKo+!2(DOEizo;7|6;yH z=oBs1%6kkUfa6dzy|FFQHH0}cE)ixCaqGY!b4HB0^Z~_-C53y396E*pDU3n;jVOg| z4nu_SC?Xc()WPhwMiPZ$(2lGPt9N3EiAUahbB53(UlpX~%N}8pMx-76Z8l`mlUx)| zr#M|f>5)DbDRHpsAc|2La)9H=RtK3B4%V64bBS$@On2RP4bvb>w8)+&kkh9*r7*0~ znQDxDk9q?NX^Ja}6oENpwv9IagUQpfK!@h^%I@k=phH7=6G4>_p*~ST|R3x_3_;iP|x-LSg=QXNL<$%J#}rk*wrDOsORvn*zmFU`b=shibF7P z4vN}Cue?89_3oV)*NPapd#=N_r=1JbVc?O(O`6O|B$ zKq6b`zDw_4F8;1@K=Ahyg54|EzD(~&ve>P{uuR-Lw%y)o_}!}|xlzXR8cV(WEuu;T_ln^#L7*L~ld3_fp z;m0?sR22-bzJ3;Ck@E#w#X%qB;>vUX0C zsFcM-QNOzSF0LgGXhJU~AZH*7LD^@rSyefYKA}UTZ5B`AhMDY zVWK7&nTdb%@1jkja?)Ct;UMMkfUCqDSUf*}cnWA=gAkdb!|&7h-f zp*QGb(cdwEuQV+;()a2NO*CJLyf-G!&r zvV5v8Y**!2CL^(GW;ZKs;t?-&z=j12`ct%u&(%}FS094K=b6Wc5%N_Ld9aCvgNU(6 zR4PcRgCOcP6Q!%DF3h7A^)&-0nnaVI>{gJHbP!cZEh5F#VIC5RQX>hL;gLi$?nD1_ z1Q|$-sGu$q<)~8>K(nDl#3z`N^47eF8F6XmLxg)nZ~~vqnbI=to*LOLQWjGs2$CSd zkFw}7wUWpPoDpFoI}NZDyic6BuinugA3d6j;ynwWiWMHgu;?Y%Qnh7Bm$J$7L9x&4 z@ds3fI?{xF0`q0}!Q^zW;9Efr8>#e|qi2mXNW6pyS^dFSxMwb6gvIAX=IS5q^4 zntyx>Tfy0=cu!evkX3hgId~w`G?(=-{&cvB&{%3KZ@|y=d z^fm2J=!Bus=jyNzitVJXp{{?32!YZbS{T;Q@qhH(Gwj9_tqN+p5C@|V@{Knr$`}an zrnD9gMyhd&>-n}-t10Q&V8fig2Tf#Ih8^oBk79u6B6-pl#aFxa%&z(z#OV})9o$h| zx-g;z=E8b;cOiHK9`{s^Q^$B~fd{Yo1hCd;=);uUa^M9=F3q){&H^PvOb#WG~ z$zM;%1G_x#jOqv5E|bJxjTbyc)gd1-T3#~x9ZslW-5bHKFk$~j3f3E^wS>qQtqYmf zS}zk}3n?@>4O?-@29f`qX5Anj!np&Z?EJH$A}RJoFX3tks4idamNV1wc5pgG)_Lkf zFIgN-d*_VBm~6z*MY7mQXd3?oe1C8!IMG90CDKW$^X%H;SXbNb`#Wn!aSy;fVSI+R zLK;*7XsDanl&YmZhEnxls)b4d-&jY9x1}uxjkPY4HaIo+ce-|FkWqk%*3#YS@QZ+x ziVVr%TEFaK0+re#rBp!*z!VDDmn^TYBT#VgFfLL;5zMAeovS9+go9%9YBQqFRKGG_ z(}|#d4F6e?4spGH1phg+iEYVp8klVk3rh%-r@cv&S8IOwIM&sY2OZ!k7!1SUQP2}% zu02LDj^kaB!-9dOfay$Yo`D~Mfuvv?{e@tja?Z9E zj{<1ct^%s?+TAa*wB5Kut0qw+4>bn7d<^nwwDvDy^6LmOB9`rrsY@(er|UT&31$d6 z5@smF*U=4>;AVqF>P=!3L3D#xG-$Y-k=N+vPz1HY0{%hLpiv3NdB_a1fl6e@@i?z| zt>(}}_)uWffMQKU)UA)VjUk2%*R45IFFqqj_Zf^3;d{6&Hh!2x=(BL28A&B+g!hkQ~%FyKBV_ zl3Y`Gio62}+@YrrNe$(Jg$fUGPO(EcI+ls4!XL!h@&k|i03zSw2=rkos+?H#vICR* z2qImT;o9NG2zMFOUc_4J16Y+WRUBaqCwC*o;oG6xaEN=d>)}A{GlM!i$^-a{w?cPp z19is`ysfHt4mq`BN6X{2)SXiinr1?^nrzb>K^$w*0o`1k+00>2mXpTZSVYu z--Zd3Qupn=2>M~QiZ{)!QJ4EA5xXG8?{B&Ic2pG5=jRBddl7}PD2a%%cJy|Y`{*}S z)KHQ#q<12cxfvnnWKh9q9E>t^fl8ucD%u2Jn+oM3-2FA(wz+I?x)@i*zv~HAj@&o$ z#X{c2W6sHYaD*hrV1kfF8kPkTYp7!6Xf}^;M$%v=J0kOMBc^^VlJ*l}dQ8X$g8XiD zK?Gcr!@3Is>Xz)+q>ut%nvH(34^Gdg2}|k6e?)r^A$KkvQjam z$H|9B?m_WBy{hMPsN1g3nn;;7RI%~M2q84{?E`=TvpPfGbR}mdoWn*UKY0q%1PCHI zX`Ny1`*if=yAxB77K}vnnk|%AIZ~~@yO0rCWfMr863L0Xqa}$8h0eo1+vK$MkUFWj!=}* zSrarFBTU5TwY@f!1@3KSB}DUHgUS~+2hfHj!9a(RT$Z~PS>&gcj{e%><<|<5sLJi)suhm) zJPOHD#>rHv6DAc^DU+xgg%hZb#Svcjt|~P|bbGOPuD&)l!q2cMmp2p(g|_(jbduq? z5}P_KqVdfYRYt(0j8s1;w@OErDmC#|s2&Mas}d9NS7cH15U4KsrrGBThLr7bz7I(c z^n*;|iE=5WclrK35!n(obmjVEhI}gVrR9!+JlQWlvMY-t8sn;ck*_d7QyryLyjO)Z zSt5immKO1hn`M&p(1W9{ygt_Fh`$QOWaRhCWcg$2)nQilBB9^RbK-es28ooG@+QQ{ zLKO}N1j&*Z)`?W#IR4r6?B$Nk*h+u3rV~g|5|R#J4hNLea~EHk=u=AyL@g{U@Qh6# z*@mD~K!tQ0tHR`&+m0nhoF@My7ij@T86cH-NWA?s&8B2F<3cnC>H!|Jj=x>RO6J4% zfAW{*5mxbGB{%w4NXG_X3W`$TP+OfutfLhgveo4=h%{pwyyv0qBCOzK2VjewPr=BU z$zU5S=cSG!#+>WTM-9Xb!c=Jvl^`h@YbR?Rvq{b#xvJd6aU+(HfTcK`StWJ8?PdKx z*3Kz96JTr8v2EM7(XnkiU+kn~+a24s-LY-kwmH3K{<%1FJ7?8>T~yVs+O^lapEsA9 z-uhXTLju*S3i($S@r>%7yp52GtuOoWR3EBD~}kO+T<1Ldr!4(BKsnr zPJvNya0FRWoFj;J^{H@q5ixV{no(JuC>|6oQuV(PFOvc3(sB4&e>&CLCAOm#xruu5 zZ#&L~_N%e10o?X^QhO(Z`j8pE#Fm%6XerHc6BXWIMX1N|49QGjYonA9u~NPn|d(gTaR+-dR|O$}_n^gMO(ZS?R?dEBYo# z!|;%H!s#ABzxssUOsrWou_vgpH0lkN7Y{tmX!)4P&+_3-FT`%Tu6YKEeRE=*hdyO( zFQ4B8kU=^cTaL`NIS%<`tzPiqadyWEJB?6Lw9q{DEgq7Bef1|8B|qLIm5x!KN5}h9 zKRn`id*1v6NzWv_80Plj9KO6ymIZ*LZVBDv3iB(b4xv^eokR3sb%_i1Za%Eq|L`SE zIY?|}DTcL@Q@UIMF<Aii+O=x0<&!0L9HN;N^qQ-bwx<#hyS$X`OvQP8%sX|7uSm|R)7I+MW-T%ea8y+U~0Hw9<*e@eKyu$6J!LpQ-g(DJN1APUi*^{ zZpmM7Tdf4aF`aPiI=3%6mdM4I^_Yi^_bGq53YN=`NPr$+JTQkr z8$V*67KcGkchWA7b`zsemmE}>6m@!4CuNAPmSUF_@IL1c>N1}6Yo9pIG=KS;?9?|u zQbwk?Neap)^n0uH)aaX-5=A8a#4fqbUaPdSZ8Siuq+cklTlk|ck>n7 zH+@%;a;Yw|%FaPEZ!g4Mwa=BlhRcQ?)Z5+VSbBd=CO3{H(w*LcPkfT@8jlAdRhiQL zJxuCTuFBPw=WNuh;Y?7umNFj^`3@~ll{jpv(4fmQnA%}_64=;_tFhyd_9zkyGb#zd z+QBwhoxEUwWKMwBdy?H?D~A&sPRECcJ=42R8syA^^+c?EhxSsRX&wCv6p{~q&MSH_ zslUE9+AVvc)SRE7G!KnnifDcQE-q~30d-5zOD+8Q$0!ZyA@@V$(2sNVO-;-=$0C5p6ZAeJB>v>NgpApS;HND}>4;DsyL>15c#x$ukjPle{xUe7MLOmidox5&I4$~` z#IF^wCMU!G_(%6?WpVjsWEsM9yGjt6mmU@S(d<2+tvTo~6==15^ILIQ0sb`|%g_*< zogrg(gyFyh{iS=Vus{}Sx-sM&^Fex8REdR=mt-tTeEHw5DjUlAhKVK^|F`0Eg{4Qh(*6CbMpKV0m?GBeSoi( zSj+cX(1j-h^e`af(Cuk3O!GPC!F+GT-1)z(XWu?p(pNTk;wyOP<@o~o08J@eP3c>s zU)`f9Z%mpuE5yh1?3h?VR&~A7l0j5|}@U8k+tg6qn#&ICE6EwajYavBqc}(%&bESBI&f4U9h5a&TPW!FT$%FuZ?&F4zfp@`oRF=v}hWK zry{>65V775)zeO=UCadF3Elk1U61a=6TX^|Pf`qLeUS{5(J)x5y3N`w>W>;JYvJT{o z#-YY0M6Ft}QsvYsz15W>z=M9*rE@l0f{IJyCd?DKhWI1kInzzEb2@nqg7R!3w7H)r z^#WLP2ZoaMyOim;^g1Bbnf_z938U`5pmd>tV84O7RM@P$q86=wXxl(|sFB<_s+}7xormm5KaD$uOAbpxk9* zkEAvhJ(ludJGSxHG%myKUMmhZ+>3TqvWF*VSXO}8E$IX*CUB-Wof04zJrPa3 zEv%jxEiH?>`F4J(Cv61&ed!gU4E0ls5d05A%Jj>4b*{n!B7lW_-x<1y$7nj(W}o7; zli4lK`|G5S1BW1A>CuG^iadtYx$bhygUf9L$x88gh7av3olBxGQ1^i}r`I&;)FG}q z<-BuZ28CVvVwAe7_-$ph?;^klnGUxM&a9i98Rti`H9ip zaF}ba>>cO&nTef{4Vn-u*d(D5vbw;irHy)iZhzlwpmsJ5HA%+4hqE@f_#T1a{iG;)nET* zKZ4Q*n%Wmvhu|9x@4y(kClEdG)rLwiC}T*d5C7rF`3Ywq;Py$(C;Ty1=#I%3@G_A2 zO7n^G12wZda*O8+JOdkS5HWVkqx98~22&ehvd3wR@Tsoae={ik`d1UyJ_7!><+hKP z1jr9vXIHNt{hMMs;AN=#HlPhv0Oh4ma8Ty9-W^^5d8fC>6vCVQVGmaeWM|ycK~h~2 zjt8~ciD3OuS@n>nX#xU|gwv!1&LkyzPtk+;c)XHHK|k)z39;kwN|TCkaPM9UkJzB^ z&%G@k`H#5IPjCG_l6+HiKgxt?mZ?@maf*s#h1kGZ3K0Py;s9znBWtgJ5YJ+&;{ zif?GSvxdpc4Y`120#f{{yWlyLY@4 zAA>3-2qPJHJtW12eZObxjw4ea&GyXgjyvH9bC_@#H&hNbBzB)0yilh0Yo1W}7VUgP zx_;d~OWvndn3W)Eq|d<-TDM4qnry7q2waGo=&j=2Due}&o2LYv1wsT~_9)Q-ZaS}> z^f0>)EUvV4;D}&$(y?`bn$3;^OmpJe?6)&!}19-@Dp^boUL?sm!fr7T8 zjIz36Xf=+83BO0K^ah@8kTv&btK2}M13p(T05{|U`rnIAk4mMq{ zb*pM+)+@Gbf6KP7h@orsg~F+5O{Hu3#hD}T?AcJqf`h+Ipc=0S{HD9Y(k2RUZMuCXGH>y{l2RpIfd9>!uAQxU{4~C%~zNb(J2`4 zaSV3-4R-B2%Cv$Qf0_9NHV`TV*H>?fZL}Y`i>Ng9NG7`3G|L#m(P@dg-q?k2r_b3A z->tzK;g+F8rQ6fY`r>%6*ad(2>Z2S*7|(sV)R>F{DoTP#Lv|FAaC&K~%Is7ZsCm*` z-4=s^`k9>F{z`z78RL5*o5lra_pt5Gsw#^GA-L*o7Q#M~Atp2%JiWy{GQM|zr8i)9 z&DYABNMmT>&ROk>$aP0)y8Mzbw*4aTDuQ#2@H&haEJ-$_s#J?1>Oo%G$!%cN%_3iQd>KM;zbT6OgZV`3cXRKf z^2t`UgF^WO<$LnRmBS5BY79$rt{Yr;9ORC{xwn6gn=)G%i*Ro)s#qL~SX|hemb($2s6$Ntz%e<5p%l;uW5c<1!bvMI3%3Sd~ zwzOp+=ps!IaVcJ!U)E1&WxhD(``X!Awl!B61HmA?v--Ai49~xnmoWcx&UVS(SAMQ- z<9JbJdY_*uAD=+)q@k_K_BQR$p1^b}WydeeFiJ}}`gd59QASru1=oPx*Z0b`i-sp{*YV^m z<~w|8H!2lrt)lbdR@dysdmZPd2v^>>Eic~UjUQn1ryGBJ@n)xBFvI-25A0zCENwDN z7JqW%y-u*bQ8(%SXU*)4ZCb|&g!yBoh^BV#`URKe@5uUnPKc!8kKTfti_?sZEf+@Qkx|FQG`c zC3SsX!U2e4INr3XNhGMOq8R2;x_H$04pmsOX2xLhn1yx9(Zj{YaLZBkZTitGYIofC zsii3Dowvfuy}Z4Vn0rHq@+@Vp(E8FJ7W~Z&12^$2kPv;)Fi^ zt}R9HNQ3GQFYFbBo2pFFH80-J5ZzUSjoqz``LT~({#LVwiFp{m4M@KmHrK88NS09y zmB%#WsT@zy- zi{ZbORK}y4`bX@dqN+bp$0yuH%O4gvUMFqMIt#Y>nFcMs8qvR(#$uQ+9YQi$BDLZv z%?B(TL6XATlV+!bJd6eC(35!zoS{NsutEh(xVEJKth;n&YNLp&csnE70$(~67irFx_eQtl!u$c6pDKb1co68)*0vlVg@rK|HpU0w#g^`Gb<{1)-Zoig*nF;5Wy z%Dr&yFBOuq1_tbZJH8g6fAfm`TkITcnBONfAGZb77wmx!L7}cv!bPhlHbRb|*-d#Qmm@$%-nwUoib5X7- zQBAqdOEAP1Mpyjf9`uzcT?a2kx#<=ueHX%_H>^H6=iTDRN<=+CoGV5+ojLe|stxsJ z&llw-s^}1piuT`;z6GF-HewLSLcJLZk3v4}Dx6lk!Hkf_e6kA&f0Tz1bw|hTn;hEV zpgyS+pcDCGshy29UcQ1|HHV%}vdBqLr#F2Y#c#p(FG-mf?lQ`%qJ5I5)Gc6kAF9I|T>F|WbFTuwHoSsws>8A8lH*sb3It2XRtZWEp)KElhW$WB^J0d;&wB8fqh*iIzp{)I3T|kBCV$v+qk(l4y+h z0KhNlZn_R9d{DtHGm8~)ygl9i+wEjzhH(ctnn)iKTmz&nQVE?>O&2x~e+7JVb{08r zD3fGACDg7@37)v`0LK$-XL**^U05V{mzRbDJIRy=8vs?3vj1gLar8f=TZ|fqbKY7h-=*IofVpW|9%{uC)u`^JT(&$cB+B$7&jx5BJA(R!c?)L^u z+D*>67sk%T$C|Tk*p$U&sv`o;j@9K>Fy#4oBTbImH9sa`o~$6nSNtTJ_PjGG`_uSjs~9R!=tda% zulkzc5X zAOz__ZQj^u8F~{Ic6A_!38Kx5Lgo7y%`FPNyV%HWMul&XQdcqOH-3U@!VIm@h}jq^ z7jcIC8ccJOGs`3Ff5zAx&c>$>5)e=TEfA36e>KMbC#*D~-F23d1vFCVw0z~GjVT?~ zOC1C0HXQwdh;i>yR{L;M8X*$^XG6ZCDZx2OtJE??v%*wK$|)t;XTsFjEZNxoh9PWd zX_hiN<4sK=DJ`ser!LF#>C^Mm((k!1Ic9Hf<5B`;Ju@eEH#rZvN88Rd`W`2+j7W24 z`zF4T^X({+P{t%_ZE0xCj;wb~)C|OO?(HWg%Kd{gMM)9;MU&3c-v@JW6s+kG3nQFz4J2olR?p#+h(;r#a#!Yc+?EnoyWAe4D zLziw|Lj<*LeYAFHMURgyPgh%u%ZokDweSP6?z%F~gQ^3wa6(6+Df3IK9Q0O#GuX7mwlf7B1oY^~@DJx1SA1G@&S8aPXa!JphHl!Icdlw;s5W2AiQ9Dl{&&F;?) z6SHSXlHoQZ(nfu&To6k;`gugkGp&s|nt7LDMZLye0O-uIQ011PUHK9HzEs>);)d#$|IQI;25 z-EV>7st&W$TQ!}FYsX2<)2F~I2b!D1(N~F~+?8eEgpm7R9mC-@NyXKVXLFWAE{UgE z)iGzupWO-7(J<$UDA0^==ye(u`!i1y@zw8duXjKPFfHr@t=13Mr5s-w`7qg8ce&i3 zca-mQ7>2F*GcKXEUAg-H*cqGgNw4Z!@%&*io0^9Q3iC;*;BH@a z5`}K`m>fkqWKCe^&pxK^as0(wB-Tpi0(O}dA!L_(R2dl!9OOtQLAg7??ba3c59To` zYc#U&>Zz+>kEDuASGCZlhlFlz-P%`3ub99OW4D#Q2TZ?Ii=e5L;E&x}&UJ^ik$C+W zk06zOVIA`jlr$Rr*89v&V4nPAGe8{B}@DnVF|2sT#zY>R* zIr!&R`7)L>l&M{tC=-XtgX7p;vG|gN=%-5S#_E`GW%WGagO9TdV_g``0l4X=C=#ou zW4gOk)gYMOOz7;VCnT^!AFh);UU_s+wab(6 zupT&8?HOipv&`++$cCZJ4*Zv>akIL536DYPR$}BO#=F3BxlSS#?T89D2zAa>v45s< z`qS~fZC**}@zqrzok#~CIh^^c_P?Shk228rS>lQ+wvx0up| z8X6um#wrudk3sZX`OrznM5{!+zCWPU4z*STNsh&z!?CmZ!iP|&n5rxePPuLu@tNE~hL(=@yo*?CJU^~K zQeteKgPBRj8n;>+r_AqZ!Vtyi<4oM6he(OY6os+~K~TwdN~d44c1P_SX?Q00Kt7uJ zkZ0k-gv5IJVeZ$3vIIu=RC~8>J*}<+LEizw1Ht+UQh^Se4C$lJR~5{`ud1gE9Xk15 zr*^eiilU;T=pAtaMNl#FG%kYpLM|ARr=!BYVwadfkyt1lBz6S`v{aJDkyyunOHavc zGFbufJs@DpqQbjX;hvu>B>k1R73hwvJ}|_M!!va~#1?x^?YhX+<23z9-t0Vm)Fh+` zfiYacpuRG$uFb-^B{REoz+_7{>E|tbrk<+Eq;+vv=wW-#utwVzJG9%i;X0c6IzzVl zj-4d2b)&q4*^yi{*UE!i`=C{`km!za;k5hujC&5g53nqbL21GCP&cubVGJQ_dch=w@2r z%F8LfBSySY?y`x>PV@w#D0%{?8qk$C%gZ6(T3dR_SX~Qhz%sBC0d*oV;+b%l0rL}# zyG!e1LX#uT;oz1oqI)t8kB{kdfjjipq0BidP`A2QfNrA5I zZ8W=``Gq+-87g5V5^dQBi*Oed7gTWu9)E%a-XGo3@-3T0H!s%WNkSB*`ZU{NJH+)X z;!gbf4W&+go7OVJ$!B#h@60U{sWkfg;57nvoj!E1Q%G4|LeZO?uDic^Ba>eeZ8c0M z*R5}PG%OpCSIkzV&R@h2UYHt^<{x$K*Zam}HiYaO?C}BpaF1NSYzj!Kkh0;HvMG>h za7Zx|Nhs_n2q=bRyJWrd0cEDqpPP(k1x(bi?ZX!0lF@UMir~p{O8x) z)ZBAUG0tgSRVPCopG<0vBc|R_`J-yPfqPa3?s+5H=CaU^^iv zn`wCprLfYqS-BnW`g^H!24gA8EMQE+L{(mEb&rkYo4cZr)rEVw@{CSTVkR@36KH}? zTRcvL@VmT=Gwg4wKok_R`3>}=?(PZ#E+6#gQQaBzYH!R_XbRYcG{_|mg=_MCUcWXifbHum~8`S>1z*-3tH z{+-R0$=sG(#TJkJ7~Z&uGa$n;W!l%|Wk$Fr$5!8jy;en_M_Q&s4Qkn1qPA`!8a!YB zZ_Pz;uHFrB4L7XFaS}SFAgDw5Ody%(IV2RL{ii0>XL0xj5OArbJAyaU?^D z-UqRN9=Q0-TaPVQ6o}nls0(sNlXcFz@$Xn))Q4z7&!CThN4Cww{jeZ>e}n$j7!G6W zHE32k&-C{;ZjnqdW4FYm9lEm0Z@A`%ZX%vhBgVP+>yFS+CwG@9FN2gClg8!b0fg)g zg)FZ^gzhR=iN?j|zjg_h7((cQfdcQ;`*(owzSgoJUxd}3-m`Zvl`SO0>XP)I>^Vw7_ z%sh5ExTz6UZ461p*AY4+_S4@Gv2V89F^XBdYD16USpoqI631t6B;N+3dzMXi?uLG; zsPjLELidM6f1y&t$Kr(_92Izp{|51&!2H@z8V>vc>7sjL%UoeeT=_g`bDAPuaSn}_ z+)~)(8NoXFe5ZSo3x{zqgvRs`MOK66TngXk`!nH0lDbARQhl5H`lBrGzKEgE%HVC( zeXAf(+X>e2#io%oudI|p*-1g!X&z~V*+XwjdjH1+l#xfqNY~(tm)M&6JJlhxxVqGG z!*X2!d8Nc!)Fa0l<4E_RVX;;Wp(hfi+@Oc%#rn@ctuX^y4bARK^wmnOQ~(-p09`!D zo2T8w((^Hk&GsFO#}?%%zgXruAB7n=MeQzyiN~qv7y}yvjuS zl<`Z-L_5i#cRT@Apz%-O{#|yU9wIPOhvXK!bXL9cWQztZ;h^cM7O8Y!Yt_d10M|ZbJ?w@r?DS9DwkJ9y@ zl96VqU^0SrHp#|s7s8runEeeRkw^ALnr#_d9Z1Ojw^tV z9Q?ioFMI_0&=QDS4QP}YceH0eoe6yujxrul{OeoJv%{Rn=Pt|Tp=Y8w?Pzh|!{TlT zL_4dl`4zF88^GP{E(LQaSe%J|a?B{h`6FHvvcI@*J-dGa-qbd=*e@3+8KBA&&_x5h zKdd7MQ#22K$PV=8229k`9Pp{Z0AAIg;{W!sYC~DO(svQV$2P>>$Va1Q)}oTk`Hx7y z9ZY%dw=q-p)Ivj^-;pXgHQ!r423N+RkeHA~LGEtx&3M$K7tORgEvy=r8mJmrz6?>I z15{{#0$UEQC^T;w7v8AWWQ-J7I>3v5TJY|t+Ph0jhpFB!D zJ+>zzfgb*{C?<5Gbl$o0Nd4&k{rvICSgsKE2Q9~m0wsyjrMfA0<^NovKlV$ zU9rb1CUa`h>)PxrB7q{p6Z%h5l-X8nf@TuQRAVGOV2fX!rkXV4D!UgO+u8Q-|Cu!W zfz!Qs`|DSuE@k{fKyO6RMd?8aK)aDez`8_tB?=%nlUm+IesExZJ-0*@uf=%EDESD@ z_B!wU_3Iska-x`@7aviVGg0NoDFFL$pY!Sw>*vt9Pz*WpChg%$k*g#BOv$p{*NWv( zO&Qa61c3oIE6OI4YtQ}7kc`klPiV_oC$DBlPAcHI=xTJ&+fDoZ;MgUUw_0k2g|$^~ zNBw4Ks>5)38alfA$zi>TK#H>2SZi8_mT-hOCe{p{Skc>veYgV(L$Q?~@tH zh1uy;y5B*N|8Ny81X-*OL3ZMk``{}7_NtY!v5A!?PQc4&F&Ozfo|7|rIAWMmlXz9BRYkI!k zMgEuM{yhr?BW<0d4J4+W(~(;j-MuhAL}+CuqtD!oPkFeKMbjy+c?n5!fzaUuFJL{q zLUC$bav}W|%nj2?oeS*CZW~%*a^Z*C{-TSDBoEXz_Md;&WaeQEc=uINva^ZZcI#nY z^|S(@cwVrk2o7(MJAia4sTQI^VL+LM8h*YP4gp)zWV!camC?c5`f9>ICQN#b6!haZ zCp0>M&SIE-caIjnI{*&^-UBp_8L$h~0rqUKTP~|HYS?)a(!h?^$|mfuI_ougLz0I$rQ`sGa+&f%70h81M$OoD^PZr;)w zT_YPer9oC7?(5La^E}mBE$Y?TnZ;yP@)-kU&_9cjP;Xn&T!Ij7qC?BEpkriaG3Oph7Ttm`^e>@$^3HFG zwH+{Qxg;u6XQOCODT*uea3$`5k-j&=py*qXv=9pxNk(xGRE)I9e)H%0u$jS$`3a#4 zT}J;MW(e3jWp2JhnToRT2t9^%ov_G0uhQwi-Z3d^>Jrz1H=Zf4i!nz$$R-&qv^eyP zi43bt+_AR(RKDm3;~s``#B-G0=j)k6+hjdF4~;e9l23&y0_%$NrY!5sk+$sr$=_{%h2J(}ye!L-U zp1k{iU{F)6dqDI0qn)u5Z@S&LEbxPCP^pNu9r+7*cKQ=IO*!n4lgk_+0DBaDsDbjt z!4A!Liu|hJ<4HSu&R@-)1q(n*Sbn#=BI%>DloN18RDy;X;>e-zv} z?HfIy$*>%fP3}mvP%sva;n8&B4VlQ;ph?tq^xX)@f~LB!Hd^CnwJV#e|BdbN>N3e& zDG4g*uv+vzOTNOr3n#D6&9~6Q zHUot7%S@ac_+32Rn~~PTPg`z2%CM8-E=;m5b?OsdU~kNwL=yd^ftRCOQ0}gxJu~1R znhB})DRb3dx+IG8YZ>yxQ#_>S>+-F=)bK3}1k4HIO#{(n)z6ZX4l$Ap(|7@aZKSgt zXpK|hHO#2>l3~b=SEhrPEQOXU4XEY?Thd6yg}i2BG44Tgwmirq(YW`s;hf}hBSZ(P zrj{x}a>3>c4tS(vMnE?8v}U2vRr6JQS7P6r76CY-854wc{Nu z6FdW0MyzfeQnjU6<=-I|G|6Y(b%j5Mo(x>46^W->rNxdg7A=*H z>(UyvKY(41hb|3klL}|HNNDSZ$!9)HU#aSm?dlnhSX3~>tF|+5t(o&wi)XUwx5E@g z!HpzbT$Y&g$cN^|&nBpL4)O*vZW)rJl#q%9wc=*><=O84fuJR)-#tqz3iPI72}{8* zx7S`pXHT3D9K7wo$FR(dojH|}E#1ioGkd@lctTV*@CbrS1#Je9aiz=WqZ zw=UN@cWK-!Ur~j_3>+*b!9`&?Eyz%vF{R4G@K^gVO`I;%Yp7C|bA8UD<{)Ekza$aP zHC@4NJRu7VL4|=f+^BA|Ue~e{+zl}AKt^_87DF}--(1qNWUdOcw8AZ(AVy5v`mWhN z4KK5hJg@x}VY*)3m@OY9o~fU%z!;ib*iVxXBzupNFkw~SE7{(VbjbX+TR%vNJI<+N ztA3j1uftuyB*5H)npjpuP<&6C*geb4TFmHuxo>s~fH88E*5|P@-Q?mm7{iIUFz%G2 z-it%Ly{&5504pI#j?UjUF3m2}0!SHXz{%ZUE2HA4?uPN4*GWpiYP-^n4B_a{-I0E3 z8ul|a>yNPL79CK&mWSq^y>ja>)q_N_b(I`oCe%INiehNK$@gr5l)@t$xl$ZdK#9JW zr>WxjmLF)mHpi~q;d;F7jPL#C!8j1c?->Ec_v^*w!I>MBaCe!Ai-n+@yOY3MyraZJ zE7MyYnm^B|R}{3AR1R6zvXdCSH2y%nLHOe{y623~g2LP!-YAqt?C{Jq4!>kYyaGzD zp#MC>^Bvd&6KxfiZ$iCYTd0NLHvk;q=%G&&U^#)(kQJ=Ui^d8`fb4pqIw~FfE~0Y& z_8b0Z_D>MG&M-d2g-7pB>aQ+HcgRaR^ z5yyQYUQ9O-nM_GTOQfaZ^s71hjRvKyY@Q;Kr0=~>Ai9z6wVPiJqluRqgC^3)u< zsln37W42)WC|g*_MlRZCZzD5e%^ukPY9uFto+Tx%WeL-*rUJ~xBdD^68HH$r1Nw01 z+;_Kd*q)cauO?G^p_3&@rpFA3&x?~~-bJL;J_cZ{_@f*;t9;kBNpmkI^Fyh`|h= z5gmM}IV*Q1fRMUx<-_Y*F~N=IVB_$*_RczYH^Tb$E%xwd1GfcU2m1Q;sLsXmJTW6* zgld=iy%Zaa5v4W2d^(vZPA>*lv#B;B?)9Td*V^)y!%+K$Y&mJbJedV_3#lj{_eBYE zU*YJupNPwlx-!wwS18Svf0Jk)-(*}d3bwacAX3yoVW_<}XQwrBW|2Q!4j+PC4kYOZ zH+WGVUCJFTtIma~crw+i(>}})RW3kifsb80{@v5#D#To9Szu$m!^vDw+3Pc7QwO1R z?gG-!3L#l6Kyzk}jZKr)kS!!Np9epg2cEEaN1O-7h`E{nB;1=8&+tQy=}AA{kxvwq zE9?<2>k;d~62(lQ+42Ncvy$9NeLzz!TE{vF7n_QAMMpv3vHx|$TQ^Duu z8ILoxcy^KY>s^hnm~}1|!6_krNF_gSDHsN=DKuU}$Wd4y+!f`Xno6h)iDS%F*}RBi z>}6o{&Iv%`cX`eNd|1Axk9_vLFb&>1K-n^;xfUiZ50_T41;uJSgu+mI$GP3wfVhB5 zE>Bot{g>o)D=in3FVr5_y ziMgb|paLNYFOxH&xySE=Nf60D2k#CkW!r(JP>>vk3N!iErhJkOjnKg?rZ*Mza2VIU zmyqrQ9LY54Jw$#@s0?ZaFszeta)wgkhDI~_FVgqkxDMv*W=o#Q&4EeW3cGe$s-&3c z0>%A#d-_*GjxnX$wuh^V0o%#gCt0nqH&e}39W`(t0H&w|D;^*CY}^sD&=E8UarhuZ z4n09kONt?F2%ik#?pW~)<55@ERRQVfI3xa=Uiiy0&Ryxv378i26((?jI7$cn9!+~g zpdBi{HYqTNMyxm;QKZ($h&DOdK5<8wYoyWxdE{Y=dZ6DC6q}?OjHoO;Bx00NLZNQ#)4TXOYW+PGLMxtck7DEuRZ+FE-4-$eh)`(L8J zwD$6l@`wy3zYH?7K^u)3}uR+988CfqNNG{2rLV(*cDGQ z22T|o1Ye|C>eyATUdBjWUsvy3ZC!e{^~n7PF#Bzu;QuPseRFm5@|DDUlH;`ucoUr9 zzP<0s^@aeF$Lv|TmFFbdJ_^-v?8N=OyJgn}dE?rra3{dI9mm_gco~}W`U^)Oyp!!S zA9voN6*p%f>~$lK&3hF-y-i}|@8<_pt51?l+}*()n@yiO!51|Ci(?oi`{s-YU-908 z-D!;1rs^Bqi)RXeD?SwRj@y@SrdogYZiH_oif>hhhhYECw_i`+ZSF*)?NPb(_W8}raj^ z_8=Tr{6GT7x9Wfr$M-yR?A6-t1zXU!@W9^nQ+H&?aY5qk&Ofy0WCy_h33_t|>-#nj zPkI|?_lf$Q{`(Wt7tcWi-1lg>XW!2;vB&cDH-69YKIi8vq~Le?f!_ns&;CeHyEwvc zRm@)tu!mtpG|(Z0H1uj*WdRQ7!Us6ngbaFZ?Elh?0s46{2o1AylBf@JR5Q_79T#N1 zwPZ7+&?J|Tp<+Zr|1-TSXu8uI6AI=jh?99JVLEE`T z;WZ-qK1UBmkbDmx!ax|%&kMrCsY9&_R^c-}nxFJ~yZ*vHdLV$L3SRTAo_G(R<_@XH=bPW~+0AF=VBGzy$N8sXF5;e13o zxet5G(ENo%HhqAM!7_y5$j&c*Yts0pIXR&>7&nxmSrAbSC9T(nLEQS!v4H-W^R-d~ zEV6Ro>1j`~DXn`ELZo@fY;S#TkqNCR^lc0tH?VoA?>)3*Ka|I>tc^q$m8v=&2MV0z zTa!yz4{z?=K?I1n)TY?4xhD;02}5IugpI<-B^Ej~u-*|W*~4~3_qbLc9cl@!_I4$B z0DdT-IpBO)lL-$d*n&S`-_uXTEU%=VNAcJIGkkQcaOfzR6$j)`x+(#`DMKL7AN8Pk zEpdDroQqU&lE$xz^F!_arSFdRZ^`(bYH9Ws60G86@1Bq(&gG%NMjX&TuB5dgrBtp25)X7l zL*&Wc+xpW*Hg6^ACF@$N<#m>hs+D+71cf~sU)aMcK zICIHv{mR-i>q;2UD`~>AYL*;xZ&S9;ar)7UCO&E+dQ2FwV5duBwMSjnYtvPf?0I)b z1e#gLezo|bAg;dOX+(Ti0C99gfegSNHLooOL6-R#oH!-;S8x|bh%0lnVFf{W9FHO zH=9V=C~sM*rK8giM@KESs;j#G@0V}M3Yd_|>t&QhBu#VX5C`KNR!j3vV4;~%l&>x} zp&xAcH5w7&M1fTygJr=0%c;`SV)7j!#VWLVz;*==OL>A-p!$rbA6;EC6sJNgDIK?HJp7AH{o#C0 z*;?Fmv#8W){ihrStP`TxIbJr#_9M>upr}X%KWc(mo-$1N3@`enw?>!PL@t+fq@Ikw zAt0**MND(O?jl_xW^~%Kbhi&{;ZH%BWZq3xj`h6aUa@?&bY^0);J_(m-mgstFU3=O z@M*DsBEQDr=L`=LG~chX63-x3o$DoZm*e%X{dq$QQGU;(I;8U40I8(&u}Vp=|Nd1h z1Fb8Q<3OQDT!C1fg-O#$kD&OZ6`iKP;qcSxKc?edy|W+7lm9)TSWrQp%y@-ilz^uW z^>vIIZnLqcRdkJOn_B?$aYeBgC9l4xjD13xD=$?Ox*qM`B-My7n)VpdfLVG<7w}}j zdaXx&%@L6y8-KMDhiZG%bGXG;+AuTg@%Q(p4)jr4x}tG8`~*ne8mxMbjM!T#%rC?$ z8(tP&BG|Q*^D&rsPBJqJ6kyihOhincVDb(PiqZzB*iwzc+Uu)=uMic<@RzBO-kCb2GV^+}g=G=h_3M;^U7od3bQ)%IeiJWzY8=UcI zxg}spGYj2?kG%l^2Bn)$1XtaFm5i$`=LQpDb0MHT!$sNQtr3)0{#3%XeKxubIz|Dx zUs|aBvlZM?jVjeVGV8A~MaNi}Q#(_??CXtLYaJf-KQWxd&eM1w*QPY@(mauu7qnRr zOYL+LMQc2lG9kPif?o|b%RZH^x5{Qc3Vz|~qMNwl=Y-Hb(FY$E&BRpg+73?$F}@*6X=diBQR6G~ehp$o&+0x70F9*T#7HqBCarO;w4 zSkHtQgsxNzDulJ=d;f@GMgius_LwFT2A?>f2X+5!o5=EHF*5`>MHMNOi46&Hh<{Zg zs7Fq;)LGI?2k9FEsM>gkOw0aMw|U7*Qi0E;%0EwxPx#wboUw`tJigHkZY*ghy0_yO z#vj+qPv_XCo~ybq8nEKJ440dULL6Fg`q-0LN)Vr&_zOw@2#DAjsf4HRFu2dN`bZ}y z>wstP*plh0S0wi;k}0cdVfpQw9dzWYcWX>z0^iAYznLu+w*Sf{Mty5*DZUl9ar?{3 ztb~t6qHNLE&FL>+Y8t%BFe~y19e5RZ63xa6l&&L*dXIx-2^C7#S)Cqc6wY0%_Gp5Q z!ZHpSNxE&(Dhj>$e9+pyyFxX`FrA83KfcpC$}?F>bUi8F@H@1!spz3Q6*8ufA9{mMubqX7$k` zWJJ3%hs`0Gm_7p&>pnTrNouOf*YS(wCZbEFc7k$%YC;R5b5?t2 zpuoe85q~19KaIF}&i%OiFA7P_E}{?Li_p+O*0aJWId8bo< zQz40A7@b!Par!77oX-X~317*E-k~&W*goXW!;tzQJed1X0qWEYbOd#^NIGMyj-LZTqWj%uBd^mm`wajH0t&s6pFP08;pRL)Kj}W#In$(75C~p$fHajvwNRW z>!~m0hevoNHe6z16&V~x!W_jj;-@ov>nH82cs%~K?v5NzKp7?S=k;>%*xP@hZ}Ml4y+2Oa$`Ya0L1{uvadp)n}Kcr6?+R71P{nf8r-Jtuh43(p{m4#Mn=-<<9cvKG`dgVQu> zOxO5=gfjbn`o>XK5F$5ChaiKv|ER|xD3rzw)j5QpEQlM<{XEcIEj0VcFZgNl*`3f7 zZ!v-z1k6lrzijK(;s~801~S9L~xhO{}4`5p#jVaXFVznL%rF+2F@~Uu&~HWNNwlPkOTg5P>YiW74QQ)Q4(fU%J<% z1^bzbTKasnD4fGqhujRGeJ5jQ5pRT^dESl8XyoeoCBiZO%N>4*83s?JW7|b56tFsv z39fA%iYT9dpR-f^D?i^}b!UPMW^>a(LUE@wCvyp`66vJK5%~Y! zywd))*~Fh{Pl2L4`g|CA+>A?3hPZfx^WTl$&Ex)V=s)}2NBLk;`Jb#QfZc)kh8Q#kyIW>;K(b(cm!a5tOIpZDD*#M26Ng>nv(@SYz8{) z4NZF7;nJQ_9cw(QhV53`I&NtugwN20^^TpP$Kj_&+UIw_G#i7uF6YoL4p3JhJ;;|% z_U1}W83;Gzoljk@Cd+hdVwoV~Xb^bRJ62+jttEyDpg@)0L~Q3UgkfI=8O`MR$aK zo?ZULVzT+BG+??3I!tHJ(f{W5_v8^P7CU>T&n0UKI}Xz+lm=q3jj3jzBOoMEQtUww z(p&WSHwAH{0z!(O3M2FTFf}tf^_-0y`5_g6!J902!yc0(9=r4l8_0K{uMRIsy~aJf zdM^{zXw+vH^-(x9r0QjYg)9j?opBDEM6QGzn)Xj zZz|>nm1t1U{yySTk5|4;s3G|{PD`KcTUJ)M4|cB}QP_ZyN|{qU%zzpeix~IRi7gR- zMG98^@E-7JW7=tvDWRZeAE;%kn=>z8=X(TG@R5ZCr4I*aTqBfnwbss+sE_+XG<5v! zl>8e*Q1$LP>e5<4n=k#f3Tzg4AGR zusbbYErn62D`ez%t%!p2E33iJ&md2Im`+n#h68yf<6+|0?+ulNw|nn|m%PJ*PDwuf zXg;e&j8=cUg{#YWGfAw>VlB7ov1^G2bhD3{v3l~QtfZT(@}_e{UQh`0-`q%Mr7x^> z$Ad}-8X(HTcerF}5X8ijd)x<%A2&*YyAhrs2UhBQAV@awP7VPB0sZfgH6*$s6LXHB z&#+GXNV8wm!oCDgAy`gnXV`R~{frfAy6`xRiGO*_kAXAg^c>scI z8TU7!cOb6QY`rt`{Y8q;7^EXF=E3O-iPZ-{G7*c7nWl^X?H1Q{oa*kM-3bOU_=EKQ zOhXD{-xqtMygATPw_WFys(^YIP73hmhOZP_bPNcfcT+OdLDt3X@kJJ%d$~e)@yB(! zeH0FtV1dAzdd4}mUiA?y6rx-4lMM_&EXZ=@>kL6>10)J9%=+4cyZ;V2S8#oyZ402U zHY$)6zmbFg>G8)8sm_e}nCvar@Jx^gvJlq8rBaB^-9KB)Hdrmk*!qDAYM!b+pGK%2 z66yq@%(EkbA9O~lVEWajhNT2nv3Sr-7=y+-gs8Ex9+nNhXH$>{jinCPPa^tZzGQUQ z>xiK)+uDRempE^#iU_ED$dKB1iG;K7yx8QJY@LXJ`Pk^H$WEJUWXv>GHTsE=>i==# zyK<~pKBYJ`ttcC9`Rs~4|B)?HTyP+`Q`Odwdv8eY$Mn=OQ@RalW{piL=ow%VytV^J zF1MHr;09WskXGJgyISVjaKuSHmM}6dRHyAg+MeW60%W5PWNbr{Xq(>sE_R z**QaR6@);F0g`oe?J#n4{@9H)Y&jAI_Ca`20Pe5;#&8X96rPLb(!zvoEFI zOWn0tIKSO}0W0X)G6Au6{+cpAvlK?70h#=JIR`Rp-IDb3-mAs-p|sr(l?p6OeIzo| zx+a?qPQ-WMAo8rn{`OrLP(}Ct@9~gC)$fOs5daKfxdk@j(xmq zq%{-dc37pz9B2OR5DO>z^dZe$*ewITO+{amlF5SG^T}@6Q=qYSl`n9mhvMfg@5%IhlRPQYKH`(*J?5jP zRLb-1)K}M`?BhmgK;zDfcp;O<;sDFQ@#5Q(2L5JoNv%diOo?<08iqzhvF)fkO{*-; zFjn^;JgGY~Qpj>P59Hydze?mwwJgR<1R0fAz?%G5IqdAFqm>VmInhOG2+Ft5OyN)6 zTLuXVwyqq4J7N`A7Y&b+uMZw^4Ua(7S!7%L`SVGJuAVHL`#Ti&?F2tM3Fd4|>S@sZ zb%uN2dl@9#ukHLLWfW(0uf2JHSSY?`!kPvy)MScuUv zHp=c!V`*Q{yjVQE@oxBN$YkVF_SU+5)#fv0b4QLj3U*e#pUd1d9ef-K` z(4)sq?7dkt7|1#p(9jI3>&!aL{S{)^r$$J6m%;Ao!0st;guCBq{sd~72K062lpMk* zkMs?I-$yK|4Ei;>Vurm*4^WqaA|C16RCk!};Mv7(YVq zOkfyeb-=`*3eRW&$k7&K|&(`Fi+069AdxYxna!N zUKBH@bHVWfJXJ;87^(I#@5>hL>`kP$462hyLUE&QH|br5*x$ilOGZp0 z;vdYpl=6i|<8Lbtkvlt$DIM*{6Aup%B8voHCq*RmQ)~>yb^VNed)5otXGBSK=Q|2P zd{cy%su1V5=qHv^=e)P1YL!MHlfojV$R}N>!)h~Q&R&bw8uU5*%N!k#aA7sioid5A z=^{jdegEgF^iL)`%D$dJg0d3+wv0yws7%8CCt;ZwgC+{lLhVn+dw3bRmNlQ#6MJ_N z&4@pHFd%DgAaTSiFXR(%0J?)D4K08GL8uefTw=T4hv+JWUayY9`E2=Zu|%-cdmE1& zQzSx76#L5NSIBG|=CTyeDVq?wn@)k0%fv^np|~MV@P=tVZ73&q3J15`Z0gCxayM?- z>EHJE^B!No>!S&6hYmi*N($#@Q>|hm6$QmnxqZxR2 zNG_=a>bHwLX;8O0mX<2@HHL_6q);rHv=$|=$jSm5Q&@SJ0+&^jG`R)(f*@E(3Z}M( zy}dvy2g6JJuqj9KE?!y}Z|qC;wq*0^=JGH7Y<~rus)tkLwuBQ@@FNYqTS#n8_%67R1WT2k&TB^ng8BL2{cdZ(J35ddq8jg(9b+;MZA(|YF_8q^zfn*-B?5W3uyNA6~iEaJn1C4kk8vo`!+<`|K z$VgTvwNzt{5G$p_NPe6-jWam=E+s6BO#2UK8?&5= z;Z01~g-n<5Puv}+j+1CQUqp7!#QCl(E;x3sB}u2-(I>0gQDWx@txzf^EV-fqk<#VIZpsELb~rUVZTu-ti;%<_n`BxUx#CGisd-WLXSo}L z8yo+SIYk!Yr&D_@g|8}ye+0zSk3twJCklX6l;s?`$A1nb7cK?>W1$fFx8dGTtm%=N z_izlNT9ae8qa5GDU;VR~NnP{lY2d1^R#ec@0){hZ)~Dx30z?2t*HFMLf^11~CpC2u zT#T-X0eguGAvSY;_DyStil_%zA_1HE*+T-vbYMusn-$whAgO~NWZD}y+gTtr{|9u| zUEtu!8*_VvdIqnuOc%b~?HSEP@PPTQJknFB4B9ziw`UG^4kdIZc8-1by4^uI(%ODj zGB&K;I>SZ$DtDQO)H(~AKURxSTkNP{$|k0Sv#uIL)*c*ovTwS$Yh7_qf~Lr@ChzPb z&*353b5{lAXcOjq)AKuJZElRe0|0WHDC2Zx1CAoRK12ING(gAZIa{oU$$l@!PiVFa zRjE3B^}5VniL_{z6dX08>f2c(!j#pm{7&?`}hww#1Q;b9Vij%h~d6n%fO!9zp`O%Iz4li&w?>4c>H9z)j`-E`_{Oh2w{w9 z5QGpRYkC38I1i}20Sn0xCIyr*r#RI?_K+qBe{f@t7`RWBG<7Wt`C3UhADt9cPX|=_ zvRvy^XH5brJG>B9n_#NQK`n_Tw z1ApCvNwk*ta|xm}gQCjjs|_cKMmY-U+Yxi8Xxc>TvOyYF#96FT7L8o+reXO^#>sE1 z+sH?8NCs4P5(-N(@#$c9QzAQ*v*gq{>I*^S>|rv!#?V{SQS*(0oRyk%$T@B(zM(>l^i3U@qNR+LU$CQF#cQC*?aYlr)_6}``Lh)*{UCw@S73dTR0e}Pyoj49wG!6rL914al2^ANbr~#}N zuqxVjGYcT>gbq;Rw6I1K(^KKE<4J`k{V@$b)DS&EbYlJrds5uBy%jmPigcap=-nnFlbxVXnBjE5 zT1)@)yT5eJTMMQGi6>KvkdL&ugSwnPQHInO(ypC9F6+NCiqp2inK>+~i8;#xJz5#L zG>#!#g0L9{mI(I39KC%k#fa>f={w8ynCW^}ilwVlgp+hcK=-&MOZT;kgNocDKj4LL z5kMI2Mp_p-SL$%$%9ls$r3b&gOMpGGg-%`jnn)JX9zJXbAH87@(Yj?E)Q4ho<%y4Sv30oSb z=TZB@z>Z$4m+U5$cx^MUz8Z+r;K^C`B+9iZU86IGn!3Jzg=C%iq=zu+0(dM*ViFVS-g$WHoRs2Vl)|@ax9+W zQskXmma7~}Orm}0AqBs!bI9;eq1106K}0R^I^ZE<Q8 z;OHrJ>(F-iyKjDL2xP}On0L8z;b=L|JBLbM{wb7F;t^Sz;0OuXS0_j;;~`aB_?b~O zWIO^ngvlMNx@a70DqNWiY9UKCt=wo_n~caQi?mE@KvL}%9(5QjS(;Pw1Swvnf>p}t z0;eE*r7XtEt+q1K)e7jv8p4`cGX+lzW5`o#tVS^)SG`hC-u8)|*AkqDGgV*gjy>k2 zI?DtVqxiaN^-dR%`L%*^$vrv~#J9@7J2*s`9jtC=Zwo?kh|1UGY`oo4J;1SrH;%vr zUL&p#+w861SSe(Mb~E5H>&0i> zOe}V1Mc-4@aQPtQkl~6a$!~BhrI6A#W~_f3QSo5a_RHPjb??3?3Wc)EO#7yIy@Dnz zB16%XCwdjY37)v|=;1=4ryV%OyFg8=ND9l-2E(T%)f?hM=3o%(*Tn|sq&A3eI@c@U zB74%KAAnk*@t4R)^h{!SNoFC<(Vz8C$wGG5s1I~a+P%qVfg91e9@ttCd1%d!^iKK_ z{K}0Kw4jaGz&g~KB+D&f7zIqC<(4n+i&6%y$TBi%X&NiTnriDrXVAEN#kmL?)-4~! z$-=bJm~8*zFm76m>PlOAt8f7otkjR1qr*OUoE&+JaH-^P$Oh?vu-z%aogy)6*TyS# z11d_%gtP6F%84MZ+tOO|DK^ekmTjD=Tj>buAxZd>m&~Qd?G(yqcQJ@xiPNC>QVHeE z7(h8RNKOpatkP_F6?4i`8Jfl=2hZu!kN2O|V463IZd%^g+hZYb{hyD5p_N4jr2fc~ zgQfAQ?kTny!EVB}#I^H~rh7{O#|{gsV`3Rw-0>0d;&*owWgmB))0YA~ zE6J2(ZL{HuNjQrOx2!qNKuHE@z{|p^CfaE;TGl5Gl(x_`>>vnpfqYWo63%Z#eMqA} z;5e~?t+g7jV3(tsK+$8fJ5g0Fsl`~wPgCiC8- zchC#`GJJ9svSOU^6*fQOm{nE+riT3IENn)Z3a;eHBm=ozuKhtjS}%XZUo zCuVh^vZdw}b~WOD(CJAd=Zr6J!-=JC8CSUcP;sP>)oR_4^sS&&^4rV?WF?boo`#rp z$h`DYPqc$rT!?l!Qpctw|3sOSEupA5-p7==P>9=R8jP7yPko4Wk5;^ENjyd4wtH}f0tKyH)^_~NX5Y!I>HO7_BS@MI=3n&x z4II1v@vFOH9cLyRQY05rKRi{Dbj_oc5IA|2gu~}Z$ceeqJCK@*O^Bfx59Fa8xAIGk zi}WRwiF@_huEef=c~CWop7Kw~%<{P6%m+9M>W)zVu)iI6f6HY*$qNA$EYTUI&V5Lp zo+s-KK{>vBqm3IV=eXdl+BA@0NoeSfezIV-g>Oas`_S)1K)`kqm-ah*_@vQ>R=;8W9Va4^&YMlY=DIk{^tvbJHWTYGB)s)E)}dW%Whv4b70f z$q9mX>Y2u`9j;ltUuBu&Yglo{=w@(UmuNkd@rF4W)v4Da5-p{Ch;h*zWq(KfoNsG= zADKFzJMjuX@o3Y|YROae_`wq|@9pWM;qfc|y%|hzc}h(@@42Kg>WaEq$mJwM5>4aE z0|&O(l1S;hR`lC)K#6?7F$Wk^uu9gklrbc!2ZlZ0EApPL{DgmA_5}__`!CG&#E3=o zom*ZrUNj4+T&apu`m0{tT{!MCjBf>!bp;W-0!qKua!8VskgF2cDK8tAyHcm#;0Ers zis*-@YhIpX*Tig_edBSESb84U;q5TuD(Z~lr?zqVh%5DVeLQN{0?(;Jvq(LW!VfhB zTMA}Q9MCg4{?!SI*D1gFsaCBS;^yp z9_rpU2~{5q?gLfiBnj=3erX6HZ^L!flGM`hK$ev)-+>urHK|G#OdSfo zQ2mGnColZZe6Wa3;_ge~^BSa(fJ4${8XT_0mfe#je>Rg1@a`at3AEKZ#^+4pM`I~kgSV5)Qfo%)I z-JS=0@h=>M*~nkVQM)25;Fv4Le8dU1O0R3H~gShLVd1hAz{G zXPZ*?!TflAu@F-Cvb2d%e&IRQLosuAbhrvH5lVM9E@p)D$npCmbd<9QGcQut|1s9q zZ|Y6&8?=D2Ak_(-6DdiE4%mTgd|SJ)~C0j0kKF*LP$zWbL^k{ zI$`#S3KdVW?*YR7{Ey;V9{kcqtB&kHYN(ge+Q&$dL05E+wbjvb&V;eunVyu-&||QL zgF9g)f5n(IUQ(NDLp3dfm&XVC>oL34b7cmH{&c|)!!sF|Z>*Iv`exGB}qa!^3 z9vpM?V8UxNZs51m_4c`K9P^t|1whDVlU2&U;idxHgXfYF!>a|~efWE#@}?r=d!oiG zhx3vSFD=|+KVaQ&S_L@``^6@>0D}rDLo0Wg{%2ihFola*@-kvGtRRYGMu4x;1IW7G z*%gLf!$kHQT6Si_!}nikjOD(a0E>1R?_^PD0vuILUkGk14HcS<3T4=1`1H6w}lA)>t_$P~1Obmd_V_IzE;EMYr$UFu>oyV&W|; z=aX`^vbxhO;g)>c#uF zp5cNn(AT$2HB9B3_?k%;nvSA;g$eu5R<0Y?r7)&6NEyf2d#%Q92yNSb69D{|Mq`s` zabCGnwKK%FH*&PGVJ?2F^`CXu;!0w%iMR0;CP zHM}HF7i@(e1zBtl@oJNqO_sHXsPs$uCEM}~G>AFxva{Fhr}EtFDE$rHG&f}lheiDZ zU5|<+nr{9q8pPa7X4VkRGHOLc?W8CQt!Q83_3J6v>sfX+kPhIyow=q9^^XJD!X?9x zz=KX`X}B^4w1#8lSvZb5^ar;Qx?_Lp+v4i^o`XV`5`~G?Y!OVXn#UWV6IBDdQ98Z+ z{n zCYYXK&RNu~hucowh_l#rv!N59y zN6}k#aO5RFbmS#^)Z-^^yhfO;>r$|EC*W7W(_3*s*XdmhnR&hxQ6(2a?4)ExS(8?H z5c!7sJH|JmIwAmJ_}&arXh7-cEi0nnNSB~*{!XsD>OhN!y52L?Z|Ph7vwDZ`2AkV) z#!Ed<^s`Rsht3XX570Kp`%m(T!bPiDTc2I0Vte)37rd8L+%?907s8eYf8I(7w1RlK z$J*L$%<7$<+Z4=~BHv%4-~DM`S2x>#1QbW_p(Uj-*NY>$G@8a?JXqXpa)_FLA_>F` zO|ta$ZM3g+({tnw)c4Q#IFNw#eSTyyq|xV~n>8x`$#$2RL86 z;?-!TaE%JPT`b4536jt~5L%UkF?^qQ@<@0pyN>wfd^d{X`~;;*G#x4)cif$JWHx#N zA3N^$2n@K?xafCr)2Cug@I@~3Te^vSM?0)f{|Nd9zq+&&bT3!>U2;!Z>}H_NKRZ`$ zpjp%legz(?SghPo{Y4d41y*wfgL|r_P6o`osvptRq%5(O6Z3~p33qOz%3jv zdQhlk#kIFL=H@IYuVj$Vg+^-M% zc-5M&dE6knS6B`qFN_A8tBuktQE@LZP}MK)jAHb^P(J1XH22+DT0Oy*$^bed(p#xc zq@@^kDkgT^T;-GLf$e{P2bO4627oxnCoDvcHG~F~M?Dscf6$!e9Vb$`tJ=eD-N`(R z)aPX8pE^MJ-lO>7)X|ilpP$GVZDD5DE}oywOC6Cvl#;t0az2QcHZ30JOb#10v@jE< zImC)9T|Z*inOON0u#uj9#>eVM&z}dDIqG$fK_5`GvY5sVw}(5+CKFLTEb8VxcqZE> zzKHv2gZs9!5Jy;ZcXis8ZTn8XG9J3Fl&Zn6d;ACu$$XjLGc>yV1G*-Ax@-bboa(kZ zXBN!*cg&e7kuKgH~1#jPt3>$^_8*E8TN;+l+uk|O|)BTGcA&`H$dON!u5q8v8 zs3zlyL+FI^*?J>@jOqd~{l;SBHTiE*?vWaAy&jkYvJS_Z58KsL8K z+ME?=M)~K_gwvg2=#J!%RkEKQDDuQFK9b1DP|m(-B&f%8G^7=#_KaU}wdD0h2N(h~ z5gpEC({Uf_ACA&YKivc3g3^C2R~X5}`5ar@x!X_}vvGoP>wcvOGfdi=2b>_mLR=CU zJ*3J1h5oO>9{8ge*W)|bOJah6nE(H>E)*Rdog@KvZWb=87WR%F-=K44Yj0P8nTg&1 zdjHSRzoT`fN1#O*FxNO0W+6&jNKvBDC?f+lWr1$18=gs($RRQzN(uHW#Fek5hs-z{81mhSRY9KX zY3|# zKM$QD;|ZGJwpjqK$#(p)bl>)aaWe0j$+k)~Z@D3JkqwYW%6B6y-x(C`mz;?ou2T57 zy|T;;=ER4_#B;14Km8hwIeTx}{*?)Tq$ACnC5|s^C+OVNeK?E9k)F%TjjSK^p2l5K zNCpHKkIyF2+O!(jiU4+tGh@0dBKxYD`3_Pr5R}{k;4r%Y`PY5wWq@c)W~4bb9cRT1 zmmHIQ5qD|tb_t=}+*eyJ_Ck}5>ldZ{5A-GKXNYKfxi-}ZiR6XK>ZJK;B70d+(|89P z7d3WY_Rivt)|9yCXF=1<+FQ43S(njehDo~W=lhx?lt8eP+lnOnb?W&x(RTpjQXRD` z?2Z-+b)2JhUs(E!>Wjz_+qM{39d~WZT?U`iAu|~S^+q${voHP?x-)~rT zAL$P`NFqIeNOeCxZsjDZqhLiAFIm?_0)&Fx2%r7da?g7DMG9{4@h^n$tHL?U4*#J< zrp{}dbYHuGIXQ~G)a)Gj3&P$CYV$tCbq&3{=Tb*U z=baBFm+@I{n0PMNY8uBR7a$0ja2O=KWlzZS%gQ?3fwIgeujDp&VVZSE<({OMnOMje zziR{8R$fWg@z}5&=)Ycf2*Zc@r948}UA>QqDKwa%@dY7-x!!6nrsjJa&@xc?Mi7FvI-UNw(* zNPhBTW6R=fx};?-?#-XFNaAY|h@&}Nmf3tCcH1na-kl&+nK|T+893?=8K`ov^`)>s z_+j|NYezd}IJqBTvRFm3>!S4yQ~Hz?BZ_Gu4S10M+aaauYBc6)x>T=dPm+H{1OAhw zPXM7UbH_bSA;u`#MpkPhSE|KMzu>9P>};cm@pow}jrl<#9v&+}Wi5XdwH^ILI7Esr zQ0AgG(EK^Yb*3(MfB}7py&-3FL$&Vb)Mrj(x+XxhsoR4oi9&2419Lq!U4ZL%2{^!$3a%Vu6B zw)snlpD*_@%Q}yBqmU{J>lYlzp_zd4@jT%3I<0r`0^5Tq$-Iwi?$_z|f_kI+`Pu{5 zU}j#)jjn883l;pfy!G148Q4}vYnZ#aHu;PFq6G?SGbt2FKd~bz%3IXyi?Dl3_q`L* z@jns%;k*QA4^Yxn=WQbUUMN)iEQx*nDUQi9)Yw@vHwV0vwV&Nb6K^~PoJ9YIj`^nF zQSHR2H#=0v{AER;kwAB^O;jN&!q-URRo5(Rk~@*4${R?1Feee{yjZRCMYQkIn`2UB z2himNaP2#Qajq4IPHP0e%AH@+kpEq{F2hLMPV(DcT0`H#*z?8b@rd*Z%JdXGFaQ%R zsqL=^RW9p9$>Wn$rT;+ECp^}FMs}Pw-_v5WE;; zSazMs-Odrx`m*9=CoH3!ATI8q2r5KJV*f-*%-|DJ z8wNj4p^^6j=?)%?mZaDnJez&_N#L;pEz}?Uv5#<@abTCuKB{%79Ce4q`zYG2Gq{i~ zHT1no{*bt8f*cVX z1mqS51VsJ6J?{T!oUhRYs4WTO1hh6oh(bU}+>1lrkSc(tsqw?nDDpsIMl#RZE#+~{ zQ)QKqd=dQ%nkC_3J3_vb;_b~z7sKI?@H@BM`&_r$yGbb6>jlT@w_~3@$PXz2;;7Q; zCTS)O_+$PVr<>178VZ7g40XXxJ_hgLSHF`UU=iCg1<=o-0|6d2V|>2b z1lmKRar_ha2Tz-l>_3xFQ%pV`a@jhB%cnA0{z_sR9MbHZGMJEEeWtPff`+h^9;H8o zAq@(^>RyjEf3-wb(88;*f@`|HeZ1M?P*Pt8iLS%6p{9lMFsTl>v2ESEn8CoYDcixd z`Dm4nSX!Jlmf$fsDo*|ZwTRaR%`M++T{^F_1KtO=kJJdjw6z!MM8-J#v=n}Bt&{k9 z`nq9yyT=6Ja@k=`1L}g$9A6yKl!yrV4I=_?|Lo-~* z!n#qMlOP`#6K|N9)Y{El`*^gcs#Mmlg4{+N@zP=YhlM7DttKRVE=P3 z>2k%{uX*Wny;i9=At?b^;hHLy2Xn^W6DcFpqPzreu4fCpjJ%Kr6L=zNTilXgl19Sa z+Da1j)V zKwCFA1fHC0gYU{TuDFOa0tI!&6NUgGwT9T7{Ct(g0Vsd!`aquTmD<0@!iv{t|1SuZ zv1VbFpKdhVTvW_+4VaM|RXT0B3=_I99*Ix&N8mrU1uGqE67bU5I5e@_owaq~h6z6< zD`J9MulK}lZQ5szq<2RW#5gb!Rl;AkYf6cF7~aIt{!s4B5$?12tE4@xyX?vRR?d4{ zZiLdc_dJA-Jn1v7>Ov40`kiY<+6K$yqUefK-R$=m-8Wh8z>2j(E!{b+|?4?CVv9lAhw>zJ+h*pemq;V0iZ0}V%E8^xg$QnRlC z+Si0Hd#Muq{11EoV3QGd$y+UfGjnlq!lWe66 z-A1NYy+mLR;NSA8c?M<@lmGV3>Fj|rzzgRD%vEyIs}^d&Vv$13f~>J=04Q6fkYJyY z9s$=qI&_6cG*Aq;JAX&@LbG0FLi}Ru1@B1jB{c*}fRXt@ye2c$#Zz)1K%jZo8-eU3 zKEz(3;iNtk$y0Vf?pM9v{(VDJ8G+6tvyrkxSAM{ryXcgiM%a%VTuoIjtQXHz8_`6N zJjZ)%WGu9^txv@PzFT}ifMxxLcZTU#eDIBIH4#+H7n2@(<*7b^{-UX8Ek`C2aMPIp zf$B@6CoWcZUXx3-oZ$BdOY=+}*V_(dvG4)jepZ!}cTak1&2HUNx~s2QVM5W-`cRPB zUt*l8*0ob^AsUP%J{Z*HsPE{yKCY{M`!81}>fu_YYHbh27)e8W;jSLJS=xDTYQlL+ z)$>~5MkATxaZU&Lc>NF7-YK}!K-=1mjgD>Gwr#Ux+g8W6Z5u0QCmq}9*tYX$@3YTe z-`REU&c0c-?pD>iX4M??dB+^{IKHp?_S++Sv2H@zQobvXjAd#bO^w6VAG)^RY;4JU z$b|1je*=ZZo~6l}Elhu^1akyP(A7=L8awK#fFq>faa}GsAi>l&D25ondNzC$c@3K5 zsQ{5KvkNnATfa$!-3o5PjP-zEuCqN)M6k}WR#9QjXyhVvCAqB%-Awe%&*)wKty5&F zH*hY^)C@xpC74e(9*{ceoH%^NnjAAl5L)PAn`SyHI!1>L!g5v7I#bgQ$qr+NO;%a! z-frriI-#T09ni!b7erj|5DeBiqO%x(fN6yNQIae6btfhT-KM~LH(Lji0;YvFiEWW1 z@s}w@Escn}s;;l8wFA-9_gbH@5pt0gor~5x!j(X}fuPBfE%)&l^Uwf)d*axVEOVPa zs)yDGxCX4M`ux{bD(51X7mz-#i>M{NiuQu>{T=dw<15_|iX~6mJQll=*8y30Uc%2s z*b)rzk85uScq`urAehQZZzT^IIjRWY77G*yuq4N1VaEw2j!CrS1|M##`tSu=!a;Q7 zh`m8&-VF7w#3dWMhAPdf(-BIQu6<=k0ckKRAK;$?;;^_Q;<+g>PL8s2euTXk#@%HN*%W)XuS~hVd5T3~ z$!F;vxJ{sAY_ionG{Ko+wMINd|%lx9*{pu>uWlKsI0{Z z@C{mkuS|I{uM_H2yqRK}^@r~R;*|G@mOrp;5a04b4cQG9H{%++3XoPs3&5s+6q*+( zjvSSRDhz6%=HLrI;i@ajpbvXim!{f~&=1 zK6|{PlB7yEXeU8$qMi&CPR#C_pmLW;>ccopPtZq#&s#4ksmaFBv ziLSS)yYW4g08myR)P}>NU^c`&J{zbB@`{E9Xf@IqODvvvYW4njlWNGuvLKzvSEusV z;i1G!D$KGqQB-)BA*#VJQAlc=3dlxgvsh6#r#xdDnB?8)_UxeodGMZInSSMRUB!@# z^y=MTP5PZ@=y(5UWib($|L_fiH0zj@8~l;X}yPD9Qs+Td99sj6~vcw|l# zDM0QpSef0Y7F}(T%5%k$R`(RipdH^w1akeqw~S8XKr?l>qn(*|Wkb&&b=phAn6&Q^(Kc7*HQ zI4~ZkYbrereQg!Yh5gfwLk!J-zIvizHsCzCrVLUW`Up7UFQ|1A6ClA4y8lwXj|~ub z)R;`=r609$OZ9=A3c~AlC1d!T((eD>z4|?#6fEV~sxvID-kN#fKR|4p<1SXs(is;Q zT|U|MZ$;xuHgQV(jdt(8(XPsWx@i7YC7RIgI2vf5xn0qGxg8ZrfC9Q2LV4=ZaZU=u zs9&9=@Qw*-E5G{-=VcvG6_|T(W;x$Ku}l zsLyPl&g@t|2O#|F^`ij^=3Z0UH;hd?T4Rh0VW-_S#K)h#7Rmt-EIR0q;QN%L%Gxgi?SL8o;5!%n4*R1qnicEXPgKg2mb%44ZD{B=rb(W@^oIaUpf| zpj!NC*9!DlduS>>(0Yt!wdKfjhYggX&?<{(o40cd;PgE?pw&4vCBkjxMb727p42j9 za&aQJ{ULQ0oJ6N-m~eovq$@guJ>- zAeEdMZ|%w*~u-PQYGMzqp5qv<>RKE`XVO;W-jHb%XIZx=P&LD7J> zfsKGF6m@OKQJqEnE^xnCa8HSYgsX6fQF%f^=8OMPJD=ntF(N7T>bj$A6?2L+S-0o` zJS*GAWUrMLFf1ZH!A0R9yhVJX1Lgo|GXBlJ*0G>&##8fR|FN-2>A!t? z9+_eG7SE-D_B&Yc)7a1Kxh&V}U2@x@Y$ytv{cwhZ3)Ay*Rju8OG%V+VeV`AM6Yy6} zp%`#o;VL$R?r^$&cMv^MD%d^d+i%Z-p#<2^NPafk6dqFBpBG(Z5L=)f6lJXy^_U)5 z0&LeTx~vWne6(1a12hn+Hf z6CJDgrrNzYwv3;nT@dOQ?K{ElKjP3ZfZM85Ui zd+ldTu`W~>Zf8B69%0lWf;L($RL+G53Nhk+xskXP`RCvJ_EL|#$vyDRuGk@RJZa-? zT4}TzBJ7BC?WCcZ(pe*=y;_{a^20w4fk`?n9Y%KGIHpe4t=VY6BZjs;EDf_hrAuSI z^WeD^O>_@W)vLDvOO#`|h0ia+MzgB-VJ8Bqc(}Wy%oH33m5i(bl#W^5?WHOV;m8gZ z&ON7M>%Hu3QC#v@U{>N`J(5YIjTqtW`b@+bf`^K#GojbcA}e2t%oA^n%hH@8*75uZ zx4>G}Cm8IqjSUfXtk}AC`q3vF`I&*6$cM%uk+)Hu3vs^g`8g#xxdr(yl*;9%NRO2H;an zC7NaRf;ETbt)#qC8)UYn&Pzw_gltuhBrDyaIrN{e1R^Nizzg|Ex(kCje#~aFyv<7Q zp{_Y(;(7qRT@xbg5M>98bz)5rMM*fSb8$*u=aBxH!xRUtu^WaGmzv66!ztqPc8$~3 z1>fa$j6s1zp>_!(jB-@bK^W+bh$GS>m9Q`D&5{9xl#z8gVzntt^G5aGb{Jx{{r;2i z?4ST`Us?SaeSe>NeYh2FVGpgxx;{d)w&f4(9R{7WghmRrZvqM4mIXNZ=?>jJejKF) zNh|q9`r_@4mBK#&>R@?4S_H%gMHLd6EbELq;Vk;bsKoHYEQ*!CaE42i@}BUw9IX^r zw9d7D$f)Ipcyk#Vc~n(8?XGWd;mT~#8}tJdv52qJpZuh=rk@x#_>!a*~|=yx@<}zjDIDzhnd`!C@qEbwnCCugQL-MG}ju zN;Hp!%n##d)5R{YafV;laf(XfJ_Em#2YW2XY)50$UQC>O9C=DKtr7J0_(EO-wRKt; zdxK**OlF;JZ5$hv{mO!&(}`5!Jk?4Q>sT0-ESxi$Riq#&mMlt;uc0G(AsuaIZjC2q z0gsC|LrFUBDl^|L{4~NyDxo>cP0%^)P(03yI>Sm5xaW5#b>Gr?O5_c3KMLX*ZTWg> zvz7iGX6|s>oR>AA%7E&3N$<`%jz8+3J+KwP;c1|vqOhMu?bD}kPQN6aPeVu8%bg4h2@6IrKtZF1`mEj0JRO6nf~8EG{+EtV8NZmL~JF^Ct@UuT(O&L)kNun_D+sIyq+@lpV~=L zyVnE{*3;+~Xv@^}^L0#Pvw~56H19BITRBBYTUkX|m(#Pa13sW<&FjS=h@Q7dmT_7i zU>et`I1{1f4=W}RuKTWPc%BF1c%B#JO*(9Mr}qC-<>S1}Atk_l*TgrF|34o?{^nSU zHiq8^QCz-X$NyJZbZ0{Q;Hu$`e0{NAShub>Hy`xW;ZxaCax{alu~N~klkT5pv^I4n zO|mxe%&%_710C>dqpB(jfuntYl#CVRZ->C(mwOcKhD2ge>w6GB``?+7ua5Mf3M8c| z2z;2En?Jt2eQjm!O?rP({PcWQ0@4gxX35S08sdWH#CWRv<$N6gh93?|c5Q%&YqP(?%H> z^`BYaaa%%w#b`3GKh`8bnC#6NR{xn1FZFSk{y4|699y0w9;RXVi;-dVO9`4lk3!_ zTeO3n;ny_V;uK>^!eW5l81o^%>d%LCfcwp+Hl_5#kNS% z#s;y+x~!&TNZ$;tdTtde=WYb2Cm*t)b#ycnibeT8YRQS;)=N)(t<>JE1ed#ZDLJ?WFcb+yyHjZ}bcH_q>L`|WZG?zHKp)+q&t|iVl zqiJg|HBZKkt8Q~_w(l4wkAUw${ee6wf;E5883ot*qx5A;`vmgWLU=cK7c{_OWO_*M zmGw{FRasIGIggIZ%FXJ4T*8LU0y&bKe3a?@XnBP*nC{9+(}RnAejpoXY?03MWPS!qxB7@yhIkMZ&q$GuT`|dwksh3Hr zK5^V$@=HSH7<|i1yB?Fp#T8c|9mk!*c@!YS8W%X;}Dtb9j4t$>L!aK33SA|$?M zy>;pnFAL|^HR}y%bi@9gCfkSa*8c zAz$_wx?hYcaIh!AWrqG50{($hvAqpSK3m1Rhq-%4B?Wpkyh182`HU9Pz5PG%+p8L0 zs0uYGa2aYYt?n}2Q{iNUIo z^-)dLnrf?)L;(EsA=>~#gqI6~hn@PGt8VhbX!j0}q_IH;M?X^&Lcrg~J${mCevK3s znhU+tf2Ph=Q=~(jFf-TDsMo0Ltr=DFgP!(aTzjfnlC;Z9UF#S6Pv8iy^tw{AnO*&O zyl9?e0uiDvt0gY4TI^kjoL!4{D^7uqTUgQtsgauKKm9G!WJ}h{f(AWt?1+X%2spO= znteop_FXJ1!!Jp^qKlH?eVkwbVIQA2a>X3ZIad|v(#7ExoR#j z!V7Z;^~#bSlQZtZt@cK$gYNyVp;=DZfp!|>oyy)iLr=a-2NL_kUn!Q&+MF0oOARoy zukzX>8{3rNfA^a4ipbh5)!`@vvZ9UNN2gC?#%BA$3K*C_NlFDtbOj}~JZ;B`_<7Qk z>TkVL5rf55hi3ZUWHgX|A!2kl4W`X14-={udWW7I$Qn`~@)dRHe*fO@PK9$BqHOIG zTt)`grUBInEV z;nwkD`)e+71%D~IkV3${P2IGx1Gy&lPM<+Y?!5yUb4$$5 zBPpOhsi2=RCzuG+EM01VtUk&xwP3WK0lL!=aS(UVFBE+c4-OJ@4ekyO77j0zUWC{K zE(%Tt?t2d$OaG5UIMEKhkq6VD3?}lea%dV;SN0AXT%Cyr+F%uyev;@AcIYy5SL}{m zsP~Vh*q^?P!f$|NK4&9D0jclM2@q2Z@Gw#EE-3gqt{90}gz2R6Y%hzAsOH68pEk}A zpA&q(gNb{7eO8$Zzd-NKpJm-b8m3GGUUH^A+KC7

1b=sMZPbu;%a=;JNn@#Wgd^ zGM{x+YYXxItnsrBr;s60-1O8+V&#Fjs`s{Y8u6tE@>G%QtN~&rTH5|OsuNA#6Ha8o zwwX~OKU$BgS7r_K8*@c}2}`T5R5v()8CGrG#-ztxH&9bG=MPp!%fpjFU?byGsbjj| zOU?>bzZH3X=!lTK(VT2*tA`_XxX`XyNiF{bTDsN^M>jF%I~viKS^y!?N|}%!TFVIa zek@O+3D&Qn5yz(YXz(dA-N&K@Kk-n4OEoomIN(>uFhU4;WE-z5(HlVf^&UKBI)W?L z+e2j>90;8q5X1Gn_S8jz3xCwt4is?j4_;mD!R%`nq=uShSp`on0{@Y-S$~bx7XwQW zlo@8AJU9TE@5bP(2TF%}W{CW|oe}S_pEez_%JK~8hJ_Px3FtX*2^s(7?@N54?_`fb zjKAvUK}$r2aCbXBTh}KAZSH977c#eVl+Zobre=hCAGt=tXZV^M^Igvu|Q$= z{lV$3uISKSjnyCFm8>3|2X!0kdvC8V2j&`R1(_2O9NeQ6PG4agUb>znP;W)6rr`

gwtTF(%R*`i|$Y~F?!cjVMxE;ARu}@== z4d%L^*(1Df5aznYAqQ*=-n+78lnD4kv0cthDN9${)+wVjwG#6)I>_HjpkACUlEA;P?v#BJ{KAj*w6kQ4+y#)OE zKm;Vet_ZKcACsO#8`vkIk0da^I>cYG?aq{KM44^xZGG@(PAnh9z$>)?0T6)Z(fdr}n1o?a^+hN;DFxR|T?~H*2N&y05-ZcGj zePw9dFU-$OApYvWpK!d?fdoayRlhL)Y%?G~JAywuUwcBogLC(A1B4(1gtjTrwnuhk zke|n|(ZD?l;65P&^OOVf2uG!_HFm6FuA!NqEkXPxfIqoM)r0ba?5X>rOs)~3-lIX@ z>7HA`{4x{!Dz|?#KWl+}i2{Gp`%L@yKJ$-ijIfOXe*U=R0>8w1E{JnG{;Pt@(7{dFG@9=W_~ z(|0CVPlyBE#eE;>eiR2U-nVLe(<&y{{!s6YAnzdWv(tA9*XClmWzw8pt2vS7ech7q z${9yLckE%VEtx%-1IsA;{L2n2z9+o)$x2n;c|txGo-J_ zk)M07BSCyh;c`W`KhU-{kf{5+=AbI0RiM|GXfO7>yjFDeT{1#m7i-I6$8;blTq%UH{2zA%0 z;on4?Y29giZ5zQGp6|gzis8T1&=*{a|@AsCJI+ zH{x>|&Vxou>v|lzIIea=%2j=XCwVo|7mH)HY#Sk)^!c))XZh(T>P}YCdAy?gb&UPs z(1TyO455|3tDz){V?&qv`&?m)#K@Ve|D&eIbSxV&!+(6B7BD!qFIRwNs&oST-h|CP7qx%jG@kmV-y@Bdd6j@A z+8vNU204<0ftT^t7ubEItEBuEy`l!ev6;&U|LXA_W{U(~iWHgM&G1K>4$HxheA}gC z&y5eWb}WO5qfFr*0q#$4EHJ*WTNdw^K7y%orU@D;O{ZStzGA${ALiBU+7cZHO+m=U zy@8s0+I`BZes<4P*vEZ@n&0o|q~NVn0*ZB-)RK&0l_5dAuURmaWu+3XcMgj1db3xf zyB0e4x^2?S*d3Qcfg$wSER3c*WL-AfXs~s=Y_K@>vwdgSts6>{GR8Wz#j~SPfQ14p zfIC9vU`jkjbK5DJQ;>%J$CfjhFojda4Ixb&`@u_v*Qi^`tq_^%4t$s8HXW?}URQ{_ z!&YA>`qtnZb}G|Z%>iLGE|lfu0Oa#bp9DJpFaq}=^S^u8Ve5yg|Hn4;~-AaiDr!Mp~sZwTx?+e1hBC$;a=7*R5 z-xqCc_N@-|a()Iu>!^_NStO-QobL{+Tqk|N5J89gh$_B3W*mzvcw^`$VbSz+d< zc?UBa4tS||yk$K`XiJD^0gw5zaNB!&kJlpnYI z&S8zyG)Mo^Na_nOD;B+WtFAG}sIhcZkpN;_jdf~wd;U_Uy@S$atT0TFQEAMs z{4pIiDZ=o6d8|B`k1@ryJ-j32#`F$#T;G2*qA0-^6Ttr^i0I?iPVKgX#ye`(rE`#N zsQ)4QU^pgW8v1Zf_5kHtQb)4jpWVZ{wxTuC71bn4Q~1FcWv%P@$ZCL<@jLOOrS^p* z^BEK{Yggnv85<>87fa?kMMb`{&`$1VR+F1ly^>*~J1u7{*X9VFMd<}hCw&istTbw^ z)+H)Bcca->0905YIu{=8iX*l6wmP$xGX5cZL#WL-AsG83R|#&15@83Epld{JWw2$& zOvnLxQH2^UE#P4;v-nj~46P-slS%A?OhH3cpn~&hdfQStRQfZ+>(+l(IQtH4Tm)_d%RW;8)4&c=B(yR(_;9Uud6R*xqE#MixzdmiGG*az@9Bul#c1cNB8XKS#CE-ebvm^duie*w1 z(OMK zqn*sXz<&JDUFqjA9a z)Xkc-)Q1<>xcm zxBT$@_w@!Ibo%uA3Z%TBxQh$7K8GhsSDP9EfIw#{GFOcs=gd<%mC}+Skd5 zKlt_EXGb7`{x7g=m9EOkkN`o^EM`AxuDI}#AND2BLr?}i-!(2n9WDg&s-;C#s#`L- z7&=kFSu7nlu~^!OtaDRHvO6kC&s50mJiR1J-9}^1t=eUPC|8dU%up;jMOyg0B?+3smM3VlX^72+%=@BWc4dnA z%n=!BM82dtFg*uS8dskpL9z_2lRhq`%i}h>Jlv?&H7@^6dJz7Hq&IPHYg}pBN~~}k z0)=rw?f^oH5i7o&M2+S>zjwTN!_V#Yd=(r^fA|=0DGP6-_(SHhsN+tSLD1IuGGSy@ zcd~G=x&zS%9*z?!L@_FcB0eVGK2VS*7W}rH$Mh8GRo&xmSvIN@dJ}UkyL@fw)f3sd zcpSYUt}P9A)S|IK3eFL5j4@#~N-ZfsHG(GeoQLhQ&{;iNB)PCHiKTbao+e2r^N9(0 zH_ye&5+hP7Vi_?tkUN1KKk_c}hf-Q0w3D6=&}_T)-E4i1YU;wrfak6HL?rjEN4zoi zgdEuv(lH;AUaUBCY>Eh*eWTa$9T%GKjxwV?r6t#X820hhkP(bk9*UfG@28RX9ZpmK z(Sz_r!k?UwgBUuLAZXP6EarW3q2HGlq;@4N2wESvWUGB&82XE%zFKqOqoGGccp z;;m6TIjek?)lkMJL(=PSv=s8r=Bk9dwwGFFh2g_Hs}Rj62>Ngec#&8#GrTxqyw_eox5)IDdxs!=b-E=DWWgS zquc4~g{u;KzaEvetrCXVLREHpM9I5F=qldp(Xx=m8R#Lj6E8Sd;mT{5tGKH78lBjM zOpLGraCC{f0SB?I!noE95MI<1lkfEMw5Ac)IP@DA%RZybRJn#J*x@Ucy*3wPW5J8o zglen@mKG{^YwL=|uX>=67Y?suV7-k+G!*9VxDI;lT3Ko!U!e}P8a%#-@cGbkwVfzf z6gKt|mF{Y2YCq|yp-`}J*-hHNug+UY&LoptvKt!q4Rt%n7RR&SG!t({lSFP|AUKiU_Nm|oLlJYYc=GLC@Q3! zXH-`Tz*)Oe!|U{7$HZ%&TZBEM>$U|pbwYD9bB-uWZk1Taxv_Aul@Yv5kAa3~#%uuM zY-YXo*>!w~Xh?Ro6!;@@D-v$iZVcgsnJF4T#Z#hEQ?o7#GTK!kSGxetkt96FuIq0S z-ujbZ3T>Iz>(p7=HMQ)t=~A~&e1=$)6A9Og<;KNjg&wH5><_()*FL+7mucHpQg*bh zbl%M9A^K@$D#_}=SUg|pxmwtY`Ph2#-}sMG-1v35c9BcgeR`EkbQyC9RNa+iAvt_^ zUC+I>-0)#%2(v$YrJ^Q(Oy>A3c0r%>BN!p%xJa2qEI1q;L_o^GT`_zE8Of-gsKM0D zG!&+0LyKjlWW?*wmA|hEEifH0;e8_olnNXK-hP)f4%#MNHFfGWDHLiS+TfxJdj{@#=ZGjdrU7H#%UP=Ur z)J0#{kfU-Y#6wvZCl>>BRaTyRv#8T9VntBkc5X_qs!R}GVZWK{Ax^%RCz92$;7ZpH6~##5*^Fthd2y}R1;NVoC(4r=QQ;*dx=8n*9}*^I&gAOfzyLdMP9QZ+ zOXHnJ&oeb%D*1To@Z}6H56F=($+F}u^A`U}cCXzx?fz(lXQWW$=#lFkn`tysIy;nA)ON#{iF>=V?Z@-^-@;=|3^wSnQsbk&D@j9g zWRzA}tqcl5kTA2uaQDU)hCN^uEj;MC7mr>+HVH20wvk zN1$6Jo)-u23TJ+m5%q6ljtwjE!zd_<@4!3l^BnAQjD8QgzsP+z+!;~ ze%di#@xIprJ%G{6-APMx#t4!tGi2q1@Sn$|c4OCo99HB)xZ|zG9$D-^Jmq@1q85AO z>@rS&L#-W)8CII)Ti$z%+yorH|tGTkcw&0_SoKoDhWs5Yg)VQ1k)k_6Xd;6Gg#AL7O#TA3l?kg4llL@ z(u5pGN}f>D6dl3uNEAS6C-=fBS5(S}Yc5)u<6XN7>dc}CFhgNTkdhLD0q{B>>XbgU z34z921K3*w2xL7%n=#7ekS!~a+6NJ}U$C@)fR#T-$|ry2n<2Glv=zSsnSYLzUm&$_ zf|Xx{v|osoUkZ%+qD^qzk=4x|r}#u&KJwKNgCZ4!QsC79gH!C-D3!cKuPB`K+4!^R zJIvx|75KyHnm6YwTdc3HWq;ishwl8d+`wK6P1)_kxo@_B`0O+O&{BRF`_N5H@+lt~y6NIMBrm(~9U1m*!KrC?E2!OuAV5ec9aRkecYQdB1(8bCs1wcVretVa z{BOKxJnBPts(VPP*S|&a0>N)LhPm%WL6SP<)lbS3___LhA6L#b@0vpKBOae0wJWhZ zoIA2KHQ+YTK z?2dB^+VZ*{bx{LPKYS==&SLqvdmMUs9{Gcp+CR6o?H}xnRETp2cAJiicrKMRy*JyL z&T$6UwSCQJ|NcCxdcZDg_}YkoZDTht@EliQH#eNjJF{QY^cB=g;Ng?mh?&%S*PkA} zP2$O#ZYQddYMuGYG>v#{R9)Nb5n=DV(?WTr#9rl>MdTh8d}a#mR4~B87y3~)6FuTc zBR<>j$1{8~pI7{qYwGdI@_K;6+U(34G`&mW0YflyWLcoF2iFs>uqTu48+xI!;)gc9 zYyHgr@zVJW!S||lvz^6^@i!6MM*@A1ODpnfRPMP4>;}}b|86c4qZnQw8KFy5`e$m* zbAol@RhfI3#kPMy*q^^YNJhibxx0!>euch=u{OBRItUE-LY#D@hJGkLM|h%UTTkieTJ&x|Zom#|#@!1djT zp$+laol1_S$Rq5MxTPABQ#>W+mm_LZ0J!w!d{9;RE*@~5Dfa}yzB)D75C^@Og&PRd z@n_0J5IuJMyqkHTfL)zFO1z*N^Vq^=?l)HTdAuG5X^%>mA$Ka1sZ|}IITZV|SWT~n z2Uk+0tD@OS1;sW!W-#x$F)N$beYarZR4xXNK4C`#R8<_3@Ml>B3_ zEawn5knGZrnJlj0BYAugEK|pWk*2bV)$$_GKz1QAKiYNd=-N%Z#GO&^19$)5C+37s zB%7b#j;8<+ARzhw0N(xYiTPhpH(^?SK=8+KFL*I(CE=%l2jfiu{>G0uWCl`ksUlJM zC)-Y{DATm$rEOy*L~?u|Ac8T@nsQ#Cw4Bw9ldHGVx0i=c;QW|h)Y2=hN<*S9GaFY}Z)ymyboKkV8|gcjKcXuEP=$#SA8b<9?H-3Z2S) z3V8M_UQ*U1%7hx7ne-{1W(FL)rit3kH&ELr@+aup4G_xxlK& znu1X&FMmS(Pxzs22Aaz9P3oJ!Nxjy8;I9<5cm3Wz_+LJe@Q=eE>w=P)V^1ck<>3L3JMo$p+DXHe=!Ud`l-^yvaj{(Eax~x-@978(^4<`CeV}LFisRxrN zONdzeZ*zwzQQYr$Ohu~=^S%o71o%4&0Tv@Mo(Ieu2k5y=h!Uzg3jelK3cr=7)roQL z%Wk1%DWRw2_6d{~&}|+U+ASTV(mi$iq)<2iB~uCa0v3^ZjN>`NTZcXG=^w=S2dRf! zRfA~T$oeO#H=O5%nIJ*sdK)FAX))=0;Q=n2w|zS8cTk&(Zs+pt=&V!tY{ z?+K~rI*;9{JiJ~=9{iFX8=&!6?6XGKA3=k)-;NCApz8{0W1}^iSRWVC#De=K^*kQS zectF>BarB=234V^cganmlOw+WKdI+}-6Biq7{|u*r^NFk#jDAcGAREisc-!z^(zA; zoCZ56K2|#{KDCA?(ZcYeus^qD;0M13Q_S(E`9Mm2jyAB#u6`c7RX}{FLFkwJRcH^* ztsc}BNa^cQ36^`hn_F9YZ0B{l*X=RaM%k3J7QcSfdipTDLmxkLgvi?8qU;=bWMOwN zS~XSKB2?+u-Zz=3+q4|AZKI-DIkQ#FsX|*dhdz8|L$3E1qcrj--QANd7M(=NJ?9#g z9<{z`gO67!wsHBC-D+V@!eba3V$5$&n}|w?wx91;lteKmi?uc?d7IF}k*l>OVGVUR z+Nhp(j@G%tE=uGGtKrf%wMx*Z9`(-`5d z8_amoFl$O={THRDYe`~>!u)Saud(JYwoBb=((}s?_MenqMRP$uOhr@H3P#4SZw{)q z?XUQ*WNA^0DNM~@<5gSQqbqpQx&q^TC13EARyo51yiGNvje*mobvvU;hGdz~&{WxT zio?X@Dhr!^ZsW$;m_eVI#oLo1;@I_!xXEfa#v;99HP#*e0MC! zl~4i|F7tE^2Bl3o8l?>gf!QH~z8E*`3u|{hF)uaR3TJty{?f;mhNiRoY?I$pZrlXbLdn3%?D>FxaU8uxk<)NIO2?R02p3~XJPdeI4up5C zO?-{B3R5%_itp9&Iu?Z@L0!p34jm?kEVU{SzYDZ}H$5@V;o6hfcHyJA~BBy>X0hI$S-J1Rh7;Jr>I zxj{qIUXs#0J0j&jAe5`p3wm)sNc-*uEptld(Y3_gN>L3m*0o?r*e-0yYMkH-~gl=h+^W^(#f7iHZ`*p;VVEII-L`ClYejr#v1sX$sW1INfz z=`FRffvKdCLZn-?wQWR@ewa3K$^E~Y>iU1vRCDYlc(5$;>&{!g)1C~|&Rchn?=RES zy&x+Q_VB#cg?J?7;t5QSaK=nF!;YX7vZ9X4e%Z{|U4irDve`BRuugunLs%=yW9)-M zwa^2iaC_vp>5TPyne7ndS&XK;{HXh-g}~FII6fx;6X8J_Cn&gNIUC`B*dMe&=dp=@ z3klQ_F&#s`H0w3yKqt7p8_hnAodo4gck1&`;&nS6^*0s~xttb5RC(~$%aA3U-9?eX zmu8>$)jk6+L+~nq`GR-qqMO2>h)<)qO+Cq2E-O4?_;fn)I?%Ya61GmPvnaasp0%fd zVh@bQG7VqcISUU~3N53Ee5)xiEX$$-*BSTofpk2;ZD(z?D!H!sB&#@G`-{txTWN4w z)}&xIihRrNYwk6aQgM`sPZN54dR{;?qf~5&Ey)#bh|MM&LU42kf-P};8KT*EsF6Hz4`fG)^4mE z5+7FE;eHzE66=XaYjh&k#~oQf=p7DIeV(uw*NuTu==%Lt|Ij*t%dF>ss?4-0H%-aH zBn-!Pk;Qtm+{#($K3K2vQ}ln>AMSRX$&@LaBBTVq?GHO%5#RO)vRgkg!yVTy!)-ZO z`<=v)HpkIE9`vo>-q`9tT-{d@sRx0(wLAz2FrQ}zB)$a_yUlXFR|L#b{XsefFd4NU zF~wNWa>KU%j|lKB%x4s=YZa_=Tl_xB#VUXp(~+F=O-6`>W8c8h^T0sbbELcfmY}1> zj_e9_X@>z~# zqAaqbP^;P=qWyK$DvL}R4rPp32q>#SDXaI|*lqE!!-vA$x?pasJN`k6}^*Efi z|In!pAgy;9sW-B=K>Uu4b&cs)OH=EaFd(88g~x9D^Dltm6H;3_;b%A*eeXcR1lwWq zFcI#dNL6SVBn_tvVZhKK6(q=9doBicIPfmwTDX*@inq*m?3==;zy z31292c&w2l3G1x!P|Ve#vb6>gB(x(Y9p{*+xRe5f2-p*C?49ddwzPL~>KZOI0zQdS zW+K8Iad9!fv>_Q^|6+I|f=U@pMV@ls~YJ6{qMlcQQ(v+Yw5&%8svLVWS7MJkq1v?WuIQq zX_m=(i(yzz=&Y7J(-c@Z16*UD$-IClXjy7zgazds6646uZ^OUr4wf{L7dAzO1XOE= zLl~vl@k296t+TJ@Z~n1A0Y2{%QqGV@|DHr(Ys27={=Kud@|_!|{2v(oB@Io!Lqlcl z&CUONOQ}Z9R%Ju=d-JQ&hAz;$tU^wfJ%ae{`QS}_#cPykiUZ?KM!1@x38TLDhm-;D5Ikj=W3 zYpz%OiZ{z-6Ea-5Sk!gT-omK-zVmIy#*-m6pV*tmggdN7@3J5{{n9+|vOks6k;Is< zFE<;iOs{>55rGTOa*fw2R8A0EZ%H(Y^VnHJ1toqg7A6v$f{_y9vTAE`Ge6oz)u_X~_$dTX#bPHH42igg{X08VJz zA-RyKyW8<0>FyGO0Bf5kKt=9B0hZ7i?m}uKur7@s>u*&hChcKWTD*?-f{24K-_`LN zJ1_Pki=%VX-I86KuQK1FakJQh*OAdVu!#rg0yGy%$J1yya(PVkKX>(orw*qo(8mm){4R~5{|N? zWE)FFC%$D5Hsk8wVthJx*~c5iIrR>o+#)gc0P6daS$qJTZUVK^oWT!anG11v+s-kmpB8J?KKlf$&P@<&zBdPS4KoCOEEW~nNBveTC)K13F7Guf=I!)+XcNXfIYAp zbYXj{ZD?pTM(T=ZQZ8oY?qPu%xh+ZO|R&` z%pI^GhCgOoR?)agE$o%Xh4O_bl)goXIN%jmcFzUPZk%fmLi?(rK2gbl=tfOezaUkg z8le~nIHrK~(EU$G4C~G>G@kJ2J_-_|0G`T8GV4{eaK=zEu%2RjJo2X8RiY$lO|m|Y zcH#nKP$4i+5^8||mf{)p#%S6l2+gt)8dY_i_9X{1ziMbS+L2Hjf*X5Xt20Sh`I^WI|~GJbRj3+H6Ft@VO{H-Z$iUG=UmocYZHvWr0} zAnz{WRr*IXm~djh3~+%|*ceh~CKhPmvg`|WQ#IX+;!bh)J!LeC$Dq)|TG03&r5q>~ z`n%d3YnR)+aSGB&{R)h|kgaPvVdSisOVH^EVO8YgDxqp^mp~KuB8PAnpSoakm6&*B zC?ruU&%^r(6`{+y!5;Di3;oJ?iM{#(Tl#1KMt0`eTT&JaqSH~H6NI{pOcKEniJ{fM z0@-s|d1fDd3;KAJnG*i0sjAH9zoD^X?bUL>e={%^ss8+t`yW{9{!RZLK-SoonT31bd+R-Gn+Y)_n&Ytk9&E)mx;AyJY574N4iavv z;Q9iJvHFkl>F26#kb;T8r^jOB0mH911~s#-6I;zQOQko){(KnIh*4-XpVd6wv@q=4 zb-#3dbh+J6*zkNf;j549ESu4EmCb2r*Pq=u+RyD=Jiv^8_0WaP;ebjiTiSjN< z5bIev?Qu$%_$gS>ch@Lzq1C&mO-Trnas~iRab1l{&@6XAvtX|RRl2;ZRWaBaIAR?S z9!A2hUThU7!I^m>zigySaO`=81k=TlRpNU2(@5C9~qjos;K4 zKr{O)88vfno1ob)d(aScsbl6DjcJxVg8wf9WnvH6cD%}7DP#DG-Dx2M`+09l;;>G! zgju_IAp;Z9;#KIFma2#cP0LI#_EmC>ZpvbrS{?@$h2Vc)U>W^=VbemO` zWILmF$(?%^%dK#!AI&mr?gDKWS%P(xN2)k}9bWP9wW|K8$xzj9?G?nZu(1AIAqjKcj_!QRk~kCxE^Dz3lw~L9Y=@JF z4~~M=OVYAZGL1ffg$_Bup>OGFzY!Tl`nsNwuobb3y@Dmo!kSPlsKFc`PA}+yKmiy9 zQnb)fu#+>V9%xk}mOgz;l(51coQDW2+BQNRN5Pz`H~?`PI=J;r4nVLyMJl+C(aR9Q z9^%4qf5gEa-VQcJ>q>NRz2NG{~-_=2H4K>Z7e@frxC@# zoUJ)i;AW>ODC$LcP_K>>i}X`;^kp5w7qTg0-Jrj+VMx-y3C@O7X{ZoB#~CsOgwSCT z1q}T#I0@5oFk;Kvt|{nY_&lp-BzPfgQ)+xmU2#%L?w~%s^Hkd}WqTVFa-*0*1zGv= z?G*L8(^OFKvOiWuVj_SH#ld0_>?m|pY;yCkeEVoxbase6e+sFIgR?{@!L3lEK=kAi zkY=;i;0M4?j3yi{wMokPbb4q^N0Ri+H!JRtiEVUTNzIFd5oEtresD+)jo?OYCs$}x zi6;mRizsz|OgZ&yRH3fXg)awV4`jUTI8d=6K|2yPZOpTk({JKerYOxrIne6_QA;aB z1=Poq;GM#Q8arKKQw1*soXa^(a4a$a89E^gm zO+6f$hzE4_%Vhmj`gv^j+( zgBNr(hRFlHu-5wJ6tL=lf&!Dm%k=+pr$f9lEc+g4eHq}V51z_33E4`xMg-Ovq>h7`sdn!)6 zvJ9JWl_5mly!hjB;AY$&F%p{qml$fkM5YG35JNoi40KIX9)z~zRlm-TCsFQLzFO7x zeszwvK0kJNZlIF8^Tw0#*fAqPRN2{Zt3f);S!B+3R)hCD8z+3=GMs9Z=MF+wyorb+ z_P?(Jm#NBn#ag7+XE{Z0Pd5AN=+EK%FzxBnc!Q5>LE9YpYDGL<2e>(mdDex@DF(4Q z!2-EClz?>ig+_lw^;*vLIMoh_58RS)`%`zQC`L|s&{s_O&Jc~T3vAiK7c*M`(i1|n z@Y)5UGp)jUR=-sy3f1fyGt3ttAYZ?TOs5Timg%Zd$3DM{hs_*&U1ONeC6#A#s~s)s zbh?>SaZnhXp_zj=I#nug0i+ymz%iFS*^hyF&^mm>z}nZLcYx;E=FUrnxD50VqXnP) z2OA2i*?=iov`GYUK}_1{h_24xfZeNuyaipMZa6lpQ7%*@mJ{8;qq;Gm%(?i8M1dl# zSs1E2oQCg>$I6IQRxY5%_>kintuv=)?o~aY=HLb66W}{{7>E9K;1jMmTcN%c`)z## z5kf@CIw_d{fY8%iU@%sxdUyrH)7NJ9wE)B?(#DY8VLq-}mGCbZ5_WodIu%&yI`k0$ zx?tK|>b)3AR|T%=t&}GN_*WW`WO0LuG9=osfSjDZdn$Kk?gXkk>xTB!=)JnnS za!k41nGsX&E4ksGd5^GO_((TOy3iCelISxEWM>M846)pfLsN=!lQ>}O*la;9nlm68Xo8cAg8?)0;Kc3+f z^id#yXGTX?z0uEDm2OPNW}RZ zTUQMg1EKdZWpXbnY({vkawBkKsp_~=pK-)^g4bGS*shv(tZz zjzk7W(=OF%@f0+NPuV{L!-@M~LCZn573LzHBo5}UD$-#0DmYrX6-`BQsM>_93ueQ~ z%3Ka*ok+7d_g`nLG+h8Wc+dGDrWjeFh+BezfKsasV>Ss~rmhi1ZSU#da^84DKg0#i zM+-?)Qz~SgW#q~y7!rLIiFU=^>q2=CAz^Q66A5lb>2Cv5`vgfxf~90fi3oK;XNRH1 zeI}pjjfFO+jd!Pwj%AdyS!CVPAuA4pu&!MXEmnYwP@?Aob4qlM2dp0aqfa1aUlujU zn@@fF@qDDW>2%A=SK9aLlU`Mtb~FJmK(W21u9`NdApxlHI{RZ}j$vFR*$D(ZQ% zCP3f1$mwpyzDmk162AL(ugTy@4FAH@)aAM}6zELG;FM#sB>Ov38*S~m;%B{uf<{R* z%@V^gAv!34p%&B2DtH-x<@-r5?FG>YEGSWGNn>I`VI68nf9tma+X4a8dNf)yT*ltd zkI_(q{m8a3_;QJ|z(Fwi`xE-LgDLkk6?|f00gxDSrSkLDl_1OBn{xrtQsq{=I|gia~zK@7N`bGBGi(ORe!F#Y=+B3$S0zMW+tMvJ=y6~&7a$5E%^)RAox z_8++s+rc8W7J0SL_;ne@?{COI$7?;9w%Zw5Z>dK<(c6({E@&V2ru^OR^!7^-<6lgm z;i__BcFkD-Jo3H6ib;Lxz5h!p9q0kd1ig{a$R<&-*0pv7VlsNVW;COJ zOwnI6NXEvi>6xUQkamPT(y~tG76?{LE(Y&mmQDgJUN%l{!Y(csU~^yRW1d6G?f)AQ zpeQ5&+egbJpQ?@<5B<-8DR36}n8nX3&m%)XBiQ;kDjgJ&CS@M#QBIe-i2EHiQ2wtc zKzp1uJ`1fjFAJaq*=_S2l_Q1greKmI#+EetR-WNOKF1k#yN2A0-KIUvRz#dAA(*Ch z=L~9z8lY}oyNeyfsudSMlWs>dQcOF6Vh)O9)L))I#n!&h*78&AeA&9V}T*=80t8K*9+ zIQUC=WFCaZiGK4wJHa{?|5Sk~y742=lJH@c#fgHxWt^Y#x&KQ^jqNQtU10r4i)>=` zfHA0vZ4^13gES*>osc?aEX;}{5&PzW2k-)(1!Iev?!#m=j>aSqkk0;R01k(JwrmvY z`q!dAy{D;|PDIdVW3!;KSBPfLe<*G5!O=K<+Pun)Jd-9+c*dJt8nzhg~Cd?cNM#&4varBiG=txrATUzHRr?5RsoPf9TbqG zq@;;AzN?C9sx541OTB@VL*tUhs>*Ea3M+4!j(tl<`O55O)_nad+wLAJHQQZ^VFG#~ z{$zLzlT}jU%ZLa|y$fVR>-pafo%9KG(IKAViqY4@R|WeDUH5v5~-9Q=ggZ_{5DQsg(|^EKcYwBY03I~{W$^?h{bY9Y_T=*9@BBHjIg>ec z(G<67kiNOqEG!I~>>!QPo*^2AxW=c_&dZOg_v2J!YY zd8qJ2{|P*~tvQ4G=E^yW!drNNJJ z=lwBzB=iFN_C#4WEL;GnE~FXg;Bur$nNxUV)W3;%Lsszp$V6!QcXnhSUy) zeL;G40;b913I%;pAiBv-3I%UbK#IZT#`LS1+0XEMAQfKn#5w$+%nu5|DfSjhO3}$Z z3MbFPDFhIT2m#4Ur~#j<4hr>(pLJA0qAIAFTj4{>=VRW|d&E?nQ9N%aMKCbeoKTFU z%oA2Y_%$(StZPB-9%0Oy#O~zV=vPf1`6F5@y14Q_w9wv{mNsUJGc@)j7JLzvPw)P! z9$sSnR{EZ;M(xbUKcSy>g1<^xhie4Bk>I%+=7tq?oMW4l99=b{=l>+ES`X)c(T}S- zzOlA}Z0ZzX0TLOz?Q}M~-vH3y6l-}8FiNhzSuYr%o4LHkBSE0RMn&mFpiJ^nh6Pd!W67^8|&&_Bdh6 zofP_kiUqinH?>52^rUWV%k=cb)|xz7K<@T+z7J}Q_;N(6Da6wjlIsqI?+mU!XmW&b znM2){xo1xzyAD~>ip(+e0Nh@I{9yOpA1RbBGQe_i8o1HLnRV=_2L>FH_5Fs-J`EB~ z8K56gY;z=4dB>E09yTXT8z~a5A=QYTd$C)t64vjLrbft^^R>uGlN!j6l#Lo>e==??dBvmn?ky8yjHy`8X515&qWrZ5p9GIC+ zk~xxQUz#T0tRr?g&{^0iecAE1@Y7A~x}SOYZ_OqMG7^{ySpW;Z>YfL~G}?iTk< z08oBEzIwkM<|Fp`&UP9UI9T;qiSl(eGBUg}JQG~Oj{$>a0i54%L0qJ%GdXjiK#U=S z$-$Y?pm;%;8tW*^+sO>Qln3>X{<9Ynz-@MKU+fRq%qO#xZu4$n-Uq4q2%N=2z$!> z4U9`k^4!i!s3jjI(veanbv8M8%2K=Gx6LyrwzLc}V0_)~-)!6TT|Z42lfSSmP?!2@6EUYipmM65n>dCrshqD-Xc-zoRT`-Zzu${2Axw$f z;7+Lt&Rsyaa}egnrFObA4#$jX+%kHR%wWfFHJw>E2!WN6U}5-x25_ZHmjGwfyQx(l z>5uy%A&>Yhl`ce47K55E`D~N0N3EyKXLvIH#Twoj&hk~OTS59NH<*tTGTWc;BkzrP zxe@A6uUm%}94<^;q`_}h;V*EK{O}992|At%6jd4>4H_p}g%i&Rip7yoRj%6~f!cgL zpxRWtYsR_R)dKD~^+>T&53-Xe_k_8rcceVLen3ya>KBaQF^$sGElFA zP>2)w*F(?x>N>KTB#}O34`I!KakyAW6+MLYCOKiX!9*27q_E7^rae08Xr-jZvnFWt z_w979)OtfbU7mWYUU2EMCOJF3VUmc=Q4pxS$bw9nPyzS26xGIh09iq>d+jy9qGZ^N zN?{Sy3i)PH+#8}E>0tX_HhW*g$);(_9@Ng={u1+0A}0Y98fXx9f5cayk=+I54U=3m z=I#vh4g4io6fp6_RuJE!)a}m+SL?NS9_>nr?7$Oh4-~Y57OIu!eaav#ABT&yzGJc? zx%9Skyi3IwOzTZ9(>;PFOy2Zn2+6KiKus3%vaEzT)OSwZgkQ&1%r176BT%-wz!u=8 zccA5;w6fk=sqUJ@2N9))>4?a-zEsWn z_ND{qSJZTFgolhr!^1it>d$_Xrj&N6EH&o3b zFyYJvY~#jSh4^IY28JS*LzlViSKhuAih0}csTAfDst^JXNsK-w;AcMOAW%VeU@%!B zEfhwIq2kJ!8-m0&V%+1cGr`=fW zTr!p!tK-TTH$+#cl(_E<^vz8aRk?9h(3FM9ouPEI_JCyJlFf|Q*GB@z2$8y)J>D7% zBt1I`eL@#~L1Te1u~R|3?*?ngfCe?6>l~dDWBss_@>wkW1ea;fp0$Yok-7rm;JIy^;`=Z z3|~$!EL>M_7dtxMfV0szOqeTkfqYm3sLOQzbExQa#e0F#q|)Ir3h3JNw(23#2~4a6 z#|`Y$eTeeK$jJ3fwN8V!+9J>g=*{2^A_Ia@#7>+bDwg9Gw~mzoG`H_ri3Ua%5S|nW-Au;c`3L+DSvDmT7 zoz-pa&mzsF*0r6P|9s?IB}qiztYI@WF1R(&nwm@`Z`o+q2Rzb|knCeHmRyHPQmhwQ z?{L)Q619mcBd5&kZ;d6H@jpt(@vh#XsdoDGcj|^uendZOHeUK`n)D_ES$eZ_0-p;G z<(k0(8pExwJ?ilgR0{UCt@$R5C5Jras?nPaYi4jREv=AztG~P6ra9ZaYhi{bBIel# zkvAxm4(^8I>Bd<~DbG7eCC|)F23jStqnHzx?@&kE_ob`isSi4owCZhqotzy)RO342 zUbgQaR!o#N=nGqTjnaT7Z(W4Wt9GoGS))WFHFE#$^kddsv>od;>dfC$0<76{0?eor z8o&Tw$1d6gUS^d21b1cX7>~|qayvu!phgzW$dZzLMYF2}8zmSp|ZWL0O$>D67MU~}_;mXf^-^7~@d z-Ie%tocv19bl*Dta%X#6UZwl)e!iR=3{kjR))*?b-v;GT#GLb$mQ;A11=gUHw{_0I z%?GKFFhm(@&c2kvn>kxqUeb{d+6Sk_aMz|>sD8maAUmywNAp3a@7l>M_3J`Ks2m!| zrN-w8yA;-t<7H>7e=fDxgUuSQpDx!C zKVj{PhbqVLC^~ziw@|~0!YJ;qo#lrMenyxsv^ma=wC9M<3Oykn&kidKr$gGStDrET zvcn9nNT6#@9fs3n$4e5*E(k=F^qFJYq8_)2k4*fLU5`b%38YBV8>|q`{NphWox(-M zCE*jZm?@Tp49P#a!92c5*R$#?H|@cP>!O-RnY#W=RTl&&%%y61;h7O4OP7qxE~YDU zlT|8mBrV!Pl$4H7@+c5sZ0KjRgnU5ZLrt@;A;>GNsLwll4g%ZYeiT1(!zrT8vluvWyq_F&)75J7h#_(l0C_X+katZ7Zi>i=H!Td z+u#eb$<%Wu`Y}U$5bQxG0oU%Kr!Na1c1Gu5+~_B~B0ey}W1TzwCQy#~W&Wzrppb~f ze$!LVIl;)qWDkBHWZmn2zp{QxbF^)HkPZMfWjAOqR1z%cved#uae8P){$CARdDlt@Is7L(eMt>0RdFflTgJty>w3h%Ld_D^64KlSK#TexxF~;vsDRBp zCBTZvOF{eTd3il@?vif4{^p8le%Kr;lp5o>G9!EvW%?kKoFu(Lyj?znov}WB!sIVl zN^;a^yP|wM9HjW}1HSvX>*Lz~y6xoCq^(5z^I>7;FROyLC4Q8PWM_mA{oTx~5t_W~ z%wW&1PScak`)d;S_R2_&)4k33?agb9{AV*_55rx~YmNNp_8`vf&adpvC^hP5ZuuwO z^HWUYhXvASI8W~{oEIXRyk+RjAP^ay9GN_b@;Uw&WU=bvPUA^i%s`Y57Jerw) zc_jTDX~9@PV#X>lNh-bYa5}y8Z~_rqog(>Yj3lj5ex~6lA%e-op*-7Kthkjl-4P<1 z4ml3JVL4gJxEZnW#95I%lVmxO?kFR|gCcx`5xWL`fucDi*!>^-U_6s_)liqm14@PI zSP)QB*Gc0rJBPuXVJtzOEhA!OOc3kbscpNm z^AJ{fm$K|48Vch}N$0lpm2JLNpK}Klas+4=m9z!>?|eRs6wFTCLxq0RT;%+UnglLV z#L{{`^UC5F#pRAf;Wx<>F|vHe)TTZ_6B}$fB!MhCsVPuEjw4ArHYD16;30TWygzJ} z-BcufU%H6+>ZEz6NSU zaV8a+pAt$Gdsq-0N7|x*^oora)CK7!-eS)%%!%%ye1u;VXuy)cJ8Ov&fL_+;K#G(zyxmeyf0b^+R&2bbvOZ>b$ z*99gq!kD;m|8~>q8QHqvf<496=)fAse>6XwUzsu*)#ch!#3;k-rg#JwZB7R54qXYV zW$*}zDs@>BY?cN+W-5Q{1>NW8>yy zKPJ)bu)^Z&eraC+mc4bS{j2dgm1v7oT*@VmmMme*vh7j9WH-|w%j7r99=-E)CCK@F zCc^s37Qp*?vy5$>SkZ`whBVZ(TJ878IYD#gp#%l$wXUknOyxD3Ovm6;l%0938LDXl zQK9Q;=^rM4(~cnXk}f&XkA9fiD28;z=EcbDy)Ld$r}hnHI5KrMlI)QQ;hj|yE_Tbv zN3Iuh6YCN1NSZd!;4__Fla6xx-Kb#?;%J?m`eLi9g}~zSkLuVW4ErnF0>u>~!{#xUtZE;`t_qlYpM-Wid= zD{R#w9#ymo@Uhy(IxKJm7TCSy`oD0qf_*iqJ=F@buVY4Yz=5mS>Mh*?XI<}>SlGz= zXdl{!sD*6g@*E9)USADIY;C&C#lC9n5iZ0fq9ANiD%M`6?%(2~&lhyxw;Zp;*kUg@ z>}+oZlOfu$7}u+$)RZfiG$}rXFG^jo$k#eRJYp7nkgnSc?_lCeh=@!tj;J-RxmCqk z2Dg0K2t20GHxJH-DVvC?YE6x1f7S}enD1$Cnuw0ZJA+}=&&5NBsjm7{3$Rm#iBI-7 zwNCd}wNCYKwT|?cG(NF$nu>15Q`|=Ca<)~1iLeZ@XQnTUMivgtp|8B8Sk@AV5#xbL z#OGVnCy72VSd9^{S?-y8PR3J--$-v7?HP6I>rqtG6;8xc9FYrDHy;|*FU4CN9{4li z$X1K;IBq&OEm9s^9|Td{RL(UfqIL`WCi1xTdpn^%t%l*5Mj>@ zq<0G7<~AFP8kc#JBWd>U)$iI=vbH9PCMQ!y=P#X~u%+P0QuL^-qDQVGj2T1U+9#H< zPS&8j91KX_ByT~p?;vs+%?jQ$8r!muA0*9=nXt5`*$)tFQ z+&-ewiq?X9uO!h;9HVg`=&4ov+DbT*%tM#);u=1&M5?)ur@&kWmOjO=6+&Q@KCxVBtDbdBTvAovVm6FWr(CXB#jtd(D36jvzR8x9+=&PJbXTl9 zTt`_2H~INvD0dFxz4&756_6VQ_)2`EJyN!d$9gnMmll0_iV%BqGc1kP3P{&FU@AI`Xdvb_pHn7MQTG~m`<3IzWi+FU zt@Gkyld45XX7?{GzfS!sJPwDP3~Bk}{*zynvewE*<5W$?+CM09Pue0Zq^m>bS1cQ| zpeth4lCQ9wUL@HgC#RHJT>>&w;_B^X8*!ESR5C`Id%I0ltj*w0fl7nA%p?k}h4zUisHuBrmjj0?Osd zRF-S;J}-B5BUwb_TyQg06M6g{kAB5_ld7{_ri^iYuhB@NT9~y>b>Y4{CFHMN!tl;< zrA!Dk0)+Af4Y)?D1M4~+{}}FM5YW$XmtOB#hg3u zUwYg0vTEc>-sK-Xtm z>$>!GgYVHrx^qisg!AFme$`C%Ou3~USCFy(rP@oyaDp{S$NN9TA4%#^)|W)7&$$XQ zGG86er;4Q^KwKL5=Vqb#5{4gN?nV*KPob8#yjfk`GgmA;C)glqJl(q6DqZU=z+{C0 zkRw1{dksp}NQ&KF>P2=tDhCGF9u(Dny9bsidkCN1$SZt5S}_<8Uo3ZS4AyohJK-Cf zcfXZ~!q$nTH;T^FS+3Hx>t7M4vDQ{qYcbBpk-!kA8PmT^#Cj1z@rx` z%_h<4aF{<}sp+Ba8X#&94#IN$`5;?28d5i;bb4GL;df5w*KEay-Muc4_NKVR7>+j_d*;b`}E4QpG+;J+&(OZMdD{IdVhhm&bz-ieYCenUF zQJyLCaSuvhBd*Q4ek zjk|sK)|6SM_cGPm*c!;3l|E0cGdn;Hnv(k5UK$)&JdYfn^{nA!)Y@7bNNfad;y=b!`7gV6=5wN89jB?^YT) zh?9F#A0NL`e5x%<%_T#0963aW!xzcw8OrAH4WwU%Y=bK%zGm|ofJlMPc7mdSbW$lJ z-Uvv*p)g(dfi1Mx$xDq5%o0oPKOn1^5J$N%PPtG5$7^BBDV^Jm~Y5*i;XAC)thPS-gfc7j|>~fK&6O`ODwep_XCCP{&AGC(0+bX>g zakrmI{c8DSjVE)@ecR>uz`<0>QFrBl>&5^Ew7amkm9KUlC9MIBa-9L)Qx;? z-AIYKLz7Yw)r?rFm5eHfz*ay{eqq1NkE|R&Xx-Or-+df5i=ch-R2{A3MupGsa zw(P<{Qi%5T=QwlVZXwKfQ4h$tN37`Ud>6Ouyg}S!^U*ylOfR6N!w0%(Zefz2e>vy# zFvnk$e4a2nE?wh%*+vXb@)4Hgu5fq4^Ggdr}R5+i>vK+lNq2n)RbUnj{pCAgi z8smp9|8`$kA$)Rez6qvBa!}D>@}fhmikz%q0kP!6seZ#h4O1%;iV`o568kD1O)~j3 z9@Vi>lb+5rUZ2ptdC?N_#SlBM#T_iPmo>I{cSfQ;^LfZ>vo1WsyRr>;4kvNoIJ943 z?yJ&eS(HCqzkR*03I|TZ4vF4GzY+dUYjEZ?Bq90e;&WNZSnHbCWCp^g9m=DgOpP;K zx-ej&HH9j_y<~^irT^}ki|9m?Di)#K8XuoU;ePs@N5}NetdiSn z@VN9XW2hHZySaAC*q=OT)dguU>mGO#mhuv?R(>0oKNnirdbSpKJJbv ze6bXz3~grw0W3h6AvaA%ugGPDJRr~Jm*E78uQ%fJW)-tJLp`w2nuSJ)rt}wD0zC-% zi)ILnh|kNpgRUcd@pgVQF_wRk^%C;eEMK#G_^`?<_q=SK1eGoG4Q?u{-8i;vvC5ca zI|NGhUiWfHP!3k|7;W-a)X@R}EEJ6{AV= zSquHUXXayn%JRt=^swLUOvg&ZT`FkNCaw(bqz~hSsAJ%pArsNGC+!Z=vZny9>P{s9 zk%I3p1*U4tEe@)0F}HD*P>1;luk3cvlHo!38|K)Hrn*U-!-}Tl9sbUqV=ky@d?*x* zWr>8P(*{SKJ?UYYz5kq%02gzJRPd2V$Hz@3^8|QFPNJAj5siZ*n@;7h_6`!oL#Egz z^*Y{S#GAzn2PyI36%8ziwTjX968Nu);Vw&W3d6d$gcboU_cCezX7$?PTXpbtWQ;M% zw+3yficvd75VME+egY8Dl#O&3sxW|!V+hjT-X3voe0=e7fY3|!?8!)XTYI2)s;83cRw3`VJ*YlD95(JS4$O`+oW395 z5mJ!GYjKGWe+oK0uhj-g<{_`wpQN|jC~i#7fb>Jr9NwlEBeq+wTCRA;HnuoOulSy3 zNq9nTPo4BA-cA|WU}*51B$31|t7s0s7*6kIqrYuwp0L#B`*S)jxsX)$`SDYg5ace$ z<+=*ns=dJK$9F`IU}V4sg;EINm%)^VL#vMi^&NQrue+0Tgu0<29q_8aC6q-P{g&-L zviBbqra?l(c1@^V#j@o}U5s?q(TmBl@XLQw<>+8y9xMKO3|oHP&i@lW!{2HlZ)E6X zVD!J$4P^}(1U|&?;;s7T%3M-Yxw~N5T*oe*KXB?oDicC@s5m7XVrpv6D_7P1HT1+F zHeq*bgRe1>Ce2j-xIuxS8IH%3X%AT&nG6j(K3?D8yWq(3rUBdFNGVF1V*9k!#kNo@ zN}IBF3_-}7;&{;;WSf1Vn)9*4rQsdq!)WTmlq2Awp(Y^6Xo>zB$dFe;6L1WB;$p85 zorVY{1*hqo+K1glD^A7mlxj-;(hf0v445LXz!@qP{;3T@D>bK^2gmWVTow5#t!_N~wCfZPB*mhLSpa+YEBhQgVkD{VOMx%I|wAUVP2~kmJMEl9LJ>Ot&7JMY$ zr#rHkStUyLY2CH%nCi6alQ3*Z9f3Z(L_Hd&C+gG-CN+*;$qL#CdRs7Opz5+JBKgU6 zyd_F%lS_8yNv)sl^nF}mhS5S-HltEA(g0_ch%`doqW~sc2_49(pD{9|BLv{|ZVPQ- zF*XZM!bi`QE&nR5`v+wy^;*-zXrcbDu<2sy~uLo)8%{eO+;Ac zcUO7$=EVuBqiKuSW0g7P3pYX+B{}M2r90wcv&{;`@YDt>s~TFPv32%Sy{7W=X5k1& zN3BGzU?nV?Wrk%k{qsrYHzP`OK=jCTJhTNs8Qa60>uMDQc(MW_iTe^X%AYkr?Zpkw zH#9us!IWcjoZIAE)QikR5PLst4Y97>`LE+5Qd}9?>))UF8SKv=z5fA8!+-gSRm>bs ze@T@8C17YuS;GcH2=RNkC5mNZSd2#ZZ(ic;YFUPrU-G_Yu5-rnkR881{JOkuCw8XQ za!18edz6mr{`;SOFd!Pv`_Nd~aUp(Ag5Rxy3A1xLqjaXYP=CCM@k}P$&hO>m{PTX& z!t*B+P$jN+!VYrRlV`sbSEZ)3HB<%3P)Sl>ysyz84av(@N(i#1qBStnvrrG!4Oj(n zl76sQa^ukiwZH6$0NBn7fqaCmfImC<$DeH&HeYlU%uo^5mo zMR8BM%Q^K+>y0aTGmJfzj(L#S1s-T2QNy)*`l=WOCUu=RbFoH zF1vu|SmoNLjXUD(4rp20kAfiAxU!agdN!W1R9aQv6^j|aa(pl!q?s$A5R?O{S?Y+t zKg%x-Fl_I3T3W4FoP|py!>T7^mfHjoFloskNDot1u!T zKRk)sIc#pYrQb5 zj(Xb#FJ2lETVqP$+XICtyMQJb=#U18*yB%>$f;OK+8icU56rVdtyki(D()xji0L9T zk}alKI+>3vh@M+2SHwTW_$4*7Pm@6S=ND09{2qKverEo7QTlp6(=*V=mZBm_*3=(C zN({DyS|7josPE`*j5JdX>>d~<3<=?*v2Q5TF;x$zQ${cY3|7i4NKk3*UOaRj{1hHx z9&Ws~64X8u#ETQr1BlF#l+za8wY2BIpN#YiCryb}PuLX&}X5T0YJks8KGF zAC!P43D@{zJawpD`t019c0J9Z1>OJ7M*)6K0rvs@^M?!m&mW`zKZ=d4p1q@)qnVAh zn6;si>wlg-R4gnpRgk``xjGs+vYEqv`Jx<`c|%l}%q0LI zU_R2p){zit8oahPs7apqTtU-_A^;3Yn%G+O_bk5hGL;vhR7~s-F&;31AE*KeaqJ7w z9{%p;S_poJy}^=LW#hB`mHXygPCKtM;1US;UGH-*2YA{QJ67r!Akl?!IRs?LiP)*Fpn5#mJX;$ zEfR!;1VHQu7(5UJiNu7Sh8TMZ^-a0``6nv;EkEZa#f%RRKxX7E%f-xzzct6q(Z9K# zlis@2FNJN1rWf;8iD#-jUy;U}iH@;9j%77LDC*vkFdGg%Ix)~8max&7p|`@Dy%=cE z;4S5-zB)_BWJ>Vz=Z{Jwb$x@ON!T4Me$ zj(8I5**A}fWzFD{7Jj-qe3!DQM&n3qg59Q}FRw#_xvY>M1vmz=4@gs;n3UeP!rf?5 zi{DgT^OfVS5C)qR6LxX9`soWXasSTI>3tRP#Js^q7aul*rnri=yJX1LY__ih@!n)5 zY$v&iGKCd%d#Wwy$R{H>3?!P4=l)}4+mUW1UV9NEjAapH`CIV5Nk|gIN*&9LqXWeX zy&B|}N7-$3IM33}nI%@KMNa4HRLict-j=Y>8c0-R*pLy=6awy?Lea4S4)^4E1v&9s zmaKiatJJanh=1$ox3*63dw)gel?!Le+}PvLhE66d((iOjj_e?fts?IFGK4K+OCbXr z`!q&O*GxYtiNf-r6MPrkEvc}+&j*z3CpGe)qxCS!ng#Ne0p>=b>~ zB*!cDK5!nwgW#&l1c2-L-msNgCL>o6BdYak*&d`9{}B2uaZz*;g$OlI(VnE6#!$*F zs)Br+m{F$Et#YQpiH0>J!m=zAv$ia$9)$K|p}@?Fk{7P2QYNbA=WkMFFXUq*FUcXM zm*&WO%Ho^K$YUih@R^Di+HTRYmh|DS#Rm%t)SVG9OuEw4;Rs5fe_TphgCwZ7Lzk3y zsrqJWcq(1$TVf6&d5)oEtv;D#*v}l%hs@bx$X`*-h9qvJf{{@+>1^{;HjUz(>!k*0 zj`?ks26%i%Cu&h%WCK?fLhE-38>06Pa7d+HgExThRtP4T$}+I_%EfWFw7knVN!o$v zqPTN*^^L&-H>H~q3G!pLHLTo zo4{ofhFd-^Rg3?1l^oeZvZ>OUKBp=JmC=|$>CN+WE2|)dVj?DKPB2((Tg)?LvJ3tx(1$Usrtyn3vWE zq`GR_bh&H~&^#QjK;2e=AuDIEyQB-3nl8StoAef&%LTw$eB#+7Wi6J2{Wj=|@*t(e zqae_KV)t&L6v}v>$|t=lA*F0una*qeT_X_wW?`}$@wRacAR{KLt01Cu{TYvrk%c)k zJm8R=>~&INz2#Pz3$$>yEdpfi{-e^6YxA+q8LqGj+v z)jUSXf3_IGX=cb25Y*fd-V|zJV*s+j+bF5-@`H{PC~)gzh^*<8-y=m$R{S}7cfZ8 zz>sd-z|IZ%SMEo-WOtEF}p8#?*s?PC-gzm z=$rfUkRC|VE?JvL+C_8>kz@s>aZ5IyWCiEX@gkrr+i^QYd(pOFiQdX7PSSPX(G7d7 zY5t=D;tT!1w;M@rAX@A{+i5C4+i5EQqZPn^NeBP+hlUxESh=c^nFGrV7E#5TQ(EMrB~RXh3BVtfw@qv9TU< z*?d*8wXW=XY1ndk=HlAXJil|j&6NO|23jBHTlwnspB=T#_xI1~=Vxxy?T>>Y)n8Ht zD_UXzYxFNP-{bzBef;1*(Y}dCrt=?K>AvwKJ$Ab;o%{WCw==Yz5zoN@?~e%P@3TR0 z+7GB6mtD4(Dt})`eUz`QupNq*E@WT18+?I%L3G}JUy17&nU56z@8qlRR(dR7hdp2D z8-}k+039t#1a-k}*nh0Li=)DhW*vQ`%)$i_(>00t$VbET>ZiF3+4c=%Y}w@HI0s(_x-a1)RoZ z9*K;`7+{`uq5p~?vZoMd93-F&Rpq2lrG3KBV0 z4hk`}gh&Kf$WQN4e|{yRp^3Ab9AWa)V+T}3gB~F}M3U#xI%+dx+*mJvXg=B%)TmoV z3gz>9hg``H2E^6k9m%?jJ6ckotOzV=RM@=a6GV+IuNgXj3_IN$-iGFlKnzs0m~AlQ z`{LaxgB#jsW6vYN1U9+#&q8X-X6VrWk%!?a94kaCk2(t;Cal$Kf+ZaP2%22$b811V z{y<4xWPqf}Wl8`pR;v*u647FVfc-pJ$`?L=91hfNNnOH>`?Zg;z#8&^B7ej{9Um%+ zd`l81C-&R`3ROiwE=joC+D#-QSJAJ(rG_+bz%a+}UJS#kdt{jE<%Py&82-tk`2Dq3 zd7W3LC&_#OE#l?e_g9(_1u1L=v#*u=__>+2$xfPZeWaX8zESuf>GU)RReJ-o99v|t z6iU+{*$jkWCRAt-M*6bXiop1hLJ|QR`cwls&!qOWJylGjZ%93_T(Sm%2_`t;HWv*_ zr-ZGZk2ZEC2nUm!#8Q2fBCud3tgc+AN}2+CC3s0HrPCc3#Q)^*wYcME4qOO)oFoouc? z+^^re%Vu5Tf>vgbUtBc~t^Zk)T*b}=UVJ{jrUBeS z!*uGr;V8>6Uu1BmNZ4AfB?6sFlU zVQyw2LbAFDvRCVy!ylVWkEteZE}-uc32t59N?o^Eb(=dsYqw@QKL8Uwy7MYAf2 z^Ao6`;e*l;lvTKWtVDgo~k#bTuc#9IEWN|L~$0LEI<%cO)1TnJ?lg#deqt5%#ktn5Vx zMsrVXBH^*(bX0&RK)DX23QAbHJsAddW!IxQlQtVwo~qlS@1_iwTX{1uLpdDLcE$R% zT1|**Su_83`5r~0v@xm~ne;)OxS}yYL%BWL4ZH&U^PqeWZL@qH)`>`8&M;Mgt;~sO zE8l?asnm&Ti~PtbBfP6sG;c$>J>W`eP3d_ro60>o(b$S&dwcRp_oC(8wYof<<`Ffb zp9*5XU4Y-45C^{b#GLYipiyz?r~aO&ZI>+SFmK7shzzWr$6KG!Nuph&Y}RDS5O^+a zl(qWON#d1#M{SoaqMQAsHxmk+||AUDcWIZyOJlGH5~L+K<|G!nYCOJ7G< z^`2-k>xcEXd0pLZkHonKvG{b zURyVe04!3BaGaF6A>}f=d&c-YeFervTXsrF#@?yuHaz9{IOjZ-p0^8 zC2jsGmU_F=H2t;HASqftgn6V5;o^7&m9je`KV}+y-`jIea6Vx zX}*M|F9ErS0m#(2N(kJX!JzGaFAyV%qDP zN9$m;vy8xvn?zlrsM3}5{#_*&6rqg(kUIOhFwvGtPAMDM6B$ZUqWO4$`8uUQnta&Mt0MmbhzjN)Fukc8A~F$)tnLtmE^H)XzY z`X0L6q#y(T6e1l+VG4qhL#p1Exx&M9{VGwf{b}s%j0`tnf$cQVE^2%Lh4rX*_BcsB zPZliCz+-be43^HJnI1&;(nlk~8}{jxL9GcS9Hjzf_7ECcz2deWg3Sqa4({2o5IV=@`UyhTz+ z{V-CL+L-<(yOJ`6Ug;-=9q^R+@DX;Fc}3(9h05?qq`x;#U2lxpAXl^8WbN;fS3+4~ zthLFfol2;#V+@v{f7HZbn%-!clxfYaU)ZS*!)|CK-GI1elZ`ZWu4OWm)cH3yNRZN} zFv90P1Na>KeWbP>Xj=;uga-r0IFhO0R86+V-a)C^9<44yz_KMXsXlM_tqGD=s~lM0 zC6`1)if@WnF}f#jCnUYQMvG>yyuYp$Eg3>%g@+$@IH`L4Og)h^^~P_gfuEj-CvT7_ zPEN{$TAZ8I2&*+A)R|%#m1&{O_+P_2+?8eH>T)@~r&ZM_w#iD;L}WG>H?r;a;3S@k z^KeXH+C`SBf?5qut@V0oIwHB5?lQ87E7NGCq?hZAd^FRPM&+>393m_J7Kj?BRp`yB zBI<6sALPym#q7tBU^8*gj5eNh*w&>@6s&T@NLo{wN`L(KK$e%QQ(&It{RD%dgz=AS zPe69!#3yGm+hDi&Q7^l$wg+x;0``8-Pf;Hg3#Tor);JAR!)m)x!_j>M6Lxes_E0T-!JR|4*DE61QwDa)1dRIf zN|0(eltf;5zyr2~Y}0^x|#N7Q@Juci>?sJu9wo zHw!I`vrU)Clr&;eoL-14j!onkLj|~Xxy`V>#}PZ7sL2#E26dBH_im2s%R!Xvu3TKj z%WL@jeoJRMr%sp|`@{nPapxs@Q^8x0K@z=+saHGyG2R>kFo~zpEL=c9F1K4B9@3hCjS{T3LM1s3{f2;DB3T zn{Thn6Ny(^(k))_t~|~jgGzW6{V9*hdLnesus-)R<+p@J6=O?;r|#j;5}wi?UM(LxXNS02TuQf4TI1)|HT96s!!+PW$GA+BIaep!J41?)!OEmtUVp&;tE%N;0mgTbxi+N|CxVU~$CSN{WYUIc!c7 zs{_i=&9ovvj0!wp5#1|~w9K>W+WHL{>HL%(st=a?%>ufZ}vJ7!W5!t5Tmy0}; ziz3Q%g5N-5??NN&%q;9lA@(ihmX(DCQGpAO;U{zVZ?bA*Q7(VeRn3|}c-;{;jC)MA z5Yx2~+|&RMoFT_z$*$z(ZA&*iy3}?L*vrl+oCY$iDV9g}bv*Ntx`$2L?DMiYxMBw! zx^*n=aj(DyTwn!oi}+G)GfSGiLg4$QJ#3N3HcMi?z}3TVIns) z?t;6?6fT?v*CW+!PB<|wI%Blp=mZa8SEnEbQ@?umY?lhR8LsCn=pjZmJAvbC=eRxb zvzP`M3&vRer4Z1if5ufm`+5{bBe>3b1mk*0n-iVpFmGkyXK4v|tsmZ?2VC9l1=J;dG=EdxOX}|LVKLU2l}*loCeM{XdMFvSv{aEBiR>H z+9Ec&Dg7S5Z6WB-e}=0uvAOvBDsSRgr(;&Pwt$VuGqf*sB}c>|h>jG!51CXCJI3-I z2KmeAwu2U0tsFfgTnBY)P3u%qu$h5)hmS~uZJ^(095Bvo?T^Iin+*vNIK5&F+QH=< z#_^~04ds(?jNSE0k!5uT>lk6oMM&m2hr&1&Dz*Bmx4yNPi||}^#$B-&gDUs#FnNY% z>dMJNt*`@9OKk;}otCl)BY(Ntkgkk+wj)BVHy*WM|M^T~?xEw@`h?{@;E~SR+7W!B z@xI{_y?cs0!LJJ8_silF{Ph(|{R-qdHQbcW(WCpse4f?4K@fZrgWIjk%=~$k+1n)} z?Q_!=n+8+0CWnMgBFFTFpw-B1)7^LNu+9fubXt*-Z+N_VGJY;v9y%D^ara&DWbXn$ zj_?%}H%Zxka>G^wZDrwZ@iRk3v6?38sm@<~gPKQPKt4txO~4^ZJZIi}ac~!?=wX^k>HP7*sQN1*8kDJZ-1 z{p%k|{%C;3X8%;4S7omxw;P4d@M6_wXJ&?~onr=g&0qz$n!oDtt_xn^39d_vtO45#g_fMAO>o@bL#m+Bo_;`F8unGw*VN za*WQH*-X{8d7$F)G-eY*bl@fIRr)?R`APtaP`os`7jy8 z&!a7iO-7`ESP-JlOtgg>sIycqts>L(d(c({wA=Y?0zYvz%wq5^A6(6|>eM!C@Mi@f zH$sIU%~A{BJDS|WZ3|rF*gpA*ycBoD%vt@KfNdmOknO?HPZFK$N~`c|!O&7*Q8Lj) zCZ$6{_=d)*rE6-8smSxXr;-eRc&pyEX`_?oiacHtZ zM)%)I`;5d6isjIQ6T`%YEzPp`qK#&&__fZC#BBuwb3;g=8pFqbF|Q$6TVgv-fDF!F zU|d1hM@D(Joo5*#&MA(H@#fqVBA7>f#MJR;B5@|w3iH#!?XVA=cYs*4d zG!cOIEU>9ffve^{vQdyiF zOH3@kP!K3WXJ4|Xa9$mV+M8p-{DDZ#n5kWe(Wfr4-gkd%@}*uqu4NaU7pH|E68`P$ z+JJI|xfV5yu#T@X*N5W)2ZkO2w%C)=PrjyitO!7q(URu1{cM#HZk7~U72yg=wN~52 z$f;9F2u)ItW-xHW!f`O`P-EE6wHvDc+S#+vT>E9o?0T^*_%sMA?XGvLH&~K(KZ#_M zJWsWZB$-O!oz+L@wA%|~hz9v<-$>~CA{hO$HWmG-yA%$IVW;-xwK8o&S+^Pe$N3E=%_ zYJ^|@g6LBTAfxmNm*l=ruJI{!r<=R^s;CsdxeTBCma<6t?q| zM*h01wLZ1-?axvE2&(!Z&go?{_^W)2@>LG7rTmKag`oCj)z|FfQ~rSdR=L4PmEFfo z@-L7JR1${MF0~`bgK8O74$Ked{$U=V>@hN!d%4K{%Er{Ql4 zG;o=uX2>V`a5B^HB2L!Ps_%)Q*Fa)3B-pBl;3Uy!N)DwXqzVW2yQ6xRLCvUD*Be=y z8Upi2rummWhmsd6;1|vi2;x^JgYqW;G@VHgA(iLPKxORagjTL=fj#wi=f)F8gz4Eax6xK- z?IzKVznQvD?*s$()Du}_5Gt?$W&WHw;twkgc89egTxNmkRT&d#esYyC2Q}!}Z)7dZ zQqV{i?;4gNzj|x0j4uL(3pHv32I2Hov<7nmLj6ax+E6TBV=_f`(q;*d)deP*%K*z9cQ6gNg+D ztuoNWPb{hH#W@KVFu0`uR8-%Ks#<&V9EwozGKIu? zO^st&h?$qG&E-QIXspA$bQETd{|%((pLQS7uh>`;uAtupwE zolo%U6BMSs{=loHw_r_WxHqZ5ObK8aEM!6$0EJGFZx=6r^*p(ERE zJ4`aoo&Bq0jP&|(8B{VV8JIp;m_H^iC7z!&pK` zezHa^{{Z-Cwr2Ry@i249k;JVY-4Q#6D8@L)G(wr0JK`9Um?6fqD>X+5^O`*hOjK@W z$KhzVSlyuQhQ2eY|5zsfjyQ(CiJaxoR9k%7K%A1cOnzLws0fZ|g`ri3Gw~h6E66=b z`6}O-rR`X5{c0lXNX4V}<3qguSw6FuBC+`T$|f-`>X<+7K5rZfWjyF+Od?T?FL{il zG+ENVKr5KD4F}s8)xRFg^AMJu*wQ{jD~#)aa+3nvE=4P(z=20*G>V-Bgq?;hSCn;^ zA9GN+4Kik8s+CytHgMCQ%osa6{?bju20gq}9c2R7p!fp8(O4{TTsvPOct~N5q`Z*e z%xJK0!8}Ur(TDwR%HKnvTNtH72io3YZbeha9%zM* zxK!2ndHZg$KC;1GpJUv`c2-z zpqz9`1zTr0HFROC4%f<*?VbLmMu;!*uMH2i{CT%j!eZ_M;IeGc@b$^fk zkA|CE55qNLG|x!Hc%e$Y)pWNz$hrR^`TltUSsHvgDk}*Mm+6edV{7}uK*pyZ9 zG%ZOSnU{_4iaOb0cOPZMIVlV^h z-vyA9KoiRnDp+)fxzQG;Ct%{OypjXZF)gZ4vy>z*c3QoqkBGKijp|=1xxkHy&*1r+v~={%M~k zrF0nGwR0u0+U5ONH=Nezfu+r?rO_kQB^HMC)OAdB;#WwS1~Ss-TmRAICJL89tHmx^ z#MYxq&LjA#kV|U+h4S}@IKzKCEy;Nc$?>uH+yQdCfY^ zPAT|yzcc@Jf2+v{EcR9hu>r-QMG|re#rb(dpdKi@&m{ap%}W+ifvUA1f}fMp;Yz-$ z9%yNwg}<^7?GxFDet3nAa-m>XfRAjKfIrl_7NdDX0;Zcx*D|i(g0I`Bit05LQx?yS zs+%%k4P}$k-;<{oOE(h@i4!|9(bDs}M6=F7lOm0L z(yKz0Rf0xg#i~N2iDP#5+lILdYLahYDIIE*%fO4otl}>qTDD18u?Xh)w^~-EvXy3*IvHM4IDIn`ZLq0cN#AI5@UTuivt5)D?B1|l)Cq@lMhbYX-NYnd%{DDPr;}RiL>7VH9yGd3NIrvh()4Q9ITp-wgr+oi*Xe6F0uiPv{=2BRNt16hVN>ylW!5e zboGTFu31XgJ5{0?YXuqGp!)e;B>KK=zQ*bRwQ-c9hhBnMR}n_-Lf9L~x?@WfNc<)BKcUW(&)z z0l&VV#0X{rf6c6uc&kK`pxhtq_X|=923x4CULrih6qpHoMkAYRIyqr@;gMj>$)K1; zbSiKX%g<3X^8ksq1aE_ISlEdNC&f|o`&;~k1cmajPzZt+r52B4o4T?1$^C2CY#!#c z0%q+2#q?Rn(HKlcK59fsD^RwS9sanYwEy*@|2^W~VPTQeI7p>dcWD15+aH87`)^A) z9uCT?Q#f~SKDuoA3voJVF+3In)SHqQzAVMh?j936u{3Ap9=Mx|-Y`JbcU*x8OKSm8 zEdexB#_oTN+L^m2+(}IDY80(tkHVMe;8O?~q4e=jNvjx`U_z|{l*B(O`fmvHrAEp8w3mBl8OR$GEX}Ww>UPtg!*}?VAm!@bvi_p*1(PuO$^h! z2tUu~`t|+9Ii4i(rNwo1cT7ZT;&=Se_TA30i7$(ZX*u*K-AiaO)mt_4luU;%e#@K+ zR;byk)s0(|OgQ*?_+Rm#We4A-qXPR82G;7-?^gBDCTLVujA)M`~+7>{VW-%njF@Yg@?uhmL^ z0Pwjr!$>*MwQ#2DzJOb8AXNw85MNGV8$@15M}94!lMJv+tvA*AF8-iLF|tq0Lj0ZmSm#dKPDy(J zKSNeiULFb`s%?87TvrkP?wraq4?@XZva^dEVKso0g9?Qm)HkVSa3w0g{NpYhNf&;R zl4Lf7L7L>X3xi57-^nmj3x6kK;jXp}dcPd%jXtleFc%y-=aloy48U+^kGHH3Qq^2V z7m~BjiK542;>hz)dUI|tY2u|h%{E0~;Lysnx>jyY%9<^%1kL#XQjg$l?w{A4k{->i zJb4iodcsb~ZQ|N!&S2fp(0{%lGBF%zx8-3N91`m$NzF`<5$TUQ)gy9T8&0?!`MD@@ zaE3PcA!okYCEMQ}>^J(zVmj`yx@3GbHNRM<6UiIe9y7}X8T>SiYDin~_LJ`yd- zmD0aY;nAci4pRfB_V~rDNv?)iz6+l81=aQbluAz(^vmnZ;;AkMNYzczddq^56q7t&dRwgu3Id&=CErPX)zCRcsj(=(Ou+%X(UA@mLK4e7vIuVV1i zd;1qQnF}Fn(x^PC3lR!*PaPgv{OY~1P5|1{R>YEB(lb}4tlEvB6Xrxl^6J{gs(PnD zmW5fVEw9OSgaJZrO#&_hP0hM^S$v9)3HV;aAy4fhZ`X(op|4+kliJM4rhanb=Mw=an=`SZ5}Yb z5?!cfgtzlTaIB5ly`{hO3dHQr(Z6N4=lECg8rBt{yGlWf&hTO+d>g+H13ng(dQ6y> zA1vM!*ST6W?}0v+CG)!^wy{Y3!)jWJctaF_?3FCw-~&hO$=)kGnOiy$Zf<%&=#FFE zF|f7sFR?Chyy%WA@oUxjpEZGSYD!NYNiAw@E$VX+I2>g^6dYw89s6yczg`Phy`XJ+ zdHs@heR$;Cv~E0-?a_Q(mjvkR^09{oUH?XZ(Jch4vU?`2;v3?+WsP?Bah*ypf3(!X z+U|RF26*1kQuTRW;kKoQiYa*?P&Uv%|AfJzmE$=SmTrhyH~t zYGg1mnN7{feg}~yd_g-Fe(!L61)M(17C-fv9rx6w<4UN>dG7Wt^+(_metXM)w=T&O zfBhe^xLaK`zS=)+TJBGq_CFDe`(LH4h^>RQz7w&utMy)zG-jMYy@DR!bJ@S%6zYC`F7p`e-bC5a84Gkdv+Va(Pv1r~T2TtCRn ze9v^8am?P{(Cz8@1g#A`gi+!i-(!Hn5@DG%&NI|WCo_jJg7l(fLYv2k+n%i*&DC6* zQZ+fuQm(O9##oHXB{6F{dneSofBEWOHLace;{>z4`&3QOS#8MN_R0T3W0=z7pKBNz z(;rR5b`x~n+D*US*+8OEM3!(vH^&F_MZl`BF#SIaerUw8Cjrsko)@;$5_20E4AUyoH6a9l!RWSZZA2L#P zF#mr#x29z6|JWjfcj~e#FMq5Z4*|spo6K?};_VcN4Ul;R?0t(-8>@{10=Z)zcFlFJ zs{31mK!YrQ24O?_{(-LinJ@ptU-K#-%4E2&2H9AhsN-hxG&$*>E?JI~qekt0_%k|4!a>V+v!hE?=7Xz0Ft~Unp#gt;SM29J`*h z`7Hd{>;z1J=^!k-Onv3(DXFH}dFI}|1n5RtnnhaT|*!8+5-6>N) z>`f4uLenI;c*`*&Wy);yTEgPoHeD+NFXo zdJe*%L3$+a44LI#wh3bc_2pX`zBh37U zYMib1l-6*2-F=kXow_{yg8<8?c6(CQ0t}lBH#)uH(qV0y>Z>ISgI+(xV z^ochT!bjlDb#O~vC$E2Fx=0S{M}akiA&%`M>5;O_FN)bAIT?w8$@5o~&+BwP6loMX3%Zos_$%Gv1x`go zfyw=cCHwLW?pLV~kpC_SbvY;+>ih^oPuTx6u=xM?9rIuMW|O+7o8lqnx3~FxLNT}o zu#!J8VG4`687SL`hLKC7ST0)z3bo_1po1BFp*S}O;u-DL-qmx2O?$=yL3^Dvq6UJq zl@0w?u+7y`wT+|emQ(w2+P9VG&NZ=l`jMZL?7L|9z4rFC=g#x9PWQ7V{`(9FJcwF^ zuX`=rb!FamN2@y<+x4y7{kBGB?|Z9zk3YN{U>Mx}O|a>Op0_`Pw;W)|;)cq5)raHd z@5x5hIut>L=w1m2P1n}ax?EB#1+wYh9GZd3$@3R*JTEPjpa{+<= z;9zGSQF**P$&CA4QUmkSKrES3XGg_>$zvy&VDOOa9SZi^dOE6_wz{J4da}NaEqt7- zcvr|bE?0BaD*})-94O&x#c?myTU|54K9%&NsGhWE@Dw0=0*<}gSswoObAw!|_U?B=Bog%2(+fgd>C5p5PD_*H)Vm7`dZbb{VxSf$iF712pK9$) z7)EzKq|}UfAQm8D9_z@N;noTH0!0+VEK zRfiWUq%E>tWS@%Hw5C!z=#C`JS!IjcRvsBLqxh)|qz#@Rw7LIgir@D;xZ+n@NcP<$ z6AQa%x*^UZVO%|?l-l>(E0Ymsu}Yw{^XgIMUtN(Z&G$qpa#B`~y3frx4bv&fEB(;2)Q`uLl354vqlEK67YYr|PPWrV@m zGVFuGy;rODG=B#4itIGChRNLGL#{Wa;cNgbu$Z#{OzZQ(yOf+KjkFu;3$K)Ky)*T@ z@KtX?ry;1hLC={4n--|;E4e`8vpS@V586f#Qtz1wnf1xHS;stS@Dl#bl3 zY$y3F6O#!QDD3`id~c*mn-ZiFRxq&YbSeJPwhC*5Nr!@2GQC05xKR%WX#t*9C2`Ha=SBL3OHow}SZD4N5na&(JSrx#}JLN4&tI@IkJB8y2 zO+ESB`%=3+$=)+b=e*(TKHOQnm;vpA?Vi#A2_Y*mnNv=EVk zpo8-4tz~<|?m3%nQR}~3OJV!Oy(~kowf5NXu{h0Dnw|?Hrhy;Jkc5-C-;LLY-bFy9 z_wB%zB8`v2aRgnE!`W-tL3 zz|oX0XAI123y?_j$f;}y7y+(0_Z{hl9)@c4!9g_BEWD;S>{#W!6NRttBeiDzgb}B1 z09t_@x&AHYZg46)zI1yf%A4`bq2KmZDAdHLxtckp%oq8JNBR^73Q0q1q3o~mis6^U zWs7t^0uKKqVc>5&ON*Uww|i$GiL#(g9sPzZM8#;H<`>WU`<^-+?o{$NtU+o1C&IH??}v<|Jqh91~+` zmU}PF3u~a{>^Vm5_6K}>haOhOJLnM0@`c^sLin>sN#2mR4Hz1gGYx**$KOLzxZ=-& zL6-0kS4EJY72Q^Z$@J+spasd)#yKnaGKs_z!^O%NTdqeXMhaZjO~VyJGYnW>8fP7H zzRjHzh77A<-IiG(pSFKs&~k>MB3EnE%y>wQd*(^20;>D-Q_<^FNT9jGsjVE_vwhy7 zl6j>qz5Hc#PA>ERYGU406EblH?5lZW>tAXor2tSzW-)Hw$k2{|1M}DT;sD9uZx_c0 zk+uOKGXL8BD_)V3pmTBJl}6wbl)s~Hn{rf(Ivxwt%{96wEW1bI*zFk;*QHfZq$@2u z-W(IY^igP4lLqIzbQrrrh+?-|2AAbmcf`sDq=VQg<3h!^Pf+X~gmj*uZ1S#{m-3d( zkU+LF1s@55g{8!|5oD$kX_X>FVFhAK3c;oafuln=#R`V1@{d2d6-cdadL|^sd)gN3 zpn{DGyG6kCksK(nDa;5b+74|v1fLc&EU=9taQn^v0?`^mWRH#@%j_Fa>W5A-BZuk? z_J@HrzEf15M%|8f9le$gwOtWBE|I9S| zzrK)_Obt!`_ieLP1JVO!8TYTOTe4&rq>%s|yadl6n8sZl0jjbPFd;!C!G;=nce3fh z%?wv}wSji6QL930YnxUp9`90{(k+c5&lzfUYq~nUw^r>+vAZ?>_3dA?BpWtq9X`h0 z^p}%gFTd+w{6E&PU-&@`0@cj-JZ$O)g>*;;#Z5aj!lX@JGhnn$)~xmpJ5+k%}sixX6jb%HG!2II?-0gd}?73TRDs(KvweJJ1?<@!z;9BHbq7A}GKcvf&|eHgg%?^6e!aQK48|zI z;qq7=BKBkDxRB*PIBIYEaFNHrVWukrh#f&P{%u~AQcMhs&T6SJRi7zt0xCLt z>C&5+u*~44+jj%BRW2w3AibU-!7ZYtRxKEf%C#&%6ehWfb)(65-fGD$i?t=}6id8n zU|ZL7aTSOV5tj2Q2CLN_^R^pkZq}qrx1KYmV_lapdl*e^+?>ekBuQ^p&eW-n7p(H8 zChcZ&n>u$j+PYnKu)t(Gn#WExCQp2!LChFqjm~LF)~?wPJc+7p4!>%g>{!!9p34zFsd0(kWDGPnM4;*8~)IIQADT5=Y?oxdzOPx{{7k??6QV7AAC zPHIC*2#uEXj`C$*ZgQwl$Nr~{_>BIpCQd)=71hsnKrlaYcdwm^r~J^JEN>JVEA1xHVfEe{nf`zj z*}rd+OONrMN{{(o{Bv9ci4xbJ?H>Jed1!Y?3C|+tcQB(}!Z8p3jO*v%5dVEwoPXvk z`**RSQ|1|J!e6V5Cp1Qz1An7JcA1{YRasb%%mIt6I@L@q3269vJlsL7XWVsN;b3k` z@7ILDBGv4Md26Dujxb&?+cT|L$#W~sO)A-FLfi`1k18Tuv$(T4I>y6eW|(rK0MBl9 zPga>E>GBR6#fhdDmGu*%uI%(U|I?Oo&zv|$+jtD`epLs1OsSKCOr3e2wyC&n%-_s7 zmYmi#GGTEm4DB%Mji9_^HkBcqtGiUSse`HhC$GBP^UyK}aIEHLXNy`aShR0N8Z*G(jYjF6{ zNvO`8=*&v36%QVooUnD4+0Yw)@LpCqxqD}JRNZ;OYSE~MZ!S7|7b`{ldWmY!{w4a0 zq5C!_bz16D%cZ|H?wlmdd-HjsG(F5dmmCotUXLiTsj@x#*?faeq(BoQzmcc=2NY67 z$nK}iS~a>YOq=v4E}8aX=oO-V!f!H;&J=irW` zW?&WXBQwKFT1;O6{g?Ch_s!=F!>={wZET(w97}&y4_qccLEB~{njd4}vc!Hekhch9 z?Fz}FR}0OtpR^z{+JOfE*>mbM+@I07B;!<! z8*$oEpo@1yh_Mie-%}D5_)m#IWOd!r_nNc!95P23;u&7ciZ_udZcN(w=V!x!5_0EtIsZPo!RImGv*kd?mdiqYQj~w^(_( zam)~tc3g8#oaX(xNS)}&V2$ZApmIe65mT@w0B&hqO)2c+p0jFA8GLpG(RUEer!L>k z(a;=I`%(o}E!^Sf%N4ossPh$6MT|t`*aHC*$R#vcx#EDr@p~?E`+_-40g|}1!5lmb z3k~fysB;4N+p*)^45nXhSBng-0hZtWEkI?v;h3r&sIsB}A87zr>Y#><5nytv!7e1~ zVt^l$P9nj==iGZhT8Bbf=FDXWL^v_h7EiQGUrafnlZ}w-6tLw;<$;Ll1?Hlw4-*#? zbj$TfvPiEmziS*Z?l&x8j)df! zXorL=xV=4rJF?8rWMK~{3jhoo+!Oif)w#7ms}i`9Qjv1Ez#|dw3bAW}gGIdB44tr3IzI48X~c z{3g|T?5qH&h@0_u0BEi=xI;&Xt^{LFMGJNbDc2e6sQ3iG2FXvuOt+l6mE zo+EOV@a9m&vakXu&Uj^8KylbO3%uxH#lr$eC+ZU}x5LzX+BL_sp`i{wUvO$_-kb(D zz1a*duE&vxjMB2uXfn%+a*c}o!;S~o!&{bqj5*{NPV6%e^3>KeT!=o@huLy;T=S2+Cqzh3M^uv zB8XZ|o4p~Uv$tK6A(mFee?QTm?TyAU`!zCi)mmK0SpR;Gzj)k|CD|{$D@`8%Wj=eE z_rQI+weA1?btn&@`>Hq)O7Of_6q1O5Ks(*_k~Y2F-`swz>OPMCw;;_SJs7~M9AP|ZGuttqM=UB#p~k?o_BTW^_E zsIoUKxS3@bQQwrM>bg0zXt@$YbR^a$LKbHGG(t|%AgFx`6H7UGaIQo*nCr-Dj8sl) zYF))y$Q9;M$CP~$gw3)n6j;c~TE3%5D@p$LATSyJOq{URtNUT=$kjg*E;c1 z+OBou>u7b?TtloezmSj&iqdp9ogXdQo)uBj*;k2Se{HF6EaG)k9D^s5POA*ef)ZgE zWD=!jO0pj*S~5BXyV0eWfCWq~FJx{WL&(@|eEzk8&}sXx;A=cIf&~+su+j+MmAOT= zA)A_ug$C`|11NkAH3?wHe znhmDs8d7iSegQ!tMRABX(~04PMjuh@$O;tkA+iW%=CoG6p}eeVi^v}weZ(l0UhQj> z!TAJ70C3YSbV0X6Z%FnC=%8- zUzR3F6dR~nT5pHy9NT0MRl=xoOXfx@oh>CRf)$cHV07TN*!6C&hIWfcjm%` zY^dw3N3Glpqi0vQYG)d-Fb3=19Y)P@z4cT^VmXey>bR_9YlT(LaNanms4L$nSx zU_WiX-E4AIp;JFQZcg&)GRwR{cgif+gy{{#+&t!p?4Z5Qy@v!AZbMa z=p9>+b`9hmSnj!b?B^NYuy)XPnLfu&B_<&dT4M|_3E}m~b^~=igYTG*T#B4AH+Kv_ zs9E@gwcly8$hq~4kL$5S;jpV@*<6{ zohTL^Z2~V8;yhH_Z^O)hp+N9gvD_) z0hr$yXtn~~Z z>9{;XPeBKGDIJk#m_QM5p;w;qIWa~thbVmiKf#)7H>G@^KY{c6Ke_<@f28|6x|-S< zn~GW5m`d9HXu1C_cwzka@lJVL?qAX8VRqfFx~*yo2!#|Cv?^kCj3a9jLP`l5z=(8X zF5GtD?ah}rHS_BCpgs2^*fKqLfIbw5=PioGE8KL?Q<==CA2YXpAKyR71lX;Iy@RlD zP?|KwhlY@B(=DgQE2H6I+pq=T(p1keg$nj(%pA3?WSH~~smX1hxzuy#`PeZf+tI*? zXVI##m&SQU-&Dqc{@oR=vRVm~Op-}OZaHB(B*aHjd6ltlA`ep6_!m+7IAprr2?BPi z6{T>LNO+MyGS7bm+J?C^|L~%^9bM{N2-sCh*jNgUEx#b%dKtg}acw8u{Ro0wiqfq;x4v!+ISF zSHwYl702?`ae5FJ?|i4T^`s~syuj(!cwr1HyReF>m!p}ELG>HTYlAltu(g`45ix|F z$&`bmI0WN_Y{bk#*wXD9qwF%)AGYXEd8}D7W@6Tr04${0g?}+tct<>vEx&N4Unu3f z1)2-DJXU$~GurU-tU>-&vwV|DY`jBY3BOaHsLqJPV08x*7=E&Q;7n+%vdlG0b*M7O zt)it`w3=fsyaN=zf07s5vvHA?=0Pcrs3-Sa8Uk*%xu<+Z17cEVi~Ab)R8Sa zF|BPT>bYoU)Lp{4CoWXnRvoxCbhw78ieNA)Iw&Z3`wR~&R$26BccaGRFis%VV83k< zYBVa-r5VZ+E~r;pGtULy#sLbd{lH=giOtJNMeYv4g}*6BP>rs}SX7VTZXD?Qm1yY2iawmMTY@ku$E)dYC+-9#I6FsapYNwN0OZk!qz zrNv%%kGuq)3vOQ))!jA2K>MFOEpe3*Sb7u?M}@Ka5YGnmLx0F{>xG7#U#00P$3_cWd==u5 zQyt|n?s9Y(NS5r7>tt`Ne_64wCn(pHfjV+~DzZ7^AT=pAKL*^jbYM{t`WB^^u6K`!hVe{T2ynHT^~6+=;d2|n}S0-|C{U=?oV(6kf*=rK;pao@^dUf zmV|vxREke>+_HJzc)9s!M%y3KSsHZ*qIu0=X6b}RILY_5wBFR&&mv|jj9S+dEH&#y zdz+@~`hYtlb$s|YnC)6oUDyL^{LfmYh8s~!CTP%=il96#%yp6~o$VzspVHs9;qk9_ znk5GMM$8EjxtAXd_=L>qn+jB_McS9QBDn3e4c??3PV5~ex+k7Ld}x>{!c~;u?GQH5jw0+(|S|)w0^KoX83}yuS_ZB{chZ&Wda@`j&WX? z0?OhX_`2lhdpHA|9Adm#%4u(<%Fo`b%q%+RdS7tdYI7XRSXyGQ?WioWw8FU7SJ0xu z+tu-R4cu5l-+ zJ{wV8yZ6Lc)DmgEIJJUFiy0*@_nwUK(N>9O;}fDqi?k|_r{!r()~~GwWW%!@kMsNU zLOAA}C(iH)KaRE<#2CPj9`ZP#v$z=J%P|Cj!TToZ|KbQ&{+#s1LU#n?%o(P6zhm7Q zHqIG^&+#XG=G=h}dIKGNz?k*`hiuuy4y#K}BYsWm)@@md37?XMS5G>pX2Lw;4W}Pp zo@{|J&a)XUTxh->U!M+O-by_hTgLS2^XJqkML(Srkk6kBf8dFh(Zwb;?UB*-q#ysK zk|43d{oZL+(2fXGqdGn>0BxZuhzJNnr6h(ioL84QC-dIvD=IzV++}NhS@|?8*B(^o5?*b+F20|wTDvoHe;!Zv-BW! z(qy9<%r@4Ta2q$b3ngT-Uj-G`4~5BZS+ySU?YAZ)Jddh%b`I?h%W?0Epos>AxhOJ) zbl*9uO(QUANpEJ?#zZcT%W-5pJx4IU^Tfl{^~)AFh$E0azz;y zDA1}Rr%dQ-?3x-#%Ls!<=d!p?@5VMJ=n2o#xja{uXRQ&t09azQ=o1$-R4q1mpr@@i z4>$Gw8wc0B(8Q3&8^B*n8@WrJE9%w9R%Qa0zbF~YNa=4nK7XU9e|v{{_pxy~HKnx5 zbz60Gc-RqNr#}%{32p8ys5JXETCEGsysk5Q+*UtO&qdCB~8JU!+EYffIC6i zGUY%`$c_@sDa5;8ST~jUl0^~rM^J`=){T_(T|>y@>03y9cOc02_DpdIb5LXRhR3XT zXqE+eF?SaHz-5nYiG!>ye3rmFfE{$1AFlk1Y*3j(l5aP{z$=+U&=OyQ8FB_s@+Xwu z;H@6Qw@ed)(ra`K%p*McQO%b#kOhpu{zqKVl!VuIrWCuNLE|HwVThiHD-sQEa&JKL zZ3aC*#ujUj2x}YO{>5Vw!%_>jbBM!dfP$YmO3tshGa8-ip~NN4HkCe;j8a4%3*8~% zM~BFDrcPKJ_N@Vn$aba9oYWSKuBNW{yn&;~q-cQeg4+ zLr5uNc@x?Z#ppjpByJo6MNzOz&mfZB@De?5e(pl^js0J%mVdoS5S;kJ&+q^M-Xs73 z(*MNF^RE}_->zQ`NFS6_SAY9@%~&ycZ-&`~S^3Z^*8Cj7!HqQBBM!znbcrW-kVpR%REO9cI&_FA7G3=Z z7(zO@j72|h39P%~Q7Om%hn0zD5Rx7}B%(p7+)?GwE4R%#SZ=dJF$|Tf_K3{WsT8J~ zMXy$wTNmz6Yldem%qnQ7=E-Gn5875l)y7*GK|6@2Zo(?Kr*Hzhl~3_VHp@%NCFVZH>L?Tp#s=mt{fCs&^f8JN<&51}=atwXTpZrThS%P{kO zuV6D}GKPf|jA@c=rU}n-3%9Y^;hT$Fx~I*y*xUkgr3@jy^i#HYCdY`1D+%_j$<+6Z z3{xUFvMX=t5ZK|@xl5TPMc#PUS14aICD|F2TEwx5A#%0U(FXLmkC0#h0=~hYr&49` z%(Tvglui6#+kl5_e&UNtb-!zL8pdDJTTT+*0N& znG&!7T#bAQ(p#*|j>NWK&o9z#_)uAU>5{3q0fqX^ZS-5&gOV=Tl%$Wt^^mJ;VSPUa zzzOzL>n8Rc_wLLK{L>8{t7p^@Xa}&aqKscsjJEg2W^I=8Mg}@R>L_J3lbWHji6h2+ zf3Y{p#UCmV{$z~v?>3lcSB{~6_L8;6*qdl$X5wRXPuHg1$; z0ET}^mbkXV%Rf1#hsKsPPeMXey5xD;szqkG4>|>LpxDi*@_AP*Pe|>e05}pR`(5aE zGdbY=AMBAi#_**}r>@dWc+#2smvZuuNECFc`T{QGuL0;5pOW=;`8BDJy$}v>a>i!7 z+QZ94q#{~7{&0%QJ=%(YOvTvVLpcI)OA}rb$iMB0Wjxkk-tOixB!3m9M~G4++HGLX zj;5-vNXlqHs+`ALnvGjuK*W1GN#sw{v4K_QMDiqr@HWjww5r5)@a}|HAhbI3FGj73 z>{y|(2Z4O(>CQ-rbN%Mkm3ZQAg0@d$Fz=KAQ&)5nWRzKqB61rFA99Z~LH3+-CZ{rW zC*-d~mB$J(@7@mL)?t;=#Fr|NdfL0$5jMluX=r?{DP}#Ib5%DQ$-dzw$y`N+Jh_R5 zZja9WSTXigMm38D!x@i(mxPmm)`wHIp(peeg?4G9L!QmkTV|7A~;-!BSKoRAQ3R2bk zgHWxy6W2j*us&aPI+Sp=^OT0L?Vu zvlZp}q*FT0E5Bx$Rnd0Bmp-1hX-gQ-Miv%M()B>fT@fN$ulG^9l|gUvJ$VjMxvjcy zGYlobNGZ(`gyb?&h*|(MJ%bB|TL=*R0y)#9gC%hAh!$)|YybzfI4R+=p7TEU1+zDO z9chWQoG#!m{eaz>VaMzVHU_&LO1nAM#cIdgl8d+8V%*myH^u77ekri_QqC$XtoS?UPFZ@j z1^b;qorc{w}pba)U+-lCbuM&e8`vG&Lapt|gRMjv74vV)Z2;7R{R^eMqqy28f z7$qtm&eFTRN5X4&hoH-1ks-z+hnaH{Gvbbs6AG3<{Q!pk%FweB`H&P+e2-}Ox1xXa zm1A*0UDirU1UHTx`oxapSu-)$^%xg*OmO`* z$dQnJ82rvO18a>wp5RvYHK5^rr3wDzwz{g`?-jF^sL{k?gyH zioTC&rtXC@T`uS(tR0SQ!<+(H9bXtqVbbRgo_g)M0ubFv{E(4SKQIFUU)F?mF7eWo zWG)y~H4Aa=xywRKa1t5!YvZpj&t12&Qf0~wHCeUsx~Gg~cs=Ay9s5fiEyK4SYFpJz zHgU4R&Q|RDjcc|ZJdZWv+{7|By+&+m+f0!cx>SKoi+ayFxT5UFP)q9h@?oV;AD(4! z+N&&@NtA8^c4)O{^iDdbYJ+E3vqf>02dXVK`5Ui90M)@O4DO1yi4Eq7-GS>4UeaFO zYc)5Mr|`6E+Jy;UAv-$*caL0Bw2u2&4Bvql0hf!pVG#{AE>vq&QZBg8(!s9_dM!D} zWVdEHE^q&ma#chH{K!xfBz3MM8+PZ;SAYuDG5w~tA!trzqup(-$v zr5#lElGaDJ`1NkT9wt^@P`9OGj|$u8H!FddxfG2qNC&#$f1l|rW7*bZfnW1>>97{7 zD?cRo#*L&Of&Zc(;VcL(!!X9{<~r=Z~6dQ zn@J+1_g^gZqnU0igV|?&C#?eMHplq_S=zow^2K07N2}#_bVJ{^wf04o z0}%tpo$l_+nHoo8$|_#zKI3J3=MwN9nMaoq`s*9YHwxtFQW#hh>78beE=@MeV}Q}& zjTizHzwltJvdk;AxDjF}e)wjV-@cCjC)}s}Ou=~UcNZxF9L(Y)8f{H0$YB4&H0q8S zZ%H_n-293ILrJ||WevPK++qwgOLO;^B9tLH#`3Af3d8gbm#nkn0)q^pD^`2F{DseZ zYGblr41O(s87FRRY%xpvxnUN&lp%bt&`JC2Krf@#94vnor#SRr??b>GaB z@ExhWGXV<3oBX^GA}I>Qhfv^hafsL-Ale%qOn__jqm!6JbzBzcL*BW`Uo5$NVu*sLt2;2z-5YZl%p zJrDx(eAzR8RQ7# zy@d<}#M;Uu*w=%mguY3FKPbbUln1_H9?Rl+T+jyek=Npax(B7bE^ARN#%uJxAgkcl z3W#{G(j(M!`bVLObUT`NmT6$7ldrG*{tGy&ej3a-pI%tbXmL#_@xsvI3(@5@biZq* zEq8y>%J%m-$9Lt#rWp4}$^8e|)bUMzhQ)Y=x;M)Y3MN7%71dJW%XwE6Zuclkc+{1ACR>^byG*xq^B zSps)oaMdHtJzw<9JMV^JQ;z|6!CO#;Prff!Uie7=f=4fCpLdE{h35rwp}Wix z9FuH@TM_oV#25E53$1vi6{N*2BU(RCq(V(ma*JWsRVx29SDR$t&0+xt{>r&*JS~m`?ErWOWOi#1Yh_lUtphJXrDiH zz#mM}Pww&@T|(q{NRwZv(W7-rXS#xeUvPjwWdECXc?GIPp;C=}X^ms(cJSkayTTY? zmtduaR-o5Al8jOnL;NT2AY9VyzSw}>F@~)`+3p&n7`xQ|HVGtJr#tgjbL;Rf!n1Z) zzfnq?M+L<+5k~r%J49qUVqX0mkH`vpyfjGju z@m?7g3ck?U2ZQ>eC&+Jqpj-3=3)&s}J+U*prl5;L18?6!3R_fX+e^IS>up$Kmqm!{ zvjCVz=0iaS8E$clRgZ*L*U+!{04G}7QtOS{k3f*v(;mD7!=wdv_ zF(NvcZ(vDdCP${GxRC5!ME=N`@KJ?%h1Hk0?#G(WL1qmj&=#XTO8s!mfI6EW1qRjmEo? z%4TB~xd(cRYj^ZiZt==^rP>U=q@VO`!YYxC(}5f00m9Bt3^(+>ZnqVT$-B96=je2n zAadI`M663Lpp4uR9Y1~Ehy0r6FUWDVzyYXb_wyFSRi$4|d|l%=JQ_G1v#2)SV7`ep zUGoB|E)3qw5fqv9g%F!kR-)2^JQ9L(3b=Z6jdDtSc|_cuqi#-Faz{T^XXaKNzCrNI z$~mObImOdC=&oNO70h<13EsuE47u7aqH+kQ+EvH-yX@4QAw$UBD zJd7dY?Tue>PngWElR*c~nk$N6mjA7q`s8iN3%NQKY<@xdk}Lq4PI!Ow|F89e@6UR{ zX*jLw>F{zk(xcs*p-oF8t&!C%EO&D&!YrDRxl2B)sBrr4QxuYCXK0gv26 zMG%v{rbDxzcB{?+&z)!M$sxY5rwqw-1v9Gk9Z=X(xt(ONq#o&w>=uM+6J1ay&r&~} zhz}vN=`^s}WTR^GZe>+-NR(=z?Wi)!gJ9r=$@0Fncv=foJUUfJ&}2NcDvcpvcxhFhA*R6@e?8E_MSfVE>Huo&>F_^w zoo}fpo80`6rW$y$d|{OmSX?-j6tg1y?P}Hf0!9*opY0L{_iwxNdu@*!j`wjgVVG6D zh@)=s-Y8kF%aJkin0hNS{0;M5SYGKGeHdSf&(s0yp-DL9$mG)!@wGXDzAme%Nv5Yc zqozsAegNl?3#1Jk#^#1yuhmT8WenX#Fkp?J^1Pw-Eu>Up)H z!$__gI_){VC3*8&0|3%eo0u4s%fE4c&$-p{{`-2Fz6+2ewk!gO&w2=}V+`N34N*6er$no1bBqJ*(zj5!3oAfFO|Sn<(|o-jcZ2X&Ord)#e2fsU|sgZT_Z zkB7*irK5fl`kWDS?1HqKlp&Q`0~h;DJ7-4eQfYFF;;JiOB~6u;EZw_6Ate3|&GhH7 z*D)hny{D`18(U9QKSNOtqqCieUM(>|6%GhB)km|Q8j4mvRB~ z5S^Q9&P5*@T8GjObQxfPS}Xmz0Y%N4X<=#3?UAl^!&x$x2A6&RUcKzT`yxrcmLH|f zJvM3*t<{@N5ybhc7|dy~mr`}8V$IflAEYkYJ{TI6W!@82U`%pE`i8sO0DE^x@OkXQ z<3?}0Nw}ro6)xsZo%O5hk#$JK^g2T%4(t9o6JLk4WA0L&_SryD=pcvDKC_SAK8F{r zPar+4kEx*1cz|(#XwzJzFj|y`nVKGAk-#Z5qm`;}4Mp-3BR{+2K~Z2-C~CummmVoO z&?uX;?z69I;`ebyl8P#VJ>rFn$sD#jy?5YdZq~UPtn9`EH`onS$6xVHti6)1_pYiC zBuQ9AZ$zPP=xti_NrDdaWm$F~PHlxMEF3Cz8#L zEE2p?pkQpV{N^4nVZ5(I1vYIi9DRxwJWsBXuAJT##F2a*eK=>3lK9cW` z){1Mfg0Ad(?iZkRpQAX3wtShOd>xuN1Ask{VG-Rh8y)7ii8ph;6RhHT=y88O(A?Q6 zlaDtR`0qf9{*K9y@yOA;J#3~mh0TXZGcXDf?|D+i8GE2hP_pr+T1sJ$DMU?=t5*BV zoRig+_Lq-A#hA>VLUq>NGR*xYb_x#NQf4u|V}8D4IQ5HW^PSQ+gB@Aryg$3;f9ii3 zt1t9KUST|X!wn5Y+F%qDJ`7<9WZu#j$;`2)TykcyJs^2!6sU(Z0(OB6=12^pOSD(6 z7i5=Lq?`OrbIHiH=$9FnW6QxL8Ryp1>c59M)o*U08QcnKDPLvdARVH`9@C06=1ut< z^;FW7O?EC3I;1BrN_1yYDHpdE*VcON1Ncw;9NQc9^Y+b|N6^=Aez|qB!jfoQycw8n&Cgmcy_fh<;__x5S(#A%s4&O(yj-NTuG27fh6@9Yv`p4$WFZu89 z(|CTs9%%YNJ)ey*JfE~a)fkeFiDAa7{Vc6GQiix$R+*M!*yM{NFCfn?i9ND178!B} zkam`-ipzD7I3u}1Ti6F6-bFV~du}H&tvYp`I~#p3p!bkN33_$LOIJymChCVW^Xjh=!y)~)NX}&U zUrmN9GS-?{tlc&!GS;eGqQ>hqU3Sw6)(cjzZI=?|GCIXpD`qB*hT}bN3SMB#dCAt_ zDueZ`pNy;3S$=8-wG`BDt(L;(n(y4TUD+FS&cbkxzW`P{bpQ!ip;^` zOxEJ$_93I~8i2|nU5?eK>Sc?KxJ-#ThyvA`sFSduL-x}yM%Jz>&a-8+&3xsvRGnJu zIzR)Z!&P(R+H}fE9Oy!nw=qg+){n!?spymS+AlfVr6j<9jY-f$iU21Mly1@)W*v=? z;RmYFX;7i42ytDA9QUU&Tcls7qq_q0D zMAhA05;a9;T&H-L+O+=xEUI=Dv=3HkepOeaUM6U?V(XEH15LDxt~7|Q+TcRcSaIn3 zXsNMq4|;Ra5l#ZL6JkAjU_L`TaOW#tj)^N z|M=1)Hl03f3sOtTDj`(lq5untV;a9ZKLVmu*)RcR>!Azi&iqG~8e{<+{ZEOuhhE=R zyswCblifkb&IrKRZKlv%=2eCKEQQ@7CIzoopr=a!543MGFnnQ{=f81j z`oLL-pLl}Qzu6J{QSj>)Ir_I7$yntMP!MY5kW*(-rM!|KhfrP+h@6P?f$o+lA;~!QWJuboYjiO8I5pyF;MYVvH!5ky!XHOjES@kOb zZc7jy?H()5Vn!FnVM6z0YvKUViQ%n;eh-Z%Pa>{B$-ocxy5xFTn?st|eqPl65a*zW zzqheyeKz+2;xw_yy683VEZ9XCXqPY+dZ_}~+5;T@e2eat!68ybQ|8a?h$+!2L(t0| zAB&FwPA-OQCH|iqD0!oycL;o9WDsiUo)asGe30LRFey-R1!24`3V;6tui74AtcdiZ zJy8DXwEsjB*S|WglA-&5iqmV=wAGQ-F}9PZOe8R>L6nE1Vk{ z=tCMJl_ey#HY8L9;tC-s6P7U68s{@OyG$+ZHO0w5Y$l@KI1y4f3@5DKTS3I$xuCe9 zxTc^~tOK&w5^(K#orh$y*IIHLcCOcCQ1H~-3u(WF=d1)WaKE1yoeJVB1j#%Mi_@>ZWZ<9C=E=Nz^J z{nnVo7<}rW)b2k4BsYgoYm(f=W=+Pl8UU^{H5-wcMcg$l)yi?huqk2E*pVbpDsx?* zDNO^~bZ%3n;k>isxwDww9hVo_x@2+Yfs%5g~^kXlO8t0G0SUbg_*PI6#4-B^1={`xwoFU{AHz(We* z=r0tIM68NYAccoc#b7eJ8_WJ`gdzz`Vs2Bt(yC6>+ZKjDP}4)Eemt2)?d-LGlH^|Z z5i{V_m@pw>^hnTkVz(6{wi|E0;uCi$E!<$lN7zTXS`SbMcGQ_?v^IcKchzCO!_s4) z3*qJ3QD|@{_Ce>tix zQ9qzK;KD{e7lF2IF+sQ16NP!2h3ngw9fV7*37_sHN^VWxjqv1ewukA0{feWH@rtBR zxz^Mr1FV0D4Ql@VqAkm(##`n%OBppm<$j}in>Gg5(wel}^f6A?V9FYcx)ec8!G@r_ z)4!~C^gQV8m~Y8gUwxI~Lel+`NY%J%e+K(kQl$%Rn>Sr)4o`ctPiv)O!lh(!x*AFI ziX+%eO{irhiHud$T~V#!a$mBUeI^U<4ZU(f#$?u%#%%`ncxsGx*^WCih4KSO(}+z| zL0jT+R0M~p%k2=_FCykYdNQB%YR7bK4rLrplvkJ+Zp}NO)bSTe@UzyLy{(Pv+qG@zlutnI)%KF z^hB17AD6}}pm_1Ah37^z`wo?R^}(13Sb0(R-qQMwCuy|5L$SUA(tZouZ+jk*8NphK z9O9T(vet4hkBiZgs$7_m6qKGc77PClHM9E_$kq7-7UWk5Ev-98eiX1#C|z+zfids9 zi0dWt*N(H54AUuq6s{1iJo5k|5NnuVpEKC8S3y6`5}v-S4JIhm95R!r^swblvNAkL zUuxD_HC2h;H7n(v0I0nL zjgCO^E6M|po2x}T0&+JAm=hV8$_hOdWDuF)oSMKKL=f7%nn1(@ES!_zplIF~DGyMx ziF^_;+dE?@WA7OGo?{Ba5hleB+u{8n=XG>1uS7L+qt6 z)3fCeM|8PQ0K={L-G5}ATcn~o%tHVG%>R(S{|TwQimQW-sfekWp{vdR>+h&lJ=*;z z8Q|MYzc~}eAPAZUL<7O739_a~O_nANpmH98f`G-y_9q))dT2ZMdWsG~M5Tp_h=9rp zVX2CxO;)=yDL|u-ucS4ezw)`;ojseCo`bjil;mZn^X28!Z};=&iv91~0agI(YkL?> zlTicfR&Y*Oe*gR)k$xx!6xDMzM*(3(upmstTS8d5gGNnW-3THN@=!CD3QzeyMV@-2 zV8yS3gHyqtiha2rk`O&LN0~vO9wNB-=zUej90WW0Dvr_vR#<6AtwE$7y;!jx#G!1V z{TQ*vc9ovmLcH4uLxo{Y!uvXm8B31%7nL6TA#f_c!UJJce9AjhuIrQ_iSp|bBe5BY z$IQl%-0?L{_Ov!-O6u$=#UfP+J11l)o)0`_h1~T`^YZEKP@N5x0*BFjOLinr&`E08 z;~Etj%cIx~6cnH~W&!19CU{Icb;4Y`_9+)7!974~>tlARJ2sY^!t!aiB(qZ&EzRcY zgmcE+ZLxR zWF})w;`T)uG$gZfMgy)!Ts0}M&fzOosPj*`!#*?aWM-UTLT007)`#qCoBE)UFsYwN zJ4n3Yq0Uq)g-|^$Q6jxbvw*@GKjYts%~u+u&$ZylW^#GVcOfhq{@%|Jj{b^R8)iyHgjOWg15nkG2awgf4fVCf!k_GJ2?)H_ zAk-Ti9Xgao)U<70D^awJCBu6Ve~vsGEWDyxpb(*Uw-ttHY!m!8z2B!rdR_m-J=)bH z_GGw_$ytnLqvBnXd$lliwopWj$tzVJU5b?=c(xk-z(*^sT&p=DIOYK($@DvcuW&Cf^aSH8B!?d~C? z4s!WL@1AQ!yV?M>AEX4m9TMD5aNEGxf>P(Q;2J&yf;B%G#EoT-OZ?s@K0|G6N?fpB z#F1Qoi-_AjR7OR2{RGTEf5pkK>?kn?DYk(GF>STxe!Gri@x)YD@hLo{>RoEY!$z6P zJ)l);IGC@YFd(rPPb#x+VlJr_Hap^8IeU#C*k|3h*!Mu=hFVmUK@nv>_Hje5WQ=#RwV4*N@U1ltWQ|yT3og`kYlscke{dHyq};SD&i)P_ z@cxJDBrT4Pr{3`Gzh52%dia*2m!(7p)%Dl@E91+MNgHxlZ*(Hsf{OIAT+}dVe603{ z_L=v22~M*WM86kx_quxVf3WrrQJO_jwr*tDwr$%sGi=+oGQ;}Awr$(Cm0{cFi#zVP z^{T2{)i|Bg-dJnRx#zd(nm%akyzUS_rgIi`uV5F^{EyL6c>~{F*2W4tAin}bt}64j zpB@4)H==WIMQ-0w(+`M*3cUEC+*c-H)-Fg^xr2(J3zLG=56g7!WL4R(-M%Q!78Q=m#-Q@nv_4KPxKTU|y8lO^EFw)#8z4iRimo{`A2)Uy0MF}2Tfi_edG(%uy*a>c;+g4|>1n|%o9D!TCxxn|?L>V+OZB_+UW{ssKjx!$uC-6@u9JhIc z3-odcQwfGM-@36Hz_5Xu9LwBzgVq9;k7Ov>gG5_(2uJPXpbt0;r{&*!^$lRrOZxbvlN&PHP8x8aP7{=W`B> zI&2Y1j%Yi?4Zuhood~b1@p1N8>c=F?Wj;>fWu#DHUBhU><(B0(;V?s0u9P zNOQsmbuksx;aB@b^FA_bK;|hI)z?->vwQc$p9j~Q=8o#C6vtwU{2UvC1s>=PKz#y1 z0AU+MHYFQGRlPc{mR8L}B!pl3I;g(8gjQs0D_LTyqt~aA?;>B0#Dh#Sz`Mux4_3ge z85_D$p9il%hFqfX{6i`TENvvRJ6HkFv_&z}FYM?841q^`BOby5M|%`Ho)&BqPMETd z>82;U;}QtPprbR+esxKuZc!Afi=DGBV2WTk8HcFP!9I}!LE9XYuf{KY1A0jOEzzvO zAL_G%-Dnp0+GV@FE07^e8IaE_w5obHiCkpSgVJE1cWfWB<0!1c!7c7XdEkRs0Q3ZO&-4xS9~G=8=SZZ7-_wS z&f|?P;uJbnHrcnfK`D97@0nhCf~!B3yW58)@{Rn{HDz`S-voY3ozX|tDKqA*lhrHX zq2IqL6J2eJ1UmXlYn@0)&GMLExIL1WXX-gO1sB19%X3TJ=qT9fFTEr`wG-LPN#2up zEosFuV}xI|53j${Jjv&OtELEvA6sF6q$3y?mvzSeJk*IUka&x9uhWp5FP?=4<&rB{^D87p0)!9nHt`cHHCl zE$}_5_r3y=0>rvM!*3Fc7P!AZlZk+^pNrtjsS?hxTZ@T5=O8-J;#)dcfaW1P8S{RF z=c8U3{pJX(pt2)HKz);l@Wc0mLn3;r47pa_;UWNj={Df5-i6>_-1Dq^4XAy|pS_C@ zNny*Jy%B+x27#ngE9npM+D)Ey#a5JnRk_X~2C3oF zW%YukfNP{)*(~B$+HPUomff(EV#5HZiCO=%rsJtrN|xjBYq6;uvJwR9VoYRdF8g>@ z?P|*&SHy4sNXS0hH5aZ0v(y`d3%HWa2U6?T**julPNQ^3LEvl z4pSE+nupgtu*}bE=&dL$82h@8VAQnZ=$XyDhsTeqEX>#SkW)4Jicy6RFTb<1vy$^R z^dxDEP5l;oU+<2W8hdj?caF$2y&0m3Bw|S@p|ft?*uhvtk&sXWQ{C|Wf9>8=r3=hy z#7*l{`EC@K;doZgC7DN1rIEQ+Cfr2QB`hhDlB1#@bn#ijUPJ-?w;k3I3(L$y$qovJ zjBz+w+2JF_fF79&={n5d8~%R_|8LXr1@=3mtMRyZWGKn2DTjRbVDKfu&oP z&MITb>l2H<4(#bu4}@zA&YJz>m*D87R*?~1jRAbO25A*Wds+lt`H(JK-)ZT=$*3Qw zb6=IMuO<0N6qxs=og%Lg4I1skpJ9Uyw*c;~5iqy`Zh9lp5lV4ZpWJ;BpF#t068$7o z)PKxMhYXHV1EzTDH+9hrB>B0;vNFGGZOo+)I8RplQq$NxC6Sj6Svci!;QWbU{itE- zFe8nAwi1ygdQ-TdU(vq2kJv~Jm|4y)^|`f|c?9BC)2`yIRPAT9gi_rf11+tKf7fy({1=I#Bw>=<}9wkqw%fG znmkIdWWe=|sp%@U4{~IGlZHsz7{-2PyOgNo>2&t0A(~3qKcX6|*TtHQ`~|&;(dXGS zW;MGn(L3x@)`X(mG8Nu!ORY1CIOwU+fCh5R34Plv@p;r=3dx0%!N$ic(8;owm<-Q> z;vr2Kl+M;;5HPlNQnC}~TGb5hwHk@x0n^pzbWw)~45t|pYX-h9hD zv+H30bLiIYt3Agh_`ZJ@GSRS=z8?=7P6Y4oF>oi*A#mfpQfrbi+Y*S}jUkx;zp%z1 z)z(l{z8YZtA@1o2iwe)jYM$L>=5L!9Of(P;Brk$31jV*vVxOfi!xK1GwkRweIO8Kq zvo?qBU1;j&5bO$bgaRwjCH3C}Jus;(VZ+l(w{56a*u$OY_4m4+8rILOx`P1|2N}+2 zszi?@N%4Cr+M7to_hgT-tgeXV@4)Eeu^6uGz9DHu@$CQpiWt`GG3J%Yx?lU`jXWQbD8IJImjjh29 zu{Co>5Cq`QbL(#F|KuigUvO$FXX;S&4KTi<`CcHN#s~2Y$~kT`97tT-%Usd@Hz$_3 z{BrGzAaH}Sy+l{t>i&R4(RH*>x%}j}aWusMk zX83{SN^KI{co`LF4BOj~yV9k-E0XXbyY?V;U(`kNE!mB$*)W;9 zcbn7168}}X(<9tGuhXMkb>~J)*bM%1jNrjvVq&|iU%tF%6FfLvm5Q542o?TAKxbB} zTo7Q~q*a~q6yctzk9MNGffi_;$hcf6xX9LbqWSxACm!T^`$XK?f6QUCAk7?jWpfCK zUcfF=3EiK+a;b_2UnJQWU)*6nUWF+mLSH;qt8j>JNZ3)uydT zNPYezMey&t_6Dm?4e3HJm`9h+uscUy zNkD`SS)0>bN{PlqKDs%RH@iCZ^8)h-!Y9Okb?~SiKW88O_{AW8RP6t`uHs~G?qurh zY-nWjW7%=|Da8I?#ATI|oc)3zqHhL6OSR>>FifzpwvZxZKDuzGV3*1nSY!<7W4ELD zbn>8e6`rU+Ix=M6UlIsERHJ34YAW1_!9JFEru(C=|tf<7e)xQs%D&K!shDg;A5HRq~d@l}QdY|qPplUO%-I&?G z6JdDYTbNE?Sd~BijZ;xqBOtf>{_?Mzu&;!33m4Fs!@bPr+i4x_kjT8$?RbXT<72?S z;cU#OU0AQ-#AGr*fg<@oqMgS7w6<4r zwUah<`bm{?{Q--3${E@jn*YbjQqfgL(M06qfRe%-1`YfJ`ujsGfpW6G;ub_eO=r=< zg)X87-X#2Rt!4yOVzz>uxlE|3T4e)H3D_;#FN+IWmO5YkeCT zA|fDb$nNCz#;!XK#6PcZE2;jvl_o5sL-;J;t0RrIE0?Bm7}z|XPhdlXluhT#*UhITzH?cai~sox=V%=O7`)a7|HM?|X1yGqxftJKp5rN(ydQS9*jTDRIlVth`n?)4JL z84FEose_3CWy~f=Y!vo-Ot~F(92pOjeWvVH&KMZ>Va93Ar|T!}Rvoj!N}* z9DacWna%)LHynwNlu_8p;f}uO^`E&7l!As~{Lon*QLZWFEH|M~kzJsC#7&Vx!Y4$! z!6!tTPLT%|rQm1G+2ZH0Z%c>i-`NuJP7$xeuRF4YVh>1LCXUHXlRLI;%vItge+lng<5OH|k`4W8n_l5cuujFvb*eJ->qEJ>?T z5mjw+ujDXiz9q7bC0(=Pus#oCZYf%$!7btf9AqllE2KoZ2;ae-Hwle%2+Z--cTT3Z z7!~rV9A8+1odf;NC}Ygr&f%z)w5qU#R-u1Y`1C{cDa3ad#e~Xy65T8x)jL+9NehN- zy%EIzE+~V5TRViTzw}v}^A`y6UmP!eaedLZZ@2%gIbZQy_!aT9fun!qj{mW@Q`yk= zr!`k_aQW%Ky-b}*fB$snHa4a^rrL**HHa!WF@Woi30&G%D3O&4CgcK>d4ZR$omE@L-_pVTc#3UE?M=qPN4cY zIqXQunp6&MaymJ==KOk;X>ay1lGF16y#ua=j?Io~W`jGh#1yrkj+6~JZ#5j`LEmb+ z(L=iahdBVj9`Q1JVh(s2?avI>hUH3PtR)vw3@1J4hO~HP8#tlE8gt3s6>37Oh#o3*%-cPPj6r^Tls|G0x+Sz_PSA&DG9Z(rIX( z{;d%P(#N<(oWk(l)%+X5gNIbE4J~1pTYbI~VqADVDLt>g9}PhSiGDCn-ZQaUhHgQF ziF1k)*^1~u3x;{)B_`FIR})QW8^b$y#e)HA$SHn2`$Mf%CbTe%#K6oSmPSICBxF8W zsy-(!@BaIZuo;D{v3O=8{CDQ8wY|X0-p#*EOF4c!`*ePrJGv{R)aaItEmS|N=JNyy zksIz@#?{Dw-IB7nR%LXLXR&#&Rsc?`njzoANHR-G=3C)~YOEL7CX85V=5gq&&7#>& zvND%9Gz0>uLj&X%K&ra~FbaHhfz7TlM9xlhph8`*!$&UZ6eSv7-WOfpAF=^C#tnOM zZQjJc70mT*!xzQ>B9j~z=wo!KQ{SD)HUO;zVAWANw#}3U(H&SVaPuAT=yeA>EIwDc z;oj(1%FcQakKuO`{T?DuhNEvXr~;Hn{HpZ}h8pd@2ha1nV^ha`@g`;eYO+6FR$N8f zsk={MvLXz6V3eMj?=s%gyKC-i?MwBmJ{JtLD)~!vWzdi)0_|sYe` zb50&g(*?XvO$Nk95S5hEDsX2-lVhkgFxlEj&Vgd4sq)rot5j3h4KwvnNtuUK{a&#< za#)ec`_>@Ke$TWcFl;LF>Rj&9IEjL_nXta+0mjEtMP>$=%L^=6;G^FYCV0xM>Hc{8 zt>G-S-@yO%N{_C7Q_MsJ0&=JLUr0j#Zzsb4ozrz`di&rmxqeMgTA4RtTaszk^9`?B z--sle&>`Gw;Xv3RLdw*~e#FD2&RLOGb#*jcRNGv{veE@DN)bW@1xY}X^P-m6_8QdC z!uFAZj+2l&+CQuD@ zM-Ie|7Ele$fo#SyR%|<8~DW?>756z z2Vu|UBNR4f;wA@s3ucd&7rU3>;JL@-r2zJN;_o-Q)C|~-z^@OgKM24tjDNo2_DFD{ z5m|}VQW6Q%Z}?>>n!hD=viyGcKC(Y-Mj;(6xAz-NrL(+jJPCFyl-I$_IwGv^Ad)2ShpxM2@9cGTvf z4rQ}yj)4Q3;nUq9^_7?EsK%KK`j=P6m$Ezl=E(KTJx)YXo zFInJ2YofUjw=q|Coc=WDtEFQKbbVJ{YV1MEW^X>Qu$_zPXgSXk^IA~a=~0z3yFBSM zL&;ehuetC9r<`a?b0Zo`Lc2#na(#1KXG!#M;^nE3PTL))&=X}iEn|nLk{B-07x82Y z04kf0vCNdejUS6Z`){0Jl;ttcuKK*?n2;M<-eIdrM=NjX-_S|v^vGBZ(cH-UqBU73 zY!_LpC~`!aIw)EIStkz1s@b!dQL;8Knz2KvybBR6ug#}kZ_+k3do|p@Fu>8_oAg4YCEuW2hur0$}zZ!Gazv?+l?kxF8SMI>+ z10D}%<+R$T?~+8UeJ!l9P&vbfRi<4J_RKxgzD@3|>HG#)?$CU0x;)CfXxB1l*D_<> zGgAFCZm<1t9bVNHt77iU!VC*yq5g)%X)@7hm-3-#^9DDo1E%R(GqpZgbJMsAj?T5K z(tY&yHMa+78O77aDE>Z-1FL^#UCRFkfH&19jm*+)IL%}Dbe5Q95zM_WwnWWa2aJfY zMqqSzbz;!ud3FU9i2b_{&YgHUk4$3H4p}zJ-D?{LI%6G~$9UR=RiO2a+&SOl-G+^t z&8R2mpeE=JZVX!V{@}Q@$4)Pm>Xk&E7y+s1DjmixZH9o=lVsIOR2PL$KAc(UfuT~)7NzfB56x-M=#MdZZY$whG}|&kjXRrn z>7}}1uVLJ6FeY5A3YM5F=PauVx45-rHF(*f2evjhGF(xb$ukm$JjS8OaahSckPD%R zNw9jcl=zI!11gnar+-lhb=EU(_7hH8;J}s)Fq$Gb)K=GHSFMy~ zu_I3F?2_jHZr}TfeCg7x!)=&w+}Y4Uuy4Qx52TtRmL7T2Dgn5p$)_x!VcX81N*j6P z_$HEvh_IW5HW~@X)w+!$E|ev-uC{}MEDRcXk&-&DS1zDSpCpLHCmu;U_tYN@j@n>5JcL;ba`R}+s~8bdeWVHQnG)6scpv8`*?%%Z8f+z@J1 z;HO<_IGj{C9wzF)=1KU6?{ndOI_i--r^@}jb`iEL)$SuUmRCczB;nD7>5{haAgSMS zIdf-gNSUx1G8(7D++MW*MU#!w*;1lLm-#Z(=JP?_Bf%KK%Cqh9Vw?T9FU1L;8a$0N zsk_{`0WHCv6HJPu5GHy~=EE%Ewi|Lv0DOkK+46!+T{`#QxKvGVA3cF-;hbf8ro>nNP`kSpUe|}VCfgQsTtz)=9=ohX+sXXNQzJzKy@bCE%^rx-i5&|6dXC7o z?{smV{HDF`lJ)reWe~NKgK($splKh^3@3R_w7c$X7nrVq_3cIQyh1&{Uf^yuMnh{J zPH?cQiH1=3_v`0MN86{;!q5hsOujT@>syPjvM`(Hj^11w^T!rv=z2vv>&TP=c zpV-}Zq>6CFFW%S53^N_Qm0mpz(6a%{Tw~XnVm2Z7qU7C<#`~aAI#hqkQQC|VeAIh^ z_&vI5H3bw9^v?lUAp#O2kAtc1g;3h+9`X^-%eWFB^fkzs@Kk+W& zmkRZ&as|9Wd4Ic{wcZ10vA5^|89jH{sbbFVgt0h2g(ytt+GJ_ZhiCy|jAo0m*Q<^F z5iV>81l;$+)jjU>8cPRxOhSHo9+%h4l~YUR1>PJu+g!mf`yQlNkYMp$CF-lJ`D?`NZ53VdSa!_az|aM*o{0Tq1seTIEv;&Xb7FK)Yrh7{31V&Ro& zB-|#Hb>krAEijL^nY>Afn22~Cl94}Lu3BGvw@6K$1zn8TyCMeNZ59@@bkyEwSSHq) zad8pxw@QHvP)S)S?CaIg9SCPvRv=pq^J2`@{346kpNEgU7)cS9(UHp&lg^51itLjf!S0>1m?=1Z1shXz(`0Uyoc03r zH|Pou=z-#%iCxFnJT5gzcZsdDhe|h?XPOpJ*Je~j<%-E8VxBr1!}zv4#|z*NP!dCw zc(P{X)el+AZa{#`diSGWtW7vz;+6vG?{(B#en1QVOsykfQ?L&|EdcUN#bH6sylOJ4 zSp3G)_|{gLusS$ZjMO^+XaLePPnyj4HaS|l#G;==TE01t)Fa{k?_NmgAl^@4lZE8mWvQ$bl+wrNVllH15#CjIynCw2yE4>q;4Y-_Mll zEz!$IyA5(%G5uE8ln(!Q^hlZ44jT6WdRSRLwk1g6113ZzRH=j2~B8gz1 z=}?Tfx-*I*QT^8UknTg!6K96gVd; znSB@^BMOPD$kty(%@ybzdoq;fk>|IjM>@qe#^nz5rnaupH%+PKW=u1h6(%%iH)u!d zeQyw&S8}BWa$RXUVu2}CGj>PkQQCI;kot!x&{m7?jxhJib;7<-`o?YitYtHM=dDQ$ z53XlI)y3MoBU5LtKK^8{P7@8sR|fRIc*~l1aXQr`+qDMS$=Y6s$9XaxS8^q(_QQCd zs3TA1HinoZlFYooJ2b?@R zp5=CN!L^WXK3cs;W449rj{k1jR9UatHBTctq^) zwPLAeR`~w{S`39Qg={AG>v`!JdiwhYeMM+!4)f_X7Mnv`?Aqb*WLFbMlIqX)l>RF2 z?9M}{akZL=e}?GKPP8SOstXeKud#`k;d`Mmd!f-g#hwGYML>Kx)eT}gQsH;MS$({R zWzv{PO|=$hjz70v`Q)`zTxU(8WBg6FI}t$q_hjFImR@3+j!1@6MRu+TaKa4Y`zdqx zU{re1u8yHsTGL_bA_H18A{=9-Z*Y^i;1&QIq*ME{uRPI3F*x}^-4jdWMuTI~Vx;x-)Uj61G7n$0<*W;g% zEf;n)LvS80m+zKDIhNj>QqqO;P79r|<(L6p9g!>i*Pnghus5C-uwkbH)hx{%^XOX?0($+ApFW6&n-;M*V9bL(qf?iU~(3Ha}sL2y1oH z#@s#5!q&E_NLu@%W7TTNYC+4UDQPGS(u$^Ai=e4{yGg5wU%IJEzb84Hhv$8?pj4mz z=>2JWvdw+wb;f_X?RoO(>wC)$;?LSkZ)DNx;V_z>!v%FOvX2PtHtImj;EgSlj|A-Y zqY-^K2lV#Hm0~Nh0OA0`X6rbhqKk5no}ubB4re9o9G{~aZH1u2F{@SYdRSqwX$6qe ze#??~3Z8d*YSCECFiX$(QVEa^@N4jrWXU=o{HE@z@IBa|=p0RFgql`7oZmZZ#LdezIWTOveyx^oYsHDAqzn|Gd6?W3pNh@=P;2ybFv(as zor4zAAB!aAaz0pET(Z~#tKci*?`zWE^n0P{#V7=a=02+)*gs z(U5qmu8tggrM(G8rNM}DVz39b=_#`QE7v>Ua)%(wMdMBj{S~f`#|oQ>O&rNlMT>|7 zj^yjsI#+WSW@G<{}MN z)-Aj^h5F~w;+1^U49(3OqxVy14xa-2_~W81+Qmla#NH%Af0j(3uK9@5lrM{xV2&EW zWD7Q#%nUib3kiQ4*~+QUugbIGUu;LCtE<7;Ns8aO)wm zngp(j8r-cCdj*R3!H2xx<@9~;qeVb_0HmFqA5=e2jLrOr1(FWpOVJkf@UY) zqGp?=vMlDIFmX3TV9SZbB&<#T62%d07J(X;le^Z*o+UP%lb?brFR%H>sH&g&M6qp? zV^K9KQGd1If?Uq83sC&QuRvtjHyFN|@CKR>-DE}`-v*Q(VwiRYrC_NYS~JAmQt;C6 zqkW|1?G-etGwTgmGk+0>tPCaJfNv$s}aek|n@((~~XF*viF_?}7S+ddYPXX<2}+KAV1sv>xL$ z>M^d{-wF)1=hNm*z>(xfg!}I@g-k1)$R$Ky&<3&Ll|9XhgsNztJ(|gW-^U#_dU$Em zpS$}Ub>g5QBr7EdI@&3=j8vttpJ|b9dQnQzj0q9?Eg~PAkbh$x;g<|eba%npu@+*s z*`*b7V5J}fki%h{lH@=p9E6q#<8;$tz$;sr@a?-O>O#~Hwp^9_BM%_65Rq){> zbc0iDs{6Gwzgy&L`^rG=q*zzqXtZ3Z3G03LT_LMCKqbs-T)H@c)a0Hv{JJMW9?7zH zbz*BJF=u>VuMvj2kgyTA-ndL17X(d^Y(NEs;ajR2^ilwi3`=twT@W|C;GC^KhdJu0 zCr84Ty!>3cT{tI`!)8&`#&UY)#Ys4~q~AqQ`3Cgz_UUjDnAqj2*5K=)Inb&P&WqTLNxS$3YN_iw4!d4A4ZJ_H zb^zaS1!`co2~`STS>nyb(vp6pG;Kq;#Qr|t{0A?1-__F#xA;w(pC60&(S zH)RnrfVm*Ng&NOc98A(Ah5Cr^O@A(gm3h!7a#I`1l*AXH^+b78Kf+vEVnoXikZWv5 zTyUVDZm3;e%l2SIi4gO^c9otNawm$Hbm9BUzerf((11sZvdha^&~}9Xz=giD5x%!c zSy{~$wMRNkcSS~UB^pjgEtQ`r`bbLMAZ98RMKR}sELSUr5>H8?2o+AQ7tW|lei>4M zfm5uJlDkt%9zOP{)aDYTV|z8!?^K4MY{An5UxMoCb)E^~(EkR)85eekmXkQBqUG;| zj8-)L$$9v@?cij9mHx!$iVuBse&sLJ!6M4eg;eXRa^btBbdWh?IG>CQzXa3`@7ybJ zaverpL|2g86EpWJ%|rJMHR}^L@rA36vHgzl6FI$^y&;7+j(V-T0|`$A{}aBQ`L%YQ zhHRnkU#it8dFWIKjsh5L!&ULe+F)QprCkH^E_f{>W6{@NHbp$yPzqMQ5LGj92wl3X zyJ_;w(vTOte@;ckOq~INqt0ZCu9(3}$V*+ZT9SKp^;Jd7)6J(%Zq0HSs3@*%9Zbo| zTvRmHT+3WfPx#)M*)Ugqq>flt!LbB=t+;8Wt-vFCG!^6q$YzM6?2%Ez0dK$JKJ8@< zM#3(>4VaHhkKtJzpy)Qj37>@zne%;EW%Yva^`c~6Idz6*Sc2q>E8qhyDoeP-lpduL z_nXv`c|f7Cgx&X880m0R2*uAm1l~?AhpWfU;=|u4(F&&|jmUjJ9@3HqZJa=xLG}dF zc2Jn26`Fe!i@Fn;)TJG{{RN@saH4Gw7{w$9OXE;yq$qAJ;(CU1)?<_rxX{DHUHYfd ze%py{=IbRkp?X%ACsZcg%M`7Da$e4w2-$O?UD`}G%*l!T%Hx45@B|yaKczfYO&$A8 zoiaCouj%8}i$wEj4*pOPW=1He>P%#W7!o}JL-zz=*<(8r@mp(_4T&W-eU25l&m`(^ zh_#P0M;ANK)OkRAD%rPr(7`$ikX@>F4W4DKEmpce^=n=`Ew+spIZ?D}OSlT;ZRsAg zwND#!b>=QAU!mitv=>_MBt9+6Qn|;E2U{jZ#%stz`IQAe;&*%$l(atNQPIQ2?alu| zJudCnnJhuo7xe*Su45F7zvK3J;f)yb5%$6^e*vC6lwr(aC`{Ck5cA=}em#Y}0nhAH z&rPZsB=3ec?8eUUM4`{W*6Y`<3(X0~@xXbyCA8Z=zai9*p?Cr39pcy(^P}H7INsIW zG8}pYUZ*>|ymOfI$Dt7m_Sf2V;MwZkgIo+;&Qr6K3oC*MC(< zZ@??cOE7?d#`*pi3{U^L_WzHL;Qwl*OIn@=Du+$qSv>Ae=Hwh>vV^0xt4O~@AVo!q zLx$>6U`feseu;)0kYlAz2BkrfR<{GZjPcUp); zm>VW~>fAbcJ807tP6)tCX&8d2Igdqv`0?n8XrH?sxm>=H<+Ssx31B)!j*OXaNJtrv zRj%L^zFn|j#+p^6APqkKFn0}m%F&R0YRMuS#@Vu)wFsMT_Cy#Wsjc=8`uHrd#9C}p$fl#kz1j=O329P0tl>LI^hOXQ@~a-TlME@`4>J47r43+YN2dd zh(dZCz8S*6| z%|bbps}nTdQ6lNDR0IYKmsuj*aD0AIx5XtdPEWN5EJvbvg!6wuVj`cfaTE%^#Sbel4zZQh#%hS;XG4M`?cgms$ zUL%th?JK|6+*XNb{>EJ_FUV_Q%aqmYCNpXd%F^&7v2Bi}4s4gPK*d@(%K*RXe94lbvCtzo7txrh$vKl6 z1sNslU$Xe3rxx=3EHwc7R}~CL=&Au%+mL~CWheciERSXZ^ZcVPTxFL`ae&Z^Ts9S; zvdZ-HGzWP@!GppL1EfHz>oWe_WUSnxB$ij|FwCvKcPw=r_FH_+hOI(>BprvTPBmnJ z316M$V-$b0hs6S9wjZ@_{-ZOX}Ixs)duQ(*PQt;;KVIHLSMx1nOqVLF{Z zEQSrfb+?RX@`N#8M61p)?m-kcoi*vk_ci(`MPZ}ys$BaA8=DS66z}^*!J%t` zH8h8WRaD!rAG*y0Hrq&~*ZB*?mlURRKOy6?blTW}JM%bJ1R9~*R{^F@jBY-h?;dpWFfc>KnFUY0clj zjGDN>pFn@{vy7bt+>E%ODKdof2UjoA5{Xlsp!;J7Y&u4tE*h`$J<12M6^L{nL+cQZ zlLxu=;yGjPOEm1BU2c4#jih+czV_$V^ImDC}-Rh<_|Ts=(BT2|nk5 zW!7F?AcdyurCUOLVja#20_b34{wJ&M#)p1&3@DDb_G>A|ok10bbN}*`AP>nB-^Jnu zzXV;psPeO0-dCWq7cqQI^qmRKP-Jn4FgeDQf9^c_5oX`GBBd3c*d*AezM6!*v z+@^)%#>$nCD09=YRDh&V*A<+x5q>1e9N61JDLO-Wci>>+Jg2JN`*$T6fw1?TtdW{_OWlG=h zWRitP+gG-%IScWN%w0Yp1^CH` zD$l1N3=g1&GB$?B3+EcR`6L8pPeUI%8TTc)KQ-=pa%L~@UX`=_#06GPF&+YY^Q}s_Dn7i^9YED)APZZp{h^eB=+S!Ub zea8ZNB!Sysj*m*`I$C0bPT7Q>ao(!rdkR)u-}qR9w-9wtGrnB6W=~$Q@sEtkN6_l! z?e{?VizJsun7D7z4G-~~{pRTX&QL^4b0=`u)ykF# z%o4c1p}aKulcQfUy+z7j&sK1E$p}!q(4yaoFT8ZrPk5DLCpZP3cIWjGT*g~S|6V-h zld6{}5?wr1MaH1CMC6P-^><%a&&{f_%C~|tm!jkxOVU^BQMLzUyuJUrKUe{%1zQ@q zxu0phYMtXv^KJ^5wdCf?Rz1 zt|l{pLbIVQlgQe|r$3amTvU@d)y*g!Zg$4Rrs${p0iTB{)0T$+>sU~bhwRg{uC%Sj z(BLB?qoQrPQcXKZ1GOy>ShY!9K>~Lb%ke?n3;tSPM-kNNi-HNt7at_`7kfsTK~2Q^ zYoR(xz=Slu86-*j%Im3xL0=T1-KLo=l2K!F_s7&%oh*`Xi|bUm^deDDGp<{ODO{l;)0hdg|xxJ#0EoX-X+yDVzt2#JBPL`mUWy=UXp${2Kuh z1iZwM~#hTQX!?RgAfp;|KoZTkB8h5&yx*ke)1r{c57f&bik@a^w_m+fGt~gNqIoi`$c-R>9k$rc<_HL?ooa2zidzQF7%_1z-Foz|#ZoWGf>+&zt zVaUj3S++$o_8|ynFwr}jL9$%Gs4j0U%qd-eH|8Z!hy*i?UCXYYRZ{IVUSW6c@SteZ zR`xe%LDnDegXd;qnsv6^(pNK=d17#5#F5tX@A&+uKW6h<=2XiiGOMzBY5xvX$D(?z zy0M_0NQ!NzA(zx5lp)j77Ect=ew%Uxp5NBz*AbeF9&Y0;elXE+O&*JNv}N8&R#|lN z`4>rv$Wy~H?yD@;2T$MiM46Y?5D3@R#wZ<(?ZP zb8D3#x|t#4kpbtCA*i$?I(@%gHkL`>(B~>hY8S)}+AwsR#SzzVb(Eo(E)d+cpqVSr zL$LAwsFn$h+Wn#3{FH_~<|3}+9 zMQ0X8Yuc%>VpVM0cE$W++twF5m5OcKwq3Dp+qRQGXPoZQeY!7C|6}aSeYxkm*4k^% z=h<<_?Mc)fGJxBPD({FA#?;-tH+9PdX3$R@72Kr0bwqJaa>7s~a;`E8++_9(XiA*@ z`$@hnlZu(jLmf2(%rHTw(Yi&^nvH43>uzF{doV2l?&}pRt_^)rdPQiMvS{Gn z^D7*GG{Foz7+)BPL<)2mUr4 zab-Y~)l2>kO6NvU`OHI4LP1Y~$03iOG$U`#B&snav>FMrdaa_G&E_UT?Pd<=$kH{b zhfuW3;Gohw%KF1@8|RsvCODn>26InI^q(Z#_NJ~^_`<&)%&rM#%3)%TM;2b3UFgPI z#s(Hht4ok6Jl4c?#?-Gae*|eiZ7BB%kbj?lN}-Gp@_q{n9m856CzdE1nvF!8P(Ma( z1ov_d)?OOp3N~?&P?g|1Ub!jJdTP)q_Uc!1MdV#zYHcqdA7D~gBE4C3Yde^PN1KWSPQ zlsEE{RZHIB{M9^#puqWdEf<5*Kj!w$=VGZ#j^Oy!Lz+NLKRnKEFuzT?mMXk6EYqrc ze;})e;|3itpriF%r*CsTzoNt97i%q$3^8-v9T}U3 z!euxV&j`!f>AZ#05Xee1!2QIoIX-%8N|w?nWFb>gPB%*)aq)GGX%7&9U|So<(1>9a?87T&I%~Uw)LPc+8YHOgTBFt2ddz( zlV>QE%yF5lgw+RTR7tt(N>K%e8Z!PtIc)xO@4*>#c+$S35MEFBjF0+!Q+!wF`cnrH z?I^{`W^xR^!RnBm+J*N(wwt}WM*l$e&1t@&;0|&< z_U@4O9eBGZ`YbDXPG`pURU_Z+P8fMk;d;nrh9_Y1{VY`8Zs~o=uS=6ykqN7dJRh|k zmCqs#zGXfcE#Ft*&(qd|gs*9}$g%l(uu?RKwC06vSzch!w7|NqDqg$>%G%o$IN7iv zNn*oek#IqTZL=o#(o|gX&n01Sl|Qe3A=jyjJB++EP`M&YIDX;o*vlCopipjJ))rS( zT5geaT6|&&uam<*eY6nCs@0irSh_Uh^*~Ooa+L#dy5W@DmW^H}r_}n2qEb>lBYK*y zT#BDh{ot)x@fn`9DmZqIplSOg-{=iq*?=)k{YI*3lP#0HoIHd5W_i12Wl_l|()b8( zyv$JS@uo7b-yOBMVwd;u=t*h3C28=6ze;!#9v6M5P7t)+POAqcy>V(%moT&gT!g|n z8`n;HK+RaU7Ur`0?CZD`K0epW+7~&%AOH8{E@KvD=GfbSXM6YslI5KSaAr9>K(?nY ztLSq>m5RD0jpp+=b3c!rUlx(g&gw&qnW$S2Ca6O&rLyI=ObD0Z5t!gW4eaF`)g#iHoH-$#C0vpa1AMquPNV} zF0SQZwTUqR13ZlGuMzsMMj~JyZQ~&COgQf&f424@@9CU%es{n0u!3GM<-3$@)ukCv zJUBx-Sld=WoQYoK+(g5qk>220K(Xx`oJ_N%|c3 z_cSx-H(6P1y#cF?RR^JD8O+uvocE;ot{;zUH$0o=gi~E;)hviMnAY78-A7Dl!WUZ0 zJ_M!&O6FSu9IeTIWRIe5;TALI4IU6$?9z#BHQ2uUOca$WX4`Va-punS)^~RGV*RKi zRfP~kigf~m?@ex=v;%8KUYcjTLn(&gZQYE{=zyyNa}Ns9IVNLLN`v9j0GA(MUExnC zXfv%D)2A8apaLAh1g6U`>?25pL*jqXe<f-uQxz#2moP}%%J7FK4coDJZ71qIRYI#g zCoFGl4)uwNYpgrQ(*d-d==+K^0c) zJ{+Q9R%?e!j;c`ZsMq{%g@jGCQmE37CL4t>QMK)-Hi11&NUzat#Y#`m0!G%MEzvZm zTBl5v?JO7cDvB`V7;`7yQHnD-T%NFDeL?PAbx`*l@zJycdjyZ z&aZG%q6_r`bLc`)Gzn;ZXhmsnd*;@FQUgc|NxC+#x)0=6b%NJfsP-hqQhGJ-2Z3jj zcd*m0cqHO$@~VoA)+4LQ&7%yS?yrYkHIQT=N0@8@aY_jBA7z%_mZE-R@FlHl{Gx=E zD1GuU)Af+;4@ymYVJx!+3)$48#5C2b)>3O&WMgorF*?V$S|QdERh>HynZeWEUlh~)uRVzCUr0J=Q zxphUOe;@rArJ7=K{iTd&HeHyHqXT+J(U^uzd#c`B7lmjqy)Dw7YXkuXS{$vdfTsa> ztUeITZKys6#349wgH3i`|DiSS<8}n0`NhZn*ktTW^zru1QEXbjLM z&6^(FWlVY3tH+MvfSMVd-5Bu`f;Jmcw4I--2|A^hOxbw4Qb+4u?Izk;rte?nvD?K~^S-==J~qe_`%xaa zi~MZBvWuA~JV;Lv#uUZXGsM)pd1s@41xF=i3W&yX>?!^#5mU*8OdW#nKZ-a$dV@%2 z*UkU>J43|J;;hNH)pL}x-d74@`}WUQ$V$bth?nn;bg0o$(*eVVx~Ld?9^Y6#a{wfb zQrtc)r5zZ^IT*4-eZWZ`(+gTCB%nDUKa@d~4f{FFyq~(iHh)4)k9>jY+Mff+&sM7% zGzySI+bRIL+`WHthBMgWU#WovJg{N*tVJlkh|viY+#*uQagYXEjx31c6<)v)sR;6t zQZBqREg*HLcAKYeH*U7QVs`d*Ew*>Vv>&T`V{Tu6Ve*%FEZPmx(o6!)jwC&!5@-3t zd{oU%NEUKbv<5BRoSM$@I?m==PP;AM3I1zJg#hxKvA!kmAW+|+_5W^4sfdV)S=d@Q zn*W#C?g^Fu1c9stw6WR31_pr^5;XV~394r(J%B#{k`S?={(-W}Dcvz&TiK`+m%RX+ z$Z5aV8!a@>9ZE(`gvO>)YjQR2_P#m3rt@}GZMXGH7DUDxkt(<6n?pFx*@_ zbIb_7ux~qlS=9u|rAKd+@sh3MlLR>4+FmAOg!K%DZQ3v+lAs!7!Y;hwi9`1!l+a4- zl>f4Alp)~O((J~V;kKSvOMrj>GgiJ)Dm?d@_xxv2Aa_rftfx)x=53Q)?0|=aK3!G8 zqj-Sp3!cX<3!Rd?!1k&5&3XP!KR#pSb zkCkUaY>uxzUXXT-Q+;hu5|l726Y|O-d^XN@hV~zFie#8tU2%y|Tj2$+gX^`rnNYRz zve+$f1PtVf$Kq1`OnmrDfQm>auR+fELt_H^`0yUD6<5)Mt!Q}bxNd@?QM9P}Gq02L zPp{Wh0!P?6Mq1ijWBoB9tY}PKz&*?FHD6-X(j&&;ig#zOC`AV{%uSuF!+X5fgojc2 z<*J4nKjIIBdnSmY#6n8PYdp#y6)7DO9Cd=C9qtOa!GE}i z&{A5kErKf?BTcu%#G6@B{v*`!aazQ{VHr&{{2UjK5^9$ zafAsDo@H+wyhpk+`P6P&U_0JQMGZdBs`Yi>e{8GZk_BrPMEIdlgU0OfWs=|1_swB) zJlpr{mFLghGkMfpig~}I8;xw=0%zW^l(k|{f`|i3s%<}<>0V1vvr!9uwxfeh7c{K8 zbuWAeNiZ426?F@mI1YK*bl94rX4DSfO9u%uePJTzt%IwepZqnya5#x|W-@x<#c@$> zOy~GNa;&r=Kp(JmhyGSkLqz_ADX7J&t%KDz2|k&KI}fLgXB0`66ax!;aN@5%XWAS* zlaj6;@*}Sv74mEn%iX?zJVn}@zJGfe4;+U>kEI?w_0nl>PTy9@L>2| z*)-ujsFL}>lrvUq4@a8@m|rqcRT0{eUMgG{L=)CKNead|3f~;=&uj-Jqr5c9f(yHj zNsa&)>F6=DZM#yWKL`Eyv)QOHhetKkE|!dl2DjC)g?(baLU1Q4xn=JswWhc;3t4G_ z!hUBKx((tMFTL08xQW zV{<-zxdEL7<%&Le{j!>CKTuG-3=_^3-(@-K7V6Pu`Rm*;fHl>##K6$A$Ut#R@ivQP z+@Ay%T;E>($;Qx!!a7}Q3H&FZ+`!TE-&@vol0pU*FS1&w=d|26dHVK4F>uPX#mct%oP-opfBt>w2Jy z3+aosqG%RJLaDBPe3cv*xkM^nDE30=O`Rv1GRS1na?xPD8I~qdkua>J#E53ZziXk6 zv?Y;AWTo-_f!h%9i}PfDHg3Wb2_2z!HSITj*j7NmfwP;<`Z+FtmnD|Z0YjTsM-U`=o;`G#~tqUt7mt)()BRg<9@Ms1;2;a^n=+ntq zG73qIB4yB#HC-fylrU=*Qkv~EVmkU;9xEC;gCCO#Sr`SmFeq0@$t%Sb~v#^A^hvn(%ymSKVy$G^^oq^tU9=f zwpKes6n6KR(Juq8z?jD?~#Oaos z95d8W+R%Rm48rWPHnFMX6j7UR_8DQaYZ! z9XUQk21;9WzbD4l&FagmL~b-gcSVe+?Fytx13@)u&XY}XlRJ;lIxZ_`0fIgZA~;_V z|20YX`=a!ILx6y=ekZBJ|I;M>kC_Aq6DJdS2RlOpLksI~GSh!T9mGw(llFhXS36ZD z9Z|(Gy-6a`_kTpxTC0i^LE8O9CWN#W2ft$P9mIr-s&6c4;$=)rjS!3So?;U(6Dk|_ zlP*)ltuv6w*iDkAE*+&3AWD7@qOBh4HTeyZjHGgM_Tur<^`!f>#3JzZ(MJMOZA9kB zE^}JU1h zeXE|&!W5GW+!CrbH$mSTbc4Pk7c6v#ziZ-K7V{Ow^$!tH_g{vTmeZX~7FOn?8%Sfs zN)G@UM~!9C;`M5xLl0RWS#;I_bg>huW87*0rG4~T4WOr$5Bw0aq$6l61 zy|a1rRv|ZC9vtaa>jr30=EN(8R5bWURk$jR(S6aKqK5EPG6g4G4iYtEzkXb;)c`UU zlO5fp0ZYNd{<3@Ks%tM6%tz`}gL=j50tSZI*5UM|lj_N^IxERCQobD?e5^p41?B@v zNnfd|=X;*^pOP$S#Kl4)tBk>P3O5vCH0QLi@p+9-^wuHP(3UJ`o(Uxa)530K^s72% zDYeOopF44@1@e9t)gQsu!5L*2EK{O8%qnB?x7BUY&Hp^OsGypzGi$sx;&gqJICs11 z1@(j-&D8i5cEKRZT}n#$$YjR3RjLWa0T9i51L`o*mWgh&{pKwHYH_Y(dJhwL{Wq8`X{GmFi zJlP|=VbE|YYVG8otOY!031ciejXK-8oYP#d9Xm}&ddjrIbGlBfq|HSeAZW@Ky{46j zJPSF*Zl~(I?O9yiv~MN`DhKy@Wly11+!Pt%8`SQP3nNNq9v?`xc^W{v+SmRbzAJab zMigyU=Y;LDzNut=n#LpV~eO`+2;(wr2%T{MJqf10;hkeHZm6l0B^;( zx>Vy0&(k$DiPU8}iqvKio$JRgx(yXWxKAqBjN*`%BUi&GJZ*<^O-r< zjQpMFX8%v~{D1E`{pNH1&n@%++$Fy$p^D>vS>xkon7L7^PAgZ2Nak=2h||bx{-&M( z`HL~Bi{qP0P3p>#%me!spJ`#HS>s%E>-a@EbY*LZ!br(|v2Og-{^l*f@zwSGMgXEl zpEAP~zQ%oMFOK3>YM|Tq7XeStqNTj=pE895_oab?BE5zB z;&r2J1`W05`v(2%I z1iep6@Y$+Ez9_vb!|KK%>#d1*iNUBRNXMkcp-PXrb*K{Fz{6#P*4|2qQ`FHPLuZL) zAgQMJp9|hjf<%?Z&Sn^8P}1;?%}zT06PQ-+M_DU1?GlxP?_1Tft}XGfVm|5LLHIt! z(;U_I-9*3|Tk^%Mp*p=ZD#5c0;ZvyUYMYA4%5R1O3`|VX<8i@UQG9zxy8BUP^)v>z zR5&9{Z;S)dw7<@GHI#U8Q@EK&O|&~BJcd!WQjs@99k_8Rj&{JnofcG8Y`?sF##=!C z_C(vHaF8^(P(xK({nyiJ; z>Z(J#vKwiI{_X=#V#n}e;)l?wL})8@GE-2MO8fpb;XPGEcN0_e>8zTwMbd>(J?(_M z?uL2kz{pq4b>1!FzYf_8RSQn3{M6z+=8zdy7Ibk#u99Xbwy`No;?Kapq zZM@R&>bf(C0PZ~`8!lrAKIrU9sS?hA0<j0pk54 z4X9U$V*`sut@Gg)5{Z<)wJ}sIpU99ASnaW1h-l0U*bSZ&Zy-DV6kNY}eEYE%GrlLJ zw>M!^xY1PNxHP7in$8~8EC}4p3yK9^(G|1ey(H{m?u<7I!49#73<|G^GO+hFjGo-4 zWQ`*CmqF^^+2I&3=t%v9&vizx2 z(k2-*iRUrihOnLk6h&6Ugi?{%N=(K-fLGo%w3n2;h+wlaE-|JWGdW}Fg!7>aeH(<`1SUtHeaZwaL|<$96*;UXKq@Z_Dx1HQ|HSc zyiLlAik+sRlZjVcikB=o`LQrR&?a%ODa=jB)%wirZ)qsP5?AQo2&9Ys=H`ioc^G}G zU|YqaxpPx%D=anJCfFToKap)@r14WoX2i$}HR^vszx7pR&9c;jd6Y3^%fjwOrL>WC zUC{rQcSzd6J4qtEp^x4;K?$$i8hEI>2}q(N#>8b=uvV46oFgY?-3qrr@iLot$_f^E zCRd$(6Z3IG2Q#FfvDU!YDLg>GV2IYaL}VnsFyD6)Nv|OR^nSQP3kUUwv?%roP01Nc zZ*F}ENMY)d7Hed>tc&$Z!)ha!`>6WK_QloQwpdZmPvuX#st?$k$_t?}Jhf!iw>eRC z92vjSDRC(FRw$2D*mKs+2fzd$m97eKR*+RZI)vf5WZeK@<>;9qj*yueW;C==PYi?j z&kRA8MoI%~-KmsRwKs))oLZo1J6ce4dOl?~Wy}#W;7u&nsclPhz}CoUvuOf7=huRlWE7*ymXJ) zKm+Uy<(R^STxEzMfDJTmSQsYSs`e0SWEGuOi^G(ZDd@K>FjYHf5L(wIf6tyRpqpP5 zs4^qoMRKoV+GW*bvKL#XOI68P$^1HpnBUpPW|1Og`p@7Ov48TdzzaI01l#O*_M%VS}@&z}!a+ zslQsJd*}Uk=~sEscC`tS?0EF2WiJXBLL}G#fGEBWV+6qy7%9x9)DZB;mNRXc5u>BI zg@w$;*1mMI3TlDANZ|!3Jn@ddEOqa(Y!Ruk+~2xyvqopB{8&nQAuax~{b|Rj~Mljjr&Hj=5u5c#I0USxl=v zMNE0NL6Hy#3BXx-*dOjHD6=ebv#2YQch*{{7m%@79F;UMdtT~uV~zT+xkX*KD_{Lp zL{&XCF7?4y3oB&passW8k?b{?>%gBrQ{l)moN`mg)O6k_)ETDm(!RaZ-$iWHhoCa8 z`(ZjJ6{{YViTO|Pvv4^)$|eGXuUH+0<0qyWeh1i(nNZWEtNr0k$eW>}z8N|EHJ%|? z;DR5_%(FukJfD_I7^<&o6g;e6ESJDO0Bi{o3$c{I{e zi#@vM$$W!7z%$IXXO_4MkozFj5BF(;aGNfN`OW!sp%MNB zRYe!AXDP|{z=9V#=0}{&uWG4rtq6M)VzqD|_Jkm!51(6@yq|}lWM8`jwUj0XZjlNR z>GA42b`+Zr&9pG+V!`0nvIToB+5197EuY6BEA>}I>Mn;`=8@a;2SG+@$%|)K7U^M+ zM8<511vDEWn(2smaSVB5{zjDD0h!z-oZLam>NU{*=Xp&WbsGx=_3e{<2a}UXl#qaz zSoPuYle&D8@53Mb*{OV=lj3YteNAolR_c#;CNuq#Ekp$V^o?`=(~~rjbe#dapHKeTwNG$QG0MT(iw4{PyM%%o#PKg zKL4UKb=Q0DMHZ0sJcy3UKkyK|+8BrOzm2hHa5mLy%NSPI6|Khqcy!DpeM`n-_~J|U zBTBfD8ADI`bIVPz={V?T{@RET1|AynHqXfZKH>}F!V%<~RrgnrG6|iF?`Rf%VY(bP z`e`PC)qJxyj>l6Q^{xM<7Z!2kh;kg%N{rb*2Y;wpNBW9)V~|U@2Io7-LE2N>kCFqq zp0)n_VzayAK62prYwW?qA-kiKdO%(CSF)`aOom>Y1>R zG$Osd5`-kQ&=CQQ#QpdeQ3V6#8xNPXKXG|1;_yS`vqJHBP7(_WE2CcxA!3b$*OzTp zxg|FmEY2Dt=V&^w#w^yi(lsuK=p;C8?uW+DJ-0l!9^Rs!?hCJcE+IhH_J1(QXjk=P zcFBh(Ut8&VAv2U`S1ad2bXRrx^o29*aCAP?y*FWds`Mpymh`I4udDGr<00za8RGdY zg;(!9vA%Z&ReRQ=BzJz_0q0`)1}rU1*|}X0oqka0K3^AOca<)7?DRCxzfH~WP4PX$ zB7Q1*e-4Q%ZZbS(e%RgU&8>y>@45Awa))?<55hx)485$H%b z66jnG`zq`8-J78Hg8ItG-&(jL>fgJQ!bptd%EDAykCF4y2n&62~ zMUg3Jp=nftXE&d24#ei95z_@&G9u)EV@pL?F+-scMad!OW=fdQ*`mowk~~~tI&zilTackO+nig^5>%_Ia%`kvK(T!A?oq$)*!s;zXy`hDwaX$zp}} znR4PWzeTc`GD~Dugz#m`Ks<>l<`!6_2FKkn8PTFds+)a#KiTbAxa`uUPxitFDZoF4 z23g;fd6Qrqg0#v1e%wHdx-s*UFKhP}<+@q(F87HW6Hi2rWbdQ36W4|;3^@A@BKx&d z;Ik(-QaYL}8AEyk_D#X~iS2vcLP9N9AiYHkOttD5Lv(4;y|zU$hPJl9;zPNl*%yh|D#N1&OvGXVu;-%VMNW zvl%EBE+zYST8hyb48^P39j7eTAB{hYMSbe zOU>qZ=tZo1=7~`kDgtbPaO}&<>-n^^gz74cD`p4ARcX>H+Jx!~zm7bqLiCL!UtSH@ zNlZmDk6nYU8)@=@B-YG!I><7?01{r@Eew)jw8C2*dbLM!#reuEQXVp?=3HV$2@+G; z70Ot=bq+jc)D@%(1T@VS`>)LqRF4;DVb!?YNzwA~y&D-rdU77=R426_bf_WOG@^3H zoe(J&jw!h?SoLJxBb8dNqbGRS2n~=JF$kE@>3f$NB=F4)CdIS{MYhJJ-e;719p^18 zSd~x=9QN{X7%vri%j0+@(^>4OD@k&A&?UXJr;KD3_2)4AT8e=i!+=H5_o@%eMfe=h zyzN3W){$?<(6-~bl03nXuw@f#wvbY!Z&ixm1Kg%>CAn_gSj?F@)!m9j)=E-mH8G3G zJ(7S`C+JaCK+}msy!8@OU&e0a z`$~0v2mzAzm0a<7jW4auixJQ_s@9h`D=N^rH8Y^5Alv&)(Z0kSO1*zj$!2f82#_lT zEl@Hh$h9vc6Z9gFDq}3AVMU#5t`eQWszlWozD^YEj1^}vs+Lwj*Uui2>V)+KlfEa$ zZP_>wL4`e*Ps#JQhtRX5P=yu{b<9jGo_Os_t%s{b99=j0tI*@HBg1_)!@A~@y7kY# zV`4MiPQx+|dTASX*1z~R++y4mkHB~=VmlS-N`KMUxeD0Aq>rk?5$Yn(zy6HYDT%R=v{mXP>H-fzT` zQ`WU35{3GvOd!SSnGGW~N?m{`GB{eJ?BR)N-Upi!oJ|9_EU2%Uk^&STma_s`5ppk? zN&oEfW#R+qpI};x`)5fj$Edg}_po?r%r!l-W=%AerQ1WUH_T{R602J)!(*-@i^DDN z4cFF!Cgb%aKe-%wDdg5WzKyiJ?6Jk04Jpav7KUC6u_;`c`WO01KTDhD2|{1Ob5dxW z+A>r&tGg}RmVgACeZ0wf-CGiX)f=|QFJGzIeSV!ylBY9?ZQY(EvZQuHBb`!_+L4yO zkawb6Lt7AB?N^?P8S?lkI0jGI+(-oZ%1IZUior=8LGGiIyVmY#Qf*O!T)*Tr`5>CH z)1}#|$u8N-zLkq1{ff=ydqy^)MTN1ct}nb!Ozx*fu_>{A=+lek-G zvjvI}4DLJPfG=K)6|Q9(%D|BnxL>yBh%V@4H?%s94mn{ij??t1<)CB`rHm-hc`;N~ z0nD&!#&2-+_v$xXBUaLMDod1p3JCKjrAz$mSa8@GAZ zj)bng^Ntqh=yDpXV(W6=(mGQyBZQQcilEtYt0=@^h~tJbKIdjRtMg3Go#9n&XjgH-Ok4GhlwkcKJuGNp zovTqOlchmG>R;QA*T!!nF|qO_=jN3fM$Z};tfE0|L*ZpmEwT|K5vDj-j%={JFRzHh zIlfOKQ5d(g&-h1_QROMAHFwt5^SwiqnH;~|jarNy{U_d8MLgdGKNu2MFdFxcyccB_qS zA?M09hv2o1BYBN8m#s?3vyC0t60dcQBf35^W#^W&b*$s)qX5gi^WTW9=L0b7K_2Jg zEUz_aP}6=s>tR1WxkK_fjbS%M+0tt9pZ0#VB&KMf%`bW0@M~3~KL6;*ek-p=jF}u* zKzk-HpN{JZ6Q5y(-Ky27;AQt$0btFW z)^Pi?b)7vA=h#A8>17PG8C79v3oM_PPw9V*sV|I9j8>4=I2%In((Tt?X-oMn#?63 zRLb5pn$oQ=DopD2oL5)kWfisEVp*5p8Qaf^T4OjVFRJMFBlM6QtLU9)GURslX&rTj zxRzXH;;8^mhj=asSdNdUPkM^l%CMX!^G?+>`Tcb;fUuPwmzCpO%81m`jg$IM;c{~| z7qe74PVr+dR~l_Fm_&J(BJg4g;d|!dKiiRu@&RzDj}$*&e;Q;H?@yuZ5)f|tM%eqN z5DE;s*K2Y<2Lx5w~75r9BEct_wD!_rRJ zH+ts8Cl1gxVdRgHlP{(iIY7SD7R*Y73;*VI+Nb%GcJmK3%G4%Y6yLA}>`ytkv5nTf z`$oOvRSC#dnVhza7l#Q!s6bq)icMDjJ)(is)_ItGDppuh+|tf|_{@T9yL!%%p6q<6 zOe%IzXk10uiB>8JHopx9l59=uFIeXE}~oAIo0hiv9sINx^qSF|oS+M{ZU z27#rQ9hQeq{QeuVS?~9k5}0L+DA7pNCLrR4e$-*whbPUZ$SJ3FaRvc?J~xoLS!ohs zv<0JEXq3Vp3XjF6xXodb!~UMB%GPYrD?`>ExCuJ*u=5NnExA z`XA3xcuCYRc|58dkCG6}fhf?WPu_I~uO!xh(DlT#=jQ!JYN(Ru^0ZqN+7Rsk=~KrFjm*g80d z5iu`?wwtgE!|uH-?fwV*1&ZSinAX4=UuqnnhwZ*aR6?nAnstUZxzF_n*S zHJ0eEb9ICDlzO4Hus`|2ujgE*fUCCJzZPV>cv?3V?jc^aI~kbTxhnxYea|FliC?h) z>og3ap2!E_WDMTCQRqz*|FoEI640j_=#y zG3L97*Wy@LvNVf|5UA(DJdS3UTt}O0=___y0=^K8p;!>8l8ok(r1tE7vNA!3+Mq{3 zqICdOOUX8ha5ROsf4b<7N080N=mbm68Jvz|)4|L7ZTLEz2;2jKQHW*-HRkK>(+l^K zjBR2g+7?^IE$ar?KFhD$CauMmzK5`B-2944;D=w0lxASE_1a87NiGzBDRjBD`Y**& z_n^rM-fCzyST5emX-=9L4CXVJP-TrMB7P(FAb~N$S0j+e4bd-IoT{~Ra{gKS2``@aoPoqu>khUc47TG&i~y2XCL*u=QD-g0sv>PM|18AEQt}Vx4o)%tjzfLB~I!_0khM z*c=D0(mpN4b!n3Xw`00N&4UNk_1V+MiOly%-Doev@QX&#AHM9INth?<2j4i90xqcv zOq1|8KWPeBU;0-wX)~9lWQ{nfs%5|Oa4RuR#LkJYSrK)PzTqfK*t{Y`kYr&;QBe*L z<@-Fj9u>5?W3^g>4>;G)GbARsicM~ zH)FEnGVaGeiKEB1kK$-my4Q(~L|2Ax^bqgXY?Z= zQ6;S7GrvnL)AMnir(Uni`yDO^XNY;2*eZQ0DQ zI}{8QkPCyWfv+Zb>~{H*EBFJbs4v;$*eB68(?YA3=(j5Ov)~nXDHIaCs|nFla3rp) zfgNG`6IBjGh#kVG~v`yuKTN{fiuCVuoQR0%3TbjEdmt2-lUXlcs`*N>x8Lz zf3yT!K6}k~@hG^4yTmTmisScSDpNJ~@@YVRdzpLH-K5oR&^U3hKRE)lwuUnOc^2}? znKhrrecsf!6cdbd+yl+W$)l{v0A1W(S%2+aQ)at2&pH0zqWeND!%|iB^9h70aSr-t z*}O_~h}5EbKWpK$Ls|R)G1=Ex>*tE{>c@OcDT=GyWj)jri zEXE@QzjHd3<>7BGP_0ULPq}L#2UX`GWwN=zaBSM>6@ImC_jr*Dd9WJUPXC^CD_#mk zMMls!Of6;{n)S{-xZYLr8DDjmU-Je8&n>{)ZJgx+9D;WMDQgEVYY$g<8$~xSK&@n( z?a3oN-ED|-n|U_EYKR)ffYAeQ5vA~W4U@zl>zCOQ;~L<#*^mzzq6ac81k+{^-C@2Y zk`g(;QpEY)F5F~e63*lNH|!9u0w_;SS}P8xt;j3Se_iK#4MBrizq=&$&_F;8|9`8( z|6KI`vk(W=ZCp`BfBBFD>Lr(j(%A#^2oqo=)9Wh#<`fb(5SzaW`3EL(aE!#zZWu33 zK$b9+JYOPSr+@vJ>bWO`>SyT6+=Kd{`1}#=%5IU?bV3vhE?}&CIllh3Zg#nzI&t;& zevR~xvFp(QYVx%f)lM=c(S8z2*l+l?uOaS&}rKC1B zR7_MNEBc6xSw1R2b3tyMSSM-Cu|^8ItQxhYjRCP=dFJicB2|~w!~{?}i*(h{Cgd}6 zQ*3~2N+@wsI-$Tfrj2V_f6s9VhafryW`WeH{#Iwz$TL!Ck;s6usj?*USF&8K^I2v) zG6}fk^nDZeul1sO9gMVLa{n>XslO^3g(V{gdsNbFb6heKMNi|(X3fEQJc#SetGIO@9nXzG=xGYYY6ambCd!2yX?3glYcAhn)`$ZqI6iLiWKX5;)qpCOxwfs;G={zY-#L;h?`G-a7 z`0{BPaLeX~SZ&lM6|_powqBq%orRsmLTXu~IK*B_iyj^(D6IOpU@r2Tz!bA|um$hVy#2N9lLZ-+T0(;o5`aS>FC{{Ht4(_14Io^cqx6N-^^d5G-EjRYIrIWV~RZ| zJIhOCbn3&t=!{*omaVHwT+TH+1C*W9W;hSM8+mu43?R6P&aVH!0tqZ5ihKpXTSK-L9A z-9>j0#^49PZ8s305f$Y(;)d7fBMG7fubFl7RVtS}5HxbJ@m**gsQntNhfOxYE>xqH zUkegsk}!+)g&J-)4OV5{ck@jS6C3ktIZB?!xJJf%fTQY>bowJI>_^iO%2`8>Vg@r@ zJB%%wa$T}kd*eGAa@-fQ-$&T{mawOSFekTCjUTxUu4Y~ILFRTrK$XZUd2dvTau@J8 zBZ9{wD+C6)Y!7!qlj#7hjQGUZ@f4)_+Sn^sV@amUurG`mz4wzG1$o411Wi2LF%s$L z=oT@DB?jvL!-nY4I$!^W4=Tuu(`>Z`M9&Dnc}@EAT6r#U81_I{Q8Ln%f73lr%)El<3E4I`2h=kEo zFIOaa$Zx3%GKox*uKaw<7(otqv(l)Yakc?f5IqT_7 z)jM=FtQOqEQp#r*+bgAaX1HFCt3TjA5O_W=_g*KU#}HW%(l^MsG93;%x4sJL+cU2+ z`?fycOS-?Y1t*bN0wA#f5L_j@;Nf=B>ttLflCzqLhkFzREy#TeSRvOCfSmpVvab=( z1d<-fXh(-q@}KI$y=?QJ!s&dJgh{p6{Zg(Q z$+~d*#{Bn_(7F)%SPh<(k*gth)@D6L)2|@YuY~Q0!>=f(2cbX<*nmt>+47pNShbEz z%$lG#x{I?=@^5@SGM^5s^xCUquIk(}#*|=PuBK&iqS3lQBF|Mao~jaR z1$EN%Nb9Vbh+@x>eiECW!pfM1?;E8T(v!iPScRr}6sc9UJCGPy1e?GG5J>G#!su+c z(nJGdU@)uenJh$vd0Uyni=(^?N~*dwc~(LmrQhoC@2~R3IxdHf?Zl3GjxFmP4y{|3 z_%&7$rxCS?^LNYx*o|Wp#w*pX#*xNM94`KYo5VEAW9>3iGKsm*-V}`=?j?&)c@8kN z>+9pEn{Ii+v{eZ69bKl8$+yvo{*=r&ss|#cm&kv)|hwdBnfXlCSyKJ7mim)3%_pijhOKx9(5*e@SeR~?(w zG0NP7dYj6r;4J;+i)fTmHf_!Ldi*bxjs}DCFQt`Zm*kxFlNmtil_LE?*Z#ze@r|TNQR2|T z${~9x7gX153$|!sGo0kZFY2REIC9r=9<~5D7~WoTm}>Dm1GDHo(sBGjr9N3Z=I|V| zCw6U>X}diU_ZCBSHH2$0Z+?sRXve7KXC?&GCt~lSou!u13bYXr44Cd78#hR&iDTic zG>bIrj~*JecTR0QB&`x^&Poj}^7dnh=SNqq!_EC7yA5eiI4X=4h`9&k@davkjLlvp zyMEk!SB*aAj=tuZsKvWF-4nAujLSv32;>}z4W`M5WGaJ}SG5E%V{y|2`N5|1UE4=L z;0Ih;%%b&}nJd^4lazr-Uqb5KHOTtZqePTf!S?oId6<96S(^!9-=@O3JAzf#yYJPg zZnfT6mT&XE>e*z%X7=H?ni6)CmCwcSPhs6bAGRq=r0V*b{d7+#<=!j^vmHHlC5^E4 zp<)k;D_6;n5%SNgY3z{i0IZV@Pq$^y9PqK+%%1E{WlZNs*G{5pw!iSX52suLWlZDe zxD;>TyrDkr&!R^@ET)xf?!u2P^rWsNe(S27ZCEN#Q`B2!n7C9=RO!n0S+q3US7k*_ zt4!llB>}G>ivLz^-ow6EvBTZGoG?EU^PK+XTrT(hMpaGXkdE zu(hV%wn$BqdIH@Hz~@dGHi+K1muh49{>qY66*xp2Z%_tDxK+4b;m$1hl6rC_x&4JE zw1eGX)5&Wt`Q1!9k7>|5wMX?36~sxF%OxxkODT(lV8fVqr=!afTIe^NW4BdkNwpsvraDF>S<4;x69aWZu` zJyT8jRydffSxV6mgRO)kJBYzW6hL&@@0b zFT7Nycyi~o?z_f99aU1D{YQsHOy<3nSG;!_Y>kb^H}rWZRcK+Smp49#y|v?%jVi@Z zbi8uBOO#u09lrQ??n2SheKPW_^Iv2yXs?fiAOF%459exTIQf+T-TfNeVfXJj--WFW zWsUhQEyb;Eog9Hij{oFA|JyY|QzHY5|H<{EWc8H}h4*fvRv`oI3-`MyDykG0@@ zHTd=*uc|8ifQY+rvO=wLN$I5Y``kxGf(aw}_~w&ncc~6pIj?*Eyz|${_^+F>wX=!$ z+uIBFAFk}7I2m6(77!S?U(+F?cE@&H#%Ku2?2kJ|W>xAk8yfU(L4H>BrnH5`ox;Jr zQj5xfEY9^($c;EaXMYqoN0Ex#N)xsg7I?v>_!^?Ju)3iFw_{Py1Qjo`go~7Qt9uvu z*qk-Kl3}RPd7UoWyk7s34B}7pvKdL5pebSEeWD=rcJv|y zbreSDB7i}@vy?<4rye7Nlvr~m_rBbS=SqIys}Vt0%MK$+$hHfvc25xp*%A!}WQ)gD<6wND(fxKzbrzVC$l71dW6qw?3b%N_sf zff#O{8x{8}jHSb0@Hj%Dxf$9}F9N%t;usjii!-ZT*LUksNqjVQ0LEDX8hcd znkR$W#7i}_}imWMw|gDgy6_^{gOsdQ|PPcBa<2z=i%ya}i>rPtxxrp)YOGeRUzl z({Z=zRXK3%AWtCDwpE#gTj?nYoN)B6#75JNt&c_M4SVClYilVN?$|n=i?j%r0Fid#SECG zGSLNvP}~##h>2sMi)!#gAIC@q{y`dNK%OKeZR>YRw_siXWuB?}Ea_U3g54tRn+{vy zty2yd8hNwj?5KVmefF{hRWTt9f6Mn)3A0A*X*I3hpHz)eS%o0F=*bxyS#WF|sIyKxk;N;3PcQgFnpW?>%(NNH@yB1MSKTC`xz`|WEI#3KY}0!JmM6!6h0d6h%M1+w=r z(4og|^AQ9oS#%Ui?^6FdQ= z*458e3;9h4trOkLA$p+WP0OTv-cTo+%q&IT?FXL1;LR%t3dB}5P5<_y)|G^<;+mN? z;RApBm`W!TgL3yUH+vL8GI-R8EK%>Byn>kOGo=+Ddh(&{gyW=hoMiFm*iTUy(2t^) zl5aS9Ej<=tE;;B~BFWdFZVr%EM_|4^x7GPi866JKA8~WK zzoGS6U|XRr&=nKrzO_W-kuXV}miBzL`Hylf&=u$@_hm-Yi3p~uc=M;3j8dJJ_BjmH zmhJ=v69=qL@y=yA8*tjxC5>D803O8g9m+9a@j}p!qEQhUG4{pQCVng)NGm6BQY(i05 zVcj%XaR##i$aUhe^o&B6hV4vb3*Qq%?-+M5WRf+qYa5&Qz+F;S@1XI`t|XTl7@#X> zRNqo_Yy*rLM9dQ5BYWYrqbv?HE#oG|QnSgeTrQZ1u>fhxKNHqTYBz{k=2()a3RUg3 zyqoqUhf;*7U88+LLUgTAYqY{$*~{3=DK&#HM$g@w`$gv)UyTQMXI;!JKDL^Rqr>Jl z=jEy2a)K*$v~CnTW&>!kvZsDIpuFN62mq zfcRrz;cj9Bw-Rr@&4TJfSQ}2c@VZ@SwjP`N`-o2U_XEkaZvQQDoQT8@z?L(T&v$Oq zqBTl4Y`>9gM}_CY;u3vO24M~n6soX-`*jBnb0q%?!tK!W{S+1o9I2)2 zSq29Nn`tdobdEi}8}@MKHZGO$xainjZ31aN9$Pq+hgBMl^cg8Im0+AIuULE!dXY{= zkwe^EzoAq6K8|5lX%VZJ;pQI61*tL|k%W)yt7|rt!W<6p$P*ct&+7dz?UM49e|Mw4 zfT;bi76gTV*Czj8)fWHh2vw@A0g+VDKC!{i>#8K=8>m-BI^e(8ebIpG=YjT9fDn>} zK1$b-RE%7BebRxWH_T;wr!HTd}NMba=D70*pNdLDb|ypk?A^W5jnU1drOAfFnE zF|fa8*>t~NWWIe|$@2bK4z?jaSNU^Yb=OoddNkABT0W+@edB-zU?UohFJy% z&D;^AtfOgAbhVH1ks8Bg1}q3v;udHH>v0Dd?xATc&T83{bTbRPrG}1g8xzyIheI%| zdPgu|&Y5}M)n;khaDp6lWTNBFbLp4IBawOR#_E#CBT&IfvkdN|P1qBuEb*XI4b*)S z8OeDEjwbVBfOrw99D2=(kyB-DOdRfgH0#Xm%fEKPa@|=37nBD5bPB`7(2t?n!-&P? z#-tDDEtlz@IznIFo@pj%WzqP(%zZ_C^K3-S{Ws7d<>$l!@vI&8nSyb9>o#WPfu^Yp zcf~{KVlIYeqal(SRa*Tj_TNc!wrEKi6IjVXldSDRDDsmDYirHb$7>E}8m8Ea=0BCR z?jC8xH9K1NM9#7=cx+6l%ry~6!;3>wCb~`vcWkRVewND`jnlM<8%*!-sLIo9(QKAM zU&N^C9%lZI{S6ZvJDz)T*a^Yy(POLR*s~|eFN};$!BLx}_c~ZP^&Ur8w2N5G?vl!Y z4y%BENE4XzqNhq342@92oK9DqbV-@IqNuwA4HIN<7f79ss0T=~6Yf<4ToS z;Mp_h9fARC+?pf~)hJcG0Ue^Q2xaNJeOKemGwE|%d-y9?+SN~uQ(dj-IgzF2){?7% zlO*$mMV+~gxvp=D#Teg|G`o>TK$wR!#qa~$RVk|lxd9y0&1t$=;I3RadS_N{DjsuUAfPQjczf<)-h$$VkE z;T@ou3*)~SjUq_`rlnN+ogBT~1a?Z1jp@z&f#qRMqIV*0!*X>sewSkxj%ihSwAj4g z^N~Wc8<|Yna%WyJF1iG<^?WyGCfRF7Ce`N0^I1kO36WtoM70_M1Rvqe%OCL}_-H_| z98|ZG^PVt+LSF+MLAgJ)@VASwU%Wl;bHG*tbgIbH76<8d%0RD6)RqK`%(?XIvBvfN&5zu69@2$)c?AxbG zEH>hqhs7Q9`i(a{Bw=5^r^2);LP>h|1RBMoP1d@u_|x-u+nOC*1#WHzuP4)|ua&<( zjQqIX1(B#jKb1DtB~k9=kYpB*HcbS+f3k09k-yglm&e%L^3t#KGs z9TH@`@m}csyfX*BVf9KO9a8+=BZ1xa!0L}d>&5P;lk~Ybr1Rf`AHrpe z>AMO+dqD~utJtE68bXtq_Tz(!-k5SD zqCxMdg2#-oRq3+hq}&9;y9j^?D*RS}R3@NiYlWvHGDWW)9c3doQ0*n?JKY<%OHJF^ zP~Mc1jLk><{?}Hdw>I^_kYL=nj4Ec?oh06{OwYQQP%XKMinCYLjpFJgd1NBe8d1`D zd;vHfT1$T^gk?=vI})j#!8u-!#fo?pl>nZ+(@s2pF(bWk7JKdeRD8b-#wczdhf-3# zacAms=NqV-l5~iOGI)HQ=See?)kkFpUqo&kmkM%tyxT4Y?XI?wRKvs)aL=j<@mkl4^yFw68?A4{IBrjD?#?CXmubLdros zw!ItuRzEJG5b{a5{Wqq#h5F6O2o3oR`cWT{Lp?zA&{J34QPYEHzY za>qxcY=~<%>UB;V7wRV}{u1rRF_! zMARz#)>XY8J3@h`r0A~KeS)Cd{)HSj!+8SEg8zmFBEz)k;5N8q4Q%P~$tX$JGNoqE zhEM}p7UY<^vbYC1x&u$|0tQKAAcBo@k9cQhj6%D|%w|BQX&1dQGI@F#?7Xeadg0WhvKzd zcHCtn2|oV?8Q^w?S|sB`G~qPf)LN>j(VKLU*Xpm>F|PQz7+Ao%3hSoqN2Dj-5+BEw ztPB)GAwH}Om&Yf8hfQ9UH5jQ2mER}OiL4!TAn!!DTIv=RUYZb;4JuEWk zFCZNTbI+Ndj(I8V&R3Rj@ZF@)W13qQ&=H?U1XZ1x<$xfc;qXg`_^JLq1;Ns+J#sbT zV?-Gv1HSMEiD{cqU8jU~q{NgkT$wF~dKvPm!m1^rFd}JMRq(96Hp_UH6ed5K={t}* zOcESS7Mwdp&I8*C72X;<#1rI<1VKVdpI`sG1n`|rksn92kIg#cPp^f)7gmJ(Dv#(p z5k^==S&4LwX3uVgszSh2O(T6nYXOnoNn~KLQ3DBmyMKaewkoeef z1;FoOvx%RJ+5;CLSOCbiLA#JtGeOLuwg{f+?@6#!g*Q-SPmq$=%*uP1>mfF^s9Sox zU7~Q_f9&4GVw!TUp}hy$-~3%}D0%+aJfU^Jf?Jacp$QxQ8X)J>q`2(q^+y?Hu^=FL4F81?*(#SE_a7tIX8u-xF8--yALfb=LeJ$IAZ*{}ZVb zDNt!$yYpw2SDqsA>4g!c38Tmc(U|8hTCkTdEsdLhdIxr$6!4rT%m@kQhw=WwHR58V z21w$ek*%C-PO{&5nM|fJ-km)@=6ti(rwABRqmwe+*&fAvv11MxfxdQn9XqqBl`_m#z>;sT{N)Kl;)fAPMbchkCm z#~H1>zu#CNiZO`H>Nh&YD>m-xe0ZeM?4hA+S2OZ?rg=itd6tr_&>+cV?jJh+=PPn_ zXV22Vy&=-ji!pgc6N$9$)NcvWEkFT89JJP>n#_)gSz5Y*=_|KSoK}F4BO%eGcaann{ zG*4|cUZ8ZXu3rXN-xQn7m%+l+qb9bRSc8oGx~NPjDG)zbNJahHRlNhcnW~3ojFxIB zVWO(-x-AN_m4uv7l%;2viXUBl*?F{H?|HOdu&3km`tj>q^EM%%K%5LsQXdt^ROr`W zLh2to0BO9Kt^zFS!L=?TCp|Gbj>1iiCRcoa0o#J_kRn%hzwvQA+lH8gwlIE}b0|rs zNG$|0nmt>AK8ZLE&^iTEx2z`NIFK+T!QNFMsG=8GLu?WQwc+GcK^Bdsy_pTDvIUq1 zBonWD1HXD~nv&_cc?+tI#j%*E`mTU`pPc%p5lhs1*usi%{7oK@!B<3q?Ct3 zUAve~#J}_P!YtKX28HwDQQ5t;ThxZ;scAX6cuv7xBJX~jh=bO8x8oseChhsfNw}%E z3_X}7a8|N&R%ax%SqwWhpx;S?O>n6}2@Z-7n!u?gUz~Ma;N*T62hyn<$O4!qF|v2Y zs3ksA_+V*ii{8EsNKZ=H8I4CqfgHa$o$ybcmTC0#`2Y)m>5tH0Y>LN_z!ag4L>h4= z%6VW2g(j;MvQaF?n6^C0FuIrW)a<;zbFT|bh1g&lL+8{2qBiJAfrHzQn73?@Y>i}_ z5}mMNmOrfp#w1L?ysZPw% zEhJ=9RafraLa9-|_%&H!w*PSiC*@<;I5 zGpUnZedU6}&+R9qbEFGb5<_)X~k?DouoTI0NgsrK|WK`>KCUvd4+dak?h19p)Dh)^482elZ|bTN?ge*6$@T#UvMB99J^a^R5#nmx#}&dz zj%WHhZ#)GFGe1SkIBJ)wfQ#INhWvh%8DGEbFw1|uP2!XE@z$xlsoKP114&1%nJC1G za(W$EtG+f@54+5&_Q+%NIyxEKCwoI#|1x`qe$kGslv8ALg&%Wh0@05>3M;oDWd7*JgoQtucA6ud& zQ#EL}wh222svmFiV~HmsvQHk37T0whpVcgp{8j~=l;&pBoOh`2GUxtp5&U}6onZ#3 zDnCLCqNfg}9*N0-8BNq4`@zx4C`f7i_UUZZ-|T%j$IA?Q$%%d=%SC40eE$GoP5@h6Mo*N zJ#go3GUvHZrn2T8@AUvI@;*y>5jKx+NNA$j1`tDf`teq)$dh}DzP)T1WF;`o+sfw4 z{H@J9&hrV)pf0+&^ekb^-ufsd>mGz%2I~;dq0)dJs1~Ts{wqG@jDUSZAc~$D&|hLy zz$?eG(+N)N>TDf}&Bm}xa81_qO1nRKQ&sewpo!u*%B7!HKf@msdURz0(px~(CZDi2 z^a5%bCo67@`T^BWT1)J5+>r>p9r@Fh?9+j$Yy0zROJtSf>rWERFID-e`)lI*a`7GU z51-=Y?l7(D(Va|A@1P6l&b~9?1Pr9X;KIJ@pdPV#zydlV)tZD&ZXooMzw!EuWUt6x zyQfK@N7^}s%AFg+!!kJ(2kiYaZ4aC*v(9(i07tU@)m>IyuC|CJ^q1VYZ!S{@9BU+SsNA^mrVAR> zxKY@c_~i6`yyZoyJmg0iVN3Ru@+61ZuC=k2{}kr*zn*y#FVge>Anx`5J`53Ppl7LP zFK6oZ#r9zMKN1Oz|K;(4K|&R+=Y^}lCu+w6u8ITuOTEkk8#KSu&zxK?!zOM_oF^yx z6opA7_KV`a$qzEFL8<14<#C;#XEK^}IXq8}zr8(NaQYH+G16gz_Gu!^ud~>ahJzoz z&cp~Iix+~$4*L7GgQ1{9Z|wrAJcHxbX?VAdnPUNrr0;lr(>k%lA-|NqC=ME_X^%9O z{+aYyN5`$k5V5|B6m6B#EMtu%=f|EE8(I!gk1}Rh5uDVcjsa!`4c|kWG|RTjeoApz=;Xh{=FG<(#E8Ufr6E zkY7zfY4MXW-s|!pNUO2`L=I9fiahtXqh%i1La4T`>X~CU^h`WBq_oTs;z<2(lqoV= z-?5$_Cu6#Os1zDbRKd+h1&Z@r#GX0H8QNqw3nZRmHad+5=M%y&`H0=+YQ1x>UOz}i zK%2{{;*#2INLGin=ZlQRnuLiSJ0-+&pW|~!)ym#!QPzHqn+!kJTy^pJN7_y9Zj(XA?dVr`akT zPrjRj>><8YeI1}2D}JdPjFUpf%@fIEF7*O8`x{E|xcHe)#1xBo0?b_uN~&mfNO$do z&=EV2(Ny!J|QoRisEKAeA`R?2Af!P&|Bz)!=&qA6vv8B0pnYYCiKtFSOBD z8y7?;uDWe!o;^=s*a8d&-Gf+l>?K7NGvDp~Qq}GWEuFWnqQS|D%Dt0b1@|!Mw1tG~ zVHw_DCEi|3G-jZ8!Xd(o>>Mi+!TQU;Fi>E$E`S$(&DrHd__qNu|H+8*KW$_}MFY4d zg8Y%3@drGfEif=xa#fR#guj#6>U)v2^g4@^@3yU^7*I{y(vRAVh~4j_+`0HqVb+Pp z-SkI^%v6=39?j%5{WO=@s>9#&$E>l_-}g(r-R~gVgsSwJq7=b6g~(C^(~-Zh3&km) zNP<~BDFd(iFY@5KQ46Fg`(H!UvZ-kBy4OHQG2@a8|$W@I^({dy{piLMp zUL>_Do{YuEQnPs{r{kERk}p*>CvB~^?uGDD94m5-yp|du_YxWk*}GrD^_UFWL8V3F zia3OA{+=FoQq#l$A}~Vdu?{O&tRs#Rd?soioabUQ%9>@q`pE_Pr@i}ex#+WK{K`|m zpG6UJ9S%c_i{8FwVS_FA_uMve%xk^9s7Va>O7B#3O0mg{+S&tpHuK>2Z9EN$i-V*m z<9Bri<`vk!S#f&0ejH1*`7X0n;B{i2qzKEE9jjI?bTDd^@DM9jiU*GLlD*1~QQ$>V zsO14sE867PvV!wU-r??wGM{<2VxUKuzbecuXhTIwpZZ1x`G$pOHO?eOu90Ju)m*dh z9LvNiU}@V7i1l-0CXbzD(;M*KBC)V>0sW|@vW0xXF9lQD;;S7TN9fZXsx+l|zrzin zbgj~t3gy*R?STUD>hpob9qN(|S8S^)$;rtFmg&QsK}tZ5LJz$-@rmBmHqg)Xp!9p! zY&07YF@?GR5WM{}xzbp*NNFBZd64@Ii)#my)R0-;Lety}8LMxTLn>A9Rn()(mZm(M zJG*ix1|9HNFJ>}9-kJA2x2<3|H?o>ToJQ+6qcHV&b3j0HUWq%_5>EHM zc;(R@G%6yq17j<*wOm5(kHeFZ#xD&KeFqDCJh7<^@xDtx&Nu!p5t~ReleH>Dq$9!^ zRdzJ(70c9o!WugR6lZAq@i&4ABi}ZAGHbd-w z7-%F7f?GjT5iz8nNkkiD_+THVL=YK^`_-oJ?2x6>6ueas#mt3}eAaQ+QZy)dU}i$! zgX0S{j`0fr&RF%w>(>Oy_`P9J^jUYgL%_BOK2A*sO_hPWKpbZmp26hi4+r-Sqwy4e z-u^i&iml0UZQI<=&xP-^a zf$nQ#)c${+MY1(@b1?hz7NqptjO=NpY)lkwzT5`IFC@ajQtuyrp8qOCBBKA#sJRh% z@l~?-#R!sYt)Wu(qy!IYbRcOoKsIRlCZ+~%Cz+w}RefK)HgdULSHEI~fYa-8ozuP0 zk7nTd4R_S_7YsaM7(V?<>tkj{n#0v5=iU3z68E>e7GN*2VmO&@iG^0suOoDze|)r-CpZU@D7zFF6upDzhC@Gnz=WtmSUgCQFD~aLPR1 zt4e^SdF;l{rUio2g|ACexK%A-M;DEd5YKxf_OJF6{;TsnsHQ{oKe|;yabo58vcNco zL^z9b7nYzU1Jlo(H!#a!4KYbTD%nhcxr7cb9`(HtqrfZc62<!&C*j>S%#+x3mw{JQzyb#~8qojbEC!i!%x3>_x=G84CnwWh<5d+07lTg7e z+1A!{1BkT!%dD*F`emq8uZJWrtt*ttVF&cFRMzwZmBlSl5^6gGuWKh060}V4E^_8c zD!)-I$ec=2xM>$>FWOY?Qd-uKAEzfe?=N{IHe}0wmCpVO+s3E83w@|PN4v58?JC42 z!5foCU>YuwzoHu+g>%zF&f=EW>Do)l5^+%M8egUAAzJRmxX^`8IhgZ$d<0UxETE`y z60TT+xPX$s`Xz^&L&;kamX{odD|UvOWBwys2{lHQHX$v{ISQBi2{mT@BRl7>$|!P% zo}#xAET{=J29!F(FTm!n@+)$NovM9ZW6NI^|B}r~9?3Wlg?@hSufc?w!IC9P$fRS4&jR&A3cPc4S znLm#K^gIZP+JyzaEkR}UFKORIJuN~d3}7qHx@g~Yvhg+ZPEX+g-%P`Eq;44bUn5Yn zQM%pr+MlAl7r$=AKW>pfgphyS_e!c#e*DhM;lBS$HoRF$T=!a!T>r|zYeoHwu*1`a zpzSKC8DprRk5t$@G@FZ?;LUZ9Bv|Mnb@DK*3RKsA0&j$5nH`H#oDe}5<-MWz%$K?z z6-wJrf-$U1iAjOEL2(P<&6$~!czO2sRz}c*d}ktP8>GKQJ1J0zE?OhOSxHQMR3Y&4 z&1NddIn+*$1nf>>#Bjkv7*cRs`)S1vc^p905i>Uf(ka~Ze{+d*S&lB6CKzH?TT-qe zm|+ynk}qVJa=7k~2t8rn$~)f+e6e^ZX4QUQ33_Q7q--CBemOB71xmS=O zhTjZJwYeXU;8dT>a9>0#;CZ;N+zczOXl0AYDu4aXwhQq}t}F$U{$cI9X7X28HO>`O zpVD#Xx*YOXkNq^qm`p3eU#a%FDNcjE*9UkpSW+v&srIEQSp-#&!&DukdSc|tZu_<| zToIR|@D?KO2W0H4W8VFndgfBUu_&kh4%-!DT^YR}{Cvx!c#Ysz+!r$6{ZY&{G!{H8 zj^jqd5THJdViv1$CX{lO1T^#$l_6G3Hy#xLqNLR= zCxXy&{z{Tao^~oq$?FH21Bw>q%&Cju*BthrQ{#*7&p~3S?ndoLRs2-gv%HO652V<~ z#=;Lk?_nj$NODKRUIeN$rSZm-9i-S(tZT~EQ~||buR-121ws6EAh9jbo2CdA&UJ-o ze1v4gtvK__2+9_^pOo@s&+hHFtM`;qE)smRxGQw}$J~kh-cOJePB{&ef*PqioUxIPHL)_QK2u zVUpcNX}2btTIW;Ft_w0r4p0C$&y&W8nl4a~f4Wt4OpM{gQcAgT($fqPhAEq?Q-MZk zO24a0OY2B8|4z0b3%a_zAN~4@N=;FS&16Sz_JrIXnF<+X4?n-2AX+}3(0(R5Slczq(dX&tiRU5y%S+xoTW^)<9QaE3NkjT9uAb{O>pCG8q< zp);b2vNIEbj21!3A>Up6<5$i|3i$(MXJ?8+wZf@)zJ_EOccK8zWwHO?X#kW&q6c|n zxOYv=>oo){ziWN6f|&4LwP#k9m=~4&+*1Uv*gKfkLun)ImV6g$*b8+V!MV+7b7x19 zXNdKZg5;x|IiS;ZiPEe;@%1Dev;^p5+TV}-_lWzMXQQe}0}GF*kQCR$P30MaCoac* z-^4JQr({THI)Rq^ma_V-`fZi4`)1UpZ2#&woH&6tsH`!u_oZ7{dZ$wi*j2RZE^oD^ zpwsm@`Xlk`SDSuUUgFia8EE44ABw$l6g%Hw*-cRpJYwEv5_#joK352R=ntxk-zE~f z8NY!o{@K;{G0W*auVe5*4_NNy`zZZ$O<+>)#9el*<{A8VCA3 zuyOz>@&;Q{@>w$w%Vp{obQX7>MD_`Z_kV*06($bCuIcd4JnzL{sedcN4qj&MlXxpy zIq;jkWOk66oJ>2UoxGqU=46E@JK4@Tnd+JSG4Qeb=C4nb#!&IQyk=)IOG91OGA9Wr zuYO8PgR7mY6}ce}X|X$OPlYXRZ_ZP(aNTDv0VWnghEltS1wc8l4Fm_~Lhu=(SVnh7qdRiBd5LR)YaRw-~!EMpw760EA4N6v17 zb~%;B2iyb0=?gj!WhH|YpZGX_cSV4*64J_tD3PdMO!@V1J~yT3laQnz6PAifk=I8B z%V&_Q*kIi1RhfLC7;6gSs7k_}_6|ZLERPirB<4aQ%2G766O-gII0&qw4NB0J zWK^_dBXMf36qOXcs8PkEkkg4LZ-YvC#Kt7BNpp`xNb2~Celnf;?ECCzpkh0SBhhH|lV z+6y^NNP)k^clcbUAGbHOwO*i2?jB`yurhF<h8mJYaalC0!G4!F0(akhdMV z&(D(qkBn1hslzQbZ(WD05IW05=8y6Gf^RzWW8bT-$x;J)i12l09oa8^jq=)8&;eq= zLFyO~k*%JNj=eorV6I>aqW#=fxj*q|utDp}^TW?tAY1E*d+DGBh%^IU3Q44i?Pgxb z6k!f%s;6Q_DPgaTQbm0EahVIA+_B3shmgtn_va!%$TeJrl_1a+GIV2cUz`;9sc0T4 z!S|x)54lF79yE|*Y#AWEZe}#~gmB0THPl+Ux-8n3F&BDDuO(HyJYrBaahjbEy>^E> z#><|@xq&?_EIcS$_Rroi*AAx(xdvljq}sB-hRChNOgc*Kv3{Z`p`rsL%bDb$L9GV$ zP$hBW%^a@hmI)_e=!&9B`fH^kdRZ4pWK{ijBTa0j?e4wS!{Aa0UU2Bl6TqXxKr&T9 zXVP{i%m|V8J@T5d{tosBJp^+#0|xKD5Iv`e{GCbpR5h+ADY!DT&rqtEF7xJG(2&EA z&wN44r#z=)T+olt@XPm4Q)h!ERTy-;%QV8tqBv_sGk)J52YIyLeok?LylHeikO!49 zRmxGT;_LvU1Hb&8m7)XCwpfMqf-K>`^#y~KpMFe>v^ZV%CNgVR z%fvoagz*o1ob8R7C!z1?pJ=j6WW~YBEq1>A(i^L%Q27UjCN#rW9! z?FfmbPKzIDu=zY*|0@7EDk zBac-U@qNKgm^)0Gi2yyo4Zz5IC~?-oQ&X(Rn7XJ8NKg|3yg7AcH5P|}z%TxJ`}37) zf_E$~d=Kt0KB1LBe*FVZYA`N{=mEhiQS{Q>E0~MBF(0dZRUNCu5+=*9-lv=9{o)w2 zk~M%BjY0@*1dcm#F{Gz5J#BiHCH6k5=l=q1K$E|-|FDVwXw0kMZZ)nbp;&EfLHU$^ zMJ464dS~bNnc~lxhAwL@>wKira`?R`vv1h*Pbe>LV3}PqvLM%vFB$Y|9aRum_uBM$ z3b9K4xn;C;S6VuhRy`|f7JBr&;)6CMx*7BS1%+ti9q#CKlvC$UMm@jO-^o}n#y(U} zbp1hX@n;v={VCJwDEs4N)cqOhWUP<=xb;LgZxfm|rVd3ldr~#c99;z={iO<6v{N)y zN-q)pe$o10BLk&&Ph=UUj&w^#9sH#_7#GkXKLB+*OMY&V+YVK8Mpn4?cEW zju>Kxla6jJtuNYc{Ix#iCtrdN0HRh00At(vgB>RV-oO?DL8LwjFn1YvOI;g2a17KfvcoQ4Fl?G;L+ zFhJ=jMO|m3iCLYcKGfZ*A-cgK3k#TTFAt;k*>_P64^CD+JcKgV$ME3R6Qz=3HoHSH zD0Ts7%avJ#KSNA68_T+A$ zO4PZU*4JNx-Sijd%oo2tz)+}Bq|Dc~gP5GB4|5jk}DZGF|LX9o*NTZm=b2v6B z>*!7uSUitoSd&y$CfENU zW+|UK)>2KV0Zl2@Of|REaVphPE!6RrIzhFxR4a9&sZO#~Yt_b5ZB;uQNDfz4864HA zQcI<&bOt9LbWzxiYS4oiSgdD?QbWb220~0pEhtVcpW?52V}Ap1-jOud>AcoK{WzqE z7;c9eMRL&IR2>*hJlZhRR^gD4n(E~G(1cOLO7%A0Q31P>j`dKaKcN>NM5U zRHri-d9*9-m78m-Gc47MtYhzbU|?tuw5j^kL+PRI&!+0jV9e1j-QN3bs($sr_@M!0 ze18U`kG{nAD#oJEWH9RJ7d;HsY(Tv<8kN~p11&X(#>>GpUN%-kOf}R}S!x)A?4$2A z*4UP5so~19)LAM4ebjo}Ob~Lj!uKdzYJ?hTs!^62t;Settco$!+4OLnrN*lX493^b z+-!e6_tvRop*#xJm>~- zOJ%E`mYS@3)(ce5)ID`5=Lp@EXg1?9NUrj{d*QQLG+jxx=aRpOf|*CX%~w+_RiIKV zHI=yArH*DDbi*_TL!*o73$)boKx&!496hvL|IE}tacZ%DM&Yc~9PNgu)@)&WjKu5o z2c7Vz%KK8&sHKIf$Wp~BV5$;Jov+GFg|JnSll}+aXQ}CGhNWhzSr-3^|7@xYEHzur zvD933k;UKU?=U!3Z?a4+C`&C4l&6+el#~QY%l)~jd4bZ@?BZFeAXy zB~B16E&d_@h`}+Vi>DO_W)$N~y_F^v&2B+nfsVV>#gI!z{H}B_E;xbm+}>)b;C3bv+rL8!WYis*JBbgn|%P#SWECp}JG2WKZ{}W~bVbqFuYC?zTscdrWn& zrS4PrTj~Mzps60R)Oz(WgLeIBx}Hi$dzI_Q(u)dc(@J;`0(q$g_AZelaxD+GiHe%| zh@~DS-W*YkbPyE`)$IqA2R!^eIunQwlIyh>M_7V-fV^7Oc?Y#qOsW|5Ov0G`W_6`R$vAL zed^pEi7W7RCJyvlRGT`+hKPG1P`jgFI9jM{0cXD#)ddY**rg+sT_&W^2n z(NZrF#QF6RSKAG@qj%S{+ePEuf7HvCdWGi1UZtPiY7c{Mhc~f0cGyV2zqrqoX_nfn zhM5X|j@PK@*VP+G7%A4hslrlkVim~Zw;1I9FZNSd>TUIorQTKVF=%-xOtFFJPwZFk zTj~S#AvN*rZjVVTt6IewVYsAerC|D#_Jlwghans z>Q@T4JO6{d5|;W+{cfp0)SvaDU}Wem^_Th^+i8GhFoRo$P~TdHPNI`n}B7+5hLP>5ebw^q*M~MwgK8sBbkEa zh>KEnZ^tJ|xYIHklJj_cpX_4lB7z+#L?=#HO3og+f_5&`NU@AYs-0;xu9v9aS$?;S zV^or5(5i}YETx*LB-3bGFViETWp|RbW{0+7A##MedA&?R+pF}4Hu6M;jX16zrt^n~ z7V<}zSVpRL-HjHOaXf)=LOo2r+NB7S!jCG83%V>`#*63iI%%E!xeV~0?skDnt zJL)(bRFwI3&YbKr9l6quv{cjRXc?W1&I}~cm`0a+u?WuIfI|&AnheFHYrQl=FUDI& zhHA;6#UYn*Eu))p3WhxnTipfx)UrTcd1@hs@JA?Hb+6b*7mc{YN2Jzux1wdoI!`L8 z#whn_w=su#99&diKkK7P>Ljdjh#dn9l_l&pt@i0^b!MiWTCMfwp(vOiwl9D|JXRXE z`i~7hShZo4bdDV6&GOGIw;RJCCBiq8Z3Z!D6yfm=@>Sk7euLy2fe;Thd`Hel%<+wywSDw@svdz($#F zFb1wi_FYYs`xQex8WF}LYQhL=0(IH`rsm{m*cPt3*zfGzT;1aKnEou{BYBw>wOhrbUy{I5anr^v4<*=1S!utyXTI=Z4*-&9u= zx4~vkpt#siq2}#J2KCYXX0TRi`sNT);*t1bnN&_<}uCzCZMl=28o?RDB&{q3& zy=dl@Yn_l+$52rtlSA4rnyEQwCr5M>dG?MB28lJr!8WY=Bwh6Vs>zFf*gdqZuIdq% zLRqql1I1KDR(6rUTER9-3}Wn$W&EQ8by!Ip0xZ;cy@|u#!X{gfCln*@q^fyd87X-^ znf%)3%05*UUE_*E#}-F}QD`Gabkw1Z8POqXZ?uSxS3?82Q8JaRaH}W?hm}TGAWCUH z%!Jl)MjHcfzB@FcnM1^Eu}Wz@Frp!hMkV!JwOZSGfM_1V`{*WkdYgdGr-q72}g@ zeMzfZgA^mnvmG5BHr-!J0VeH39`uh}7onOu&M>T9n08kqvTnT1T}Pxmx@WomnPm5L zCNDl2Y@Z8YzYd*~gvqRmB4mS`O-qqJv#)K7iN>n~u?PheZy+tFcd?*5e z(p)>9v_I36z9tqVW+8_H;VXM2B@@uF`E$^1Y_j{U03arUfm&`sp7u&h)M}ChIe! zAZ9XNoslt`Wp?}6n7EKZa*hrpyE&-pT$pxs#+^0gms%R@WOhb~LBj@S*#@vnbS9Vb zwhZO!9@Uo<)$^baGIDUgvHki+cdUJ_3VZz15!oK-C|n3*UL7%hjbq2g*wPuA;O-O+ zt7;y#L>@_~fM~uJuby9z?IZdPA2P5{uTjK1BfEsw-}Pr>wd0~%kDTp9m0F{DRfj3{$6zE-I?EQMfqL&eebU;pHcM*dp~{Q>Gn@mPnzycfa$U7En|yV? zqyv^{VhSD5$lm3$qvf?7(w@{h3SeNZkFUczrN2NN+Hb^}s0Bqw%kPaKu#uck3T%dk zKEEat(y!{f?D){F{nZ_s@&Ua@4!|%4pZE2b>wQpeUP;PWN9^WU9DU|d>1e#6xmFbX z(78MM;4a&$&HH~id#AQenFVFKZ5ZKD^yLPG_SB(S7Met?9i<#;YxS45(aZoiXTDH$ z*iTnNQ&mH=i?ekQ*0~N_WS=oppf^0$$b`C;Ox7$!fxRx$;ci`at*We=d9bMFk#Y6PTC8dIo2_L$vPhdwgQ5d#W zEE*B8cXLJjI@lTev{PJgfeukFC#k%|_9zl4hcF=1I%Q^@jTq(XQ2o zoet#8ss~WZNWkrZD;jC2Qe%?Ws!x#`7uerM7G#(E>1qGcK#~2X_E`m0{f#tDT6oka z`D(rhZMxHpE3NnHBA5g6sd|;>SI{w|D0a=rKgL=Matbh%>cdpF_tlg(jM#i??c= zkiZ~CFCx?~?yRG{4i~asv;nYA6CZks?dtsxhie?9; z`D;0LkQeq38{_q4Gi?Tx1bEoTc}VX4ZEjChoa7?)+%Tv%^upE zd5UNp9%rwhJ+1Gg%0ekmXd8o_9APJ;TA*|`_%XO9YK8tE?K_CZa}RAFjN*H?s^fZW zcd=>U#if^$OeSXBwxX>8Q3Nu;*b< zn6wPbkGgmI;ZB%T%dcnX%BnSzK}M}2552e1-XL&@?JzYbToS^psI9={=~dm;_VlXx z4oxFXC-lW1dj`*<-G-F3J(`s2^Nz0ZibnSx*EJeaz1z^AN)&`Smb zquJ0B<$jyYe=ZBqf{*7D06>AHb9R97Y`o)qGT!I#T=(9O_jx=Y(~^z`($}ec8eTgh zO(8GB-(tL*izRWqX0&a$3D~%HmB1@OwA}#mF;HzcfibQU%*Vik$rwy}AA?v-T9}M` z4C2@8x&runEExDZ6yjh2G=$-B9GnGhVFYxAk-P-oG_q^q=krnkohswyz_1k+e7e(? zo=8JfM%#9eK|)p8Uj1|owsdS*S%zDdp?gMU&EPYgvO3T`@u}^j5c`l@l%XF^(8X9b zfm4j+v-kx_g_+r>VB%YV_uB;_@li+`x(OPL+XTtu(y%m@7KDbCkb>_TRYK#ebW9u* z1Rq_G4MLM3G)+gcc)HyP%{Xj@=0P|v^enaNSqr>9K2uT)(}Qq=CP?21Ed`AANqpBT zQ~A_Rkn4SFBbiQnNGQFGd)zG&x1v_TL;ZDW0K7G zGQ17aGh;VF`*FV56tC%vse}$&*8Tm+dSpQmGi`@75Sl^bk-Grrfd{goCFJ6BKXijU z7!3JvE=<9@sZa#dU>5#f0s&YEC9nw2hqX|OB~@Tq7hq}AVF%2B=V2!7LdtjGLf8+p zHPbIZ#y$_p{6an(aoq(8d=8%ru}FUmzldK9F<8n;{1SdCwqPxE2*_aOZDAsg>U_#Z&{AH=s0VMz~TS(RAYBM9fC2=ikIa1i0&fbegu z7x-^HitsOu0RI^_{JkQ<-@6j}X!!d&@b{|@f4>Ou_pc8Bng2@oHzP}*L}qTo=T9O0 z+Y$b!5&j*>)}09dvk3q5$i^2C{udGcmk=rf`eiI@FH*mPl&>PiZhX22;opbwzg92s z-*gn=zc~W@18n#QMuLA(B@EW^4{_ihS{?qO5#Z0N4*#%!CH!w8OW#IjzK75EBmD0p z{2w6vA0k^nM)*HL_&-B7eva^ef$)EcP<@3ZeT`)uz|y`!IKM@hze9k(NBDohQhux# z_?I0;_?Jh3f4B|*S&`r$Q3)e8{G%NBM^}e`bOiXvREK};zY_jmk)^*OGylZre>d~P7+*{f&I>|zGEBzbAml8Eq!dH0#G1CkI2nZ8`3BRn5fhRP zFUdwM%7&$7!%B%YeO_BOt|8gjjjFrfm5n^`p>i|^4?6~0vL=wmn!*{Z8H{4hVLVHP z$*ctwvlC!CYY7*!Rxq2L2p6%FUwyw+8cM|J zC=q8s7uE{~u--6{^+Ac~2UA&pl!yT^mkoq@Y!J+6gJCrr0*|mP*vv+v#TW)pvEi_j zodqwj5%4A(h0-zFwH7;FYw?0>EuM0%#Urk@SnXPitHZ3tyRH#!E9B4A%?V_y~t`|?Vt(AZCRu%8iz{mcm1&#I371^-Iy$07FP z5&MaV{W*yJB*gw)#Qr?QellX8gV_5K`#i)xAF-c;*cTx7(-8X-#J&)*FGB2#5&HmQ ze?DSgS})kIK8o0{iGcluHuke4VLzu5=4$LOad&K$xV*LYR{S#vSJ7WDaV*LwZ{VQVq8)E$j zV*O{mV155l#QFgTYpT(UZ`LC3*Vb3rCJ{0cl*q8C*es zZbBCKzPgYD8AJh+K+@JcNk!N3Pq5OoqNrd_5OhYpko{Yx`YoY}VV@c)DQO z$KZ?k$iIgO8@#U1+5`_r0_c%Scr;ULUOyIupiWc8(?#$nQrT-hbCl5NBJU3IXW66H$&TTLIWNXf?DVE zIxFg=o|Efn-IJM?&%*MzFx1qoLD=TA61_oq$`|Xiuz3f|80U*iF`P_sp-gc))Aop& zly0cc!oK<9sD+#0>2b-hBM8q_!p=%~HZ$HAZ*%H7Uwq=sAUuyH#s}dAUwo~+L3lCL zy@XBhQXQ?_m6_;Ew2Sk~YEa zh(-F6NJaK!Ht;pDi`}c?Yv5p79R+&aZWHXQqj9fgCi{}@QeNK#Z(v{E%xow!Qc_ZE zKvu!op)NM6tBWZqmGD+(V_)M?CmZX2H6}b>_BHY~#)kPC`WkJ7w>gZRpTgGtz1o+I z9C*i;9}*2i@v6A^67Q z#Wx?)n~$pAe7wb{sQgdt%cnk-Vg%u{6yph*X^IR_iYLWb1qnVg#Un?}_h56s^(mhj zgwI19@`MsGGQ*qVO)*wOZkV9g!6q-U2}M1`CX2-DWuK+FWy`wH#8^a`;<@`jL z!P}s%Y>NgX4Jvs$yu#bVo4f=3%ui;DcVrgt#FBXzb~5kEGI<8;#k;Wq{0uga_hMJ^ z-s}e6hb`xQ*+YCF+sFs8eS9!`j}Kv=@S*H0p2fcB!`QEUIJfu+p3X<|3_gmV&PVex zd@L{HX?`Amo@et{_+28u3l&bv(i!g~t3c9H$<`(JloMn81TLPI(cB5xf%VViAq5 zd;{MI7CV{!!5_y|9Gl1PMs7U;@oXb2)*(BDYv7--4wiA*O%xt$Z6a z;AOCpKgGA>xViw=A(yW~Z>y45;OIFY$JRUeHQL+ppe!uqJMgV3P7RI0;WHN=H#E*;>N);Ajq`H0GtS4!8T%Jk zgb9?USFu&=;2q5uk9Y|7=w8Oi9xy<2$gBN<0q9-W80^L;HXrvOA8nrPL7v(C*-NW5 zy8gu=oO1WL(!dLC@!TV-NO?$xv{`v5)1^!KF z+pZG+yN<=SMar+y@xaSB`1dXTeTRSFoLK zgkP3}wki#Nt%TpUP}VFe3;jgxdHh}pe~b-PqM!UgF6OtD@F%&8WQYIqDasavzg^G7 zCu>Fq{cA)9vDKpk?ZJ?#^N^{FK~v||rp}uZWBNRHSP-%w1P zr=qNppIQR3j63oh!wg4$z3`{wr+ToRU9^6g@B$2VqKV<~k(N*n7sI3cHOwJs6N{E) z33sDc{5pRFR|plfMPb^;m8C#VA=-QpV~3k2qTouf@_SZaJad(6lFeiVT?| zo2`OIX(_UEhDuS(K#BY2OB}bpkoQ4oV`3w!l)nrO`77W7JQHHYELY#A(;GK z{+`~gc^u!56dcn{`TN*?ft~eX2WMkruv*tSKLO46tJpkF|JDTI!p4cU$c_fssWoZZ ziP;SMG7XVo(x`1esbnJ4lV%H73^_|ATx<-b9O3e6!ScFsOKjE9oQ?x3leDC0dsXC< zkm%BYT*(wUrR1Nu{!~1^*HLSb8Do9cMrLBI3BCkZq!a8~nI|(bdq`nU{h)1-Sw1SzX$#)Rkw{Y&K^DIiVlXGxj~?s(bx;J|JFGIu zl0s>0PXqGf;?;k>pe}X^JGNrNOSsLpAA2>?498N@O zc?V+oyAY33(Tu+j-T4PFn12YP`Nwb${{*J-zyGVH0rNYbZ5SEtMKhTWsgy#HXY)2BTM@6JSFcAZMTM-kOiIPsL zo5(Cjb(2t?ef(2Y-2|55D0U*tbl-N=1Jh1OfWc@8p28q97S6$zea1gWwVcjk_!l_9 z#-V!t%D+T)j)(57uQmW)XvNOa^~R9oyNbO=4`ocJ$N+Gi`U?3Zbc)P_)TytrEeZ|J zGX4#6&%l9ysUF@;XvP-t?`TkCcOW)oJYuw=Af2^2@I7+CZs!4Pr`@_e*gCt`#cr)X zV6Ap(yU~4gO^dUi`$zmEgtQi;sW54}hK~rv!QnF0jDlc{M#kJ#`dvjAKFx4wuT+5K;T80KSEkgsc49RUVCwhRd=UJN-LlaNFvp#hqNWNi{cWM~rNLk6L7V(|vnD3eOZo?y85-a1Ib=(Y~~ zP=nT&P!;SYgvYRP?x>y+3j7iXy>@o95UB=;RL$0}`zsgGw#)v+7h^|wF)0`WeTL65 z(EFThj)V!22L)(z2C=i)D7*&PEH)cHU|;h@p6m|%zoCqSNQPrY3bYW7ptWcWy@d}3 zieq7fXaW;NbI21XK!G?8rioN25iOux91rtED_AH_gv-TAuuimwheR8AUbKT(L>jy) z(%}=)o+;6RC5n?-6VZva5S>{Y(S>ytUD>H3gY^*I*dTE#J4^IrqeLbfBf7H*q6a%q zoW=s;3^q&jW3xpsHdpj!^F$xENc3e(MSrwSXR^D*0QQ&|#P*26>|HUGeJqBtuf+(i zFi1%hXY*t+il>Ot{8%xDH^-nQU5w)=iwQhaOys@9IedVa#D|GU*G zVtDDiSb|k=?Y^1h|iLj0>)K)PLu4A|Bwpe6ILRJw|j#b2z zV-+#=we~Y8^^0v4F-7)K=@j`HhE9?7G~rl&iVg`p&>Llj9IhBx$X?Vo(+iig=lHMu zHypjlQg#95aO!(VcG`Xb+itgb54PBD$zrD^WGT&B_VIW8`y=G&i<9lidp6&3t7+2k zo2>MZsZ9kn3{Ge zhU}8JE%``QcF8=HT{3)TkToT{8P5u)_!=G-<-&DR>TEL`%|}jbrX)39NvV5UU&v+;SmK zUtyd?JxuLxFI$Rh1fneP3x>gDLV)CgQn8YK=0-${wM0) z2-ZpKwh3Rra;;e&tyyi6{su?0yiREtFgs=Zg=N?|_k`<{4w(P8^(b~XTDD~LH!$F3 zskR=m7FnCv@#8Al37Ima6e^T3Ts1+$mYE*>ZIu~=zb9sRooVt|iWRhn(3nz=hhS+D zv)q^>4h1sK)(XshlFxE9H9k~9LJj{wm!0TKboDUN)x)HkdYD9d*nsAL!}KuWPWYW_+4kbeR(nGJl2R8>u-vkP1I_bzmf5}ljBKE~d*d6>0b z%i3*X?P}Si6fY@iB}*eVl08ZH(TW9YPv*fQb&N}9Cli-YALCQvOrNFQmRQp1jv~WK zu|kvA=cK1tt~reJc~Y#-%b^FkX`N`Ie<#FlVx1A@E?e|0KkK>>7}CtQFsxmnZAUEHj`SK%A;J>o6h`o7Vjat3*MpX&W;N~{MXWPNAx7F9 z>$VmL{ftBnQ2NtWRdV$Vf~{)VHf38F`<`HnP3~<|iUY zK6m2HQz+i-S1aD!#QH}@TKPlu%(XM%P;6H?;Q!`7~+ zMNkja5Li_Z#1WGn2;y7_;#~;hT?i}(f&_vf5h1AAS{DM#fq=biLojG9JF9IvrUzHT zAIn%jThi1@ifTpYWwc{i;>AG*MR6I7*u;iJ?z;4um27BcEO9O_$^0!NKAB}@>Io^g zx1oV1Bnyz-HZ5J7ix^r__>+``xG~HXNLy)!A2w5BqhLykwH$`G^?A~;{W0m4>?~Wt z^_rXA07|x_4mtW6WFzQjB>jxC<=gI1oHKITd>y?Oim_uUUc4QHjyhS@lPclI%-CwR zSuW;IZI)A;v!oGZW9erk{fr7{pM#?>_F;B5O=8;fd|p<@Rj9~C|>VZ8*qSd27TTi zaHtW>anQdF2mO`MNUVnA#99m=?uL%yUg#$7gVV(Q&__H7+2SEMU#y1<#lsjbJObB? zM`49{3^t1(JS!@3px*#*pjmxiJP!XAo8UX~1pFX2GcKNFme|5l#8a%f*v?K9PvaoJ zgXM{5Sb=z!m5CSF_2MPAOzdJS#U8d=yv*(tudsW>tL#Csn>{J^vK`_zwp+Z;_KP>z zXW}jPjd+LsAl_wvi2YoMkGUb<=N|C^j}ssA2I3>$LVV6o60c#%@L!%GzTu<9cYK=o zi5H9Sd5Ji{E5r|cmiUohDSqKMi(mOI;x~S$_yf84H@{bMzEMj4gj9T+jNva!i|>_j z{7o4zJTg(F$P{tBY$Q6!#^O}Go-Uh<-tt({S2hs?WmAzQn~8H|3$Z|+Ag+?F#kI1f zxK6ebOXZ1TxjaeSE!&8PWLvROwiB<(bn&(9B)*gF#R1tt{31^lf5?t9UUrd9WLMc- zcEePr94UJsqP-B)0dltNE9c67a-QrjFOz4=o8&;bPG-qUd6s-qj*vT%dY2p}zmj8= zkQ0<4$10CJTgAz7DoKu4C&)>vwaiu-a7eO0qB0&&RGN4>^Nsh zu+DML64`0cQ~=M%FzO`eY6o6$KK!VskK*8Rct<-^@o+ut(JoX1YYqbh(}7YpQ`+gW zU@&X0^Tx7!9A__%J?OY|@$61?_AU|}EcOjF6hcUd=Wz~Q0&n2ZC326ZQ{;;ZwHS+6IF4chpXKH#aU8{XKFXQPj^jO@zFF*gW(Y%=7)HGb-Ro_{$IwG1#gEF)%dgCbB})YND^xssw&7>XDbfhb24#FN;`VLA+S%v??Ao*3OY2 zu5-fBIPptpnoj)TmUqBubAouyX>+{T=CnDE--WstM**`~082%@NB~RDb6OQIFALQn z=TfV%lsGvH+KWV*>{D@}+Eh}gHsuM`Ccg^RCijJEQ^TB6;#CHmuGg93WF>5|%abG7 z2_gymYpKnl+SF1Qt;5w=wFWVw92U91AtzmZCmM)ka?;i3AtzmZ&xuBuGN7Z{DjI7iy_hnB=J#SF$ySGWu7#VUY#GoWL zkPD%OTm)@V76-^HV3@oT#>lH+qP!Y%S1TK{~!{u^0+=%qI$lG9}ydAd6 zJ7K%L19r$&@Vs0NyD%1iN3PXjn#?Fv>@+lOC!B(AcF$^FcDu;VwbWS2 zX4NI;TzB{ zY=kaY@ELL&^pTsPuY3{)$}Ny3w<4z7AzSXy_>4epOn{D>?*h7@J)Wa^H_*XCLKa%# zIeNmYFWDTln8c~c;&^??3Z)2JYSVez+H%b)1MkzIJ@V-U(bCp5Cb2ZyMOTs}hxJTm z6EbDHXCSxTMmCYba+uK0c1{&3oU`Rqxz4HKGq$mFmcxm3owOX9)Aiit(1@<*Er)oA zhkC&_jb>_ErP2EGy1yUMi;cF#ACJ8S`8-JZ0!sZ$&`9n=F8v2a$(P|=`6?94eVQw@ z9^b{;lW9P6XHTZ0w!1P|B3hvwG5w52Uyi*ibodwp`Qk+4qq}x$(=P^AOO8&Wi3icz zS*h=2@5p3;vXZTLWU@$G-B?Z~%cUKe%pYWVbj=U4DUtSHhEEsNm=C~Q1vDc->+xYT z0yGPdZfBWVeJTJmObhMidFIb0yWfVnt2E|mX;CGs0s zA-{*!@&G&`e}rxFCwNBw47=s;@D9SdU;Y6f$Uosz`4@aC|AyZbhrg9@zslT*ZP2?n zf%ddqs29MVaXb!>`^~1u{bn=9{bti7EZwOt5N$+Ty=(Lrch~6KqMg7#LLn^C{+xsr zFd0X5vRk_$J#5z~9sgrfeeY7$+CM7N9gB1q)=gIPVMMLSmZaiHk`%zd5`2Ua{P~rv zR0~yEkd;RgCVW+)1wi^_NC0FF0Z@(r7|CpUrm3+tQ;aGBP$2;@NB~R}fGR2R8KHf? zSHKdu$;EyUE!`^#$WBbcKa!xgFtEK#X& zlWGdfR5Mtin!_q}9IRI@+`ZW)4yFobI+z+T$HCM@?j}1puW~T;phdDVB|HDHYX=?{ z?L`NT={*jn9bld~S#(59N5eT9Ns0Zj_y1Py&9>iF-%UaIb#g30va=8WXJq;eX9{xW zGB|z{n-vMh3o6-#S?l1q$XgBZHSE&R&PLaSu5-{gq3hhO@J5hbM9GVT@CscoL5Bq| zm+B4O;f6%Z!~bxj{)?~=-l+a5wmK1opfx0@Hqc(B!64NEMyQif06KzSb%G+*87fp4 zxKMS4Md^G!|ysI7CbxjsY zWxBi9s+w4)*Obb>bx5UN$1uoisV0>^SyL=G)N);}-3jeBvBi<3^EyUWja|UP#itzzBOYBpTm$wsTSY`j{>^3}bpR6WEh)O~D*x}VKf53q~X zgX~)MFuPej%I?5uX^nb}tyMwxkg`wNdLA0WL}=DAUO>*m2+=10hZonV5PL zP88im4=~tJ#}P2uXvYzt&--mhAcoC$)$d|g{bq#JkF5o-4v9z+*Sm_iUI=lTR>ZsA z70orGr&h$K>_%}qDI&%M>+QC%PH>T}h%6r!tGXiE{UJrP%lOMJV-J>L=d7)WXE;va z-HsFZAi{lz?F6ohB(|%Onc6+LlU51vy4GGjs5ScIt96Ir0p9h`JwQ5LZ;5NCv-EQ8 zW)#0|&{A!OH1#xeQ9Ixa3`7Q~XVD3G0d4h*P@;CBef|$zpk77?U=LiU-at=qFDzC2 z;1=~7+^$|noBgKi1s1tpV1?@iUf_Cxx42&5?XDNNG|UTp&>h+z#G$>H9@^Krp8b`2 zwpCz^eU7uwc2O-a(0*6j3+(L-?aQ5buoE)=ZX3XRA^~`BCA&`pcs~NTi9IlmJo*jn zK~xUsFD5wn9Aw!ji5k_wKXJO=eJp(ZmhOc1#Q zU!xp-i_ydZj3$0S8TuK={a;|a`VC%Cf52Y#H@t}=alqj4iy`1oLo%=7VGWHq*4T() zKEunJ8?o$o!(ttccrAbTK@v=Wk)n_2i_$a+T}E5hRzRX7Ya?JHe@65}>HOA>K!0%~ z&_6>FC~F>yK-ux32$cE4BG68*Y3d~Ui!-%pY8#SM7VpUEQ21C35CakWy-wVsn77t< zi+i28rKU4F$dTBtj>HOtdE-H5Fdh%hU~EE(jU*-qpSgIVu9=IiY;wfOi!kZid>GRA z?^yR-eQ$8i2=0hL%POq#2Fi6yq2;!|=fn<5(DLG=W^B8B8^rL#c5bOgCD< zCC2e^t#Ja}WVC`ijMlK$Xaie}cJQ>32G1Gku+QiS`;83v(C7po8=c_`qYHd(bVU(7 zg>mClH=cRI6``kH5qi#@#Q4IU#Q55s#P~Q&goGU)=}wlEEGgy~dgKU-r&ZHwbbZr#APWu?4>i(z88wo?giEMtnZ#0V6!JanAwdKbZBF|t~yte=QcwFGLk;}Bi$ zI7Ig&OJB0B)2>K@@t;cevbIjI1lg-}-5q3m((NI0FWv78ve%sVuh%hE_C}SXbVg)H z=`h@(1#m4~U)>!tauH@f+L3&;BU7M@Q2;%RsnFM$hPI>-Mj1tLo^d|fl5&K;0u~rE z(T>c4>y25k#JB*K85hEB#%#FHxJW}vi*L(Zhvzoe;aTE3JPVwe4w?wR-*tWN7h}X& zZ4d5r?ZM^ZY%va@nhJgF2{0ER`$=twXS@T@CI_I6$iO#kfZmF*MfGhB&O1T&E?wUX zvi)>@Kgd2vPn;QKAJ*Y_d~_JT>&O4xcfAfh#}zJEV;}{=nulO5fK&s0K;tqv*|;1H z{}s^FxC#atSHlS7T9{~D581{IaDj0nTx2YT`NmCfopB4?VyuJ}#;vf@xDD0lH z?XbaE<@&HQTp#u#*N2_&`mlGpklyV=x-!g%-RJ_jQA`jMHIVBaAX8zPI7duEASb#r znb+0wVJFrxlX3di=`H{58?-Lu+Pa{D3nTGUpLP%8GjrIjS|1!wFqU(Qy>}$Gy z6J+1g^}8VZo~{Ri?1%JC?8kAK{;87v9Av-H9w3<{TV|+a_A4#xel(6gGBSOVHjGtM z+G_Q>!XRzyAA(#F+;{-aHXcNM-GI8f8OQ#OP;NX9v#`~3jVIu0<4L&D*b3{7?P#u_g(r=t zVXLtNb{J^_)JRkZ%5N1I`ai0Emhc+%`BbWx1gAbjwy<>3ZPq+6Q+nm_8 zolI=o&cwEzOl;e>tt+-|I}>xB{5R^^_c?XmoV)f_S9SHvU)AdFg_8O}5sjy#tjkeE+?P30t4 zT|uce?66(>fvx2_f6_p0oW8jsP@h?5`DkLyYvQcK383=*cx5#9$oe^ z2f1oY-B^#d?_@Q$ubcPZLLBWvM#~ zX74eNs=600v^pNklt8v4bF+sa?B13Wa%5xTKZ9R^umMQzk}AX^^ORd`Bk_10C*$vO%9oA&%(M~# zpU$21z|#Jr_4cxYy5;sVc|$#5KEp~%6aXdg;2tn?-!B-LT`|Q#D$v5=wS8}y^g4*) zNTs)jsb>GDM8IBdt%WM36YcK7blwx#ev*N-f)OdSlu>e507kxvb}(@pde>h6NZqla z@elcOrqFy+@`OpY0!$Rp6{Qxr%9b57Nlt$nxMw~%BGv+FGoB>@kn9X18fwCntcng2bWfq{& zI*ql`c}lj9Vl1B9jn~iXn698PZzzy%9HXBL5w@#$$suq0?R!`loR}W!telXWcb4!M zsEG;B8V%mmvo8Kw|HyALc2!d{in+HAIo6RZhKMo`*D2(SvwSZ>p% z@{@nN&RGL*O{UBaT7wr(B^q?CVNbJd{LD{BV3~ZfkN05Kv2Ya==_D>l4>yM(UWZ5K z14rgVN50x4<{l7r?utBZ|BEzt){MMF!Qc`Ej!+O%_PO-W_lP-%2ANZgNNa}6@b`-~ znUacV@9q3f{vd_7ydbm_UTJYDt2FN4Xg?LZP(or)G=PzD+6t3-b6(~Jtv>^=p6ao0 zSVNR!dLP6(2Kr%y5GZ-9kY*I)Z()XkmOhxbh>Nf=)o2jFHNJJGe2(z3uocitX}zm| z>gYk_=uvX~M<;Be+AyJzBXJaTNJUm|+HrWIyv4ST_0l6_YV1KG%UXYo$DBR~_Ye

>uT2K}eIDlNvaZCeX;-3O9p0 znRQ3i#I2VuuM#7E7mBQ8IqpScJh-1z@l(t)-_pI8Al&&7w!2LpH0imv0_WB{%40PN z4%%ZC2R=x;MTwjA!R@E)5PQ(81NepsfJRcxzh|>>cLF$f1dw!7&~avR!Su(ZLw|LC z7@8H@9_Q&D4}=`6q*)#3H&2OQmiEz^0dE@11%*utL1h4LgSqfp(tuaZQu_Sj&+u&yN2)rFn`jByV^RKo*0=|rV60(g7V>ge$FlJ z*!(!xC4DoD@{^J1)LfX=PSGjeqMeFy6`6?&=|Spp?pJ1K>Ru( zI&3~|-_janWcYcDN&FIt^fB@MpJUVq_L-t_DC@=sH(sI0nk?kwdt>@p~t!wQayl@YUcp#)j{;iF3T9G6SAGkGiT_p(#HfQ|E({!P3$REQ8{=8p`ztN9G_w6l`*i8OLc?bLYU6T@lh@9^|ETXBNMI`5i9sudIe? zK~^A~iB&Dt5M4p5?1Ss?ZTvYJ6h72YdjdG#Ji7O|GCVP&&C^U;&Kma~!NN)0*G_CKO5mXK%%*=k z2)vD8CM)i4pzm&oNwQM2sWe@~?d}x()o2aTx6(M^bCjJ>(!;@l}Xg0;mlp+WVzzcF?CN!IKnIZhDR_P(8Z{ z&KH3wL8B=~c(-i93z=(wLFZ}dfdGiP2H$=wn0M$ zdaah4JBZrF;AKYV>PZ^A5X~jaE=x^#*{pD>1;7=QvUtsMSvEJse8h*!s@B~6nef=U za;h{tM>e+`o!3bbF?N%EvY>0pOK9(bR(57b%guUddF^~m;>du*w}7~NjlbaEHPhk@ zo9@!;iR3#NJMK1m9L;`00>dt(WgNvWnr8~@4FQ#wi994#SE43{3eUtINkW&M8bas9 zeruDprLwR;BG!BBrBHlucsfAIrize3`l36zF%|ahikM3$G1CSI4blvoK9)O{>*#kw zd60~5>0!Te6fD{hG$RBfkRu!-pd%zBs3LGhbBtn)a*P7{=l=Tk)hEXdaM~_5L*VROSSU`UxkN zPw>m47YYIi5{0o0=t4uX`L}m}Pzs|V_R6G$P_2G36e~`kA=OA>j8Su_ChVwImp>bf zaRgh9M(1oqFVL8`;c}41kCwnlH8Sf-23P~hW~L0}7EvRLY2z^7@UZ%g@U2&xC-P0D zRml*#J;@P!YvIc*$lr|5WHsxFn_c>t_8TC!nfvqBDsN+q=!QR@2(j343-%t-;TGKW zeT3z(Vs5`?{i4yZAdK*vLh!hogk`x?KF9mwVQ|a);*}6G6a|~O-pW|5zK0uf$leJ*@$cBLVE zoc|e`{>;e?cC@iFP~Jo5sGy8Y65RhB2y$Niex8x%oqS#D>#{oyb}jQ?10~1QsV}+x zR6*0`*Z+gdAktn6>j4Y|l>EJ!rT@*1{Eh2!w6JylZ)i!iva~!16Vew9R0K?qJ|qDL zY49-1aAfU-82g+7Z%VNt%|r`TWI*oGmB=mSMpA~o0(dm zn7YJ4OMEyhk4Xc*uxMO%tcDS4wJ~U6jX&3bpj|1aoeIu&xhk}+j8dCSSdu^m$HMh4 zs=$~iZC*6GYWP%8tcPz79agKVOp^VoazHpQQ&LK%#1P-LpCP|T^B{!qu}Ukc@LVCL z`lqe(I?h}al;$1l9rghAiC;iEsS01{#Bc7q3lhco#E zz9;ouQiIO7bda5{3rRKM@=M{5?`vT0hrf}Xwc!m*{w79y&_*?&Q!3@@T&oSC)MKTH*Q)aa{p(SU z%^5Oi^KD`8Z=rdqf7`;!-_TnZr~f>TtK(&*L79+3ck1=Ew5uQAwe8OPciV~0RlzW| zt*=z)ZOjWVY}ckDJ~eZ|5Wj%@k~lSKXmh&Zc;(H=$0ko!-XZ--F45xq&49JRtx-#~ zt`?qjFM1qg^H_x1@n7QOd(I>n)UaxR3_Y`)olZE_kn=g!Fk4%fWr}19Eo8}Uw3bbv z8;zY@*K73eveizYUKK>^e?h692=YsRpZ*NkTaZT?*tz&t`kAOgW_G>8qsu`#`)V;y zOR^t}s)pLGe5R40>YjL5mJ`P5Y@m3^^x1@$uv|&kKY!<6N7gtKB=4YtuE7T8D=>^l zMul#o64inBAlC)m?4FWAbO+I}#BqxIc7tmx% z$1o0|i84<;l$4^EjP7pK!}lQQsF~|F6?hfspU6$I&Ti;5U|MhWm-$m$O>nH|_BA z)0>ZvSjNx9FZ7{xzxxBffd_rhko1pO{jT%U6-HVp5+Q?>+uPQPxZ4OzTW{Yz#7KWA zM^q3Gkx4o)qz6dzO?;hs~%HGY}5c_VNQW6pqACqX>N&bd_WY063d- z5%XPHhMI9_lR53uN~)Aup*!kI;+?rTb&raIjoQSiWr6mD$)n1jswcL4>YN`cCe zMUk-eTGImE<%N`{6vb#2vojd$6Zs>!L}|)GQzvL+FF$OaNOriQrKl9v4irlzg_ZCY zIZzU9!oKj;4lzr^WJ5+Xh9_Qig=Sn-)}?mcTFbJzt9ECh!EtH@^d>Wrq!uX)V6|#; zXLNgGS<@hG-j1o!zFT*!QS&pE;aTQDZt)UtzU^d5fFN(>lLS#zNbHfVGoO~oNYc_r#-pYo>z38@sE?Ei<^fsKFZ={H@o2XYe6Q<7Mh)nGO z+$`HtYu0>YZUCs$7$igWVEriy+cr0>ejK|`UhQY$)Jk}Bu2MUEST}gAqaT~HR-=_7(Z9JxO^lRNEm&JG z9d=+_W3AFB*rhO=wHJ?phE?KzMVm_jy2OBg4D-v{#GiTyQV=!rTYk*c;&7Y;Liet1 zTqPAwoypcw?-2sCmnwnq9t#f;R=9X>DhFBntW6=UpsT~u(oq7aC@~`)?yzou-X=f$ z$%bM;DQ}3ytO`_EIZ4UiQ@%zQ6FWXL3fb|qrOrf1`3z>;idB9OHCwtYv}a@!{-E%| zdW=y?UDme)9q7-(A<0Rrz3;%fNiEkF{qNTD=>8zWbz40HR-KyUSt|RXE!+XX!TnsajdajwYb_$wuK5ojWZ9B{ zxbK_&i@c49MYQ~8FJrBY6VuNEd@Q5boTd7HO)~Et4J-h<3WspXOYkdz6_=TF+RTQ5 zeQoYA!G(r2Z5G^xgUiXaxG(;8#)7(>qlnESx$0&T zAVepNE77IqF$5QETp*|)+cJLETbYOKLxN{?E7{IA!jC1*85}Z8R<{x*XU%l#S1Lrb zyoe?eXPh~TifCNha&Itbq0NmS+7{xVG09J2s?5FQDP*C0A@Y@lmuck9W;=K5XH|tO z4+)9VqLjj9=&*;p04yRnv#)_H972U+7oO41^W;LN^YK1j(1RzX-ZO>V=C)w6rBA4u z68ZM3AJ{{-!!;W=Eez?Wbr)NEG%@zXz|l*&wJtp`y%^ zFq-Ph!AyTJZIqTc+=&`;%``vIIAw6UVwHcWI;-4?!l?IG`9j0x5gr{B-Nug#PRpXp zT&)V&@D((LJCQK~V_LV~Ua|OL(0?)2-9C}Ff(F7A=k3&W&HO<;WU3}b?5Nr$*iTpo zST(__TvA(QpP6F7f7Wn3g4Gh`u5ZvVoZ)n{m_yJBrfEwlj}989gr8f0cY_<#+-(V0 zj!pn$>~!NAGxg}j*!4Me9$+DBO%JxG@&z_uBG;P6#?g6+*gOa#Se}}O$Gz_pPWYC3 z>(7j6K5@=9jO3^AO*daUG?@~lzC3!5o5lvwKMNwIZYCs*cyUz~nP_#^u%!g6@!K+*68s9rm(=TJ#)}5;vXWr_|w( zrAtns!dtWOGjEu}?tqFt-R9fq-?u^KxBd{TKJ2u3`8ZTk9CO4SSS+^j^CB_`;1+5a zMVb~CzkBO-RN33rq}nTJ+qa|ZpZnfQLeva`)nAF}Kv~x&hpQ4u34^fMT)i`g zun0tbyJtVu1lJ=7Wa!kOK2(_%oaIsx6U5cmD7iTGgtdwnUBC>0m3KDkue6vQ9#%KF zH+-=wPa)@4^h_U~_JeiKp*QB6 zCZ=5wcfjT&W2Tnlc&mwwI_}>ItB+lvppB#Nls4`+o$eIqD=az!U$Bo7cGe0Jr z;+`E)09{<2cTB^MP2B0s?Lun6mo{sKAyL0-( zjsqc#f+@gpa^m)hngYeXU4nVrbNbUv0A<7% z{kSE9y*2DQ37)0nzA5DjVs&#G;*HL(4lCbwig49cbZTNg+;Tks8Bh;2`yJY381X+*{`u$o5onlq`nM=-X6n+1{3|D;xj zz`_dt<*V# zssY4?G0^^$W~xg_9Bk8u#iK5-j?nQPU-D4W#61ST6b0Hz;TR|MfjkO;R$$oqX&Pp}4P5?}KgCe33lPvq zR>28k-O;q6b=%Xd12ecT_+|)9gwftnib7iS3#ykX(R>rE!}$uq6w(fLRT_)TF`nnf zUa|sbguF5KoT&Lr_nf%-S?;(%a{-XB1!8%$1IeqA7xqAsvcLv(lu;wYiT33cBOw*Q zn5AMmVE5_+zfMOQ{GR^)YnE0UbGj<=oq9KZr{4c|amC`-H{$Kre-~G>6=eS@uJ{z1 z1K+)0>?R|1Yw%$;QW!B35zR^LcPAN^*U+80s=r#{e*k|`8Z%jjQ#6N7XP$L&WIw+> zeSqwuy@OnU-~^8s{z_-bEl+~FM5@T1H#PlWC3>w^5-1B8 zlSyOm9fi)n58_pL%ehjBS(d5TQ2cF{P2Gc~kn~?lASjo4+6#v~?iN!AkN4sB+rcO?VM%JePnN?>i{bM$?ERY8O zva&pKJ?Q?(rAUH35gD0qQWVK9w+Uma8r}ITl>fF{BsqCjFnR5Q9MAom-rMWr7w|3$ z1+XRX6P6T9qDbg`644I7mzjCfyD}3dF9!F%uvJnFcTO!_;-o}}Iup|(BsKO2jOdSr z_Eg3!e$({|_4BiKspNxBEebV_t#h$PgzmU}9C?#+?9e2fv%&sUa{BigvC#R>o?%hvL zYVCG_H%GF`y*D+4KsmD4>_?_VH~-Wh8E=f(gf8irNfimFAB~?DmO{c^VhxV>%(bJh zBZz8^a#zGMfu%QzmO!el5|^&?3zH;z#A}7RI@4~$~_HB&CD_GUF%pA{4208-M>6MuN}&HI|W0nc;|(PP=DBz5n}AIr%m|Twj{+ z7?p|LZ2X7^3p3@VSWY5Uo`RyCxLG1=G^&^K4$8E8<8S7yAl)~vCVd`SxO9U@S=qF_ znTt^rV}fE_kxo6LvxfdB4kk#JFijuzG-O+Z%RWDE1aausU*m<)FA?%l=3+R4qzUDl zEi1!BwacI{2z*7_e91dz8yjF?QIS*jQ^GMXG-S>-glv3ec#<8M-oTJYW!m6NwC&P{ zSK>8xOOu;~s|5SlA)zWo^0IY%=v3)D_L@mjeZ^9aR2{bjp1*OML|(C^ZjuS92{@`7 zGV`J_UDyY!cvRhY)|2=iZY$6NgYN<8Cp~#jz>lt<2i|IhRUY;b?JVLwYHGTF|3^*y zpW>Q@$^Gl~H_;pH2N00vzqy)B-R=K50lu%W{=2;9#2{#5@}JAAJN36V&Is1<4zMJw zkpr%M*8U42TT4hC9fjOGDN-ksfl(dE;wspskypIhyb;k@B|2~^sY3?2O$~l~Mx(_% ziH+xKH7ILSxfhv@=QHB@f}0u+k+Z=YAtvAC^rY`SUT^xV+0((_uD`(k8@F~0w&FS% ziF%j9LW1+Km`ZG%47p}>u0&TMAu{kKmbd=a@?Lv`QQlKVnDnDPklioDW>$Ad_$dGq z82a&R@$OZBhOvDnsy9=r$F@LJE&Qsna~DRrUE?qY5v?-XFXZaC0`+?2EDu;f$bKcy}5)9@v*=KPt`BOsL= z=#YO%oRyf7O%=Y$h+owCSJGfDgWE!3Lw5_XaMfgvG~aQ0VF!*OMO!%s*px1q0-~Dw zehYmbXq{%|KtUR5ai+_FKUnZ3)t@%yge%z*qGw9tEMhF0IVajFm*k`{`RvPZ+f(9R z8@Qi*?^IqWpunwC7snK$4F{d z<46_O9Z^2)Aqh8G?UuZhIA7LLjQvF`2z5OPBhQ|d(Ij1rSjvK|Li_WL6%_C8sQX6F zj58Lm+RE`ZEoXX&j!W0DT_&RGK7?vk7YhfU-i2n6je;AzqTTj~hfXXbW2=RTaZ2C~ z)MhS$u{>|NCORw=vAuD;W5AquWH+>Q?711h8XF{r!*Yay8#>sJTWqU=yU zr?^y)Rim=)2#vcmp(9=r0sXl#YmgNy>uJ*;75W_?&;wLYiG3X^U`+C;iZdS?^6o%d z#-zzKqVAEnwYE^WXQ_MY>ml6+&Z2!AZ}r{dfuz}VrhUV$B<7t1g#A8c>YiAFGIvN} z7~iseare{XjN>4=POLfG~-m}s*qSAF! zU&qmfDuCcFAMH)T;FP;cEg)qni-YTmCE^_H=_LYjYu{;g&Dn7y!?lEuKXJchOJ~8EypXjbk1#!Bt3hm9a*C;Ye9%m`dH16Xh%9ueCADeLi3{P zmaf28lHMP0+Q_FmJebpk<=R**Oo!hDsgi{qh4Y~@cCx{aIzFZ+z_y-K z*Pl-BEypua^^BPaJ&!Kra+|R*{4?R$hklK*Fc|$n%P5f?#I80PGOru-4~9wV_0V*2`4zu%R)lVr{&N_krYd`y+i1q8-C2cOS>%P$v>Z+PoD zx!sADTpD0kxb=WH*qMk?#dp1tuIzvDbKST+vE7E_pFkj|Re7h-xlJ=Z?yLDC?$8U^ zzpix?xZS>DwN%_{E;dekAx^kH;hN~9=i>Wq@D7=3mjo5BTyFt$xbE97vzry$`;!=~ z@P~Po37>Nxo}P=c`T6hLZSclc2$@#n9=0a4{Q@|Sqlu?ixfDds3*k?B(>v8q351;s zRePNo zo^_C|G4n^R!v*vF)P!%31Ld#Uv@z_?^#Rx=#Rq7xTX3&$AdjTIF&xeKvha_(ApQh+ z+OT)x9e74}a7@PVnFTG_5!^u}3NIQV&S`#3o7@rT@pcQA0K;Y8cIb{BF^&Y`lWF1S z@`Te2HHZca=q)xf8|O@G=hji+)~Fd{CrRg#vt{T49P$ojxC2-?yQuyg@(vb}Qe??+ z!4PN>iVmoq;c(l))9GYE4ypE~e4cb+w+5&!D#$JNm_!US8)t2)f{hO5I2-A2H2hfT z;wo)O<_!Jra>A;T^BHe_`p^@QaGIGp3)}-pJ)eYis-6n2n zoiN}DYOEY`We>Zu24?q7;Ckf}l4ZVs!>qS*M($BeXR6I}0ro9rhs3AvJgKLkRDRiR zDgLptM$yl+I*n*Sust<#;5(w9F37f=^2go$ljtcJ?*jN831o8-mr`a%`_FvihGLse zv>p~)yj0oQag*s1u6<0GmfwczW#*tws&vJ8LdDdSHQjxbqW!+?D#^^}|LC9r(n#i8 zbppG{{!TNhAb@~u{}+M`Aq!hWM-O2;8+$`X3#b1Ze*9PDp`t7QA1v^iPKD)=zyheU z7vL)lO{8dYN>V!PIWZJ6-WPV8FjvFQ&NZ9gj~uT?#QiV3h_~WcH!|wxS~BiP`exVL zUMHJQ*W2m2y*+<{>H zWy?0EL0{s_y?BZ!{ci{OjkXt_%it#mgLkvT>b2bnzq6Qk18 zH{SE4Kr;#Og9Rum6G*uG6VxGtT;;cv#heJCz#XHy^MdBBjaO%S`R>+z{?-(lai^5a z#tFXXL~HM=`7u7FM~OR{y*cF2W1z%Hs{|C(xU|XlJUmr^wE9=oTJI6Vm>@G#C>1IR zBYahwd$=lUs$u$%Sxphjn;(oMA8?WJ~j$w5w3&uMKS$7ap?nK z)mTUL3U9!}?OtSFdc>fzq064Kf&!{eQ*j|Fu!n zwCzyEfAAf6;VPRe!4+@LE7h&Ug@mR^&;NwV>qyfEtr_1~#lM8l+*-QE-dp*~;crq> zTvR)^D*h^~_%$qjx&Av?C7i(ZSI_Hl7R$-I>x$ps`}bTR`aM@Ei~-zIFiUMGq!mJ~DIus**NT zfGEz@EBnM8lS{>#p-vigmAo-pKV>%nf9VXYu-Hn*aC(H@>oG(5QuUD-k61 zp(Rp<#62IJ3Z=0?)FqE{y=g*72JOZ*mu;95PUfHDrr#Iq@Dvg0jE#HFd|1!IWVen+ z9;O1UHA*73fV-?TJvExdq-nL?^#B^a_o)~!%Q_t!7?b5s1W^L(xHmz+%Hjcvh9hR? zm0PL;Doyn1L<90`E@z|rY=X0tgE9NM%GdpnzO`1KIRD4*Zk3D1(gFWl7w#2R2jSEg zkKzsiJU01Zmo2$M)%lrt--=c}S@7rT!f{0bs1L@N>*uXiXdTt(Kwp{sGpR=`n@E(6 z4&|C&M9~hD4y7FnsVN(XMw`^S9m-S-5bYbiv7BlPtj|pW#`nG=7Sh z^}&T2lzYOH$+yS|jmTofy0F4rvyV%X9Dmw02e56E8EX@5s$d_UF04VSpY;_M#2#*g z{U?3=bdI>?`_q zf;XE5t;gzLY$yaW+R8A^ByG1%oUEt4e>|yQS0xblX^7LlGI|;w`T_rn)+28gGG{HK z8u}HZpWjjW@POT{H$tp7T$YM*o`J}Hm7zcP%Z#ItW@7Pj3!Ugk>w zYe|oU@9YGxqJu{*YcnR7YXLOlaiE4CgIu@j=6rwuS_mdl=-O!}h%a~&hX`w4U#2bt zT5YXj&O1kX2R69baw84PaK-_%E&wEF7b&OVccwA!5nc@Q-`;H2u=jyW-M9bYBmYUn z8S!@V1Ha2UZQwva=Kl*m^8d>_HYZ8T4KgByjwnE(Q66cw33m1i8LglfiwaQ_3VP6$ zo>96$w;XYnwPV8~^yMSjhU6{wg(6X5o!p=PO|0i*sFj}qszdAqwuV$NREQYOv0|q5 z%9cS5TC=R)vu6&0T@t&>5Zs-gF*nEBRNVme7Qru!dimXi7GJ)yJ}!dG7V5LFCDIkS z$2W$5V=PZKWg%f`8InLsLZmAtAT1K^fWyN45tzW*;sOC!W+desmj+oiwC6ipQez&U zICWKOE1$QO@sQk+ks=s@x$5^$b66l0&5XQ{u(SEFCs%kdnHx^lsO-n3h{G zE&V((Y)y%dI^NZE_x|Rrl)Wc-cikw>k{ha+HNnquoizjc^B{hh^yA)pg`m<1TlEr`#w9u-3mm3QWNGB`k z)d9|+e~O`FEie+mr1@vGL0pYj>Cb3|_&C1i{b+|cJNfTUjxBeHmhJ!1!uC(BgoJe| zNB-SuOo02pwGM2ZoDFTA|F0AH&%?ht?YA4w68doOiH)^s!x~cK!BD_@LJ&xC)1oBd zc6g9+CMOdUS+TJNF-`oh(7Dl7=YTl+Z^6B4IxEZ;dNf?@35(5n=YW=`4{5&7ZF5p- zl2|J|8vYSGv+X}yI~&(N^0}YS`y)WMfD@w2zB44(y{144p)x=^`LVY;`Hu4KClXqg z#Sr4*{TNctz?v8@9c(UiKHe`=cBdS8qQDcO7b0KM#FOGvSZ0s<-w_N{?wb65Rl5{0 zz7@OZ;I-85vi%n@+X}ZOfw@#K?ScBNiub8RT_LTee_~kgYLYzU$$JyVYSBN$`28w( zeFEq#w&X0!9k%2sWDiaaa{W0e>*`ln=VQz`545NYS4f^Iz|Mb}g#Nsrm9!CyEMX-< z#!KfdAIY)}(9)KNeSlAQC9cravX+X~8WV9=ip-vmc6XMF%q&4)`t_(WoX@r;K?<*y zo*_+PtYs;Yxo~xA$d@fCSs%(LNpg0HdU|;aOI}*7vRH6+8eY7SVZ8o)?zEMC0OHbZ zmgSroM#-C%p0jvGvTibd#AacnWBlv&%5<4aSrTMq1-xoT>#_y6n^%TAgP{9M2#beQ z4tlezr3NZ^W%W#O?E-tF;DeCI#C+j$1syM2bb;k~Rgy$-%H}M}G`%H}e+XGli^^xV zju`4bN>?9O!O;6L$ zGC%eb?FLn?{u?lCR8L#c_mQ%?V4Q^zY<1Tp^|+>dcQa4VYl-0m=~FEMR~@IOGHnUA zkf+c`O$6O4>(k8$9~sXk5w~A1PcpLJO1p2WbW|@t3EN?&vI=inZER{RWfC)fzGGiP z>Y@v{udYSFPEPMIQN-X#>6O z#~xdDQKdQI3{|$s1!jX3sfCjfC2LCi0%ISu z3n(2HwCRkj6vn6yEmZ?K6(rB8M=m!PJ-mgDfm8u=F4)+QJrShpJnF9`&Qq7ys)!6d zbv$NfmudWQNyrV&-Ys_Vjm7&DZQ6OiKCLvUrz&4it`+V?H{}w2cV-j!3g&I-GkImg zBAi|%`fpUTAQ8J=vTZuoRcLqVA-;r)`LWJCa7WS2>dSTmBLsE}BFH`z0lByJ5dsGZ z5xu*$yl_=0?y4WM1vYklMIsg;UHuzX5{* zC?kZ79oWF|k5aO0i>yXmukc{qF zVg=p@PKgjP!H~+FdoY{$K!+4(CvJ+3*%OW1G(KN-XHiGCAf8=i*bLClVDK5HipmBn z5qEqB2QI0!fKAPeUp6C?wc{NYa+H{vUUQ?hPys`4^te`0d=2_G0UJg?BkPIw*crLu zJEDy@X?P~^95rz~av2}QyS7|}M|P=YHQa)ZIcXj2OdH-T!?1`eQ*4G;!E3+zxJg9z zc=Oh|4=R|B8M~N2BJig>t}yeP3?P8!s?F;cPL6bOkO0u(g3JLR}uK z!`qsUn;(AJ56lidP^ZjsFm)&DyVZ-r=Sp!ZH9|Q|+dQb25OKWAvEEi@+HE*4)j@jc zGJ(IZYP1=-c^QpRs!jZI;&j|1wj7Wy%hnLt0y$50;b>=jQhG+PE)HF^ooJ>u=kQwp zV%rb0JT>Fd9A3f#TSnV)Svbr~?YA>T{Z8wIdkkGUgRi30;x5vtJ zp~A=8V`zWgyMR4%YO(6WhcMX?E_AD4?|b3phP?>LSaIc)jVLof+HZ$BS3#Bc__Id5 zcMXqdY9nY%b2j*g!kX7i6dWnE(L~auyk=OSyT^fy)FbPkJPV6D?I`V9JEYRI;zkn; zr}3GPvl|FYX(R3BU%7#!}=S zjMi9`Mgk}VMjPSM_l*dnE@@-;-)BaTpeZ9U=(oyLYYGfP%h$> zjwqG9lK~QqsvMEeSC?_{0Rr3?xd)&zBaSW2((&}XHsPXzyqVzChW5*b;(6= zNZgqLiK0+UP|B&wI2Z!~w2Ry`(Qe}Qb(Oj>f5sQ2-YUze%gkZ?yaT4yS z0>K1%t;rjz%GC4u2O7dUS9Lu!?-Ns{fKcLxUKQ>h4Hnj6Et!07q|_N3L0_RVL*$H1 z!?uliP3#|wdK)6+2_ye*jDuy)jyKK7!t4!p-6kRFdi~a=7A0-GX<}VE)L> zuPvvF_8lOwVNrrl2fYpu8^3%^ zkyPe%gpQ}}n69Qw#o&PNE!t0c32VeJjJ(~(Q;;&8#;Cf`XdR?~%o#(Fj!q%UMm)c`>zch?mZ7Nq z?zlvug-TAu6hTA~O_XAPs1-WgCSPtXa(DCn`}Y;&^TfkJ1TU%8AwRe~*iLc6f8z0) zbCPwU!~Z$cjtION;pgSRW7Oc$7b#kYHq>q3PcJ6K|1PBvd8K#x(~rp&<6z>?vwBxr zSnc?VwdGwNuGeU!u6rAp{vb)iTJ^3XSJy}+mb=UoKEp67*=l5M>b|76Fv3k@;`4-u zJQe4zD0oiF6pQ0a9xi{;t|r6rZk=^b;^5509tn)z?LZi&sgp{-G>jg7zWC0=K00pV zRONjCfuoQNd+pemV_-fY&xn>nEptosX| z2ZMo<1y5F)?k^2_zs>W+5$P+ zu@$RRBbf))@m4DyEqr0ZK5be|MFx}920FH?ZVd7+ys>U@Cc#o29Ug;mb*e{Dk+td7 zCCsE^Tzl%kuwje`2d!ICWq1@B6owWJ50Sew1KR0GHjTgUgz`vOhL4by8(}ewTKk%C zk3h21F)|BU@RzFt*I2ht z!9Sz9jN!^$0zQmM^kV2w}RJ9@c%eR6(cRuRyx+rK^|+zQOO#jLNA@m-{nL9b<2-xgzBb! z^dW@~YOcT(j)5-BS-FoJite3PfUd7{i-%57APf-aocS2zhf?vz9Hj0EyQzEu<~IwM z3`*AhOy+T!&`}sOS}xUQ=c{DBl-j7W*296KRj?NXhZn9qO-e0jihAe`~BSP8oZB6M-$+rg{KENcVOhE?`R5w5x^!HV}%3IQE5U` z(dP6#QiyvlW~<5)#g(x$)-$oOTNmAIHFw%NM0ek;8%gF>mNbxZ(`Glym39&I>2o(f zW`0;ik$WcUi}yghLd7P(f_Gbcb{4n9FKN;HXwdsG4*$vTTbVH;$=Cyp449|zYp&az zd|*>O=}vQvJ<=y)s37C*9VFSli*)a>Q+3J7wBj`%ewg48e=EC>)^f~v<=aHl$`@z2 z(n;P?#1c3;+9nXsmJv==);ApE_;WesG+CRV<20f^=EC4Yzf1EdA!Fujwy0D6wDn+fNC~0ilu_~`A zP92M%!KuFG+n26seL6erQPS58_$;Cs>;O(aKUShrk{T~?Y5D>U+qfb3=vPEHK&%en03grsFcnz(XzpEaGYjK-@ z=T0(WS~$j+`4x;FRUdKD97p4dqizMQE>)w-Vo^(RBYTKTegr9D$iu}O!G0;sqbD3T zXla@t?wKq%tAvwI8G9Kpgh`ajeTo&MTT9lQ^k6Vz<{-?L81_yT^ zXJKdyYv|VrneX5#E{#~YzT!NQukl&_9a61*_|EBMI6?t4!(H@)FY0eQ@;kyp`1n9M z@-0$)a)+Ry)_9}fAumZ{sk0wD8l?CX4w*ym5?#4NZZSgMsLiplFUPERQ(Qx>Md7u3 zX-)Q%M83*;`?+|T2U*Db>PgPw>be2Xla8sZ+DD^eq4rLC%ZPI~x+b5Dnm2}7u}ohh zk_XTY+tSPO36K1LHsqJ(TTsspB=&;H@m9rurl-Pf*;Yz@RpazAV>=*M>Ij!`{QZz%A^#6;Zxt42w{_bl1cF0whv067Ymma--Jx*TpuyeU-6gm~ zaCdiicL@Y`=e+d~Nsd%g&;;@6%50|E*-$yg_j^)q5PKRW&_ zY$ycI(3Go$P|MqxeL@$9N0g_MhjAqk-wUwA5|`4-Nx;;UG?y|z7DU!`*JEk3)JQ^} zVRSgR-JBTnV6g?cfHuxf`hC7Vko~}MDnY)rk3}8_Xh%qSHlru2Fr7Q#RLR>FBEwbD z3>NRd;SjO!o;YYLMOm7>)$wQiI5iFmE^O8{;~1(I>GPHaSH9Vi+Mpi63$4;jjYF)H zpf$n$77?^XyK@ozMRr8sw?TcR12|K?QYSyqj~?Tz+JXhYkYmi1g&I_h!JEIe499*` z&d(csXyPVlwLz=cNzuD#;-=8evG*newMn8}?95Wszjy-T29YhYGLlR=^rQre0<6s? zEGsc5j|e$^fK-uqI*>l$1}=g137AnQxz(}aS0`U4o%-W+axwEQ8M-|@>2T@}&`uzp z!TU_{b315{pps`vvJdVUE@MC{85{%?KvT z(weG>wmtwYt`yn?UBYH=;Nt19I8~Vl0x8A}$MJSV|H84vLUb~)ZI(G{A*sA`(;hcGQEEdm8E|teL1cmN#CCW@|#@O~cQWKS{j0CF3$I@$YhbE|)7-9)|=|yO9g`rN#0oz}q zi!1!^DIFU{ad zp(NBSD$JMOdv;hwet?kUoS1Gl~~vWw+b+wyB@#zX2juz|nFq z92?l*i#s6K0W17039ic2{=9Bz7;uc{Q@#XFBr{y9W2 zzr`?odV>AlG^T!)Jkh>U9Bo4V2@dly?8tdXz1yUD>kSQa4aF`*O_VH^mXFTS4Ma~I zvBxxs`(}7#?lV*UdKy2_@)|4-c-CCelD`O1*3+<>;|SNp26QT$(mes_YMGu$FVCWX>z2Z?clts#iOg9VO!Q z&GMwlaYis-bH(=BPrRZ|iQ~S&1TPlTgaTETsGpY{zmpNVSTep^H+PtZK%jw!KpZ;M zMv1IZwbY4;Adln||D-DB3#XDj`vfsC{M@fdU6G~j1v#)2NA}9#&vsctIL3pHF0Z(T zHCx4es`3aEJzrc?MQe4GoyY;pcLQxHM^ZL#f!vxheft`i$pWNtPkJGYH-$;PC6k(Nur{C#b z7Cet2CxvKS$z1Fcn2o7a)9^Ej$~4A73dK1GjUj6HaJ{(t`OIyTGnB9zk4z`2gc_rU zUWmxR!zjTwO)M|XSs%koE5%366$kdF_VrNl1WgeP@g!*(9(RxTegTBd;Ux^@;~U4m zPjam>|8DzY=uh}Y^&9aqPF;q)yC9H)& zoiqP^6~?5>>5w$V{3mrBq2;H1vZdkCP8P#H^s!8xe!1?QFc-yY`uI3S$EldaiLAXs zyoYmj`y?z0KI@|$tvS97$Ob*N@*6s*4U>C=EC(Bj*aUSnY=RaQC>`8i&35ojnk&*x zv}wFRJ2nmpxjc1{s9vEf>cEI;X9{L?CM74l`HIhe^C-ZX)`&t)D=pC=Rd=><5#gRf z8>O+$ROfe1sBbszN!qi7i^X+)evMl4XIj3om07qWFRp%3j~N4D&FZe|$nWJS)9IvA z_r`j`IkjT9Q$gK&&_O0yr@Y9|x(8zj`AZJoXas(pYc>SuwsfO+oK18Az-y2B?G4=} zJx+w%X@nd|XhSYp$lU$`*&f83h9BFKMb{p9(d9Ti9}9Wmsk(p1=}x(L*yBa>r}HA} zN)O@&QE-h`V-_!E7h+_DxnPp?$T|j^(^Vu~#{1tZxhtR|fD~HcO?b9plP&|A^xP0f z_6DIa3HvzldwWb*(w_(()+plFWgxi;Oqn>gr8D+zT+1Rq%{~g4Uy6K}5y9--LUMD> zjmyPV31UXFw!&0FFi}{KpY5@9oUCzGC6Kt7AiK~;$YVlvssD;aL3zhXi}nODXUXw} z0r=;A#*t{n`0t*FKJ0a2y<;&vccM+EIL zmUOOgj?eIJojQx7ELA=U;t_>P1G`kBN>A zb7uW0!43IYMkFjMK!PQv=zFypGy620lM>!UzLFik^A?2Uj?hYo)DelcWb!Vc6;5pm zQ(K~Gmu4lXyd^+eSVLQ?&7OcS`1BZKO~7_b)E%B9k!s1}aHIiy6qhiN0my~SewZ`p zIQ8Q7<#Zmpa!3$sKj)n5h<+j%d-Zf%XF#7|!SS<$_x(Ppu>Aubmhl2$7{AbJk)8>g zW_14u;lTqAF9V-Eb8+aWoyW7r_}VBrAY9{~$L>tk?Zg8YKPNYsx=!y$^!2ibf{Gju zb6b|wM$P@h7)weW#vt3BAaO2VBp2d@H3z@0UpjN6u8aPFHK%abuGN)s@GpDX1Mm?R zg0}%xF##hZoM0_X$EJhPlCxxH(Ven`C*3iRvd&*um9F?2{vxpsO=bY@Ud0@VwD{{DkR&RT|!FRe(8JS^0xKBQGEXQIpiEr~5;$QB9 zR>PEnlZ)uRl|_iAwaZDl*a_kKiETJBK&iugyB1$TqW;dB1gf!miOIv4Q`yT>hnhGv zGTjn%$sK<#eTVoDegDnr$LS4{!@=V@hcy44eD*)N?7xTT{{`XWsg`zVi#S1>Mp8-i z@P-H}+*pP%xwR1}khq$kCf{jJupd9;OwG&T32G|JDaBIaW}W~b`S;|?Tk3(S?= ziAyEA0%9QJTvp6y$Yfsf<7G{6if3Ki-{)mbxn(x%;v$WG#R%!KL9gbIXIz_)9g|$9 zj~TsM?;iWkA4ady7(v?%jF^&+92eI+x@&6PwUgiNeqotBd2$h!^m&83iJ&{KcESlg zB^tk%pz00uCb~gJOWtB26UO&POWg*r%Bfc0E8U!!n;BXC?mHeAwcFP${x7ldO`t||4&(s#&L8_0e3-gYGIG|_iSn>jZbORUs8{L z>A_wq?8a{WK%Kq^$`h#eN`#-xao!pw`X-c4Yl%yge$J*qm`~UljAQo4yBVPpKSNR1 zPC@q-@Xg~XI6);9d?bLl9Wy4$#Y9%sX4V$CYxk2}`C%2&A)2 ztZh5Jk;L(=0I1>Hc@&nQc04w&%;WEs#nu$_%eIy2nEQ_1F&`l=WxlBfH(aDeSc*sU zJOq}c9jR?$&C1P61WpgjHx4@WJ6Qd=Kut+J)ZWTGhCV5okf8kIt`l7 z+$*mo}tUVTO6purt=?!RlIkOb2G!>hsV4UsN>vNvS9I~u z+$7cG$0$o3rHjgu55~M_y`SqnuJVu>w@aW0J+b>~h@wczr+nU^JCO+hYojxssFa>= zP?Y=gFRw&J%%tYH1C#9yz^C)PMSXvEKIl=&lnRz!cFYfa{^&i8TG z|3}?%h9habwd#6VXq`fiHep3S`t06sKGW8fzziidId6#dssJ4SPk9khLN{aXf!`*j zCeruM2A2xGxz-Qmbmh7#af1Tb7PH?$u}BUhgGp6%*{LrZMK5Txjd^=dpA*v`d7AI`R)U{kX6=wbP!L)dn?ey~|Ic<73d z@0C-SBm==YF>v`xK_cZOVW~hAGnbd!qtCmn3lsg>pwP51ZHj62)mob5QSkK!;el*{ z*kUF=mn8FiE(0G$wR>xG`DAX!x^K%7t3q@F4EY}+&xEe0V^HXN$-OwFJk$cuT z9ggxJSvuC|)x8lM@x=OunujQgAh6NR^H4F%n1Hb!%O893-KfD2#`A2sNUjVzUG9JQ zh0J63E%vObpMzyP5uLujjXB$2HOzHdjMBY&`9j~s^DFgH1M^Th{~EBxNML5NwGLXN z$iHbRq>($S1Y{9HNRcNNa-=}w9&tlRnMXqicsJxg265Ve#62rS_iG-?2#*kuQSrBw^b-{dLH?BjoLgduu2E}o$k8|n zinD_srk`1HNPy% zO$PW*ocE3yD!zZ({spLjH~nKbgdr1^r*R5JiWb4d@K4{}F;Yvr0nJnihhIHa!?RBnP|) zqJPaZ0oG@PsD7Dl5FuIdR~HQVYz?#r-{EHiT85UShAd2L!J|}C=Bbickmex*=VXD4 zXbO}l&L|2TDEH{0;$nGUflbRH2;eKsY{1nJGai(6Y1^hR)3PM3i6Q7DCzN?iC?X+d z94P0+d4eR8=%GVb5^{t@(i$IfL6X-NdPGR# z{v+f8ZQ37rCm$QKXH)P6=uxiOOp7ydCh3=|6Q8bKmC<}{KDNTy!Xh}>gx{E+fM%1{ zk|Eosk-i+N0*Nb|Z-WjWl`*g4&ZEp6mFs7D?S;m!>$*gL`TFfYK=@b3?$gDql7n^Z zZ}7mL$$vq|iYd!E{QoLh<#~)ZTL24VDsN-8mKC&M<``s%Jk z)bC6zwr;ndQKwY<8yi*1N>r*>*ym?dS3U>8O42StX_}uKT4G6IX|*xBY*yT7JnB-! zONbv27I<#1xlXh^zCC6*PVyaYT#c2>@k8lXQ9T->xVS3}jz=~Rhd1q5?~}EqrfZR{ zodgExm^4TJvVGhka5KfRVeD;deqYzsj(W61W*fDG9=+HDarAa3m$tPx<9VdC2Aezr zoQae!ccWfXy?!irj9nN#2!b{z&xFVL$IhTr{(P+?GHgs=M)tUdA*AmW8>!ao5$k~2 zG2-3X<=v@q!0)1LekL2`C(jk@5oA0tQiD3!{sK>wY^UB6>rhL4D#94$a0{7!M<$HF zdoa7^DcYm-$~Qrq6?0YJwNwuV%yt>JnY1kJMRmrnX^^7x$HVodWaI4QSqkgbe*v?Z z5aWvNRFt)#Ai+)=G4LMfCmLKrSw^0p3*yrxW%4eHqp*sXpS{=AG>DlCboAgk@glVSNb8KprtWcWw*3Q1bWaRfpS~-Nql01plQk0SnmB7B0 z9gL5#lLPbfa~p~TCn%&J+371&Ax=LVyvr@rA{|eis4rTOtY9m0&`pM0Ls*6(({EWH zA(T_o3XplK3WSw`!dE1UDjpAUG|NxzRU{UaGn0`l=bR{PZoF|I?)!b?lzXdtYguEX z9H@|L_^MoHvZH#zodl-Z=U+Y6gD@kf)uN7^-AP1hG8W7<%@T^2Jf;0 zzn*>j*5$~Ripozp#S8(@h7cEPQaRxoR(F#6b%laH{Ch@TC`r`sXc;ms~|uL&>6xwYzB;vFtyXv zci0Xs88^EQvAF7!^_8GSRAE->UBp$2<6pnhfVWDxHt`hcsRML|{hW?n2jV`1o{c>T zPK`Z3;TiJ=+m1Y8%8t4bZYEt5c&ZINUik%Fj>_AK4j^CoMR1}5yhDhte-Jqih6B9) zvH>rsb>HHz9bf&H^#CulIN;DcZV5(Q@@;C9P{D^P9l1pTIK(*P<0-N8B zWcTNt*ngzg!4FiQh4iUpFDEKuq4_Lp_~kL-;)YBX3cE~Xo4i7$Mf}QB>e)1Y&6N2A z{YJfu%Q|aw_cA|$?XsRe!>GHV=7onKOzHCu|8wYxgVh&L^yvt8j5nEta#saN{wh|4 zh*?X8QfS;{S948YYX=`4LFUA*+0VcSVBZ<9jRB?#R4jFt%*TaI&u90)xF2n0%~MC| zw)~-lKCosi9>ZMMPM0Wj>QyHF-aCMK0L|T}3_RU&y3dPa`Rc1K#J!J&?Bl4F8z7O$ z&{~Kh&$4-c@B*Ey{O9pBd$Bssn>O>)%BE(Ik|Om@MHk)s7J%!?aCZ+@C{1;%Da4Q_ z*BxAX(IPA*nqpPMAc6L$L3krh8y1esA-Z&x#CLk3gBq(1BKDTH=}<9&vt0FP&H7Xt zP6QANRP#D_@WlxnJ*2BV2h8sWFn;_AbK(;Du!EWSaC#)f@2 z=M@#ie14x>e2CQ(5x_(LKzhXI-=Su3*_maZ1^DA@IclbFb%lwHt3sApbsLRczX)&= z(N?1piyYM&K%M?^WCvRD4=WgHc#9QkoOCdn!FOAy^u=vZOl8lA_38~P82iDY99uI= zFP!cyn{RZRWFUdK*&56|i>pM&xAIB~X_YGNw|!L)5(^6ir>S`;m298y`@tI$+JbEE%vh*CbE{gM#8vQ*|cOPT($FGYiPdx^pw zKSp{N&LyqqTh@>X73RgoYJ@|Jts~cut0xi{d=_5R;5)YQNNMMd^@wAJnZH}zZw;<% zP%bXn?Ef-a;)CLTI5$V*BMFM((BObPP4#TREgm6@p?YiBvr1Mz^#)Mq6|o6N?h$_R zY7>g?K}?AfDfuKH1}`7|8K&ZE6(nsgdu{0M#TBB`F0J6v063Bn#|OsTMh-czg$~Hk9{4D%y9sN z+l+awWY{r=vbYU8-HwlVU7$X^(>6ET&jqhx?|T~Mw9tBi*1pk$p8aa2Mg)Sp{wE`S z>Oj~#j}&jEL-Wxp<;X2MD%Wn2Z(c?P11v#+Oi?RCLonK0+u zK@5Yq{~p}lp<|#1Uy}kKe2~LdWB1Nog++@Rp?k&0S6PERgDI`g4t->SW&JwvH!q-NP`k(;ms#HLj z)66Fx>yff^!a2?SVWf8tJ|UY4kV zRm?hfNKwL+gE)$$BM(0@uZ^)bG%U6QA|3dNeKtbTYzNMYcqB3W;p;PyM)@(uctXN0G}{r*$ZrSc0FwH9fFEBllPeL0!~1MX z6EemiQ?|=Z?QUbcdrXE{R#&^l%Pk~k!j@nJ(#;8ED-nh?gYoN|RdtLN)P(@lMFCn9 z9@dUcts$~Wu?Lmd+UR>euyIXp!|ely0izA&P}`Nf?#4`exyVZ8OEf$oe_{Mx7xlr*NSE-9?KPs$Zr1kRLZRi ze*dX>;Js7>C)2TEmsP-Vm-aE)@r}S0 zWRgkE>sE|>WATXNi*gh~BjV7s$x@VTaVO;2UHHhgdV zZ_9pypZ}xp=P##z^P;E00(0u^e~VKcOboz%*ve)GHvjwR!2jgb0Yf}UX8;;D=0|d> z7$}4y-`an3sxfHe>Hv`}r!vgix?DM5wcA3gtgx~k5eAL5QXk8#upzushfc@pymQH` zZozTGBUPG$xKNK`|K*_5aiZh!*I|aoMayH5sjk-z^@m$vDOPyRYB;xzO`nWzJBBSS z`SSSM8p*aQBiR{sxAm)68`fnPkCdGU>_-2e%ZD9@cJThaMih^_3nhmK&O?Xv`hNfV zk>Lqp()#F}mvl!o&xPs3N6;g4TlO+^s@JzVWaby9IwJiC|1s~7*$udDMLH7-1q3Q-5j>$VZJzj6=p@bhpb$ zr(7ze#K}uemvg+uft9;{gq(C!?3snwRVvG+uECraH+MHQGV1uDPR~>%D_0;*ZqP_o zQ-2clM-B#k@GeBaB;E$-C#eW4GOsTlJ!tL}<4&SUf@(EG$_9aEN=@Cv4$G$UvoK*H z1()J%93rbwQ;6j!YRS!h4_Sk_d^n|1x3RPYcYLT{JFfad852c%swmG zO6m+urNpr^;)x8dMOg2QsWeUVh&)F8K@?5s=M#!=B5hsdhljyG-g)_KjFZD7$htY= zIAFL~;n)pYUCpS+(U&-3UVLtnti_=gMR^ucr+Z>r4McR$ zzNo9YnSHbs7894zgeBRogHQyrrOzWdn8Y&RrJQ{?&6>mBD@ej_z)a?b$X91PO)^P6u8PIuYqEU@NjVr^o>~m&Gdf67 zU&BkJezOpPAr3&-?eagA$_|J2mr_kF#W<%mgGYVwo7WfXZIp!Zd$+?tZwvG%DLhv8 zxXnaTNkN9PCJYZXt*#we>_R7!ky0lG?H?u$R*lcm{Z7A5yU?6ri|;-Vnc7M;y{&0w(nlZ@51~yBoWkr< zwKyVUZ_Xmsv`h(S*z4p>QmmW!9N#A^jJ=dsjRC!!;U`*dvR9db&EzM?|$0fNb8?_r>Mu?duRb~h%#G23LEwibrXkH@z*pYsBfgR5bzr4p@uhF z%}dW9jpjfxQ3&qy5}|BPK;zb5HVqZ1pQvLKUU6d|&+GfR0gZ z1hlT5Elc6ttK7@gJ8+l`wbJI~ve17tAVN9*qtyV`+)?GlH?#t{!oa>>Yk>pQ`3{-NR zFq`T%1>1w!^ed01XFjCWGzdF)JSi0d9bsR?J9h6_ri9t%m9IlBkg=_Z;x+_eZC^$6|{ z{mmcl&3<%A?u1pt>AY@y8j#!NfEzYG$sgryunf@>#TB_reD%#E4Czgp(X{AD;{1SI zc1@92wA5RTbXxh1U2^G1BxH|1JyMF0e*azO39|I1VC-hL;I(~oRQEX}*JrtJZd}@P zpJCU6VN;~GZYv~ctnbgrlrkk!S}_-6*)im;T#D+x>&r40$na57GmdwieuJB|Pe4&& zaAX-siX7L+SHOw5oY35^smIb}8D)=SsoNTaiQLhs%42N1fQ{inyZKZPX*0J6ITYWp z?qAuQ6=U;tMP##SN=~pmQuB`YSb}x>V{PRt^UGvYWfCT-f7VhLr@E*C>L0=)@GSby zUAo+_!aFVLKew+tFccsr^U$^)FK~mJD+jS`qp?z2Xof|+l=j;)Mirs-DJU=ufni7u zOr!Wv6wox%lUqHWP;ng_3i=K?J$QhUG4Lk@G2dZdI4lz;7Cm3%x_{1TXLOT4!;!Vk zDd^Yg83+yciP%ueaMH3|**hrP?>4a%tWDsK#E@s{Lb9v4QM45jDrXN>J$)Qe&A}Ws z53VfYYYci=;?|ZqU4w}vTQyfZr#K#Gbw|EGE0kW6!}0t9xc{<$ZZNWzpP)u5zKgIw z;Fy4DiqaDfvrAsTZ34W4%j*eD!47Xu2? z&u4>f-7HeSt7=bVQwOleGP$+TmK-t|sa@ANkx$}Y6`SVunyaOomrX5<8@Ca{7j_Zh zr0w_Z9HXF-e1o&XKwJ{V?VswyFryqv9Dn%GPba!;ajx8rvX{9oVxQ%e7AEuA5kd@Q z4caxQZH1z&O#<+U-rhIK7is^|9>MXGGOwTS1!+nXQGCn0`PgQ3tu9?oV*6PD{ZB%& z!&pk@Un_6CzJVh3z5}1#Tp;*D02Elv-ac&Vz)lF)90b>ysMTPdPCj3d-RHqdroBNVVxl!^5lyPWi^oWh(+zuBRVvZ&j;iBs_?7A`x} zh~V;ko;yN40|q;GKAqo$--N1X2%dY3SNYpve2e-XT**R$mAUb{5YeXlKN=1Hl4%vA z<{b%`OdtPCWGelCH>h6&*f3WGFLRE%368>dILaTe~$LR}l!Y-+nk zYOjNAO5W00d$-|W81!fkPQz&SH*s~SOE%aXozt;di}aFS@2&MnZH+E{v>*I4#@}`3 z4MJvm`C4~r`aldK0zH6{G-er@d3F3OcpV(XCvB0Eb5d>R+dN$}6JFaBUaM0hZKJJu z#+hWtO_6R_JcNIoS z&Py+!rHm*O+E^;8IJ=97dkO`N?`r1e%aW$PR!&EiDVJ4$4-l|dMuvAwb0)=?N0nsC z`|RgVCz}6h4sBs)TcD1x3MGYAO9Is;Og{czQ_(!SjJRimY)iAKzhO=VLS;icB>)r2 ztOm#$6GC&{l@?$@MZMlEdR|s)fo3;%Ost|WREMdiHB;j z8)hc)k(}t44y=vRZms|j_Sl^y$EB>B)v{zj!VwqsSkfY3p~9*3^F0Lu+px{C8U>D- zjgAqkl^pG~YKL#@uxyRYR@`l!aF=TqCP(eCh^XHd-f&toDrr82M|oNk4Nt;{NF?Zm zVgv_v18!F{33~Es1DJ<*WBSt=go;R~O7gr&V&~qw_9f!#s;eo8i=c^|G^S6$9#g1- z>NE=lp5cVZ4SZosXTCC;>$Eb3<7!!%qI~KNM&X=JCiyrEY*ls4!8ik?euKurY1!XO z+z3V}&Z6^3wTtFv+b~}HF}lTrsOX~-d<<7-M=o;LdjuUAXca1OsBRx79vQm$USo&G+ zY_H<0oVzBeTc|{vsKJBcn#+ozms}bi<5Ns zyQkKG>y-}9W-mNyC!=t;2F|xGctBprFnGDY@00f!0prLMtoO*7)YzN!z~nVN$Y=QM zr|~Q9FN24su{YU)OBB$93F!Unb~NA(lKWSYyHIP|UEh6zsh0NUap)9dp~uk_8IO>{ zeEf-IiB#yXGkla-A~?i8Q%hoxgGs;}gv@~6H1)PE;BBJl?bCaLJz|)rZ&#uw(q$XtD$4Fj7y^(s*>n3S?o!Hrx8UTE^Sx z3^?{j32K4-uEtga=_bUXsJ`O4UJB*?82Oet0gsRJ3t6+&jk?Q!^{Ex(;wUCx1AnIY zk*Fl%J_79|BT_=Qliz~K@rFn$TZwy$kmPl^SZ8CUqF;d|md-RxNh+f|e21`n+1tzd zq;(3$uf&@l4^2r|2|zzyTBbpsy9j*u@|ti+;_;zd@FWVgue${uSf7Rmr&uxDiYv7@ z^y;JR#X5^krC0E>K~esvr+0yzkXVKO$p{5~Ugvde81{@H$gNnF@mJ z!gin3zf9C<`&oSMIwr8C3Bbk1Y3&yl#Skw(Z8!RS&W@;qF^^HEfaaLQEW-MgH!gJ{ z2cp!W(zPO{Qrjk?`19tyL{8ASBDfdR_`Mo}n&s%DivF46>=Y<-)a?5NfUb0#_7Mh@ zW&~FtmhKwyqq6<%8?~r(_Zh6}6FME;T|m2121B`1-_JwK5HA;5Jka%kCVyQ)Lagdy zDS?y4&8>h^-U`D3Y=O(&$TYD{NohTbUbdp5d5;rLW4mGVp`7_tyH05(H_@Z8m@=Cp zjSJOuL17IG&fIgseb(*Uyb$u!z?22shR>Iwy%hXL_oLB^jCf1BrW+#WiY#7&m&f{0 zLX_U2^f5fYXVFg0$eCfXFSM@>d)mp$x84A@0tz-^L=&Pff7*l?dt_2V$-x3O;$H%l zqzb0ICVJCT;MvJLH%7XMzjZ{ZEvR{)H9w>?Zc=X!-hL^gK1s8Q`k^<_qCK>vbb+a& z*Zmeb3RSblR{*h?37I(PQWT4Q~&-UpEQ&yqPjTj|qBw6fUqI#V1O@JyxYY}zEyDTXJ z%Afc7H0ZSt&O{DIY6Fdb(mhcN<+~-8AB`5Zy&6ySc|y|%qOoy-|KWCaHGr?8EWitK znR=wZQ<}2L#pIK_CDSZPcPe)q`;BJP#uXkbh-{?~AzjFn>Ww`6GtQg&H+{|~0IX2o zp84$|;{hq{a~j9IP-=yB!C#6~YD?*XvUNj+NShgwVvI$PR{5EzER*QEX)lul6xnAglA zf&D2$`(J!eO?_owiH@uHi^}d`qyS zQ&Iw&Q^EpLLeCA65Ncp}`W*g3DtdX%zaf>NS;VTha)*BnB)Y>8c0d7!RN}CIL#o^~ z_n)~Nx`s#+De}zB0M4j8I7j8Mi?<;P8leX`D-^^fNtUqOP+IVafEfGApS?ONk@D11 zKfa;Vj_6n3LspD*goEGFB-KQ+K5`8rfA#4Gb#rL8D=ymW4)(!wsT<@gpBU5oha$W8 zS#`S)CSK~|Jl~gLrF8j#6Qpv6f(`fIA;QD1z6`v4cm5Zovb+hfsY^Hh1F4Dw=ly}r z=@m<2|Ath`So+6!j=zu^{TEV6mV8?q7}H-~q<_K0gY{`qr`V4o=Q(5 z?)>58W1X45g~R93+wg4>uKp?WPe?(`aG&DoAjd!jRw=aSBswRz*?nOMs)U4?BR9(C zF4>Z?15tu(t&lv^=Glz=<*#{$s7c219v|X%zvA~t8@(CaMhuA*bNp)AsapL1^zz&! znx|ucA$0=`sfPcBm|q&)ocw=wr&?D2IkF&#UXmP0joKff<~ei<0ws(rj4a5Y14P@6 z{{g5B)=Ko&h1LQoIrV-zIf)AljV?7#H4QI@kK-#@w;Q9z09aq|)P&3QmkF-NjrT0a ziNOT3TQ^Jg(W>I}*TYp<{W#YI{s|8)Vb%BY|!^+9gdl z0l6E;P@%NnNk9q%CMh3hPgLqQ^RAWVNU!!^!Kthvw&V_&AA0XwQAkCuc7xeT?m#aNf zLTx3(zcFNRNj5-2s=Jba6(hr^C&vTmI_+X}$kj;5vGm#7@Q;t1lJWL#qP6cxz)r$N z$d(2b^=T@T79NK>BRWImDd+<$v&2&+d#n@;&F2wcvcFJ89i#Y$QLKkRCKjNEm37Au zM$aC7y$99<$(9Pl%Af{h)f5A)Q7y`3rKl^J*`$~7AbG><13(OzC3pY3I#r|iN1bXm z)oD3cohtml>XiOp>U2hrnD*c5l>KnGi6*zwf&@Q;#z05z^SldN0tj&)5}fhp z5m19O{sbg&#$S+Iv_#K0Qs5^;rjDnAdOs&78deFBtdoKQ@dnS@I{MDr zxlrx*=frI0BEbQZaN;QQR;sJR3HO~~Hg)1Fq6s}2AwI6wlqoc#xJ)LNSHq@~U_;=c z2;VJ*8QlJ`x@$rH=a>#bAH{iKj=gr#+ANmJYrk5zd@vq;h?UGLxZu(zr%7dOy z48id`=(F2~5%snsgmrJVq}Ml`)CP+X_)3Rn{Gm=$FOySDwmu|$I-P^;)w~A{e^60p zaE-A7)xqALA(7l;)%2{WxK5QlfFS;KJ+&ewf+Js~=g`Cx+z+0SbS>g3A^Hi|gg2N4 z0_}EENJcX}P{p6J5sjBrm1$2DM{huljF)k@{0Wy}N~S=lKLQR%Z;%^jv+FYs<5oJL zGmr(K+t*~`3B8$k&DT!0{Q^MGNWA{kPDUEk96_`r#=AARc3m5hy;VHc*>g5&{EGie z{~>woO>*GpH6F-k^z733l@JA_XYzoI`bM=Ymt(<}OHxr@8Wv!qY9({TLz8#Mat}un zTkuz$rVvOt6JM)kkiy+G{mOc>x-km=w>VuP|3{q8{Uc6A&_slPGM>NcCpSxU?3nNc zHf}IE-CX~=N{jFjMr$_UL{96vgD*7&=!yY*e2iCgh{H(~q($Mev^MDAo34h$1&aNZ zk7!7ZNNr*Q$s{530$qTSN%9d<8v*OH!h?4ivi5f)HvC*nrnX!>?5785PJmLpBHRfa zcJ6O@Q?Z5a8hH7%m@3Kq$33Vsshv(&cqxds|PqG|~i^zpHuZz4t4fY=! zoM6m+EMep%D2D4GSUmsG8ou*H#ACylOJ;5Mgq1>`mekbjv(ux1YL{D)BOm7x+=w0e zsrQEXifqDIKk$fGyVE^%tt(xubEYkhMy9O8;onTp8}-scG=`PeIz{qJN5cBy|iSeWw9L@d5qP-9YIW&e@d&E)p)&Pu7NxG_Zuk<-Vs$JbxlHL4G+dl%gpbI?~fKm z#qPpxfoVt7kN_@<;L4Nt2%f_|i`B~4L%23dxxp+x>w?AJ4>zKxLHZp*KNXCWQz3T( zf5m5<`dO^YQ3T8v26eHo2n#zu zs3BFQH$&gL5^T_)|KOk#`JYm)y~X8U-T5W=ide^Vd13o&K=Q9@7W8C+ku_rcp3Xvgel+gXMWoxBqW2j?umR4u}4B^l3Q>LMnyqPVI>Hnt|kpv>UD=2v|w#o zSG#2j)}|_$v&k}#kV6^F&q$gu9A%F9d&4Ie-4s?71x={ud6$u3B(=!L>=6Hpr14I) zK0R+z*5zzqBxSVJN2e=fit>)0u?76qroz8O?z9B{)cx6E8UQibjZFa z#L1XZIONGNI6a$6yEq#uV#%dsbc=ww^Wn28M7$Rz`1z-uS6DKSd=}r~PPB~#L6JAe zI2mUOre)V2avbSVFohmZKvKNOHnaMV-cZJzIe@QqpbaEMWL~=T1mpug5kZTQb9nOz zjhvU{!e|_EMRoli&bqcK#Z&pqq!M5z)u+y^7UedgJ&vdxV5^@*oD#2%l$~dY+Yrik zRhlFW3;2WHorzuwO>Kj$ut%8N%q_?=A#c?#Fl~oP+#bQCD~P8n0)Co}Mmr=lHYB`8 zJOoxetaOP2)b-`IT8FM1Ng>#!27;&G;tVz2KHjXi(va2$?Qz$I7S&aem1D<22uUg7 zd_)ntZf=<{Y~eUJ$+)xrXOE2h?%Bk6J8(PK?H^vRccwHxJq%45nU-5xF z8r=ES%k^77;pIg|uFgonHXI_o8HgK^!B&1i4n@OXIC z^hUw#_ul4B{BdENz|Q@gLHjybqyNWVyCv&(7Dd|`0iJ-y|HOvwWMEz7SDwEBb7aC4s8Xrhb4keaX5Pr`^;8JZOwExcQ9Wu86e!WBO#Su*f{mmxbc(q#J8l^%J z_&ud)$dAfQAt4E_G`A8R4g;lWn0+}8v^ZYWNYlA_`S|k38=hqe0N8zspLi1E@^-B& zFi+FZ{G5Kg-u9UFlIhq9HlwcZzh4$NKUj578UXEp?5=?|AH^`%2t+>&;@U-fow; zJPv~tyA%9g@Is#iu#tBHtO(=a2$SHWh7M4cdCRt#pFOp^(I8~FdYSLXVz&3BX+4-4 zwFK_q=*}B!xzr|Gh21G^Ar+p`Aac{kDvmGE&7%A0DGQGqv&8YztczG3*CxzdL?M32yf*zr?UON*(D8f6d)S5E8wyrFVQZB^?Zc>WV12`i%&O5 zG>tYGX{9K2DluBiW;7$dr-N&4tkgGucBec+rm39`z@J!&ijC*bWT7ZNuZ!b3+OYRU zgs;>2+A6?kG&9aUNZ~2{mDAurI+rb|4jHbhifV;(yOk!AbF2Za>A$_0oT&Nzt zc-Y>gatFAqJyqj-`-`C)aSu=F>?t<5;(EbF<~nLgTI$|&7K-B|y8>hWBR~hz@E{mcPA>9`TIL>)eIa#s7n=AuVI`G^ensXXAN&S+!(Pj}O zi#<%3Dg1nFKm5>fL(ht!QIIt4?&m)T8 zapz@h_-(1FMk57jFUcZSr8V+#hqft7b^9sIYL}~aaCm8}Yk3t~Z$K30oQelrq8M0& z3fH{4^4BDWaJq`u09}JE0+X%E7EolET8&J<Qz*jD~i%J z;RaT-o33KX(+Atqr!PdMPuSVw0do5z{JtiR!qWYwPug%^#7u48EC6ma|Wla z)?eY^;r^ssYqfs%;4(TgZQA1heYFvPQ9LPx&5bg?>02Pd?Z5pq^jy1q?Dp6Hg2Wp^3ooE5+|^-}O$6>nx#)&=)%cj7n`8y@g zQFbr!AdzjA^8P8ouz~;ard&tJZiiHAjP0L%2wIYwhcMTm>DzNB`pG*{kb%Vl_eQA} zNyj+`(r@ja+Q=4f%wBfI2D`g!Q39z8=GsGc!F`j8J?Gt5w5Aio|3%w71=kj~Yn!oc z+t!M0+g!1o72D2=ZQHhObH$mltfpT!&yf$qHk&W1sU!_)Ywmlzgxd%W7Sy8Xd$gn3d$f`N{H0KRJ@tiJApZrwZ}jsX1OR_MN5?oVml%ztkh(?pgB-2 zDc^zl?f1T`*N$+#))ogI&5^NkgzC81Mu-`yvB68 zmlvxr{(4Uf@5sQkM9~7u2kKFk=9y!no@_rr`(r4@%oSZip|+NAvPOop16KMh*{*H8 z5M@kij}(HnK`Fbht~GbCiw?=F^T8$W#C~}61GE>cny;|hoCa(iuyQu`c~TrK2;B8a zMNNN`mWxdB%{dltssY5dUUjLx{8y*hfMKPf`2EdKz%$;CjzfjyBlN{FjXJLEE_+Qi z=>}7=iD=abGs(ps2HWkyr|SfGxpQyex4Mh)FzT8k1*y{fwjabV4Lj~t9(vMCs3a9G*oCqoGX*U*0|`> zU2WqHQ&Pt-zU7N$!=3zr)igqk-YUu;Z!IuHCF$&~*u@rT+1JwIrR?mb=H5oud0Un!d(Cq&`r>eA zb)t7Ax8bYMF>SPjT0&Q1WJNmhZ76K^Wyy-tF%@yGYu4g4w~q>ITL@ zK!0E^`clI=bJO9XNww?miD%}cOHYv=e$_IbUfM}WXJkZ&BU@D0yk;}euW5&HihB?m z-fok?fK=-Wx`x@x$i>@O5a%uUfem_rsI7|Us^9v22o85*Dc*>n3k-PaVhnCM!*}AU zeqcS9zd*8cxpGW=o(CU`#w`n=trfX=+13bMg+Y{6as%?Fb?|? zc|Go)k|PcFR+Z00s2_2u|72{|{-!Kzi~Lvdg73?C^hI@@zp_xAgru^2 z=GIRpuNv@roxaTV|SC9v-E zlk6i~=u#M5ScRHt2_c z6e@2PAZ9Ow$)5%(O@y%+#M8ux#v*pD&?Yu4k8W;+3Tmj#PcB+ee@}R%R~VT^GI)+& zw{!@)sl;!p(PXhb7OUiJ_XU_HS~B)-bdWrsM_8XL9q9>_wpZSSHvUIq<=BhH@*jy+ zLRSa*>cE#B*Y)xY`EA{QNvx>+D)g>~49|uyKTLX5d_nIgy$P@O?^$iC`^}XGxvz!k zbk>~xeXL&|z%x&X7jawj}VCSA{e*Flk;ulYZZq za8-?KtPBYWNoa|P(^bkkCXZ|5Zcx)75>KC?U}gq2mW{1;gfWDsIM$;+t1KF>@Oiu5 z_TozDnk#UtbNT}3PgH6`>(aYA(fA0+3Zhgs?x8>YfpgoV&yo_wxjXJkeOIR4c90dfmdMMW zq27@DwXs$&F5q`4KY=~`1 z+dB2x8auD4qHQ~N{0Rg=L|m~Ay8qpSXI*0zbxQ%TkwCMSK?MLrIVM_}3vfw0ZcXRl zmF!Fb@}~3IG=gp$Vy(#sMIqRb+s6ffmL!OVKgipE$k+ek;1)TE*bua04x23$-w&A~ zZ;HSfM&`@0&cO~nG1X5ajTXWIn2kEeb_pm0DI+earFpZ1Y$>8_LjhsceYcnSG9?o?>9aJs0#fEGQMx8Z9QMR3EJ9bmX5=u6+bH(Nnmt00( zImBIkbNVKEjL4glLr*Y{T_gVblr0?LmvRs<81>z9HrI4Pmvl4LNSp0+EzTZmFOJ8@ zCrVZc=^SnT*&?t|aO2S);Bxm+9nD%d4(q?dw)t2>t}XSW9jl#Dx}t19Y7ln2vv>(y zK9jLkjcu9Z!dCH36o#4!oDhinWTx{!9=fgtx2jg^g!FrT#zw`$hT%*pK|rMH8H!`2`8-t)9K zYa{FZ^i}uE{DA^U8gcByZ+?cSyyP$$hjPf&y+$?B&a5z6GTqaF;7XcXt{%Ugpy{>_ zlvUMz?VR43phO40*||C}4yH?Sf|j~9Mkbd;NRERjxb1xN{cJbaRz-9DK zKOj4LXBe;@y)y{V!0Jw&f5+M$0enXLCeK4+`TU&^!}3X+m*90r<4a!CkHAMExc?2| zdKanJMWPd;e{pOW7d%?Y4C&=@1r2R*p9ig=zE5qoM!9DRJ*yfYKkIpemvgcQ-r^x3 z^U96*Z+A}&`rqMuE19=W9(yrP28vE04z-*@m7N|LQ3`alb{VB|NdnXAf6DZMpOjz| z#-5^mny^hvW%L&M*Jck{B6BV#>mnl(Q*G4S7AXG(b^JT3GB5T98;w6ik`Z0LZBjcqN21g*Y3P$r zprA}u`VibF1Nkh9&nH7%DB6>{wxyC}GJ`*a3d^9TtVFWlU3>&-9YBjlY6+s(6SVmb zaFwCW$|0@#0{+GuZYT`+s zzk~28`%^nYB=4d=kZ_M;d9jh^yk@J7CM;AL+0>*xpnNHh&fWWieZ70avNOzR@_}+K z^^y#S-kWlZ+eJwl0ng$a_3Y7q?Y{ra^5rkU@&;V!mvC$DJq9qq`mBuZ-Mfzd3Yzzu zc)?s@eX|!ANVp}G*6N}K%P)|{I4THK!Ys6tl58j&(hh*$sjDgJf@ZPw5f+%mEuLKJ zHZfI|3O24-og!PN7%+!N#vBHo7k004O&AqTrsEBtEWojs6iykYatg+xx1wTu`?S%( z;3qIuq8%A4KyVCT2E9|nFdpVqE+MV(q3N9z!h5F$A%Af{E}y>$n- zq*&Fo#E@G{YX%Q-x^1d^yq>Cmjd*TypPMoX< z=)CiZ0&(i4|7$&TQo2aCUq?H_R6{GYNpY>#t9Sz?+xlrJ%`rU}1)iu>8K zoY0p1?nh-(tg5WGWa{aW=@jX9gZZoQPiDI44W1UsIt{ly0kk;Ur&6x8|B0Y__e@#gSI;uYEMqm(f!r>%R*G~F{_S-@J1#^Q#6}oq$$|@Dhvfn1xGMz>tc(!1 zw`Z#%xgbkgq$7F-HSI>)tRBW^^Q60DYh%W*gwFWQIEa&F=2Mm8z|AcM1Uy@on}f!+|EP8g+^zS9TlGc(fov;|nKL$cwu$|AfS&XFJ!xON9Q=c9vQW$1Nio!OF5vWIK@ED8zllq zqGOrQ!tcv3CRAF6<%V7FmuO}2J7)($eq~M#>?wC_4ZDnUIJgVq;wM1|CBW`w7$JJv zNH?)^=MiMm^++53@6GnW%#@9uc0zV7$z&y zB~(~Izi=pN;|<2o1=W9DD3~y}rv+H3;6XsZVwadXx-f&z$2wwR(V5(s&MQs{vh@!3 z{U#P6iW)J3D3Ei;!n!cWQsHuKnS_>|38E`;=7jLzT!|g$=8>nS$lUT`E^OT4+aZ74 zZ!2|Texfm(ZwtS3WhP%oiJT}#Q<+36otjaE(-mgTqz0}n_1+Q2YgHlR7esCgQWp@y zzGpj{a!z7oL_Ntwo4;19>QYd80Uq893s)Yq79QbdMu2t%b797oa%6*J2Bl_3EV1aP zsK>*e54az;Y!h_Z6OQi_Ga_X`#yn7Vm^Bf8U`Rr7urd`;35*)C{XVsqP^c+VvM?+$b=d+u8bvEF|{E{JJ%n>_rPavuU- ze?NJb#rR@epyJwD&g&%fcs=|v)6H3)81*9WZN-7XsfaP%t+^H@W~-a7b9aGji%_;22rMM?p6$slL8Qco~RZstnp+yoR~SBs8;pSk2P8;e2WcYd0zBPBU5Xu{YSLwS#TIJi0JzHq8i5I>R#xsS*8dy0vYY$reF6>y z6ov!@r1QUU4p&qVvil#g*P=Mm7ejruCKY_JxqnN(;u$C6EmRUp-YzfE5BU18pnG$J zPd#``(F9A(X;exN;C)+|XVDLsd&zJ@HY^t)`&}mgOtyzbX$T-IH}`Jc==^kjcD290 zy7v2eLG5$>ATh?5A6tt?H7pP}u^A`}PGkwhvhHgGD2#K)L_lO&_GvMnW<_IUVUnH< zpMV43M&{`!=M!U#!&uHcKqs=Olah`!ptwL|4w89Fi6eN-V}XpL-KALQqsGwZ6M0Gq z#~XSCY<8sJp12Be#MJlmL%1g8|ESJj{!4XMGZu8?8%^A#z~OQNB%(uRw}GwJ*rb|L znUbUQ*p;kOon=-gg3`uGPCuSTmw0|>@gmEI~*nmh*ITNE*d zIlGx7Cfyc6&^KCpGg)C3&7GSS+Ox>EMgAZXxoOa!!w;?3TKFD^%@4VlNePX>!u(ru zSwq!1#nkXJ2j}#y8mS{(K@^Y#21l>Jtc7kFWw69xO(Qo`Rv252i=9Dl+LpMhatIJD zvf?$%;HtsqvxRMN2*!-$k{F^LuX93h$EnAD!;p|78l@bM%d0kTX!@m3$&yS;ytgs% zktTpnF3T*-0POuuN8rKPB`7xdrGb7}Xvv(^HKt-jPpE-w#oBIXwQ1~TEaw@UA;vQd z>7qns;)dQC3=FeTCn5x`fz}nkfxWU~W?fZwvap4L`mewuaaRl6{s13Z1HCKinJRsT zb*a_0g{Cd^5@u(VADnPEFA(uIJ}}NvZ-5f!>A*i&0o|KI)hCt_OMPlW7TjQ;7Tn-~ zTg{(b3O|%yK{*-iC$A#+F5UI|$lI0pAp7Vg&L|7ae9`I$LQt`yb0t@*gYCp_h`RKa z(}k3A0x&u48tYHCF!-uRQ3Hx0q7Qbr-Oa$(lD=Xq8;vxGb}+ zyIN0-Cx~%P7{S3`8SaW-P7TZiy~;$aNk1abhq6uX3u~C0xQ^%O4S)Iwd089Bm|9NH zFY#OPDkMp5OTx13-E)^o)L3+CL#db`TT4I{~8-oGZS;WV*p)2K~7n@LdyUExZ8 z9WS-%J9wsC!IjUNp3296`O(-qoi;NKVe~Mdsb=zsZDRdlE0sJ)ZSe8E*EQ9X@?>_r z4z}ya&R@ZR#ic8fjS`^Ikct@V(qR>P9_W8CwBvP=&W9CSHXpqpwbnOOUIG|MbRowV zI$r>FY}cd*`KexHn_k8H1)o?XW8X#fYMoRXH2Iwx*2U&PyG)LF+VtNq*%#MxcLAka zRIDwvpq`j6p<@0mxcsyDyLDR%e`$iD6P`F@Luv`f9;YL(W7Q3M6CgIATZhlT1{JWZ zQY*U2Az#R_al>&kfMkbcm(+ddK`#U^lFM0 ztghxcjak$)wJr<98HidFkz_>1&+}#c^s1X}eQT-&2Xut(4zUSOD`UgY^OW^QF)2+q zSfqIH7DtO0Ot_z^i+PPD?;KdE^O6~`qC$3=^S~yQ?w>l-Pxy+cU{A)m@e0-*U8=#6chQnl%j~i}u8RiPB z6ouC49B+YgKlqTugR>~$9VuI4B_hFWE#Z)HN)KJpYrvq4KBQVt}l;1v73cHs#a}7 z8D<)$QEF5R)g9mapL3r7lqv+CxSvXX<~-B1KtKlnEAN*dU%80=|9Qu3r+aw+AQDFX zIMT<-20@95kVM7u+0x12s2HF`rGTJ=!Jx(Ct$fCL69UGkIAO^*3e;@QYs#u!2dnMG zqF|uIJLwtJ&efZvw43eJI?L3~SH5O4Q`5&MW%-%YZzT2xeNMLBuDPDQuU%(u0I6;} zpua<%I+-yui`y!o79E?>RwW&)Gl&1&?O}?w5pYO47Q%A0?~P0|d@R7j>xAxMY|&Kv zkuRv9I%T@F>lN{cZcr?^#&e6Ep;(4V@`!b)6zP#{lP_$OY?Cf@lWbEie3J0(j@a-5 z25Ps{0_j6uVtC!6_QJN@vEtD7=Y>DicL<%0AqY}u<77rDbzib!6)@`Ws5x>*Se1Ec zZG9OhIAm_K*u3?*dx>iO9QNYwJ`!R5)UmLCk@)h>nTQsADc-DYyuw)358PxZGT|%+ z)l={xab!t;D8a<0D9)AcR>F#DcYA8uTe5Z;`c<}S*=t#Y`RuT7W5R@Vk9Zc66BLS* zuALz^Na;u{{VRYxdq5x^sND-kMh}8FBrO;fjxTc)s_j^Zc@1cxnhZ~xNn!W%FOgWO zo?kuHw?cG3|CqK!C&Of`3N?N^71atfn^re|BtW~~U)XRr=6AJ1% zx4CT`G0)st6~SpO%#Yw%%bigZ*UEt#kM8O0|7-XUF+zdV$>Gg*h0D;!2U$JM2?M=# z*%Kt!5&v8M3XSY#E+kBj@0&L5%(2*jszIfyil5hIMEkp6J#nyJwRqKzCZUR^R>m5n z=*C$!LI_AGL6#nSnIj!X!L7dy$8r@DcaW!V1l81ROuYA7>8^fV&oGqCc^NDtP1A!m z7e8N#*GXg9ijMQq@0>V-tIbmKHylN*P3KhC;7+%S-Z)3sp=giz@SXU^dG>Rd#CO9u zNJ)Rnf`Qx9C}Ur$8R8Qg+n2+OvtqryHkV9TP-4Cm*2GG5@keF5O_1jPw0SF0C$q(W z&af_(8Wz&6+L$oMHcGKg=W3iSY?lo>6=BZDdY!sbd}^EOa>jJ~v)T4E8_vs`8|V*H z?-pXbS%c)0`1J~eW>}t8_Y$AXiO;|3CO$_xTdIw*LnxFi@i(oq2ky2iUH8cCUh{Tl zKrLl0wJ3T9A2g#etb_JAKR0U?Hk^7zV>Dy1fm#p{hBew2VaWDvejh3d9!!#>pykVh zwO^qOYUy#e4bNSNH{~^ZAyuysjffxYIZ&ZQF&sJFB_=_u46H0Pppx0Z&y@^Bh%gPP zt{o3d^8N)20~;@opX@oyd3^}?@Tr~P!=PluB?~uQ(b*_^(A|QcwvNVdY^W97>H)uB zQMhKkkh7nyD`G~AYr=y4bXtv*lpkcn4h|(ROEd}z#hF^I6O0T|{s6DF;3S~cb5|R* zeyI#yy@g<`)z41~<#2?iWcwv{49{>+mhsPN&lcWqkOyaHmOmoXCH#RgenZUDpr}rgFdPm2R3v_%EHF4(fxDFk{_jC3jh={ok75v{9a${v2@pC&jRMa7;s+U@`ex_r(> zk3z9%OUT-w>xCWZBM9od;mu7RurQBI zEN)7+jm#~Au_*4Iz-3)?E?b}K?Oh%v{Bh7LxZc3IVEx=^R=;)iw; zr_9`}h2D_d#U@eK#Mr|aSLG|vE7cc^k#$c_oABa^Xb=6AbA!PAp?s=dUn&j1|-0(*;^#(V~KO;>;^E2mK`Aitns<*pQEy1EfmMgU7{D-VoJ`%Y7S27C5oLNuW z^E_8m8|flgcArTpFGf7so#k|9FS+@?^*4BC5MNQzf|`Lv#t(ELfd97 z<>sgC*0h%9I5<=>a!ms)c*bW8eNJr3VE}Ez(+V;EZx9BB*CB?(_2lW_r9rZ$i73+v z6S^rZ@v&m4I#C-3bj`Ty9saYo(9Jm$mt)UKoGpWU)Q)yxtfuC4hXV8nhUX&8aAAgM z2&0@+EVk<8E@ZZ`3LAL#>u6F)$C(L&E@=tBT^h)Z-CGin^+wt6Y~v-8-3oqN4%>0k zws^J>=hoJ|P1sUx!E+FbX$)w!tL+!!-bV{I3ixX;%n3rO$skUQ*>6#^6j7Be`YdzW z61K>mjmnW(X-7<;npop3Z z3ZY{)t{$~sLD{z~0_E@^2plT~jAsn#&Z0LDU6+3K1~o_2w+yx>?(YktE8=f`s_Z`p zrWbC?3EQXsB%OEr44Wqa+V|_{tJeV_c$pxaD+o;T8Y{@VtpGbn;2dfHQsiAXCNFaC|iv9;GK%<+$IWihEq<#0NS0P~T$iVut z{+UoW${=0QK#Zj_0%T3JQ2Wv#AC4IQLN&AfAaIDpeF*^|S${z|`w<*i_1YtWYD+p- zk#?KG-0^|+75q7nc4NTYQS~~~=UR}4=!aaaG5kx3SSo zsd{Ju`DVI3?*iHFR_Cs=I@3J67&XwZD=~1Y8Kz8~!imGtlt0RT@kG^&kgRj<$ol$Q zb~1%5-Zk%+XTNaICtV3*7aPT)9Yu6R%Bw1$4K=BPC=qohn`y70WYA4Y`}((5RW*(1 z8y(T2R>fwuu>>2$7{kI6*<0H;hC;&MpDawI20W zz$7j!QK{WK)K!QlR|r*SR2j-0E7e6PTe0em_@%Pu$ih9k*x5K56WC6BLWOx% zHQMo*Ny%*lq`EslxI35ZT!2;u$R6knVob%C(9F$f#ui(Ryrh<@JwNgW z^<&8u<7?{T_5S;RM8W_0i?HB)dz6a{1f;3(zazd?R1h(AcKPq_*>VP?H`?;k=xlrA zxLpdcumvL(Ryi@SF!CHwe(G;1dKV>*b#g69*Fm$O{>WXCkW&9OL1g#HyzsyT5Go?0 z!6(sNh56FinsD$X^ZvWQ(IpwS#zkp6 z9Em8dotbIQ{+a3akdJj`JMD2f>-N{Es#L1DjkQ-g{>=&7%OWQp+8@?Y#+Xk+@@%sT ztDXG;g44hGne4Vk-ZP_8clnk=TMeG-W3+XdikQyzcAn=t$BgW&gB58jSL*G);`-V!H2S>#H~o*wY!UXH(IA8jK>U0IiPY zF_>#FEV!xT8G0tu3=L+O>m5;Sp7}A1sq{5}``eV+J10goHeYZ&oZ@3@lf|NJyM{(q z#|wHl+{@`Yc1AeXUc`AA502EoGL!8i2kEwLPSCrQ?bOGmI>lxjRob;|k901ZqBkd} zZJikGTkr5)JJL6GUgf!}txcMn)q7;FuFV>zsUHvJ7+jB-$;~``UM*vdB=iGltJkt~ z*(W%E`BSscD($_ACLR6Z46~+GlB0e&W+ew-9lysw_uwP9 zy?iw9)Rj+7j4&TS%biq8Yph!vM$7%RTHhD14iF47g#Bb_3LmF+z~r7*DfV?U)K2Lf zic@h++s<;ax6jT%+j}a%&46D&(n;X1ie=*{%sh+Gpm$# zqzSm&v#Fm|Qx04_h|=Q^`*%(&g^R7kP6R5>mT};U&P{2YMiPEQP;#88OTMFQkYla- z2u<;8@xbvl6q8$8T9;hbXJ8tZZQ}mN8 zdf~h5%h}4oiR+7=@&}SjLsg!;^NDnCr5G2#TMey(wke@j$_Y&hLI1lrIm0)VGha5o zVNiQslM?KS;4>1go6W z+s?aygql)QR&{uYWZt`RG4_H0?#aoZ&db26<&^rQQfiQL71f3gZ+@RwDlzic`~_xG zGF{GpwG^8JF*p(b`UjhqI$gXLSw( zVC_qR)hml*tn}8zwOC88k~lbROXN7rEfW=q0fM`jNJ_daXe=ue3uOiRh-cK4KMrxW zfeg$l=fhPz43CNRBrz@)7#aX&5*dH6EcO?3o6J^-6KBeid^KQrr)4FdZAIgR>Hu#T z9wULoqZ#2+TL%`Q)VCd^D?!c*Acm8~{LUZ|@2ix6dE01JZgK?Af=L_9EX$gxF#Dig z`<%k75Fw*~7vvEY`}ddp3L`y(InztVK_rXrariPcj<*z)ZgwE*NJIF*cQadkge-C( z>VPluM3n?gncjx!Ks;fVLl!z0z7X|C7AupMQkF@ELk6fAu%|_=A%#J6apPY4U#fF0 z>n|)_!b-CSWasov_Jv>TBHDQi?4Fv%n@}9;xN2WK|E!PRRd{s47w}z5lQ}B8LI{6b;U7Ur%O_4C-l>k0TOMlz$-EH{K1>2nf$sbm8M}jPR&#h;^#@CK0Nf zqV7Z_GvMc%9yuk`wxJZrXtYP>4ENjYGL-XhZ-{ZWXR>R0#7)HAqsqc_ztdbF`Ka&I z-X(eq9+kQiF4irc6i+~UluC!aSSIJ@kEYL5vlBg_+Chk2k+PrMavT{ zwXQB$0>ck^A(6p}wzvFF$6NHM^%Xz)d-{4`e_u8j2Q%Ayh`ZSLc7az2YX=rO0a+~h zd+@sQ1K6+ZPU53}BA02O>0DOxa%lj*vvYaY&e$_AW>)npE3V=LoKWRe=1cmh_7zw6 zVm3J0`4Db!Z0QZ#63PgdP~#QMukz0AgAXwLt@aA}bqgofAex99EnArzKM0G*4~n$~ z>&I6d7{~m`A)Z2S%GWEa>P5DP6IqEZTSLIsx|Dm!$}6Ri0Qi*R{%o9}d`)F#L+$wW zm;+5enG!36?K6lcf}zXU>U;EW1_{3y{E;FnK3&GVC&r~@Oi7o~+!xa&G%Fs1Xtg|- zRr0MmP>uL7+3ze}Y9^g~qSH(WQ@OKixC?&K5{i>3|n&8`3mF z%5WSOgnvTk%n?E@0&%uJ-R7=FT748Om~2m*nB_>mS>T-TXekv zs;~_UXAUCrPO>=k&|B~@TktVE3n$#OU0WO%w3WkgBgn+(&kT2W*m(0+ zvh{QJ;9scKB)W42oh2^2I2D4k<5pP$j7MZgRJe4?cX&>7Y7llL#n#)hHYtD{_Z24+ z=DLAd3@QgTU_3Wr>yM^n3boX@i5`(bTy!oeI=)fbZ3(sff`(TJh=MnRU->NS`t?of z0FfJgT>R8Y1^lDCl3zHr!94anT&B;U{i$)*KWmdT*>R<_;5Mz#hvlE2#L%W(DYFw* zfG8v@?RZ}qZ?pnb|3R}fLr6@p)Of!eu`Q9&-?F-~w_4)~>{B^|h-}QLd^VqQuT*qY z>oj-9Wn5u1G-YXy3ni1QVsTe-02*dnZXC@6axD)+StcEM>%OxCiLSPmjw!u~i5_3j zX@<(AE>ylhtXGmgIMkDG_A*9J^*{2o$3JJ<1VEN8T$2sp#h5j}D(BZ5y5*TxGEGiL zrW_(GBg8ypW?DH*j6qt5wg<>_uNu~^moHy0U(pU^h&FQGXp8)UW8Y|oYvfRzPT-Z! zuwK1#s;I^Xj{{x7;d7}NToa7@0S^RQcNrSZZA-dqPIt_I#A?^;ERhqarl{=R>mSeQ z`^#1%B)T^Cu|Xev$A@J%3mkKdU{MfupRgT5Eh}7r8WXjroR3Bu%^5!p;Si>?{&Ma( z<^;-=5>nVVmcHvs0F3`Kb6D3erCsFOIGV5R_f$rDXw)&$RyH<1*rzgRJ2SVEBpZ>A z7fp#<#>jq}^NX&m(^R(EKRv=io^@}=6Zb7<+ESt+d=I0dmsC@dsjtABamWSX&WV7l zQJ0!)Z%R`~iH`~Pu8S%hzKgGJF%Yq?$GoerX)%y^wm51d@BiEpufDi`=}cumXoI)S zhgAH^Q}0Rs3OulVU}}lWGvS{Ad&*W;rj1Z(%Mz8A2B%gtp)O5BE6y@9%~!Lt;b~%M zG2V==T0I76E+&`W zv>)q5|D(kowJIrgYq=F++ZONOJDY$Jx z@|Z6LWs1OM*?~Q`Qq~wou)5`dUBee304u$J+nV}>jS$0Pwvv-T6QnKWS5D9zfsvt= z))U3_b&+@4kh#~e=nIGy@f`KZpZ5x#_p+feqpj~S?^Hh%oNX8C9?9q)Sx|9dDqAhR zeTL(+9@|4k$uQR-HnQEHTuH3JcW0mbE66*$aIjtyIWwXptn*n~9k?-`dom`Mpzmr2 z-n=T((;YhUYY72%5l=eLTXvFnnpllv&LRAi%X5dRC$q%1NsWNTD|}h{7_u47Y*OSQ zkU(}4I(RZvrF9?{VS?d|3h7<`phHdZgyt+^yE`zUb)p1Z+aJ0)!u)i!3*o(FtvA94 zb}8OTddR;0!Rz4K>f=$ugZoK7$=v9=-+=gQ;)cT+@sF5|@&bKCN}7|jT$4>4-JXj~ ztEde4ADK9*=yj=b>vHL^ zGU$&qrB|8MT-(}lo)OHOS~cTxNoiQ5h(*NRZc2ng+Kds?bUtsx6j`Zu+af;ZWZF6- zBdW^1KAGjFo;F4o{L@8g!C##KZy`XX*$D$vwhU|5xe;foz7QxKZkeS~ba5nvRbfR0 zbEI!He`m4!9MO8=x<=>^R=2QLJ=Ff)(g7TvITHd0uDZCt4hDx{{N)!CMiv)kR23e3(gsc;+R^7mi9`HQt@ ztioK>2VeJ4@4{pOb2mu>Ky5zYi<)~8w*c}}#$+zjhy)**>!Bu1{5nDgATS_v9c2v= zsZZlaoji4}kBpsYp1p13)&pPakz^#!p`9u`SPIZD(Qx?1Bd7u(_JRA;grc!XDXEM$M7Ha(3o@rcx>pJ>Iq+{#@})s&AUU6GY0Q0s zjQ8FKPe*}0wmKvRGqi!Z(v>|Hw(eF{(xSHD0toux;UJFyVp+KK7M<-m;oA+vs@5WKQNK12e#39+TCr#mc{d`h0U$t*g)c5M;SA_ z5g3jWFuVTjc64$1L;bB#iTm}eVOtN7)*yQPS7rHqUH(+Pt(^OI(09d7G-zM!(mRr? z5!!a1+7p7VWUo8<@6nV|$NqMfWs|VNy{N^GLED zMVH;%0#*eiKKFvLT`41ylW@EohLT%2N9b8J{T%X|y>%nzlgOhiO7;3ibrd_c@!~XD z?`;#Kl<^^G3RGt~isMY#Ei;z-EfXU)lq-l;e$Zk%@4mx-m=vhJzqC@(h>i)#@()kT z<8Pd^gPH3gpj(%O251ZaIwT4W#KECZ;E?g6h!0!{a6mhsNeyIaOLCGY1*=$7?lomd zt^IQ#r%wtd8YRHtEwm(;TOEm5_2ZcLlOQ+pDlBF9T(+W_VZxOfGYuBBv%sH@C!LAL zau&sp5ggS*jycOa5Q7(ya8*Igk*Rhkcx?}_F+!v_WTZbr(eC#(@h*0uWCX)!gv74} z`q38PV+O>o2FA(=j;iGy{K-2|p1XyZbHpCe1?Fae#J2~*w}-e^H#%I-JE+b(xXe4i z&pXJjJ0h5KqzAZT0^B(OZ3r$~K-@c!+;$-N*AkP{c?TH2!GwpSuf#iHg`i#8h_?v* z195kbO!quH!;~I~-y`o}_rA!Z3W2eZq6+*}Z#T{bgaU{x1SsntfeyDa1^x!;_b+u+ zd#ghzzIs*197C5MEt<2};c_3>->lrR?5CV-R6Ib^02f|SYVmcc#Ng;g?VND9Nlx0g zeA!atP%ZY^B4a5e{MIy=t%7;=1;=WsQ5`t$n%aO^3EbG&f^wlzyA@SmhjkiO zdO9`ocx*$vPNwP5+r)?8p<1=+h-E$&Gs4-OoTcw^U)L8*Yxkf4^(7VV>JRw2@=Lhn z(Ln)^AEoClVgbiD!&`VsK`nspEvm*S4?qDCaW5K*MGO(8WmMLt8I8igERGgV$E*;I zN=~9fhkP(>#jLc&P!joQl(9xu@;!d~0y^=qb?cVsw=FB5W1iA38@VDUF}|o$cz`*g z_b>{9Ut1d#+gLE?7!%LqYd=>XBs6M12;jV(l^f-HEIP>22BZ;6j|Og>0&3D@%R_1L zg-_QwnJElw*aPt;?yX3S$Sed02Qn0~v~aQLDk1{noBIV6_Z*cS5kZ!r_Y{#*9~*mjbpbpmQZlNMbGANbjU|H0L-qoE!O}e^HuK_sF$(j zR$||fpMqe|UJxT_b96pZj*>8~Xzy?dIG#NexJc`{N0sE+k5|0=U;WfF<6a1>e_ps)3klF>#35 zES(WZG=dh@ch%Y==|;z5z`p0_wygdCG2{F0tGJU2spG)(y2pv;1_+kRfd=xjK?Pbu zfGxs7m7F3iJw=T4v;y~hjo6pgry^yQi~Lg^Sh` zq1vfQmK++kDXDU)d_o;d$lDQ&QKmc6t~N`7eOm{qZ8-ipJr?FD_}M2DlT!Kz~peT8ww<3;J)VXEXquv=t@k5mP5b03z z7Ofl_)!fOQ$G9c8MgOPRKA=h-yJ&Zd&&T*B!x%+)s&)(8P3&8Y3L>b47OO(HB@}UF zGCZ-3hHH?=wighY&6*_6tE%MDwvg7C?4DEBbdoL&*`smFSE|91Af$GBGwZWHUBg=N#13R41^bqodunpsT{tHl?f5 z%(0Zzl&)bT^2GCzGLV`#);cS&??-skj?CG(bCLVPxQF(WP5bCA7x^+Z*ZMjde$lRdu0Z11U(ks0!Dw_grm%$aOO%QFA z!R6CWNN!X_2Yi?-CMQUBCewN9kC&t-LqAj`h}cu90D@W2?{dLk4ZvOv;9dN!d=cW@8ihPxaeyG5|mg3wgz$f}uIPxz3^{Wv2R;B)?wf_4pFY<+$o$^sz(fWZJ zbl4kJgF(E|TjF}k_%^t3J$Yiia2*8{CjAW|`7JK#hh5Z*x6q6If6?|1N}@$emu~H{ zZQHhO+qUgpwr$(CZQHh8yIj?EZvP$K_W@4CS}W!gtdTi$Fdi>K{YkR$iStOc>-;bd8`<)IO3-XldtR8^k{Rqmb$?r1&ycdOt6D0t1uy z$)orhg*lglN%;0sdVf6Ude=?#$)WgKFL`o;K1UOd@5%dFfEyv>AW|UYYY7j0uZe!> zjt!WS^2oKG029zV8A|_s=VPA|?@eT>XNDJNUtNB&svUg5sd2^gMLbRJBe=YBvpA!b0!ox^*BU#$ zo13n47U9@dmPsVDIL5RjvI0gjJ*Fk5VU|Hv^W|NVlx=)q(nwBB%KH?&PS!gIgaY%1 zPBKt0@oo(LAlxddGxU-*_Awyo-E9c=HZTd#N0+jGqN`dH%seg?#~!Q~)w38Wm3X*O zRmLjw&NdE*Vk8&6CG{cCHm*cSxfnrlmQpvmKwEF)?9gOu2{|XNA?WZ(xE5s#j5-?` zw|h}lo2%w3a7C#XZ$qg3wJku{R>|lhb?9YcmwdS6*c7z=zk|X5pzNJE;j4$h0RXlz z{ySI-c{vFuSr=>T|G`qId$|8v2Y8{YMi&93vko41#EK0dz? zdo~TyM6bit{PC9?o|i4BS&o;j#x1T>UhYRqzh?K{XqdHW89m$y1Xm-TcOCidv5=HV z@R!#6s2fpaQCh=x^u7CQ+gJLr#XxTCgA-V`n;{*KHmuwe(^#zBelw2)+?-czp}!a% zsrlZ}+oN+|&^&!}rx>1b`M;;3xj!Mere_EtxF%-|A-HabP_G48a(hzk>}iogwAikN z&~hDjNlpTt|GA^%+1v` zn*2*pQvi8j{4*tSn2fAY?mRFVxKHB<>${b&g!y@NOL!0;ga#VV zEMmwytju4aCS?URMLK;N#URN3ATR_f77@>GRPV0sT+(R+pwn^-CJX)}tP2)TAq9Xs zesF`7%qZ3)*Nl z5y68fP2;@&D(BbyefeVMBH8>B8JQ;f83_?^uq5d%AoYf^puZ%9LI-q5<<2g{H}-& z@@kjoge>FAbOeUY6+tZuvvVXKlQU3NjhhRMWif4)3hL|n6_IsX0m&=*Fpu&i@*~br z)sqClG=)cr4H>KRvp$@TK= zd1un@+W2(+2c{6aLsE#cdqq&C#9mN?M^;?pl@Y3F6712{hzI70^Z|*8qW29@X2eZK zQ^9J)UYYT=2P|m1sdvAroC7a+-Z|g?L2ULZ`FN7A2pzv>u>H}s{y7=M|AcZn>kiD` zkqYsp-a&mP&1l^VMyr1r^X?{k1|at4eqm@s8kzEm|@SU;aWKyi3PbqRR5No z-k+Zx=qxv5he?&=DVKXMtbkIK9lz$AE+4#>jHA?17gvdf1i?$rIqy-p6#;V!4OCZ?akmkfn41X8=TL;%zmu5)=`2OIHjyCMixAP^2m# zNm!v0=bz~cOKG;ZY=9V9MBrCc9wDaSJX>jFaxryb&Gy{9K<55Zax z@GisbxQBU!B%SJH*$Q6*tMLTa)>S+yhA*2ToV?Ug4kj5FsHlvfY#g7D$?}^~K)Yl} z#b9veX?RG#zOre}wwShh>MkX=Cw`(CV_n72PFr6#(tfh0oxTdgT8l%xF41jVex~D% zsey!jn(3({LA5%+Qp4EpN%QVdv(Q$PP)#KSV_wu8K$jAfUf@#B25Z)^f+c<#&1AkE z@H??(4ai<hel1=R&80v|*Rs1;o@7`!c*xxk#ON zkYhf;ehysvNo_OB9vEPU#F`ALKciQbZ+)dcN#zi8n75$j z4f$-KhN9*&P@oGpvUp!Y8?C4IzK}X-WA?r{cBm51eK&IO;`Y8MV5gPJeSct&SS;1W z7$sXM)kPU4Tg=x@>ojk?PCmNNVV7J(>)uMzgw@dJpx`(X{RN70rgEsYiPpV02f0OH zFOFK7D@STkbi9-II8OixA@}~J82G1Tjh#2fF09}XRB(VkJWTPh!E|I%D{q}UESoh- z^VG2u+)A*B#pw4kANYeG1~*650FU@MPnN=uf<{Cb;gESJh-QU=k_nHrjE>ygCbSjC zB?5MX*lFLo7R@6#$}`ZXTMzaU;k`zNJK%kQj|(qWr-wcK#{nBI(tw+Ax88(RfN?h-Wt_}mCB*?nHKM8@+4aJ>Li!jA@V0a(sy#bk)1%gyZFrS}C>~GFwf2&bM{;A7 z8}xCu(VdH`ZL`CX$&}%l$gp-yl*Sw9^6+(O zJgJIW>Nwi$g_|={U(xlJMJ?nT?Emie{l{39+!|BE^IJdJ{F(&cAlC=O9nGA9hyf|*emj#Ea5h+%*gB?4VMuBka0dAxmX zDMI5}BoHWXnP2`~qmD5!Mf4&&Q9Bi4&GYIgt8=2Z|YXaAZm zq#kSbVc!hIiy}&vd$_ZAD)N-`4#lfE`@ILXGtl?_U^$j6kyM` zzm9u6n=axV&YxWyvZLGJd+)BPNW8mfpRrKip-8;P12RCLiQZc9R}3LD0aj#++?42m zg7U?21W?wIItz6rsFI=#3ktm?p}HT2ki7tH!!yb3_!-IYuA&n}rIP%zcAt=yKO!nq zK+yBzl1z>e5~+-jj)H2A5GD$5mX>%q*j4X$B+841U7@8r?Ra12&#bpof4i2iU zF;a~l>2~=hVX4AbW!KIvX^)cr`%P%RsuR>>maGhPCM+^?V^WSK<_X*l8AMw_yn2-6 zK`9B64FZK71ZZt-mBx_a9VmafS*bJU3%mlll0aKkMnxs`qGLNNOJSEZ zHMt6`+3iW4x|nqe+9by1gp$t)mbnyhJps;OpyVaT?< za3`%2H(F@TFl;FU-Q4#SSx#`Qn{VVuHosI4FcocgmM}M4z>Yp78FjC%C}`mfxF zACgs$=Skc-_ene#9=a>o74R)EIA3yZ%Fj$3K?Lo`!{qGPxnaQD`+J4VjO=@mFXH;~K|s5g3c22a z$D&N|9RtmMrQBxFCNCA%9xy>Lp@wQBJfPGig5#4x3(48RMW8xEyQhL?g0|sy75>B= z+IRUB?PEdc9ljO*#GNGk_Dc%mJKz*jnkFHP?h0eOBSY{VsU(ci&(eicF~iz@k>QOO zmEG=BH(A>pOV%9HtJCY#6ZwSkMY~7+?zj0Tr;CkzPxq-M{KxrQn)PFV(h#`tcRbBY zOQG(?vL9Ezm4$KhKyYNjC~33W&~1f4KQ(*cFrC$KyY0BbV$=BCW{+_`AKlHkbJKUS zFZbd_sYRkgrXI4>0WnYJ_vWFmUO#lZj9exJ&Gl7d_7B-XR-v(r)0T&?gGk6FSA!;U zg{W$J>55j;Fyaf?LM01K(#%G1U3*)I9^^R7dR%-~d_In1wM>b$7+bSfq+X9Duw;BZ z;YvVKCrh#6CVj(FO4IwTGRoE11op5i(KdEzrW^{B(wMm>C9D=MW-*g5-CJH&Z9BPF zNoU%-Sz_3Y!KQ88qhpPTw|1YwlXdwV?=WYx>C-mwtCs*(~5?eW^+xFj&+oZXLP9S|IEdn7J#I zw+I&N%3q9rmaA#hmF%EyJe;P8!_liZ6tDUKI?mUwk!6IUofo4zRScECYoE6SCR>+< z&@scw3_%89R0hrd@>hrfXq58Rs;6J5QRZ^?`sjR37OE)m?p0B!bn$^rrgNcV1l;1{ za0=bQJreKHD&#!Y6sTuf0_LU|)jEP0Ho~=EkORt>7kk0xFE*4$Ov@8%2gke@2Rmx} zKIjx?sopVr{(Dq8$woEF{u7cDhcQcb^Ql;mLy(-|yD}86L21t2{yId};rE=DQr7mP=djKeSxPl=1CdsMWln{s`dy~xs>ju)*NtJ=Sx4hTSh$|Gk%*4CoKT&02U}`jTj!Of3x~j z4Q)V9w0vQlM}ow0GPcnGJlx@3GJCw1Ak?ecBV64lI`)4;C}U6vPt(i)aIb^s-2k~q zAYX!TZ-eX4gWUzX7lGc*ivrBP0Z0~E(CL>T^o@D}G{#v0oMQg*+5)2weZ%B0M(8W` z2AGZe>h^mR1<1Ysqg^z22UrmezR<0_~ z%oL>4>~-87;cn<*l!CJo>4>Z=pmKW0`sY0n_ zkuEx$b27=2HoFXa@4k5Nsv$2>QnvZrt-eAnIhlqKPZzv0r}1_>ozAr1dd&Xjz&PFB z`ucnT`9a-T0c6_yWkT?=;-mD5N=wcGm4(g&B<8Z+qfz-WUx6B9(huLVfx!%Q|IzDCvoxZ&V1fhYXl&DB4 zDGF62uRwd#uTcB~NKswTJvUUH|B6ePEP19f2Kn7W%B)^h1Mcg`8juu5ln;$Amz9$g zZWN_QWKf92{!6d;MTEFSb2f6IVj`AZ(j!*%CE&`a{bfoltY{X)keL@(ibWv1hIGht zv#XoIU0nbq8io{2ehsQ1D>#%Kpr0z^tR7)vtyg^}-WQ#|Pj3=6=b4DIoL5^^T(n+- zU_CeY)vL`=wziSnmNu0Iuj1pt0=A@1G#vsKB(q@E`g$GjD7y%;rNVFmU$*KT$!wh&Y+f2SM1XE@7g`EM(~)R}7$) zm-0b7s|rDdpo7xZG8a;tkiYs>YEpYmh_*ia{E;-nvoLYuu}7qKA8UC^S4xQnE@(5qf&mcb?95YyaJ!wY5+%{gy(2n`NOMa-wn6O zm*9-^=%2P`G?EX$`1%9O)RAH{iJfGdDZ!EAwx~>4jM@UJ5i55ceUp3Ixtw`Lh-Jfm zHek$0{14-<@K)olXgA_6szaVGIRrH`+rdFM>lUY(ob|7YPN<@mEN_#y$g7^pYT|aBmE;IFon{R15@lZa!6$seP>P z@9MrCCA4zDX)+~j6YOC(#Gcf9pqO#?7`*jyzVTO#-m$eSoj=Cbqp<$Q&Cz&E zC{`uRvE*%3o&!cnm12NsD*7tYW?0>5(pEEcj$n4e-61@23^Ica%fkNZIFBWVxk~Mi zQq8^wD4%-&sAtwU@LPF@nITm2SkyHwq#|omU9*N#8ozGR8N#d0(bQFj*z(sTn-VeA zgmD+2fZhIPPxNrF>9#bj$|!j(j%}4b##W^O^6Pa zTs`R%F=MQobVE29u$|BN%#6b(?b5^1?gFJMU8dU+HiWpT@)cBT!r6_|B`Qg!IXqru z9`E+gskg3lV#tOqQ;TzF8e#Vg8_KneLetA%`T4PGphQR_)ae|xExgvnd9@23ZT`{F zv$CH=UD}({Ye`p@h+$~~Z(C-NrqYxEmP0VJ9A*xQ9gerdS686Bxdy#OztU~A8&M4b z{G~>eOM=Nd6s$9O$D@aCi!n08a0$lQ1D2zVO1m7pVphncN*2y4hjcc zwry=6B|p8|TsSij)AiV?%rU0&%Hep2)p=gKAH-kuq}~?7OhIkR6REzZ4)7Aa1L7R> zVs*YT^xLgq_0Tu_5G(JR{36tB-zuTO92_si7xvEu3yULBr%_~sx^y>nidXvLjgzU~ zg?Wk&v0USxQ}C#F##%ku26%A`Nn|k>QK-YlBzq{XOz(cw*tz=1_Mv*ta6~vq6?Z_1 zxUN*xL%=?f=Z>1g9?+$O zk~|ut*8I!PwxRi;y#*YVI7}H3lq45dH+twL_)C`e3LuxNEs0l<`@1CJo|h9>IPB2d zfq!$!)AvSyf6+%=@ux96Yj42%{vV=m+c7(5_768=gRu^!$!M!^sx9$~L(K(dnq;|q;TZyX+al+{mZ;N4PeG~C zChk_1fU)9i){qPanrs?E(3rE38*C%+P|Et}sASdZCeyO+{|^^5`{1SJp4Pu>K*}Mc z6XpJN#OPoPXL-*&%{(N1z*-gW$^4J>Yo<*$YlcqBmDgmQo9fC%FRf~0b<+saWep_+ z2ab$K=!oF?3N=D?9kt($Nm=efW7_~-)Bc^$f}kyK>+p@2Fa)H?q7&655&DY?ZW`ZB zhNO)|)`X->a)!PM`psB}96NK4lqH$9bm`CNN;L9yisIJ!zEaVdc$szgHuIu3)5gHB ze}Q_Y&K1EOqBg!5$%2y(SRJ>u7)W&B651#hpvS5on_{z7TGV@|4BD~h(en_An_%ia z4@(=vPDQC2u8RppfLZ_K3Yf1v@{+n!)fxei+ zmE!emPXy#vUvr=na%b=Z89Ei+Yq~QN#mg@cl28oIWDm%$O6-f4(;$8DztEm7#knSER&H!{8$PqxmPN4DA~L=snJ?% zkaF^Y^L*RYT1!GCmv9OJ8}t$NWD`yf^(N`2?Gy9~4W!$O?DN9(cg><`sv1DbjO2mp zyI=u~d5VZ1bf*xM)6+oHwGKl^i3~~t35%E#S2=}cLDadZVjPbxggJCGohHyH1|-+Q zIzyzA^W2H-3@a%1^tb{+D(Ub?kz163w>fi{I4&7V1S&FIfCyr;p1ME5+Yj-cK4a?m zQLr`iz1VDgmw9ApG9sPkp!ZaJ4NmbvicA`6u3ZYpU}#Dg0MFCbg>25jh&L+a@_EC4 zO)!h;_~204O69{a3Iasr&^pHcPVl3da8Qk-)ssaIX*R|Jl`qK_aeWxu;NY6z>{<2` z?Z5dCt|0m|5wSu_;q$_d{e_Cn$tF>agpF=yPFb>wvQ)zP3XQtVn~~CO8$-?r09OM% zh0DmzIy|r~u(lvG`#65!H3GP*+&-@*s(AB7>Iy;@rn_W#wN5Kaf)+y*2z>d>$d6Nb z6gU<6(0Ng^bn&(&s86FMC$Yr4&%xE#XRxua!5x_@SHt&@{pN;ce*gp0t78)t$xOp-QCx;98tN%+JW z-}X^t4O-Y!mLjo7R)!^OxO>hFEV4^*Nulz};GyWty)X=*eMnT6#B;526HTt$vL$NpvxYg9FtP->{el zT(e6DS!qr@Q6}23c&LNN0zY>V`Pz_tLIgoXieGS?l0lV_W|RrBIU;Sa_>&{L#*uI# z6tU1)W`Tzcb4pOxnGFO;h>C};g{5nNmdEtmD3~Y48bnA4uF&jeg{@rKIv`x%*Y7p4 zz;C}b2wffDW>Ac>tel8ZlcuRT?`W#Z&{$CeoxY`LGsrm}xm}F}{f$dBjDdNWF#`tG z4%|AjB%H!7<3J=ap~C3j!&9<5W9|_WZuuFM(KYp)l%Sv#VnFGYK?cQ&G5yTEPib0` zwhB&DgW4`qSTg)LP#QXNl)Rr$P?`DJIaPN!9%UBBo1Y0ZFv~ef{Q4)P4~3?79s2OG z=qC|L5Gp;J0v3sZ03T*jL*6ym!1z2LW59b+!WOc;3s~`Bbs0O@*rtfZN20rcl9{7A zY9pnj)S@Nn+@KdWVWeaSB)-YB zQHF`g;u=E6e};HMrsdwT79Ex-f1|P_Mn!yVgAi*@Jl#o?BQb^4U>Yl~&kJuM>#v<0 zZV*fi=IX?tN-We_Qz_=+oK<{=pqIHguc3#V~Me|1W7fqrh;Z!ixnfo#`S zxPsEeU`eMw@?p4{A?bn|>|!03=J`pzZHRQI8Qt{<+20$DzA?OzbH zS*D$P2d0VLZITyBMY5I7n#@bIH7jo(8UTHe7(9xdNLC_LbTWcn2)4&}iMoW#PB^Dj zxL@b}X{w-*j==1S4C5knboR&LU)^|yx|0j*ZXDpaihZQ6W)!kLf3d8-5cx>e1;#q} z=3lk49&qIr`+|M17<>drl6lV9U_Iyp#&h^Xv7yQa<+_=^Bx|!~&`Kz-#1*D_D=S2X ze7ZwxK*l4I2Gp_%ghfD)_w3kYGf90>@Z3Ad7TUIeafv@9TZND8zi}`_S2%%Nk@qhz zk~p3G>sLGmm!PNuY~~R^ms@V~M`hH4tNR%(;pz1G2)f08#RKT+c0^$If++?y?-3U; zu@RhAf@DeTw$BZXpde$QKx6;m2oW=k(%`td;a?t6_wGD(r+AJjsjS z_JNZwT1u9$-?$t94wVTKS+vUDP35B$wiwlL{xQdYO%OCT1v(Kd`Xt};q#*t=ezwRS zp^!DePHPyT8-9A`PgdU52xp8aGXqh|}z`xu$(|hM%Zd`f`2i{gwMuGJ#z98o5FE?(z zC3@~7JVVOwurdbGtZ%V~{g)dD-d@aEqeXRqc?a;zjjLy-Crbr1r8~f#BCx9sDzJH} zZw@lEqXR##Q7*$Ch99=9v#f;XNErM9{J)LLeFz+JDQs}Y^=_Ic5XArxsCpb9lcC` z-MIDs{g3&}>H9Q`tLv$iESI!^=@plYW$tIAJklpMa%Z&Qt=s$?KSNIVQ7o9ZIWd|S ziO#oS(fXb5{@pD2+pyH@eM;09Wp;M>w0HfSJu!NHgah3;F?t%^EHXP|jZ$>Z+7twO z9dsotQz~+;#y-crB5(4oAic=`5BvQQHzg9C5^%gAmfZ9)A??f|Vou4OgrRAkOF=nGtxG}q>|t^iohOWk7cX0cnzef}dV8NNR-43>oobs1w^neHjbO<# zWUWC4_KKixt6erKpTUU|!7nQF;>j}6JUdx27x=bXzl3xnlr&o? z{`6J}T*031vaN;8a0(#G5)7SOOIwlVFng8ygo;5$s2Buq?ibyN14AG6q)#lgB<^`C z(n@FWj$i}G)S_Chml3>gXNAciIm32Lwbw;juU16;7%$%sbgEBG}<3~KJWnSR0bv78AK-= zg}jx0^#|0YlFl+jl@ldIIeh`N$(i?Nx1mBfQ{d;p&X-2wCYWsgbzenck)tMcNT46PLZID6moMx z|A#g3QZ$*%BX9!ijWC$4yeqvGpodjpC3;5YU~~A4lM;ur)1q}k$GLj~_eS+Rvq{Y) z`bN1u^u~yC8q$jLgrF7BblEpS4dn}>+5s+=Z%Vz2g-m%;9%(T7=BW5N z#^o&cBzce2wNVp#jJqD~<83dH#Vcv+L5z)>;2LcyJnf=6^r{`+hOz>Q2C(*k77ObaU5Xq45uIf>dK1J@Spn{wUw*{dCXI$lTmvmx4%_WZM!g#%$K9=z ze<2GDL-a3$J|Ya5bG?4?UA&Ck;TDhK%QHr&QjEn?(y7L*R8Ij8Ma}bZc;{Ngn5b^$ zJK2PXS14>m!{ebSu@|hh4(-uL2nDPlo_|H|tk@Y|A`mpQaR1%ebaNf0Mr76(w6yee zch?siNJ&{uro2O8ZE{^`akf-hSJ_oe+Ox29paxx%2@95401&66n$QP2J4v9IzlePt zJFIoxTiquVMKSFgcF}ty*DRgFUZz@^_Igc;^A;&jI(f~)9PC~YqN ztrQ+}3^jOsqyOjX?-;TuH|fU^%_4}M#FjzrG?3%RANf!vzaB79UEeA2I65N(k{G0> z){&)wMzd*&+k z@c~wRf4rZBd2Hv4JC&X|jI89dpP6l!>yCDvBPDoLpWQKv}prAb@5)xM6 zhT=&|akY$&FyONZ?|OpTzTf%W93vXqsUDfVTe=;(hA+_4a zD(y`3UU7eoxfV8F7g1|;#o4hlcf(q4Q-C5yfwz&VtWGbTnBPkwRtDcaQ2t)o5NjPQ z@O!EHPWWWj0Z9s_=ak)+j@lV{}<&P>lAk#KK8)lTwf>+&$Grq}jOHeY$ zeKK3LkqP~wr1AR5IrXqo*_h@&c1fZV46`I^+hke0)}dv~0^6CYe1}X2?&9^U5QW{? zqQ3mJlPY$}`9qV~FEpjaheP}pG?NN(>~whibiVZrww(ewsH2R-st=sC-&CBlu>1gi znp@Ol+I?n2EG2^AOZ*9TX3(G~7E`;x4zollX8y{6(m(iKIE zgl^yFqr62k{%QiGGfhFIUo@h6(AjezqoaAKh@7RdN4i?o@aym|+(LTsi2-M2F`zAD zqak>Q{Q_f!KS`t++Q!Ejf+zIt!5i+53ttCp4zVWG6&D_|NivQ{q(>+hlBOx z*&q20DVMo-%Cj#)_G{7(sGS!8@?DtEFIxH+JpB+j0F#%)sBZ*qT+m#y{(Ar(+1*lv zt}jTtTm37ZC@;VG*!iEJoPybNi$?A&;G7S!M0I> z9;f|WQoR_NOnG4y7xS3251VoaN2B&H%-Fh~qn=LrPd!i^eEuiFoDwLcQSEm`5Ni-4wo*fspm{Y{p zSHkO0^M$a)R~)N9n!A{*Ud$KR3zP5}ELYn5Ao!j&{L^dMzh-G_YZlCAkP~`+b;u2C z&nxDEczXnBkql-I`I;gz+u)z6YZdf43unC>> zzM*Z)wsJyPa%N={M~|;Y9{0j1F)Ug8N5)w$SZ1{APdCANE^7<`4Q>D{Sz|1$;Z)EU z?p%BE>+xZTUQQ?*0vUDPXGhZ6gb0NH5TY9|RMNY!VAcWzY;35$1!_R(xj1Hj{5?GN zq~M(dXNKrq`8_;03snEdIhX}-E-qaE12u52e`N{c=e|3*stLg>(R(R#zz;WWfSDP= zD>XlR^4{Yifx9kh0Nq@q1+X4tTnC_$c$P=1|J-yR*<*HB0*LagI5@heDR`M6{x z6vge-{HL*pNyw)#|LGuL@1j8TaBF5I&4Y)Gq@iIr3F(SrrGH~}0b52Iz&%xDa?sFF zGRP6lJ@gRjCEicZ7(ewGzu#%GsWm89JXh0AAtpl|JR-QZ7&4;A7ePe6cRkXprRzBn z)X-|WqMosnhJU+$qBMPrt>yw^YM+@`YO>HU{W4UUZp=eDk~~e00>DEd{il&T+AP~4 z#atzT;ejT7yJEHWT;0*N%&XE?0GHds z9|E|!(yqXmM|^+}MSvGt+P%qFjir|!HxR(5()E`ydsw0FF#)Cc+n`dTBfbvx0WcUNuW;!%-n8d zI$%*XxHv+(tOh*&Oup|QXsx_*6vl8y0(c$cxxn?&ey`N{lFd3Up62jkb-^`hmH_oJSm0+N$N&*FbbQD4Xof@F?RJrUZKq$i3mio?iS^9kxP-H7SW z%9iLQkI%?h7%E1kRvnIMsyvBT9)%`K;ZbUCO1Qf~M(t=J4t&|eyU}K@AZ6UXpvw$| z-na+fk%%0Y&=g2)A5O+V9a%Q@gp(Ta%7+t#k@>_v^iuII(Orw^)mqgJw$a(3l zPf&$t4}lfxx5o9YxARFs55B!VK=jgRVWUq?kpSMSaEJJCeJr^CR?zcaKGXt~X4;gc z)DOe=-Kcc@NU@6U@*V3YdD`Jrp!4L{&Sl;& zxA@5PtuRI?{Xvy+4&8A0{37%!IIHuLO@^Nd3m4As`H635;Sv#-5zDko6xqM?`gW zeq1F4o4kOoGwVBWzAJXiVbJFl%kmCAFo0Kh#Xt@i0K100{sx1CqX>and^shBx(3l2 zFHGiI80J_6aXRHy!hl&E((5oQ6NsCH<%fwDNU};;-WxejJY1llzg044>J=`&rqVyG zS|IcaNWIBu4%!Kj^!^#?&9(SLD<$SAU!iuF>cHarT_lg^K_X@f13PvO# zd*cmN%D*p!e&OIm@8aUul?6`EJm(3VQiDc0 z<)n(4KQY*NP#0C~2yr^ZoDoTX7Ff_pj{eT%WuCF=(UwT^4WjkQn&B3_8MSybLc|cI z8<+tJw2ba0eM5MGwzE=SNGnec6E|3K zhi0}V=@cSM7}QZ#&WG~!DfZ%=t+Kg#%u#lxev3V$`|E1nX0XPgVWIQte4AX&gJAfuUm0EwxR$!3kyrJ9OWN?V9)!P<*V1VpM1 zMHP>lGD$8p^8oc3l_$?65gkGq&zW*~EJ{Dw^YUD9t9HyOB;mof4o7$o4onndX_sx? zqp?L+kf^qrk7qN7MYOm|JsX5U^zEbYA6;Rhp*OG!rW5Z9 zLj$g{+>#7LsZ3LzfmmFS8v2uIkWx+m)*y*KNk{T;J0 z-=z$Z@1(ec*-3UqMh)rRCHVJBc7-Y--W~$snnzQ@c#+>D@TRw7O1rDcocFS-(SC= zd7I0DFhN=KQ9#BRF*cQp0m3fB_K5u_0Q0stzzqk%x$)+7u7VXWO5nPtbIcx zNfU zAM?ExlH8-}GJ_RrcSdM|?#fCgd%LAA#vzbCvs*4DT}|#$c2!M>+p$OQh4FhKRoDEo zCvc55DvTF{jqQRDzBovK^0|*nkr=RKwH$XPi9RWz_Gsu1ZqL?69g4)0A6-=5$|PPb za1zO;`%_(4cys6c3~rzJa|;c>7Y=k!x(T8APBi+yZGKZbEA+=*;+}>1p6Zq;JWLB8 zFqc30zTb8Ejt!K#RCuGgoJ*Dnu?y_{`4XU^wYutW>Q{DpK}7Z$@t{o5hN!9Lb`0iR zLdSJJC(U*YR~Y_WESJ!*TYb&>uwplwc^QTS)bmv(6JbjbCX`n}G&(pvZQq5AiUG6X zHQCB_rT%twF5>y`HoOnm8`gmRm_55bv^7s?%sI$VCz6VD$qc{hyhwQjOld0fa6mke zKdMG1iY3B(igIWfhIm0!uGmusB`a$~cc>3el@V3yn&ejK2hgOLL!N2-S>Eh(M3!W@ zU0r#^B<#%>TftvBB>}fN`#1FA-)R62!i3s)UthVQ(=LYgGc4k#_7_lrcWJ9|K#1&@ ziIlaJaA4Er_v6~YkARrWC#=@yJuB;zMOS0nJCd{IJ!XXWByW3ql)&cU(YDA5mP4E_ zhLK-P;zwYUGhpbOT0a7JWau4t2~g+DGPP#I2Qn^yi0bG)Ho{1N_rV@~v8&;680*Ng zmi8dn+u<>FRFnp2_dmiykIg3z>)e$~H&DKJ)YTA06fRWzZ9t!_`ToX6xAdP7{2V-! z|3%w72Fcd6+24)Rw%w;~^R#W-wr$(CZQHg_+qP}Jeb4)!cxLAN{h@YLMeK@-s8~Dm z%FJv1Rw2){m)TO47{x(z!Uc7E@I z0RXI^{5T5#m3|L-Bg>zTd>d;;8zEO)`~P)5_cLW`V5?ww!*yKr8-qhg@oNMWGfU~I z)r&=P7Z;l0@d6(v$Nrs5J*(wVBcWnUNK*@Rw`npzD{5AjI4{yR=pw;PE;N{!$dG7EzUT6KfKeGUM+Rvfi2!Nu;?hP`RZ!XLv+d!v? ztLQKM=vKLs9HNUFskq^0bMryr#_Rc=xcxG(%F>QWL#1IL{qm&PuDwHm_W^tXn-OM2 z3C2Zr>Znot#fM+Fq5$J`8e$^LT7?l<__!C3mXiXWtrDx1SJ*+!S|Fv~ipKvPCgjqWJ6| zIMjiw)GP3liCdAyg+D>!(Pck@gz;;M$>VK`!i)@qQtAft(bA3DPU;P97OUxmPFnE8 zVz9883-#rl_Tv`^U%6J<%ktKg%vfkE4J=;2_AC}UHOjU1Gy<te z!R5+B9wPJF*)q^%szeU%pgFmqo%~PMF*CxWq1&3R4as;93$IS9druLMSWy_|MTN&! z9`M5SAk5ZqUFgYe?f3IQ^s^Mmr{Y^wZM*q|&>JvOAehCQXCsfEEEFo=}H?y48OB=@dr5FF*}9$N!?n~Y&%z*Hy9>e5mAYo-?f7x;-+bfHb$ zSLvH?mp{6yA;P4waTGb7*4tic_N{vqi9%ipr1-n+#TG_GEzr`ET(WUxELpaFg7+o! zzb-)m%#gjX(2&2tYKQH0Kz4^*Gv-cHya2+|`^^lBeZh0xEyVoUUaXhsGis!4_2&cO z8J0oT9IEu^JVNG-6v=Nb-eK29Xl>C1dde7%Ei!4S=L8!o-BFsPd?EK9CZkc1Lg@_7 zk&v<`bH(Wi-AjS=CT{!0^t@_L4(#&Z-f00!`&RgZxX_-E$lGyw@j|vCdjb9`-H~}| zP&}oD{0f%_@fp}g`4D(h0Et-M5neV0A+?p)JxC_zKr#@b{KbI01@0F>Ia;UbD?J@i zeX0HMNUvt{F6BaI@)MwFewuH9uvi~T@^?=4Z9JK~ldpOdy0Oci<{j*amjlV9h;@)B zKXX2+k?6fk#mmFMW7ULfa(dy2aTJ$?RqxNl0VKtQxqT2RIxmn?_33qN!!kLt<@d@P zX7J)fYo%dRvQ@Ebp$2#Qq?%u~lk!IYM4Ot+nta)Yp!CsF;Sn>8<=^#XIBCXJ%yl+{ zQ6m+R4v`Dpu4+xA8mT(3F)7msc5TfZHDDChQE5F9pN$sSjvL#8Wo?zyK8I;k*WEVt zw@HQNIi=yf)Uy|AwUrxgeCDiV1WgjU(@1erBlR`;4wU{5yP?7B=0vrjHS(;mlWsKFkx9%cH=%^sTE*8RxmC~GB6OWDB;3^$HBabf3Re}Z4~}owXbkU4hm5Y2 zoURlCUG$7k9zSN$u3A*t7@;DWEfA`bU#Kz~tg_w4Ux~E}sb8QlNw)^8OY`)8;pBJ? z02J73gzM9<*-bHe5?>x|YQ9)S&I^WK-YMgwj6tCN2r<8q^;k}$Wk_c`T?D6$)4D^m zR&EgZ;`!GTh$+-B)VoL6MxOo}WY#(C8H08EXPAO`sTF-XK}ZS_xJf^0(bV$o_*zG?(c%+W`77~-f6k-Fvyy%Ff9=>l#$l$IGhe4e8hd#+A6#5uC>LtOi#F|F4cuME&gFNGbbaoX$@k3CTkH*mxFZkK*k*enHf>vT$e~ zbU~k~#Wgv(zB2NaCFlugr$Z$XoZBjN0>7tsw2(XF5$sW85V|7b?yTkW{LQuoe46$A z^uLd~2WyXK>GnEqryN`*YGkg?Bs52~faHtLTQzn-x7M~d;zzDRXg{BI?psX^aXk*2 zs#NT*nVCQpaittW9p*MTLj-mFgU!W8UDJXHdOS?(Snb;2=Uo_&eNAK|Ov+{*DzRo1 zV$jG=p@(k>E^p&YF8R(-N<8F6?~S>f`3vp~p)PxYnl=*bXk2@;)o~B6_y^xTx`g=R zSLcQq4g84XwW$Fq$gS+%HgL(pW5#_~<!?m6g%@>oAe%l8uT^Hyi&ixnTG9PN@(Q000>u0KoiTsWp@wjO+z$tZen{%^Ym3 zg`KPo{$H=DDunAlYjct2ruxQc98x=dM1Lg2SYj*$(eYhfa}JgjIZIM$NyPYO3o~jP zg6r@_iaHH<-cuCyTq&rBTx#l|-X;K_k8Nhe>eu(V#5pKk6!6n zX<4t^JT^x-4KkdMCk6m4weCDLed`~t99K{KIIbqd{jDL-T@lMyheKZ86kAgf&o6d7 zTa-J1Tc6xKK(0N$V4tdi2A#)$XmR&8;oj|lkdht(A`<;EWtI+vr#*==7!5tqyv*!TB zO}hh?wD2m#(;Y~Azenjgsq%VKW2dKcJjCjHD+Rc{H)8|!0xKY;zJx-~Ms{u{q~PLY zqo>OSt%Q@F&O)?^o*0kO2DB7I{}Rk-Ziz=cf__$zo*gPmPR>FgA}>xbXVhb^cY}_F zuG|hbNGuS3F`mH?OpL2Zj#}8fc(>EaB^irQJY8=EhLdOrcHEqk zfnbQ48YOPz0DX2|QLQLwE;jDAk>6C#OmpefN<3xW^SM|hDf;30GT&=Tfq-MwcI%^& z|Gl8ZRG76)TdqubUm0nDcZ_*bNwrFqR3;9(ZGscWh;K_LDHg^;2s=ejs`l1QCW_a= z{g6IDPv$QH-v(lw^5KO(b%UfV4@xCY+?6sysk8Y=L#O-ikb?*w=JjYUbYl z98mYf-=vy=Ze~jVheA6$byA94xBmjN3G-H1Ws$27o-u&@y)!ghz`oqw zbvVQh_~j|n`;0*WOKvtq_w0#Zag-O8e?IEUqXf!USezlaf**`i6oPsKJdUVnmKYyY zbZ`!JVW-8OAa2$kwtpsPY-q>=WU7F$e1h)V1LT^JQzDVtB2 zAeEqlEd4XKA2{ZQP@G{a4S0vt!+mplp}?M3tTaAuHtb4u%Xf;nWIJtnc}|=uc*eIF zGF1OWCpXu-%6jJY+JUf^um~mQdY{9nR2~7LYzV0p(|0x5YPl#ZO&nn1LNq)9aTMC0?h;p6rGi^ z3;_g`CMZDn;gY=w_Ht1VJ<5hZtCd>uxK}k__NRmbQ&VgBv{3ChrQOh8)UZ~R7P`Uj z-`u`>o1Rzj1Qo-rz2<_=sf*#35E|m+CKH0nA>Q3Ezyuo8%C>tgadnAna1pH#v(tq^% zZGn{^;<94%Ubmpy;L7aY@j!pa0B8Qhw^puEje~{-fP-D%{I*R?p3H!6~0w?yX05^yRIxOU&q86Mmci#qLncL<$P7>WmZ=Y$TWD7WPP4ij_HpZ ztfzZFk4WoytGdSH7*SYIRk|3r=t2!aVVMnEGG#eD))69YQQa$AjUH*F@p#Fby1w4+s8vj< znHV*@tN4tcRtPGLFD68yn?I~LGz|a!bqUp|=pw!HCzKg{9}z%1Gf2V?1D&QM8H`#{ ze#^)kq}X%BvpH=hHpiD$QHLHduY1o9JWuQ;Is>tzOcUyNQVL=}2^9&8)1Wbmom#}ypPZM#Fgds= z(tl!7q@c>+SJc^Z#Dsb~Jr6&ACBSxhP!E4;qctBn$*v&b z)tI(km+n{>W~wbM$QA-BAuSz{QHQpSi1z!``S%l2ze8zDrhB%8lXy6e`~F&$v0S{i zcE<-oyaWmm^wSUox=(p0WQRj19w_; z`!4^yjhhB$Is$eTKw>o&oK|rER1(rj0rx zkwb!%k)37o8^XWqc4CArUUCd?L7cWa*wvN@wqMXIU}j@uHU`%Fo$e-=faN+2*GMQfAid0$e&#+XvB=_iLzSka+l(9atu$2HL zy>^P|>iJo`VAWTR6~Iq^nlfsYJ~IVzx*H;dXA8Q{h%sI6RfuL3;ADSMF~akRRv9$LjgXu2NM?J-PfsN76Ez)T-y}JXt4|RL;OWbSwD{nB z#_T<^EM^9$dc>7Lnx4QIe?FzN^mh&jr_L&(Vr}B>0;P{5(Cb}M!AxRF06HLT84hUb zt@9g^c|vB?yDr`O%f6C488alV7Tcz<-B*2s{O^6an`;pbBLk8#U#_x7OS8Gv zW&TiecoE7nDwQN}`JVhn)ne;Y5oQgqaxu*&`dJXh=yqb#3sm*hhR$F z?950%DMG~|U_6Q<=U^{kwc?JP26LnYQaB+(yj3k1)lXf?+CdJaCN8eY+WI(J-*LIk zjVp&_P&(xb)Nk7MR+w8Mz2VojC2P*Wv3Nnn#%X@RDk;+k5C)JF1TuW7Ms6(uWpW zjnT`0UhnTd{Ud-rg#4=Q?@}}u`j**V-IIDQ@*CVbm`=8L@}0lJ$90_UmNgId&jq8g zl)A;bIRg8qfV{NQv*td7nYNL`tP%8_#|En?wvURXItRvDpt?8;@14w?A@{QK2ep8_ zB+_j}X1OpU<@7U|D-mHw zobSH4b0yMY8-5xT_g4$9j3~|`hk=?uYHu!EEj)n&H^F2)$`Te!8&ZG^0ZIO3mFg6h zfc>(dM*=IS8~O%4gPM? z;p9U!r(vPj&m@c<4_xG7L3J`OAoZq9s2-6fYvu6kflzLs$J)stKnh(yML3Fv&^*v9 zxTk43^o*n$ZrJe%K}^r`Gzydcb}|vWZmoIA=|*jRd0hW>QAdHTer-l;MzK*5 z+l);?3-(r5#-_BHwJ^j&fl`{eIs3$lU1qydLwXK5a!888lY2gMx5`95ze{y@C=D_g zFv$?fkm9%LdsP?tdrn4Bhod1(y(X-&2Oa#=1XF5! zHlRrcC{pyY$7bVrW&hO7EWTc>CoBzhq}!$HH&KF%7$X8SA*n7vUVme~s$*BZ;jNh1 z3Y5XFpAbQB>5AEfbHBK~q8E;|V%V`TGl4?@kD(}E1RQd};Y0kap*n6v2n=`E!aw%M zyw&7)l70A+-OSdNk;0EvgvKw@S!PFcFZ5a2W|50=g2$t6nD1cOBpK{;*OJ*$d7-$! z>65i24IDxlt7Z;XkRf7HBOJ)Gav$f7)m73EXfqzS5Db$ z(e&y;Jvk;WWnkf_%Tv&GxN~uPoKpa9~1+AlBNh0%*M(2>EtZm6tW-P? zWHGuy;@X|}oO&c*$db+ZD}G#N_sw^&qCEiLM?f8bic+dg)EK9g*yLuE;lh)G^&m3M zXom~}l3ORggsWO)UwJzmbLrMeh>>0EcfayM93{tzD(LXd2L4%+WsY()6?(n$%wS0R zWt`xG2@~kh{qRG_h-Vys0mq4SRQyNd`1bM(#+TnX&%%qyhe1>~zs{m5+GhEzm1D;+ zPo9ULD4fNI!dIHGP5~BkyIA_AV}NkTDMFG@^=-okcJw=55sdYRB^IeaIQTB0jS|H- z|Gv&$4LmY;A}gKT^D-vSOwa5K$XC@c?cFt#cbLzNqR#bJ;#%jpazmjwI;j=L1z=pD_cKyHuO2QC*JIBPQhLvUnQHM!(B>WP$;+zBmES3Wt& zLo=_9nK*tI>0gfYz1)&tZ@>&Ia8)Psi3BtT<>-yDv&c$z=HhjGb0_rfnGofzda^^1>G5=-D;o#48j=mGb4}`+sjZFLD{Q6~+_oi!rV;iPm#x5 zC5S-O2Dxx%W8ul86%E7EAWAbbDFfCaMcF4ZC@2_{H!#x}r(E=5M^&@|Sk^x?Bmpsa z85(Iqez%h1-U=e=#auy%^fS9d_RSlW1mfcK=S`)-lSePDP26?xn6&gN05>ep0ZOUmQW>)E4|JTT8N(d6NhqMs-j~}) z7Gqanmyky!?F&sIF&RPNEV#1EB!PeDmM@r%hvhpYG0kJVZ?VjgCk`XI5+Lrvea{UY zvX@#}>M>}#DE9=^#8XufBigjYs|@7$9gPXj8_?%6hgX>7D#AQN$X`9gUu5={b4sp( z0|Cc_I>58OU*!3WnJ;+l)S?uDgENoW=!_^H$zsGl9(=P^l9?Xg4bc;3kDrey86Ugn z4>fv15@uJ1=Vr&d)Iq3Sr$>d#D8Fek4!#?A2bxJHlfXkTOZ>qOJJ8$c_9IiGGiS?otlPWaCZj24Jh%6fhkt7q9JI+HDkhu70qaSvogdpElT#^* zKW1bJkk+fOIpsb|CzDH}*yWPJl7o}p%7-c(_NNfSg$~@?m!Qndk~gT8(BIvo>1_D!E&vI1KA~7rCRxeZH=wIV(6&-vLa21}if1 zX;;?`0N-d9%O_MEe$o*LE7m8lu@M=^<($HOuo^Mk?P%9l_~g@j&sjo9>@OmhXVs?Q z0~^UE0iML;+5EqBXwn!|ZH>)?t0>U#)O0n?^*Xnr=9J>~^MVr2V+sa$oGdPcH#kXH zQzrH4oi>gEMCphK^3MtpufK?7ET-(Mfr6P+ZlQ)dISacUF${1vXGcL!!IhI`;vbj9 z)IpGHB0(!j%c9)2ul3VZ50l(t16K~l`*5!ZLw^`cmcc4+7|_8|G8h^6M4&M%RN1p@ z#FaCn-=@Z2(uSB)5>_RP)D@L8??h@V0fK`oLGa^Sb;q*F!o>lgL zC5u6@E*^6X0~O?inz;)&Xj$E1+Ft}Ie~C%f8-v;fDGWBZ?U#fO?iUOf2OwjGr)XYX5J(JvP@Nt|RF?}u1Zl}$DgT8G~o^1Q~XgO9uYzwz!Z2t`7p2tAPG;gSD zHU%mXw2*4{ikHM&R{3HYhOq8taOzY1xgDn*N8dba67Ls76Mj|T&PvR|PlYNSle5N^ zX7uEV8mcTHeuBt)nT6bw5LL7afGB$$CFSmr!{z1iinTeA&wTa%UOT<-qDjFO=1*}7 zrKqW$#c|=zFs3vGb&Zgn%JYWiL~vC_s|$K}HT7gAuQPd{j=c-!u$uy4Qm03MT~4+C zcG6bWMQ+5z2^4F?rOt7iN?-x=yvpcXi)tvyje03=(8d>i346{kt^T&iD_-rGzt}W; zvd-w@t-lV!@NWeJzl$Xw9caem8Fr1c9D2{FiqTX;Nwv$UQdZhs5h(z8o;~qiJr2N% zRL4OfLEPcupGY+DRxde+bOth8-M^;5vqCxu^_YM?GMnQn;wj7sGFRyEbxSH}L@y1-}#|a0`l#yveI@ zXIJUp6agzg6wIBzFgTT(`T;IAG~$*zJbDZurH9_Hq}3Qxtb9HEHgF*9;ai-yA~E-H zsi;jo@f25r*};xtz*z1w04+jnMgrP4M8FtPR$ZHa12AR0HjM?6K0tRg>l6(s5>IhW z_vkw8D94b+pgX{E%L?kFe{%cMu?77{P-WL9?({m4nzVhz5w^b=u~K*i+kbKA0wub` z%sReY-C0g_=Zt6y*t5EqSu7o{RbXe0h#J7Nz85N?5&TBpFGg&x&KC{bQ!^eFIGgz9 zR*W6+M$j*Xs0_B2TNV&-CG~@<5UjjaYW~iG>#+TbU!RjafVt!mZtFbPM9OqOs*DZg!MjVdluT? ztj@rzy`l#z>)cIGhRrF96OIkO_ZKShySB!h07qW|$5)=cjtz zVcjDwb9#dpA}{b<<1;(&3ht3PMWnAN?&&%u*BK_c5t^MT(fOpc`$MXAvDt+sqC@TC z?wGMi3uk`s(9?6UPjhha%Q%k2F8u^N{eE4kw7j8jVc4GD1KeUsA55Rf1~*pT@!p^m zke}H5LFi^+AC_;9C;{b2l+V4pD$GgM*2BhTJ|Q^XO@KSoU4 zVzs+f8EWDPYoM@zg?2vB_=V&`X0JwN*o&*PO={-nmPjXQ4NceBC49(AS}tm_l7M&) zF)RS$(@1s^b8|vaTixlwk_Re}`qCpnb}Yzc?EHoV$7%g0$xZ3~@r9wFrO`I0Xe~uE zg;+-0cA??)#OiGQh6oCV+XK(Smlm5++eDP|R#Ns5NM= zI;T9LP-<^?LOHB)GH*xA?4rxN$#EJ}fM9n0Y{R|GjUgx78$)o&D~UpSiIeml?Xt)ECOM_Az7HbCz>e>83C#J_hD z-LXQ>ajJ7&+?rpvdr_>9fS&9SZ1j@~E)JktQ(fxUIic5WC2;&RXZWIlDysvi!0Zlw z3k5$?@&@I4!x(yGWAD_3c>|Y%XZ8ru^3DNa2eCuNxXhVi(S}l^&^NSjpN)@ixmpfWABk@3!=j&&Pr?zs+}VOB!|~t9 z)OxV=sRRqy-r@bq$?Z9S|2Ve=pGFVdsd!L_?dyRF*_jQ1|KQN1%-A^}z*l7i@AK*B zX@KZv!}cG6=D*Q-xW@Lk`ME2e5Bw`+Ya0J$+7`U}=j$C0+AMsMgV-7 z$3Pi73C>UGIi4M1oBT^W7#Wd0nDLW$cpxl0%BNQacz1?{cx7?-@`Ob{*Qy_q*2{c6 zvbelVl4G2VidvYNE00k8!g`Lqa@yD)=7GnhV$_5?cTI_rC#HQY%)+?IoLXxq40py) zMpJO~ZDX_@4UgU*5#k&ot5cT=dGe7L@dGE7S<~+H z@t1Vy@k6um0@p?XRF|E)RNDZ)RCR7Kv~XY2`l-LRsMEr{Fcj- z7-OV&`Cz82MyCDNQ4$2e=1Jy4#AK+dKq2!1A}`6i#E8PS6-?_bncp&pG(|bJ!!%-Z zMamQjQy;<5M=)kb`*CcllgF!fltC`WRU2`2@?teqGg@k~U3bZyl%%=PvY_0ZiiC(- z$&WYw7kzy$hd!>mkl72rd29h0Wp>Nlw){6%jaRa+sY6SOk}!5E{&25STuz-=%&!_$ zB=Qmv_tUV6rr7)uRT0JT8Pk3{@P12#AwHe=awKL~sY{B~-W16(i%0Il;A1I3$Fee3 zc$TH!mKeGB!bn!tXRH=dW5ki|AvTWf7rB+xK(DdVvAeX8=kt<54FHQ9fTEIw&v z2UBC$`&xYNxLyY{W@2CKv2ly&Y4r5dK}w4L@3<@!ob(;-_5K^_(v=dXfi;3LybUM@rEiCA zo4NOb$J`uLOHC^MP6XdUud81Rw6HR`()UDARXD^SuR;kVEw)D_wRU8lQNfl$!ZlYM zvdPXXkkPI(SM2hg%k5>-gvyV|;1wHzYiwfN^BTQ9>B;zh>cjm3-fQ*Jg4RMv69GZ{ zgqMeR8U$5_nVBHdfXWg3)X!fGn!xNT+g#Flqo>|&TnBt?v@M|R3BlOpiXJn%CkKLN z_-vqEIj~>v${+5nCeBs1Lr%-GLs|b)e({xl;jP}ssBPKTQ12@3%tPDJQ@w+w-MI5} zGQigt0jXVH^qJD-3bGwPR56#eeBj$krJXW_1-X?t(}L`&p=+hi3Pa6ZnPEnl_Gf9G z8t?7bSx0~M{-tF)7ing41)WZTDt3}j%xIT)VBGu z->InV6{P)ssi@K<3V>eXa7I+@Riya~SfWG4^Po*2O=g=1>xpG0Mv^L7Vqn1x2FYbX zm9WFE%DV64K=3y;bPL)T7D;LOZ`y#*g>fuYeg;AX-zB_-NshX7EL@}GwTfv>R;Bv# zJEDh5P_A&&K@W-8Ny;}Y$Ao!Ow(3i->H+&}g{Zl^7y159RIjmP*st6j<(2 z3jOm8hoMiof+r^)Rrs!wxl3D-T*mf%!(DE*{4$^t1WblQL!1=GJ!NvA&Q202x?_Hq zy21%SdYA5rxaLggCwf%7sWgQD-p57Z>C;2u8Jbbl93%$eQ9&5=g7nVYV+dCC^i`|Q z?|01AhfPtO+)tT12$Ape$9BymmFer2zoq<87xyA6@+K^jwtq*t{Dq=&qk+;ha9yu?;r3>UjwXdAV1GYMiwN(FSM2ms>k+dpL zk|K$Tn(HT(+2;3ML;_QL`#`tlTl{*<8mUA~vbbDHFOAM^oTjHZ2bSEG$^7(yNQuJ6 z3>Fk8j72i)iE*4)m2jk>s`C5g{x^w^$=LlibDt5ZtO0R@7E^u@e_B{IdX&A=A6$kb zS3J|gs_1iPvC`-fSL2Xxo>i}lHw}anO)j+Qf-(&ZM*4#dYtG{NZP_!jG8W>2_F69b zr_F6`hYt$d*Y?A?h~$i!aP81egfgqy5ZqINn1?>HV}YO=QD$L+88GehB(YV@9V@G@ znwF-6Lvs1)9&znr^R%@5h^^a>s@dgb^*!e#SYm~@Ir14R-R=6Z#vC21u3eAE!NteY zyXu_f4K>dJHD@}QMGhUvRaS59GNk#5O8kaq~$310l zhHaW2zgMGfG?$xKw&wC1w}po34)_V@rwbDulpI{2b?iZ6jpBf!<*Q9l*5-Zdr6rT1 z+89FJnW}fQ~*!i4ocYW89`nW>+J* zluHE@xtTSpCCy2-=aquzwdBQ0l9Bq@mY&LMeM&<46_-BRHDkL0FAknyk7J?PE7VI| z3hniS?4hX3YLaCmri<+d%rnEPTO`+XO|;81lu8R1g}Y*s8Azg4)!uf5GN&%=T5k4` z3?5!WBl~Cm@3pp+)HqKhr&TbNbZ6Kx%49A~D&s7-+vPwFVVhjUA+a&g76SU6@ zmLQ0Q@&!Q%lD30y4~JT3o=npOSO?lu3b+u2TnQqyC&0E-W0O)(uAHzW3AVIx#!^R| zRFjVdFRV}nWenW6HW4)=Yq5)1lGJgH!WW~jF#&VQSi&K!L)&ro`{EKeLqT8B6S(@X zWET!9xUCr5uNa4VFATG!4Y0%yIJ1XX`L76DS?&+As0Usu!e(&5t(!>budCw~eKeQl7| zIR>Au7*?eBCn4*y#`HK8e4~DwNP9=iCQNpIfn1=g`Pv8?PfnO-g>08i`G-tQvt2-r zBX@UL(GN2}iQV8f-*v2oJm257Zr`dH%SJT~apJ0b_;Truxc71g0n*o9Buv^TbdwcGtUbjyxoM8(TZQRoAHY* zg~hI@+V=+zDrbJMP>l?7#zpE%&1^pOt34WoOJ$>MD^rO3cZzwo8ax0i}>o!jWzzD3SDww2J;{)DI-{Z z;kEm47dDQ*z~DnBT*82HABJvW)%t-aa<6$RNJ4MdmWG^0u*QqbP*f^rlq~n6!ieFM z5!(1?fl=VGe%p^L{UTBC-2R?UFeS6oM#M#4;DuX&G5K|N$f|u4_p%|4x~=QVZ!yT` zffHveW!$>(K4jWuJM1~%k6j(+S8G4~cS+J`g*dG;5cEKgmYEawHGW02c&7?+0i<(YGpzwoKB~$l-$#fe z<=SbCOe0mn+APFowW6xtN+%pvvW#7lq{`Y(D&U3u8C#xgfzP=%4InI~exOlXEEm*0 zukq%g&(cqjD|-nU(ffyf&o4I#?=~8Z7wfXNXQco1UHntcVk0~!r2qo}B>(g&{~La& z|I*e!Mkpf(haXea|IwC`mJOB=GS_~MRi@S2Ad@GEI28F4W;?Pe6j3V?ejdqmh-gOm zO`|s4Q}g=f;#GUN&MOzEuOKE0(I6Hg52T%Omy{+)oIiNn;cOO%%XH>XWXWdF*Vm&9 z0BfzBAnxz7-F*V-X6%v=mco(q?qIfEVD;|w&c2hPU+Dxx!lw{{|o z{tf@6V@MJ4bO3Ve*l(rwi--zYLYg4Tw_wR6JDOWbo`0%r#S-@;6_~g;>8ln-400QB z*h}}7!{wfH!)`U+QO|z$94Wt;TBKZ`+>PHRIERAC?%ML&NZjSBIGtl$ycyG9hs0r) z$!cre7X+FvvuM9o)4LvRj1U1@mL?I!Ljv89Lo3as1ET<$mtzJ0po?B=x4!sQUFq?Y z=R9fIR!>NWF!tw`N^nA~%PLCuoANbWz|gw5TWT5+mF*C1h11FsFbe2xa{W8%6(T~e z41o%@>hojcdLpU8tzf@dBD7TRv)!E;Na0=vSm;)F#4l z+GF@$@-~cB+g_;%mqTRfd3a`Yimez-3NzK2YJUo5vQy;_rQgx_jPl4Ynv>Ru_UM|r z@_ho5w1a_0>cG53g*F9jEcT_^X?C6h3p7=p&E;|7$+uN#BZ)Q6a1p2tPCm*Ej{Hs% z?T6Yd1oa*Hwro(OTq|p)-{KlUImc(L4Ej)=%TR`MAE;_%yS@@sB-Y99OXf|;3uLlJ zq%J`^=bSv(7+-E4*gmq$pxgn28vSf*RFu*44-0XJaDOAuCbu&APZL)#VrI?=J5`P$WDf9K|Q5ZU+(0z@d>P++?)B>{v^PlS7ee9^LV5t zH)>)AGsQL!83cDoE4X7=HYp;s<(;mYI>aR~S1H_^^uhN=t^hD|=TVRF58wC7aH};q*BB09OyhaN_n{X+FIL3!0Uv<> zj1d1gCLw@9>$!md0Gj`$*~k7LrU30f(~p^@ks+Uxqm9+i>|_1^j0_4|GDv*LT=QD( z&Q<B~1z`?(?CmlIfaZFuU8#;NuhE+yDjMsa!O^3WH3m~qXcfXZ<@iqx--mC-moM* zpr4Ev;-L#*P{yk&=M&$mOq<6Twn8~)w~*A+-f0W6Kq>TOqeR~|2m7IjVv=Su`Z?nD zBEa%dgy>VWPL0aK`7z!CC-!&#$OSIJkH{ymE;0P%u;WNbMV$p&oKDbeN3z$MPb}>~ zY2pys2&RkUq!-h0_S5Od8%)tLCL)~jTn{6gJey_Upfai21tQbxs$%o6!Xv_=S32`n zs;-H;u#Pv9>W~enoc2d7LGvQ8*2xPZu%Z{!Sgi_1q1J^n2j8I-u+(*iim000e9zmW z4}aCxZE_}RkKbHvBX85+DUn(uO)=kZYs(b)2~#u`RTQ(Uyn77#@im#Y(X1u`E1HRD z$(p^{8@o34u+5o?6OEia0TR}pYL@2sFRr2;OzU3K=Jb1a+55*;y33ANc0vod00CZpbC9Y_nNtMtlL6)#Eh-{=`n1x^H z%|Sb6FsoMTBW(TpYneMcWl{mW)Yq@(Az*q}3aTi(hc_zG-G=I!fs*Hc;4%IYvfu3d zU)UcZ6a5I;>|e0s{BI%u7b@ewMJ(}83H8Mi)6N{sLN3d%VLDL|2CRi2&8JC#k6(lw zS1#I>8`V(X@-no>cj>d||BG+07j8!c-#ClO7ieQ==k|iLJI%)A_`GFC2Y?fxr7kpz zSan5n5C)62&d30ZLwKX!cD)}5$^}Dp@b5`hXp4W0J&HZGm)rQW&=rWh3py%pnd)H- z21~kw+>B7R1dfZk2y~J)Ln8D97s`Z5fhpc&9(tf**)yU{pa?&uLpCO*TP_3Wszl?A z5Jf4vOwtTkzX$a`KHMOkm^_V4k;r4b>M-~29n+r)wh5`uhojDdL?^$jV=c#$vplSGQ$3!VC zG#xpSjifoHdko&EmNP_bwzcQI!yusP%JY0s)j`bx9=13eBe((2zpKOa4#7eK`obRU zgH^tZZRKvv5`!JZr0(69zUUA^DbqQHfG9+DU5VK(ANT6(2KYtz7$B?yMy1PX}nq5bn7{xm& zidH9B!Y_d;*M*_fYi1qf3nP#S^qL6SrMvi{*ZM*D&jfJ2h;OX%@>;)6pW5*Bk3{6bP!a@?-?=OP#Nxa=i1EN^WvhKe3-efcB_WJ$+>4iqUr=$PLeuy+}&EGZ0Eod#) zql&RZW(|JP?OCWqe3-d4;lOPCQqWUl`18ic3Ahu=i8hRGfRX zfYu7j8Tm|4IvMRNbui-KZx(MzPaUP~*mO?M3sBx>L_< z_}!LRHTfJwWI-l`9OuS`9ACjJJVmn%`E3~cbh<%vw-YuN;8v( zIxlbF&1fpBvEkx$&(%|sGj~hUK5LMVgG0h@wZ%qqM^3txjj|@jQ5vEd zUgewJGK4nA-Q}Zdrml*EBh(-NS^Zh!=B5-TiT1CHY%ZX+gXCZ|9C0E&=Jda6eSyg3x3EA#|N4r2zB=2GWE ztB1xA*cIqCx=Ub$I|wqDDn@1Y9>!1)&2-;g3_B;t8h#!374A0&xW*)LfU+h5k>M2< zr!ST=e@BYm&RJI$2Ar7>y46cWE+BtZXAB$bp1-|72AL|1AVIbZSiH9t z9RD`8#l$Ll-K&3vXy<-4Vb`4Zf3gy*`YSa%ex88)pO?|U_9Xn5{Kc$nE&uu+ zlxPBRC~<67tispzk48!dli4rE-WbA|E|*bM~M*ZApihuf7nm|_P=Ls_TRjD zRSVg~V-)YLO&eO8ioLV)6aBwrYR*9@^sXR4uH>M8hA1F4zdLd!&gwf>wFp*9#mq4+ z){4y(kec4)4EQzZl?rm{*BZ@AQp^%gi5Op3rA-(T#&W(+9#l;DbDq~h!t!5{Qg<_- z?YkYa9o%N#`*pfKpILnEH=K3J^%MS_#S_tp@y8beL)|74k%N_(v5kw)QIE~XW5*VX zTs^JX{`Irj#{etdi-ifnvE2=YnGwI7VY}h%HW-itGsZU_fr|H0iATG)h2D111&v<6 zjaPi|!t&nXMbL-+fTumq1LCRQMd9$a9Fe(jU5w6Iv+IvuzHr@)PH@hw-*3d>Y1og1 zF|qn`E9vop;7X!a;Z zo&U%*V$cON-@-bc#48yk0@No-ff-YYT3b{oNI?E~D^*ko$Xp)A$=uJ73GqZ$;JwN9 z&)>3Oq%cF?^I85)jP#|wIMYmmtgcg2(J1EfF7@z^vBc!$(ewm$@hKwtQ(lSxkF|G- z&MfHqeLL*fX2;1B+eyc^ZQFQ~bjP-B+qP}nw!Qm3XMcO|b8h!HMy-olRb!1B>!v3D z^Ec}a_#~Jmxp0=CS_FR4FQyXoP9(a5oTr)dq?KZ-1&gFEMGBg^QjwI1jz}#3!9f2S99W0pqp$^tml~3j#THk`Wl!-d}J{2J&xfmg=32 z<$JjpAlt`V^`=m@qUW?3-9UoDWwKo1UJd&QDY|x zToE9?1c-J}W9P#jsS^{+vn-)7x-cGrT*=#z&ZNY2!T4!&KpF|r6p1XT+W_V^3kOqk zVv-Wnjbp`1fqAgxu>AC_Z0|{Qtf*&b?#3DVQK(z4r#=+%CbANidZ=NC2sx|N=M%`% z&c%Xjo+JWFqUR0LmX3J{3KUB@W1*G{9Jg4Ond*UyvKjrjvKeCOsOzL2sCE)_^vfJ? z5I$658El5}zs(H|m9s0$Ye3e5^f+trF-ea=d`AY-DlVr;Q-X!H)7mL(cFwJaLYr@` zaOpT5u+_j#H#5uHuFPpFYC#fg#f`%j+-E=0&@W{Q!Zhp5TvcLGuXoB&ml^(5OQJ42 z^oP1JQtm(#gw)BGv{J@X#x0GVI8;p73kO*S_iTlRDk>54%{@4^ww?|ETMmV>D(XJk zbubywG2<5&VqQpZr5wfMaJseQ2k1wMP-?@wVL5LV^`F7nFRFWNu2zZznNY14PN{j6V3U=>YFmEFG?J7+B_s<&zKxE9w+CKR)6>yek~>s7qXJByNM0 zTKb|9c+NJc=faT;*X!hGrKAjz(})^QY1T|-%EU-tN86@udRAnWDbqji6;ocsbs=Go zb`+Ea^*Qw#$SZa-Fe7if4nK|Tbp;@gX@xEGt?|z!mK}@ka8W&c?V$Q7s%p(rs(LS& z_5Anlojh0N+oPNjDhhNdy*}moh@}M>i}Jzicl$lBCME!VkY3{qYJVX?^~ZZfd9s+* zGMUXDCkuVK+wg}hsWz!_^T)Z$#x4JvoyTDssV2V?$0Eg+ZfPI3f{tH0xliyzS-I?L zB)qroXlQakrDu40ef1HoP@^{+N@V#B^2nd4_AP$)XJcgHe68@z49hhKV=Mb2udlQ_S4vUCh+>(D<3|+1@p730gfcYhYR*A#5DPWQNKPi31B3o ziulyu9Ttw=s1^Cj%@80q!g5O#u~*LszpRD5#!E6KVVh|k*KJ0xD)CfmH*H#_M9W4( zTQ@M89c8&ro0K0BgFjxmA!F4J88b1RL_e5L4JC8o1~FNZ0>9+C4VI~LkCV?|VysJ} z_~TSg+_SAyqi~I*&sqdkOZP!n9r2GxRZlpCniPkjnN)_yjnkrxjHR4;#Fp~+e`z#F zv1o{kMO;AR@l%?dWDih*fpis5U>4_nq1M41@Va!HPIIitldp1Oxr zs194#!E+VG;xFv}vbv7Xy+H#2`&5elb=EV01?$6;A&@yp{#Jk#g}v3|5$KV`H~?Tg zTTbj#Y4$KZ)jE%RL;=);sNH=m-Tf@7ntca12kw000#}FTHbynDCPVTec%km>_9rzCl-Y?nfv6y@OjY> z*|enngi;K%NU$Vf0`D}z238(XrB%5!P9|Ih^W70)yD$BVw9|fE_r4hJ2tH3uFjyNe z!D}`l3Vn4KviB*i z=u7%vQG6WHX@d$TEq`9Z`#_={Ol92a4vc(yni~XVREL-d?F{JNQY+b4Gay$|=oF7^ zXokr8wN@$aS7OrO=_@FSF~AttC&k05#)*>hpYRO%p6WtG7z<6_x#`$mKZH^gb=-Sj z&O&^t-kH_DCTE`UiTSGN$gi~ z!AgFH+wB-IUlniw*FPfb5k1zuz%jq38lye1_!LPQ+fhD|2@v4hZm={@7J8&^{BEDLX{*hZJ$71?Vc$M+9>WWAX&IGgA6y=2-ZSdj zuIB&!nU&5WlfE;ymUBsQym8On{r<^2eVySn{drz$_M@VQ-(;iwcON5!YllKFWQW91 zIyFa&9E%@}=;CzvP839Lo0 zI?dJm&&RCBQ+bGR7M2le7Y~*_viina^ zcGNZu3yn50KN}~S5~-9$*_0_|(i{p`D_;hx#Y@Jn4lyY#!wj;uiI~`g8D97^o6yuv zog_BTyucAZF5Z!SxMX>Z;vrTM{9n7iS|~l=Xpv;lZTh zyeNRU(%;=!>|W@ ztXD72!_J{6BgizDl-vWiPogbN0l89fpMUmdMlC#Bw`%t=XCWVGOQA|puDYeqU-#BR z7l-)rk%W$*OjY9FUFwy>8D4x#sW{7Yrx2x>xfCtCqhWAneQL3c9H2Ajsm1%GacXA^ zB7To8r0#Ui+l)mHTGtLY2DAz)kA=%r855X$curD<8HHSs%Kd zNhl$P-5}>xvnbS|csbGFEddAHRafC)shJe5=!XhB2i{UFv%>FC0DunCysQqIE!xnt zL_?xuvoKm~QWu-w$F484eC(0&@f3JwGVR~jKKw2butx&vd7$xDMuT1*l|edbA*Nm2~OIX4pC@f%jf7`J|YOqa(Y-&U8x zw$xwhfZ1|8<^d2F*RM*0R48~0BjmMQGNFw?W$OW_JPXXpdmek~YZsA0wVBGj@;pw8 zwCde+K$ovw*~~5N(hQefW*Fv;4Kz?nesN_U9au!z%3D;6q9&u=1Msvbkt3f3Ozm?6 zy27tZU-*1}8EgqJ$xxBtp&`5x`3Up7BM7T}sJ93sSb^|vto_FN0>mH5tJ~R}Ft_CS zkt&B4Z>e)6ncM8pnv@6(%CtnLEBm}gAe{8NTFFrds_h&2hu))V$J{ER^Y>3n5u1h+ zNWhQ11O|AMtr$esBxcg@B?Ei{^`utxK#6JxRQF0>;OP=bVLxlZ!EUvDeQMA+WrK9u zlEfLT9_IpjxlT%6@aalkc)xz_M+3e{B?q!^G0DGQNPGjp3VB|kSib=Kc7U&_KjJA% zp7a=}lk>sD>lPYz8+%9isdEPVCSIkX5$}6*3Kv4sG6nf3 z$tf@-tzvTEh7y}|UBTuOn-j6tR|G@`RqZFGV_9gd;l~7SyVke;TM;!$EbDm!-ZkaX zlIXx{+WSH(3wV!Pr{+d<^eZUl4AT+C^lOd=_YB759a>1u-(y#(0}g47e^is=*;6hW zbJjW=Zq}HZR}`c}5FpuuJTzQd!wHuHw9KNywT|k=njZ%plG}H@I3Mkz4i_x^{M$i; zOXNhcVVapptt!WAOFTXLGmX?pi(@-8L(OSueXPpq3g(7lk}ej0%G?I_o(&x`?V!=+0ts~INcD*GV8!{kR?Iye|I0U{EiNui?YS2fI2(J$DbS;S)c+p%cjSh{pNjVb!*`q-M2)8k=0Y`X{BplB^& zD8iC8GWC#pn3R(yKqe_*4NgziFAU5b{@BK_dn8BVo)d+RckO`&Kp4XHybhV3|be%~Tjl|-V%}+{YoHmi$53|I?P4<_; zQF9H$t2-(?j^9JSS2HR4AkcaQ?H<fmg@dmp zr}mJ$4Kp~Kkso!0HyBBuUOHWmsl6`uZ@S<%qXgfda&E!pAH-5c(93GF0>rbO!-Nk% zs9BOMPQDMHQ|Iv`%XoHY_R2zSMYGfA^4-gb7^MN|h`A)oD-f#K6sBpF4w!enV9g6j z>&42fJDu51mz@v5W97t}s5*w9n7xo3OizeNj2AdZnBOYw7htZ0({N;)WpG<%a7PxB zm^VZK{T_;FGDml@S>K4}IyJb2I;<0XFi*Q^Hj{_$6yAQEQU}$>9-wD>8JN^hng9{T zwVRN&sdHDa>Z0`%0}LiG<(4Hl{1+z?qQ_%JUY^5;Yg2w%1=Ys7C|%fH;Aru*nu#Kaa`Dj#m#y^L9$n#Wx0ppIvW`+hrW zm-v!K=>y7$T=CqCF7Mb0nKv0zAr=H1x>;Tj@=|q<2}^;`RKK?(f!T(4(e0{Ma{=)sk zzsvpjq59uq4a%E)$p147RV|z>{+AwTG95YtR|S9g`}@*%zUZ{KDIv)z_c@i$CbzIK z(6+rW#Wu;-u4;;W{=%7pd0t}9R$$=0i=JkSVk^uLSy>oDSXe|vKtyB^?D}1>_Zw(u z>T1SjNAWJPCHZcVCT-f){rvv@eSb8Wp8THG@rm%me0Yj9BLbDY4=y8RfK*n}5Qc){ zbRX@BDX~*pYENQ?3%YneLc*PC%9y^G)Ls?E%*YBwZg4)%oeSs&3FJhbmNdi#y0JBp z-(-MK31D?^)kNGu2KWYIuhSN=ij6;>S`1Sb02{TtZ4c+YHMg2F8~AdL0Ve%T&gJ^* z4N}{+ak29g5w6m1^HHT5h0e{*mFawy;dc>vMt7fc7w5cbtJUr27?+~vS2g!seG6~~ z?SMhu&CSxsw1z@fBUJ0=BbhA4s`G5! zm!Y=?Z>e$S&}l8E$v8sM1K&!}UCDbhzhL>FnBT0yCb+7#$`svM!~!bZT_>ZQd;ytu zjP<#7u>mL~ide>gqkt?6d%(gDX$VDS+n7FsQ}3P#6HcJx>?6yc$&gH~d^^8vfC)TR zXTK>T^+ra&QV==H<6;>)pleo}sB_Q|af^oFdslwJE9%cYGs+8K5K*>hns;rENy_7T zJlPU?%c&}o^+x{#*EpW1gBF>~q2)}4E;WfIDGm-MILtCu1{n}R8>i$bEkYt#;=+0)UDSB*giJr5c6e(ug8Okky_RgE&wnUwQ~UU#dh{ zrc=0D3-)kwrv4kmrRMr~L_ij=-OD+cSQ>ud(hn=MszOH>uMan7vC6!cqcQc zQB414!;&i0$SUgZpRo|24vtny+p>u{;)mZcC6J z8ou+s?h;KLVBuJd&|bS!Ij{O>qt{E)4)-ptSuOYThx!QwiN-QtLH$(G6tWvzx8Gt^SLdj_N+8(>-{PBB~dH&xuhfJ>>a*o0@sAHlX;l~*5X||x*cwx z@6da=r_~%HiFT>-5Mj0Rs{fQ@q#IiZ;sm}t?Vg;^)eLNd+h%bVYcLTwr7c zJ4TI!elt#2Hf-Bpz!*eq=cqDyE-|KbFJZn76HOcA1a;fv{fS{*(Q>k)!}Brq>!w4p z?oF2TGN%>=bZ#(ay^HEE)op8?h@@*9H3>IvYE{`-c0&3N6;MX>Ezru|;YATKr^q-P zXa%9f{+gJZw0s-Er=69Ow`ad~Rtlr_&vrbrXUWagzw{U@Xg?2e{*1FIPiBX=F0fVX zC}>rOwe}LhZ0r{Z36WK+R-DNBxSAKfQSg`lOw8?s7NnbdL5?rp-9t(KR~Htkk58iABdo_rhL32+(b+PAno9??aYr$dqi3* ztI#-a*t99q$n~6CVW}(k0nrjyc&I7@&it+!R_CJhvtFW_H#x-NC8Q}Uy&u(o91=EZ zd9`?AxP8k)St^ONFM3Xpf5uc6QDf$koB~fJh#s;Q+sfEWX@cC%ie26Ub~m}N(RUSF z8AR(iIC&QL`nWQ7az7Zw9$Hsg!<)fmg}ydbs<5E*IF->c;fBv=HQ$*G9_i1VV4b^hHYFhBq{;_jh2AOv(vVlO zoAjUeyuFKFxKO3!bgWr~3sWYhVf0|w0T4AYx|p-?rf+{3KeYrW!+a}~dXjqYplIN! z_w=pEYFs(kLA9XEc~*2`FxUSq?F}~no{vLixB_R%{+l#Nq0jRv1!vhXH~Zt<0H6&u zZvbXux{uVU;k!j+x#NM{c~6xtcx^goWne>A3ca4Z1&MkZS9ifCPK+1)`e2IEml@NR zliFeRx@&igo{epya~*+ zjP#ErNnAEXhO7ji_$a;n-8#*n+6= zA;Ye=qh4M~0u=x~Y;0CR_zs6dpJ|}-F?6)!7$RNPwIgK>@rjkPH?P@lhqI>8e^IIqy7Xo zg1e@S0pO?=d3vmp#KB$BAj_2JBqB@x>2{vU%To50CD;|oI#U6smH8zFM#|xh zZfMJjgG0iHnQx{sc;cw#WaQyc2tc>WT&M()n&exXn}d@!lMA3|y0i0C%bJ z90z6{qhP|My$qdUaCfwdSohw3d=Gs8AL7*i_&jt4 z=YCCyKYmQe zijAOzRIY@U#AvyUgpy=!{DchCrSTO5v+ojJUVBJ6Y)_bN?*Yo*?$JyjGkDxH-js9QH zlZ5Q0nPf1mE=2*k`BNCrpe{gv_xXeMUuRF*pz6_}>hlLekg}Zbgk4-FFQsBT2y;l@ z>Umg@G>L!L$~wp!Lgz_-ve(1r5ASH*r`c!khcH=ws2R|=T)6F+-R^*wXzFR`HP(er7Om%y6$aTrgXr^;M*uLtT(AN(9oF0y_*XHTV_VrLxq znLs!KJ+mtPNLeu2_L?DEUyr}RoghoKVQSe^Uq5ULl=u*KM$zlmkw)yCvoLZ*v*>_k z%Jzl&n*Gc!2%t2|xE@JU^c9F0Gch4b;>(RWnMZ4xxI|HOlJ zpK2g))!cYl9$;XL)a0!?CM{O4npS$5kuq98d$j{bt?9)%e~|<>i<^gxCoP;-AIJ*n z1Uub_8&6@Qp>a}D2QDDR6F)e&Q48SB7%O-AZt27=RxOap+fLEQ)0>oJJUt>xA0aSK~qb`9}-! zF^wt~^5h7uhP64l?-6&xIXV%9Z&@FvqggKSr{RV8aC{9NWd~OyR`E)J62AQ&N~6d; zRT2L9>Tm0_=pQI@PwPz!!k0?OFIpAWVnBs!O@NZ+lcH=)#aqRLJ`44#IjEX_m@aiH z3$XSh7W#GXTYeL@rj%zrxe1cUX2}o+jR!XmIDeGm0)_`1MjQX743?U^rHg)_bI=Ei z)fFpna;RsNM>kL#&sfbFA8Lv_jLiSP90UZSjC*S#+Noh zERXYp*&o+z3XKu5QcYyeXs58;-P!Y7#6>iloIG`YxM{iAJC*sZU6k3wd`K1yBuh8$ zDQah@Dm5{Iu9gvOnH+OPC{$GCvKo@Ck4@^kf<6L}vk@_n1F0BYH57kWx(>wZfizJi z-#}3+iPx4vv1o*X8nj_Tzcvua!WYHO)DUpzu?OpMQlhqU zVapKO)(Dn67$)__;7OK-{k2Eo?dcI08fY zW7ffsJ&90=Jl6batBf+IWP5x2D%vJcgK^RjG}n35)bbF-ELFc`$*DghAc=)m3g_>Auit8|{gCKVGx&+sCsYjq>?AX}so ze}xUND+XPpVX`J##5#WV?e>RD&-#X}gW&SL?yLi)EN~ujbn_^;Vk#MDqo)Ri2!EJ{ zh0sQ69W|7R4aY0>5AW>TzH$ic+cSl_Qc=N8vZ3 zaw5DB^|jeqs-+{UR9&ya72}NXAl%0NTMFct{N&%>&yMXK5F*0lRgf!#wR@lyA{-y3 zGD=Rcc)OC}6{U94m6;}EX2P`vyPK#FN+ve1>w7~bY?r!&uz*frA zR_pTpv5~V^)=3j*b=z*BypDKLq=o3<&17G~bB;%g64DgjzV8nkEGN;+Q;y+qt<-6! zDs!431@B=^y&+n1Dr0^5Dm^Wu&$;{LYwmx=$tbHW-iYdV-0Y=I&W;bAv&t}*`epQ( z2&lhhCsEHy^Zkopc9@aV+ygUo5D+lqXDxexBU2lTo`jt>CYvobkiCCPhN?~12PejLtJfJ*c2o{>*FH+S(uHCq@~ip_ z+#@Awjz_vfIy!D^2cgx<=>V7DGfUo)N>X6v*ILAVq_T`mLs%2Q5)(vGMt5uu$rVdk zgncG5th}b*cLHJ7;ASTm{eg`Ol1!UUb-?Q>zBAj*iuiePKb`EdveA%Rz^L==9$@Me4$9XXSacujCf< zB|iHLdi^(p8-1yz;lItHpv3(r^%B-Om#Gyw3_<0`84nTID+9&A~RIBz}c z7zGHX(x3K21U+_6gV~OIj$p#^I+&mg_fk_dr zidx&Lz=K1{$QMmrhB!Vl-%@}HvMSH%Ufc2d5o}?h!%(o3i>z5gp$*Ky2%A``6DO^% zdq!{cCMrSwSc08q-O&)hvLTvi0{1}+qNogh`R5&D9R^=5CJ3(gJg4s7;D+j??1DCwqtQ30a`izVS`$7p(q340qHfqgBx zrZ&%&gUI}_ZNqfGSD=qI+HZm8xM1b}x}MVV#rzR3$3U6bK&4et@h7A0Jdsb>U5sSi z_^4!nWRA}2rj2gt_=Kohad_BVVuV|aU*OnE8Hgy8j7ixzB+_m->*4-z-k;x^r>p8u znD+3-^U87)I(Zc!;t(ii)ew5}?G0p~j1IY?s$Mjv*QlOJwA^D_f>Uev z-Uv0Efd>vEJ3lkD>O5oeESI}_!W9T4Dj8 zM&s&T_5^SnI)3sBP>Y$9aJ162P>ChO9#P*+%pO5rL$O{#dqgYEOY{Pyv~W81%(Zso zJ;mK>5Jx5iuFGLxVNO1YzTF?Xh~NGS5+XeykseIXHW|RI?(9382#J&sqm3xsU&#-S z?#^-ti_vrWn`TiblMxVl*TxWa^yg1U%{=iU>~QET?$D}$33GdsWWUS*8trR`VKmlGPL-N6)AYU$T$V0=)6hvIrH^L_Z@ax#xGv*7LnqI&xyHA7v zVJ;AlA3T!}630&@`0Gk&3KIuWD4TO240Add-;(jIPA2l&#r)+(#!R)BK4*$A!W+Z~ zGV%~Qw>LZA)}?Rw{iL7_ji3uV@tRqY`GxI)l4PaW36s}$en<4JNJePg7jUofhUd|6 z9wRBnaBtb15A!F90OW#B6MBW@Z)TCr-%Riek2bWL_`j@k^=y1LtQ|e2i&b8NPpwD4 zr_N(jgy`5mt{96|gNgEKpO1SpCvRmSq6+C&-@jW6`YkFpXtGN(ea-;I*=ZP6A`wfw z{9<AVPMe!8{yRE^)^4|Zp-h1;P2TPgRS z+P_y%6BIM#A3sK!e*94S@5zn+ix2#NgmRr4uRRp)-uRy3S%CL#rNSffx_WEKp{NX@q(R0_ov`3%wwkw#_yjN0Vc z7?7}Lk4k5;F&dV^9k!VpKzN;=xDcmSWr|!qduNFTUT55YA7=okI^DA0Hc6sETa@|q zcZ!LibYXIF>3`nVL!b0f3w#7`{UI3mbN1Qo_Y%0Jw>VBuvBUb&%=a0s>w~^`xJ>jx z?}OTBciZO6?)Q@E%S@C2ElBL1**}08M|_K`h}S2Avcqn8LS{Y^fCM0wCnlnzLRKO_ zI|LOXPaH8$&C^Rv$`L6FV$dr(Rl;5zaTYQSKq?|_l^<(t6&}M3{ZUSXx97?o{v?P(}s)nnG+9z7d zWNZPIUa%A+CAdP9j8jJwQ0bUTlpH+sD;!RPKrp_7;Q&Uc3T{`^LreV3xRB_P$a?u& zEpUchWz4>;POnmYnXooh9H;lNYPq_u^EquLR~!Jg4lch>7t*#i4f1K60U@Bn`h;ka6>WMT5`xD zc}RW}rvVwsse4@@J#}(}!h4WAJA|}l$di@~n0KbCBd~z^AEwD&jdTe^LiBOd=Ajsv zPbF|~4v`sK)v5Z&&Bm^Pn?P?T2(Ygyk+sUWG?rIlsAU}PPtB^vgJry zCTlqvU{Dl>Q_r4IHt-+b+EyROS-)8NXf6xYA zaRmu+v`o?a!TyZT`jQ9h*k~sK^e%#_YpYNOWJ!AjVMhkCyjObDateiqWKT`M3nZ=A zeQac0c^%0onX-4Dx#tuH_E0AuNQwHinX|>-4vv&GxPnQ zyz-arvGCx_e`wyJz{3Y8?1VXuxW5jfkv%bmaBZQS6rhKjn~^3hioi_Lq9LEwUO3h@ zQxxvvt5k@z#{j^<83sD^sr1%D-Fs@=s&| z)3CCp!wt66ReXOLlWoDpS|AL)T2N%Zm#BNJF5}uevt%U0*|H*FIis|VRcG7`h z&A2L~3F~a;c&u?+V*c;>*EXL`9W+ZW?m39&MfT;){H)iBrfkGN3Pe?$$f^%A-HJMgjSFF1E z1ccV!1l&XN-a)%FYav@UlVoA((*!!Dwm~L`^b58^F;AJ$vMy{RYaaMjqcPpkRQDf< zX>$}TDFM=<=gH8mg9)TjFs9M%vCL#-8(=*H@$Ql{&OXV4k*yP z`J)Ymk=LH~??I&Mc2CSvoR&< z+VpDhjmdqmv-=IUhF6_{D!RJG7#FD%2xQ;xWcbU!$SdUFD2Oke>Ll7@$}(x zu>0KLHQZ*5?IM>1G4I5O>#(Qo=}O%^)fuZC&wB8IQllj2a*@1IzkM{tt1OI+>NExQ zB2$nKJgyt7A1qhZ|8cL z#6yACuLnMVYpso|4K_rq;MZjyp{IMH=k@U!%V(7kUN`d)4DS+)@3DSzd+%dQ;5;}V zKu*{=)>e)B+E$pYU;i_QYmSQcYNxWlI3IR;DCs$a>3WRG49SKUD-^yY3fAdJV#4JOlzZo~Gt>L$>qS4e_06?&FTdc8NXPQG!A}38 z7mg{CXvi|FU7tQJXH~t%go`tf=1d3Ut$_SFa!2X}Mad zlQnLg7JJ;K3dsfL4a-k^YMt|y=>}wQZEwGI`;o?_P3y<<;y%RLAK21U`jU`^!wmTb zlJ-+&a~3ki@=f`sNdnxm-+N$98L$k2P4@E2D+0}lHj~z!kcH;I;zZXI1b(4%}OI8H<#3batSD%znhPmo9hEIV@3b1l~ zU1mi4n||u2<8?;*O(&aZD@zn^P?v5nCp0eXfXb{8pk#RuQtNF(StP+Xj30@YJw(f4^QHIuA59}o$ChLE1^7<|n<5rb z5nx#n-A~3Fr=uLw*!@ecMMs-n@8D5f0z{BvB^57J0T#z)kn9{d<%FUxj}i+|X_ngI z)77_dSJg1d?{fMUkd2org~=zEE2+ojWlSx~+et3xN|4Eo$5vxIckg0{%Pe3fe~JDP zX5Zzda9l{cW3dY}Yn~bS%M9o#7Bqv7w`VeZCI(Q}GjMDj=qZ6W%`L+S&FPmncnFfm z!#eYv`3TW4V{&y~9I59V>&_oI%RG_AH5t$7GAhN4rDZ|?Alkj?!$_2!cVCuW zi6vxZ{jAs>l`r6D7tMHW_fW{v5duX`qtb@+9K6R&sT0Q!6{f_cBr(=~vjv%vajY+B zd@}L^7zku1`^$|la9>lAC=?JC4i;2_({1x#2Fv{}8ZZI6I*I428XD78758fES(R?u zZ~58}^EMK0NQ1#8xBcqUVDx4arkV9Ar)Srp^V18=(@wIP!+8kLYuF?scl$M1bYw2N zEaxTf9g2?p--JE>vPn1FLaT6Cdck!L*o--<$W{130YK1l+(kw)W3$D5btmxo@_vTk8M!^){)-I_dzlQO6+D9rD>ZG~ZlCothKiaKmhUC4&t3n^)1X;{Hp z-ZS4KY*PASI2qLRvifEmZdVFBQ}0Vt$4pPqFB#VQ1VvYzVx=;{CN=@eepr^`_y@95 z_U&MK@M`(P7MOJP1Xp!`Qe`N&j$AQ9ZqwXXqK#lPp7TjsRa}81at2lDz`xWXBt~al z<^d|eeWnZrbURs=78RimtN#E^o3rs*e;A#V7|Jz-PR5;(bm_g@vCvk%T&wMz_=@BF zrj(Q*Ny$tbW{}YPTwP|)9)*7)E_#pp&cRFNs@I|wQ*H$FhLtmt;?TlNC##?R0K!Yj zC!+We-Al{I_j<(2eTW@>`%z~2*$e%p%Z@MWQDvWVJJEg1jn!wzY7o)e#(g%cX_{f5 zxBuaulVgL2tHHgt^PonV<*xPZ*dh$?%^)hP|HXFPX?h85_JESCioxvTx>RQ8m6u(e z)X5arH_i7Z6r*K$v_t4OQbgMxj+=OExA|&`@{S%dC2YbwA#Yf?6^qKkSpxu;ran6( z&5QEa@KlFyBQE|DQgbJ3G=>rHVc&?Z%FY#K~#%FEy+c`dM23XH{k^3j$W_ z-vYo@uj)&l+U&1hU96w2ii*!@3XVU!qU>T$c&vNbIjh`-GuO@+OYLgEI18J$br7Zj zW!Vx}M+(yQm=6u2kods?(z*nlqTY$xJ`yC|V}m_bTTB|5Ct<3rlU!B`eBmiIRy=~ztr}wjmmfNhfD;0$+zZe_8^x-P$862X^ zS(&<2{WbmnZE}FC{Ei@utZsDhsw7*mj4FN1Wo-CLVfOeJKijv;6t0XPUU-Y|(Db%d zd{;9??EHSe{{JzZnu>p`Q~yI_cK>_+Z}EkWCe{Yd{}_b^*8fE*bfOn9GIFtT`OkY- z6J-~BYZE~iQ`7${jdYRkl>}o%%3NqFZ?tUuN$>L;$OyFs@Ph~xtPKwPX*ciOmW~ro zmY#4~?QsM4LFwQMAuy|Lx;37jJOTW6_Xhh5dqOcIwWHpXRmc>i-` z^LK3$-x2=J-oN)1d?N8%3R`e`pDHjo1s4FGfk57Wio;H6G$Set>H#+utZyc(%^ONImI`9Ekor|!(cXwAm9 z?R>HAif!ArovPTjZQDu3wry8zch2b3eY(f!yB_Bc?3;bFp7&X6PT^)VJawA8ljPbz z52;HaODt=}lMZJ(^yOsem7k*}>kVWZ7+U%1`zc)yDr?-4W+}gYtK+_TDe}}AjAl!m zS~L=2K;GDjv$H090&QTCX8)YCX5taEVz*KcHc$O*FTrtAnpLd(`}UkQ6aYhR;xBK_ zQT*3y+<$EC_(lvrVg79G$Wa0Tnf-sh#{Iwknu5Hry^W2jv5TpRxRbrB!+&D9G-0z; zmOQ_I-*-BjS7>{)&~1RPbh=Yv!t4uxS3@^Hb5+>Cjd&=w5sXK zEp3cJ#4341)r~A<^4CkmHi~33u@;t#Y-Nn2WHUXbM4QgD#9U*(L2HAb;vrav2dy1y?vD=VDfG{8TYMVWY!!U0-BEHze zrd}HSdtr^B=&6HK(R_>H7*hAp@o%36KR|o5LTZ!u2p?Z_$8R2)KN|agL+|6hl*D|O zH2A8%_o&8bdQ^k3y|noMhBt!wk}d!8=9;_U-sM4mq5mob`40b{KlX;$o^{{hxY-eu z^IiIj7*IBl$^8@>v90t#Dr!&jMkp$v_JAt-U3pp>@g+YugZ9~5kW151bjlq6M?xfC zJYGFseyCHzb5I_e{8#x9WhyfjL!yBK^&o8`u^=|}KmfCjn1!+f??v#k}qwJiE8S;=W;boY1y3IJoKWeY;HmdmZvepDOk67K@D7VGyhJNo$k0QTV zbV;&=$0t9Y$-eM5{Ohn`E)#imZk!e)S+&&c3i;tGkB*+HOY4J$0&){hUfa?FHJie0 zDd*#m;Uf@DQmpy7uu>K^-p08IQ(J*MfxXD-CmUw7+NS<=kqT+XW{tA2o}1j3fdb0i&s-O(eP7cK0#%bsR-S>^PV_zL&M)~k*QgbNVFv69Q9Cp#M4iNwU} zBqK*wf+x28yU%>kp}A2T*Bq1Y3X{kE|c~ez#=Ug zBQ<%(u9TsXB10|Ji^5E9jY(Qxt6q<#wV9pFi0Rx+T&);ojZ8JEJ0M-XG%W9pTGdV^ zo3ET~i~|Nk=csBQzC@&Jj8`DUeKWnv6x zg_W8hkY=u==mC!Ya8cu!2}!af2TPC$b}r5Vob2Ytc7)4Y_5n(f}KVKa>&Q&ziN8UIYIbfE-1W;M-m zEA$%W5c}1&h-lpovEVfvm8Se80jbXPV!qV?H1|qsNcllBDPlJZOJ|S9zlb& z^$qZm(y%&pNQ7~t`F9r0GM+D}-P*pxWOZ8`e-?pG?r2H>(|8#MP+_EW##%*9I4!>k ziGNXeXFPK}1fRD_EqO|bl3Hqpu()5J2-=41Ke13^7MXM^juS z`mk)~VjM!2Vlk`s=#1*aLl;&f`6~rtz{2x2V=)4guvw-OgNnx0;Uhn)H!^8k?7B=% zrp>s{ruk@eNmppo4I=@QDFMITxa??}Q{yBit@jCTkkmL`32j_(39ZoNAb|~sNj+8T zIEwJL^|-CaXi~A3acj~tkBsFnDI^>h*oJZ6XV7-4iQ6hFnniqq{uErref7~v@N@FZq3@Vtt9vc#W{d7`vivQ&(n`^3suKL(xnRqZ-fA|Y3M&z zM=bdaqcM8n{t^+9Mo6|yd^LR1Ny%7n^Y){|)c^KHr%{IH3X3WV@EOG57YHG=4P8`Z^ zfXs;68I5#W*zkPc3(#})DHr=dCg1Ul0GzId-wX$<1T@(Ke_r;dtFQCRPAa8{dYzik zvJG|ns8d86va@xkQ&VpKil<#MaF8>Ldgd6C^d<2&{8G8B&kh_-sQ~Vf9^|XNx-*pq zUw~L?p)uxJf}s$pHAu=aBuA;~N3PlE<3McVk{-42B)RZIyf{j{)>awZx}-yX&KvMG zr8S%*6i2Ib?BFy|R_&cmpcg>()ZbZ{+};yuZr!jH^~74+?t>KL^Pc+iRuEBON)aHZCBIcvbDm3RsiUwNyJTDl-r(@45j zcbS4c7jt|?z&BAX`OivX&>b^7SVxOD^}1$dWiG~Me5^Y1&8$RlBz{}9Sm{$Bzb#hx zX-ktP#qM7ig_2ufa!aJml59*%2B!j}k;@g8G_}Q~*hwz5s)yA#m>R2aZ(Cym)yF=+ zWt?Pp?v1bN^vdzAI+mzAB9Ca@Rt}cQ8LlKHemBQA@*l)6Mp8a!EiP4yhHBq8e6u&A zhKJX0Fl7hG7GNl&EQ~dzf4hm0JWRkCZns?jc*S2PQ7P)>XNIzXVhfq^{`wXbEN)*bX$xCz^EN|+Dq52_vxlx9ezf^5qWvs)Cl)4O+27Wf*F>KaDhAe!V92KoTFJ5fdR8I>dcx{n?J3z{5z%dL{~PXCcb#i8%Usd z;U!Y-ESo10{Dc9edjfUo8vLdEbPIL_(;eiVjb2th-ABv*<@Ul==}ra8mj&8JNJw71 zzWo~)(Jtl>|SA>AZ6fY9C=-(p`718zLZIdP~4gYu;ZS+!J(F^ z&||NIa1id4=1?Tw@kf~NpG3+^{ZO*Z4H1D;aha=5v~!m{=G3M(VJz|ZKnbY5N z!~rfRC$AWL0a-Vy}x)X zzcZ}6|J`KwnJ`%SzD@hb9Apn9(dlu2Es{on6i8$g2TL2>Iig2^7Ee9Ik@RekZwizRY;SA3Vaw-M0_MeX(aO=xGEDq`G5OnJ!S3Th>=?!2u#+SHQ=~;z_!Bs+{#@V>4_(m=E7EY02 zEQt2tz6H}t=aPi)iI3JBhTLJQaV#Zh51+a4MsMNPc{yDyO>&ls6n065;W-aa?9bz1 zD;~iYgFsFky|>GkJ#TLNj(w&df1$UmKnjc1qAmwxomJGz{_7xZ7VkJHP+=wrK&BlH zV-Iko9jxQY7HrLlw;>>1LmAFrisG}z^37-qwLh_IYRe>fF$S~-%2>uS0ek}XULEm< zbN9f13bGr^^vIb}Uc40T+oH{o(jsJeY1Eu}cn>x9gE9?H0CHBGTvig9oFt~)1aMX| z;(nG7R&HwOO%`v!{T(zfgZ!@a9V`8)Wfu|VZ9!KQ!$j*%LRU=fDB?|nPCpF+L(e(Y z$OFm39LQVWnSq!56Q-^st|&rJxn0{C&5u=Q z0)bl}P=Uv;@LbRBesYQc-aq+GLvU8u=3O3fOlL$Z(gzc8AJDE(iVGZN&$-Mkc$mEl zC#9@~c+yy$|1^cIZ9?VgrJ{w07#JqduKLkHk_<@shxmmLLCoa%P-FbBJ~T z20W-nK%fgWWtzs2VA$YNqC{lG_6V>KH2iQmLwd@*aMOmXs#@R; zXzd*T9SnGE2F)-NzH4@P+@jG{kPrLkSp5E5-^F=qY5qw&Jr;X0wE>C8Bjt~OkNv?A zmfP28H(c)SoUDIuuGo>_2!1xclUNh8Ytie%_C;1hBc;w9PtRl-=Te+p$jl#dU`JKI zie992KO&k$I1nPt_>fQOII##)POY2axg=M2O}6#d5uJC2Ko+U8!dlKqjK8yvWX4zc8gvibw)g+HOs;0HMu}b+Dxe%moaHBPs+@&N2So8$FO*jdw+`b z;uY-<%{>lLRVR00Q(rG?SjW73iKx;Gu3?<6*;`%S+YO^?8(V#-aUhl1D%&~Qt+Es7 zXo>Blo-342{Czzlymsa5i0|L{EtgM^{Hton$ql%>>P6OE)yCGO4<4?Z@r7d&p=(NS zD?$3mJDtMD54iXP{Qr*Tn`cm<)v!Q7nYche=KoF6+}6m_&eTc4(Ae70-1I+hY&4)f ze%!8K_GX?eiK*~y+-#1dh3B1ur$j=EY5?FZ=PD734G$sZCsag0hY_dFK-ZsOc)!++Kf!@Jub_>zQb`_dG|M<)KB zjfWpc+Q>zjDU9Ma8a!~Cx&XX}7eAF3P)IK`zyuj$@?8fTyW0ku>$n>Pn(MmjjB=qk z_I>Y^Jq7*}5R~m=h)zHgf~l`^(}qgGo4+gVbTlF71FrA7YfbNizvHl*N$&%@qjrOj znX;P}aNSV{x1)UHiy8l$$o-=ZOQ1j2^u8z{w@o3k6zxN$ZwBq73+wSklnLiXI{cMp zjrX#@_7=DOMjO1>*^n-VYmJvQbEoqL@S(Cb;xLCHcUo>xfn|yTOBj@ z4^Jo?BgqDoC=bn8eLtj-Z85qwxqFHz4gKj53(8Yi3+O41fIfKgZtcXmx|}uU*4-2v zJi7)-dPo0)7<>?`*zFawL1Absp|r#WjDb^0F>pZ3zj-(79upF@C)>q>%`+QBa8fN2 z>;>>?f^Mzcj1V^J1w6@9K|VEgpv7O*Qe^2VzgS^cFNEV23Iy{O5@~!&kFV!Qf%RK4 z7^BMMcM9JVmE`z@GLefr83^vI_b{sEV_%KDG!i4VJd{;pEf?9YtTz_LP_z_kFIMO3 z3#~H$G><`@DzTwh$UeE9*H>$qr^Pj31QxYey3n3j=0Q1%7x0Muh_eOT0EMlIZS{u4 z*i>@_Vp1cPtYGNpthV;@Tsik)%hI}~9u*BAOF3aow5E0c5STc@=zVOr!xN~7POq#F z3^K_q2hWRy2krdnY6UETSJwIb_^d*j1A3haR!}ideW&8-1Gk?J2V2lY&#?8}LX5E& z*V_Aoy$bh!+(&d|T*CVj3$XuHUBXY-!~y_X39$4v{C{ zlR)zmquh|J)x2XN91D?pVbow zcNGObkp%J~sE5alg{*nk$Z0yi6w_y*HM=p9mk z&njc2)zmYvCFc!tFHRHkb2*K}%&G+Zk2TMs3#G|Jq^+In(I%6%wXl*;N1o`u^-W=a zyO_oh)h#Lt_rX7uhk6~l129#)f>Kqp_FtpuiB%NIXe;VqGPBlI^PxgXvMLs6HX-;oaoY8hc>ENq|>UR)oHZ_p!P~4+0^Gl)T*pu z_h2~s-t-mwKJyrrLn3-p9N@#b`qmKl&IwDz>^#GPcD%Y3M97iNN9 zJW5#M!lJL4%3?;3(3&^;j26wzE=K&>Nc=sob{nm#2h=0@gYY7&6YZrfA3^oXlZ~|8 zt0O=d@}82KG7m|-n2NVxMow+*t*DX~k*1`VuUL1Ki?%snJ+VvP#4p-LTi(CJ!*h*g z_jfInmho@ZKL$3Z$j5lq(m5V0RC*jo97S1xVdhhQMnB&kRi;h>H!8{C*jk+YaT?ZX z+T=@!ET_yPgR74iY32IRGUC0lyE>MZ8jCl2j;}XcBq>rVD37Xo<~`gBVb5Syx)hTs zbqdA*!E4EwL1`_m-|ZY+w;q1r6n{Pqb0dpd)a!AXq>$C?aLQjD7lp;tEg5-X ztP+QRm#}KXEx2ug%Dh=>Lf#ZqM5Cn1pVZmW>~UT(Gnp_*a%}Rj!Daj+`B3mmXW^Qq zCKKH1-83OiKd$xXE0gUCG{8vP%Vbf#XI`Cvo4Pl)_WkfLoJT5H=h=`c zAiT5~I*ZW1YSR%8#CGwmMlrf~(*{pnVCmLfkeQ$4XF2LJR~M`!eqSD<#~OaZmqKy2 z97HXD5nlPO^W33~ydQxg5C)i`ti4!)Hjr1j6knY2{1-0p|5zgpeJXT=B|jJkw7xL{ zo(dPOFGt8~9}V41QXE*0&IfwLdD6r19id#Pe8NDe;sNFh0W6Z~b9z(XK;k){uf-GQ!2N zEPiZKrm0>O^+27=jMu3@mI& zZ%9uLNG>sKG*wOyn8N<669eIJ{1w&Xmbg!^ti~qFN0%CxDrMxFsfCuVv{rf6;7Pjd zl&%n{7n^}LuB?GQ^>0-c7W&`b>Ud9VFULUy>r!idYhCVT_0`CSRmgZip@v2f>W$3< zJ$FdjN3RzqXjzj2v?OF(BOWBPDbu(fPUG8Hp+OXE>xL=`}}xYB`bO(zjqu0}Ug zvooenMWHuXphsc8q$dWATsC*r11X9**;aE32*du+WW0cR2WYwjen?nDNSt;{AjovX z9Ix*Vxl^w6il~ny!9`&~(gSj+8yNOxnl|1SJKon!?6x=>$4KAa>+N#WSWBww*HHE# zw`$Wk+@*M30UOV(xjBfh5xjp^GGX7i8_Qzu;lF={oZ`FeI_WsMG!_TzTGqj?K&})D z`qqtAP1Yw%`q?|#%1iUl!q*(!_~BW{vj}#8-8dfl38zsoVn-g zL;PqfX+>(S*Q%6#pzm_JzDUu4OJLR|Ejk)oJM9cr&)_B7O2>^9f*~B(r}eP!2c+fs zstYSy^cEPkKac3_W>2`(Hl)W};pH|v18a1Y!__i%&xTCzM{W2g2Lm!oJZW|A%(~YF zz}IG@TY`XV1jCUYgD!J=Nd_SJDN?I6bpcnM z@za}<<0vI@(i~f_^fh`Sr=5X{!9L`HIBTNf}iR8B`!*ivSkFq6&`u*PRS|YEx zg&3{OL~{E&=Z>gne343BgoL}>dyB6`zX=Oqm2cYvx@DG)v>eZYGUN?}?c@Y9k{5=k zo{B5e7g|dj?Nzx;e^_>?Z3aMekWNsvGg6l3Vty3W-Xmp6Zgn?X=_!@to6t#cMhC=A z2u2IUtOSgeV-nqE6oINiK8q$EE4}|m1^XXm9t6ZMM1!Bi0Kf+V(*18q3?inst{(q0 zCp7=BoB&9d1Ta`;2uW`=p;<^XOLEgyD`X&L6jke=vWCOU+E9&NOj^I4nwX^(lTn6g zU^?`HC#eL(Sm`Ki%**|O!5A7Rfg+)ViJ;622!o~0tN!-9Hf7etxr=ArN$u=z%ypjO zawp(*THymqDJ$1o+>g|wP2Q7k8;i`W@BCw~|Fnz2u-T@{t{=Zw$?PK=!jIiUKfuBJ zQi3_ezS!EO5x&9u(t=ssrV8~_TI@`X5ac(hv%lXtcK*G3-Upg@N=UK)6gotBbub6U zIG`8gKRr74Zs_O)f^QQKx%PIDkh@CWg9P~&j{oG>{FJc3OW7M?-ehngyxA2bzE>dj zBgqT}e?#8;R2KAXLV`VdVPZ%*Fn79Z`}MsUNO5>5`T7CN^;!%m4?Uspd1K9R6@);{B*xGU;W1tsj-ZPft zw-;$V423nCju`7O6W*mtTBLnqQ@;kN z_lpC^ZVCfHkbz+48unabd0YN07mmzj5Lhg<`|nC;K*Qoj97?hk$$LV zY?V6Ji86Sz1alFGl?-GS%~AzLejS`f8jO~<7sw)g>xV^^CYnQbG2w^#B%oK;(vB9a z+xtg4LtzaOD(TK?#-nYnNm)CVv*&2kd3L)X*H=ypsq^iI@lz>RW!Pq2!@ZY7^_ytr z1%M(O#WbaWcNUf=L`#4cR2!C3SI2+=l?^DObSxMd?#!+Y~UdA?onY>?yqg)VVU6DJ^%wbvx`5#M?(Jm>t2~vN_OZl z$#BRclMezH(XUVeOI}oKDh(Mx9DG4h@))+8OD5|ypw{g=1V20i4TlbqmG90LQV2Z& zp`@YAFFC1W6Fn*_50D&<*g;?!6CK^XeL6U4ReS?pS$2@fw_4;i6Z@<-Gk2!OOPQZV z=162lrJ89HrIW%M`)Q7HwRcZo2p;2uim~+*8u`j5J{^Yw|u&f{0 z3}b5(tbM`vtRLXod>*0$`rr?$AF=~Gw}|lHXv?+XWGnl|IA7!>U)pNleHS>}e=}iN zVzGH~`2FRrUKsfc_Ig*TymYG#=|FJ4C^loEnhTpUm8e6jPYwYTvh5DR;z!gnR#4W| z^6hL#$f(uAe?FI9NHvvTD}D*1PA?>3s#Ch)l@cf_2TK*gs>YW@ETKJZC+do^>A>hU zhlH0>K=e@W74sdVyKtICwDdO^JK#^-u{~>AkOyKGnNE<4Fy6)K_aBF=R@HAKBs4})S^2ImqPnHz z{qv#~JBL|1BE`*%=ZjBND+(c>oPei6zw6Ug+hzsWVB#tVzaE_FpQW0({Nl!Z$K)?; z6m9HQC%sIiA-3<{bh>!lk3MrGO=#MPC#hI$Xy(UjImhn|h>I`x3ydA|x82-T=ABPD zjLfRf!?8$iQx29EcLG!@R8&@uNI8??FCGvlLU0^mhISE2cXo-B`dDy`cl5cK6s7BA zliY+0M<)|&6B(ME@)S`~60GB`$KVYxOFI%-YnR|KlQy?sBkZr z>uB{>EK;O(A4aox(M&^ecXwH$h0a0-Ys!Z7Z!&pDy5%o0;^1cmXwib_8DgjzvaP~7 zrUttA)4V2H*Q%gc>zPlfOWQUZXb401ZfmvP<{m9-weqk;RgKHm~&0()BnI_(Lk%>{)fdhY|^4H2Yw!N>K|{FXX!s{2b6* znHDNu5I0ZIS>4;TAupOVN4q%Db^uNy>Bd-GoGb7;YG^wM$+oiXC7)4jC|p(>-u_Br z%G*eb(Z(o-9uPZC9&P@Tt$2?=xG&!80BwhiJK=$P9`^GDhiG`RM6<++1;A zeSDP&je3e2e2N;qcqP`1#u#4pIVWsNXTrG*{#Euxd?;>%&*ij(8>v4TRdL(&QCHx1 zYW~LU8(>2F?|vX<@%)0sdl0J}ZT56uE8(lL=iZ>k8|K7E8qKp}I#NGe{&ABD4v5-YwiKteh#Of$nu468$%!f{)n7WfV#x=e zTc=IRBd0NI!PuH|XLU-lgcbOoFroriC#w}3^ankobE-a8j+H56$-@QCLAIJWU&0z$ z!ze4+oTQ=cc9xB8uZ$_GrPWwehe^XcUXB?=DVLLqQFF}VhuRjGh0X5{W_gIxK1*ml z>rb+I^BC51^>MtL9`5<8{?5<%o^%t4>I)sajkKXNb(TKC*@NP%V;J+^EWq2&)=0WqOUhqKhKQK@Ezo-&nA zxh;c#h=Rc!QCrxF#)0Y5QQ;cF5>S8gua*~fIl6+;BxGcK%=F7~;d52m32c| z+6H87h&qpswDZ6?P;95gcKUdrZfC`I`nExa`nF*{hab~1Z=VZ8Uordz^VmvRPuih4NJEUg^<#SZbLz`TNH{)1YP1pR`vS$E z5qt2;Njx0;rVtt%E^DKfuRr`Jst-i&CljEQirlBIve3Hy)?v*>a`~}ZG)2irTw`PB zCfr&=52cu$TNNp@&~1fXTG%BJHA|$;Sq#!@N3UgDk>wu|zse^tO2-bNwlouC$%vpL zFD(o4)=3K)O0?SWVKm_`)&&JuhhQU=IkK%*gYSEBA}QIj%?^f4Jfb*}K)9{ema6;X|=#VN-5~PqYTd_u?#BhgMFk=tsHUj?hay0PUm^F;6(`&U(_(L=9RQ*Tvp%78ZJscUsiXO8ntB=Ldz>t zjkUB5w@5X!Q|~0nqOxQQ08=CH!s|?IcWbqiY@<&!{DB0O4hi|&v6*1q{#mXB9~%y{ zWD4k`+wx73v#PerHG`0e{_Ct59VKB7zTV2-4v}Xz&ksETC`Bb^Kj58{DV%L`36miw zpX^ph?0E@e_hx>I5&TG4v~m|G-RBbPIN(}0TNg!cd6HMds`q>(<1lxw5T3dGcVFPL zL}rTY?c1@0#1!z3_4lE4oEiYZRkdcT#W0|OFHA{CR9Qmy#y%w_Vnd-nNX9CQNY^e6 z4{MH@|8G|&dcYrHqsWb9v`lFY3KB4R9*S&WK_WA z!>3C%0;wTHvuuaNfM_AkGBg!hlR;xZWxb#vD2{2Csslo9;;kpR{4Oj^MxElnHS!E= z(MIh)%v*A7eYG2OpJYGTZC62J(J(;;4yV?YH8@CYedQZhpFm)g`KO8pN$ee!8#-R9 zgRTJk10I+kfrMlmiM9>vhzHLCuWaE_0MhO|vtQnBsE3r$tJnhTBn+RLp_Nrwy<+_e zYXSOLhwyML+6R_^;*Ii$O#g@Dfc)?b`Ugm_N*(kH!V#o|;cnnao4H#p$|(ib_Lvc|{YW*mWCTqQO-Spyw24c*BW8Ze|b)UwjS$qw&9 zY@*6A?jQfb{$R1kGLiYR@n)FsPQ7pG{bM-<9CJgBM%Wl764Q=F?MR#}H= z(>ef|4AD|5(e?{M;d#dy8i@AT>BMSkk#j?(Bim6br}HpQmYVO`p(gMr@(RA?0ip7)RNO(Rh(LqHA|3%DUqz!AT`+o`%rP zsZq_M!_ZQ!EG1j3NnfK{ywl}#=4y7u%HixXThlIt$_L6c%ZB$q*uoSX`bFl*nOn6L z5$c&gW8L_{26_wRdI^_Lw3)xS)|KIJ&)O~eGZJX;T)oNMC-K93CPPgShVtNyS475+ zlWJHjYv7S}o5g(kDu)k(6of8|@W#8<28Xg$xHQ#MVoMF3G}kIegbFr}7o(l2aT_|U z?2H1HNPqc#uiva_gjcGj_s>;zKO<{JTma8V&{u2t3OL27&o*I9= zs!V_510gQokIavsHFoSj*HMEFMTs& zy%1XyfEWI%Tu4DHXm-(~(-XS8Eu>ywdXWbY{+?SsdjF#Xf9Q@A3}Lv>(k z1G-X+(YTtJL68l)BsbOCxN<{sX9IN4N=IIoSCah{(j7(cSO{DXJmwKi+)WX&;uX=J zV|?3myhUr|Abb|`OY&=Fr~)Gu5qpC!Fd;Z~!7tcTaNFq&VF0;8It>ybmq~(&+Ch{g z-E%yeY%R;e{;WjAg`=g!DkvLtmP_P??8)cn4R4Te(sZk+l{Z?jd43(8A_P-m^%L#! zO3Eaj$Xp=hQ?AG@k(pwuG36a%p-HJbR3lf_nRxj#R1L;XSK;O=qWxHwa@vL9lp|C6 z&nFN9jc0hrL>&#bvAcHBj=guMbDvINir(sX@u0(gP9y<_K(MO}nD?c13Q;HFOf$K{ z;isOE3Q{PMG4Y_GTt=hUw`h2LU}#{!0AMhm<^WrDDR25B@Xv>R)rVp+p^o%CH>A7^ z=s5zgxv5@C7q$F%tTg1!>4jTce;))FVthF*jDFK;tqBsOj zJYh(zBdyb=GM)t9$R%7IYcogcC$*tI31aYE{0LRp!~5C7WqDqy@J5nju%eAg@aSD1 zWGUf$=mq66)2i?=jHz%1Zyq5mcrwl*Y*gl9QLcqqG8{j~VO&uP$Ghn&idQQqc{`_O zS~*x`c382%!a>K`PHenqMot74&MVnM&8&hk_kc|J<<9z(Y5)>jVUo}Ezt5tFUs@KP z%1<}_MWW)DhW8`t(bBGvzTys51Y(!dhKrYI8qcH-DC!;Eg0zJFk(?TRJLAMU$(Dae zEd$g$ZN$^IG`jpJy>!ihF=u(LZuVuECl*EzP01HH@J;qzj>yto>GQz9&RI0r!cwzF zrKrQr1z~1Dx+2nFg?RN3vV1NZ|7!~T4|WQ51)k&=1`yC+4j>@C|CT9GOi9Y#@;_;- zLt4;2DoaZO<}Doe$EM~apb?<6@6qxhD;i%#GaB60G5SxXcTSr0$cPV zS8yvDrHw@^IwHw1$IF6h+sotmr4~Cn-R2)vz0bGvjx@7oSh6yE9O7d%4QU1!&v8Q8;mg^?4I^zPLe z=@(wltXjw7INgfGKFrZVDS{>DO0fe|ZrOvQdFOmyH4B}Kqhy@gc}qa;GL`nSiF5O? zwF~tD%aLRJv}>2zL8L472_g(l>ZuF9e#NaRx9Vutluvt8zn&h&z#}(D-Pw^C1IL(q z;|+E^Hu9?u11!4)4Kg{%vSba}lqWiQ4#tr9kVQSaj_y6X;#}F^;EA zf~qa^t<>JzfZzNp>0O%?6oqM%=t<=(PIT>0=Hb>IUb*N>-(c(M8{xPUcda;ym3$sy z>ybRk^o&H>sJNx$u0FBV`wFu8klfo>6WAGUg&pG&k=!uJfzDCW+U3j_YcE7<)b5b^AMv19mPb`<< zZK-fVMrAM^*TRY|p5N2(a^+mN>KJ>AP0yP1KvgcI&at6}r|5s2GPCsUDh6+qW|~I} zXG>Na$~4(mDxIOVRBI7#W6|5niWd~m2ZQ&lODNN3 z*~w#XEcT{p4toX;8nD;W4>VGB49B#fL6}fP>O$ZXrkUU z14yIplB#xWUZK_;=R|+^-9t3mnrioh1vYLcgFZ$#%NE0~&zH#dX*PpIUNHFSe~?dE zXh00inL@K*%Z()Nx@oH7p{k|rnV@)Qkkl2WhtY?N1aDCdxJ5OQ6%@yNl*^RKrad;~ z=_Pe|D1*RGK93Z%Q^m6?8!>~@KBZ6Ye=+?#vk2mj4dzT4OpFvn)IKGFl#!1la$=ZP zP;uq-zik%o2*H*Y=ESGhUCRi|aNe1kW-if!nUNzqcywCVNW?;k@})~G&2I)zOasI; z5D-}_R+H~}WtGEA@!r{ZBVe>En|Yy2oQNo?ni+d(TCdB{M~l}rVpvCuVFm%upo)<- zK_S8ghM4P-VMyjqH5spmnUMlfU7|jLn{8~4=R5eeUb}OQj5imKn!Q;8R{A|aZ751% zFccTo8>VGy;A$faDD;U9U9tRNcic*8w($JlLXr7!Qd)!#Ve4-pO3IBbWr71^A>B;~Z|@4v!QMCZ30KB5q`zq1As;a8D>zUa z#3NpXujkTY-nhO50!F-{F~xzhjnF3|W5^j{ZB5D}|zLtVdO`^!tmq6GJ3*o>Q^ z&WyoRSqN;)i9 z{LtF;o_oW-V!RxdZvnvySUgA8a1 z$B^Uim^h3|l*I-dc?sAqJ+GqURluUEam;A0qhSpVn&X1t1~oiM7a|JfwGQ5qxsCTQ zR_ELJIhe43gUgpiNOELKE3^yGpg!bGY3HcxC%J67m9h^)5J+1z4{&spquZf{pb=4} zN-(|kOSKG3>g@HIs}g=mVuzIcq2Gl797kj@b#2a(wV7zSxU?*zbMeIRaD)vw+b1GB4u(XCX_7W49E zP|`;;tLw(oHDktSQXWrTL@XzK=R3cXK;4wbVJD^U(UTIl*N+}x^@Kx@;la0Ll5gEH zt&5O=x$1$i#85WS3h`vnZr!eupFxM6Cfu&Jif{;g9AX{Q3-7L?2L~n^glnaA;U;^` z;v6T7?k&3o9eweKR#n_sZn*z!k7Npt_=fNf97in=p)rO(29yNAVXF!W%a*hq{~MN& zVOGs5Wp^TvIS5|cpvkfz0hP$GRE07XRjLC)mbT@lEoZP$A3DHY)cCzoP(V|>hBD8@ zDPa10k{}{uPbYeoJncsKM*59`3@BXcp_utyXMy@1r2-w)^>J*WB(}H#W7M>Y+jPB? zAz?~PT0KMzXryF8teT(sL>Tq{ywEMWbY)mO@v#KFpsL5otYma7@BLLUy4U1J_`@{t z`9?4Bs(1@xOK1abT}T_gLsiRtUKir};8sIG0vKXTHdoBhsmgXWZ{rL>=1w< zEf3-ImqjS4so}`qT|BI)iW^Ub) zQ)1WnlIy)3NoNc4#~-eo*&c?ODJ2<+dz?%%1)gr@P@}L(B>fMBFozWz2o$n-CTS? zECf8~s$zdwe?|B4FT?XwhQ87D8s1!|4SD^py(NVEBK;1xe5@ZcPIDT{3>`ENDFU_f z|Jq)Bp$q(;{4FTO==D`_y8-c?{0*o?!2WRmf?QDSKAL^Wh4_yAw!QfcR_zJe)_Tvs z<_Xf&&WJEQ$@&0c(E>0FQ3Gs%<(+w0WMk=+G1w1J=bKf*7Y`X?f5b?9)CICa;v$5e z{vq-UUD)eCGfj-^NKsOmCyES$m3AzESv|T(Xx+QvWS!b$!{gV?f+Ree#rF#w32*h9 zHhgf*#;$}H$B5;?TLY_Y8Ho=pxVK2

8fi-LR6&_%V1(HV$885MWjyV*7 z`<5FYHAHRC{_(*O8z@(ifUf|Oy*FnAfj4<9}O9H61|L0 zS6{{6ATv=d1fL{7ZW5==iM zW6m@;xRmQu#}g+9>SD?3N8p$TeHHiDb0l1CbU4f&X%BaceckAB`Vh@lr0X(0;^-tj zr&4Sum5GwCYf?X=l~Sn>C5+G`VPtUzI_Pojd+l%!ek`d)N0@j3H;BZw|9yL$AhFl) zK#O%49tX$vrO8aZr5Qw})Rn~#Q0mxW!JFMJFPzS&hz zWyi2fr{}_GPW#;O|Ca(EJsB(3PbD2e8PgvQHp-G5WuiaO@$cg=wH)Nbl*0C{_*?r( zhEblJb(~pr$0_t*A+e4#$L6vLEqm_mk^Jr1_4#@AaO|=ptwg3p`?KYwk38$oo8obu z67m7!r)@iB-sQc#YzJ<5E%r~z!foPe0=#(f7f~F$K^XwJG3hoal#x+QS$O{H^Rb;pr74r^o*nMQ)1ILvyA&v zL-2iNTsp#0r#*~XDm;(Z_wFdJ7BMJ7RnA#a{85VI99g4tXp%cspV@eaWs1_51PuN| zEoPEl!+a1ZyJU0)FWmyTcsw4K>}qZuhv~MnSlvF; z`s3|FO3ZO5Z+wy~KE*Sgb^aTvUIs8CyqPgpkAdj0t`6|zOphcpF2-;nmFq}4$LL!X z&rW||z5VX`IBxi(;Yy*8lgnRlbFeVQg%oA6XZFw6MDcv^mK%SW@CqBd8wAo%Qk)E# z%<}LsCrgm6UmPe4;>pcl?qyHeE~#Gj zrTFL35h~r+CGyWR-*8`7`MMRp441?2UhPndJGqVT0`|Q;r?4KF2A7`?qE7RmB0i|? zqp{m`qAW2CX8?NcUkK`$Ts&i{XfCb^os@a538j>$S7gq^Kxi zg1DJw_!(`$4ZZO_VARkTSr16QnxC-cbWNd8LKySwkW-v07lx<(QU~TFRO^uow3)TYis9#Mml+Suw0^sO(7Xr5J zsC_}e8bE6D)HbBoe0j6f_Tarh``gV^n`3$0l`*qpV(=y(jqI>vp8a9;XE7h#=BoDj zo#`F%^H{3PM)p9k+}nRnr{P?tu8O!w+O>ghE>&!BU%lGk`dcxb#=r+*fo?Sayx9El zX8NlO4q6G`W)EC-4dl`ay2=B5?hSN012X>(bgXBwmznp(%=%<)aV~p$j&$jd^4yU7 z$H1MJMG-}|9!pgw%3}okI|RTD_#_CAJjvRJm2Y7g!%Jx z0~86Da`$dd;?jfv-HElTzN2(2{2WgLp1bI|Xnw4D@?M<~X4~&M^$1|AdV+PDpsMLL zdC<7zOn~m;GJ#9zw;M?>X5~B)ixoV*R0O+0v*IAyf(C?rcCg^Kddy!A9+H`AedIp79Z7uspBUbsy4`r(rA^3#ntkpcM(`M;h5PlWkUyO2k(f=!*j6px3X{m&ET2^3*r< zTCE3|*p$L2TIZ3c^UUOVw(_!=eN_xn!;Jl7Z4%&Wm}bvVbgvbpiC8exYQMjAtHdyP@JT>- zi-7nRaBmH{NAd%-Q~rQ4_7eM=0@Yl-BWE+X-ANX6r}|nS7}pVLCz{6>wgyT2YYnd# zoDR;NI8=Lk!{LVPG8<8oF2)1$bpCC&POa+W&at&|6ZIBsv-9mSj|M1nWj&o@{g_;o zE#8WoqcoMiGEqLumy;<^La;h>0DT*@sb*kB>dED!jOg@&9Q7@Mz06IaP{7ISQ2#-&H@fChzXg1HLqGR}@|neebXnMe z^m&b~!!aU&6QihuipW*UWtCA6n4UfD;u5zR_yqA7YWF@s^!G}Lq`*4LhtAexK7k~c zMKr4=Flm}u@Ca}X7ecMtJQPmz6~VRp31^rC8_$5_2J}hkt{$uoQDfx1edD=0=-ZwQ z3-|y-D2!QR=*cV^U!v?@yYj$;IYzEXH@wu zkp>_7;ZyR2gUqcR7taNvZo zSS2>s2-|u3{#>D%KtH&1O2QQL22I*N$F?_580*-_^u6S({x7P)(Ay@=IYg}&{n&B} zPY4-Ype;&QJRd>(1ePXQzvaD!5N(`yi_#XZgV;Y^M5{7LJ(%5Z?U_v+g6=(dWkB3S z+;c}5$;8GpM(6G#j3|t`Bm`9gJT!cNGZgOBlaQprmdbK4MJ_yCq4lTDiKwkEjS3XR z(Y14w0==@lc^HW{=szy7?|$>J`7m&51R5prdRJ}i@*{s7>p#lZYxRQ|mhHh{x^608 zk2F>fpgum%CQBb_)(2<7Fue0?U&Fq}U99=2F9XX`Kl(jmTOyj1Nk*lnet>%c_xjBo zUBc>rZ#i4cqK*g$x5i)e0zzATY(}p^m`JTj@XyJ!`eno6+sxu$ zTbTIjNqsz}JaXksumeO1$Xd=j)Q04@8n_sF!|1>Oyh8LuUmtFq+4e+UA7w8Nx{;9E z6q~*2z60-w)johevhf!!HsKTZreF{;VappaS1XAoh*(h$YnZ<&0A+;XporyV09*(Z z?G2YawXhR&0kPK3?9*}4=Bbe>SX-4@p=o=fvfoa|`L&>bD2UiF${cIe_u-?aVGlmI zVh4E$4Q~h#^sBcKiJKE(>zPQrM3V6ADD2aNv&_U1AZK{G;*5e;pr7}@7hBDVC+>;L z8QkJCJ7eSCZ52+%koFJ6^oO`f{g1>%jSs8|b2EXJ;NVlgv2Y)u16w%nrQJY0_Q0(cmw9P&)~YRp?y)$+@Qw?9DwL)7LaeeNB`CrTN` zCT5RGCWlH3n$K0X$BN9FHOdW&?^!u?oqramx(pPJO}7~vG>NLGE?gxIIm++!ZLIZUCAm%((v@f)4BoTS+NS5Q~)M@(%Ux989H;smT~4REgc{I_#FgGJ}4-PuYi zC}YnbygW-!zx)+rE>=p*Ih5#SFP@9obati9I{p+Z;TVlO{Kj@LgA>~w%D`)Q3Lw{` z?;IM7#%F@5q0dG&19z>O2KGOs*1?bO39+^f!9sR7Pp+IdYto<-k%K93+H2G8GYqjv z)`2h({?hWFVjqp-Mjg6R&c57UW-=sn8KsEoDsMN*JkROm{n4~dj#}BZbSu-S%k7;S z&W)gu)OBNi&!Wj=O-o+TreoD;fqBo_WXW1h)}DO65)qXrrlNXr)wo`574W8h1XT8N2NlqJ%C4#-IYP_ z9rp0&PNCvdrbc(3Qg#3qQ_&vM0@+Y=nnvrY-gi^%zqx~CYRNeiHYto-u(9~@K=&OA zig2?JXB~8lk@k`v&aIQ`&sF&Zzj53rJ=@`(t=_N2QodzJzZq9abr2c;c?`8?h5Fp_TmcED_w(Hg4;wZ znm8JOs(Y3(+(HK%O|4+|5{=~5lh4)^voV62c&nHv?hMxT5#|XSKuFXc^I8ls&5{F# zXgpfu*UpDVkV-oXS?uFZH)TpNVLu-2DI3$z^o3kG3N>eA-< z=+?x0SlDV%7>hLG&jxB9KDp+q*v2}XO^|sgZb(>TmPy1&RDL(JPKZ&3!~XM~)%t+31OS35^! z1q7p&d}Z@n;{6*E1;LLpZw3E#xw(~DBEopsuxC9}hUdFzsjuyf7?@=d`fc~@feU`o zR>UTQ-^8O@#ORxQZpN)nyZ3!Z#dK5pg{`baS;kO3(Hyb{uqE~A8rEVl&VQAz(YbmA zPIQOS>_kT0390wt+uTu4+(|q!PCN!X!csl*fj7;grG} z3K#MBKQI|bbg>S$fsr=RbOjiZ;}}okp|CzF*a&(DplIn z9SJL0er5Mz;t|8|r+RK*{vUpKpLNHtBPsjXgAQ506j@zk?|vMvi|l5=3aO7e6MWQ= zWu8^+?Os8>`GPK1UD5AMSTDLBo*63M!l-Lj%FZ{%0X8(jKsXhV>npmA52plwOZp$dRcuxsYET?H@W*YGH|mt#VtaPgwSk0>q2S>kV5r#!5y@q5Vq!3HORp zGe5>cBIYY$HcE#Bue0GJwE@Q8ES7@ zo26mco-StLyCui3WtbQFs9B*<4_4@73922kWvCdd6o5B zL*Et84$6e$z_Dk-hEk3^kA1S8jGRn|Wxy@VRGqW&?}554R1!Ji*_(9CDJQZg4is`{ckFeuAcg>m zGqhks@hwNZ{1NNxj{tJ1JCPzUOol5Vp^vfOTbCS+q@ZrfLB??S%?3te*U?icYT+}1 zI9q!m(v?^$;X|`wY&NPXcm!261s}cjmml5{EL;DcqM>X>41o>-sj~hc-uyAl%KnJh z8u12HvM3=f;Zp&(O%Zm6X(RcG9_{4ggT)*?aP`73npaFyLe{iNuKE0pHN-i?9Gi2q zM-hjiq;3cMc>(C}Ox2sALMJ=jBb&&XIVe4H3!UY}j}2l3@HpKGd@{n&>=!ka0clAc z2*s6TP6C9dXzR1nzvTxRR(QB+L6+NY$B_2WT1wECmbo`|(W!&c0872ds4I$XF@@x| zMZY5$sqd%UOvu+8INbt{D?3Sg#%zRI#g(Yb=?fmXI<*5Xt8s6{i#3_C90|$xVqAxx z>x3?X)~x|swNf>ulW$0h3_~sW&h-CCxs7)B1+N?85DeWSq3tE5o}#gu1lDG4K9k-( zQeLFQM^$5de6Ef16-hmq(9{03l5Of})J1~vr*5Fhr(Xa1*&h@-hIn_;eGKE%+!Blt zk|vzq!i85*d{XXm6OMqvR~RTSoiUWMYoF*smJb>6A>why1iaE7e>;h1{GM32Hi>H8 zImfjzL1+4n>%yi0Rf$e0s-m$C%^gs*I26ce_5E5 zX=(Cwo`gCuC})lK1BGuI@o#oX_^p-t8?Ke}#3F-2_WU2tCo$i+{uj{p$_LJdKe)mU8 zWEr(5g7y9rcXKX=`&rGJ`&&cZgy8n#d1Wc58kWhT@_5sB7Mh~+H;^@l{7{DdNanW$=^o_~K1830 zOcS3*I4!33C1JNZDrVdwi$v2x7J%SagZ1{1zqRJz)jZf1-n~3HiU_hr zfuo}tM)m#_h)3jOOGo?Ul$v{4LhWCn2Va7oh}x869Dm1Z>oxKz7mz)p`HdWzhgVMv z+qmNc+wY*zl)Gq+ji>kY`=mq@DN5Rug>W1L#Dq*{OeuJ)?LE>g|{v+bChq~MLY>T4% zgZT}(|Jcg*+mj8P^H&=WWo>W#6yhBLHXrR^Hvu5%1=`{xm@ayLCQCRJcXbrGBCC1Yf9V_YdcPiH8{=gP6Ak2QtZZg{XN2Z4kD5mpGf}KAZT)}sA zQ0~F4*h7@`hgWd#1LO*Fo`-r0(d1fe{-YkFCDWIf`c5hREj=?g`NS#UX(qiv?O(=G z^9r#g!+^Yy9E)Wz60}ErDC+l-$ltmhOm(9Wl5cr?mnjr--DwtU2TH8)_kcOp9)Z#` z;tPJ$``1AH?70y?2Fgq;h6kHx6=%^e{|HTAQm$AktK?p~(@574nUZ7R+LM)#0UPul z1ieYo#)82e)wxTnHDd@Z8n$;$;rtjFr^>Z};ru*o>j%hsVmPK^r zMOY;1IdyXFMv6}5t#wf1J-du^_cLjNdY8iSv<<>(meFmUTaDgfAV^>9Lyfh7GOg^= z#@kh&ZOqR4r-#6XsM<g#o|H!8oph9l5VTax2!C6&E*lD;=|I4XK|8Z$=2Rj+p!JG^c(}3iY|uujp7i( zVHkPMV{e^84$8*pot0TymmvhFf)JegMM@-FdBUVo^OIJyf_8@@)ehhnTnjc86Jo?% zW_}Q1mU272`9|7v>9>Qz1z9-z*7v7b&&kN%srZzrqr-qpZw&K(>g~TEziuD~*Vpwo zuMyr264uK0fAtCrP3w{TGWZLU%g$Vl=Q#^yFV)(ThTj0IdIa~rPE@8qrj!V!gAPB) zIXTZ3q%ww$Do}vI2G=zyDH$b8faVHP3AXe9j18J;SAT(c!F>)?yEF?jT5k24a z{j#TGMRE0yFEpm+xk~e|#pNno{?Pu@Z|qy-7N+A8ly)AvUe5KU;80#TKS~yY%#P5E zz|;&#Gk3c;0H|jrRu=U9X5r-K)5Qr==p_2^bfP)-p~A^Qvnb6$)1`5FMduSE(&yo- zit39cB+ayA93ZSHvk*6@Q|&BpWK-IMMgj!`atnT)_bs)|Q1!2c5GDzjcft*( zCYYRKWn`ZN2^bogkvV>JDUNN_ty+kZ7|RQ6ims{08A#@nR(W}BtWbj9F@}CmNQqzY-|%~%sc)_Fd!9~G zjKAqHl#xdy;oe!lV6w|!b@Xp z5BZ_T_@y2&19x`W;Q^69ma|rm9hpC-w;P{sfjdNfo~b9qYj~P!BP>M~V3XX~_Q%UH zb?(9Pbu-blGU-NKzNbD*C}n~Ct02klQp)?a^N&K)pNN3rUElj&xUgmYj7Nis-)+Hg z8OZ`C94ouN3k2-FX15hKcu|~C4RD@WIrcQM`3=CL*pHr@Fz+WV(v1*%Cjz4M!yNVSL1hLZ6UcQc6uF5w&OtH% z`l($$pj<(H67wBZas+*q2H{x)zM~1cZOY;;ll~szS~lU3bk~Pa=V2DJPs;~ahNXUXQ7)CdVG)*5Oe|&}mM?8o@+1o@{B4iQg5K&JGHCi6=akwOK0SV4pHj1H z4$~SwZFsH*p;4U!xRXJe0&6_ASwK<*LRaNj^(Z8Re0pRm9fDVU$G#FqI6J>&5WEaK zB!g-`m&`JL^dM~+v00Fn?boZObXN5-*17atJu3)PW!X`!fE z7Tc1Dd+VQiE4vT_yF!u<<1q!xk_w;I9ibq%LiGg45<5wAsq9F4P|1-}xEcxl))Ct= zGQZ|4EH9T?xw-;W#7B%FrK^z-O~*t<`|mQTpc!%Yf}Y|bUwl^TsRSuvqTFZnuRD`p z!qN^rE+qZ9jnvd4 z-EpN#GoNI7xgBI6 z=$_8wZ%PZQ=kI$tWiW6tUJB#;sNK_0ew}mC9bjspf3>Fkuvszf9)mGOOq`~|*-jp3 zW&|VnkxfKXi#nahD3YUl!c>wSA)W@IGUul4?Nt^ruYyt#gsH9G*Z-E()r!Ell_I`Z zm#}%{FDeu3^q-0{pWB_)E(O|we0}Q=CC#T-GM8PEHH+segT5gi#b2s2D<}i&xK@u7 zbPOyvlaCUf>3t?KWf!x{*5R%nnJ2oueR<@ziH2!jkQ@+3viX3mO!f0)Hh;y&fc#jC zl;4XC>luT8yc;D(=UEtWG>P=6svpH;V8OVWwYavlx)EvU9NQ)>6cjs z;Sg~bHrHt>HbY^b?uNRF$oO;)*#1DKOVb^md*k2x*Up46GRpbVp7FE(2{qNY z56BfJmQNUYf~RsA1HAjf#5i+&yD3xTOWM*Nn=t7~!aND|fKS%C{l3UC}6M}??bX6BBJ^XW4D^$m2Zd>2I@wync zc*89M=)e=@V+fLQ2QaUKV690IQdTj2O%##Btw|<>-GtdEpPh=OOCh*)bL%MdvP8eb z;q2q}EH4YDx%VXh(Vv@0{0lP=SaufC6dL$W5VTWz%2#bHdD3RangN9ur?*f~iCE6d zJkfAW#q0BA_B(9@XGpbXb^)~487QxIbt&oT;`$iq{nCf{jCZY+%&C%_MzN`+&)vk# zn?(LN|2-CduTiuXA@v?RlUMW3o$$$-L3vYziY@SYQ)W(lm3Q_hN$S8Hb4yCPlTq=J z%uzBK{lJ#D2l3z|aRhPJm1?a(j}#q|CtI<5*r^lE9`%Bt|C4XMc4zd`3=w~JCh)QB z+o@gHSv@v(R^f|BG}eM6b^+C07Ocu{zI7IWTPSh}qFhj=SZjDq1W+e*!mAeRy;UG* zT}V8QZo!yWdc^mU#^9CN0mlMSvmh|QOr$DDVY`+&eZ-18Cx&VWj@qdKl zSpk_SqM$MwI^<S%sk^~2^LH~!X6=5dK}MG@oSHlip{=YJMduYiYQ1N-T&kVE zu*I8r!b#)l5%%S5B&-%%V@cNn!m_y+uDzpmTits6O3jZm<_Yctk~tyEqso`P;wG>& zG4q+D_cp1|`-Yg$C$HY9j-EdsCC&UF95NgzOeURJHHgX zu|1Q&&?Z*a69B^n%W7Yx&gRznc`mC{^X%lDWk|o56~CJm|Ea*dC)~UfqS+3}rfcx( zSifZ-Vd)FKg?TQoe+RR3mI-!!fL;l}*kS{gh4W5+oAZKb#`-Uh3S1ogJGXOcGkL*j z+qVke42ZpGBWN`*aae(jI}xw$O$!kDol{17bNHM1=S z`@8_6L1j$e#C16?&_^>KK-_`9&Leb;kBdZhn%CWFHLp<19sekM+zLWMFy`WWjk;D=Bw0BjtYEr8UCVG1cHdQ@(GNh*X^t;? z&Mn`9KhK&J)?>DaF)Sw$nca-UmR?9-BWAS~WA29dPnQ&u^X^50TKQn?>M+CcF?dx6Su3)jd=B$|w589ovgI8wlp_f~WPm`R17boZ!dn*2Z z-EYR!J6P@M+Itp2Ce5oJkFeH~D-MLGq~DamtS74VCH`tcWXmkodP-K`M6nG$sH*8y zyF1HNz8iA7b{DYfC*thxKH-ME_M;X(kjPI;+wje>Fq7wvL5##*@2Sr_(5A~he_W+UU#PH*y1bp=ot(2@qwAp z4{Ssg=f8DY!p546?C26~3wK`T4BFM74h*@o2IIX%?;>S={Crj>87Ui?Z#g+jR79_pxX?Dy5AD|D7&kI8a&SkrlvA$B98?9)-Gxc81PO zP`E1y!&rSTdQZ2|W!brZjp&_HnGn=zMB&ku{&6N1K{4O_?O z7z4pca*bl*rQmIu!PnF#XVNu2gw5uNlf3QqGJiBmg?lA9P`yD@YBo#>f_UIE)LP}* zLQmNtH3>Q!t0aY&{qT`*?+F8Q)?ZvlVY1`8w6WzS^#W-(h*@~lJrt;cMo{7;VT1-#a5)g#d#(oa;L>|gxo2NLh&7A4a;#!zEX{9^OgCOzV^h| zUAnon@MTS#h{lxsV_i(hJh*R43@&F0oN;$1=&vqX^i^FcRf0P^m%KuVvkbP5@Kjnx z|2ivM@{f;iD}0|1+{f_X*D>84PjsV2E12PX7rdO>YTn@U3qyLj4`w$BDAsv!Te$iw zB>>^SAAum<2jCe~4D)NBD2wMJmiws|bUzrDPdw?L zUNEr8j|V7g@0&vM3Qj(UxN4IkT)Nx-!Yo_BhGn)0UFgQquG=W+u)j9X2k`uYIa- zEVD>X?lM_&c0j6qr50sIKyf9+@#SNM@P8jo@R0ehpm8HLe9q+rgT^*jBJEE&#VkZy z_PFTaD_9b_fG-%h=ScVrlCbE&AW>1SyySOfxATaqaUsnAP|+}Aj_RuKYLjU?EXij4 z!oFDY=+{YNcy~~Vs#z=Ny(|(kQ*yfzZUffd5UZs(zU7X)>r4us%^v*;QvQ}fhq`M1 zsbY^^+hlu-NJj#=!CPkx8M%X=6#vu%66R{yRNhu%L)(3UmoUKt@;shvgRXpWW#vn*b@ShW9qM7|qGDsz$JTrt== zIYVHjf_S}TWZ~%??Y)Xi<-r!asSWBZbvT>;^hTTjd|O9uP5e7+_k?YA%bXyvU1oNq z{#~dk=9rBBPwQ~)t#;qRWr19<&s%B~hnawZh>UEBPHDl1rv$lqRk+k!+WZZw9us)w<+uS)-1Ire7pI%f|7Xdk?2$*Z6xuJ;ysqriJJhZ1c{gL`*}K}|RZEYiFHeAE>WfMr%62?nNbR{qu8(0_ znN;L7D*Qt#MX41}6tHucm6Zxsxf96M9hAATH*99%%EXyQmZb?tiu+b4O)y66m27pO ze+jbe8GD_MtSe(`t)3Wm+e76x&E8!5q+iflo~#Lw7bb+e^p@aA82#SZg4VwOg$*BI3<| zQl6U{8z?=?c%>P$>sB{g8j5r&c@DW!uX^H_sq)Rcp`Wmi@fIzR&A6myhDefJLbuQ^ zR{D0|VGA_^qQM?9yh&UR4V2h|(%tTKZ#EpsVIY_CwW!d-TjH zzjCYpHjJ}AXq<g4GiWO$3G{Yt^UdQP_zMcHJo8rz z?0}C3ddAlBJB?Ss*Z-8%+OHL%WUkH>`C=qhxpNoDSprz@UX-vG?48M_VqL zuf_OKH=S;CcAgtbJK`)RVSbt4A!*!+9|bI{*>s5)i!V0zAVNpx;{V^{6jUZ$;RFT* z6!H&M_|Hfk#H{TM4XhPS3>+N|Z2w#8(5z~uv>}SdYn3_HMoa`IPT1tI&az7WCtrq8 zKs?f;h#aAmokeacXW zwp~$O_~`s;J?X1#KPvQw;;nN33*1+Yy|b+~%;*xGQ{nHKz7(wYW7>tMwlf>YLLyJ3 zPsWhRqmT=YCloiHEs`L6EQp9s?_4Vg_R95)lT%Zd41+rAu6Eare(9D(7sKfas1PZU zEwst#_ZIu*w-zGOR+W-l4seMJd*;0*=hS@drmTt)8{Hb{spZObHijGM#)2jH;Rb6N zyfDW>H*3{!wMMJZ8d+-*043fOGDIpvd%YoSs8`B?WpOfONb^**NPJ1>dihBer%eX<=(>XNtDtn4ZNiZb*H`#ll{jp{=oSvuk zf!t=2oQ?@rs)JOKB08#1_F7L{DHRHg0WT_1hC;*4pMM~+t2@1g*{KKO*^IlRt*9r; z;`Bi~oFilJeJX_X#ROGE;JIvOzlIkMEYLg0s8*E>#6RK60;KFTxvBa3OP%X~yfzFu zQh6lz&R0Ptb{Apcm8_D%2Dg;7#eC>z-@&YB?I3N!FSBa7omY;&$W!AFM5uc;gimaL z(Vd@cwv6Krx|fFRw4hrmSZ>@!T7S)^bWAcwIa4=!;DDgIh_TZ+c3iq8=1ccHq&CRH zC+~@}`=k<`X#k_FoK^Bm@7!W-536*+hqw)>2w`ANeVE^XY8Nuf&_w4%s_%P$IwF2I23BBkFyD1-3Q(fr)beIocypp>#K>D znYJ~^sv^6&a@~$mphvpd@4dHdL%o*`>be-pggU_I7vh=F`n64#%Z|C4jeJud%5kMg zQjqPtxujkb{wK*d{o3;AU){VEm;P(kF|;Q@@JFxZQ0eXgsN`E=Ito#?B}te?GJWWC7{C^Nk~ zyf54*fD^Cmo6laSLo?lveN>?3v2MVnNDoNsK@b~7GiUJ>W;5s2VCttko}Br~Mjj{C%V@2j+J4W^34VRQ;EOg7T&~L6#OUMk-u^=m*xrceUG8}tt7@rYV=#O z+>Yi+8@&AF>EZ$44aso2>X!U&RortjYU`~x{1c7odoc?0*%#Wo@Dj=Q*^S7*82+<6 z8hdL*_&pcuJO9mJDZ{T&lKanxeOt@-nMue$-1~7$2>66!P?6u9oOsQpe(g>2ECy-a zKhgahh}GNgZGIzU_+E+4ysN!e_Rwl)N}FMsFY}Ss z(yFF(nG)uF{EgPqhWh+fV4C$~a|^U2iYl)bjX@Tx);wAp6EIi`tC$Q8=EZEOVLI8e zh|F|rj`#_aD`bC~PEmOSG9~V6Y5E-WChfSR7&PjN)7WC>wd++Rm%kGxRTz~!hpFUF z0`d#<>n#>)UN76nbe&wSS647MwnzFO*~eR^l(ys$LON^Cw2XO*#DVQYJ@i~MRKdF# z@N|*Sb6dk+B~Puv@Gc5L`2>k@6a~uw2`@&bvR}M&W!x$o?zF<#!3C1McIkmPQpH_9UGSm!L=50iqf&;U}n9)Pp1(^ zuok8>^e1{~@ zs-n~|n1)eG3;N{Lb#mn+PVX+ow^)3<0DEouqxwsT`Q>GC589%pVdbo#%fjt2L2w;FJfm8 z)P3*M2J@Dvve*_)Xrza`PC-kvNip{)I`*i_HG6#j`K#Xr(>C|=W6;C8wuwDk0Ia_> z<4TcP#b$d<+p4d;d*NmtV)BMYGBV#rl^&%9ZUw4TogJfq{+`ZYGn^>wyE~xx-Zg^& zv1Ls_Q+wT~+FX2%N9Mp}+V)(_M-d?h2D0-m-X-rg3DUBmGp@~L*gSmc$dWHqH7}2i zl_CNG@!R}Y2sixYKhjSICMwBm>U^oPyJ0nstYJ|AA}fAyD@2=U63zqk`e7nDgfT2Gp4!Z2+mcVS%X`&?opan9Oq|O;pYi_ zG7J}{OI*WqkY|aZxU#wp1{U|(III|KEJethBp=;yTqaWJT2lBN(`zPDI_yAna0u? zlm3JUhhry@vJ%29`eHd(YG+aDvT7iI`fUJom<7;ni;Z(5TJN3U8e>OG7xdfLbTf{7 zR@wLfJ`6H1Y6`=3RWx)H*bAQ;39fS)Rj zcaL<%pae%O+5yWFB(Z0M*buvUxjlYHU_Lgb?A)J9EYtGCkkMe{NoN)R;OWMw3R68;v6+Js%Fo7tk9}XM^heD0&&s>)yDLC$vi}MVHN7^sDIw*4^0!St)^kv zf64brujp?!<}J;->6Ub4a_|E!iyMY7Dw)9sYw8r~jC`nsx@+tot@#9$#1R0{y{S{V zy{g;grh}H87)+irvrAGV#8i~i7pd~)vQ_0)y1UKRwp%IsI2(rlKo2AJ&{{7nMvK$r zqd42qF;>oY>^`EKiD}Y?(H&>)z@|!1Q?g`h^j9A@+OJb@`kB?PBsuTuva$A#g8}l=709 zniJ7LXe)H)aq&3dVcWR(W$g>x^Ri`!Cvp8?97Jydu(D5IBzENScB^Qzum-|^qxPKn zg3?1X&K(76y#qIPb?bm|D{g;uMDa`T1xi~GTTr9U8qD^}Gl?2;*GkW z%R+!v)qr66x2RP;*b0#!E=NcFqKx^_)M-EYw1sJuCf1~?Xi2mq+riapz|N@ZW~eKV zAA&c^#+_DD*KjyA;9orbN9Gm^0BL73l;nw9c-m0DbM(oS=?S?F&E8CQOJ5Tl%4Ouv zzAgOmC-&yhtjZ#GGDo8);jYv1FK#dI`mo1V%kvw;O4+mH`YUA8Bj3Dj&(6#VXw#am zprn0K%kL_ud-ZJq4vnYWZbgOwfeX#sB3QFR1l|+HT;7(-dQevVl1CL$^62_tqIE~` z+8p9p3PgSzVgV6sw2!hJr6dE}8PKhyy!+p)s}k#g?HR5!vXwDq~CS2`Zohs?jk zn-`AEd-eVn#q6d?!4aA!ION>*tW5w#eS0BaES(4)p_yxCN^}3ah)0l|L1kp?hvd_O z50q1^;Rd#H126b?A7Zvo)Zc4J{ngHEN-hM**n))=XEB0}c4+*EFFBsSypDJ#k(+KD z`*H?i*w#kAWd|nK+vc(86lH@rA=!~NeC9LF6fSo~-anvC6tT7{W>T*yBY#yM|EXz& zMGmNL@NqzTU48=9+36wRQ4Qq{Tj+kMbX1y|Vs(sQH`ji*v`k7?T{VOZ9Eg27_Gy`FHsL^5)%ZO$Nd{ zQSFGFjVBVo+lBCjWqxLSxWSzNKt!d~4yvm>TLVDcm}m#`D@wN1 z)20N65b^@FDIS zfdlDUfnw}U`cJS&_h*B!8u!@zYZpQ*`l+9KaERdzl0!RmM*r0+bcCdC$7%R)zhpZB z@@l-tu~~&Rd=Eo^N-9;O(hAXD050 zkxPRXFHgiopu)mQJ2@IWtR(;Im4|>vdRsivSuP1O_15oqd5r`w&%As=&6$Kq1v}$a zul%k1VA&{5?$9;QGhVv+|KaVOzbgU0J?(VIwr$()*h$AWPi)(^)p63X?GxM9iS3R% z{pS0wJ8SN&`T5SO`Wvch@A~ZLL8zqzT^SQ({Ur0_i8h4<$8To1bKr9E-v(@L=rU0| zZnC-(Gzj8Wd62Tz9*;Tco{iOk{hzg>-?=F1HWV0`8~Xp0#z-Du2l!v0bIo^`%^^HN zn^AXLH5fYpd}^{NZR;XQ0VB!+#71*avprfe7m5y1bQ6!Yxtz4Kv?nIq^S7^Mz;yHc zfSnnX#bthq0G$0$MzEwDb|R#PKx0e!>XL0Xo$&8I*Q^Nty9XwK*#k9kZTrojt}2r} z$wJV|7IN9uU1T}S)1uNOanmp^5!f7;rcx?TS_f6R{Gsrb)!A)ZjrqdLRWfBo7`J z_?72pG5qKQ>Ae279Iw4F-Dr9iZ?5)8D%Bb8&?R+f>tQcnP&%8R`dt%aDw?1=ow~$l zDzl!asn-ZwW|PD#zIYrfVB!&C1pkgYb|0XfwNxG}v+3^2OCF>C>v{qXcG1jwg*{4x z#iF#TYk_S^4wy}Sa0V0$LU)zR#aioN`~3cj>CIQ57uSIVf`;hSXs>jf$mDY6GdY7? zd;srBwiy2Xwp37aSJPi6N15YxZ0^%N;J#b=h#xQcpw{Kn;o-f+ppMXY?%bvMnHgX+ zBYWodEK9y5f2sLzZ(n#gMMLRLxwOR$xhg9Z6G>ODh5ODnemHM`s2~@#rgb<_0>Xc5 zs5BkF-m0{Um#(vh#02`)>RaG9I#aSg?QPg%TSCH^Ax&eH7FYLRe9FquNoM_SW6gm# z9Y9HYRNg)EDy0Kf@x@=g?I@dMQgEWEqyF*MGIBY%D6AyzRg%xiU-zBTE+VGLSqJ)q@mUP&dm%=zVdX|vQ?OC#ggYlibTIk}{C7v?hR_PQ zv0c5zvR}Pz^?pFw!SuXUg3IrZPs??~+X24Q`WAr1d`m#4zD4Z;l_2)ZWk>4kgyc(oU}48rRrUX!J`5yqD`|oZ%<|G@E~2E4HDX`jVQBQN#i(0 zub9XUs2_9Cc4%VX%+#q7>Htw|T}E0*oRd8Tjh!A>>)S{tSoU5I!b_)QT%=#*7Yn25 z23;a|DHM*@=Ly?vyz-YquGR;DL)ahEgcI^q&1@IU7IVKaroiXhHIEmETzz4Yk0x+_ z)x5@Mwu9a2*B7POQDyGNVSG8wp18u?=bUT;S0)TS+t4Knuj1#B&>4w}?CiqbspqfZ z<~GE-B&;gipnU)}jyHR%pnSdlxT|Q#1Go0=q3!3Kc7Yhc_HeI8PDc*=_P0sDfq23B z_*s`xETOi!lo|NQqX(DBfQ0l1>BGg7Bt4^gq57@}g!@Ym6LwB@`FqeCUJ4^8l`a{w zx{_1$s?!FVW3*JRH?Ifsov51+Z(XZan~u~Awo*KJl!s37BaWOzgBk);-GU-HQWcTP!JVaG(I}(Ol#PffYzvA0CXlBzR|$-_(BC; z7vNvuFBVSVC@?1>jnXt%$Z(AOgQoQwvqT<77fGHj_Yyo`6!$BO*K8b#;m(O%bzy)s z-*iF6A7M5*Xjge6sC^^KDcT|6GBh&J1AdLYhyT$#{h2jbqA_c8Qr}`q3TEt=2a0k_ zKbexNj?K8(I*)>Jox6tcfw#}I-BeOZ5HER@K89%NtYz)w&pv!FWl5-ooPr^WG4d{& zD}5joHr^zt{Z~t?c52dz)xkkzNvns~5oKh4r}@9{)&DRoaP-RFoW2|)bU0vOdjBIm zSa}Cyv;SoWj0(W8#Pz4@T3@#f2P@oy17sybk8`9&g9+S8dv>Gvdz(jmKA}F*KlMo5=+jr_q^!YIoA|w!+ z@OI1Ut2Y8PIYMRiT#8WI3)OlrK^F3rl@8DuF*Wd&9I^D%3Y{JI+=`&&IRg^-_J$i8 zq9icGjjImMptI$t-cb^3fF{}U?~OSXz3-Ym10)1`>4?g0oSUuoTZZC%|B%m!Upx$Qe-4`DAY!uPn{G*#TsJs zrH92jQPCy^X%)saO3BbZ%ptoTag4P#A|*Pqn=Ss7>cml8$wCinB~Yry`aK70#quRHhqW zFG}iWzfrb(;46FQYV2ZVSenB{c)z}654w^g9P`!jKYh zQ=VoBrm#eQ&BUrx_0clo7BzWR*om7vazK+EP|v_%rq9-+zHxAd7yP-m9m`WgfAp3x zRqbeMr^%BkMOc)*DQ{MMz8Y5>k=^74X&dNuPhIg{EKB`qp2>SmJ3Qlsl{v1kj}2W!*6X+8~j_v-~sJsuK8cczn4g#%Al`l_uyU4u-Ha5A7*6x0pqB&PJyZ zwhZRk;vG}7smNpT<`vpBd45}RhONKi1Bf_@^#$^RM%nMl`#N{J;lIwtN&98bMlSvp5Y1&bRRc~EBLqz^D2@7Bf= zwc)3>Z0m?GCRi&A+te7pK`W4hJUTPF4Xo@u$VM#$DuX(4!jDz7?o9@VfdCYSWJ(lv>$ z3Sj2F$ei2M+e~|HSvf-|dNoSvv+TO)pkac$;TcA+SjLL(%xzQkZ(_uchMqLxJaYyh zS2reFtBYC~mhaDIqNiEwC!D98tH8e&+$NK^M4)%l`N-Rzz)oB7m#xuxH%Ha)w@}$t zC9u+#ws9=KgnW9_h&Jng8O9Lsx)v<9UQ1vj%|+L*7x2+xVHm)&)>nl>|VoD)>w;tKRca8|KdS&;uwPncDr`R&|2z35A7 zPH1JM9e2A65TJ&c^ZPCm^zeO!O$uJo`0Vp_C!FvPd|MZFgO$;5pD_7w27lhYvP~HV z+{}W|ImbBDb)0nQS(8_rOnfuzx{@Hw@;$`Pp=a&t?!e&`Elv{g8K7`bcsxl;-v@?< zhog4>khPAkYQQqjN-gndr<2}a0=y~|0YU+V61R#`QL^3eVw|JR^&1g10-lj&<=Vbq zXrsIjLieXs)fC@yoZz|kHLSNsLfrU&`$LmEQF}RI7khCd9T0V6KQA1IG&>Tv+>8P} z!4mHGqF&rm)+SkqZ^+v8nt7majvHJTM@73MUMq{-Icvn*y>l;=XJGep$52loI1t!& zHfb<{D4b=D^^pO54~Hy|ceavJxwnQnh|nQxLxD@sL&aIh5fUP-ddx)lxgB#nT=y{| z{9!d|t{2?u8vJ<&iK8QT(nJB0ja^BO5O)ka1^VI@2i}#^4-Ep-mAMusSlQH*){S@5 zjRpk#0}e{;ZI5LpE5ZZv0$G}m0v!aNi6;3adWQ<1rN-uWp^cuE5>_$5TUVlzbAl2C zSaM8JI1X$KHs}TRC9l@+C8i82gLg1dns6nxIAh}4B%|Yh*nXTx2X+6*!=?vA#ZGYa zRgI5%{lSF|&TEJ@HZ9<59%#!1br0uePy;5uKTo~$6l{kBd@!(DDKIdL z|DmUz>Hn27TIPfG{1S$oY;mV>X9SdvjbpEPe2if~;>;eO zbGN1D#4a)%rFa^j@^YYNU#XsZ8A>EM3)ouXF-BBWZdifsaYA$8Pb=hu&cE&l2o&!` zEjopRTk<(#1^fc_uOsCGn`T`1w6{1{VKesDw_LbPc}9ay_~vR%LeMAXfb89hOHM?@ z6-mM;4;+SkQv(k#E+SBG4Yc6kIEPdv`>z668m#-Fs3XZ0(bI}JA@HxJsA4g=I)ig6 z;Y>zWxMJ9BuBc+z?9I7_ayW00Vtr;-kYX8ZuDD`-Ca2Ef8f>nRVw!x8Y3Ox>%f6pg zua2u`do7-M(X0GX?K1KYb*}_A?Hxg|JxTBO`^B}>fk~e10Xu{3_D8gBtgigX)dxZR zIA3m?eSKU{?%I=C-S&(QJ5Q-y}#Yh;9IXG&$fq@$h^14nB2U^tlsS| zq#ipZ`xeI!qUj%42KDO0c69bi)VRaBYtj7e>*HiTp|l&r&4LrtrgTEr{o9ILAwGF8 z5Z5k)vcAbV<(@Bq{J#idv-gtEe7C%xo(99ax9^Zb&oPR=u!fRj(Wl10HicD@rF;ka zW6$Xza%1Q0d|6PUI>G0&>;szYp%0J&1psfCu{tGJpQ3?Vw9rq^0Y}pg{==uK)k(45 z5$8Job0(!+b~KSYXna(?C{@9MGOCTDTuD5?pppA*BXR{-#Z0cNTmbsH<&wNqg{^egx!|D(G@0u{ z_QMwZ_EGbop}|4S?N9HXI@Z1TuR=03yac2i(bBcm)ID=qRa}>3Y6YB%yoQD9zP?T7 z3!z!A>pD?DL~X8@dtSwk81xR6;NvWb*rte(eho$PODi{ZET%mzUN1VY}C)W zjuA8>Mny}%B|^!utH700k2f-3TX0ik!xEt+TuBG}2=vwWbQNI@^2e)O9aEL(@BLTdVn6h;7_N>HNsj}R?Px1Nc(vI!{!AKBRsE+LSgUHd6xLLnC`+_;|QOo;ZFdq+OzF!NWx(&Ql96a)37NWDPFmjpQn{zw|x+ocDKl zni0^8z{^~vrNAb^l$D*+e4S|K9XOqDHO&EpCCQx!sS)-D1k1Ecwz5?qz(D4J8)WTU zDWuMDI0dGy$N3fBxxWCNRGOu`a!-NeLpkK3e@>jo0an>bX8h!^#H~e6)zO!w3OQp@ z(1AmIqnm{fk!suouoe_2H3Tg38(-6P-H&DW=yZJKBwi=5k9Na+8{&w#utp+=eK)wFNgyOaN6$D-pP z?!28d#y-&LoY)+rl*F@3n_!wB2Wf@nB&apZr>41n)jUaaC`s}BH&mszo;7KJu=V0N zrkQ^D9<8#@9bE7rdz8uBxBhp~i=F$bM|J5rlBz2q+SH-h)O`aq{YvtKRKrVX$cp$? zis&d!!frJTt#0wr%$t$RTCvjOcrb@{3KXjo-et%t#~Okz$J)d+(%SCV`Y1y zgJY+)ngJsFr~4E1EddyO8O99?!E_d8V`wp}2GNlBJt9P+Kp$zj4XV8J_$g{->w3gi zn>ks;-rzbFyzE^3c$=r!9&at+>Rtj~Z@h!!`h=9Dwk3&!b6L2vf>bZ6dn9 z!u3yFuPTd#Yd+>R6eq}FWlaiu-4GP01Z1x8kmVU63h80Zi|7BH<6V^?_UC3x ztpTvFAUJUsFtlI<4&MzLJ1N5Z`VX^t;q@;Z(0nNk4;Ze)dD8**fw&R=>H{4>=;5D* zAmKay@Xu0E->1h{bY{D&gsNa>|~4_ zx-P{KZ(T&(9pl*edPYaJ=qS_W#L1BKG+&UC*A-lHuFzZBnK4Scls>*Ri%E%|Fhvec z^2RSur}^rP$-8%n(I@74iRdKd=xB!4h}+B$FM09Pqi+RMkW|BOTG${_|u3O1w*8C(Y92G&JgJ198}M zZEwi@9-^lzI{7e-dhE&&juV^2lj5sAlkA=*ve!c@1MlD4>p~>jfWsk@6zA#&|6lwj z5IU7-+l450AwM;i?xM>M@{)10cr`hE3zZWv%HZJCmvDd^0*3vQMm62l6O4En_pO!n zRx<zeDm)aTE?gLAufz2t@e=v1?MX~QFwn=Dk9#d*4A~Sqky$HFwXe2za zuk7oKt+%v7P5Feck}H_d4RrSSVN|OD`r>-w6il@rmL(i(Oj{LnYqp<4ORWj>+-v}k zrnVS&kBM#Sw-Eb<)yq&jGi-%S^`*ilxnU)zik|D?swIl;7!Qx%$Ey(9?b@)Lj>w3! z46@f33u2RaK9b5R61$~joIQ3H!GDsY65EoscbB|;n>Lx)ZduTAkckQS_%v2N9>k56 z{|tyq?S}k={vNDfMv%96vioh@z=68hrTPNEF0e*+U#?d<^ongrb!pTfQJ5OF?c+v` z&wcM`|9+sY2?t=yeJ>4gfMZ%ZIC3xYmOTVlwRUN9A1tHbU4B)$vUyHiq?|fy4v4!p z7|Zwv=#6yxXYzcGI6bs@eZM08d``eeOD=-r4E@2*zkPg(;Os<{BL3agGf{B8FJz-Zy4b`(gL4K6xO-#WAg%$HH?+z4E@-n6I#HFL&0>biG7x+eF)* zF0YMWZ2pS-E>|x}>Mi;4hyL-q^26WESAPQc$!DU6p4H0$(cT!C6Cuy!jq=kAj6wR- zD;g%$HMn-|8DE|IkoSxbxQu>wh$GyGJ#A0LPo-?|{D@r?tdXbSC_6S|RRr6x-}N;l zwi&XvOr{3Vwg^4auJBRMl-}$+O&hn-D(H2|fzQr^UH5(aep!d)du?U9+Tlik))iq@#$0;f2JKkEj(Laj4NgQc?4(~i0xXk20cH1eEYOIpM#_I z3FPf6;AW-#<@g*WBBDsjEv2m5jrevQSp2#;tRP%m*SGG0z&(oRCyNk2iEGq9I$97} zfP=7QrkqXX(J7$<$n}Vqp&aDDt(8&!z2vM@ERflJe;t}OsPkGbrqw}uH1WLW=VIQ` zv7nf0NyY5XV%)H;(@iYI0_nmA8>?N5g-uiQGnXMtL$rxpgsqYiH|^(*Vxl~*AaR+f zja<+ZtZ3H_$~@dBWWTr%eO}PN`IaJ*i~T=)6Rqm;w0iLIxye}dC#f16?jq}#zMINV z8aT~enUZX9jp@vN=}_>sln2(uMaWmk5T!{b{4y;*N$))2Lp-?O=FC}-=4l0n0|q8Q*uZ$>8$Rfx z4E^APJ`8u{52ml3^q4=@w0okV^$@MDku;~m^3aOo;Lu_1hc8& zbo0t)%^$!k;A+K`QjY8dFt?xi!oT@LBn1(JCz3SEM7}h;Ze;He`emw4wl{ET(2L3! zTX6f#D>6TSA=bs()%&~3KQOE&@^Q_!ep*%>-OcZP7M^+t zBj~=oKy}v4AGd7b)FVb{1Ek?faUz@p2I17BdmO>DM(tEksxY<2?Oaf*utH&=PMloD zTbA(h;e!0VB1V?&$6Q|avE6ai>%c62BMpQZlAAVk3wRoG<(e>}k1x$|tUgNwR&EFF zu0|{R3>r(-(P##Ra6`1=xGjUyv?N@NUvM6 zrxBIQ5MTqC(qOS8!_zRGD+uE^uVI|0 z6;d9PQ*W>pwK(jZOUt#-&;V;0lgA?^5N#P%?pW)IHH%ue*Rk(ri*i#S)gAFwqo*1Z`j{)UlY7d?Kc;Dwmhh8$k>?BhK{`XEQe znr|S#)QMP27c0a?2g!FMoz=qmIwnJuxEtXObfmGe8bKt~B5HX_t^{Q2>pQxC&q-6% zT^SW*)VR7MuMrOHhPNEid*$JYA(@z=BwH0j3}ZSVhq{){si@DO@Iv4jxyNxCGhNeI zV_nU!OIEm~$p0}J)b?O!=F%B;kCdsBiP;WLl}5uq01TTV$et)PV~*8>*XfOuk2)rn z!{78&fOiyW%{w-0yQpJqiiIB$!6LN8noUh=K&d#Cb!9-~B!%80TpoxB5&Ol4@C$`i<} zy-Q}zt)PJ9K)s?qTCB>7YPLyL9ygfl{9u7yXyQyq?`Mt5H|-xaR^?Xo6lme3tyZD z$EnGA554iiG)4}g_9q47Bt=ZW&}1^`Z=QWYO&A+rVji-1l1$XBid}V}b2g(PXDU)> zOJQN@OwKs0LkP!0FI2S|+=6|$lca{mZ#v9)J?N`L(`2XxBl}uY=-f7REEtvGT;CRk z)duvB;KMZvqp*+)-xf#?om!mcwab-W9kTg!z8$4#;7VFNzBC~*iQ5*8#0v|;umTGf zt-JEJb^^waiu(Kr{>Wr~l)>$ie4jnGEqMAn*mkI26qtT+L`BbBzF0(~CwsWQMEO#c z`?r`cg1vCW`!=_fy-2$t{9D1^F!vAery-Ae>|Qx#kP-&u(*er|81>#LPu^a~3*`*3 zj#Yd|FnAZl_F+)@S4E|*Fp+k77HKtQ)EHCqclO0s$QcR~deCV7q~Cx|RGFAwPcILa zZ8@tTV0A1Iu zJRKm2UM>Rn1E#>KCnW_79V80E%G9bG$#b1enGJHxFub7`+CfbGAngp%HxuDb%KM>L zCZSi>&1JHGPeX$%WLi*N0G76&e8sXQUH>ZS*$ld|nUr_ZisIs8s8z1Vcf(1|6iqzp*L%AAd)JEDg;92S z-K)g&BXz98iAd@hNPnAou@E}G4uwzbu61E=u%M)@N=V7aed@w^LyPf7Q*|L#Jrqel zg`_Ovj>ULok--qH!2x3*zGKnE2vV#@h(?QyX49ey@0V5nwi<;FB632zL@Ne4v3`Gw zl=YO24Lup6@l?%4xQSBTBWFbtLRG)b$^#VR$J>@Nqx!`W%^;7VwIn05rXhXF+ zYf~u03)K6>dr>#DL^~!^t24|{F{4E1m|6*)M#;QnRFZ6x0dG6mQL|tj#yghP2jL zX`ZJ>96jlSZob*2A%vA7^^yo5Mg=_u8JI}Eg`q_m^ztOCl@uyPYK2^pY)>eF2{@oA zf@5ATyFyHhV~`9K5+ebo5yeuCin>NpR1zj;wlEkY z4W9wb*hhPj)PvfMnxy@s9|hQln?lnx>D7wE+{Yteu5D>{i7U;U)!BFrF3o#KYkrWG z3I(JYn`` z>MBv&S}(|8`o_k@kx@MzhA0ym{`3&GZOVkvEJoKTu^m$~YEx$Ux=haoyp-#LwzvaY zBgRLDT4#^SiFk|jntIz4j4KM-IATrOz#9E*#Nwr9jqJ#m?#>=%`M1=#cQM#o)}Ik; z*{1gc)*(0HRFxL5TzNY~&Z7)OAE&_7U(ms)MZK*)-lh!7*7RrV+4poa=>+qpwB&Dh z8YQ+}l*Q5jxz4ZlNFwaj`C+gr%Q_1^NymJ2P-M9OcVt(?syZlgi3ZY zC19Mvi==;r6DzDF$FsyGx<_PL;4$eN&n|lM-*yP zogAv}%D)l{N|TZF*DF&w|JFC40xdT7&oOh3RK_E6BLtsczI zhADlF6cFs!a?#S9>Rm64&eSv!Lxv1N1Uv-v^6u&r>tKHF{c*a5-OvoX-?;6L=#{;I z9hl*^=_@vA=*?t^k|G`^9Q#oVw0VyK|2t9kpQ+xc zTU$35S7&2a^Zx>~Yr}f#sbhXRSa_w3U!>c^((*72ngPlguyRsGb6PW#sYn@N5o5Y% zx~b~&;<(xE2I+JlC&(&j+)Pm!uS;8H#LSiToTMb2uthH7--Dz7c|EYnPyt{#N_R8f zuLYmC{#8GBKa?f>yBUc1mRlJQPZpdf!eL@bYsn30=Y?UmPa3WfsqWY1+`Z{*sFDZsyT<`4m$Nj)w%H z93fIDPys3kSISV?5fc%Jl``{G8w!w&_t&WQln=;G!7%*xo$~I>MWMonW;W`Ss04BMFh98VzOge zaOlc3QD&@}ZQ#c$IH^qJs4By8SA1V(n$NjvV1xT@@}VH^m(=>0nfw~!s-v}6rNF5j7&>?iXDl^`cY~0jKY0TNb)1#X7YYGZ? zQ0MV-)}URIU-+bQ`Vo~!<(o3NAd_#sV`9rr#=%SkXl2tLjUl3!NzEG8g?o_nXP`dHWib#m2?6uPv znV{2Sk^G$z$~0jqE{RV~_+17zxUQA9SxK;2)X~jhNq+KoY`DXN0d*WilbK>W-whX= zht!It9sRn(wYKta9(Sf}J9~#|^F`ilU*mQjKGPKm@6p(5)b&KU(dqDW*eA~{+Z0ZV zWz_X;dkbf$jzG#4nP!_%dj)OoP*>)witL}5=E`|B7Qa$~#-q*l8zz#FTS^K!?BZET ztY^iU#?LKX7`t4*XUt?ekS;P8LS3(PcM3rKaI+ZDrqTTgXy=m(2!Xmv4G}d5ujL)sFg<#fyje1#obe(w_wf zoQ3xqcG(Mw@umsgo+foad}+$AV1{F9&xaxF9cQVnE23@OWp4Iib<>JOyM@x4ijVQ+ zyo1rHw6Rg9k;X*3i2l=f-gcNg#cT+cn>gnwol>dk5%S9?x1yM3UP z#S?=@;YN8?#2_0;ClC?gEpWS~N0n|%!hfB=;QA{LQhs!)j4l>MbY=$_6ZZA9e}7?6 z?c{4_DLKgCF&wQkvgRo{Sbd3$-`)4X^RGKVekqPWxur%7Sh>UgP#@i`7c)LeTE!*| zkkSDPyMCbmTex!&r~?V}ea8Gd^ZW*@AEeW$eZ8a}b$>b$b8^Ohq-vG5kyB8t`RZ2s zJ5Y2Vvql4sTA@s0LH#$Jb)~#!XLecDnRPgiNm;FhaY%`BSM9{I?eYB~9b&0sSs?4? z#wCXCp14(4+7+z%lx?~;t}_9StQ``-I==R}(knZ=t_p5^o=W|9m8n`o_RAVN zIH$RPjGbvy%M-tkZLDW}dSfD%d*rNRI!%rpPGay=ljoJmhlPO}6wcO!mTPQORPD;q z#zxZzgYWR7)d}u5-z++1%NP~G1-QGNw~(|vS_odV3wiN`iCpZ@m$T|k;{(W?ZHMkMClR1!S>)FV@hXl62dtc zwFCAg>5V!R*o2KDb|QzTU0${5{w|rC=}hfZo62YSYVyDb2$8|1MWjujGeDBn0WR^^ zUWcyJuR4m}cpOg8fX!)s$-wNzfHwPM8)sK}bKPW$hLJVv&LU3wMXPeQZo*#IUZs2f zb)537;!BUdbAS?Ai-JEgEPo$qGqEu>o4w&Fs|hQsCxORYQ?EQn8kbZ+2F3!X?;6@- za&(qLgUj^c&mmVlJ`-*3m1H1MSph(R@owe@1HkX9zACFWy0993a=dnY#LEuk7SA!p zziY>BV}qq4aT&PMvP^iLOLbw7$s1%}DrL1d%Vl3K?gXIyzKkVPG5Gt(#1QnBP%KoM zHi2vtC#m@M*qJ1N?|r%C{wiXrCW2SiXfT%y@&=jLoc7;E7AX5S0a>-emVIkMl@KSQ zC*80%y!lcSytm145Z2amOJGajNSf@U1!BU8c#5NZGsj1-bHN%XtNmabjzY>1HQN%p zavVxTb_1SPa*^(3Ay^X+aUv<}-nJxddj|dGdjDyogd`Pm`2_5^_r!XH4qRDR#d-vjGIg3u2Q zgYzX+0lft>5+W&ZtD;#^iutH~>O6d2dU;Hwae^ULi_StvseP?w#5xTQn!r&UWT^?m z#LH20mdNHP7v^8epTkC-tec=0yi>kNusC{3_j+RUqfUfHo~29RBX}n1WER`h#Nveh zhAr;#A;9IGwa(Yy-(WsLcIM4#DnGc)Uur=|YO=dWzc$^#R+9UwO!AypNKS@djX#o( z>d{ZD4IBPG#-BjX<*&<`XGi}D zrrdq){a+U+0+&|q%*Tub;+!$(B-XNXGKlVdw;Xq4tU`x_q}`%SZmjM`?hskTm{Xs0 z7C;q?F+Pj2*Qpr9sRbH#vYFg5M|Z{pPjYfE=pGMhZ%1r~%Kloo@y#DJX4jiO0l6U zR};_3(>%o)!XgdW{Fwde>vf)5(G`*2`ot_}Hcg%^ayuIKwhphpJeQ}v*KW<6--RE< zLk1oGkO9aAt-O6kWYtwJiv^W4)4`qT8)1J7Vz61U5Qap=xrl~Lcnuqn%Zd5{*VV-3 z$a@G(2JC^97w6XRPRI5P_Pmj$_}lo(2F$qI3E}<>la`NQGnJMkv_@W*rO!!x9Z2uW zOiIWX;`*O-x%zFR9d?3`cQ)HFtEPOTRX8Cw^^sq3;)}66totqT+KaO~d`EgyZ)kV6 zBJKnGt>AYj{9^Ae^nulR?3kY;(grd^bdr)}T7&E&MFv-%-SKOyd~S|)_%+(?&<>78 zm`9x%0YEpFK9B9lw|duOxclz;8a~wW5pW^zz1;mOJ@$RNcRk}`;~fr87XGsD$0}v! zL|=S})by1lp=iv%TCe@>KVN+r84rDF$=&>vDSHNZy$=4nee6Gv;B1eehv_f-m?g~r zRMt+x+}YAx%)!#x!Of9^^}hxzG~`tArBFUC5}Qdvy|O59@Iyo55)-k}zCMI03idKd zh#*s|oMp_L@#@m6-MNd?g6M_Gh5I`k$4T8Of6MZ`OO-A(B@Z0CNn@67hA1Kw}kPWm%1aI7Yw*kQjajS0$3JLgnkF7TmlWEAiIy zU;w2L6{?jb*UqI3ZX=YS71o^}uN-%(wKBZ5-s6h|t>`!kQ0tl1vvK#V4Y`DC&DTkX zU!zH^SuIVdTtzp^W%nj4HRQt|o!+Fgl|>nWiCHf{l+bk+8Dx~u?y{^j0E$NHKXgbi zK$%_r3KF4%TXqn&40`OG4YJC5uVi{D5q!gfcPg2+x3(@#-PARSSOCp7o$9>~r1~qcyI%6NQV)h(I2_S=d_T(`jx^H`4YS!}Kr1+m-~E~Hdw#ld`mf$7 z!S%;^73Qq-U%sSfCSau@>#YJDy=S&4^P9@)XdA5nIBAa?6yTL5mDV?cGknu>1B=00 zOi4hhM(m2E7 zNoh*EH7v*{V(R0O^22Z200I)w3D$X* zdhzYzY`#YJSsGvPZ@a?Aw`7a{GiZvYj@^MqhyH+K=JH zmz3-R+#tq9%v6$XWu_-_(Ivwy$1Cv)TtP=ddOPmkd> zqDV*}1}B4|A<7^`2IGqzQ^v|$jA#e$DK^yDwYWFHUAGscX^1BWIu*ujShrYT^t#Sl z>t3kX+0?-A@;>)u$y-nj_?x``z2EVB&iA?DJ;{F%KG_1z&tCh1^%ri;8~8?4!aG6| zk|l`saD*VGtbp!$M2Xnl_!i)i36G8vzF^R4)`zTe<)$T;t#ebf1*T^_fax@;8fBzlA4232V>Ic3WIX=ZX5+_EE96hv9%&m>(py1W~3sEGv41Q(@SKkSd(ya zB{yaO#^o~%<_E{hjP!ZuJjKN`S7y3XghHN0Im>ytbs#f719L&zQf127FoI60>Rb%) z%vRgHML?9TRII!n+fqebsE99*oMFf{`XV*)OolnWl+$3UkW;ilBYgF2iB|9ep7qxZ z=FaTm*~j;#MMysV)8Q!BB!P5zZfy8zsJz~yMaev-B+g!)K&-dyj9$fzi35Ct&yO<2 z-QIF=@>*72I;OoMewJ1{NSb z(20)mu{ZBL{-IM`ggpl3HU+V6=%WSb3X7SkT-U90ItIrMEzLG<<%+0N4C}Kn}uI{y?#rtA+$jQr4)na+o85!xHGr;tIONc6vN{-SA z;J5MhjUQQldUzRtXmsJQsMSV`;%Z^!_^Uja8MJPGUPx-51Xw&EjMgzN6E^0`wy7b^ zt6JjgMd-4Mha_(>Dtag>$x2nXK;lR@^~TMKX%!?5I5$Gp!jG!4=-~+SvJ}$A>f}sD zpq|1R@RMgDke?#e(@gI%swc;*jBhlkQl%Plu{8Ht{ns!5<##mKUzYezCI`;Eka?*E z7)g(;8GnWUqarK@6|k9yaflnGE~0TC%^Wl!VNtzZ9n(BYHcoSqt$|+VEA*#fBm`vT zg3f*n1FPpX1P~=5_t>0S)TRx%W81cECmq|itvj~Wv2EM7ZJT#& ztJ9sgr)H|A-ueDituOlz?7i1I*L9rImgsVrQtqyvo@5* zk0uy#x}SIwBu~|#%}u1Di7QPcS~=C{6B@ltlIpkiVHg3-X*f++JghC*je{-RlExAs zMB)W|TlE=8d{lRj0Q9!l!ChRK%8onZ=CJ!Nh*yT;O%e@0^b_fN@^sbnYdbALkG0b-R&PSpA6!Jl<#hLw#f5X>X;Pj< zjN17j5Py;=c4J0se%zg|6?t|K#Q(^qLszw^3**+-{^*yA^ctre#9PlY(wjRw#&7&v ztxpj=MzO@T*>7-B)vWBel`n|;aCs;MVK_(W7K0A$xhY&?kqBv2DwC~pg#{|>> z33+BhOo`lqli6!S*d@PUjD1I8sUSY5X_I+RWh`BI(K~`d z3@tm&0MJG!W9sqTw5?Vx$kk`NszDhZZQ@o{2>~t+k63<)+^m_X>N~@>32$wE3iHk< z^ER=y&lztEIR~@Z3ajvu=85rBEkKq|nr3Sf14ebcCD*YUV-hwveZQtMODdX{{*-@) z@$!D5#tQizAscj}LciJduclBKI>_b|>9~E|NOxFL#GGggpTxG?1YwsuId4o;CvqHH zb<2sQ*Lh$$TsP!xsU*VnmRnpm|6hczfQ^at;wZbSR|Ys3oL;eE;G_JEQ=a&+CLKRTsQ>S~&JIzJQ|< z-!11ot-qxaR{Rp&jwI8PtnahmtLk3)IsuP=SSAYefVp(pwc{tJo-uLDx^j8J9D5aAQc&&nub z0e*H2Hsr6oDKyj#SuMHTRH;>L=r;FO#CGt_WJXjHq@UO~N-C@3`k_sTV4)&Rsdfwv zc=8vqy~H!l4F~%@ySF-`=T5;#WMN{*NJ9?&=mPufU_=EE!YoCZhf!ZmXv6dLB;Id2 z>jCGDg`M}z!ix^ob^dUT0xyJGB{nU49p&$2DW<`)JWyRQp^;q3wnIr)<6r!U3$2CW zoN{y)DYIPJvtyhx`gjgjP&O2-wxNZwD-xwS(lj27XKcBs$;Z&s@Uv7257dNgEsG|O zZ#Jgz_Vjt2Elx!{pMSNLtXUl(WzZ5;_NY!B-6&algiKUG)6*%Bpg+tn&P5vURJFk35{AeHz#-PahV9ZM5X3g*d=gU ze(ZN@8$>OwUZyOmHcnsfYKwOr2bsOSGhV(&j%*W(d8-#>+N&(6%4X*_Kh*`&!(iN$ zp50`XmRW6=c!Kq&G5?tdW@je*`qmpeaQaobm+z=(5~XrF(vI?~->R&IdX=pI#R)cf zZhGp~Ey6;TH6HCX_oGp~l2`r`{mP-N=X>*X=BhZ?T)yr0q=t^A4Oeh0QTR%7x3M_Kdb%nUa^}Xud|eAnwF*=@HMG_r4%PCs zQsvDnHIp{@H^banx@Nx@&g;~jnsC4`Fp}TVUC(CmhGdAe>&RrE-A=K|Beg~dxIZ>jJaM=&qnQ*9J3T^Vzbzc&sY%<|ak>eUhPnAy&EYUVpp@2i|0 z#~|H8jV~cr6xQR~ms;1D-0?nn{1xPPJ!cUh{y53frwRYiQBz8tNBf6CPYr#F%QoRn z8L?4(Dw*;{Joe+Bej0r*0BxFbD;f?a8H}ku0^1UzZyKefu zFYnu7Aa-YuSX1NDkIg|ket&Ae-MIR)pTMI+ka1u&+ZM3;ZMsvsx9!w^xxeW62f-Pk zV$7q6Zg+onU%>sF@g-c3_|b)Yc_-SStTVpfeF%oDzdT(tg`3Xj=Sdu+)rq=3fcxL! zzMeXg-C)oO-yyZrEWmZRb5UIPt&hT&8z3Y!E_!AOO;}4Pe*TkX2s;|O?6DhGc_QF6 zdzdf92%Yfw&q;uk{5av4m=imJ(CB;yk@cUPe?8P!Bx~uSt%RpI3jCmzrmAZFMzpeH zhR*xNo-;`!A1L+u1H1Js?RD}Qrx^liy0;A2vrK+=O#IWw-B1wGTsHJ0SEQBIJ$QTK zseH=RwB@ENU2FY|G7<$nIT5h})XvXFQa}yd$!h!tG*PZ$_0HVwVb0CiBM12X7^vE{ zApXKJ?OT^eOKBttyI>zWhcpIB9q?lGp!tgpg#0mBoZVO!Hhum=IEo?WtL-z_%@FZ! z77fuzTekY9H(EkM%_C<>cheqt!j;%1-1hlDzu@bb-a2LVW~^gxmve2rV@W~tGzuD(%hYGqN4uh>F+4>e{0mdcCxK+#GEvhQVf$}pSYA~ ziT7r7&BzN-Z|6y~l*F6JRc&(Z#ad@;Ly`LTUx4M#SQ4G^;mFSZ=FOPD)%;C=DL%^j zx;O=OXgsV}+Z*WY-W%#ob%F1t8gq1+yI?V?%O??C^~SU1{UWI8W3Xs@)Tb8X)w6`_ zexv|D>@9cTfiq{m-iyqJ&70)}GdG~w2%3Im=Nsbj8}q0GUK(-~4hOYAlBi+ZU=+C{ zGS{6=bdBRWX1z(*Faitv-QZ+Ti23U)TKLX1ymuae=hcw<)|*=fMC66_Sfbw%uWrcY zZ3rzvTRZmp2U4!f($^ z!4bMFFh}d+hc-^IJFD>nKZa1G7MFX+8$I6|{UF5_x`#@4j#nF9=q=8r7RiZ10ddBg z_7YpB1WlhrSPV0JQ+xCSVhC+q*$=X>sKNEgrre0}2XN8Zex+zbm;CG#9&klxS-y4zWL>6v>O=9rfXD9Oal@f$&K5_YAV>qI(@Nqapiy2O2AWsX)zn)w ztav`hqj)Qb-7KyO`%16S$R!+CDexnFqqA7#T5qsq;2XSA3>Ok>`nD8|B}&2wH+ICB z9k%t$K&m)ZP&P9eHm3Pyl7EvmMg+lNGVxITYcB1OCXb2)NN-_54pwWb8#d}_T6!Zg zS0(8wng7HoS0mN!V;&A9YQpJC9{y#jyE;s*l)`w{Mw_qd)gX|9X%#hsOnWk%Gcl6P z5>sZxo@$J$ZbX=EM@A5A!7B&j)-kHXo0S{3Pp4c$(WqleW@GjgkQIg<`|(cJp7RIX z520Q+*hc8M71rzQg%5B}ICDN`>;v%Rip$7NN=GrG1X_G(z!CigjbP1&_RWo3BB<%=|R@YY4dGgjs;jY9d=r{vmkSng(*LS8Jg`-qAMz< zGMvVO$%r*fp8^$-RwpXk8i%l8`5rkpZu}fRyqfKSG&gMB8e*@+eThaylv-x^0v~tq zhd|Jgnm7z^=$aBiZE2XRVCxcuTw{=ARz)(S@@s1%TUEry=*T48hC>uOwx<=ehK`@w z_|HF7?leqK91xBN@^dZRH{<`PlePmG?#Zkm8J(1=J}bW|gU%OC^Lo(>yPi^f7xdbZ z{gmD~OYJ=itGnsGnmi9ydadAzZqNA2&EN}GMAB|J6(3GdBy`^?uHWE3kN%lg}Kr( zmTm(O`W=h5ABf|HBs{8y`Eyg72ji#-!5h&7EKj-<1|mK4fA8Qc#t%cG!g?tV$Eyk~ti{;>M~XJWEZQf7=zjJ<9iR zueCW>j0l^Zq%djPamwWfVlWM@7$)=7ZWNL^gknFEr6}t72BE*fCx0kp&RJrL4UIts z=9+{C3m>H=(#lhM;4OFS})f!)DZc`mmKPv@xy-f6?{E>ZC)nCjk&3eFV;Oj60|->}lIzWGt^ zSx0BT3`RDrfj#a;c#}sQUq-t%yJ2pRn>6t+8v92MIn%WcIqT)r?es&NJ75<;z9Eai zieJL>TQy1htYtjy=Cfa>V{!}vPk}p`ao3rSr@1^SPg#F$o}z+c4i9|&IoBP{D`vAb zvNdr^^pjk|#qdE9Th{Fq2*ka_vEnY_1lN>%rX?Tten=`8O??ZJrRUi$cgGy)OPubP zGWw(=d8P*(?~nG4Ol*Yt)G6MvMtMAxA{1<`)s&D;H>p1vhdb+ApTw21ydLvp(lbuC zO~z-ADnc45-%5L$;>rRanVa~t6MMaIa1^0VOwGJ7$#NFE+0TAZsZ6~DF*^hNd*`Q? zy33w)@<2kj5{usm{yf(?eKIfZne(}F1i*l;L7a30jIqk!dONH(7l!^pQ8B7x2mFYLt~j}N7%=V|H$xqQALX(pRTzVn2*QQ#KV({YGHM8> zTaTT%<+7uBbQmsMcoFlS8nm+WN!F8E^kl72sg^MVJ+-h-a_Rt}a;iM7Z7tuB`H^aL zSl0s-oN?#@>H{X53a>91)7=GbKwEh1-S@6v)Y5uLjP?`6-{D~lqr2}GSt~5vBKlTx zSH*bKtMKty0nw9Q05vYgW-FdPbA3B&DwCqMKpT4$+*S#8TX}AA+p!NF*{&+%ID49E z9#?r)O~7moTlJE3ZZXFi0xoa*VaJYO|7Ke%dWz=t;K}a0d;U`Dr1R2Z`Al#ND_lh| z?eD_IG4ZmDaaWr&50>Tmf*-9cv!gbUf06$GG^9QTS{V2*4cYsbh8X|n(vbhE5BcvX zWLMYAS5^I0=l*dg`c;Z^{mq z4Y8siW=UBuu#hGiwquUFc&;OcH+s8+B`%IPn-}=bch;BAF27tf^z|3}&vUQOUH3o5 zWdE*nvHY*gf$5tb(0&;(h`WDGab){o->YCg`W`e^x;$% zyb8OTJ!SshbFX@I-8% zQwNlW@SZuH`9g5`b5Wn4;0QvvQLawlb3PRF;FTSqVvfz%ihpU15@|ej{b<)ghkxWt zK}f|uU4!^y7^=0x9BW9`wS8~Y7?5A@Zfx>Bpq+0;ZJv#~N z59U(cd!iuvw&Kp zgPr3*$eoe=fPwsgiQVkX5TZt@?9SG({=FeX$CEXCyMF`dZO$He?GEigf|+scuZ=mG z-drLSf4f#-JOTM}?pMcj93Q%!%Rni=J@em@AP=Ab?Cnf2-Gc$tUc}oI5CqE2gXLov zu><9;q08T9yqACJc7WS`!zrKO&E@;Ye6QMI2>*pJ1d+%&)+(7lLxWnp*q_3Kq^eD5 zZXyy(4QlLI>%k2bhJ`BK!FMPW`XD8=;BH=YYufdptafdtw{RUDD|a40>qn^UscKRA za#AynPqM`glYnRi^z`MS%P#6*5An;XhW^SP;itN_kiJKYSuEJF=@+-Ib`FV}b74{m z)vR^NKL!j+wYBSHCny3150VTG#5IyFTGXfvh8+dGVrZAH45*92^1L^>1P@#s<5i+r zit)r9OYO%L0)d|WXoMJzuGfG?MIl{ zSE(3neNySNA<~DOX)7pz{8?@lquxwijVMmJN%;tOtw7^H%NW>&<(^ESL^O$_cSJUdODGC}n9!<}kvw&XKM_ zbX03?W1(BA(r%vRFGUKmMf{o7FyJpyPR)dsxPheo@v+zTu?@O7Bqr_?SR?37SVIX0 za*+j|qP2tA5sVtj+?kBc69FWP8f26x#o5Ec{dv8^=a%Apf5tgu-D78eTG4#kEb6D; z#v2YsFQub1i6>ZwX(9MkORKlmG9Y-HHLf8e9}Z^PR7&8af1VcL+rB1G=lwR6isA4s zk+wWZR&nA8Mkm?T{mNmvu6}umUXS2`f9iG?523U?Bcl5eKVQA!MdiLtYHCQHGWEiO z=xT79(L7fj;Z;q2gt5QSL#3DV9pw5bKQqX>gB{xw@{wcD9lC)*RGp z_%OBl*f3r%Rw*+{?4*OMMJF^{=PitMU`a!Y8~I;DDr!CW#zvo(u(-f(Hshm~ zr54#T_I6_wbyj40~8FRK}if>xzX_l7!1x~Obmx0U3IGvK6P%;2zAd;qWDK3;zr$&!rv`K zC(yTV`4L#o<9c-E8b;& zbp|cfd@%z`59-x@5#iK)2@QX87ivk1W4L5QVyS;22#QOdat-F0Z5zmE=TscYa%+vI zKVV=4ME3(@1L}{6?$t32hu3i&T&3}7`-1ApC6do&M!BqM{w`|0q(wOb`Dpv#@+;p7 z3Dpl}Vf8Ec{|-lW{!WG^8m5BTb`#g2{et@|zuC+MC_VZOpdy$PFcHG|jv=bJH+|g{ zJ+6Ew`71v9uU89$FgQZ}J!K>~e*kd`BXc1U7HLW53;et4Xy5bi)qQwWfMj;?x6(uY z^}WX+B1YTb4fHKLV;3=z!UwH?RKBvm^gKptXvX=7QP{#+6z^{J@AY%R_B`a}-nPu( zRf6c6^37YVi5rB_qAuN>+S`i#@@~8fpD^9g`3FPVzj#n|?_p8j#;bFg@>77{#kWSk zPE~ZshQu0OZn!--Dpv?}@tns;PBL|oveL=f-53Pub`Z;E!ab0aQMZNI)cX{qsk*WP z6(o{SkD#EN`yc}Zl7f8>1;vxpCP^nt4hV0}P!#@t2C9#uaLa{_AAqPg9WjR*kN>EG zF7BGY4-bw#)*O@wP_-2+;Sz7EOd{RXF4U&onSUlwX_FB-u23Pu9Y!kct6EWd2 z*}786gZ*vPPx~VIxCOR@OMF5alZ_S`N_q=GZO__zdZIabZEcG0)a9P;?(8`r$c0wd zSFOrEsE+rW9w`rR!hYD;SzgY3l?02IoAPp5YC`MbGVfuip-dO7SywWNiq6LCPG#( zAtUA~8$#4ns(=uHsT9&8q=r`}>Eh1O;uOk|UF0N4NEfSH!y}f3uzIo9DU~*QswGCL zRcbvd;pfO24eFn2l%j4ZRJ@86V`D%OXIhO0JZ!~1s_R(PB8;A@n0=_sC1Xb-H!%#aQBW;;DXRG+;CPj1;59S4=_@I(Pf9(jv(h1-7;?0VAxJenxtCyFRjAGD0}51OPfW6*7P6M7+`!PD z?l>Sm&=>zn&SLdLjYTg~DGNkjQ@t#P;j~I$&*E27vT)Z=Y~S4(@491SSv0zF?UYWj zwBj+005A~Zhi38zS=Tvs#wHa0%v%u^S3Wv)=!FdBqvnNEdGqfgxxKoqN}UvxuWqna zVB$d$HU?x1B;BTSs(@afSf~|w{OQvRKvI+=Mxdge!x1yQX`=TJxk+S`#bb08>s`hP z20+RX|I(PRDQ9?nYpH2(XJN28LLXS|J-;%Vs;B9;vdMAb_pC@i$T!Mg>JK}noq zPCMC!kU0@;j|qn(TGFtm3Tj{3oLqLZxJtu%o0vbD2(kVNQL96|<#uIe#PG~Dd!!mK z>Rhn@ez8+O>Ke}O8)CM@n)2d2oiGXf=rWnS?xZ|PX==H=(vE3R%pDN4fH-!nl8yn8 zBNLl!Bx~fo{)*}4CCx#S($^i< zEbtP?smpt(KHWDj{vxX&l+{b`@a_>WPM)$Xxadlm=CNJDJ2KN*0X`_pmJ6re!3$(? zC%@-NU8KFb6Qs9g#7jI4ZM-iH3(S7a1o0Sxm+hms#&^KWf+(I5?bmKJ`m3JIesJgD z+}kdSnK+B7#I**?`460xGTLIza3AF7^Ib# z5a1JLrQA3z(yyBOK|601##<6S+ei54g(T?SZNh02B&;L;v5uSG0s+fNbK$bL2+Ch< zM*W20pkYRPG~$F04O9AI%qkB)_7i^UfR-DlE}RB{AQxe4jHm~-de5K*vL1rv5NNUS zLx>8N4Ibox<~{6vzUu2bR>!i0~V)B4bl6F0*8Th&e z1N*^{1l7BXKoSBaC1aOOSfJ7dq$piV2y@L|>B!)0{NWe&p$f0jF-dx@Av0pJUfa^p zA7&j=5!2UUE$36d!9Au8$htYF=uPS-{GK`rYSUuoO$;wm-For?A3pCPO4oBGgYH>`^9-DkV&8iR z^-1uU#yP&mL%Bobk=pZ%T3ylx#jjWQkgXHsuWaYKnDX2*th!vuA>)NLZFWpC+wf~m z`gyFfP$}z17l;!)(7-GE4G^lE#}xO0Nqgps;GBJmabpv_ib>AJ3&krcqHvnuhv+bA zPCv8+N^h7s!iEec_G01}t4-+sk(YViHq(74H%i7j7O;tC5d~%=POa&n7WJOn^YOzi zu+`>Gv(~`E8|D#nw=KP<RN?tNKH>7og?Ze}d|n4~xM%*+llEb77gsnw#(S6<$4=TjrGUMR%slGXsQxN7 ziguXMc_Dbq+Xx%-06hJ{sw5;4c|WL?Q-)u|B8Vx?=L8X5e;R(FnGNB0I^UR4u(+|v z8>ayq$Q@~z*u)D3Yik;aLJMCQyOTj3HVaGIug#DIh`*-aEgPhaet4L<7pjO_t0I(6 zDtN)c-<M@Q9TG`)L(*N`r3}eJeT-j?!R;*qAkF$kB*z^Xlpc*w&Z72hCAs#Q^z_gf7AiSY zwLRmb3g&P*dAC4AMdb8auD4s^ZfP=i7k3{n%l5b9>y&#LBYuiKm?J-2xgoN5_P?-{ z6K~T{#}QU~M`BFU03M!ufs~Rvh}56b0m0-7!DJBN;^C^Gj3ggp$;*lbE&P#5x9m3g z63k|lrKw&m#sb3eso;D);QTt!4kOSydSV48XoFGqDZU=dPWS&1Y|D#3q`6~`3p`&u zUrEky%^3(xMnax2Plx9Cu=NL0?+kH4LPk72QOO4!{QugyyzVL226VNM98Jl3M9Z(b zI}rDVrw{#lApgefhalZU-IMHr5*-TMvk(Ra94+57B|WS9A_|Tx9>9HZ1&97pVbLj( z&Y(~)5aq>lDS2v$KOXK=L3SyXWmfD$h5Bdj)9i+WmELembwTMb`Dqc~42oQM;Zyd8 zla&^8OMGIF%LI&)UJz-K`b7at<{!vqK@^%&4$P+6b4)TL%e79A)TTxFcb9&preV)D z=>;`2?2q!K#d5DdlJO5aRCma=AIxkw(-Zw(hCinr&%`cJz~~1 z;E%c-+v(9DiOCa&Avm*mpQx;yBsWOs#mzqYK2J{rQhgEY(m3EJVmC;k9zc1*Q9eXs zCf7}?dmz>)wgI)iRCURB<#ieC(dt2Q^`V7XT;n?Q;MVt8TUtLyxE+aq?*TaQ8}6(%;?SJW>+5&^N0* zQVg`*VsOD5tvJqiH08FWSfRkTOxRN(^E=xOiL4Lw(*tbvb5G|GWv*m`-aV;4X!=6d zFtKjj5O+vZPHq7`5$juPvb!brvs63bdx8hqEMl48Hz=nU`UlNjH8Z=^Ph=ZEu^lq) z7p7AOj3uzLW0(w(4nPH60^ z(7St z;);9&k^6)}w(wU8Ggnv>%e7;7COAYu)KSnT&c=VeX-%1a86c88AP$cn`N6CbtW!Eu ztf;ty_aLC}!1_7iJ^xcgk=v7H?t+t@pyWn5RnB%B@bAJPdysG+{GovkS`j`6s7I%? zzQ}c>U;u)kzjp`2BD-jl9-l+R!)FCSq|v&l>U@^Uc{{?U>nQ!#v+y2{oUw}+5flAr z+5Qu39|6=jxwiej5(YIQgd$Qub6lJYMF4FtN=3^cwulr0N?^tAY(sCd<& zN3$dW(v0Vz#e0a{Mrs|V!GK@Ea`gRF_no;blOWU{Fv(0Pa<3h(MM_|8&iTQ~w$n1G z8MgMB4c}d`o%D!Tv&s9ZwuXG0SsunzBPe!Mh-TVG+xpTpW6VS2mJ{6bB7~*iB=5G5 zZ)g7=L}~hd03XCN;+WtnPf>_U6Ku(`dH0-Sz=wM%C~C_<0TBr;xXLMUu&KFvqgoSa ziXxHAGF2oC!A&$5(_*PtS*J-K;GO0}PCY$AENK!J_ z`!on85(O$G`zEcT(n|KFj?or!SDQ`F(B@Wx@gYQnoX?9>6%+bc&Mou=&#^&tL zOIG|VaZ!NzF=KM#*Gx;!MN8B#4JV`)_skXv3-qZ>Mm1}X!p~Xqa(r{%6W8QKdL*8Y zSf>ta4^2?^!sPfGaw2W+EP%geK0cjq-}(Q=;uAamt`5Wf_~AwL zGk5m-KMn0Lp7`cC{@=Y8@z&X8XK5vNve|Z=G;s2rOPmW}3W}Kb6oMiGQJQ*~_9#6+(LggNTV?Y14y`f{Nb%HaBO_-cpKfZJkZd2z@{J z-FXCjzaDPC!2v z(uC;#>k%I;;j<}bAKe*|%6-Gm+#A{B_1kgd9o@U|4|z{1(CQQo zkM~R@p>Za3{<4E~|CS?-&NC;kTNok#Fe-|Va!^m){5{@7iEX2%DuYYpVyh zu1wCP$X0@Bh!Ja0U>XJ5MNHa6&GM1?wuAHedS(z;-wUr%viBjflG0>JBk;>P5b|nZF{jC_Ne-sx= zj;&!DvP0aG$Pc9fCd~s?A&OZSXUDa@-ZFVIbe;-EI-NQCn{oyVfoYQotrYD8O_ybO zB;u(X^<<{D`cWL6TfqZGw96QrO!sumy2%tebb_1&)NL$JNvz-|-jN39h&fNTjTyN! z;Bx}Z`PjW0+Ie)<6$9E439$oS5h_uoX-kkYvOpD^HpO{YEWFSy5u>y==*g%_(Oqn6 z@K%+58C6>h+(>I2hOD>PGO*xA8AY})a;{zWC$}x0gxloSP8kPhi>4Z>h}htGr#b4&rt*@l zG#)IwMx*UZnGLFF?c-Mopw*kqyF(=mu^ZGh&QAAavD4HHpV3m;ulSgcQ65Kji(m6y zj%Svz+0~uYqn7as**3=YW5r6qz-tp=Ct$N8XyeIn&v+TT9a|FM#%lX|S&qHry!+=r4`ADv3qW^OY?SB zs!c|iY?H}9!e_>Ylv{d4(L+xZP{KJ6GYePwj%J{IUlk=}S{&7Xz=07ER(S+pqjJxM z5il;K))N;{^^Uct)CfPV{)P8#Ymqt?xOO3bJga-AMEYo}tIlcP65?8x8DO5^537IFnk?bWHzDd}%KFHRS z3{~NNjGswQyNZ_2mTw9x?WVnY*y{Hk1jcMbzhT3 z_q)(4n$~fIHIt6MtVy)=dKeCg4lBA~AjO9E_(W`zKA`quONvFyvuaJ~LeO1u05PpM zoz7mbLYIgD)a@{CQI4fHs*Wyq{GlsE$s+C4%%PWLu{43zec0Z3 zBrVjA31Ajp5Q)E2guvHo6FU zQ--HqCZsN5+Ei(_uH39VjH9Kb+NKdDY@)}BchT9q&){i_3^JS9u=-<@_?EkwTB~B7 zf5UNIop`*d_Ge8)mq&Kwj}GrXV>ha=F(#%d;twW^0s<=PSD%mMa>Qwqwl+Cim!)Pi%QZA(b+T4iFATJz87}e!#MT#0mfg}8{Fp8bp3_|Ci~I1o^G#d1i=8Oz8wsOL>t`Pzon0SxWu#AU z!pTr42Rota^Z}BugU!ZVDf*$R_@AVDkCHjv$k?D<$qj+_zImZK^x^{YsHao>u$)Bc zI)P+foZT#ZxQ|8EGe9CRPX8dV0pIHN5(!Dz{Ik&In-Y**F!5( zPmFSSth=7J7j)om0wDgaBkiE#Mj%->sp5h^T{4FML7^qleW{1|-NJZ}Cfh%QekH_& z&2A_Uy-Tq6!8``&uB9i?Pc%U2+N(_4=T^1Uw|g;_@ip}6HGAA2&!Ns%iG!%YABo5Y z=-xYIn+Sc#f;Y=j%bZnKOY1?6b`mhI6N0;BH1^>??H;=jp>6!3n99;W=GZ=^ki0V< zkipOkjkebnv#0B#Sa(kMEfIrp$Tl*lzXRgy-teQ0gVudk=0l=prVSfMUu|7EO74o^ zyd+&3tia)_RrW)|QF+e`uBk)EXTpbm;v?20oB+xDlhI=0INt5ENx+fC<&=Q%OlpAsv8IE?S5qCr2<3yk!-(L@MbX64~zCMzRX}k zavWzVn#Fn+{6+J7K_(INetH}@ZYN26?j|Fc)9_9`SK!DL=SF&tv(yyHD@bH2=$rHD z1Ipo>7QBY@=9y^xiRv&my%D4dz`9FDy662sc1(_k4WbGI zn}4`{rgfC4hk>YnvN%Q-L_w~~ARmh+cBGN~r88YbG{XI8H41e;#^{VYo{6MwZNw`W z6Tk3TzsMpuh5aO5jj0z|Y_Dg&h{2XAR!a9FP9P{j+ta*=P+Hvi%NQ33>UGcF-1cvXPMo%FV&e|OFyN;Uo*ms+xcPY;YpqHtbw4b2p9r1CTn)91o&~$lo ztX=Lt7QPe+q0w?Z3KNV9$zz4)S{$C5G|Z2qw7u`!Mh_C%nIY_iC)S}wXQ70XKYG?t zbpWuCUyiyMGs)pd>ac?gS4qt2?ApNg4J!Uu?KOIzgU>Tlr$><~%0A4XBQY`ARGC2g zX~Ke*J_abD$BDK|#m0PBaU=fgOk4}zWCTu3XHUGY4spZ;<*Y}MkVR3WV%UbzwgGt* z&3-pNgpYZHt?rINzd>Br-edQQ#0k@f&&6UC^HgU-2KMDiop9yF1AK6JL7`s}#{}27vs&4PHqK58&+vk{-*0oSnp@qD8 z-9~YMCsU{en>y5{uiGVp^5X7e$(10_D&4mHN6|k4M;|}e>1!t5gr7@#JxiX?ZQ%C# zv@ho==Pc)U&hbh0{x>)PVjGqcpehVwc+yxv9IFefV5~gnfCVTIEG4b&XFRC$Wu)tf z_zkfFtCe`A4q;m%3-l)3mzsa!=X~@WV7`S}Kd=z($TQZ>o^Hq45C~rs-+uIU`1m&& zBaV4UWQyI`;dmXD`cl4QPQ9Aa*JTD?2h1|Yh(SCxWv zPd;~%30_;Y7<0r!51@PD+S=;Al z4{^4vr@_BCIcbzmZS{5l5tWX4cB=ZM$M<|`A#h#S zZKKPEr<|Vmg)&Ni(h3fwL_zjZ*gW_;xFNVmb}XG1MoBKH+a0oSC_(?!(&6dZrm12W z{5OFp12FpB1@Xf};8V4*$Z%fQ5ley)yh=eSvKtd-!72 ziX^A$e-0smgD|3a*IN+s7Rm3`D=7H5N(v!iVjp_bVuZT`_wti#o**Kad4{reaY2Ke zIO$-sL@qMF(sAkMC6jW+g7su8A6(tE5B$jprL4`YT*RIWJXYKjuNn2b`+20xGfh?j zk}R$zbL6s-XSN1Sk0@Cs&--~ohPB#xjCa_WOG_42HoAbz>IIh3JXroyQ0k6AJiTBE z5j0!F5QHzf(Fp*>e8@% zC)kA~Qv%7g9Z{!>0U`u$Reu5XnG8442SSxz+>|l>4&UN9_w1+V)fq^&ULY?Tv&t|u z3R(Dc$hmg{SwgDv(Hs#GU=A^-e&Mf@Q-C;&LBjmx*(C`?q9!F+K(Hd(ub7cYJz4Tc zoG>Y3R#YlhML{kk5>gH_FvC$Slr=dq^3=aHVIr%c|G<7`|rHaD{rM%1dMz4>MR z-JiKeb?3?;f|+#O>jCmfq= zbqFPP7l?<{^&pI70TeZPEVO$SydD(BN1Yw!;f|4X27nWx%qnwz@@K?LLR^@F-a0Mv zUyeya9HJ4X9GC_l08QRugxzLBfbS)E`Od{c*+=l6%)dw$bC8#E(b*MY6O6e88TJ%A zon(euDynQ{0L@jBtpF?4SzeeWNSIFB;lhf-(3n{n=CUrMKhmhIn${$ zT&TO{*Fj=5R)7PC#QfxAV^Kt}mR_rZE*tvt1~B+<9AMzaJqf4+xlg0=vC)MlQrQ0flE$P@L+~ZK2gGHnclRqBcK8 z20JZoVn^h+(^>*nL+$8CO4V;VDdT80RHNZkGBQjVJ>IK^4hw9S&ADs|NJf8MpkKd3 z&>oZG8|f{<^>%l)R#-@Fq>%m&gCTi$#7r>W9ZBz^u*>nag2{Uk@aG_P7vFt{2NNuD zU4&$dmX*kD-CSr@f~wrT(a7t~3}!znm0*TTG}Te8VI2Gbv%!!JM3T(~`oj zpE@ZQqry7)-RWs;kuC|WsL?+C5=+ptFPn=eoIoIY-lvxW!or2S?DwjT{PCrBrMDq@ zNn#_Ue35m+1G^v<@TnsG0U{^yLEKk*EWI%#JDIGNwhRKSit#l!k(n$=PwbpDNS_{v z;~OTyaM&zAruk5VXPErp#y=i;tx?6dvMRGO^QV4QMtASr zbUV=bag?5OULKO~;jf$z^O$}@{HEUEf2c`*^P<%hsE6U1!^u8d6B4#cZeqx>(a7b?88Y3bzf=h&@DAEv^vH(tR-5p%dfO zNtFse)$7HGa+)*Med2fxVyVd8nTn&}t00T*Q@W&(+Lf>CA4?UTf=N_5m(zA0s8w83 zk5*e>h$1UyN^gr!OSwtRaN9A8Vw+*GF3Y$!vl}~ChMt&h^U=*3SBJbUNRHO3k3W4b z(-$ao>k36vo{BpP2LLY%nzl5j=M)OE)oJLfjH7=KDnKHv^A zyG7-JV{L}iwx$-pEAB@W{nUro9xyH$3vuMHfYg#wYb7xa`7PJesDK*Tk_~Bp2K8%7 zaGCPhSR){n&DtSQ!a?j+8^~>9yRuX zrC3*SC_$o|1z6iv2qlIR;IdDwG(M+8OT~wjsE_Feh=f0vv-#;xjJCALG62<%hHeUT zl!3Ch@!)Q$1Zx=#de9D{a4c;4Yi4QSUL+Van%8SACyIp55Y-H$8aGTDD;Oi@hJ^sw zN`ty4mjL?`2ds9J)yJ;^*&b*C?21|5XDI`c;sp+Ao#6bYEZGBg^rHmu=nSQ%`QIXP!T4q^?XJ-6%y zbupzRos#D3lb7G7P#c#PPl`~rCS7EE$?@LpH{*6)i@a`8poYAt#THe)L}+~E2^yz7 zP9l#YON(j4SCZAQ#E?avakiqfLttLR9-cOm0Af|hzZ8R1wtzk={N9buDYc_AZ-P5T zSeY>vK9YJ?1a^3Y--95fDyP33s|Ht?s#whtKotuylV^-xuPOAxatdW|D`}dac$WHL zv;%kqDPcG$Q5B)-zU=`?j>Pivw8@!Qe^^&p!xjplT1GKC@elAg3nFDt2(6u|B+g;PXtm@|H`K+5nN4r*LfR}WW zt?@9e5Mu{8#gs3WDybX#!byT2`Khi%^5UgE+;_c)~z%rJcUhHD%VV~wG z_$v1)&f*;VM|=*h=tR{ib6rRwiW({P`rhf*-R#EpcPz;%@BxJmV66{P-Q)|2?r~>4 zWa9-&pmZnLol#SMiYu0li z=Q%j(JxPE~pmq%gwGGI4c8N>sjNw&rWg(J1^1G7UIawmRwm049zyAkb{#Vb6dZc*f z3JL%K3IhP({2zgr|IK-k$jklrm$Hk!wTYmMsi}$Mf4i@#PB@{Mp#1jaLh?!_#UP}D ztP@R38)6|KN`%J)a1myK1d)XME-huSNPCXv-lXrj?ic$7uv4Sc;VlypXe#D%xIZ0v z%1_mF>!-BiF2A~jK}v`YQ{CS^zdn22dQIKZZsUL5&#C}$-z%bA+LMGny9?*y#Sq2p zK*WyDhy7rHF<^`!;3+vsH-@R;#1Yb&gx$3lM~^Q-dUz7WCSk-7M>2YFl^jf9VA*qx z0?A1{EIkk^8!KZ_H3r*@9!UmwHuN$ZN}lCf9Y6C`9l0VD4yn$f)t;sYzm$B<_68;` zi6~$om+Ce9>+;bv zc{mApun=n-8Ln}Sqn{3KO zyXn{kQHJccAnO=Ba7}5<`IO1m%d;nADt+1xA|9%z&q_Sl;N1^o*nR};Ld`N%YqZY! zl0kDK<<6mEg3UL-z4O*SouArjvvfWYr8r?+aj*?tQ<4aLRm4Up34GGLNt+}KL7G^k z4fH}32{Pi^Lu9a-lY&?<+1PZ#vdau5Qkz}6iNZqY>b}7cwfLL}1=chE@(I?5*3_!P9pQjvHPYzm*IPKJA3$QmJh~YO4-2`C%MNh!WF#N_}S{9^Nhat){>@ z=YdS=b>XQaa23y=#UC?j=at`d8aoZN%sf%Yy*Di(B8~GRWCQcXf#Ci3Wf}E}V8n#i zz$aul+D}R4TxA|ETlNe=qAVSt?QVESY0>J&!!XnvUEkfoD6<&9lU8s`t7DXdSE^JAqLDdO-%kXCVE4IF&C@>>Q}T)~z*#2Qi$cl~U>pyRNYJ%A1?=87Y=M;p1N6vxI0HyEJ;H3+bKV>W^LFdEGEJZ{33EKVPzw zHQ7QfTUICXpQX1l*!-{hq<`=UCGm?=`2>i4B4eH~TNh8wOX%yvdWQi&puW4|msT-~ z?DXf`+T(6*4gJ^!g}9wpTGJZV1{|oEz<&5QXSnwPTM>ravfK`MM}JiG1aSAC8_}sr z!hG*qECZA^hJva!60pi6u2mSXQ{>H(U>+`Lu#R?`FQG0VPa}lBb0LSTT;W&))XmY# zCdu?hh^#-s&$?5WCk=T+Udt25) z30?~_b-H6YzQ+l4#hAZ?ubqLp&&{sN^ya1d4C8%=D8Av67&8@7Uf#fTRf&ApbYKDp zNl6Jwo*H|@+$k}PA-L1g1}V}c$zPl@NC;1kF`_;4JKaMv9vJ9<@~4CNX0eO~Ter}d z4i?qJixmQ^hY{6SSRQt&m5rJd553mUF17SpdrPuQgU;X+Ol?B+RmyW!r;hT3=OArDHi2k9vM0RQ(pkY8URWB>#JfB^;oVD+DU2Mmm? z49rZFJ?#G{!$R3c4p{)fHh+0*@B z2t9Sfy_|%^usMtOix{TaT$Bayeb%;=Dbwfl6%TV;6ThF&H=sUl2!}F{Bsrrr8H6_#?*n6T2^$NK5r*G)siV4yxN66HuGT^RDjJOEgiA=@^_{GkGe&je zsTP=>Qxj=)s|}`%Bic>E!hu-sD%&u)fh+K)Q;5|%T}QJg?>f#%r+M+u%OZk!%UA zH=euiSh;GUcSbyiD;V2_DgK};gSiL zX`k03$`7C&S94K_Gu15ypXt=tS28x78*LHVTdNhS6EQJ|(hEiHtdG`1 zVDv}3L6>X&1cM*!tzxb0g1}TM-?vkoJ&FA~IpC*blUN^cOS+v|Cb>yjq|B7g8k*1~!yB zQN^#r5G_p@M@#0B_gg>J6X3?h#setX>V;jkO2VWU*ac*-wW3u6%_ zdUy^o9u_1kOXA_~gyIJaK5R>z@iorXm86TL^9Qi8;5WijBYtq@xPhhBJ^TNzbZUXc zBwXZwmCghJp!c7w^#32+%xFUED<7xunc;OXCuKnd_!EbZPfd^z24b*6QtX=$1d9WJ zqO?ock^v`WIN<&9HU!;O9T5Z7QuZF!l>dViPN7(1-cq-1<;9#aEzJPcUUky<`~CZv_uGx{=6xDRk55`?@x=>YV-go8c;#S}N*_ce6c7ge z4g(K=xc{I@G6aDsIw@a$M*&~Kqp3Tqymnc@?B2Y`C1q-)&9gFMHg&=QpEq=<<6Rp? zKK?0;F6Ve}oFRKZ&&4OV$JpE?w<5`Nb^0$xj8Z#S4%wq}5EUx#wt$jHW~FS(J|N5N zE`+Z3=}yGEA&UOq9#w5hHYUvMfrit)VlWL-qzSe5c^)RW=uR76<$(u&?i7?syVgD@ ztkT_pkN&_D6ODRnuRx<)7>?~^$+=fBo^IxpOqZ-~%ETySP|ef4*PcJGF43R_hw6DA zN;gGL%A;jat;>Q@ujmd7o_4?Gp;6XAhAk^(5UrqIV_((d!m(b)6mZ=tjXDTVcMcn6 zgQOFZa;w084c0@12j|$zBs}8n04OV-kTTgu**%GeeM+*7pi6fDGDDP&&SF`U%^c82 zg9&TVD#^yiGg_hsI_!BZJ030W(jml7fASXiGWtS#=)jYvJVf=Z!JPp~SC>640-XtU zIwJTGk~UktASAIfY1%K{LRe}0VD!Vo3VlYaqt!t|X}htL6jr+RZ&iA-Fq39lyp-5V z-71?U>VzVUY2 z5EIH`rVyFg-zNXbUV&RK(g0dHhF6ngEgAfP6-HmFpRUb z=@b`(Hi-UZ4ejbtK;J&VYL<)Rbs~G!p)&FoPn4o*BC%f6=_WknNLb7QdOlbN?&}+(DMiqi85SZ~ zir{0Z3%g7}k&R`GSJ7RJZNpCdTSjS>j#isy1)@?lx94Cqby`R;hac=!&Ew1M2e;GD zWMY9!{}MWCkxg$IDU?owxfjl?SQ#>au~-i>(8oya7-6x@Z_Y=MGQ4EcZyFpuMbasH zUJ)zXM|1OK43*_1CCaZXs3cBvzWoRclY=IYttX4+T~td%OUOiG6vkX|&O zboWLuD4>7Z@F5L9n2i~5N4R(KKoiT0V%g`R17S046qFC-A&g~bqUC)qPw+jlv0W%~ zgnEA!EnUd)rjsC(7JY0H22#WJW}ca-;VK>))z7CV*2RP-JJQIZKE$#X*h>j&NST=9 z=`Je=JvYe<11g0jm%5LK2H2ZLX;O|G!Yc-|SgGbwOP-RsV$_*T zAV^edQaKcnGvk|9P*IJdE?UE;DKn;=rbyO&p5a(UiYUa;m*;(54xieg85vmTd}cxi zwgfLwiy9$gjG2;b%CINoh2hXb2Zey^QkiU9muIz@pB z+YvpkP{fGZ3ObYsQ65r_>?*Cm5JKXe^?brfUW;2o20e;xgjkW3g@mQN z2L||@Fb<|*B4BB&+p;fJEu2C35ojiprnjutGIVSulO{*Y>D_WLR3u$L3NhFie9&26 zB4s88K{o0fb9?mlI!u~A$X;^B)S_}UWey)adENVKh8u`zPV12Q;P@7uPSI*rqmIR} zjD5v^N(PjWj+1;tD~pbYL;XiD8=-DJb}8(cHfuQ{!roQ`s}PQ@Jz3R`r|>^CmwFwO8Yq);nrYPvKp4r*e_kxlVG^Ed>yy!~ z9oaNU3%m@MbX(M+_-1s&SHR_Tqt8xvx6 z=hh>sSD?ixKxu)m$ESPjWu7+q3KdxQ;6r6oDjRaE_72!u7Xt`i}U6l@0w?);cazBfh0HQj6-Ve6|L- zqc!8Jx1U4BKdPs8?(6wy->$qn3SaFV(l^m!8WO2}vdaQq>1~QOEO_-Zo$Q3j+Eq%b zk(d&L{X&r7PoCnNMM~bjnaa(Z|Q@sufQ?7N8aG43LikzQQBL+B>fU%=|)oBsPavzmdZ^5whG^Hm`Ftr z#f&V*nu@Q6w6nXayV`p=!$;&58cg_8ex@#JlDkVkC-q zu?&?DiR?Y%X1Z!F8!@IZV+L>Aq}}``l%Lo!`+NJO-TkH^?ykx^?nddN&=Uhhi5~Gy z?nSP0NzYTDq+ZGAFwCwASBXK6#IGU~(&BKwVk?FWv`W&)wQ=mS7i@xX(t7A+jfMUY zQ&Yk^Ay(d;LNUC!+Nk04pWm7LYnEp~7WnXsW2;NhUK$!;CaW9x`ovGOut3eNX`KEt zks<_Yr4ITox#^!l!xAh?AH&eLOhFuObq>}x6=#APCL0F6ZIq()*}utLT+u`mDdA5F zWmtJuig*ntEUHqXxO?~?Y2d;!y}q`tIG?JQ{;US|bY>Ecrk&AgEn`Gk(RLV)PTQwO z)k0Bpf3=cmJCFGW`VV?h-{0yP7J0~#Y^TaCXtXKBH>r%dF~mn%E{SJ~3;F?fu~l(ntMP)O;B*SJ*XbhB$W3n#W(-%}tW)=(H$_ zJ_q%Ac!!u=vOT$_`Ut*>!|zEgwj%t%1*{Cb^%L_Jh5L{jtLEy_!`SUEgar4U>5Oxx+4i~-lQs1*AZ)@dvrjW`nv18p88Ng)mVF`_~GvlpSY z3x%E|=uE(3*%P?le*ZprL#+``fwF`=q25keK$^HV#Y~KE+@#R$Cy`~KNy@n~4C8ms z+@W1CVDV)d(v>+gI08Y_PHp=XSMCRjNk0*!>TlPB`?m0RqVF70#AC} zJKch;cNX0yOmgrJHj@XNpxRUG5`cHpM2ZSh+>>h3#B#Q8`S_TjUf97bex**)Ij*Zm zQHX&l1h9BG{x?Qb8am`jo?$0o+>L1B`gvBdbAf@c_f#P5tidMe*%MG%j2f@v%YC z2~)0_b#%uSc!vzO_+I#L*r|KAPjGMOy+XGRm^T}b&W|ECKGBZdpPT+O#+9~spbJ++ zw#ne4v9n7s6}k_wYaq0Nm-Y}F!uoX?Ya=IvO(^S480$lsb{cCYFCd10j)+O_P&-wh z_ZyQm-7hTN|MCQL@*>>Wc|p(07}b=h=k!OCkD5kfbXH>M4@JRqS@(*+5Z1q5Y#XQc z;s?%OU*&fU6gveHi?X&F&+a!nDR=#Xms7&Z^Ru8nE1rU619MR{W4p2}IBLRpkn3}0 zGl8-wLcr$2ekuzFa=^MB+#;hgQP5daosG%9)|TEX7~dMSzIeh~>=qwWP1&%KNG&-8 z&HTF!^8MW5QBh;!4M@O#1*rI>i;i5HVazhEbgiywFh|p^jJ(dQr#;QLdXs%0VXok@ zma~>w)R&okqF&UMlQCZcnO}G@$?>JQEu?aUsSVDJ&~gV|bSWwL$(L$#3COKWb|oJb z6Q_#ME$|tx*?Gdw0e*C`fOzMQicbk#w4S=5Iiq4v-lsY(y8D{LX0J;j5n_$Nb1ui6 z1{pKWZ%ZWDa5CmaKoP!@Q|gN!b%#oQz)@cSU6Snh<>QJcj)wF)2f86zUlaht@gu6p zmkj)s?`M25&6~@jCc>Py$W|o#U{rrGr17;p_o4I{HlhTdfH- z-A0;`p3;B#rh)kUGaQ_%_v3S{rdZ_ zXZlKw+6``oxopr2RiQ}F11G$}RVLaW^6~!~v%nc8QdiOtdUwFm)m7dRQgTZd7zNfp zg?qruhZ2L&>1&^+LtB_EIY8NQ}q>{1o57Mp=D#b0pr2Y;&8>9;BS=6u|_hrw~ z(7oajOrh?HH9RHkEVrmi>f-b1i@Ir3}*B_i&B(OsBgkkm@YIgHWkS-WF2c zZvGp<%s;UCrdw$+o zVA%lNofZM%x@cTim3G3^fxjW4OEbeQ_=xz4rrgnEJO1cc=cT97Fw`L1@9$Fu)caC8 zz?wYNb~#y)H8u5Fb4)QEeuf@?{0m|APHz13Y60`jto;Z~RpwWO`D7)0uuFjclX`LP z)*S6CS9NZ*c#fm%X~Oiz${AAb`- zO^Euej!~&R_jF0F!RpkaSi`NK{DTL)sSnWfB;t-Bh;IipVhZ;~nJ3R${*ig9&N|T! zy8e-n$DGj63Ro?I53$h_|_VQsIi~w&Ym)l4|=!GOmZIkE3s`Cu_w$<>1S=nz~ZC?gK67+$9;^E4qAF zN^QfuDy*w~3+BzJTQG92>SbQ#8{rPb$R`jo4<6aOG3sl?gWJ6M9IRJYa{}V3suD+s z1;^_68(Md)S2?C3R8+{6nebVPGVgR@z1#oGn{#H5tNnzi@|d(a9(!O4)ftstftkLD zgT8F0JJI%3#hG4D2>zaHV`#=1;%4g&!=E^PTee>Kj8 z+1P<5ThZifK+#>Xo+<*u>yR{?p^35@3adds0C(hzH2;e>=u*D{oArR*tb_n}r`i7B zI{RZB%<(M5w|6j%*O(v>U&^!N1HBTad6+1KIY7;AAvOFWE2c$+-CWTaa*-J^2rAS4 zd&E7GcUu8#J*ViEl{CBp7o@B)Jd%Ig?pjTFmYl*h+bpdp(4Oq)4>UKWh8~^+VLXo| zx+3#DvP#D-M`zR!QKT;!MlosI0HaIpm|g>oHEe4q^j+gBk(&9%&3QMH4xGLCp|1cr zW1AI$_{C9ggq_LwrJ!FBtan8WYq-mk+?0`KNs8U^}fLx ze)})jU*Fu{VWc6WPsZBm3vhQ$^Af8BT#Q*|KUm_bH?sf2EJm)}p7J_a?B zg0r}3Y=7dh(}KQwt`&B3AHAVd)h{e22`B5{ZiXGUBGt;=hJf zf~i(U=?DY2QP>EqA9?!TmNNI>8GIsc??=KrK_M*qploV3ON z}Mq;G&SymJL$Nazj$^k(f{(h_D!>Fq>%|mraO`c{4*py12Ck zy^#0Hpr&O756?yf)DnrfCckw>(`zGj=hX-Q$+l(Ji+SDJHH?HGc!J(;=ktX7XZN+| zwa4z^$tjKpN}tn+kmcY6UTu`{0-q17U&kPM-z#m34D8(3>j zzEg8mT56>lk26Q*tKJV|C_AOHRC5M~bpdW$eB?Mf`%LKolND7Boax-1jMnO226H5y zc6%)!hV}0g!<*`*M2LZHSS4l7dVuc88qsTMgyKp8`FOiDg84Oqk7}MJleSKIpPK9w zwIGTbaMxakb#uarLMJ8*9>aqAhF5gz5@`QTM>7wlWU)49q(5Ue*zf*jAQdj zb{<7vp~ffz@9LVI1b|jfs*Doh-Lc30CWTogD0+m)vSnz92mK~3ISEW#`$TnfL~@>T zw@y=XvLQ=HseHK0L}dc~Lgl4}$Vw)emFr&(X-l-euD8%Px? zb!X=yR+M)J6Vt$xsK_mlBu=P9o@BWVO|3>uUw^4qxYioVzJ!)6Dk(%?P z`mvsN6wuBdjz%T&-7<~!=?vR?qUS;?k`#eRS$;W_3@aiRA(;qiaM*^XVV~*=U`d0U zE?S|#?su*^Q;mWjU8WU%<&%pQ=POO3f*MtkXvT`LTBE$l^-$X|QDoiQ*?(zV{!HL{ zgi4V|h#)nhEwntz4<~-6j6Xo=Y->bazE=j<82}GGOngb`FcN3licDLR6=ACycUOx! z2Pmzj%Z}0{Ce&lwm1JixanT!)yE0DaQBYnQH^os=yuNBc>W#4jy_E(ly{Q81rrzPo zhkVrr-x}lv-x~1`0i`9<5k`F@`Syq5ABe%E$sK+R_sqm;3_USYqb1!L&EFwwQ}34% zfbRwd!Jx}2akh8yC*3)H5&7#$zM;N;ufE9u^rhT!fARVAi@y>7knGR7rxv)pJc|tK z*~<^=IcN?(zhMu(hnzOBR+T1267Ju&OAPYg$#v>K3<{Lkh6VfWI|KZt-r;|xK_pAK z{u%tW&m>z_a0y=)BjE{i0py`)wCC+|U>$08YAwU4B>z&M$x4ykzG!_cA$gqD9>OF* z+0Yfba<4N?kaDx_LcmPkFhVCH;g^@Y4-O ztRLeB&yJ#_WY)p$d?}-S_Zr?Mc9bh`ldW^Drlb>^TWH&Vgia`8IHaL0_J?^k@qw3s zs8dxoUeDQ2*YTukbDxkU;oN;ZxK#V_=z{12ExnPjoaaz#Lx^lQ1XZDpE=UkhmI`em z;_%ORF5w02^X)RIF>K9dAw8dCVvAxC%aW_9W%cR8i>*XeqXoWrBc&NrGGwS{+o7hg zBNguM7MuDH=Cu5=X;Tc?XG?RMCjs9z3ayMs^WT=TeV2Z;JJ@jBa!<0nuw)vkHWz(J z$@%eLqd*b!K}pZvKvg^zBf=TAf{;>=cb ztvw>GJ+arc!kdwJ8$NTj$e)))mFhY)tyAuQ30YeN6{MGqLGbSc&j zTbM@-+ablwX)}(9##zmSa{M76u3jVcam=Zn_J^HQxxaW5Cbo!xz$lU5y2(T#(DPzu z!X??Sr%dbO{;vcANNELd5(h8g(Q9`eE3Qk?$4nNFx1>KP3f{u!Mf|9r+Q2&dNMuW_ z^WWlKU?1M-u;8-@XMpBJ^PX@r+SP#kpihBApq4xF;KO;qi33$S!PIdm5crO$Y~$Q= z33r8p`&l}{*Fgq|F&mEPAW_gMTNjagSTI}=vWO|a_#Wt&Jh7EFjBvr1m?I%MWA+P4 z5KA1PQJArWg7ja#yu|8_4YN}jqxKQ#F1?=j(J_0AhvlU`?}egzC3UtKQq(>>2HL9U zSpY3C23WU+f49~By~+u8#hQoI13oK2`2GOk7bn^W&t5BVsf>ZVfnn>@g>*3lTTMim z9K8c(dYrj|>jqt#OF1uQ_|fW$Y1w)o_T>K5meFk7?EK;J?oxAji|JMWW#6` zwnP8A{Zq&BxqN&GH|`|)e>-7|41zc7!~f%LDFFa1|3m1|(b>Y;!p>IF#K6(fz}D=4 z1PYqa-v21v-@FMDH>Qqm!#`ji!U8ftga^n&MMywMFodx50+n0STO`1bhIoSgs+*Qt z3!2wb2_O14GF7fo5WvZW&D%O!*G8ANT~=K#E!Vbg|2zUNZC@8$T7I3(m@=ke!e!Lo zx4BPqZ@o{scRl>{y!Wr`_@VW4uU-d7^E?_<^px(rN%{MV>Ta>z2cn?5%kg)shF|Z2 z;eF2r^1SYeyN^cn_UK{1)`Iff8UI;_3j1A;Oubcr^D9O6-S3@#d`J3z)dZ-eT(JW8 z-5sR8r6m65)V{t#eQ)&hyE6jQSJM2J&)f05k;DIJfd7s}t=@-%|6UCAoxf(`eRYBV z7VO!jJn{3s(SN_$(f@$=DNWxd&-`9o)<{+l{GOj}sPSq+lMLq$gu5?vMO6 z#5lM&Oi)f#?S@jJG^Sz^Eogv>Q8MoqL`JDpJ;MpHM9D0lCkdHVF>e&)3n`^=&J|Qb z%`_c%ra$&n5Xlr^EJ?jcd#tMeLnmLEp_@U9t~w#&*g)u_Yn}Qb*^BkWss(xCUZQY9 z<)f6MP4wveQ4*?M+D*rP1V!&zA;r|O7+(c)i0sN^9(s}h~a*i_4X`@Y(T}y_LdU10ZO=6?R)jOacdk&$x(J;^f zGg0{;FG3VJlwuXwgGikbxfxh9B!+%+ueIIt_k|q2S?sV!Ms@n(bzP=G-%g1E2ioL} zyGGyJ01{9#AMG(Op9rM|$kpF#OE@S@t9s`+l^-+a71|Oy4N_gA$|79QO@!q+vDuAG zYM$HB<%GhyC1gOHp+j9>Pc4x{6f#S7?5cePw6^IJeYp<3Jxvt2m5LUlfm?Ckmb-W5 z0uE7W8}E5Z9&KjimS!>M3VIQWH{`2}_Lq$t$^4=TXu-UCV#ngMqQQJP_9@X$sk)9I zk)+MFYB!dBQ|*_+DgLbb$bP=aovGWifAt}Yhr*nz9y5}A^aEuv@y{$r6VO+*O3P$xW}uKbiM5H@?h^ykbLq@g zUUMSsC2sE(P#2L)6n5O>9|U%yGiowEZkYf=HRg@ENlV#-Aab6kR)o=arXiBM)RfJR08g#2rr7(~iDUl>W2Ha|{AZivc!xaOX+<0-L zSyn>=WK-gHv>hv8<7{|Wt}IHMLO(4H?#YtkFLNaN;mOwlur&yd;;9aolD8Q?%4=tVaS$FB9m? zL-V;UP_;>sWntWpRA(h z!A3HY!Xq3)J8YPfriK_z_?|*EM~vkmZM77o`%3!hjj0_(V<9)bHs)AFh*rFZYgV|| z$aJoCjDVB&Y~C#e?D(W8V$(~NpV09!&nWm$yeC_I9xUK`p16gg^|0$-yKn=2cqEz+~~lFoSYuA6^Z2rY@dpyr5@5s_eH0zt9ha=ecHTww*gC zwhyhIzZX%0QPbUG?9eY_rI6^xk8NB6R5T63U^dllOFv{PF?{YPcMW-V0vY z6ch;5M(Bm2`q}qPu#(q}(8cFIWLccP9E|>g4DR;R(d=RHreu$vMLHVMORQ?!$RS7; zFIzzy8;agEv?6szml8HP@%R8Kn6#8hEA18JP?MbsiHF8&1`pYlEi>sOP>I^;%ucT5 zR#Yf&|EB3AyB3QlCvsggE%l{)=`89zG^K3jeJG!8g7l^c>eoE<(ZGn>A4+Te!n8`PMNonvt4IxX4j#(0~G)0Q|PM`%r+JvH*}qj6Ubwu zp!@HB*&8XteBdU8%;fz3&|CPUm^3_J44oD(}EW{l6G#(nNpq*r^P z*1H3FjM$!~J+8>QK|8i~n8J=d4k~tBKL^ZQJV{c%-;R!m#EH@vZux-a^q(?f6`@cy z*~*P<<(Bnrs^W)~;|aCPakbQxTP5$C5g*L=BdCLvQ(SNbjq93P$sy~!oYC|i3@02T zcGWoU!23B?oWbCta$Ob|`itZn95*ahDCy~FY01Ma!07XN%&p>E&R0n3^YraAzR1Vi zf#uNAMyb58eXKvE1uUf|Dtmm<=R>8uG6g3KDXE4`sP#dyY@AjvDdi_#w3YR-^PXIX zWYdGNoTL$S&a{oJ6p!nmmj1hEa4k9D!2c47KYup06D$_u6lbDo$|w50PTV zVLz$-k&%FqlxM1UIcl8mkw)!`^H(n&ar;ySaHZ5NdiF?a>}j8q65HNL(2MFEceB(Q ztG2FAh2+c$BpDGvA=wd8nbh8>7Bs$0kA5s)Wv6=)5lnGfgvcmOJLCkw^4Mn#HbEhq3WiNIu25! z^s>dE2BTALrH)Dtrcrq7U|k2BQFc?szzrs$>gI@&8z}9+mM#2U5dV8E^4Fod0BZ&B zt^+&HAO{*7Y{$2%Vc>N3-16Iig&i;5$=|fFGbZ`Ms?rc`5@nVn7I)%Y(!M^!(O@du>ES7{fI3) zzUw}lC)jS1_^q|*0}rbzG%hqQOEKLYBW^za8m3mPA+O!X9oH{-?5xu*e1C7UL5i+8 zfK_Ty0{juHfPUY>a#-ZX&Cs9A-l>Kj8EeuiBSUY0dO+Vh@Rs}@#e$`&XSj|kwtJmH zEB%t)nPpL<&TYXD-p~juD{~6ou#)QzmhNNU^m*u1!3ige-7=Ey&JehfCuMBUZNMx(Le{}t&`i`Ico4Q)FZqc$%VX3bl@hj5P1JiDA zmGg5)68@8lRw)9tc?5rH4o0>7`wfdQ?t)7#W5>Cg3eMS|SsUj!#^^7TOQ%sj5?_gG}4=J~6u#r&e>~dpKaNPdo`cqdrMvmG7!zTk7$_q@L zQwsY7SA1l4HL9#w5$?DHSsY9q*=bgUIJLIH%D0W~3hkW9HnKaL)|1wOMu_oc`fbtZrEI%JiC%~Tq zq5lt(VLydr+=M9%b=LMZUUG9wS}Rq}OQWXEr3$T03vHkVQp3uM>b1}M5Z+7ctB>B| zZ&rpiTb7KZ3v1u*Rkz3P=gxP}t0{Y)&q+WSKsH?(*g{N@z|)>Sz$^Ivh&?FdIsx0E zJOHr(2tzeX5fra{?9j2hJS896u+W=3Cm!7x`O|ySvAghX9@N2-yK)4+N^KnFjf2Tu z22}N^+4H?q${SCwd<6Z0)W`iXKyRUy9_*f#!16%@KrVzKVsCBpuKZaRl#f&aIqFBN z;I6`17nF}&K^e+NuOPqtSs2u}cmY236LN@e<@^%Vw{$@s^%Hc6uSTpM=)KjKsLV%A z}2d*3(B$~@`5-=gNL8zN%fv_>bvR&znJl~9@!111@6G2-^?OCb= zlJ7X8&4nK`xbmRYEqM#Im%Y!g48 zTyQn7Do0XjhH5m8c%u^qDrD9IoYS~usAsh@StgmBACcv#qO;7{@rf<-Fz$l97)gl~ zsunRYBXf>~W?*mNvC5o8CoS{x^+D%*vm+}gSYfKidMZfEx{jj5O0Mvtq{Vm}Np-PA zTbbY``92P+tA@~KC{nZf3e=>kh+)E$rGSu60Z_t#@Nz3LORHZKYjb0P+MX8r?dgvZ zSGl;FY5km}E%RsLu(Hc=g<{Ep@Qaa?TJ?+WTPE=HOHh*NJN=*cLGMobk-R|=!naP zlvF03kp~AMVy%zsMw&MAv*_i*M_)SiE8o=2Tomcgc7ieqZM%J<;#{q-`T3bjYI6RU zNn3u|Q+3PJ4Wt(2b_$d@Xq?S7jxo_}g`NGD=t02zigV&f=JKOrhA>gCbQzb1G4U1L zb%QY3jL}VvUAwTo_5MSu^oB+q%aX-~O%k*vxrasfF2_lt`2Y(`wHdH#=hTJ}my@Px zi3J&6R9S1+a!r6YvDDdRH;9c~m&Hg6hsjGH!gs*8e|aoQ&N(T;8SQz3Lr=S>iT@$A zX_yO{gz4aUEFL*8$muFN-rX;GG4g{T5ldi?Y}cYYkllIL7SyKFfNf5&x&0RE_4#4A zqqjn;&DS2!kj^Ulg&I*9j}>*$^v{eDY>rtREpPgy(EG*e1tS%?MjJDjN(}DVmq586 z(t`)4-=q>JYbCkJMlOvr-+|04boHWVSGgjUOXqr7{g3BMHrur}V?%jr9hBHMVAR0= z%ifAR8D=Y+aP!2f4D&#CWh(X>B$@qlJNk;fi?~jK)Fnt2FeO`~GLzgvT>Aopd!T>bCUH}QX$vG>ENOH!gt|O~*ag3DiHgq>>4-kG+Ba&6tCNZpnbrjlBM_VJ{NqggQ^433VO= z_41)bji{0lhiSz?RF#w6{0)6#@G_ykF=eNHW-!`IaZe%KJR0>Roust2-ZCW;hzLymm;k=`|7d%s;LO6VYd7h1(y?vZb~?6g+ji0&+qP}n$rIat z;&f~$JMXUludnu3`}D0^wT{;DT=zZ3nAbQ+U#bpSXU-1XzXt>>KX{w3wb3G5 zha5L#qGyKq24=d>YB;gh64^kXKBpmx5n~}V=Cti!hrA+3my69ynJ3g&Te4&uHfD3x zM=J$Oc+ut_$y$7P8k;Yf(3W0gn~kTp_%gk*-+T~q=p}Q>p-&oYdfnpcZRcX);u_&0 z21U-C884`X>nH*tqX30AMV=MZBMs(P7Z0MdWC@pP8!c|p1~-e#@WphU_0Ga3*rf-f zQbjt&)teghYG@PZk&*Z{$Wr!^=9{%KBZr)ALZ|dCBJ+VQ_i%G~107R{gTJuxz?$C|FnKDTCRQSK|4V@Igk zL}|iG@$;1~e{Dx#n3@7cpZXU1xVbi)0p;r~(iHaND@)|P zx_ej(chBu%iSL;yLAYxdxLVs|(eHztA-AI-e(%bXKSsJz^o*9p=la<`Q?hM(r~dBs zvkp{I=nj_~me}m;QaOeb`JiA^@5pIG#7-}CW58&0eatp-s(LyTVdFt{73Y3OV*DiF7OK7K~yExF3$>;jS3H4RXsu69ey2EwY!?m z@eJ%nN*b5%YC#sJ5ipgMM4b?tYbz(O=xp48m0K@7y{nLMMOi*k!xotGfpOw@P zgS+DUA8AYh>8~D9ev?iPjk-F?cN*ElxI&tfTe-?rlV)9mYU50`TdGnHWHnA`7A?-9 z-d#`V=di~Y_Xlo0KuWhu+K#*Ew)L%aXH(+^qDdN;!-h?RAv7wC(i;XBrB zY{IwDqC1e7i-ND1Gn*a)Hq+LzZm*}Emk+4me6ZB7|kh5Nv8GB_{UGH6gJ-9 zdLB;CuTYp82*T{C91iIkt)XHVF^g6wUU@(+7X!#cCJ9 zU&TXosb@H5W(9AUa$P2|dE_<$?M{c@ig5_yE}rww`}l6TH!19g1wAnuoG`>pFbi)% z;3oo~)3P&p6a@KTM=rTXJgDRf-dh>Lp}lb2uDpym(Lu*bm7T89q8MO}s*#+hT)@WA5mlaQ<&$eq|P#c|ITdMwxy`$=vepU;`NL#3JHQYlYzx<4Ozb3n_Xmd@8mfA78 zZ0}rkkFu2;5t-I@!1Myu%1M0}ub0d)xjrHyml>SA98!FEd5o-jFZ=C9(^R*Kg z{OWu^hG7Xv)qv<7@et^TzH!~?gc|V$_b&1x`7=y={BMl;R@2UVO5D!;=Dp}-%Wk|A zt94b@z`jQ9yqS-l{od~K`ntToxHIeHl#e}?rflEak)}P0<8OBhfNW-8j;Ag9`l4;H z+K{OA^`#jT1HF&p4mwXj9pq~0jU25Hd@xN{1Ed$gCoiiA;_9K$?+&zUWjLf^?yY8r^3}M=#a>)|TUOcsNLn$rytdekRHmSFy6+q`IR+KLD zosRW!?c!LmavYH#d$)W%&AoB>$|AeO&>6*SK6~y>o``7IStQF;Ws*|D7a%gXVQH%PFLB5~98~ z^f3`ypa+O+wq{>9uPtUdiOqTe;&MdDenI_>;J&!l_f$B$oN+l_Zijn8 zpkm=FOIJuIPE3^9$NQ0I=f(HQen#Ny^M>M&p-=8eW?ITY1}RIzWJ%qx6|(e+yeN6r zO$n0lOsvfTp9rHj5Xnr>S}*!}d6BI@b0hW#bgqRrAaNaJeAX*N~nl#2sS zT-e1vfXLz-(lf8?my092V}mKG-?rJRyDmMI!4~-@tNu;IAa-(7K0lK!B@&xncp1`M zVE(v+UHe;B)PBMWTiQ=&JPy@IK6ES5GC9j;rY^ZtlVBNMIW-Ip)H{ZO+q+LWhi;#t zg1pCP?tJe!{bu>>F1UPRMR+ zu#B*jl$!mlO+Y!vADo;1PIxTj7Y?^rq{z&u=Kyp<7tyGkOGGPk%3y`R7Kk+5mIoYo zh&zc`X3%0!uWhG{k|zoDXPa#~L_?@o%9)g&T|x&k4-fhyvwg|rV%@DK)bJSoHf3Db zMvz(g^nf;02RAYt4nq(u;fHPg1ov=nA{NUu zE2P$WT?9YE$0g3}8AymRg|&Rv0&uhy?Q8o07`wN|<{Yv1VGq^sPK{QVu|$IJCYd2W zEYYwZ-q1GtEDzi&B z{H=gBWuw422i4@!i6OJjO-IV#u)<(C=714LWc1i9u1HpQEO>Y3oH5@Wc#yL5MHQ%d zwlAkd(@1HV78&1`(3-N}xZbC&2(WLbbRl3OMbSCpTICKj@`()YT1SrmFv zIpLeT)T_c|EWcT++X7EU5Wl+w5sKnS@E57&d1MUn3i-laf9c+XCA9Z{2v>&jskl`a z;)RAhp=!~`S?7}pv$iu!(3hwUlHvj0nwAH<4SjxR^(CT6J&n!A65s6KFG@U7?F*JR2^JFc3wVP{NlR$PTf&}pG~yd_yt4r5!U>YPbFZKPe4E&Tq-P3 zgCtIDs>1K|B7UOxWLmo3@msQ80z$`1Im?o2x(U@>1ysU++JH3Ziz|rXz)yV7{{G-( z?E@2gkc2dc(9;>MT(bQM(=sHtj)K5HNQQs+M9B)kAaR6}@S;6YBba!%Um_V8G>%hpij z4qm%p1@3!>PMUO(HIMw$)4nXYhaVFL>8`@yROWqUbnK-iv3_|>=oiIH3B@TGqMAZw zKZG-l2Z>fH+qmfm7ThC}Xo4cC;M9Oa@iB6~Tu|Au(4vanHN$Kfk1|>5Qt1`gvJ3 zHcJW_gcnVmCIbo9P(~U>N{Edq!HSsF=4zH9DJ<{4u!R!V*>?5zdJ@UNFG1xI<0vV`(+bFHG0*BYw{KJ8a*14tES_PLS%`gmM*{O5)oZJ#bk$L$bhMjx70?j3=Ad zYuCYH&NUomRf#&Y>cv!;YC~eIA<9>a&E=}(%v=+j1%-NV0)=_l_1nAJVw*bIq9KX) zpBYnPV}wo`^+l_T^QeL>c4tjK^-e;d>EHrF1-qzU_j`pVdu;hjVut2uR3-RC1Is!V zojmBoqRjdL*?ywkQg;5e?d&q{Qe#$5rBGwBJ(|pE`HVt=I)^I*&FtFTECE$Ydduz~ z_O%5rV#Ez&G|YnUk8!C}M_7;RxXm0)9H)t&!^UNR-kg|sBd!gvC#vaJ79rFWl`gaC z{lLU^%W6VlarGg|zi@ZXLWF$s?!St9kpSfHe}2#K7XMTBzE2AZXZ`};0|e65+mNd# zKCeJuV(np#s?zAxr)U=abDM&yaOxxx32mj?#wK!0HRd zN=Rs(UqwLu{Z+%q8awVieZZzI)h^l9x5>VQT)C`7Uf$c<)|Q6#h>KsN(>+PI(Q@cf zLx8(63vqz==Wmpoi<3BQZ7x2ocfU`g7Nv|nS_?b;D{g4-42UB8m$o4F5qUK=zD3u= zkx1GZWi^8G!Q>H_e0BOJ$HP(dbr-omsyv*wDMjtY!ePIo@WtOYNeJP7^7E9b$RE-b z$+a_%i;|k32dEg|X;APphspqY8D zq&Fo2t=H^h{z2p|?yWQ)zjzWn)cd_L<@S8R33i3S34q3bWxrxH|K?j~PeVI#QeIy_ zbfuHtga^pqqzA|qf!A2t8rA4BI}2gZbgCL0C+9RWFKB&usk%!4?(P_OUBk&#s$KH>8*k*@r z@)W1Pz0UoLv)rl?9tE|@O<>gS&u6_JpOTL2+*-FQ$;YMz0g$usoSzZGB13X=%X5C@ zwMTaL-#u5pYx?twA~H56Ur^m_^@v$<_BcuD65W&Fz#;&#a(hNUeH`5e_1>_j9kh?6 z$+s#%BMk+xzXR}SAgA5=R?(MTWK}Is#lNvE!d5D+z`bEwvYz!2z6NhXfGhzaR6U^+8zynw47_5DDIBUKexw9 zf6^m!qOZ5t^Zbli2Du`;=HqwZ)>uVs3q=LjyH9AM63F6L_3MJ%{9h-ZxZ60{<)7-+BZ}<31r^$on*2zGZB=cKOk$t8^%cpp97w|a=(5_$nJhK> z(Iw5q38)`d))?4EO<6`MSwd{-9(Xjwg-W)CU5gp&sYqnCA|*i^i(gBa^BPlzBa7YZ zS8%eI$ZX=>egIo1=G2n^CR;En3g;*h$@dF$9-ZNvB7zNlBG!u%*L;Fb5I=)YoVBM_ z^;?(kHa}+y(*83xAbK-WWxrxT#M zWBkCO;XF-I644QTkycCfh44SW#{ZBL0X07(^C3Y%DAE60#4KfdCl}KHyuRHk2SX=A z7yJJUOw_P;QCUL!lrv-g4y1x2hr7h>O61E882=FZ6Eu&Eylwz%!FXF%%`s-m%$Uy8 z?feTxtGUHOySiGQQKETCtBx2vQaG-n3ElEkJ0IcAZ)G`G_hX@1=i@PZ%zHF8!DX5I z*v~xI`ElF)%J-4We7gH?Zvq5;pFp%skoeDG-4BVq!r{93gmEe{?9QSNRbhD$zx(k& zTk}s$a0Cdy0k)uWJ>8J*yD0PSr6KYV3eXd=@?-9?lO58ty2EWaxhYJ!bJuGi0d3-D zTR45k4!I#SZ#Uoqmk4=35Jt0pvGI}y62Ke4<0Cbx&+-e#LjOZ)(w{XBfQNi7$OC}$ z5uHc}^qd0{iT9}tKhWUhztI*;4hn-ra5EOq83Rp8mVhXaOv_RuwY`xN2=HdTe3R;A z2`;e#OO`#$M#QyTrE4zT`o&GF;Hjw;yl70x!orCtQsX&^&bj&p85gmRD7gtKu}S!JOGFqs?)co@UzsYWjC-Qx2P67`&S* zp7EmDktCUUjUbye`Zy*@n#K~(F0!H;*SJJ(kcl{ReymNCLr&ckCDk;ud>9%owdfJf zj2aJP2C9`TijZ2>nJK1FldWNEb-|E4n4CAkktDzFCV>?hpH(-eC7#ZSK5R-gZM$o5 z)l`I&4d2pf64V-9qiFzbA5%KS!7><6jT`6?-joes505A)No4yo&?61A%B!`aE*E9a zy3o>ib9pK!N;aa)q9DRL|8KQkBq{Jm3y+VoeZsy9UuqXo`K+fTb{e0gQ}V$#jVRjK z!nKC2-Ty~me&5Iv&``_HV#m;p)76*?c?6tqiDV_|ED@i(rHl<-7E?~kZK5Ue#zol3 zBC;Br0#og{WT;A_GE=~~lT11cO1rvg%djR=a+-4ba!Zt14e)mDhi83V6*9=3Gl&GI zzv74raN&)$x8SVV2RCua`rRuuF%%LXljxxcpu(NIk?k5{Wj6_Y)dMWvFze0L!x79o zXaQP(yY@z_qnSm;B)q!NZd)>6(b+cHPQa7dRtoE+hkctGEU40?3-H#hOb4%VWvCT! zr_b=yCmwL-Y(DCMCkHvelcPw!OB#O)8i(j1K^U7~7(JZ8d_DdK)t$d274$>~a_33G zv;upVaI4!KPyS|6)}FEJAw1;CGt>0HXQ*#ymH(ylH{i@e7~rWhd&7d{gDo=)N96Z~YzfRTIQ^ra1CnTEF_+ln3Xxhm) zNA5w%N$xD_ft)%vi8cv?pa@XgZ`|H~E6M5T+xz>dar7q;{fXWZ0H12T2cDW2B-~w6L*!K{S9k zC#+PIkCW2S>fR+Vd3u{gENn)RLgU>D$$t&<=4>y8NZ1ZKmPoGd8jZNi zC+NB)$XIMEx{fR>l(!$c3Ke`@jB?hdJ+q3AH-lZUATrd6+*Do1gaMXUJjzk|ddQ;4 zK9-EzRJ|2*tK_+m-+nzk-kgDTNxcO~y=cG%UH5}Vb4G;jUtS%w!CCi0$yjpQs`NVc zVx!)~veF~kHBMQ^*AJB?C{$c$07u=UBdh}gw&oc?inkmU)Ur=w6%bJK)a2<(9!f0^37 zO-xh#6R44k_fQkTho5YWIcZ=d8s7 zqD9=4=wb{M4NylGXD&nUs#_E&Eph0pj6Chr{Ln~vFs{BrrEdA@dl+Rxz z(wv{=Do}E*s4>m~B<~8!d;#eSu8h99zyMTA+~qG1y02$*M)LWTZiPCxs+?1TPAk6=sF~ond&1ag)3+lf`>If6VbAXL+lWjji$6U06 zE^aCejG-#tl;d8PJ~l@Y9&Td0eV!l-wdNWbw3l3ZU(o3ILl$TndUq76;I^B$iYzjB z_?YUDuA2qyhj%}cqu1=&Qqk|KOlwYf23h>yY*DKP@@&IsjwpdCS%4tsCmsx=V7XI?W{j&YW{r1dN=ARyMf6ouLooBe7e0H3#7W{iY@cTH= z+3yXcL>Wp{y8^M$uJ-u2HWe6Yn4F=HRGW-C&|sXV%PjiekUQ35XziK}k+{HHRY~iL z3^d6+-BEq=c54E<0?J!jv!1$k zQYY^<6n+F+(b;W(|X|nRc@N+4RC}>EtO_VC}VQCgT=Y zstGD5c*wG=i4OPlY%_OmJo3?|Y^*plR#VkJr2q7hAYqv4eoz#?g@SCoQj~=#gq}U5 zT6n0Aez`|W8g~F|T5;0Cr*&w0_s%xB7K07eKq4nI5qb zySohESa8zkn~sdiKV-_{6_rod16GrAAQ)?gSBp%J$073? zklG@i2-9=*D!CHAJe1LO)tM??Kv3AQ#n-;< zLECRMQ8-9@mCzlaewtD)It#5sq3j4-@j7pL67L8jJdc!o&(uL$ti~kTM%p4F#pL>% zfJbjP=ZT>?=NKyeD6}-utV4wTbYR7Hg_No!$63}B!Fh&66_&(%tNHcGI-lR}n07CGlh%W|yh~BzNvXOr zgOLZ+v4syfLm@(x+3Y_bVj^wUe%YYTFw*FcrVgPd%qc6c6YIKr7$lsoy& z&zQDbR%jQ#Q>qrM>T@@6(?hcuU!d8(H{vIoq6HB(*(G>Ni=rh@i5VG&mdrn&!=qlY z%&%b-{mEVlhHh}M-guaw6Y-P$=h*JuP2OQIx2imj7aT~ zxJSMrx|>C<7eDvHaSkpsUEMLl=S7^wy(}=c2JdV4&Zqw}NiSpO(wya4=trO6wTUT2 zo!E==?2J5ETF{x3gk&D-rb1Q7%zneo5nXZ>fe{U2Dq7L@c-I*_>$j# z9k#9o(-5rntjGDHo$8gitDS$(Wc{L_@|U<%w)|Ai_)0kHEq)=e{9Fn3d=?<^3(0-E z{(26c)CGnpcn!ZMB2lRv49ZZa2!;z4AF7B&AgsiTdu==2Dz>v$2r2udNh_OrWeB0k zph%T6#g$S|uYDf$-^GRo{&FhpZG$Z(B>>J;*%~w7sWQQI>cb@sG$lvj@?g(={ zTY%Et&k}=bp65d#&>Y)M!b zw@WIlP&I9bZj`M}XyEX=ICI$+X`47y3T2L$2Wx@bG~Rd3Z|oJacgSYAL&drg1T?9( z_;d>45$l%<=|bDyHArmncLcUbFOVQ#BEDw_G)sHAOI^_iyG=ad3mr~R+46;}l&~R5 zTT#&^N+2yhdg#bEi+wl7%A=GV)v6agsE9;f&MTiv5cf3tvixm9E@>j)JB>o0IiD}k z>_$BzKx4rFvvxhB257OsKYne7145{PxuawHOP z6g}y%o-#PL*kVJTK6QQ~KaPHkQO7nCg$UntZ|(v%g|d+Z-$Gsh+v`C4EJ9qNoz=xO zr>o75PkJwFLBkzcRVuo(kqAo_bm-d5*Mw zantiNpPQfwpk-9o02l8;+8BQ=Cd{uwni7RAhB1maf8pit?A!l&~xk z)@`TtYvFW!#Oq-ZzM5m4MHLUKq%E@F)J8UCbQyAMWu4-X5`*z&vIp{QSr>&kcYbZz zLot%Ol>q>-IRP=8dxnoTl_J9`%>D?c)7&%;663T>)5;%S8HFV$T8yTV>ULgk`BYy@ zB76fx()OK=K%I|RjBz@`eT7>yxkX{YN#j;rj1_T?G=RSlpQcz4d;Af%5rvcHe5Eqv zTPP+t>MKU8a||jZL*Y)k@Z!cn^H%q2uG6S^ZjAI9GFprx0*A>?IB$Ddm)PHM;w+;8 zr>~yXdn|KdPd5R`xZipOVYT331v2vZ_fIZ$DS0s_q?GkG{`Tz6Xbj6eM^4CMJ>Weh zG<<8OsGi-T`pqG3jl51MGxzqOuC4%4IW>REbeyuvp&P+P8AtvV%{ z4?fan;;o!)ORQr`SF2)2HCzYrp)B64QnjYijf1q;A%5d|QXoR^nhB!(HPYurZBgZ?`b@;%T6L<;O?KjY(5@Af8t+I02|q z%(|`;ro%L$(XR91GQP-78;)^|qOYd{cm_L6!>Nt$j*><=F44%!C8)~Pt4$af0v!={ zD4ru)sgmbk@lcmJsWvGC<4Zfa;xE{oOyJF``(y|}7tm!b3blHd-VTH$PQ6@jLvN^+*mU`@3Y@v`6#En%CY4&mjM3a`N;c(h(eDrup({7`0FI2 z=Es8^)1gQH>Ur}MN1=}t73%nk*r7A!4s@g8rKe)#`qxSR?rooU#W1`pG^5sc+-y>n zSnL|q*RHgFjPD|Fgvxu-*94(b749jPd(+p5h(;%5^C2Gk1x+DSLBW0)kLWsUr>d|z z173Hba1xFOPaVA+c&*=#FVnO zDV7o$b8i8HiIqIFLpGLz17fCy?+I*HZiX`A_KZ0^N|6o18BxVnIaqNs`)kXabX@9Z z+~8TH-kwBjs}-lZF;g!}?dsX8yQJ{j6xVfTdrbODCrJ)Z^YaFc1dpo#$34QW-5%4P zJ}2y9dYt18X$2k7Xrl_G{Z;C#weEvL6^^5?Hy2!km(&`%SGSpN&J5E!?gd?&K{V%c zWbjwZX>UFDX%d+CPF!Ct4_`hkIqI=t&Ps=hOdd>oFl9!~vW6OMahe58vK%9=OsQ+i zzt$|iCL5KaFdC!^tr|*Ljxlp14_n6w4=vg-&_{{cj?wS zM9;iqOo)Wkb4}=xoqz3g|+PJCxweWPY#oG$F=9i#d&+XcB`c&iY4v|JI!^LdBYFJ_hE1CY1 z#?UU~Bb7ua33JAyq@`d^gnm5e%sM8j{>#AGQ1%FZbWeQmypGn-lHD#%9thSZTv8c! z#)hvs)nNYib8GrxC1>7+kqB6AxR_8#gEYS(+rA zeTGTyhT;O#{CSW=<5GYmRoOU(;qT+C_sbxLQsK;s;)|m0WywgIwEHJNwQD2X7o>mN z^uZ@_&VEB8fWV~am*NkGnVd(CWqP~;)Lwr?Wd?(F5TDOIP!$Q~(^`70KmTg!@&2p` zfT@C|0c+`TKvYw^!{fe?%`P7i43|O7oWDA@mdG#^Wibb`K$9l&LhAI&)#wXS9B?O! z)x|x)oL9WxkFkbW!pV~E|ZG{?WaJ#dD3oFm?DMLdFUW-&S+nVkV!K}l9d^DqC)+XX8+;z{J=Iy?y5QQSE8aPrQOb-|_aO?I^q9fHxd#Tjd|IrP$EtFk{}} zI#-fkKh`mBtgc^0<_||fZxn|;>dTS7zQPwGakj=Dny)aO|Ji0(tKkPqW74LSp?ZX? zYo@(hD_F%JK3~BW*j8^bqQDS5=Cd7-hXNyVpB1;ln{MQ)ny^MXd*Gn zNmeRmq!bR;I6C43mD}=TltTl?#;8bV0AhP$Dw|e(_|7Basi{7(V;Yz;q0nC4;fk2q zk)kKIHD8e;*o1H1+w*4%md_he8)DWCd0}nNena=0+~9#B&z3Fjy%yP~+9R#MXUvJ9 zzqB0L)(t6ZZqjf-mV^@}r}sj{04)T-Iq3?kA=bF7l^#Nzw6(uY1<@7ZFa+5!nTD36 zlRe6lL+5Vu2bj_(FhuSWw(y8On#iMUh`vdd$>PFpqyuA042KDT4nJ-Ogb&lisLf_s zCkvp>G{S>f@n|d|v@FrR%CNmH;R93xIhTkfogLHu-GEFDDt3U%&;7tHgviPh>k^XX zl%BX|gWjdnin*GDeh*XID{aCSD8&i4pi7?T-ncWb5B`RYMpDEjut;8Z2h7|Oc{zaR z?lWox&D^;_&G`Hk(5GdM*z+lD8(yO9VTGTht4S;6#Kk@eicTR&30FSHb>qsNoWGzf zh7Zk5Ub9mRVc4gUp=wQJdFP6BPT{pp8T>YHWNftOZE#BTO)(ug7J9+>*QY6#_v4xd zQtnl=T1nwFylG7*6z<-m-MHeU=}cJWotK9QkHr!t-0eC)#ir{9J*PG=@zKq%C^y(5 z!;fv~O~uT{ir-*1JHuA0E_g(XeH5=8k&+#t^qb>2xJ8TJQBF;f0t>^Q+2s5No?X?m zJ@3Fzaa7IqLsejU&0tJQ!kE7vbixi`T>gMZ#1PNc;(%iTah~C{SwvfYv&bDoCAln3 z+OMl^+h}07@f(@YN9ORs7ax#HK}YeQ5vo|CbVPAfV`7`ACq>;>sK^`1wZE1R;yQdR zu9*Z^WR&RKEG~`4`>EF`yw(SwO#$69z(y{#tbo&+R>1A{o+3$T7L*5yMN{%rCzkfj zC0D3Ucl`nU{MJj}N4>r+HG9M>`~g1Rwp8rQHdfCWtPPV7ACNqfvZA;8DEvN%8akd+ zO88k}P7#E{TzsI%BPBisi=y4JEQ4^nq7#~PMwWS!juMskjjT_kt2f25Lwe$%-;$Cx z8KAKhcH!zE;c`O^|LXYmlE=n+*e%~wh|IPuie$t$RiHO`$F0$hZYM3r%-G8e)i}XQ zE9%t!nh%3x;kNosBcDmt#t7$m>&FVFh`rzI_JL9OqaQ5uPA|HN9Po*jnM2#^<`rk> zw*z8+V@;7vwTErB#|_x~ELJcL@3Zgw-$=h(5_0uhxivq76F{JVizqz-!7TwWRyWRf zU-%P6efmYv8udV6I9GGDc?8;eAD7o0w$*)?`5^YZU>3BzP4JWJvjDMS67Cl8EJDR|`!fy-E0m?XL`}c@FW?`rIirO3X3Z3lNAR5o|9{Tlt!^_zK_BPH;y;za7># zPfLHtgZM!HA1nMn;LclbcCwalffwz&M9}QNCh(e?h*&ziSlSu?A0?z&Ob|cLWh@!f zE3=D#DP8&o-J^8HM{ z2r+1jYtBj-vDMQ+%BW6lB7SN^mw{&-f^b!S3c-9$T=Mf;V`tsiEnzo?3H45KmbQVF zF;joRUh$FVXsWvO0xS0nIZ0D3QYvlepep%r(&KdIWr*Z3t^NT#3d$KlGhWo5(+%-) z?dN6SMKx2@28vEJu+`H;tBf!QYZi26bT^ddj@SNz_x{;f6+O;sR|K(s5X+CrE z8av3jQfR-@cWjlzP1*doSL~NK?jhkOaGUJ$ZeCpx;&v6GomZ!Vve&BehJF9N*kdPy z2Xkx-d=23(MA|YL&61%POGDW|12>e`R|hS9Wb;gq*-ba0)-gBInRm)5%p{ADIH293 z3WL&h6LoeKcbkAjyV@^6_rJ7p4rZ z*h}&6+q>Jx1MQ>yW1tBIKb!cU*&7p3Wjj76sI%acXi?rY>@iQ6O7v z%~F0spIVvLWb#j1Ine#SF;wUbEvn*^e8l#mR`JG(H~%}KcS`f~!AhZn&t-$I`aDIA zy4ZAm8@jl-yD594tlVr-*wXupqxt}84Im8VAwtc|4-KtXdyPpPIf*OnZbPCayKlT$ zz)-e=>~*mmsrrxU_O$ZPS9T8al8_fnvvT$fQY63F`B-FcaYwlH>9N9RpXCdov=5AH zCc~$c^aBT}AL+jGDW-Q+W72LkL0n#Fhvde@z*uSLNevQ9csDx7iL@JLG^Q{Ob5q{_ zkWB!hDN_K|qye^b+A!}}wrZs){0AoFGwu1UW*=cTV~D&&?7VZ>r@g|FLvlnvJLpGm z-sdGdHsoqK9V>fwNeA84^vA(ijHnY1scYjpY!O&7R@Vqe* zB95syKsMK%69>{@rnsBaPb9CUTOPsJTcwx3#H*!f6`)&+7L!ra5{Iv*5+rM?l@yz% z7pI$<@yYv3^geBOHa4=r-1c?Musr2BUwZC5xo=-J#`b>nfr2#N>>wWPHxf0C?>o2g z2hHrm9O`ruh2aQ8?vkhbFoiJ$D_qLhr2rlgk9JcC?cnmEIiM0ZOzsbveMI@MAc2)_ zx@vZ*r+8@3*gf>I_K=y`%&#R;J!Ja5x-sc(${w3!);sZY?h{N1?i0r9h)<)R-roT^ zw+Ul;L@=U+^W?3kiA!H*uitJLsR%nTdxDz$gqs8j06{L`prDU_ia+8Ig7Cq6A)#Lt zBfGVU4BwosNrM@+r`uaPv{kXtVS@^@^{Hf~w)_FCK$bx9u56JP+CLtp^zE>mW#i0f z3EHIJ@;D~+0`ybq)OOffs+`EAib-%fV}T`!Sb%~4 zS4L3IB(gF9{8f!Fp>a~AXmC>NT?3!ZJFz1f#f(oh7XyvuYR*_Y%7nK;GqzOeG^~Cq z%+ZQsi>%%o9_*GdC=37mPugI4(W zOLR}uQmw<0BdCV8i8GWT24mvsp{}*XA7?I26LMMWa3SNtunv^f_?o8pxpJVY*w?^_ z9r#XzN)|Nsf*@6Pdk;QIquiAw<_`f;&n<4*srJs3M)u6b$B=!YTqfiWpGA~FgR;q* zveg`MC2dqWKJIsDU?Hz%C+bdg3kaRT7BCNWczn6mGaco5)T*oRQ z<;GMEKl^=Qj7U@cuEm2u0(C8y*q-|#>1VasvdLnO?vH<~N`WT9c7~Er1NyeA8a%I6 z>8z>tkyw`b;}AEH!mQ;_b4UOVR&J~Pqiu=BamFbpcUksSxl(2|H#U^8bUDn6F_&p0 z32z+ibRjv~&&}kh~9%6%y&)Fe6`&@7{6ZZ(ruOtoF zdWf3pdnnkSq}rdfg%~#&a16j&XcUCA5A4s7)Ibl>!JWBEpz3=dKX*sE(4e)4Zk>{T zDrAlS9vs})KqmGl9K?FrYsL3!Diz!n;1_FGa3q_}uz32rSSO{`Ga`T0tlPUIQOrF- zu=94YT`JIZ)xdEkf#DL`86mLXJFhbhGdcp_?yf!qE;i{je%26_V`_)ZQ6*c4ftAS; zHPVVV;<9rn?@6B?9@ z>1yz3bSl1QTg6EKqr9OhXiTevMa^7o9#wR>I3kr(VZAUfGGKS1;b-JUZl}alqQm=( zq5M*u&iFbcD@v=2asf{zb82hsW2v~!1Al>T_na)$)etp%tc$K#j1e_$_#ira(mnWCj*0@i&TibCzT9(HC>;5)A{%l1=Mh2gcBal+0gNO z0FE=%F4yfDjV!zS7n;s;O1I!Adu1bQp=_KJpHEU)%fwRj1Gd={ zwq)&|cy_KNjuQ!)rp}e_kI4>+b89C@c0h0VodNfKM1A?ji52a>C!^bsRi0a2`45k} zzs7XEPW0-y(Y~`OZF9*UG-R*Oc88K$j{_+12z}lHKHj?sSKf8~Uh;$MWckiV3xM7wKxYLvKLHR8DRoTC9ynL~ZY_W|?e7#XW>svt_E2CG>i1}l! zt2pTQ$b4W_^THKqBH6xo)7xtLV*Yj=fJ}$V$?@U7J;&T&gaWl(l5L?3zM()jMaHAv z=OA)rHAf^}IQtNcIfbAytH~Icbx<{}!CZBv?E|Rn`75TIHl6U6UG8X#+bX|L$CT~p zi|^5tx0#8HC4ae5lF%3Dj#@{6?kN5~pFyz~93Y!MQdpzRrIX-BZ@ipqo{BszL1g@ZnEE2iwn?u2`V+ReQ1z*l^EGqs#;ljTw&u2Xl z(Kvs*`;>v4C-JUQjWSArXC;fvDY)5IyIrYePI(FEJ3 zRVHw}q{K>xsg%`>-$ z-$s1ic$}!KtJ1bZgGU$Eh|p1Uh7~EQwk44>x3*`|RyYRala}~L{}g7wi1rNeur*ZP zf~P7QXeGc}Zz-{o$r!JN2^NbT3T4GhUJjL3Vl`VPHzehXTwsN~T&Xprg4a?uHd;Z| zCLud)T7j=#cG{*k)ffFwWwRQ)PCfW0&T3%40!zLUowxF6*OU#TwgS8*dU+mA{b=9P zjz0$*D_9Oy)+oS<_)N_0SL%B5t(}{Yk#zl)b&W5di%GqeU{tj3xRqz}Sw!Skks7`o zcNiB~8c$Mx7HwAtp>y(ab_uTfeum+218lCDqbu_vERWEi@sG6DegMP?-;?1opBo;Q zkq>|F%Gv<`o&Wy`4gbm9|39>SQ*@?Xw`C!=ZR3sYij6n6^~N?TNhKBAwr$&XDz@#U zqR!XnKjWNp(HA|s|1qAMef_Mp_S|dDHK$r`2uH{#T;9k3x54F4lKp?mKC8pVqdQ}N zOk8yExH1HElVtl>|N3*GlNe^0>`zrS;hc2Xi&DIA=i1EUpf@odxXz3^CgWLTYN4Sa zS%OXlAfZtYHPzI7nBFMpC@-(SOP@TxdoeY=)IFHgQOM>yob-K3cQ0E-Gekqd|&C;cB%NOP1DLr@EhmpMOl)4cF?7qg+!aH4f0tH`Kp4 zhK}}kgK=&I-oqj;=2Y_Ggzwpl7<7kSk}E7hn#AtNJG*L|M$I=@ z@`$VIkYIqxGN}#j+!)scC*4y|q?=?_kM3CVkZz=odmrDki_D@mOx64poUA1#mL$tp z*RHS5{rqPHeN4ya{1;_f@w%xg9Jo?|*DvqcaY~c6GW~pQ_~w)a^l=!z^pT?9oSx!w zIu}1yYMyhIsJP3*l{Zc_;u%52C;PA zIo0Y5_2(o_sx-8854Rgy3B1C>^IVGv6(X-T|2o8^(K=?f5i%axReN|<%L6DD?&bph z*B*#d1~{Da*zRR_apMm3cPoYImRoZ6)*!>(6^`O{Af|LVdIP)}{JF|18OuU7fMLU; z&PCHIMqL^Uv@Ef%(mIKq!bj83k9anBXb5JI#GMhE$SUA79pKgv* z)Wy-r!hq0V@~W4ULyizX>Td&KZ$LnVS}6(~2~BzU36tz@N&H zWKs&SrldCN;Fp<+LTkleZ0Mz(^v+}mR`yFUoldZ18RU~dc3O4kyBrl>;maAIj&5Vx zqVjfRbUA*VG+5EEA;)KTWIfVAODT z8XuIkqrpx~tP#ItEh|^5kQZm0UlJN9RJ;Z%W_TCx>RBLrTc(d!Pa9q%pI zM}dXIeDj3uQ@M@#R1l=My<}xtS||nI)nm(+9`H=#Q?%{g-rbLRciEepa?AwSCIJP= zCL7qWd*EmY->B}2(PqJ)x{21QndN(K9jY|@ej>1#;0X;%`1@-5-DBgpZl8lKW5 z(*-M0ADhGR4!AhXH5;c*^vl^J=Z6Yfeb7RCD7k>FRM$96YrN|0+zs$}ufrRh&S0?8 z;&eB-mdNPjf#^A2*hvqKckB|H-QxG{vLMqs|Kbz`CVrp#9WGJ}T4p*i{O}gV1P7PQ zE4rx%FtZXRNuII8wsZ zi3!r|+q50ZMrd7kl*Xh5Bc?rdkyn?Z2fVBC_nnKHFRi4fNo|uC{TX%m_CpB1XLnOI zaxJ8^m_uk2N-lxnxwu1iUR%}g$!Bg9Hg2UW2?HCTIn%#HAI3vkyPHpDN;&qDA zuQV}VIWf!Aw#;UtEp;HEqq6(;jYE1Qvi#aRzF(4^lUII6^A$ZDg;4thrylxGLTADm zVaqXpLWaNAlN-yx5w*fPd#TtCl4-a@)uc^t*<2J6Lnz2=G#LmTCjXgt#1R9m!dmN` zk~Nrch+V7-LeQA9Kv_o#2O7a)KE&cAMlCNln`_5@I?40RL7ixUlaa4O> z&PD^q=?Jw26x+2&NWzrxACE!|A2_Er93 zP5A^JRu+|VT+Mo=s&>SLZtj)xXGv8IIV~k1UxYj9VDIbpIu{f-G@wmsxl_Z?e~ewx z&o0J?lT*|m(Oybz9O#l~N2SW~|8(@8FmI(+F3kZFC_P7?e{Gj6oZGVxZDRNSQsgme z+n2Inj=Q0qVdXip?p%r2l-SZTwIt@(*L<^=u`Mnud#>|!bajl%QYt=;mYP&>lt-9) z+2AFHW--o7VM%Bq)nC$*$GSZa0KS;urSRLQJxwAY(HGG-i7A=SJ+_k<$YebY$1^4p znkdqm-jj_7P|P*Kq<|D{O=9&d4JKO<#L`2KT)^vZ%~D5H?m4-uX$+!Sz*DyHEml;! z;RgdxX@<2>i2f~kD}gS^-~Ed+d{ugb9#aIK`?W?vu(B7j4}by(4q>>qZH>=t6=!1G);Uh}WHJ)a z>>wfUO7V;?_mc_YWNl73*C^aMwENceQgmDA8?2Svf1WpPKx{DuQ~JhB?v)d|G8-A3 zN;pI*G@psC{4$`>AfO>-x8Kha(9aNv;tR7B2x4L>9vI7aF9^k$m4V(p?aO!aCa1k>Zb-l>QL}V1` z2m2JnsOZlN+o-f(pT52-3@#^N5kktd9?QIB9orVlZf@x5e$^*zykUg2B7?+z6o@bA~2DMtzLOoCrH$(10X(sH28^vy}x{{Jf zWPuX4t0aC@{jWD(n%#z_0C^xSFD%|f zelyJ!*$ZOnUwCqUpa9aq2QuUbk?P#*?0=Ks_@bV`*DkWIQ}sDmO(gj8MfczIza2~r ztpD}1)uG+~SuM6@PT@#tuLgq>X$%I8kQw{|gAa~Pj)x+S`GH2akTgQ-&X|I|IsT=_ zrDkOXqfNbx4vn^=p?CoUEocG;y=-x1fTqG_PP6HvQANu#;r(#JI9Ub)Eaz(9<}%%J z%H_lJb5=jO0r^Ent*%`?ydqHj7Z2;&u_*sV>_}67i&m;<{&OSdW)C|<7aubcYd+nXVul-B>su)rr z`ke`0Abwyd@KhYf=WGD%H6Kb~05uTrO{URPF|`QB9!emo!-rTeYw;=^vTOc|mcio6 zQsZv~YZrQ`S%(`!*W6V*;XKGoi`dr(4chk6wS| z5Z_Z7q%Z1$TE|n{kt(e$>O1jXR9LyMg_|rzm8F|-Ul&@Ci0mq}|9K&PLvPN$>>+J2 zEL!MkUyk`W)WAxF{ueNg30 z!Wr*2c`SI5PZRg1LqeOXP@#1LCv;g!^+t)tTi4YEjRkoh|Ri}hkSM`IKR0PJD-9Kv_zvI%-Tc?5{2WjXmr+!@yM|b zOa6Mo(L~a>HK5Asf(8l4>$el@Z>~5RZAA=|IX`kl{ww*kMT`Ja2<;{pA0`h%qCBih zb3a|e1R0u{jC~;1Wjuj&acZF>hTm0wd{%$sML4)4PF%1Lxkwp4J2*kK(@KY zI5SZ;>b*yiHqIR@ksvLcMac2sU{rz=_PRF4Pk-T&Kt`K2v^6CJxFCz<_?IXC$xKtnfl7@uynPlAxGq`BM__L zi-|A6cfXFj#ol9Ks$4Zg$uHlt@>W=PX*JwnVM@DL-7L!UP-L6VvLqdV`3*+tPVr9P4}$g8|j<~Imjk(V{pr_xC%*>2^QjaaNfvu?%78?>-UuN(}cUzu-CQ{p@AzhN&(;`!tsoo<9CZ)m^AqjV!R(LWI>yso@sv3{W zS4V}PS%$U45^7v=TzXR;>bK+?*-E>FFHqbCai1aSr=o~=xq^_Tk3L=w3dbxm7f#@ z2^OF`bB!Ci$htkk>SXc{;<&V0-z4Bzgk9H2$(L}u)8qCM4J9eqS(Fl)$&x8vk{7Aw z^6Cg>(3N52>$Z#|6-{8@J*?H(jNa|l+VHf?eOV=5DmNFF7ZGBcXZM7Ywrkd{Z!(sR zBFh>n$-sqNe7~#^B1}5{(WJ@Sos%4=OT~u9Tr(Y&sX@((7*dQ*DlQGCS2G< zmovw7%zDFEfT3K;KkJyl9f1+IoLWEKi+4XDSaV`@&2#y}^kXbl71hcIaSkyD$B;!3 z>0)BssHd<)vTibB+(rj<)JJY`&sFN41}1EmR8^AT& zN=sYE6MndVE7Bw5A93|!CJb0^>7<0N1`_tlWYM-iCrN>OjjON{yA?;IT)ynoG71Bcv#|E7RdL!o{qQz#59I%%bv3p4RU89#>B{eKD|iV3Ctt z`yCzEKBtPtPwKGD6LxaXV?%7d)=}-!j{%WC$=DM?uzqR(dy;-jbwSsiVTZ z_%QsLe4(MzyI2EwfK#S&**pJs*Ph;Qn++q()OT#QGbh*ewTa7nmXv+4di@LhQrwML zcrPXsG)E4E79+kMvg!5(Z$*j@AAm&pMmnti6?yHeuAE%YcT#mO6p_K z+E26O%BdX;UUBFt$0kgB9^KsdBzr_%`}VH*WoGDrV1DcQd;(adJX}TqTWj23?LO@h zN_nPzh9cP}7Eb}c({#W}8h?T3wd6&Gc~&;kcA_ut?2K33%-z)(@n+}&WE{fdm;qkq z@hdso0wlT11>e@`OBvJ)N^)nun7=D+Xg-5Jm7<7yn!=%xUOg@gg8qTr_>M6K7ujeX zN^%Ffk)@6>AF3Hc`K7gC=6(;AE7IKTA&X@y4&(T2NRb)f^+F7Vp7IOsbLsIQSRwAv|P1xaPgIOx8 zY1T-%ZQw&*T_8JCP54vg#hG1tg+v$-b;Cy4qbE9yH-=wjmVFD2)|dVRRs_!F+c`yg zWUtwk#_af3!wuK(iyS8*-c-5Z@7U1xR2fLW9mv4+o~W&k$Qtn8R=kVJv8D>r4$1B0 zM!Z0iCZx$R-R{gfM6x;#75sj;TvFM@unP0xa6d|9zr$qx8bKp-I^e=-Eo)ZWjVT+Y zEjt0A6>EwMp%AQtc-DBsIZXc8vwU>MH||0Wl3#UNB^(^~H#6LRvck%g#@(M?&FmM= z><$XJIPf74d(+ePdJf`oO^Z1BMvQ71SCQ$al`Vtdo!fAA0`OuN?r7OF^daFQA$9IFTzQ78Zh&jQ|&Cro?ZY+M< zdxT@SPnvGN68_iI_@D8RlWTP@*QX>CH2S}5Jfvdt zX%G6Zmy(*6leWqK7GY{iZv?@F#4Cg2V{*u|X2E?9fqBeIdRpm*w! zmA4#wACt!k5Vu2+5MT%5KILWfpti$^y(Aa)BDbjVkaJ1>Z_t-tN*R4*P4$vGY=>_^ z8a{R3HhY^y_gDnI#Fd|Hu$a8$i!;&v^O*7CIxb=7J{_?5gE2|mkcUj%sDn`*ZXA%R zL?j*ph;&sHK*xnS1s=|G%*@B$=M^yV88C?r# zaAys#T#d$_uJb89yGW(woqlpARRGBquKHeD?rY zk{P{ezGi32^+VK)P{c(X_j#lTqeaW%fMVgo)y1@>oy|#Gu^oX_6v?xIHolDPpDy5j z6JjgXO*F+-w1FHDbtUTT1qDVr8te0C;VsgHpwi0%bh;F<3iTO5ri_*zpY^q}`y#wH zX0$(LGnCq%X+ECa)o6EgQr=4PAcx|D!uT4)b)A>lwOrXzL2wQR8DH=jaLpC^!o(B} za98^1ydzZ}xG@*6eU@2~VgR3_0^gh(=`ywNIYZ8RJJpc0XBjnLTBD*%^N>U#n;6aj z-+fQGmP19qMjD0(o1^>RVMyanlEhzh8Ip+;%i*so&WM zy7oKYoG=c$n3)O5svnB}TeQM34*C>W+9B8rq}@Q<>G32@5l0rN8s!PbViD@|IFjZ4Ql|1d&Bf6IRZnEFch&v1By^FalGr z!7pvzh_RYR`a|h$Hr`|rI?*_*L`#vWWDyUYuB>RC3WU3ewbEMl#an*I@({=rymY{D zRgk2??lsm9?tB0r$UdaBL5p%cvNzt`xgIT4=mGALw-y)LW)~I8QzJQmVWlz?&`};% zaH)?}RrW-@fpnupV>5(LEyUuE}>HwAtn7!aYGoudL1BI0;liwT7Lyqu({B>cd50@LP0PId|Xx7-y z=j6nnTviSp`iN#@kPE#FB_J+$m<*xxiOnZ(r-|eN|5dI>7B(c1$fs)OQ*0?zV0#Lt zHHjNvWi3pe#$4B{f2`q6w5LO@m{Px4kKK(=3&upW$2w^ySJ?}9Nz`i0EYbR)!UWG@ z43dY|H)HOHNP6oj^?mnF=%trz^swe+Xak*zd14x|mr*@n_m^`)jg3o9x{g3cd~B{I zcN3WoG-{EC;JNlfK|+9ZF8}xq`J7x4r=HFdubJ}$;?37g{4vMQxpG<;)6r;$BJR3< zRzOjZX>n(2>!By{A`CxiJy;f7dc2|B zsCb1Ot-&Ge5$|C=EQKT79X1#w#jvYHvh&NQ@vRvl zZi8-{U8QHt`>;lFDS)UMKeDR~@B{-dy^nZ%DQkL{Z(GJdbY;X{(`}qCq%tp+q!xT0 zVHd2L)2uY`-^k@H$$2u;TQu(-203QuqyWh8KZIGMB#w19#X}x3C85#s!KR0l0*kVB zyf~gdhzM>BskYbnszKk-y^Rz?Isj4oDzwJMdr0^Lqp-!X`!v{<4v&?Sd8u{07IK)Z z(3}!k4)u2D%X^?4JJ&8s`vBB|z3VKR?fEdyF_`ofD>*&tLq9IAWheJ^!beWwdEWG) z-%=e(rphLvj6&dWjmniD30=2A*dQr7CNJ@0>X4bb<^xup#=lq_2M-N#aIik7#c`aaH7;^mamfX$Wd*7kUF*KLcBncc)a*nLVG|E#%z=!O$tA0pW4%IZmCD{oJ z6A)g6FaCXuE3il7AQ+%|e$o>O^csuGYrn1^|2DLlhBUsyYr_q_Ir8|oS##(kZOW)a zhWbv7MicX1nm#hu2W4sOooZ1}$f@bfYB;aw*`Zc&ve0N7Y0(ZZM-OqPC@L2vDhEQI zu{Wznj8{4}Z+y;Z{P6G!Dsu=)x7*-a&NcGG2m z_MwmXF28rpK(X6p|MjD%8Kqm)go`2`r~P3;2T z7g=5VCI2XA=)UnDV=0q*Ps zvSN_6wE_Lpz&HNXLlSi}F>?MqkTLxWv#40gqY9$&DZyAKeM66L@()$lG!ET~6F00T zBNIh&2bX1?JI~iyyRd3(gZh9$1yjU+dot%ZJ6l{Ene@S7gbf% zVT%4Xw|Yaesh>7CKYnQ6ufDfiwbE&I^-cWt5)3&#fz0Bh;Fhe0=^!&j<4<5uJ*Q0X5*xux4q}NPL$Tf!dAPlU1Ig*Qsy?r(@lG!74lZa8X9I!c=#S04R|Vb>o=)z zG^NH{f+&1$$s(Ao4R=%_LvDs!Ae37y!y`*Ysi18EfNqeDzS@2YFaUrQ;@A}3pAE4A z0zNY4ztbW2{qd!ltX@FjhRcJ;Pv69hoWxxpXsp+jHPEOP>mwJ(-)}m*$Qt`)u!V=Z z?wb*5aCNCwqjA09$wy$0i1nKwkCmWXk@~ZhqQ82y8Gr_2h-3YY_4b7&?hP!FOYrP_ z$H26-CFJv$su^)h=2%(no-cRZIs7H0@lZgc2Qz+uZ!xV0tn0jns-r&{{C?Mbp{Myz z58M{6dv8IkwjnXW(?gO=fnY&QC^g4bhcmNOmB=q<@z)-FyM`H#Byn$k(nAt>Yz1q+ zCr%Iv-B{$grQ2eAzNk&PhhR2yEOJ??JZO0wl0^ifTO9v|?f=P>CcCM5ozFb!{`8Ib z_vH!mzw$&`>z_P%FQn7a!q7ru2o6pPwf^KCVt(HSmo`+q^9#GS=gzRgY+_04K)PB= zeu89;adW?de98+q!^r)G;&np7WA>14+v%9%_^@Qw!e{{U z_NE66A)FxKz!iK)p7+8pgzj@%6(PXPiRK#+-rUuRf$EG4^3yb-qTmy#q+@F5LOg|f zmOG}u45?%yR{Jeg7-CyLn^^@t%MMQu!alufK^loGKlO;L~=dSb(0X zGM7JoV2O!fL5RaS-uHOXOCUcU*&a-H>j+kAHsFIr=ey(z=tgcZX_H$9x=bDOe6EeB z3qV-X?di=BI|l&f`zA;BVgdJZQL!IP5Svp&u)jV(J{DN{^Rd-&gEg#U@_ZGrq6z-L_Z z{afRj-(mT5(>JAQ$CQQ7V`>$B>0Sv$9dSxN6;9frf#n7( zNOIqjoiC6-c7Af+<{{Np21;4 z$=B!%Dt>vw4Y+NIF2Dz=;p4YzgY{xd5V+A_LLX)*d-EfCSA3@*n#CT-R8?%rB#G-G zyTRLb?H373XXW2q7!7l#dV!o;sp!}KorY#OM(UNeEo&{tbfOc~8+joe6o|mpi3RV; zyoSk_6+IA15n2hwS0~@&|7;Dx_-861b~}dnL*>RX(GwJgyTS z-(KZnB?DK8a78g$%-(s7gA9sRN{Wg9D%!-#T;6ny^^}tVd71KK1T9-_0iK?Crh}^q z@AMmr1^bElAI^YUcv#t>f^aJKL_5F8+whfPg%o<^z7t=IemllEC#-|tL)C&srN62<0NGg4n;-L?CKjOaXVH)~jiXbFx zWHhw?KvBP+iGWAb*#tPbLM0Ydm`YVpf@UIQ13(}ssTq&g;YwJeGbQ{-uc9{V@8jd8 zs*lS@-LA>kzi+#rzCC0r=0~bC0UMfk+qIue^lNJ-^zXrCSRz6LDsUy!_7xC)Fk)!o z%*yhqjmNS~%7YM#*d%kz;Fx%kbOfH}GKqSxJzcWd={70Ad6(gHbD-b6q59 z2FOMOcg<}udt$ZGu8{7c)jv^GksFk8*c^lGF)4G}nN+Rh5mb~ver%|PzWnq-+xR*eVwG)x!&Lr~h7Yt!s0!#rh+=Y*5 z2_A1Q=BCqJ7SXTOCd#ATqXczg+=q}H3-yj8^Jp0NNX?6dJN$8RYz$?3nuD~OEA)l> z$S%y2dv(AcS=WZXgU{iZzS^ty;StcdU?;eADFL_=5&v zvgsr9jGYVGCO6?ePim%Qx2qlw8aFLCKf$0&BNMYx{xccJhgB+4W1?v^;t-upcgKt& zFEw_*sm(2{=4)6ejTomHgLP=F*$B_w9-}xqC-Iaag42U!HuErr z)|@1?Ru;(;Ry_h`4ZgG0M`CnuDSx}97(WJQ>ikT(Bfm zcW+k3Z$`NfJ^Fq+CA)T9p6`D6Y?1whsHw)PaQpq*&Js+qmcOMV2zGTrv=Q()dYra@ z_kz|g5bS_QC&*+IGuJe_VO{ImcD7FJNl4WTiN02{B@Dw4*d~mrb4B0ItK}=)vH^dj zdQwM{wo^jpr(czqSfez#R1yRX7NOsvZ+|Y;G;<7#K8_?j(jFlif!-%P;s{JAN z8c78wdDX)igV55WSttYW!ZC~@(%mH{dGYqPVhpv3%m@-#qQJSH!u_X4dk&?p+`R;Q zb_K1vG55EmVLt7bQ6U1Jm^&5yt9C4JZa-J5X`jT9Hv5~gTK;TnU zPnkbJdAolmF(z2s>W2ltHZ0dVnT3^*qj(4rWfLxvemiW$n??W`aDbAxgRMoihh{2_ zn`1;cKs@KIj>Ex6r(mJ6VojZM%(36ho~w=}zLRmDaBI$1(|xUp&=S9YOSPnM1r?UT z#uqK7rJ}>yUUFxPx0C-Rp0IMD8&zI5eD6QmqVB&+q@C!T+8x@~7z57}wkfBkb&XU*mS7uw;nUu@2R-ZG-~Xr4bQ;GJ_goH7!LL z!K(H9$Q5Nehil%6y|cU8L))~Pyoi%~O0tdP$v7Yf8Ur{_)biMte!VF`!D*q#lw*cu-W~Hol>~vs1S_2^ z!y~jGy}<<(u7zJE227f@2coXayKAw3HHT(x7169*vpgRw)Sym{OwprP_1E1vlJ9a} z?3*)evZTp0>w#g&8KC*W5N-mmd36YH72_6 zz$Vz}V2PG>5{|;P>#H!zN6N06Y+=O=?t7rG^op-KAbd{Kinfs13?^1Y7hx)XL?wpd zTPT%+XpVyE`CXa26EHv25IMbpZ;>ij1ih3>r5{ga9NnapQ#QViGkGQ{f`7EEYPC-GYY!%ivMA1+!NwBT1l3q4h9m5JIdXnEh& z#@IP5%)&`2d02=!2diGxVcOYOc)nv8EZQv7tI{2x<8mFK%K5Dq6}di(^tAC`E@#~Z zGKB|by6^B&Rk7&K3uq)Jd@Jf?>~&UqK_y_+)zqGK%B)qZo}+J+e6%nPnFK3I1*wQt zCz49XFM_JG@p-bRg&T(|vV-o3iaZNuCC5XaYEte=r{#I&%Ceo!qwZZ4A4;R>2JL4{ze#=DK&#k(l&^g_bWoPPG^T45W#s0NLy#H zYyrbn*n-)00M1~4VzdY0l9F?A-SaP@Gzz>ww}h!|DoSw0LVgG+FVmtoR}wlJyH=(H z%1T-YP4(|@>|`GC``@O(#4UgH1x{H7jFLVrh2&mFYbhzz*E>xrEdOxN`p{4z<# zAJvAq1wep6`qUAd!JDq7{t@r)t|cqGJ}Q_#7v<<)iJrl$uOiTIPg4M0)rSi_ZgAbT zn(Qy}w^l~34m>gM=bfdohgRKEWX;Atm9*}4p|OKpIy*QSv!k1EFz|HCc==meD<;l^tx99EY=Ch0{CApQ~mvy$#pO4>QI8Uw+RR#??k2KrI<@qxKlW zKc!bdteW#{lQTlU+V=~llHW^lD?#Zwo$?yK@(Pl8l!AdgP*QO@C*FfO)O$s#=*mMe z!~Kj<$W|(&Il3Z0VcI?3pxlhwm$%M4n(hjxamlTG0gZJ+bBi$&iMZkhk4}5I1vp{~ zhRbbf0a;^@g}HA4mX<$cP0%|~k)OG$I&_KmUe@;q<%gDzK9<&P^$XTR1-zX{jJ`Un z^MSOjskSl%7Dmz;fq2Gc4u(tsjg}T8dociuF9_Z8JE{3aIUJnf>XGtWg7-N4STzpr z(Nk@N!fkDbpZIFGHt8zVr&~I%UEUIX-vsCAR`j`!TgnoFIBW%xN;4$vQwP3TZdAO_ z4Ni|HuY}F_W~hKYu3o%*q80=u^`H??uq8*Y@2hFoR9soz6`Ycjd7MGNho24B#67dR zjWQ)4fIDGW*=oBgaBAzP9jP5!p!zaU>ld|=4YsrO!+YIRla`rq-*Dz|J$a+d5T0v@ zAV-4z_K0l{aCtkOl_Tnv`}~d&mamhnN$A@X@yz?8+x3{}5{}_*wmBRv6l1P1yiMSk zo54+g0T?XmGJjXwFIQ9iyYodUcXVnpQq}$DH-ntos25%_|0g(R@cwz<`1yvFfdBHv z?EeQiRgdl0@c}`XtDHvX>J$&&H5^PE6k?&Vu#;>}+3j|JnNy%Y({o;ex zx13;^Msyp?iP+Zad3loVdAZRstLOXi=|3yoY+}#}=I1I!Wz^H#Mviv10XDhq&F7t6 z^a-zWi^^6!WTgpDj#~qx008K4@E9wKZ*v!kYjKk{CyJZLTbETITaB8cHQ+Z?@phsF(pWtcH(3#oby9v;BPjk8 zqC(UFgotPiHNhDOshhPsR`K?`+l6KeMeuRASSw4NiI!)y^Oaw37kz)4e@cP}VZ5~y z{yp~Zw%At}IsXy#>*7eYxm0Zl3?>X1H>7FGYvJx{COnWe9Pu;t)~4UvlKJ7jH4l)V z#*fNWFg3VG<#?A$$gTp=--HrIXMkXqymNOSa`=<4@#3xnE;BKpEk{wDO^T|{b$oVZ zc=jBU1rx67R4AqP$hWWv6#|}QX~aeV?d8+n_uNG@zeWnZu1<&pKDGvyrv$iKW;A4K zT1L~>m*B_8_}z_-Dy>TXO%IMVb@FZg8tsw;%8#edXa~AMS=Fjtoh9KUTQ>ua5xz+p z<2XIdTXaXsDf=a)dZcU3a{ie5@AOisblHr}Sq;mvs>Ve_CD(aa_ZqT;ZxMd-;I1~% z+9OBkG8?(BWIeH`G1Pi|t1vSWoZ>cjrZJ42;yOD5@{YBl$gJT4T-l1X+RzAtVa4vM zHZTf&Z~BGl6$z0}xQN^1;cWkt_!c$k-Cx{`T>0>{SnMAh;6sogwqk0u0e!=Aoj9v+!jR zF>m+~jv(g%C4$gZmD1H5*$KcHPViAd&thD;L=#~{T@%; zFZGbijgDJE+Uulglr0EY`WAYnf_R5F2ks@B>^%zU&7auDp9mCk$qhbc;PJUB|C zziI})C&yiuyXfX!+-oD9QVq+07u?9cn9z>i0Y+y*`nL~UH^!W}jVg!f)7WfD#=n34 zpQVHSbUst~Q{nFE)4~M!|F?Af$7^~))lvyf9F4D!9!^F~1W8mDj?BVE5RcG308AvL zA6!)zW+%_JaTz~3V>vaYfb4_pLx@(zl16!7I9C1{SdyNp;YSpt9;^IpU~Bt$?m5Nh zeR+N?@c#OY`9*CDn==B4Zv>HQCmdo=ZbM@)9RiKNiDN!sO-=#t^_xd(Z{A62sNT;d zywOlB!kzqrW@24+{3MxWIszETXsiD<%sPEz&R$R0brN%+Wf*apU&{OErr<@V1g7naO7pkg(=G|p6 zQZ!pB8hI%hHN2*{0f+A~79xwSKHsdmfxcmVtZLGoyduRMN8OL@D2Ij2UcqiTOvp}z=-v=R{=z7KvesGfgL8LLNlP^$O<@RS%5jp~ zoum|8X4&>*&K;o(U4}>}C%!j>7$_JCmJ!l|5D{%aODj@i=J)y+0aX#rke}k2Ofo%{ zyl8XNKn^-0Sw&p5e2f!xn_!_UVgirJEJ+kzVmu-Gl_58&=3Lj1<*%QT;?kr~tHs&r z_{9+}auqWeO$)UOEFS!MNCzdUrGZQASZi$o<>GYpIr{~_(PR}kJFbL|g}5vCSu8XA zHL(X<@JG8N`gCmwf_eaK% zv#dQW4hgncR=oX4pO_!qNNR*2mj4*x{$=_RAC_Lgg|MS&WeaqC)~`?OFgL2lrCgcd z5NZ>eSO}Q8>o8ZUPb1cRkC?6m-cnDIOZH(ngCY?dr%R_-&{CP4Ya$;nfXQ>|1kYc+ zr8)v13NSfWmVrYxg6}6s`GWX5Q!bWKUAgNBF94^}7dZV8w8WV}^t&zMvK{2gT}hcd z0cgl97VFSKfYKv+fHmcM6_bcOAMcKVg_4(~-ezTyl?)+gW1!8TeQso&K+d6fx(5N~ zwwyJ*-L3@QEQecGUe5vr#wj+|e^-|KXRdU70k?bnxh5F!33x>RuPfGnwCD}&6hCPM zpQM8Shz19$Bd|~8$Zfc+wrnSqlos^@Z{`xizvRP|j@d+OZ zK8phRcdXYT%`rnhbQ9_j_vwkyUt!uJ-$xu1t~!HtCGl_lg6jZ>Hxc1MVkVj=znJZN zsV_rbOTx2IwY7EfMn%v@))>dZyUHTabQ51g7y}yYe*5jiv3y+T2sQj^l06Gq@-Gsi z^cq8xqP#n_Ht8dYjZ$8ic>*HSCNMA|I>(!mKR@JS8`p&4^2_x&+RusqjGn!0RllhT zN7zOWdZ9ea$Z=LaF!hxeWBv7Pw3^o&0l>P*q@wG)tHH)v6G?^V=?x)zT|(0GbzohZ z4epI+P$p-@e#J+>*kZ@2&w`WHTx8;ARrAQ8=szx;5b{1I{R8=O9%rq7LpvFLkBD!F zaj0;vk9#QI)3*1&NPDN?%%W~>H0jv3?c|MZ+eXKB$F^hs) z^j{czt=fNKk#2`$T;wS#y9sGR4?oI#%g|eag=sBP=w5sO!6EN-+JfJRI-Etey@5jz z$WSAh`9a`I_8rp`YP z1FiWkFCob-3SNt(&)FB23N@7&v`p=XouGHk0Q4ue+L z(U$MIjQe{oQ~h68x&O-9HMTWV{cox|AHuIsj+XrXLCb zNP5x{Yy1?~1fE4?D+wG4j=w~mA@{j(;U%;}6m>E=vWVQr7DqXEx;N3@AXu>)*dV*81Pt544Ek}!D zC_-J*Flr3d$M|t~wSWg~4MnF2vvpcIgV~hwknC32 z+gY^My1ECuZ+G5@u~%mN`Rnsd`OakSrZ9Ua+KZeB_D8P4nm2Pc$7Xrmk_QF*INjn` zg#6ii*kOu)HuPL#=jYuyHqy!SF75TOLFH@GFPH(vPz>{yU$zU zcs!Q{%+jW^D4w?+8FLm2DTPp;YH6EFjwePpCOpMvoYo!j`+gC6puiQC)y?Kv!6mQBSfhqMe?k0b z%UY^r2q-Nipr{DzH?s@=r5^^Cqp~=*rkdC{Yni`?=@1Nq=zt7^NHn=|(DD5WqKXvG zI`8kVCxcNu4^_xO{~*5JD5{dQ!uq_h<0zv`3IksBMjAlKKi((!!jo+k(cBb0TDu)P z#xMix3bgFHhgZg6IsU|xt+pjpoI4J#=xeeBfA{&3%ps{yU0=Gdc>7|k)3@_olnZY! zKNWCWEd2KKEvLf9aSgb%ze+-AwDyadt$53!k2_Bh&Sf_(l4?wUp1AML-jqp_w?+Cv z`4p)%rD^98m}Y$&MH{qyi1OA=q%Om6uZOYl*2DkVoPM$6dIO8^+!z0WfqqEG_@fN( zVY?tx1O{ONz`0aHY7FDZJ^0DRQGRJVEa_ZPH#+P`6AAVGv{24TE+4s_5x%aqUQNIl zVZI{X8;FQ*m*xrbSK22ipuqTt2^z+i2Jbm zaYiKw5gu;55)Uq}*?`c633!8gOq$Lv(9ZcAvG$;ju+GGhc?D9X9=ds)&U|4o*C7l& z?jMBu4`q-O8cPo|gc?$FT{47%D7 zr{bQM$cnG3^LRcXDmL+bMa~smrWg;jnjt=#i9T~?xFW;*CQXmLF%L5urRk56!kI+s zJJk@D7KBY~Fqe0kR~n2l3oj;-?KnHKwhow514IoArSGp9*1BOhK?933I76ZO*+;mP zf?@aVsK%1EbduHPjPr&lvUsJf#j{`~KUU-qdEWl#<@MMbUFScepw4%pRQO*PNikPD zJJ0`CA8S?Q|HCQrJzRG(h7S~qezpD!OxIJP;_vn8Wcr#mv=qyk7A^ zO`aemh5zjlK|NAqLZQB*n?P}sz2WI#p7(irQ@;n|=C~FZi2#Mkkrl&Xx83BFtrV|@ ze}wkqiQ5Bp%&1SY!quBBs5AdHY92SF(Mw-$oYtFe9K~WYcqDm2%DgbHx+GI^2xpL+ zx@_t9FdOWzAaCq(+O7B_+p^a>L8&I82{kV?ibVjj$+@puBNOGKVv~wVRJ!x|bMq2_ChSJ0949VRpQJha z&yc&=z(CfIx=V7`eHOP<2(IXd(s_s$y8F*4+xa!LnO)|xMni*vw`yCaQ_8 z@;3cpFY#T`SfaY!&5yuMXUlNKc~Py$3N@`YW}7(-wnao_CN^5IO*BFS(5ZFi^lG{^hpI_{t6%jq77z^FAL zs1?G`Gbt0DS)a@#VWuYOx8*^4L8dnRUaRNV`7MZJu8it9P+k~(>GbjNeTd#{ry(cZJSuu9rZv^8#IKpZUe3p`zw$hBanAzFIwRo)051Z8+ zD&@nLZ2YJij=?0^mH%D0Wz*_{`JQn)5*CLHiY{^j0+t}>4srrv=y5VeD#F4ctF$#gcm@@2E*9gek-_C>Se95mM4vho3avU*RXIN-S)KezG_ zh9PYF+nf_`9&hbeX!|bjTe{>v_Lm71!I^ASz&Y|CRzwutM;Y9q_0$*CqM~f0$IVnf&Sk!SD}$vptu3 zKwOJ|CFNbB{CeNXZtR}owPqOmH@rgJ?9zKr;1Z+5;Ev>uJWI87;698 z3M-YCT5Lpw2n{3^Ko;Ic4hh7>02SVf8}gtgjv$T*>Om0~7CQPvwDYy+Zl6?U$3vTa z-2S~1J>}hdf1h)ogNeuQdLRbDXHswGGsQW z0%{VxvVZUh_6_oki4y zM`zdq|5egQWB3%mA${l06H0IP1`Bz4tYJ@V{!)a5wTu2NdUOUmsJN}UxVV}I6@CQe zK0TC|w6DU!dtUC5de9LssGJ1fuKHfwhvAR8x@<==8Ge-df|M@tvc?LseRjN3Ls6a4 z-%R;g`OEZrfo7-4i(}}yux1}#Uv*(3Q1;Pca9PLBX-oTUtSE;^2;Rutpyz-+hcFHB zAj;3^Y~>K?^W{z7FGntw72Il_%jkAp;t)zEu_5w~*l??)Ez(sSB=Bm;HyD;~n)&Z^ z9o}LwTN{Z2i1SC%z_zMCeQ5;d6Zs`}*DSi0GF+zLd+epGzD+D`i#JZCU$mF5U!tT%w70x(lF5M?;tqeia@^a5ioEW^E%@GlaJ{GQ^GUuWl=$A190tGe zB^{BYOzw~l4&>60AlQKFHG)J1he#=2r3+r;CX6m{_ zg&HB}INOPm6Tu?2fo%%+B#W%s`53>yAtW`jz&NP}TI4z%i?@%<;c3w?NSH|=vNUd0 zps6r+zH(;kqt?aYt#Hu72RN-KCVtVVRm_^Q3;D+`|N5lVSx7O>W%|->+Ta0W!CZ`< zY(Q%ti1?&qsQH-=IhoNl^40LCU5DUp&mrps3q9Y`NAxLI+Kn|cS=BEePsS^?DP+k1Md{)asX;p+$+h`6s0wO;u_wegd6oW0xDqlXl*ra0Q^kprS zG}lSvg8193*JOIIN+76HpfxQy6;(Z?#Bc>GRzWqjj4_ouMbK!eN7TDv82$+b9aEcF|ULoJa+8zLoL>P+_G!Kuno1(Ry&jf!Y$dr zt})~*H2|EMaAr-m`b-_3(R;6K+{ zGLm@&lhdeMQ?8q=4sjI$)||YgcXehxcv?A=ldR%C5)aNIIn3LtEGf0{8*h~@aek;?#GvvShnf<5#iA*hPc#b>9PqavLNi5@r(cH7rr7g(M#V9!MBBvP?0cD}v zdtJTyu}P$Z%M77v6%5{t?6#$zOg7dG<79JhxjXV8$mkN(C{ z&0j>eXA=r++&Th}VMh1_6l2u)VW>*7V;zT}tqH64jvsT41GM=CCb#yZ3>$PDe(8CbQa;}*Fpi0c7XdM4&jOE zVPs!15Lu`uw>UThti;511R;J_TymAEbl87|%N33YM6ii-&gjxZ30|n{mG~AyqsR-q=Jz zws1BRD#;ev3~uVqqRfFul^PS{wB^1Fg+)glr90m^76o1BA`KN|loqt6_Rb8DOq7ov zCLhLewHyQyLwakn9cZJ&>TUKm(}K4k#HY>MIlcH2Xq6r?2kaHA*}-a-D@kcQtRnf# ztd&lq3>|jk4OWTg{bG0w4>-bE*rfAe5bJ~^3GXRZ*!}klT#-L1ocyGrfaD=(g#8Y@ z%$K~J^n@kbMFKL~G)#?JDQr729zI{%L{-9xNpcHRMf~k01~`Qc6!{H&cvr1tVnS|) z)IAOhQ@i4{5=VRuy5Fz4TQ|TVF-6-#D0849y5*i`$rVHs(qWHP;s`Pny|iy*o3(3V z1f7=188TDO^)JPdd23c^UKF5<7ju|6`kO|;gFTJYdb%@Akm2> zXoPOcO1>W>L;lyoLg$y{I?W`En~qv`4O)C9*;fszYwZ+k6Ae$=2<{QY&7*S080k_P zDXSUy3yzPkZl-WTSE-fnBhZ*mXm+YQz~ zlxuMel?CW%A_l0^;D`75QW=C0onguCCkigNTVO;hgxGpoJUtThBcB9m2-)~Z#gZ1z!E!P(11p?c2$N&8@N|CqPltY7%LB$iB zF*^HkZvZ0L)J>KpHE8#Ze`@}6FQ&i7M6FeFdihRc!ugRRE=Lx!mgf8=RfjlD>)=Y- zW0Tw*z8D2jx7h@YuqaLX(zJeP{)O3IH+#4m+<>;=SzIWH4V9QXC0yBORj7o4;1 zX$!^rQQMY%34~EoU5M|}lT|}S^=1|FgXe7xt{~YP*N!4RC=r+q1GA zgQUQAV2yeI3ESZYg2){)#jj};l9$UD(Xer7$2|?8XYE3%X2WlRW`ym}ax@_gxPiQAwqE|5C}-2FvYATmr)gQ;7dm(koPZz;d^{43^0wrCPTS zNq#fbK$>|zp<(}9_kmvYIijBjbUc=gfhv$0$n&`p_()1To&6Jbu^2S(3dlT35{Q2| z!R;K$Gv-PGf4zg{mbJgn9#h?Z&Gh^Ju6ClTY=oVO%TR8peQ*d4SE9M>(Yhk3@V}rl zB^GGwMr2$-Hf*F>C75g@Lfb%$28`7~oJUv6j0S{*1_A*jzYh4#iRdk0w0iiF#r~{-Q1na=;@ry$*Pi*O@wX^5kRRF;AJ&mboOELK^A-+~ z!1@!Fu%MvnP?!8rRBSS`eyD!mC~ALtK16bV4c$NyNbZ5|CraXqkp~)&qK`m0LhF%Z zzAjBfH0S_?f$gV)`AgbB!S$sdW%AvJl<9;oX@VLZKLkP#kwOlwGWn77q)o#FRrZ>1 z@msl=*p{+Y1FDfuIMkE|men2ydU^{ME=L0SfRJAdAMG#;>TJGDWy^!N^1I&*sYwu((}-tgHEK~3+R_EHGp;)(!KW=?m*rR z^Dm)sWl;8n`m&<1$dUAQ!{u%#T2;oi(b4ws}72;?%z%x2zCN*9?%66M%4*v#`isc+8nSnBf(Jww+;+2A#jlf&q1nC^U7P0aW#F8pH;<&i znOJ-|fm+DOuv}<@Y8+owfj?Hn5cBUi3_1uLHMmC7`4HLFP#IyEDvbZ zzP?9%k16QUzlENbbs zS?Pq8zlZK&?d`J6Xd1TXQDIvWfB3&IrTC4>{PV}9L|!>ZkE2ooFqYhyp|gKeN&_N~ zvWXIp+%n=Lg19C_an+=X=$#9=D@5u?ROyZ(HjMQfW!H=`&lM-qV)fP*16$kHWG0O& zjw*yXHkr4Qwwc4WZAdYGK){3_3?xnqNanAY$a1t?dAgO38FlK(BYL@(Ql|H{EqG;G z98s)JBXv-$$>wwJP22?j+0eWSzT)pJ$-z2$8KJ8S^R(|C=KP%}rf-OTt29kHY>uwA zNls<9FL0~7w{x582gr`~Y4@d5{1d8CpR;_za24(*f_BRjE8>u-?9x!CRq3?w^UiPG z8og6*#h@L?w$1OjDxhU=>SQzCi~N{|O@TBR;s?EWh4`iti#iD|>`2!#NzGh=3xnqV zh|}np(`ywUZQ40F@p9pfzr3i&y*5OzSx0xulaZGafv#e2)*JFIMqy6jXAJiukdpyB z3iSlca0GQKZ`dkLV*Cl#5**2H#Nz4oRDtjuZAc#deQMhoooaDT$h4of>^rY(iT*64 z`OHVv@9rI){3w$Y%Pir9l+%rLDLr9aH80@Kl)-=NIXQ)09lm89I~Z`f(y=o)=(y$k zkb}0P@M~9#>^FZK)LeRsURZR>BLrDLQs0C`M73XfqkPtIFVQphD5EA%oM)+=pHjZ~ zPq(}9t#d{Snd`=D71CqEY~N9K!%N8e=~DY@=8Wy94}Ae#$}Xh*OE(4jFgTvN4$PjC zyQGrl&RiNr*0IMyLKQt36UdbeEQ;lI1d1!T)6VTZFxaf#@!;Cp-B}6TytI6WnyIGRE`3`i@+& zcU&#Ab(NiTFWHStH_vgrL$oTIV-34;Lm0GNOSzqqtrBqrE8-G zE^nEJ_xW)vn{7=`JVU&$37ocNTB>bF6owv#6!uTGgG>OTt`Ti&H7fvQ#RGF@l)#Y` z(umAWi4tBI4x^yfgF~!>d?%8bi|9~o$W5iQ3$1Sa*S^Fy1phjO`=LGU;aq6?-NP+p z9ukqt`8$jsih2)-YY*~{2TI&{v=gl#4Beex%~-u5f%MRQfn)T_E&Q~fx*oo^4|+A`chL!#KN9@Pwj2EWp!XH_>&M=}&jasQ z5WgU#C_Nu3>{mMFD5Pkb?}(-FeoVMKW##C8lB&LO)E^(s zDk1*LGI#7|s9)-p18b?BuVGEtdn#!MTT{+Hl2#-BwAr`%6md5+n)~llPp^&6tY5O7 zs-`qAaPpo)>#5^nZTBq`$_D6N_ne94JxuqhqD4Zj(IV`cTq~%`B?=g$7xX}t>QLQW zy0Ho>I8l9yu`0X~t|xeAgFnbB$?vYj(zQ$c|1sR0`a)$uWmGvULD4Lmh`C6~3xV7%8vat3ZE>Aya((a!8It08*;NH@Ix7#A zi=H`r!J1FTI>qgi>O10$?j)FoB(uGE0^tflbR|4AVd5?FPK#rg>Mn)Zv^mMZJ+?!b z&BYv8iXbGjwyd~$sf>>#A}K6as$FXC`c)2$@e1~fKK)f`G0G(epMg?HdoP4_&WoHN1+tBhsTi9#%Gmf;RlUV!?M4l&NqUzEpy z6wL^pH9v|eFs|VP15bD=sjsNia+q(r3JsGnp+efWd-89&!OD{Ke7XlmT;MbnQWs|V zw;g4C9Xa1*E2vCvLtx@-?8?6HZN8pPqL5j@%=IL#8NZ z`<2g+jnSs_oD}&^vkhxTbgC;BDzo%Xz}O<$s*sfk-7giLOj(`AqrB`SZvz$kEKnF6 zMY%~26%~m`O0$u_r1`abfmwsM1JAOXD@^8$&D8+n|3zQB3_3xl?*VJ`)OjTzGE)A znyI6)mTr|F$wmCdKTdM`;)`(daty(Bo6HHMAb8*%yQ4?OU4#GPs(Ghq*@Oi-Ik-1wR zY3>pIwuh!$qW+xln-R2oM$oOI00XHpjqjLiv!2>nk_Ae}40CDUFN3&QpTo05)0`I% zm_Y|*+oac&<>C#2v5DVaOGrhfieDzpUY#P?;qAcJOn^6QQTzxglEpa+cGJt`InpD4 z;R8X7+OHCLog?VUyP=5v(+!sO4z#xReL;+9!sps|Z@HN&o6!@?whyc>fm#`SIn!6b zhXB%cq?!JC?`G!g^8ZS?YBk<9(Z0EWY2kRQPY^#^qrgikBG{i+s+4qLlth(l zlyqckbIE135W2qyWtH~^hxaz5cfYm1pX=dAEvM^y?~+fCb61q)*$7$+NZ}j2tS^3h zGr7EbM>qao9k7PHC+N0AAwlGnp2ZI*B|)lEPBu}TVL#BU>~l2?NroA8g)x^{2od*V zf-o{*_7g$oAaAMjePh7*=4z=tuDI>L{de;ZR+spmaqRi&4m&~eG#diJax`y$7^K`K z#iX0{kQ!;n+n+uA!K@U{F~hKNOw(1+X=7H@ERn3Fu`1GFc^@3_%Oi?0mU7Eq({f221XvvYg zXxby;BiS00*?O=UP*;jd<|3jR^HCI5sGW#DZOS&auUzz)XCyEOWAzeve*HK`o-sQV zlznPml;L!WdV8w6Hcj1I{O^?Zy@EvTAi=IVy;;foNf2ZN2TA4Rf@{rZ!9a-AX*-|3ZIHK_{*qwG`1N?mS4Pv_2B=Df{9(D1X zC-1uMgD|J|MP|?bFjDh?HgD%6OS|b590NydFs^~&urH6{04$1WSMLdSSMM2@+#QEv z5jY|vs(i)HpLk7i7f`+9?n`irbni=Ps18cAv1su~qpLF~KX-zDQSC1nLJdMOcUd1S zk?0z-Wta$ZfgFUH&Vjw><@3v7x~GH5p862`5zSA-OGx!%Y$nsYIWc{m@q(%Y?C1cc zeM3>dWA6j5w$t_vziht=kH)`7>Izn8jGEbhiagO>h;j}u0E=>4Jup?p$k2OR?|bNm z`>S3N`eoOfmo1X)v)f%LY#$2{GU=I(v(qq*aH0~O#D|{$(WkqJh%-~rp>a;9P6gQU zq(?on&Z0uGOR!KR3rUJcF1dWM*B$@X{yB+wuX@iWYt{;csWbEKV{)4<0kb{bvkl}B z3OZQeWK;+9i}$RZ-E-mu$V_Qb{?hX<{`&pSvgNxW{z&}EM!mwv==rS}_AAc+SgNl+ zhG}eM?z6j#u8tRdiA<9W zaMtAogP{MBJyFOLo`r91QJFu?(yqF%E3Hh-k*Sqs<@wO_2V&QaXqRC9{GQO=bIVjg zcaP#!`xyU6vwjmXMq9WGubmqdZew*sORdo6iD(yGHx^m#wD4py?3QR(yi|AOW19_S zFGM?I9`KmEA{0J4E}T;Y=eigkN$*$Ki&CT--0ml1{XLZie)Ui8aCLvr3$qSrb(#WN zvx;{yJqg8FCSe$4_Xf_fGgdjcUQarAltk?yDApNGNz#xSp+DU-DM zSIh7FRCK1N3Od+-)&M=P1}eEYxTcaS_t*i*1&YA9--mgRckPwnEn(9*8|Mc?PXC?R@As&jdxt09{FXlX;RYwYgnjw_Bt3)) zO}_TKH-+Rx2}w}Vuf$XoYc`Jg-!V>EgWfwI2_@lt9{@qp)OfE~01?f>_^VTJ|9K&I zk7iW9<3qB&pUiGN?4S0KnM zt={YW`$FiI@{c%Pv-F5P^O^|ZD>+=J2H3+`>@z&(^> zD`>)K@8*xS6uR`}Q^(mn<wR4JmaIM&tSd8)$Nk#NUO%89U*_MRa- zw4_THPSw!}7*F$XwB{8Zk`&7vqIOulBcxZi^S>!}tSV&TpNuFamoARw`pTqQs>k*Y z9rkc5tm0kVC5IZbluD}Uu&=AEqFfk^ahEn%7uS1y9u~V++sqi`cFW*Cw?5cO1gUin zm62f=RM=wPhwog!lt>5E)cOnUw7Vv`=$bv{r!Uy2bv8K)`uI_0T*9-18eh%J?l7mt z_^_)INdfe>2yUUDcVg%>l*ejJyK4R5WC{(5 z@$0o0C_^j{FLt*HVcc2N5jRZ5`SUUqEE?!gp2mX{8(N|vKL`S;RXy|EcE0i~3&<7- zE39BzB#Q_bvNUw;;vo-qC30_dli&9ep3XDrh+@k=u0Nvn4cb>tm7U7%P>sFB)<6??CS35>mKC-$pH8#t*#URawX9 z(aBg9b`o`tH*K>n2ENhy*Z#Y3%{(ZjqkFzAN+rAzP- zTO69o%tl@_1X^ZOc-D;$Is*DaDmz>nbfjb*R7-YR@q6gE76_#fTov+2j)>HZEXX(i#M><7@GmpgL5FzZs<@nG z{K8;J*1GxS1!#K2HgE3NrM!%nuwaqjPC??#DHOs^rR7ou2;XciCJ2UY$W0Khi5Cbk zw#&KkG_RpK!(@@&g-MZaGovdPhw2E|o{FM|iUczHSJd_LBS)K|Vn}!$tg49x)xwyZ zK}MOlD$`$Q#0ixPqD1MUPMYedwvOB9Wa2h1#PGYsBJ(e#OOw z5u;OBSIFMM+J#I(YP=M@k2$Uz7~-6smx_wWm76P>w70I$%Kg%5sWcdo5+8V2Mw$Za zf_MyshuhX}mBS!&YKFEWGjf4Am<=RQ3OO$|>!e!-#=qNaqVnUuXaZU z`z!j45wpzZ=1$e(TDgaVGyf$! z#gWWhkz8vR9r0-rb(&m9B6kceguRYpsW_nqX2eD5h0l;Bo9s_zYAoMS-AaWXV`EDf zA)RLR_g@5+1{G#Rbde5EH-Kmgk#iw6k-^LGS}801aA*Rv>`>B{vWN_hU{#2R8#llg z)_xdv%`S2fcErfwP&e~Jg0o~!D|ViVb{cA=?pf?{#*j@C@Jr{%DA3jxPKAd>r&pL|q$f&?2Ep^@*Uc*xVFIQ5;yC3=kdkiiFC_<}^R$Z)xOx)z7=Tl=V znVZaIvK)ZWbk9N3X*+BvuV_~p3R+2}aye@NfK0mOiL*Jf%1^4rt>`ksw~S-0q(@pzCSDni z)uF$@r(r)m6=Al`yxvR4c$fvYtjh)aS+Ah$4^m!ig6?O4VlRjzL^{52vf?2dk2Fr; zI4Ky}5(^55^A8yMcvJ5qQF&>E<9!S!l+gy2aDcm@$*t(>aBwBi>g=EiVFv@j4~*}B zN#A>6+MqVpT1d2oTbQ%0`i-p*9kqM zupP&V1E((Rk;}zB+1UkstFRjLzdORacEL{&FzsdTzOmaHEHkLWSSK5GF&^Nv5lm5V zdcleB9C~#2s5m$`{(7HU_q&IaI1jIh7t-DuqWN!=Nm5aLMBNr_9TNz~JPqB`5xUF- z99uP6PRh^ZF9@lGTX}DJd*HR`sv421rqE#*pac?5)VS|jjQ)BH_0|ugTurzAP(Ky}n0){YKF0Ngs-88sN^T}|4FGu+^f-(C%}sIkOD zIDt0##qj;BXrBjpaBCdL9iP>i6a312!=V;P8V1!HX|4cQ4s*Uor(<8@2Cj4nkBg(}MeIGN+-=sr`DXqUtY2%m@zpSCA)D2?z&>z}#x`X8S8uh6Liz5Y*L zMrk@NftIB)ZgxEnYTiGWFjfQ#QkcwBHQwkenl13IAV=K6w0yaRG-t&q<4Zy!@9=Q@|E)0&I#V2mTHtku_IY+0a$mdvB%9qO#Tx%-@iUn*I{m=D9edpBHDu3f$c&Yjd z`zBWUJW`2K_DcJYQHs1*vCF$o*~{r{tpY58@~**eNtN~~_d&pL^MZi?8}U*J2m#qE43HXaulLGQUI4>aUkVql;5lx+?f8L0(WW? z1e_OcK!${~*>u`{ffz^b*RxP-p*`!$A8XE!2)-};5rrL4Z$6a*q;M;%H$!@Ma9wpk zm;y;%n&0(|aoRs25PPw@OPzRzMXI1uo8g1*$SlaESg^il;oR&Cx{K- zxCSD((BD;erPV4@3B!aH>@OwBa;=iSJ8JIL-g4#|Ld#X1DsJ_a*Q(y~UPxcKn$cN# zT(IdoCH)I~;z~E1%bapg0bP4)@5`t1ZVgLn@1B8^?0kEs>EEO7+K)+oYHm?yC{NHU z9ev4gp)`yklY%k_Z$RJ5n3F1ndVS^Fvc2k4b$G^}!*{FL7a{|eIbL)ly5TVkajeLU9D9hzrPg`ag*`tt96%dn$bhLt{WH0*y=|Ww#w=9o&nMnSLY6Xtz7z{jOeGVv!S|-lX zhUl=so;M*MdPc1z=b+WP4IEf#mWF@os@zp_fDQZ9rvC??%&%v<`17nj%xhZBp}HT$ zNiuE${z+5yQyx@pSNR_RMSv3iNm2C^ZXLY(-9hwIa`m)dPBlEm$v++Y!rjRJ7c+PD zncN>tSQ>!CwvZ>79;D@pD`HT8b}h0#Y1SGyb^^Q70=Ot;>A)g0SlkP(e>^%5r0!H8 zv^I1OARP>{h%8|A3Nh?12wBxfbp0#vV$QH*_*Bu;^Lg$JOf)~v$S16Lz@-Bp(v)6R zpt|+}lHqsQ@GP4}3Ulr`wTlP5Wa2r1z@#D4C-fZq5<5M1a5x;j%!JQ2*5rrK7|>OY z4L?~$tj;^(pW3H~Qm7-M3gl8v-g(#v! zFg}p!-|x;4Ry^`8Ma{^nweVydHJlA6)*@8U!ZOTI&U2a;9xwzb=A*Ec@f8hDC2-pe zGsjv1p?E`xdh*DKCIRZb^#f^E28g}HiFzzhTc0GCEv+Q9;UwC-*?J}=*V1XH-22-Z zr`9${Thlnyb>~rHT_QBF}9$p`AM#ZNp)H&|GH}FZzwd+k%w_z z6r&uK_Uc&ds0pkXJ-u+9zc$kQLir2gWT{763`2?!efXz4_h86gLgL!qOez^>yal_Q zu#*edwgiS*4=R^>COxrw1t8YXSy9%VClnRhpF#6oF3*yZQ&c70O|l#mkHb8-wbx zA`u($*XK-FrnlN8$hMyB($!K=tyst#r}vh?*jFXzinS!S2vXjsJ>FPIyyMOxI9*yA zx_&{A+TfZ_*6J=SL(#-7Hc6CalCfdvGU1|%@XR|>EL|zjS}Yjm%bDb2PPckYWabH! z!;U@L?PKdU)UJai>+LqxDEPA++Ul%s!d5oHan##y^juxFy2@LUvE$sBsLG}rvvj(s z`tCMz2LtDz&8fo2+?3SPUW~~UJ3>z&N>^KNo{@a)_*A;p0#cr<|5(zCVoGvqeI|<9 z8fMKF+jC9D+ux7m{;lU|72ZGz_huzl#H*#NERD28r+=!v=m>Prp0kpUaFDub8r(xv zG%9z60f3;ehIhlXZ$(&xXl~Y2pB>8JNp|ottsq~{m;X+d#d|Q3YHXMUQ?Q5sotG?2 zE~Sh?rY%{*P)Y}d@c9p`;iA56e$d&`VI}IZ-`wHQWYBY*5#|}?l;_(t zU6zZO30^EQvG}T>fxi4f9d@ItVvP!n6)X%QJ%5xL;V~eLx3XcA^zdhP6nmh9cOi+v zo791`x$$ZKMWhVv)Uln)kux%|Zet+~fk3HwxP3|s8tX-nqjv$w4QsLR)My8jQDvj| zRlNPGFq#kYzHFX@dmc3ewiTD*2IsdCv=yk@ry#*UTLA)6i8}<;cAqJzg*m&p0vy9% zshH;eyLqwSy$D90c$#vGtTINbAhzbZZwph+cBRO6N^zMgoY%3Jz?7$*f(}x9@vfZ5 z9`M7UW zjw4sYlf>ZS#aHI@XTiGo);M#!uI!z&#NCHlpzriS%8!8C5I^kLCVuZeP7>3Yktto$ zkvjslF|~OPc3sZ4F_=DAT97ZB!eEtol4TUYtm*C<>Tbj!yIw*;y2;!AZpD1^>yema zE&m{bS5kg*YN(!$7uyr_UyQIozh~so+FTSwuZM@bCj&;x)7)V&Yf)@W*>y7M>FQsp ztVQmLrv;VC;=wif1zeZ7;;X!}CWtVLVW8gaQoEGS_JLwnA(|8{VoHjD#6lOfBn5t9 znragS8AL3-3)jzILt=Gja_Ndn3O17!uE7>k5goB+4+}P;XEn?oGu}0c1o>RM8eVbZ z3i^NPp43Ro>YD|fGZmEL3d^-pTJphpl)HYD2XsOnQs5P~d7lKh>xsSiB<563*i$(^ z%mrx`bPh_)_mj9p(tmhV)#b?DEk7<0i-uX;08tB41aubyB=$ zWagPqtu*6*0ZVk4aEqGcj61*Wun4MT<~5EM;}2uiT!e&p89t9_}S5Gl_j>Og1<25XK-L(c^xPyd7#KLc1Tcq8QT zJ7VJLXM1UGh@ZV$d;qwiq*_3Wj_>EibS+JkeqkqO2zt^>aTHO)u~5OWP|koIy-@No zh-xNEY9<2OuoE$JU05igW{Z=-qqBj$e8S2nn!WGt7NEYE^-p}e%TJapeDaucNJIhx z`SSB{E?^(vO83O1M`T?y#Z8cp<1}sC&4JI1k0QAq&cN5_JT8y`I*HB;50yv7up*fc z^nhs3xNoewXGnW`MU2uM(LbSzswvel%GSJnWZl-h`V{(qMGD`rw?=Boy$3L5F1e5Z zzd)+aD5_4OlD_q&zWJ)qES?;{iZE?E1S*8$j29EV+>+Wf%>H)DM%rWLXO%S_{%C7D zg`*52Qac5{KtB@6e`}&o+HJpM6&lCJhhr3cl5!0M$WoIDIh--us+}ANY%{V8FPh{= z`hQe377m_ritw--(78R%lYQJ9Lg}jZ#)MWIsi%Ll<2#k~B6UCRQ6RcHStSXo*rcLL zEUY3LR#kNz0p0KzZrr%EadiWBeaWVw9%oa@1Iu6mRDQkd%>N1|&XYDIh!1P} z?%z=YZn<9!Rwh8foDEmxfwBU4kMHmrcSp6yIEGU*!DL7S$@-w-t&q*{ZmTzM3e5yj zNJbcMx&0sKxg&;srb{!Sfc76}?a;1}GKO#r%m4IbI$m7#f^MEqAU~0F@;;*Eo$;*j zPF$f@?v&0~u{B~?6x9SjRG?k;i_Nb?ntp2mvM5nlRzVEv2N{fTR+SOwj^grT4artW zPhk~%NOh@xB@|YO48`9&&P9^y(*2~C{Qd<^76Zv-!}xLb$D`TmfwB#X7DzdUAd%QN z@R+va4ucd8#4ho>c_JsdwTpmm#tq-@VJkjiumF9 z4YcmM{K{_=Um{6;of;~;xOM(l)DuXcZcHGWfp#A_Iw*J7c>yYa_BhA{1z21PNaPA= z2#=>6Fxj=JWNFBnmQ(D~#;&gs9ikP(G^TRP`k=}3Dg0r+G%DJ`Qm|%HgEQ$8P~@%9 z8MAW0)w@EXPqd122ho_2ESP{pguwxzbBoa0>@4qOmTVCuu)io^5SRs@ET7E|anCQJ zU$A|b2Sb<%rC^UGuU&0UIOXgeh1Y|&c9HUe*(_Zf%{bh~O}^IQM_R}R?Qo*asqs`> zMP$A+>+a3=h?YQA0*!2IbFP^P*bL&GzdEL4&+zIRMs5{$!TIa?W)R2^H()=GY!Ta) zY^TuV9Cl$)%n8-H@Ye^Mi|vCLd)K(}&8++<(pgO7PE!9X59qm2LSYT*X+^-Z-s`&I zTv!JrxHU6r`m>G8_79pJSUhd%rq$j+Lpmw@8{*5LjvT(qD%{4frorsmQ#4S6s zE(*M#6by@u_2Mtjin_X7{GADW570}LYw?&L0zlsCso_?$@Te?u&S`6qb6ZknC63rQ zsZA~xkpoCE&t zHCF9axzNp@af!4zgxDR}891h1?%{XI&@f(@HjWRUZmkjmbW>cBpP+*l*|xW}rm)W^RnvEL_C{>-WM z$%5aTsq~#MmsM3v78rJ9#*H?M6qbf#`w@5dtN5$xQC%rcK$D+pJGh*qcEaBuEd9O}eMaP% zRq6g?c2gZyHYxYS#lx4hmYg-7K7MrS=mRYv-d(nt+{f-rP1T$kb}>@>`{#zTEB2hm zG0FWs7JnLI<3?DOO;e8^Nmtq*8BOko-FJV`sCV$dv(IwytWK?l}LeT*Fb| zuW3GT|3%9#Rn4Vo9t)$Q0bt>A{QZ`$ipr(nY)@EZ;k$SE8Q{g2oD2NqdzFrO)@qDz zW?pC-Vb&_ourEW=<{-nBxc0c~m{};0K+~Wv0>`V;a zW9O5M=Mo^@DO_)L_q-@)v|iKDnJ;?8X%h;q7PIe|A3Olcf?NQ)=7^B3m&^WdSe{j^ zLKH!f{I%h8j`)e4UsTNP!9BGA!2UYHYm6sQhzUGCXCBllyqKu)9iH73Z!$Sn08ZHr zoxb;Y_n_T^Aj>Mn5Ntl^8qwtAZngD?S4K@(PgQ`q|1JYzSZ$);xIVaf2_irghZTdA zUG$*@H0g)|l+aEl$CGZZ9c;vexNB8hfRB*o0afjvIaD~(+{~Z1A<%Sg2Lm=WI19MW zTeyI2Kj=wWtpK+hUIYMSP9cB4^9?tc^i7vkL&7t18w-QW`Pzd#TgrHfzyS#f1|w`7my{*&@+*Ou5nc zOD7OVko(;S3ygC%=!UGa6V2P)PIEa=dzx`OyS<$~wGe<@5)lMPhA9r-gdZ_Ype2`l z<9=q)O^m}00!aj5m>44Yx=5d!8&1M!$pV(t2s0F%@O-cR-jO-nw59OU5W z$%`?X9f7e?pCPwKDbZbort3a+@h`^(0TQ6FPC%;eL5j-E;m3F7TA~sWXRVNCY1y&>&UzKt z--np%>b0xzPU#lPvY%`v+?7 zhL7N??dkJwXT=QEX5_(Y@H#Qj4fp3iZ)OeVsNb}fgb>=RBtk`@>5RJuN~I;PCLv?L zN=4;JtKmKIb}%E%bMYbmJVQoumLH@Bx^m+8!MuD40tv(%n1cvJucI7hmUl-y32&!m7h#+@FRB zOAtG^?}krUr@eXj2<=(YNox}IC!VJDUYnA!;l!uIgN3e(k>+&%#M=jjP)^-h&0TiY zh4a}u<@x+JNAr&t|6*4qDXdLPyQ#0*ENAG^#WwjT3W}G-d){G6X4lK|c=o(bTa#Mi z(cE~i3J(s{ZY|q?=F}=xXT-`!X+!XA?+iFD{+d|@Yx9}Aa^|}f-caqx%sdb%+}k_( zGBCH_07b~t<1n?>$Z~LcG+^Sf^Olkq_U)I|N&`zihSo;`4*ElO88~PZ`K>l9HJ9|^ zjvm?6a2R92vOfT?HanJ!S`4lYRkfE34Bkj2e`CaMCx&L&P_V>g-5unw(CsZ#^1^GD zC1x2BolfB5QB!I^`BEwWvbxL8I==mE8+Cqk{8D@y1f@|n$q$-q9y#3)QC&UuI%IPi z&OVUjx#F>_&OKnmt69iuyg*WQ$p48#^5?x%@L=3~0%QW&{$?xtK;Fc5M{+tNp9XS) zk02gkLq3YO8Wmd{-&ja4wwa&O_=a0{@Ji;~E$hZS7ix#raZQ7!J!iO&<1stPp7(lf zi)SwL{>xl;aL#fcA+NbWc_f2;dXElGA7ZY{t?VcB4NfN=(BFFBdezW~!fmr$%aQ=? z_$H?Gb9xZ$xjn{RR;Snd*D~Fq(cn|2PJ79TiIR~wre957-j9gC#*@ko6m(iAR_r(Y z^yAl{%A1>Ww8n`A%Vz&UUyl(;duE|eTMYxC`S+RP5u`suo4fkh>_-m1z*MKF3XKi2 zn#uS?ju39WTGLaBPdVr^-0k9+C#J=a@lR+n-&k%?1w_a&V;&W`|OY6hiPHbVyPSE9U zL0w_tz^_S3$+x(<)h}q}+x}+BacR)RXSj*VkZa)ChvC?&n#Q=)G(9@%ou5UK-CP%Q zR@~gy+-f_>6o-*`fQrx;3|fQC4SZnxd&o;XY_8z|h1ZTxPoD?dgs74K-vt z@8p8xFjcJ0ocifIRuI$+V6R}jH-7W)3LiV%rV*^_N$Go5o2FPKj@Tk82n%~6IFy`Z zyh}EZZKnY-76d7gV_Ykx;+}2ZJjka$!gW2|juB5$)ATEIBt6jbJ4*N3oo@e`@xpI? z3^y+*NDrg*;3RrY5ceDK(Nghs=R`u8ZQlKQRr&5&Tj9CQ(3{TYbPLECbUTKTIiONk&wZ9nCUq9Y%sBJodlAVZ{<_=LcLCTyH zHwZd|8LfTCy*<~-Qn?`;bb&=y}*X%UwIg^Gg@GGxj<>q}{dhu)ARZ2E3F6g|S8Vc8gY7*{{|e5G#!8 zP_?91`|s^rSqZ+pR2+F@y{{43_jGOc4i5Y}#Yua5pk;Ze{pHd<$xA_Y+G0!HUjW(>c-$0Wc1 zet$3lmN~m-*526c!FZ;~o9=ph94?xSi*+Em)e6d`K)kkrR{qrs@KJX?I!>|?zhNKNaF1pWL;vxkar5_xW-Q@F z^DgoVkkVIZ=}vNb>=b=BLRAFGd_AIApZNa22zw;{O%&VrBESwJ2#CP{A7TGbzp^Uz zQ*|_T^e;XMW>R=NLN+TD3~T7!U8NL3RUtqx$S((g^(iCvKrd;+v<&)r_to3}{Q8A- z&vV{7aZ~oeX-oEkK>1^LN&w1QtV8S7_S0p@S;rIN*XPp;9|+IQZvs`1ZEt}uG(s!J z0INO5#Aa0oCQi{Cj~vw`c(>Aq|_uE;^9J` zQ3V(F@mdG{ zZSca2Y%GrrC%kN^nn_e4bD>p%7k3_f1Fi%c5!&x_cmH)XchuTzQ{`myf1017u5o6!I=RyA#vv9z@g@lIX2Vm!vvg zQ#Vv_a1yXQG_|ct*=4fitk$Bm-Kbw_t=XhFq@9xFjap?IJ!?XFFDzwqod)qg___v664LLCZD1NbCR0aF0oBrJ}M!) zgDK>P*?Gwd@H@CH3^<$5fr>2N2Zv5O@i*+-33Yfx*iE=0T%`wuLF6!36VD(7QqK~@ zBhMnkCC^cLepF2i2hKi32hxb$LONm`F=Nl^#ybwzbnbGRMz5_GDP^{_?ADoIkx><` z!wv>(fr-~yQCE&xhAYG*3Jz&`Z;mFl2H8o!i8*MC8r<8hC7uHfuWrk&%{&<#X1N<>4#L(>Ax9TN2v9#*$Wei7O+l^=}+D3ILWrS&hnDW+$gLI5Erw@|f5v zx%rG_=6Xwh{*{EwH$t7xo@`TXICzp498_-`ZTwS- zm`jE(%{(CKa0bhO|95`Kw94*3W3A2xi)X)h$T%nx??5#+alB1Eq(1h`I<#| zBrlU}UQM`}n9&tiDZGqGXSTB#*{$h;e-=dC!Cq6nmaSxEsQo zG6voe6&eO_(09-_0)~2lj@X-~>5PM-#r0PEd%qY$&&U;*b^W1eu)Ka<8juwfSE1EM z?%7wycm!o{XrSHP7V+-G?Ffm^;(US?C1`jfbVO9czlJ};d7ZK1A-izR$wT|4om5;W z?7G7DZ1*UA;aD>K><*(E?3Jv)Mjv-b_aVql%qIGFnt$QaQ1}xHWFi&8ayJgIk zosMV9xpqiw{255HY%$RsH!9q`-n za(A%t|HAuKzi{pK zaqa)*@Bk^7Xn@kTtOEU7xkb3gN8HDI4|m^t`7eq@VoHjs^?l7WC;Z=`Dmc41nHt(! z+L<#*7@FAIJBWIi8oRpuM{h>f(C&LZ;D5lznlxcNR90O1#nX>YPeF~MKXQ=prxUue?SdwEKDZc8?f5*-VGB*Mby_&#NOUvhnEUbe-W``s1Qpw3Bs zYQ%Myqb*QHcazW5sL$!%BV=_;?!>Q@xU9>U6msDe+D~le{JUr6D@c%=k-9UV=PjDc zIlIUG5RL0Dn7gaLW?*}@i_@F6WdBgb@Qv+eu-0EZsd4p+@FQgKFPPi0x_7_*6!!f3 zBPNi1yfXnQFgN!WM%YEmP;=54;zvC-Q}`mD^%?Q0DdCS_s9tfx_8Kbol|R>Gb#Kr8 z5${K4?N7)2)xr8jWSw1=|5dmWy5_I?{i69x;;skq8ZB3K4;-G|BQvW!fMq{|9mFeLveK$nf}dS=gS*GC ze!v}lY{{#C0P)FjE0CrM-RbdXIK>Kg`I$MIVdWV%8dF$UkJwq7h?gO_6nCyw?-?;e`+GP06Z7vE`vEX;L{Sx(;dcGPFhYnH`v`&o<$19SWw zwiICiGs=2;^VI^FRx>kq^USh!sgf+#QCjE3vhK-h;WP_z{fyIeD>zZum<3?fd$snI z;_ou{c};yg^Br=Em{V+Bw|S-V$BbYY=|dMIGo!Gy<|94v$X3qim_phGHMzzCISpoQ zbm-o<5cE|~JrZ!4yNS0+_65AYf9u2@)7fL`FUBwwBp$1XTDH)Yi8yr#9>Cfq)8lJV zg~w=99cj`Uw?z5kN-VX-(yjnu;&oGV(p0mYQ)A<@pq$f;DHGgGrTmz=G#RZ^Ss0dt zy516R*qkQiz7}!i2uc2B^%GksG4Vb($B3tNSLA#%`E|D{x4_Dswaw)@s5as(^AV{J z5C_ML#K9r~!`O9t5$U3=)TA+w3bN60+Th?9I<}f-s|Lu1febnpyDYB7JxnU>w;I!vkYEXh6P zViN6~$u;D*#q5>~;b;cD-GN&$&$CxBzaRm)Q^LpU?$jpsAsyrnA$aVpVsJ+*&Zjy1 zCBm&*u(t=!LjIjNY42t>y5T@IT1<2l_f74n1IanI9pY0gNVUf*J)MnN`z@{`51Ro* z_&z^!oOkf1xr4x#@hs0NesQSAmQJH=!{L<~gp2d09F%8AAI~#+FyefdLjp?#+H@^4 zXv@az6!g&sLc2U=No;xAUGP{)F3Val(_F_$%Lna8 z_u#ADv%oWx#z=~!KFevWdwQ>yK6>*%?NaHclRSATp$+7A#UuUWMti0bx)V~=zg%R% zdlq*2Ht`O-AY*U%gRk~K0NoqAU`Ej-`}J3a1u}oAHoIn#ejA#O4oHHa9#*%i`R)0g zqh87&dXhncBUWr8KboPWdg&QzGRs~$=V`1qiPL%J=#7U`{y>&^x&|0yg=j*yySkG4)gAagR!A&b=Ec4FVt9T=+p zRS?m%Mw%cpRj9R;^&;T{s&1gdJDbN>q=3De0EZg2<bTbc$23cv3sTyhFk#FWx=|$T@2#&Nl1Y$ztUe@oGo!cSFTNmD0{onR zwYE>*Oj%2}>&V0p=`sQ{#5oK7wj;6fi29JOQxT@8dtQyrYnWX5W8C zHuVwgBjLQGe3tfwy*L&>^q39}q4W8F8?2%*DpQ^JEvj6u@>o;)i;dRlceg{1zMp zVJp5URYpdEu~5%Um>k&T472W> zVOD13mch5Fixt?#tepvHLt~=rj>AC{1k302p&u>C-C)B6YPA)yt-@f^*NGHMCpp(* zT~{fgZOfkjsQR(0CNiyTnW9g;txTO7=)?8Xv^wdy9ZIgci@_HS6+DA`IgNXo;^iBz zWle6Gwo`~YvU6xX%ucDV zhe=txpR9?&u8*A;zCb_^eTSgBLS568lzFaljyfCjQcjl!2BieLBG)oU>vd}4#u_A> zBO)c+N{TEv=A>10q!~6^w={`;_p-LjONKI$^KY2AHrFfI7R4BI0AGfjwqy$dn?y^N z#OM=pVG8GJS! zw|wA{S8=fp$=<%}W5u~+k5L#Y5rMCGF4$Y5ZxNJ&cZcaxw*$7*-uwA~`K~4ekdAU% zx$>4I6W}1MI-Yi`vV2ZZrUWEAor9=_lDKa1$i6z?oY4h2#QPWw+Jm;wY{?5NP63TT zEit6I0n$)nyBW`9MPAeC9Y(F%JFuAf9yuI7L@(*RCFXC z?y+j`0-Ydzp~-KJZCQL%`Gt~x2;$?O`X(X@12r4N@JnI=z@3}jN{|nm%8?J1K=B%{ z0!E+U9~ZE^^f|dhVU|ylTJ_4GjHqe}(i9eH>9H(zkCsOjI?0B=Or70)0n{80s(EpB4ylByOL-NwV% zfrHN^r71PX-w0*F5P)?j2nzTq^3T*QpzT7 zt@V*jQCT%Q*F+b1l*DhSaSYY*BRBf?XBh%+iL1w7ix18!TZG9>=Ug(4fGh4^m1E3a z_QU}6nMK~2MXO*oHdj!`DPewme-gMpj-c`zR=Q;I820Fe&gJs?+H8XA3T{_$O4nYd zM^K1-w0qq3;}f3khWPz#cJCY4y}sZV)JYF~M|c)6z>R_K1{DB8qwZ;kH~9K%ri#s> z^Vg^!Y6D*FgE_Cd{h(aWahpGaFMh?~uxsv%?n;KOya`IFh-L5^A#h0f}LhTbpovU z47cQE3jqRYEq+GR0aiik{2AraIkK(++5`3Z^%RQ{geYt9+%X;QOyqQ8$Rq8yB0wv? zH!?qw#$d#|uR*BWoN6ug?!_aY$VE7&Ju=>f2C0f(#MRP_#!OCWX8NGfHpq^4O4{ds zfANGzMp$&;Z>X5Q5syp7^UXw$2X=vDD!=8}f>8V_Rmjmfa6h>WJx_ZsIVPz{CG-(} z@r9f48FGG{qYP&{J-5UlzaCzd-{j{ACC@Ynw3C`hhEj z`B#?guR)BFC}*7Z{wSSpt(&Z|DN$PQTC4YqWjZL2c3vJXf`A;{Zyzyqf;SsNx3aXY zR2Z^BS-F(k+ft_tws@`ha(l@g#qih!fkf)>zwmN}CAViIn3?)aleux0+qL9QoU9{&mBNfPByX)j|#5&bkcXyTM)~s}O*cPaJpu z7c*!4Z3E$*QRvb6x>dXH0{n!!c7ow9;?nh%$QQ<#pJ8rlI5Xe>$9<%fQZ}%cVNtW9 zi;<$7PtH4sRG>aMxqyimIqEIO!&Q~4qAI9geV(-)_Fs#^UP>3pGb}Tnr>QIrU-Tr! z*V-rR#>&#if4RK(OesE{eB%&%zggnn_jeF`Cvyfv2SZ~EQw9f1a|WaDxd<0aI|hJ| z^8eK@wp7Pa#qy(sC9fRu#x25Tf}8tSuZseFgdtA9$OiriaXCXOeJjdLMVqbTA~me_Yk{fUSn*m^`Cc z_G1kXF%untUnV=qljNr+I@G|!{DG8-_$Z9>+N6+MV3ynSgF%w;s0BqDyWTg%jt!Uy zC!&{wfhiBc#{MNY;ep;Bg$|G1(wjIJ*%&43ArnMq%tMUI%*<`TLG+so#+-CgS|iRu zZI>YBIwXVwn-?rSZ%rJ_>Z79s>mP6`WPCT;0?3^$JEUSAKjM<#C^E^pQqo!!am-sc zoxQ{izd>A_1b@pKJ5%1*m<&H$pKfO@=HNiNE>G#I7tI3xaa0T5;_1MgvAp!T^=H8+ z2FlfrG?>IJx$yhF9+R+?8@g^Ad8iD{(8b(KYkD)=0wF2MW-+nOjc%7&VFUZXvUMp1 z49uP}1j!+W=3$wUnJwy1G8C$5WbLE)WT`X2mW&m8CA7XmmBDZyr0&n#1}R8j*HVLx zMEPTGQYNdgCJF!Wg5{J%y!WS1VncH=6p7X_xy1lHU@$Gu^TF2Iav5_zbP{U37VJn2 zc51Hi*OkCL!yXlIxHOryHLEGS%{z%`#i%CViQK4UskOGl%#Wdr>iAdIK|3;UE%%I2 zG%j0%nht_xe;;J|7CxwQEsic~w%Rp(w(2!*cFJ{75atc4@Q5q5a{mOZUS9=#Mh$U6 zv*;;S{!hu$pGofF3H>26?7o&0Q59)XGgPa7IaZp*M`w>z9tbrw9$?4loe|Ec+e+8$ z+j*?&l+Qm@S%UW>h^>wpXUNTMbF>IC8voE$3+0$*-@w@KBwc3*J>3L`AYu4~#A5ga z-NExMEfERpyTW$}rI%B{PRf$WORX}hSZf@oMWz3_E(@A*0Q4Wjb|a&_DD|lmfA=?V z_WD|kUaFB{OU5DTelxVXc6&x^p~>t>x0dcgy^ro{8<)0WuwWH9&elPa7G`X^A#bb* zh9Eq3`$oToTav=$8ZbOE*=TxqSG$!c9Km<3MEig5W@xjaiiXZNn^nw_&RCeM9;>1h zCjsGQb^Tk7pzJD0%|d=MiIjHPEJTy27W=dL53K;46_MynY}iEE6=Ex8{HR~h7PZ*f zQpc4Cck{-?ry~Nc#{ z9`5Te$^{wnWu}$HBOT8IGA2lb=;9p^m33y?HL(pf{4v;vFEojr70q2gQ5K{I>3++x11+F&i4*{EOYEt{7D$4j2iNq!OJYhLQDBsvS;rFHB zh!*rJP%qf|Y49d%m~(UPMFXTN{IJysJ1q>%0Q?`p)2yb@WT{^OJl_C|T?Wd)iBdz( zSg&TR(yplWUphittD^T&Uq5D;3s1>d8(IGnx*yMqXd_A?o~`abGj}e~O>IfH?#1$P zHR|*OrB_=es5@7Mc4W`8d_m6cid_N4O*m(xgbd#km+V4Yk#6VW1c=DDgmxoDo09VV z?;oUjS^QlgWkng}`M9}gX<6=)nTV)RQ*yZEmSK|01d3G4`M}0Q$3#fKR=^nT#L{@*T*-C{c$!o{}q#U93#R zL4xJFk86crn9rNcFwVaoN-B?OTG+sgmL=NGOOtjTryCYY>OE$s?v_f6@#Z)=Prsl` z^I<2IeM4Ly65oIeNVqHp?)fP8`Yx$+D)hD;=}2P-*u{1mPy!abWNaH*W*yIKt?#Hk*; zU?+g_%n6QM!88U64d6Y%^BXJ~|52)*sOSE3_!mqV7;rR_ z7Xtl3_Y&HB7)&{QVy0l0NHS7mFayvmz(VNd-A8}G4!pG@j1zR)_K`eCK6)O56H*Q- zQON#D8W||!Cia#bP7>PG%Y}En(gX6p!t);tyFE3R^`>uz-PZTO7x8}+9wkW?&Ht3~ zBvotY^=~?zHCTT=E@NF5w|sgr0_2F3?o%j1*+pt{@~UmF-Kn)1ineB192OCooPQEe zRv>$}rGOi`0QEvgP%F4Zv4z-MtM~L)f%jE)HlP11W`O%mLL^tE>;Ye8g$Drv<+DPc zICZ|p?*rqtV^tdi<)NR=NB7l1@+uaqinSF6@}fT=FfgynB7Z;202KR<;p=24>fUXE zW0UhJ!X`6Sa6pkUr$NAMDNIejHM^z~*~;!C$`dWu6*kFd-I7tOo)e$)v8){~T$g2< z7hSE*j?;Y>)R2^Gu*1Fx7JNk-z#!{Qrv_YB+lDp?|5jY5rOo)C=m=)1vj`1Z zYah0i@P3up!m{m3BPAo*r-+9a&EP*>Sl_u+@W|O`^p7vdhuTL4EPf_(IW_F!@#Iqn zfon=Tnqd4#0>=(y&f&k|nKly&Nn5=vB7O0qxj;?5KN9JiXtMHHo0zT$cNz)`^9}@1 zu!w9v5j9x!F{3C0MSY2vN-#^IBM!zpcRCCo&#COvICjBGLKBwMB43j0PV>L;b1QT6 z_pEWLp2Y@QO0Av|Re=wE?T5n*{bgtMxL|E{4l>j(sKimuOiJHexx^dhzBqzdhYACj zj?W<8k9ipy?wIRGn{BIzaW%plJsR79WQcpiI{w8WWdB2fU;5v!TX` zG4sdftVHHuK!hQX(liauu@~o_&N*$5F9Pdn>oV2QGh}+-E~m=rye;+!p_Qm+G8E-n zx6Q$Q6Q`hubd+to@V6J@ZA8G>g2)9~`spIezk)TyReRRdUYx~Yp#xc3r3zQj)+m}q z7ER8O_`@9-%pyOUC#3vy)+Y*btk2%V(~mU|zc=_fE2z4}c`@FNVHF(e==IcT@#$Pu zOp=wvBSf=La`SoL_{5=;5#qP%@CjBZJLL?bGgPxbWJg?;MqKF4a+RK19Wgkmq-XH+ z_gN_YbP4w@LOt4iYfdh3_6hIqUQDgB_P|cJ1BF~&Qhrw;+!70EIc=F++_}M{3n8WT z-Zq@>ecKbd=S&cpJ0)+Mp2zK?vJOk|ES5`UC)aDJ2hEZQtm%<-dAx#uKyc)KTt8RCW!8Ht+0qS zJ2=cHR)C6@5D5qW21*@8MHiTSH6M+N=@&p?+Nm5)o;$sto#s5#bv!*ea=gJ4fOcLH zZYV1Xsfls68rLBUp$+QT+Fhe2nfn)8S`Y zT;9F7+dq#n@xz34f)KzNcn0~mceY3GFX4T0w}~$NE))Wouw0!4OHF2Uj3gfMVvHVgIw3FM z(6U;T$>LAec(=BNgdv%O9MtO)!~ZipnoFL@`u`h}sRHQylu1%xdk;%QJ+L68ffp%tOc+ z?5YwQ`tkp|4mx!2#743A=MPoNSt&IX6=oGgB>Sp?e2Hx52LVK1Ga>Lt0`)8-H2mZ_ z_xptK5qh^9* zIf|?}6tP(SzU<&eDW;!s5Tuk=ETDTnivDOx{VS0FL}f&Aww_s znFpp(=SR%tz*E*0&(4R_V@;3$8(y!gP3cL$2aDctNRYZpXQU*AGHkJ`2x9?plqSqy z^i8Gf^q@pogagIRgCkNtcG@JT%de@DxCJuz;4_`*uNUYUEjloUVRR< zzQYJ7>5;@Vv+C-WK!fpGO0=oAAzGF(I33O*{saan(KKT`fxS!|PX3Q6g~`=M@Lw{; z{9d$NBrS&Ap1Ob0o}157V?P(RGGYv-$tEj^&F ztUQpeq@f6)JForQ2_35CG$Gn!-3%kaGC24GKQ(8ZyH-Eg%&-p>>Yh8lLA1r@8fKGj zM!(=qFb-J{4XK|Mfhg5>(CtryK<9Sa1mSsX z`6b(rmxSQG)XSHDGi|C%394(KK9PfXh!ChT1) z0Wm(pF+f~)A2P%15gwx(qV6q>A>!SCiI^TQRa4I=aBuNx!^%|$XX^?@cVSL#fmfPa z71Y41I-3oDAylN5!`v9`9E)s(mEw@nlRD9=JQS}d=A4w=n!Jj>4lO|RK_!orwrDPM z>@4LQh~g8W=rh!_weoK1D@?PqkKbMJ_@vQ?q%jnGeWTn&Nv35njO(jYe8euZ51Vah z%diibJ>(o^4lYj=0}jU_6%Bfeu4Uaz2!vMm_=!CvtvZ#_NO6N@3}rp~ zaU{@#B>VMW&gLQ%hh~r8m4*lw1cdUxuQZZ&mM)h6v(~8TI-`9z+T`j_i#LiAlYjP` z2Si#*N~(&`hBwa|)izpLHVK6JZC!Io_c!TVRs>Xj135Y@R?T##o=%`tVN~~L#^>jG z&WnwmCES}aRHM>)O;LY75|LY_q0l8n8@ip;*UQzi&2gW*rv*+vji710KZ)bOgEs%qx<_a zl9SsRIV{H8!nkK^5vb7pAZ5jpE3Q$WmJ-M+J>94_WJeEXpr5?EHCP{~%+MH3D|*YD ztN5)$Qa?XNL@|eB-w{_nc&Oji3qJ>|8tnQF~%l%z6PAln`>jSMwbsx{B zeGHxQa|7&UnWEn?_FT`)lb)Bd0dJt`q?eI|ANi`ZumcYaoC=QfrWkTL#0m6!fUZVn zg9=ATM(XH_BdN;~AZ$hws>$=w)YJg$eTr)NqhFBeqXZ=b^m_!h`y5<;qs5qd?=6dK zLO|Wo`C)EUscM7q(U=x?io3gjeTd8#-ZlJLc~L<>?TaIvedF{x*FnL7>=XC7Aa)V@ zbY};p^aXi#TQ>$nVQW&0ea6lCyS6Y|JI#?4l5YQ+{P(7?0#}{g+hx=w&Q+nqB5ie_drRoA`OG|&c1+E4Maf0`d9a#tAy?tVtTqBDpUcb_r4=j-2pwt@1(8PP z@vmHrYPq%%PC4Rprf~A=Nu;JP6E%=cYE`ytVmVhAQotLh4EiTzJE@cOuy-6CVz5iA zBqv-h{9TojwWybZT`_px$AUlPo288pP}+e^Kz#>Xqr2o% zg2^}{d_pbruWClX10}$9H~?C~fL>ehs4$=^TXbpQiCIu@AxC3mK&CAYxUc}Fo*`cZ z7~wjuS(F)ZBUkY2k94)+W;}xRoK=LCblid_E^puER+GxeeYlbRKFC{iDObJ-Gg4n{6l4?)%w-V3F@l@I}ZZ*<8lHx3Ls6D!)S;s%{3c$R# z?ubWZ`v}cG|A*=Qu(nQ>?zgV-@V)O&{r^`xzs>Efjg8Fp9slJpob@en{6_)JFj}}; z-X?3*W5$zM|COvDBxQLOgq+{yTb8zWFm^oYjK4Nz#ZuJM3%C72`a-On2E_X|hkN*v zS*T|)5`wtEcv|)M8X&4u zr$|@sA|X}0DB-S14c6HhKx6)5f!Rb7t90!~t%8IuJ_!e|o64>HDbm8Nd&N>tjB-7p z5R@Kv)f<cvq^MQDD*YJt+UnN%}Cf+qpoz|DPazZOg5^$S^k z`^=v>t7M#Z14DrD9$5F4M!nX7>m(XWd4WR&lLC5!DvZm#Vp)4?8p)09D*REq13oCb zW&*rj$_b4lVUW9LIrJyZll8DCk^*ZiQtjKry=+<6JkpU+`%*y$upsEnlkpzsPARwXCjZyo-xqN&}srIEd!=dk3^}CfMoM88ma-)_e6e(AjGQ zM*&KcY2qDlpsbSDd!&W-OaFWg2)`sVSEAosW=s%8LbZsC*H|Bu^OrOA8whdtppY;pw~uHu>Jj7OI%3yBO_26Y^t zpVf_@+lS)mm!ia|M$`&{9rqUZwu5##(l$DqGX@=4@K&SFPldSaNNaL*Mt2Efy8c*y zRpMlAbz|C-`7LJj)JXql9lQk)Q||)779aRuGmi^RT&jb{JOKx@IsD+$Q;g`FS37!$X5Wre1>{|c#rJL9HS0q-x%p! zMUpmD(CTsVi2ODvg>dK&+asa6PDdLX*1`L9C1J5NioRQ>JO&Ygpx=&9nA{k)2Nkc@ zHALH){Y$O+61{w*{sAU+AWF4Y0vHDKya_?)j&MO^pT6l3v(FMquhP|DkVNbAXbnd* zWc=3V8ZD}`9E)65PA9-J-_VZUS2;Mf+0K|G%RZ1Eprj*tZpIc~n@2JtA(mfNMe z!#nxcn>xeLB@&wHKg*;IPi_CJ+xgEi*(zr+p8VVcuYBQ>|wJee-D3<5D>LOUk zNbo4uRce>3R5r?;%CxJNH2##Me7$W?7{#Xz?t^_>e|w!|dQ890ux!3vSGoL9Be=w2 z+LfSH(Qnz*CdO6ucx@Hoc_wRJh63G*jk!SfRP58C?GCb~e2MnIalhmC%L6TVkK`~i$GtcHMff?s=F51-r?Rhpp zckZyy(Yi^2$64S%HNd-GL5wka!aqUtUHe3Ea!54tf`hzmc?1v6Rf1EP^2MSF$6l@n|!+r_5eU5>1wP` zVnkC?q^EsQIPUD*{v)~R^8Th4BIAyy~XNHa78Q4vG~4t5yO+&FNh$dU<|QW}J|n<75ea-Gx^dgMcWHwE|g0*tIh2ge!aW924u#3jDx`*M0fh(r`qe3VEebHs||m6mf#HEFtA;y;ke^cdmbRYRU zRV|8LOjHpRpMf3acaRx~!x+~BY5+|63u(6M9c?FGPtT|c1Am?Jooy!(?hMqV_s)Lt zKJO|;Q8yKx(|ft_S0*3^AzAD-8izo08Zdx394@qpZPa$*Xch}N4|zM75X|zi}JY*G3Ak7+n?C(Y@XS2T!2E9*np(!-Y&l^d{a}h&K;L8ZcOCk{#P_ zrRs0Y|AC1}c8xqkznaz?XAGF5SixdupJ1n09{1O!+ov?-FbOl&w_PD%N93#+tj`vH z!X6I0q^?4mk-d&(uC?n%Um9dMF3G=Cv{)=4UVrK?%;P8_6kYVp%fD2)U&s0opeLz| z+8`QQ&-@reGC_-a5rP$fAg`;;XV%PpRJI@bE5*+MdE}V|o@FYrOLizTJQ<6WsvE}* zf3#S@0j6)MIJ|b8?P?X0`2|fq{yUa+42~4{W;0Qp5qd7jaEfK469sLk0HeWP>`Kz# zT?x3T8 ztsBJ-h8s;QKnb=_h1|wZn+NL+I+%s$fgDl|A|w?uCN&twQ&AewXh^7vPl;Mq)H+~y z2e(_p@#7@;yudCT$VMBr2Gwag=vDPbcn+T|Ybn5vHWH>!@#K*mj715TgtOG1`Rgce zTy5%tvJHd>nw0rzHlIs_F)cZ9Hn^|M1P+4>8BI`Y?C#-p0j6LdM$|POR zpZtYvCbO_nA4-PR=24SmbZcZ*Tyc7LyYETr^Np=pLr0@uIlU5@&=&rjZ=I9opp|fd zQevIb^vED5G2r7d^D<|2C_5+MV-o9GEjvS0$T@OyDm9Q(ZZtJr@Qk??>WOV!t~F$KNAm9*_~pm!-4FL$JOWqnlLLL0SGNW7=Uq1bYYM*kHB}l?qjBVfY6MRO7rEyH0zPgc?IQr2C1%<5t9}X$INZ zQnH~nyIG8d$PC$xI_ZMVDUq%Z(>u#zWGC$f%yANM0j{b7eQ zNdLhY^vR$W!$cev#h-)~|HPfBPPhonl;g%@jguM7;;c_ z;J1ccP+BlGaJ(0RIMWoNPgKus3p7zo0|m$=%WG5y?aMw4);wlckD7&!|tgtan?*eoV1l=|_8x z52kGcqt1YuJ#8&N!`5h!%tr^>qH02hu0j>Eywy-vR=02j(w2-&P%#4GZzm-;PUqO_ z>^P!0`#m9%jK{7)Gh2)TN~*}>{Rol3_0o%zC^N^xg1N)U*azqWn$^%aDt)h2bCfv1 z0doE4^c7d2TUP!oYamhxosSeZLb$$Uns^pXtnlIqR{^|NxXNmj+Gq)Oyu~g&z!5et zzv>(w;H(1c@3Yj;Gh%%$UC$VdMK#1KlPZC#0=0(K z{p4LjoZQ2-w=nj`xqHIf0%z^5F7m^1z{q9mTu@C!DH%13DO3X~5oHr*U(gAb&<~3- z(ISoKt&|D;Xf5{3p?gW~$m54(luO&X2xKFfY5L4D9UYjCJwb8kD;X-396m*zmQsNX zRlmFZ0Pa&0n&V2oNc)S!5)R;3uK87aub=RqCMSg#T~Kdny;}c2cOFxr8@R||6^vuc)G*=aF)Fg_moAhcm=~>bDehVhmbU4 z3JYU}D6<4REPN^w3dH=f`d{j$Y9qg=%_}g*>Iu$l`uZf9@{}t1o6M#|8!#?JFryMd zBFL$ zfuXjjJW|vr7oup9C+ZA(?G#CBFjGoCcqj6eU~@lBLcI(_FT#A&7&~sA47>cXAuiux z^A~!nZ}8t?%=XhqoM2YmQX>R!JHh_Fv*`R>(YpE80CK-4HAMe+4M0HORMFi1|Fr%p zma3S-Xx_nrR%*$b8hn+Da&y7(YUNGJd=`0n0ekuU$Tt~A?N-vx4lEgf4?*uL-P|OH z${nr}-U8mi14hYkb20YeOr!0rv=g@*#tc)RuTL;v%UY+%T@zZK-Vlsq4}?DdXkpJn zeenTm7~G4{0~4&4B(3E-e@+gSy5i!T)6l)@;L2bpY{Au8`cP@?fz|oisy#-()%*jF zzIZ+&<|ayb?K=qtJ-n0F-witCVm>1jiEZ(tZo(Cvmaoz2tbl^PZbc-qW3C;Rs&eJc z6qBX8DS<6=uxLvfP3P@bVH$XHI3syIk|juLbwA_l#R}s#x}*+U?%K)vqji&M2kUtP z&rW9@&yC z)~Gz12M$jvbBb)21uESH^rtz9z$cw#=)Wx2zVBvCk5_P^CzDHhvs#U^=yhz}t~_uK zQqig65)o0QtgGNQ?%~u!wt}@}EK(37nrn3;sS{p!{dhNh5&`Z<3c$?(M?MZF- z@zog88^rPiK(n@vF>dx0{|lU=>b3Kb0Qcw2qyAJ`O~;1R5*qKURUBkzR}*Fj{RM~D z>}Ae3$los?Lqr}i^{SEs_#q2{WCcr|rVH;+HxpxhHRVTAy>i7^`n_oRUbf$aYeHO> z&v;s*YT}D9BKFLr4KuO*Ws>xep_ycM@t#t%wTG}}9{;xb1@H$gD?A+2)$-9Y_qudd zOPjKRO4Imw$jV%rA&-+unY;!fn5H&D=)w%0{CPXVa!xmX?R`S)-3f~Wm6AV_4h?e5BroS^et(7V7sU_R zhSS8!$vPH9;d}1#plTc2xVuNf_M@3TnHaf zneHefCy|6eaSGn%M2^pf=lMGrJexh8aR%V;h>uLc?O= z+Xq6$X#9TBIh`4W_^agMzkY)E%a$=?1!FC+^R%ob0y%)vI)2cJiH=D+Q#{Ub&6beXAPeebE2{WEm=-wKAqzp6!&lBO&YKk{dB zOVNY@6hWx3e{QpI`;esp1WB&66w8a;eBg+A8kJyk$CM7;C#9D;lK4B&C)q*T7KXWS z!d8fb^R(wqnGK`kGZ;d2d~M|d?ywXJ;G7%nok%I9$X%b zA*iq0&cLNNs#1;}ycy?0f+msdIAp(g1Njx`f*nq!rW3=7eJ$C91Lm;T2935%J<}r1 zv)kpM4_j#MOn4SJ&kW~=mku3_8Wccgwcwn?=t#NXx`xYEe6SWTq4wTzJB2r#4pF2o zEvYvunIgf!G_M$8ZQj8Z>6vcllK%M_&sjBkb)|{=9-jg0V*QYX8izFnvVU$(d+(7? zoWv!}?4!L0jh1RyT5=JzC(Y6CED3pX$?E28VC-q9y@$)r1um4UKhmpsX;F$vH#b{r zx{HqSEBheG;<<>S22JJ5p&okT3qB<8T{J6Z3u>+)A+=5zEyc7|yG$p67B=3ws0=

FY)nOS)CZMv}uv z$W?0}G#Zzh~X4Tk2t{ezWW~>QSQ2R+7!e z>UCHX&l+&H?~&B6WhvC6$;jf(1#EqhDum`&OVjSoD=|ceKoCTM@M2haeniOg#pXyB zUaSJ}&&v3=;2Nk~i@ZFRUdzFoCmeU}tx8_>rl+HUa=TSz;yUe=$k4!$ivT`LARj4V(BS|&OJ zY!h8eq_XxsQU<3Ooa`eeJKSO>J=}68-(Gzut2RmOJEXcc@dH##Cc$1~C$&A*_A|cK zBTELC2ZS&Wqw6D{Fx%u--F=`iur#)Qr8zGN?uuZAF)^maq(x}vGE8czB9OSc)O0Cy zj0F?ixUH>rdVs=#g{v0S5TRin0&`jLFU7mk<$BbE2$J?E`eKb1HMN}7S;H_nC0xvi z!C`?|U!yz#CQ#x)>cVXQM>8)Gf+b%B^WRnk z@oDo$<{t%;&Y}i!hdR9wHeA16tp9?i(w;{THfApl)|x<<56CxCoh=VoJf>O*4Yq8l zC1S-8r7pXhV#^5UkBLn(Q<}sNHyXt_^NcAWZL$TR?9E`G3}|qn`%TeNW!Hyjqh{(S z{+LOp>ufWTM{&Lm_bZ4CCv8FyC0t~5RGxv>dl68Ogg&o2PaTx&Y}A!JB`2(O5@|dg z{V^<6DkY&xRg7*e&m1@V!;#K4f2o3*dm2OGq@Z|c2l>iX&sndm5nLap!<|tLnh_<_ zYzUIXic9l^o^#nth)O?+F(S2%@>c;I5sY+}uTYzHRI~jOn;zOZXtD8b`<$`)Xn!5P zrpZL}1^I$tr@sp4PZZ5s#_|fa7>St7HR!sa=2W$D`n|mgZhvPmA=K6UEq7?l+I~V( z4Q)yfp(sQ365~nkXpzv%aht6y>7}K4PEQFa(y6T+6JriU#V^e>MDvrxc=w9bAoHlN zV6nGpkv(igXWyn0dlrQ~Ap@yPr~aJqY_-_bq$%<311kY#z^+nFUc7&=cm_z*hHRLNzg} zL2_Fh+^lwbd}n9DV}0K0=u>wtro3SoYa*0o0UCW@sK|;i`w3ly*xfqg*_EmaLTiiu zNIRZ7!*)$9zFACF7;8CHpm=ihO#A0ZwRDu>#ar{AXOCo8odB6In7kqQVdlJ|oyGYo zo!Lk^T_YC39V~$pa`GbNd%gOY>q%Dd`4vDb6N()xFFt)F~5`%*0YG z79{fs7VB0}G*q4}*!LCFEB{HstxOLm9Q0!4U;y?a95m_QZvz@Jk7F*njv)-`z`^gC zrkxg|4EhzqK*Eiyh(Enbxh!)$0{w&ZIw)3M$ng;c={sMdfXBLHz=J}@N@CEh>>%zE z-ML-+-+Z!g3wlR@&@CC{1wGdLg_j9{JCB8nz>r+9X+YP0bM}DXx>aVHhsm5SEg7Dk zUB9SJiu1ur+g@kAKTnY0TAFmU=uA4fHoD7?s2g!i3~i^Sv&q?k$#9(*VrAcjz#bKo zgmdMZ4Q?qckSOYa zSmtV5sTd_RRor`Rw^*%t2*l}wNN+#qI8-Nos!S$Og4~|9T`ZR>f;?}glt+)D`ogx8 ze+!=_|DwVJ)X$g#b4!uKgmwPPLIlPB+tli=QfhRGT4aX?$2Q@mPyqGi%3PVovN1oA z0c32tfF9wGWr91k=OdL?jS5jlxf}&XTpsgs@Y3R{YyC24opVb+1so+7pbgErzmizIG5HNcPS=2nYUbe4FbbP#!cD_p?F2t@A9!e{h&7X?ihU^evQ~Amv;* zTS$73Ee9c8Kj)Y$U3AD-3m`FTN*vsdoMXJ6C8{_${`dD8$H)nBH=g5`pAltDH5p<* zR`{uWH{(~m;m8Tb7urtJ4wZOZ`%%5iRAd;}Lbw)0r&@vwnJ4Q`!ZuqPF{q+;i2kX_ zRh#|g1`|VThn?X=61!c*Sn0+C)NX-&1JT=Agulbx;E|7H)2*jsGx&-@CJx7yyHJ3_=B-96D)kkQB!)TW>kFS)(% zDAzDriua#BV*7BEk4~T9nda*^F~f*uu3D_{^6m%eNh)mH6n6=byu!MC3R!-`_T}22O8a`RWYR` zpdo%JOd47+8KbzhXe0?}j?*=&&D+rOm>R!>eK2^l`K!6wS2vv@mBJ5dsyrRHVQKA6 zGyNKY`N(9Ka72l~#7zkWzpwuOZ>F4baXfJU*obQb;cR*Y>}LBSvFN(l31xFH8C;)! z8Of(*(q|c)3{K>9!M%YiM}ngomAcl_-B3}`@nPh#!nfh~`5gV65FqG< zO}gRk5$*H{=lJxTdC4`qv^95~a4T(TKITm6u#E7dgd5KbQemE%^;=aygC`b{=E^ zf;(NpV%e(VVbU1o3mueJarRI4a0`=7n%C|m8U-wfP(UG%wrIigEo0P7Bc{XBo-+Rz zj($_nF|p|HO^wCD$Gup$p-UGwSJ^K|4g$bNXLi&kesd>R_)YXfX=Pp)q3qB#OY(-q zH75`PX3d0K;rfX=RfUr5TR;ru&R>95bLD6fCmy()5)fd(VtZwGlvHbs zsPrB~rsy4xQIS*R?2n35vc4G$j{B z@+Ud=Cwbzn$matlKb#>2Vfy*I3FHz+H1guJ!IaaA(l6*Ft-RiZ#$I>IuE7+n4=A8u zKFMEo#CXhA4gJgJse`Ll(sJ|5Jbv~Dhrdq~TlXNxW)#=r$Q6PyY82KkQlwFe&>DNQ z&C<+SVXP0#u!|U0P0%X|NGgNM86FEysS5KJvo$)i3La*Ue%bqN>ucVFo+4}fk)G?% zBiCSBv`ZSVncvSY7T5S))>i5ro(!v96i%F^YNoC{hoTX^Lri(wDxgfeFrHH89<&pewrbBa6G;tG)2f)@Wy&~nF|J6!io5kQKPZ^f|RxUugi zb+UB~hWX+?BG%fq` zA4M2$i7LlAeysz6&1*ax7kFC-cwV8KsTkKPJ}sA|@59Ju|G*}wFSJTQSXKki6R z{9g3kbD~}jSoS~EukC_Zq!ea@oDIDjC(MJuuAt05q#L`%lphhVic+u}m1!E^u>ZMO z+01bJ&JRU3dBtI98lxWgIm4D!=Quzeb3oR2(!kb5&p@8(nHN+nfAeoKqg)fwSe7w)|3xcz`<(__Hv4To^B1Fzg8t9cee@+p|wk(@jq zayp==AR2M{t#rUE4|BGVxTQoi5F^s*z zG7YH%7ifU_@g2>phwnz#_oYPdp3&}wef8tF&LvOS_FAon&Ge=7OxPA??=13GsZ{ncbY{+WgV0u>VpK)q z0+UDfs#LH5EWnEWb-Xx0m$C*FVogjKs z3+NqFsHimY1QkE(UCk_R8sN(?#D*3RlE64hagkV|Wgx*ZLDe`DndiVj`?!?KK`rIA z_7mx^qDT5wu}EqAWuC`!P$G6yWa4f`HpAbUQ_JT1=SM_lJj&~5jN9qXiHDO%RGeI& zFs6n1E?iv$BOs5b?UQ-U-BW72N5Bi(?gbkIb(d|fHIVn49S#;knZ2yT9P$%vm|%WQ z8(O!sgM+xTc!LsgS2LkpYe8duBf~9(hBC2ovWY|osw)T@hXB8xULRjMfPwZB2Sj}E68wQtyJ)#Oa;8EC*F^{q)%yUv-Qab`W6h!Ac z!N>8?6`moVLv0UrDu@~S@ELoj7dRQb7xiwv{0Me(KRvBJwF@_spSPY>#)>}JQ*PRt zeBj31i*LMe%eZDPM`;i34sp}v(muK00uin~COcMi`hmSIrb(rIKL^@sI<^!2x{Dk= z<85Aeq988KiU&yw^i@yHbJm4SfiGINz`TqfX5)@ES`ajsy zzoDpprT;=vN%*+(NF-Wmi~7^&ydI9b(yso4dSb+~ z2sM-zgmr`Yrk+ZqQ8Xzaq3;~_z`y+QHL4rY6WceM(B`H0)L8p+4c(}>>mbHhv=cV0 zufI`LzwjMY!eE1iC-+~tzZ-1P1?Lz!{A#`S(4N+{E5>uvul3Mg7VVU5c<$+0t*##< zgE3uUgiCt_$9%Lt_ZGtIWvVY3}scZVnqm|3-xYe*(E=$5MBY1cU4M z+_kgdTGtvL{Z7xX_*T2^9*2jLlK|zUOjIAiX$30Qu!Pu7!$Y0-3+P$drzUy?zVL}#mApQ~r zE42-n%8Rdb@MFwn`Wv@9;Q}HHe-rt_xLCL?F511&q%JJV;91LIxbGO0ywtonsZrov zfNJdtU6q1Vpb-Rl__`2ZD;kR!AtN;pp;I88(~JKo+8n)sEF(3`03ayrvO^eBohoR$ zO|Iu1^uLbhA8|Yka7||UZTF$`y{kv~e^)RCWn~2U|MmT3sc8P=5dA3wjE*MKS1M;w zrdIU}5xY@Gqm~R$>99OcPPs%!l-l;^pq=9!pkLZeCcO5+3Q1NQ^O zx`z>oiM4Jq=p?girrX;iC+qd#CdUWJ4soaKbVLfwguX;iN0yrwqD+Or+ds zSPUci+edZCX;bbFCz;qyG06W9if8VwAd0+az5yIGC^^a-Jt2gTIOV7qwxTetjT2r3 z3TSe8Kx>trGQSMo@9lbOTO~X!%vPFNhCL%GziIUemJ`X>#4{T7GA#DFZ}+pqYRqcl zGsuP^&AM=4FjH*S3QSPKE`xJW%8C*RmFWg6F9UsV11s}D|Ks#DKr32B$|PEL9&HjN zyMs~g*%mR)!9_Ovar-a_HAKW$38+9?V(;EpAtK8PB(N!_Khu0gan_u%8S%_-tA%L> zORxCHLydJRF`Olo1t$=t{#OGP0{e|t!&CNw>2vX!rprp`h?TbrnsS&%4d=W=2&`2S zH=w6733+}UJ_iY$rFi6NOyII#r*jQi}k`3 zQ1^39pG;bA!*<}0gzAgd4;NK5FQ4_X1}C?vge;a$?}_Vg-#K|Z)d7rUl+6V@+#GXP zycEF~t=++CcUe7GQ%KQbD#nyGh3nG1M)M7pt*}gas&rxW8PwmezoCBh*n1io|1o=! z(0Sr231RZYK75Sw4^ep5ge^aT9{n*0flCBK4-Nik%_)hwW#hD_Qkh&JORJQ0xdM;V`~5N_?oOz0#RDS2KPV^hLcQw4DzLvCXRojS@kZNJ+xN@3 zdK&xeV`}@+S=Pj^RzZ`F1vJ4lTtAKlsaOyhZoX{R1wkFgy{+om8Ig6%tc3rw>_c*^GF?&$<0H{2 zL?Xf?$*Fa#LLvDv`B6c}YC~t(Y5gfjlV2C+>YvhT14D14Ex$*_3waIPJ18$4TSg4? z2U-=H$&WsgXQYd3b#o68Zig^a*m2jggypB6X=YIrVjx+ALg5wSa>^Rjik;N9^Cm=aWHpd_n7 zToJ4SKZYr)FgK%A{W$$3d8`t&4XI|8W+#7g9)5FTktuf;+_fX+zENDDEd9mh)S9gZ zZwwoJ*Q&SlE=ex(-e*36&;K3p{-;)F+o6!kemgo0eOojAFN~_7tipe1RD)FC957W; zKC5*ot5}=usa7m5g8O7}N@@6C#-=s%^69?-rREv?qLba+Q!MPkk$JihZ-i5KB#f;&OmqL?y8f7+ zdy>`t=v(;lyxT!x)BkNTEw?ui25vJbE`A-qUxs1z9pdYj;_p&{)@Kgd?mtJTK*3a~ zjE1Hd*VhV(WJ;74i`npdrVv$_vl!LO7YtdR?y*kcM-l#V)%7ijbaDB-W)T`V~Loo;*xft z1%~MMgaVPzR-W`H{|$0(&dL*M@{}YX(@iSypKnr*b=KQ};`v0jCJxcGFuK(j9V z8yZW;7G>=VV)jBwTz?cH%1qpB8FcoMM(XNUY*x}*bL<-~_mR}er`77CS=Aa}1v^sF zV$tabKCbHbqRJ!{NKwhsoDX&Z z_HpqmrBpG*iZzGfuM|WU9e=Iluen+%algMwvrDmj8Nt}AWqUZ2W-oNtOScG{BbMr< z@%}LuFKE}KcIBIw`OA$79|0)q%Q#z8k(~C1@+hoTt&_6>Wm0!Qpl}v1Y&?0p2^c&< zvXn28iM$ni4z6MY1FfQ*M4VARJ2;$wPdy=k7c+JPuR5;-$ zzcIQaYsg;8gVjBy2Mk@c`c{}H_b`FOc(O<{qB)(pQd$&6^5FmAA1M3n%WzfdYhmK@vz-|Dhz_>7L=Ejx>8WpJw#gYfGZ$b=SBuJIiB99|>(p}< z45q%CL5pU(s)1~{Sef?cR=Eyl*7QNJTgvdMbJ{?v2$2XnesSIAAr;MJ4Qp17GK@UC zR1Q90Al6ztVy3Ftp3Aj?JDx2h`O!j1xCjDI|4- z&}&qKPL3bd8m1@#M};i~H8oxmZ+t{@JwXn!AnUF-<a`R4aPc`?mCmEd{4{Z1l(bw$@bL*K%@n zi7A3*Pb8n{#5REd={m^b%*){BmyQN(2a5i@f$*wZcP>C}4@HG>2D#eZd=R}njIu=! z*M-519e<2?b|^70(^1RN;iH=f6SkP}FOF}8G z{|d@gyEHVonyvyyM~DFMJkegDWYg&{!fUYSTOtQQaX2lWQx8EXoZ6se%nTrd9xxaVeo=aEVh!S#M(*Z^WDc|aavxh_Sl_Tu z)#5h95y5TyCE`ukz#AwsyyLqu#!UYgFyjzNIv?ik4r?d`y<2tp3v^^KbOjYZ>}9AM z&jsx2<%hvG$f4&d@%0eG`fHu&WQv*&R23DHR$VOd8Hf2kL@ zdNH;I_^cNa+(yUMAP^NFF`ao8SfF%8*q_-+jBi|olI9-a)w+~hg)#`DjE zSC1|0*ZbRB&W~(aXNV8V82>D2J zSK>&))2cq=Cer@Wf!2H-7{ahHI)c7^g{ypCXyUd?*IObYqVJ`PsYgua?{iHGa zj9S@?z_f%%GB$2n`=$=Vc?o74jkMoG-N$)Eu<4+wLt5kIO(_8qIE}i^GLl(YsZ-oU zZCt(%8Kv}y4l+EYC!!}6Plcx$mUuN9YozOXt>WFBlcEXNL|u}@#PlH3=#}L5hXk(W9pv4qC-CV3cWE4CiNoMdA;_v zfsayq^?`^-K1Z_tnOa|YG?NN*k<4>4lB`rx1F9KMZU0Z3 zWp`PCNQhiGPAQv7YWBf4a;c;^t-dMu0E_uDP6hCJJdB5$;e?CiorYo9xx=pM4`B5& z<=&;RfD6IW)M*S!nvi*_`K0mGE%d_jeo>=TWptJyL^#sYZC?rB+ zu`*&7a+dBtHWE@bFdn&1j2C0f?gS>|j8nxjNcK4Pq*2VwRU#yWOA)!G$(wK!V8Qo5 zh2>!eL8RbvghSp~5!{vK9LMa!?r_D=fhSH3=ZVeA6%>Iv1LWQL8M)rx;LKH88E)YY zsG9*H{Ph5d_CE?(-m2e2oys$@ zhFWK_cqgN8mZz)@vh!;}K>qXjW=+`pbj{r;{R#O2hn3zFGZ6LU#`Jh-AE=2F<%Wfl zv6ay&NNKYt=sg(Tf<1sxjeF<#wr z%lBUUd35yzGI@qUwTXA8l`pj({0pNo_u1#)E7I^VYnj*gw&VVHRU-f21f72>(tje- z|Ear0DM@~Z+#-M4=qd)w!~Z-qN15e`MZgVBj1>zi6i*TY51ZK7GjSiITaYr1ZN2kp zbpMvX#5FF_Ot(4U?7SXxy8L}uIjQ+^Tw6NOYs6YiS2f8Dv*J`3 zHy67UD`_AS5{${wPG8ZtXqSLrN2HBT#VE9pMjfoyw*i3WCO!)v*|Sl9$>_Y4E37$G z-(1`8R+@>@qbeQ65UfA_)xU$X0WC7;6r^kCT17&}sD1Cbaj0yg9H^eutPWd*B(_3T z$(v^3d;}qO2^FD_@73&`0k*jbT@gKM%5jK>Hiu4jb<1tTvDq}W6ZZ#wPnCaMP1B>F5qRSomKDhcjoz~}?ecZ%V(%(J9+^3_ohV(FZ1 z8BA9~Bo!}HXP)k5&s?E1-38cSwg&uvjJ;!!WZ|~0-CgRcF56a@tt@ugwr$(hW!tuG z+qP}n>bkk_Irn~N?;ElA7x^zEBQi2_&GoK1=9uGYf8ekDgBPc^+Z5ym`drg9v(2Aq ztmtlR_o>LNw3Dj^C7*{WF2+{)i#fT_bc2mPgfM9IOO>`?78?^IX#I?jEsrca{H>BY zt`WS_hCs`*^h4AG3nuOGGU=3&iMmPKOrXj~v)U49nThO4uFB*^U9EW8VtSN1VA5#e z-%kGi817eE8GD&a9?wOLKaJ<-l_LPwX^8x|L#0VzfrFj{4|@}Jy;OJE@gWJyz%Z|q^+S6m9{}9AzLvas> z?5cyGL39JP=LOZTC}n_!vS0?5AHX7fjNw-K{GBk8&6FliU;`OJ@MsPtxnlu9A zv#GT-FlBPtX6A9k^2MAPo2{16=d69_2O6*yxyM%K$czW zan^n2brz>g_rosvcTE~Z&~;nkVGWptT}f_;TV?jLBD6*?$a$kS1U4dfC(L{&p~a3WD~QqTv>Q;;*IG}Nn+Q{BK9+DjWx zV4|J~lnT=39$~Dm$eB!RSw(aB&D>}`*|n2rOOSZLvDaz>mAA(lr20b>>N$B?_Y9xb ziN##*9VevmYM7uFX`Rhm^P$SR4B0b7pI7!-6E87k20VoCLJetS&oDRYliM4GY_iLP z)+CWWV4>11A{1Jfb_c*dXIJM$E;Bakd3m!OA(j?!)`#@VD-AI?Oe{UC<0r`-{05;R z_ncP;xVhpi^FQlfYdOiUxWsf?8~2i}>gnS;Ic=!?+b81>i|7whe03nKBgVxXtlysBBvqX(JpAPfe48r=zz>^3_##3z5(l<#_`;-%rp?B;txW zT7Za3-XA1gf_j&tMDa@GG-5Z$NN0|JcHd<}g|uGM(SVdRFP z8n3z02b7S4;u^y$;cg4xoiYiXxq90Z@dex3CyR>jpW2MV=oa8zn; z^El*|Vndm+5t$(rCRY;2ObjL>{JFg{`fc)7Hhab!M37TQQplWk@u8b520#$+)Ig@I z_AoSASE$YDOH=6ePCKBhcx{{Q7W89gAo|7tnYUL4Jb4nCBjCm^J?{&fu58bQYz~9% zmiL9B)yy|Ol|Ax3@#mMu+(Kd}O(uDR)CazI)jm3zcc@P43yZe^Ph~F`K%3DnR#d<- zSc*78*#30Hl?>ORZZ%|FAQr|h851{zJyjBbtcJ}XL|-M5Nr5+O`r}sK>NYhmTqoIG z0mh7eFkuMOJdqt3qy)9&rjT~S^!-_j=Zmy9Z-lH3OiB%GZXXp|nzmNT&{vHXdX^1o zt}rzy7H!d#w0<&N%&DU(?iJ}YE0^2o9x4fL?e%3=bAZBW)*LY~pJ)Y%`(Kelqsq`sa>)l*mB(;Ybv?&2Ohr0TP?BX1$Wf*C4dR@v(0_j}U)KYF(A({2? zIC%23jq9FI)(vXFF|=$@5Vv@6lkjm}U)uV{L4aG|SlaJBoLV#bp;0)Ez^l=@#DvM! zQVf4Dw+7ZPW#t$=09qEIsU!zEr3ND0y=bN8Qhl#lubSFPm|KA2bRG%s^OjD_As|lx zdYkM~dJYIfX{F&Xn#@h3J|6{~RYkxp(gXGWY;K;b5@=Ja2H~wHKJ6OABk%fPZo5Vr zt=4#2ET`4;KWzh>JVjc*V{=;0S0&=3S5vluXq!brPR~Qz4uY8$8jn2SL^vs<{N(l4 zZ99aii4F`w50i=%B7-;{6)7VZ`Wvfu#grH2N4Cp$i{eGbdi9jym#Q~OoH-MWEiq~z zh3Q(YhEkUa)XdGvWw?#??U|u|QkW#_V^$NA5aZI%q?wAEYKu6eg7#B8a-8{mbBd|w zT|&Q-l_@&d*Ur|CR29@<&-*Eyl>$;bUM|O)#&=CtH5xD7dEVG`S>9$Q-vB(ViCBo% zi$k#*b|F-)W^#l|>!Is8jD*@oIQiyIGCetCF+bX~{8pXt+IB8)##S9}WOBWeO%yzG z@9jdHErLB7l`9vz4sl?%QLx@R7dGA@l{K`Z0RrzR6ZfVMJ$L41n`Marve{`4bcCJT zbpBon>b;E_lr)L?^5BKKJ5`l;w}ypX1Jg_MMe#)k)AMH~;VFNy5dp8Z!+VtWK82dM zGn#DWP3s!ZtuyER!+WLXbn0yFO>LB={>R}y*xexmJLPo6#@ zZE@(*yj!R+GW+iQe}>8EHNQ#jhG}gc*Ye*$F4fk_pIj)lw}(;rm2>#e;2tXTmrKxU zINILrnY6VPAuI|P*Bt4p*hV5vL><-VPSqdsT^fphcbQkjOrw_wKAe%xx9YLOa_}4O z4mR(O>#TN$-4ttNn<;d0==wpWHf>)KWr14-xT+fq&Ep7Yob#=A7QmcI|Bc&5_48%p z%A;IVjE*k5q|wtV^03%~mRC)?%2X+mhE}~2Q$_w~2JhpN;YyLQ7x^lRSa55&GDoq!rmapxM z`K*Z}xJ;mH86wtTwJMuoAeWv`s7*x-8Y?BN%}NEc=PqAiZm@r$aPr{2mP&9{&Ge;y z3O{Bf(+n_8%5p5Y<}ftwyNGSE)bU_i?l`TDt`XMUGumrTm|6!S$*2!FU_8T_sCni_m)m-*yy(#S{xGRy}Dcvus(q6^g!7 z5Zwd**1)(H&QL(>@o3dBp-+vg+{jNo+437y*o-ch+&L;cRU~Y! zaqpF6q^m!F)i}?fz#|nz@2PRAK%CpjGaO(oj5bO-RQc<;&k~#?#=v&yz0DAUhKvb= zJiBjDrn|CP3u{{Qg*C4PSPptIhRoN2!jG|Xk*wbW?ex)O(j6++eH@rW71MiP& zTh-H}cpU_6$UPof2wkZ)Exyl>ptO9}q_?~!-~58Z5FmE~^()0%BRx5ej;!eHhcGrZ zg0tIZ%8CDYAMnc%!*pNsPY-LnaMC_}B-~4(yF*lGncP5tY!mB!{sI-Xg}$HVrT&BV`BA3UB|Wwdc!y}M6OXE<9uDTU_b<$=FHSlA zV(3@*`U-H1AIo78#~=w^G@$8byIUW;M(D;)mH5Oy^Z6e;a{n*@S;#%EYXNhQ<$$6D z!M_tE{+krw^nXbK`WoPJvahmo27Jia(-=|#uKXOj|7FiYGJVx@m)q;b--%|NLd%01 z!83N=?r;Ozf6m|ck$Xs`nFnes3H>X8hMTd?tgwtQE`bm6mcDM|kR&QAh{7W$s~|&x zmL~!Fla+jtEjvgtHm-0^D;}gft{ocT^};a8RL( za&~&#kT_pr9Hs;1>0w!Qxs2D%JBHQ*v|-(u_T8e%%pN$M>ZzJrk@jQnY~AtvXSp#a zH6`-6bx-{fgc?t#1cdzTI4TgbY5}`QX)+^W4(BJhC#)ovuv>%)fnw7M$b@A9Us#gk zNw$F~*`?`igj6Cx;9VVo91Lk4(^_>v6bh@rvE#`0+rK1+Pg~7){Qrog20|P}|Etmd zKNN@m_Ga<@UvC!SkU%16(7QZk1qBTSO}7M{IM)m4huqLAKl67d_3X9LB&yu5aBs$ zo6OAGvIRYpFQ=7*s;P z2>X&8k^M`|>C~}Dd^ZX-BE*~lH{P2;sXTii1m1vw;h#L6v8Pcaidc-55rr+UJu&3^ zxAwn?rMpR0z$CRp^y8!fgv1f}|8UNkCF&EAgp0~86oK>|*8IjwRl6lEB@QT!M;9au zkw)-Hvyl6#fl^d=jYeTuiv&34|2#wgXBA|U=i=rXSTs>Ud)B|5EhtGV{?CDbWP*eO z@^6IUFHkX7>hX6{5F`X!gg7HDZr+JaA0v#zqX9iK0r@pZaDQhp1IV9Nu5X|H$gCS$d6rh~R-8++%*rfU#-rl#l*A71CXp9?< z-qcF9*j5DN=B8#0SzI^?4I@0%6xgo5b67|4sJdk5Kt^XrMK=*Qk=oux?K;P*!lqqi z6A3OYh=3&0r=JqBoPKQ=ovz_sN`Qv6>gC-t3DWFfyj1IOPz1GH*;fZMi$2Z){<>~C z?_i|6LNHQvyfL*FODdk_Xt@R>(SG0o1FC{Em6P3?z(UQK@-^O8+fGL<^BYHsiP>r_ zW0HHDG)K`Zpb=DC^u4K3)n}0uyyoA4;*#%a96kY%dMTH~h#D+60WXtjem2jlQK zc(_ERcb!_6)&)>q;V>rxL0rO&*z>Z^(dmu)+s+yCNvHPaq;i^)8tLGI+d=uOQEZh? ze)?WMRMtPMeTF_1x=g*T1m^NF(J1670Ns%bad7$_Lj2CGT!En>X1s`6zPjR`XtAEt z`&QCE!d=>Firx;&#P~47n(7)5d?rOp(!{`j-41(Z0j2=UZ$Mrt85d#5?gsJ{N(Ot9 z$AI)pCqG>{-~11rCh@3Z?ua;t4@o=y=<0$AGfwwO$P`^BQGhuBF z2e`RD8}#}pz8L17=B$2lVLnXkzTO;{;{#mU23h2#7r^fZrx{M(YH2pNLu2sM@DU~n zTlY1=3FRZ36_GFQRMtNenwRtw+#f_h$r&x*G zd)-J5IV8V*aIa)b)h}w!4po$b6Pcr*YT?JNopco)6Y7{cfGo$#!-wz%=ZXVX3BEni z6`1+kZd~Wq;Yuss>yTC7p8$v>21mt)k~r%v@^afC6U?@;wW^ugfqFySIK#C#uAShG zJML3eC>fNNWU81cRqi^Wt$2>+5`Win`?c>u{hJkuQ#g?83MV24FpV0F=l0WdvgE~o zg;lu^rqZt098hzPm1s3Mx^=@@N{{rwyl|DiOEKrn4&dM^=ujq@>*L8d{_nuzP@@WZ z!D?7DeN+oTR+?*2K$H_Qw&>Xkb1eVbfRrGn{uh%&(38baV^cURG6!WMuT@hl4dod@ z_5IK~*vi<#Tl$8d=TUtf{Z$B!k+bq@!wAWDGe_bXFrwUmXOR2g153#skXr5 zbmgj1lun!Z;ki*CWpywltxz)ALh3^p^OXE`dU)nZ1IPg7pt};x^Jdx@IpCLqscW!x z@>APT6=5e+vM(^XF?(#m(>V+U!c($?H;BlNrN#xbCJiOdQClU>L0CaDA;OKo$c}~V zye0NPJHHVIEena|aZBs@^&cI=U^XGgCwvf}HMM`V)Ym%+#>K`8@V#1u)TIs;$5b`} zVnZVypjrWi^eVPLHWR$yPg&+RtuzbT+$EeQ5nxMcVc;@*xM7 z0o(O*SqY7zpG%=CDz!RIP80q9e%w{UAZuy5;Em3ygq@@C)+Z2&tFRzm3wY9-UUgW% z=Jcrhe7l@S|E5xFt`F7}YNok3&Vsyb*WV)r+6=S7s@EgE7nS59H9=sakM{h)YC{zD zP8M|A(<1=t028WqIz})(lRZ_zIB?Wiri(!-ra~PU$&yRySt*?~8+Uy3MyVoy?Gjn5 ztMvw>JPR9O9#VGT8`4xId+&7h?$;4dDzj+0buiUdlU`?B9ax7ax}2q>tyUJ{a&%Nu zu!02FpZTy=K?&cY1(gn*PCwi*JV5o?mY>UBb)Z=F|3NLKfbA}7wYh2DsB&mFP6j32!~$ph$+59}LXDJc%>;uX{R9S#%`=i+0|nsiq64Vp`94)Ha~-$h9BvTUa(1W@9RZZ=erj z0F9)!zFUh_G!YOH^ zAkaD{&U%a=DjXt-ntH$07adJI&oVbJS*}8rJyZcE)d!Ei?+&;+)kDkO-m>BQsaU-q1FbxRA5h8r_JHjlxGsaOgpnQpZimpvCJ z?o5(x+JaS_$B#IsI#s$CK&nGP~SozvL{^e z#3RM~E-zu$z3)&@IjvBhkF766tI^hnIXuuu@qNAZKJ3i9hd8r}2{ZE4!an&X@Zc1& zAY!uF1}K-~{>f+m9e0|V|0}aYz|JnNHTop|Bjhid{xy0%XE^a_>W2wNbyF-3u4!+E z=`J5Mni&nu)Bq3h-!s>XhS<%aUR+A>6m|quM!OjCCc=zd5!{=Rc@jvQk$ifv?3f=Z z{20H-6&i$Yiz(Woxy&+z!Z4)uVlc8rA{gsh{@k7t4Qo8%0>h#!czyZw?96Y zyYMY1uFlWN#~4(m4z!c5I?h)e3jCEd#CC8D#GK#I;BF_ynK*Kl@8?BH%>CsAbd~RW z$XyJ>=dA=E_qpl0m; z3RKq{H7=-J{7#K>>DcSMmp@-zWbD>rOrs&@>z{cp0+>l;x5#4JH=7q5L{=jgV(P_s zw$|;-L7zzI`P&8{f5BiV%>E-K!pZ$L=$9EKgYYxHM-*NxRIo>%^?A8Kk1V?`W3VM6 z%UNyMUVea-XXv{2uplv>@ok1?`Oy{``5|Srm_(axK9!=SDH%b@S;f9878|zEQ!`*$iB9VH;TJx%$*3eBx*!=MR z$J<+pLry(Jmiq)8JlQ-?Ffo(!7z4CW^6XK$)$j`Vz7RQwYUaQUU*nlAz=)=$D6?Xd zf{@}pe+V8qWfTo4by$yAg;D;+k28mGx0~U^hr?zMXz|V7Rf_Wf-NJg|@Xp;&hVTy5 z!G3|wLJ6sa4))#NbAizAKL-){!pDNXDCw8*}%7ALPSd*LGn`O@i_q!*)am@GV<8QrUz> z!8(18);uolVZMCB|B17P(+u{;X3{fX`Jg}AQoARie;4cNIpL;_zY6nPsRgee6asG; z8SqzkCpxFt3%hWFs7e6bck|q-t0(JGqkDncnU#k@YnIxI?e`ZR_g&9hm~CW-w^#sk^w19LJ`Rf-% zCH@?eU>dCOm~75@3|ksI{iH1kqETPqDyrOgQ3-#dz@n3R@@8%2-r#kF z)i_?BS?I7rU2j*J?Fhs5)EJ!dzQl@nO=;4(*6OdCR-B05krSp`R#@3HVG1DQ!51Wc zn~nH>(MO*i=@f-=nD8ywR^Pa{GvFiMfpgEM4{wf}2RFR%yGcstP9{nTVAG68Nz z;?G~aykbKA*6?$?y^2|pKXZCKGWqd5BCQ&R%;YBu`N!ICUaCSKqauo3UcOo22H%~) zr0=f7BgV=TD^e@|bb->8x9r=u$^vd(2MdPD>)7^#^lg&fC)JvQ4ii+5$=yWkFY9Ef-@LGA7s`vy z)3qqm$ZO~`LFlvo$*Kv-YC$TBk|MA8YVb|vdEja})=HF8LwP$8mH38l{t;f#>F#NV z&^1mSmRz4ArM$WJu(AMwS8D75P-9Io-y@3^HXqpc7|7W^bWQ^LqVj|BoRj3}7x=j`0f#}FW)v=B1 zHtG$TPSIsei|`^_vBAD%0_-pK4flqGR4e)wS(7)Xa*|tPAJ4_A4Ysj?*I+R}@A99p z91~O*WexmN(>Pg9rLX%S@k+ zUqol?bbtSc-~&T}i~J80Xzv0dKV_I1sfpvnURD&xhRXU6EW9(*L1NtH8`b|Q_+*G$ zGXe!4++KunagB=iLaW`QSLayjOts;?L8&Y8#%gSVt*Q$8fcd#2AqZlF^*$|{J8_{H zj#Q@&ztQoHHB@6Yva%`KWdmD$>!)4;{ai#$SxiuJM$dM_yS9VO-$>I8h zshj)n8`Spgy2iNeeDqQ+){zY3vOf#Oe=~49jJs9A;G~1SK4fsJaW)99#z#p7>zO5c zMe704-b^ROF=0VM5cg%8Y)#JrhZIP`^Ds0VYBjCL?Sj7+G4!mm(dWCYXjqHd3Lp<~ zwi^Z2%jSls!q^KvVgI4}AOlsO-WeycA{ej8M77+g3&>Ht!mV-u)znJogy} zv#!g*LR_{Rlxb#aY@o+8=on%thV=%-EU0ye)!{s|#zk^bUPp`i0wfX{5pjtEbMU#) z*IUT>BFupw>Z4G*8#yT|b(KB&=EjG3enIgAsS{M27Wq^cjBY~a&aYIP{Y>1$ARjXi zd>6d>k16St0VuJaJ}g-ftKWfPsIW<9a!KcAXt)a!{TT;KrrsV>7I4Pi^ig$=Q{vJV zvoS#5`HHcy3{xZ=xFo%nN`g6h^TM!VBA+%RWI;l&JR?;u=fDSF6>nH(bWqrnJwhjG z&-w^1B~lH@yO$t`w8ACcz%1p*t|n3!f;Lk+xz9yzuzn1NXbc5%C;U?g`Pr6jE?|jp zXWT{QwgrZNMC=*#=tfllkT80r?(l`MA?(6u5Z|t038fK31TwSdg6z`wwxO(bG0b$x zrt+Pp?nx4T47rqq)@KVbpO?6#iG^*Sr$YwxsrkraU=G;dM9&R_^trpdz64QBd%Ac@c`(Ehr*FAPAEaWoB%jKhv3I zzj_js7`arKDHq|NVO2v0F07uznGGc#nFQYhcvgYWw^9baeJNvf2uwNKP|D zTa{jJ)aTq{L=o0Vh6BZjecv!B3`)fVR7pzLk(iA5riUZUJ$Gk+LOEMjZ~+yuHT}iy z(x1(JH8ZF@AY*>D1_N2p=m>pA-GnH~%+z;SLZ#zU3Jct)E3n|v7!juG>z>KjXw4SFt!*@tKnmL=mw>s{lI7Ja z?I?>AlAc?#2342Uln~$a56vZ7{O}*nhnuN+3$W*@MspUUX(>Vxg@`0tfE`V zei&oa2{YFyH*h-UjsZVEAy{x~vl9Vcfv4{1D7~GGHVk(GtpWp-aVu_2h-sQ+>0+`I zu=j_()U+t=k+6lmA_}ZXNgkOulZTKbwcVoQmTahI1^2!~X1$YTjZ`lgb>Z5>;*_j; z?XypsEmc%0x(+@YOj`VSS`OC7RDSiBypPz18J$lJg&N5cV)R;K`=*x#`QWyj1WGv< zkbFqIz@2b^@J#-FqcluI%+;vcYu)V=@*rBQ_*@6vO=feE?=61ihH5itD5#cOgf;A| znzY^_sxaLMK3;zYUI+mgaIuC!Bt7?^J5=$o2_C@n8iitA52yYkk0g^l*rGV{2kZ`3 zp~Uue_NK}(7SQrQody!v8Z!Tfw($R4@Bgn@qosf>hQbXiMiv439YxD& zwk6yegb-P)J4siwCr1smg-rf1eo?Y5Zm58qB^B{QfLSXk~0(j>*Es-$gE6H@v|ZdK6h``+f&(KwjJ-W0(FtZsM&8RKeQoBieamb@OV?D zM;!D*j(^$g3G-D7TB%c>X#Z5IcX|W5XpzdK1uuy* z!{4J!EpUaf`|jN%bxF@8%N3SkR}QONHV?|IRXkPA$wxkpaRGUB=A;ZC15d+oUUG~p3|yK8ju?>(7K)Y#;n=%MyrjCL#S%e z&HHY#F=}^p)FYbL9hV7feCbHIa{z6UKQp+rL;frT;&)*$tSS0fmK3B6FrfCc2Pztd z*I3@6A+pC~Rfzf8ZC_quGLr95S3aGV!A@_^!)x+%Mp8YfO8@>6?tLKVJI4g90#BZk zkMAow4Vb+!m9?^`z)vere?PYMC7GWW0V0uNKJrNJZ5Zlq(DJnew=50e*oyWf0$u2{ z@CkUhmjXGq2TN(G*PIx-P~3t~?(bvOh#mff3Wx)x8&onzhTjw(2(+^T@XRAbP<^y& z+C;u&gf@iGsQgqh%|d_c54ic0P-uz8VCpf*hjo>}79>~ME-otT?aUHt!z?Wr@nWrxdA&TX4sMUqO z{sjcK{5Xqx#FSN=$ zb@!z7dNiy4dawqH;U`P6Wtt@3C#%$=Wr*Sf(0`O$g`+2ADd?42EDc%QWq5*#hD!Aj*rgNE`Ti~j~6M$cpNNI zFVk?+7L_olOo#^Cq%j`1HZcmv7;5Yg{3@7r{z;nGcFy`2ilm%c2$pvrVBoemVtsi9 zHNRG1I<|`(;&5mpS)e*L`-{@5hQpGEX@1_)JnR6%K;~&hh*%>IbJ5)bh7O(5jtixa z-4mNlIEeWp12c`z&Cx9ig+qWq3tf#Ot#}~g$}x$jWXM&_g{0S(WtNmqjiik6*&Ux%PPxgP ze7Wic76yToiVH56ULY<$YrQuV`O!s$fh%e95}Jh@c?x(u=`c=}lfGd|k|T3^?Oc^! z`Xf0iiIMlhXfCM3Yn+98eA7hPQTm~e?A+N2kZcLcUa(If5AgQYsnfJpSOItkCOds# zRYT&u`j~6zN{H6}w5+bIV{onz8b>`zvEA%gcQXs6bxeE3xlbSdFc`jQ$dtyd-eZBS zK7WDTnz<>8YPOXhUbn~S;RNA6f9j*{-g4>)j|cg|3omd(*5cs>v54>35&X>Zp*(zT zD>VGWRW-+IG6HGE)q>B-GbH(ZQloea4B|S%w@H0<@TY@aESB# z?u2mW?eKKwZ$xnMcjbIL&+`VDRr7_xoGX-&KscAHLdDxwUl$fE2{r-NL>gU+=GDA|0 zY2%kN+@b|_1l4OKG7ghyI^>3_@eIwz5jc@i9nG7Qa^$lie^hlo!AA za&&U=u-k)<1}ZT(o=~mKwEIO;nI3KbWQ7&BA?TVxvs!*rseMH*bf#*$t))K36@7uY zRNy2w?epU5#IM9SJjK}F+xH~R&f!2Fn~9$hjUTXsiK|cCMhN0)%E#6r@(D=5{XckWsL`wPog}J1$C&4 zCBTa;;ZqUgy$RuoiSPn|S2A>_06^LI%xlIfKOHdLc50|Q7GBR=uGPA+U!(hShQw=z zWp!?1Q4Vpb(oJS?Kdkq08Jm%p7ChY&(CUc~_@v-}X9nEByxmai`9$&YR$Bu4a8CP; zC6U*JA!^WzPc0SKfND!?v_aBugIA_@Jl!tV9jY!iqqiSoyt$rMx{MjSe{qhyfB*;D z8GF!nbn6_i-Ix9BMf3T7f*n;ku%mbyVYU35d~5xI;MvY-s(|gc2+yf+$C|Yf?uP0Y%kQ z?q#@DQbJ6$O*rHcozjh@R>U4h?5-W)PapdM_SnytH&o^}HNKo|o04!xy(OXfX$TE* zoBz$HW1`zE!R+3Ncebz?cFeB}^`i2S2Tjyhb#WY{%g$m`Vb6{0LL=dhsybG01D&?U zQaay88V4xSWxF*4a#wf-G?l6#-1h0qW}IFU)`nNU&_gU(28Kf|)hXC&ON^np%Q%JA zNVFnM{j}=C594$b+EneKhRoT#{|x*qe_23(K}H2Q-y{Sk9{t-C4dwr7stHuqu*F`) z{2S$5n7EWazz|FPJA*wLX%#NSf}jbC#2Kba0LPX-fG@7GA(k!HIk4dz!?d+cMRfep zm*;_y4s)w-vep+$Pu=w-!C4fViC)uP4+6t=CpQ{*hm&d*t;te2-)H)D+UM{2^H;mi z>(gTWH`?KP0IP0wkcMMG4O`G-i&7k6>*n&G#TKbydh2DoA5Y_#nOJZqEzmCK>>&bi zI7xX$T-dWQv$aSb`6FY+q@##p_E^F!D*z;R%F$W~W`gS6oZisB#aX{R>H25WUuM6c zPYx$al<1#Rz$J3bp5+`SB;T_#W#Bb1r)iv^27)d!4BWj#LD1T#LqbRRgQl+H%e59@@Uf;O!wl3Bya3`Ef{HQfiA@3SroJRpMEz*SDf z9nO0A@e9fOj=FVr`~b%dmXR8mzEcaH*lP1k^Y0(}TW99w9!nB+^9cYV((;RAZr|`E z*}ef+?^w5|3jW&RN8GV+?eCQr=x~wvZOOAg%Fh(8n66UP@)qQLO^0YA=o8h*h8?(> zcu^MOM-e4;I>_p@_A$#(R;vWU_0}og8j$V0=u{g%L`j%2*^8#la|0jdT2q<0+3gc_ zbJJ#Cj6jyBx*IJfCZ1oVi#){0Nh%%)m+6cucr^5h$0&6uBQZ<`+WGc9EwNTYq##&{ zje7iTbYX^BfEEz+XS0^=p6MG0Q23qSXr@r(Sa^&#^mwy3B^?009rTw%U!C3Gpxr|@ zV9L|M)6#wP7Pb68&kc8)T_xj&VKaKJ!Q#TntjL1+l&a6`!C^k|WH4NjWJWQv$kfjj z9fK^|PyhB}SzAP(9t(FWTLdkdb5=51d`6ih{!~^rzQP36u!9ZssYu2+Aqq6eF;0Sv zC`Lo;j3nlxS$|gR{ifUgSzujONleBWxO&B8bQ*FOlyUgTX*BJ~*wId?)V+FSjeuv7 zoP?lyq&j>o300vXc#d;}>P`rHULAf6jXsTu-LjSz4i}3__2`w73U`0b=)#V@R3y|_ zl9*TYE5CZrvJ%FWqMU)s29Y%(MIOJqxJ%L*w5* z7QzbL{X&0vRYJs>q9%m0v@&4%*PuvSPD z$r-Zd`_uf?Y>2I z39lxy9JcPgGw(a5IgYP=ygxyE%-FHk?3$oJT?P6%Ixw02ec222Rr@tCpaSltWN(^+ zEU&i&M7M zS0&3}gZFF3o%$up&?Ql?H)lM&T<||9%_H=WD4G4tkho%*#_WpCqC>UhSWpllN9eS2 z6Vf5*s$K+VH4x5n7MDw_ibqBm|6ux{`Rf>AlpM(^yvCa0>8`(+0KRRXNl{)(ogi6*+dJtsZ2I%s6}27{{3O*+GFa){ec=QiM~YZDO48N{t-Kd}qQ z6~PcocaX@UAg|1khZX!VjFBFeG;VznDX}*<80)hYG;v7WTU<~Wq;%q0oe%ERp~4OCPva*!nJtM#9k56~wEUK>$6&;nq?@m4k@VS2$LehH#m7SJ3f-U1%=vd8;B+13y^s|m3!0+R5ryJKg)E?j^_HNQj=k{+;foMHII#Am#?#~v zuxkxs_+JS87IIfG!cp69=D$=5SGNQ&rmVF)f2ekl`Xj6rVdIGt`Sm}n z|NqF66Nu(jRR&H>8vavE#6NB|{~QzX-*7CNb|W=fzdt*bkfs_j;B*7B{CTKxWJ-9Z zX)o-X(oF(MtH!H?e+hLKet06%g}z-xGiFiH5K$~jIZsWbI&@vHI~YG5SJQ2OtI*r* zL%4#`!-FP6quChbMpi>^Wxhj$p;oJ@vE1tlMTIa>pv1IdAgOPq6ez=30Pi zR&bZ5N%AI1RM@Dbb*%phMxx$lvGrLTNXT9;lhk4Iyk#Hvaw*!pWnORljL3 z#1!Y7*ZI9P&2GT9lg`g6iN7!NulV1ai>0SPY&rM5#xeR9a_+%=b`q9zbi}6UmTqJ! zmwNbzB(ys8fm|x*0$RH?%Er7?%M}dP-4)eAEH4SVh+}51{jAU*tay8f{esbcj-8CQ z;_b&BsYKD8!U>34D#B$E#@A0`RIZi=ND46bU$ZJfrxKJs^bTfFCnWBEX)LKUj}`Y9 zhZ8VG*f{Y6hR7uKnvgAT1l^$3rq?RWRuj3XZCCGk=9e~ua6MxyJ5W!t&h^+GIQB`+Ycr7oIPwT z&zB<0E>aU~g5EU*#z~e2DEILST~-mM=M; zZ*4jBtxWgL9^YY5`&S-lQ2W#EX(Gj8BLVf0VY3~|kcbl-k%Q{e&6i-^vhjc|rCD-f zrTMgoR%~9eZN%uS6co9IH!fOAk_{$?xwE9i3}=IFT@LCgkz_`>3_c;@pqrZ$i?>u8 z`7z_ixwA18B{cCFyFx`w1J8sSVe60@B23wyPAq6ruzBeWDmBwBZyJqoVB#T7deWMT zZp3QnZa6=)sBS3t8a24d}t)RXUPl#j(@sZOYp$2hx#jCya<@{6Y_$ zAih+!|C3D=Sk~uwoh!azxt4GdoiWW4T}NlqJk;PyI=#am2xU@Xm~dR?W=@PIrA&;p zGngD{CM!a`+|?RzXFKM+)9FbE#s8Wk|wR4 zoAvCdIVmF}Nota52^j-3MT<_tbDsz?GDoK4Nj(gWI4|spg*CT^j0gOQq%4!sCa+>Y z*Ewa1AA)a_08$=ZOdGVUSkoRXh{Yz=;ew{1LjOrj)njI2L5$g~NQ;)rv}q-(+{BD4 z)sg5UoKuS*HMs4^MLR?1ycBA(mg-h;|3fyqgL7cf_B+4&YZeF!kI1Nn4_9F2se$r_p|%NFde>2HQ1H8wLb37X&TEajq<$WeaLZ9x1}gDx>`tH`cgBPJ+Ntis=|b9X0Z{wmOF3A!BscrklMfckm^Hzc#+#b-a?g^W;9^# z9yf3GUZT6;P{C8l@Pjf{1oD=(;%(Mbik;zdEf60spdaR_fJwgWP+FHv@5N?DMdrCl zGiq+-o};_y@b)GA=_$Wtl32I+Kn?F};1?KqJtJn|%!X-A8v{N}W~6AXnDBI~IWKP*%0J;76jr`+hJr^wsu_fq(-Ey~071LO9BQwV_mmPkURR1Xt)*x9ge26U6j%zdjc zy%0`))u}i~;<%BK?ee6|GeRHj?z&$9<0;Q6WmFs#KZzpVz+&$A4kVxf zcs!a(lf+qvvT~6r<%@%nYaXObZV@RXv%0|LpmMmQR{D87IFG=UQZ%^TP{~5OD#pb& zahTJWO9Us^(t|c*GJ4EeJ2YgJ- zuHg*YJfxaskEzQl7Ksbpy)blK9*j)4r)=6~M$^%#ClJ%77BZfN-xb~0a~p2_cwVW# zl8ndm6v~rar&+z58Iss)c#9`#DKCTgT&or0i06&YXgwX}VQKL$x9YawXifN=UluEk zk7(&Ua3a~81yESk1Cd+7MF{L(Yq&|wTWH2TvUj9lNvBJ<#dVANcmS(!)CKzZ+>d`e z?oHQmmtsPPPn4vS!|Q2~Q-U(#w(r;IUsJI2@?)Z=1{p3jjuV2ll3A~$Mzl@3UZhM& z&&;9P>lt3Wtr*NdO9~qP#F>}I8RLp}ZtV|Ip9obUYXq zYF61Z=K0lkrS&1M{}by&z1AWCKZ}mT+Nvsx2BRo9X<|<`esE#58x#r>s?Ko7SjGxx#e74LXq>Ts$f`jmu^n zylt3dSt=5kzlmAY!)}Ud;f8}$1(AQOu+n3VTrI`Haf&4c&b)@p6c+_esD)0>` ztV4SW*(D=N7T9ACwQW)<2u*fRn%oL4xic%$#_VDRvBFMvS=-ja2TjEt2E{W0$|vQV zdnTkOWZDx&=S3GF`Qp@=5wmOzt{$lcIX}V9gb}704Bv=e1{TuOvNT0;Yi=7DoaMep z{3j=^H`)nG)tV1MXWd2R8Q9ijRs;qJ^D1>t9*ukL(HJ}suZRQMw`c-5Eq8^yFgfiC znU^b~u|1dlXgqs9)IN`Q$|{t_;%@ZWwT(EhJg25e59{j5jAR{~7N&&Bt9WR7&@wr| zN>)oH0J9uwE5z&SuImM}tQD*o*^C8{-^kF;jkf4QMY-#_H+0xGA{#>{wX!hh^XdlvXM?NHdEJt^%?OT1!(Rt73X3pSmnITuF%f zOes_NRe$PtdM%AVS@yu9>FmUQH@QTE_)6L^Y#ezxZ7s@ZaK}Y%dLr@al*igBPj!x3 zoFAwDMnxKXrwk5XX)M{F*BxqyxH#l~Y6tzMDFgN6srU)MDqNaP=nMX(v8Lzr&2ZTc zKm33WX49zdJq8chOzQQ^(&E*lv)Rk%4Zj;)otK`dSr$&m7in;>kk4FuksgKCvb9=g z?zU+Ud`_mnKPIxr{4H#&I2Y@2hdzo+5OAVBu&j)USE9^>KR0Wid^0$3?yz6K+{jPl zihk4WKsvZQ^K|d-9M~t2f^xqWVxYO}P9OkC1-jg!DE>gAtwLhhqki*$vEgA5@(c?F zNTlzmS9cKgi9|Cg(TBX;T?HPQcW2m(W2f64R2Cyfj}^? zj6fttvnv$E$ZLZf8=sd!Z>4Jaq^)gv+^{AREoF!TC9ic|YYEUV%>vDdfn zassZrV=|t9{4DA9i`?j#giaKyyxIxJes=HzIp&n9e1N=+DPeK{U|X3S*z|*27Dd;&YY|>D=AY?otxY4dC6 z6{TISK`c-8Kzs}Rcoc&6KHjrtzW16Q`22=X0jZ6IS+P*;nPKI;`n<2nSA)?nv=4`) z*59C8Um*Xy)ocF!qp|<{Sd0C8tM~7Sj1KlLu15B*|24JV&I4Nv)t?+O&6b_0(2HqA z+}HqAxVf=bEm_hofijhpnklN@;%B0)o;%m1J;e$| zy!4P4$M}c!h_$<(Bn;vE_*VokU2)FQU{C@!QO`<-VG>cgOdfN9tDGFuSj;L~r>t|>{xpSa7*U!`mLHFr^P|j+rn#=%{6N8we?5(fud35 zlJsu@IlSEqXi{o{U_m7R%OoHw~5bRJDGXN)ZRJNsL%taR784{Y_9P_9q_ zw3QVlGiKZ^QjO>c+>9a?H^7#aZqfN7$yw(}0D9FHtj!!lcpVd^uYrw!$ag*nR}DX+ zN+RO>iH21s)ogA(>P>8p6oh|>*k8Z9yLmL^X{DAY9d@ZTGN_?UKYNUqHL*8DkikKK z!#kDin}Fo+sf2BQl>=eY$5v1dJ{W>rDb22gOg5x>=#9$pa+A47RxK6)FSr+pT_G>X zV^Z#G4qAZA-*tpBz^2@n2G%b%MAT!kk4_2<{_*@-{jRLp3Ul|wLQ`zGIWVo&y^KsZ ze2T;7YqN)@YYun-UQJ4#=9LScd^1KUy)!SQ2sM1gdo`Dm$h(K;7RHD`tU{^#YXD=|$Ad-|)lfZod z&$TDw!cG3EGP29Wl69-hs2zSpbjAA}x zMyEb2{o(}axTJg;kKwO5Q(%U9OsJGg&BFOJ>^-xZB-5(YG3mfB(k`i_j@UVejEsWT z5&y8Vuc*1h;Hl)`*E9D&xQWgX+PoVhs-18>u~u2FMz-OzNc8m~3^tkY=6MM+gz~h5 zfx5(_dek;^mfi9^^@~tREy+cDR>aLh0>PO-jng7%6aHfBmpY}^AdB*gzO#73X^fpR zYv~WS5=z&3pt5zz>tn5ZJSG)6WPP~1G%M(XsGnxg<&&tJyEMX|aAySYCRAhtMVIbi z(RYuI`+(ct;Fn&BGnj3XrmB(l%!~WZ&3fnOkEsYC*}^x#Ds7`WM}J~adL;vZxvz)M zAo$}e1AhE-9T3m5#z+1Zp0B?*8~-MUUQ|k4{2x|68x<7|R6UA%oB=Q}v8*b{73CeI zYV>{yF~P+LOPJxH-%e8W`__QSakwJ8>#k+~nm=ZZx##W5qX-+B(4{k)@1-+O5KjzG z21|7=AVQK6ECY@=>)$18cE{$n3+40k_POgv)_pKSHq+UlQMdX}YS>rC7!v1aZz5_( z)qW-dR9J5(U4Dfzu}5Q3ei7&aepHck_F`mwCB~ED+WkSPoFw_;P|9Hd@M5?#tTd5& z1gHEYnNkEPZO0I5$&rgNcD@p0Iv8dDNeVU`gXXQ+X3>v*2*Em_cNehj-|?ULu_XasmF zJc|_NW=Aq@I2@0NIWJ)^1S`tIf_l<{8cY#fjX<}WL0V)EKyH)kL$cSuD0et2!Ef3) zf{N#BOhHr7N--Yda@GIELZe0JD$OwnlxaG_Y+Pn9KBixg?0jk}883r}*9o`i-pY>v z1Kb-UrQFYj&Wegtp1ETb69)}@TINVpJ1UUUiovX465Y4KN+3&)4XN!dY*Ig_K0gX) zLnyA;q}4V?Tw&0T#X>3}HD1+kw!IL?g=27*6;_(o^?28qZlCQU`be2A_^Cdrb~9cJ z4E9%iXsJduNU4qXd5)`4tV`LP@_8FZbYj!*Z=))4i3b+*Ch#jV>o|@YZ!k8PLxm@b zL#5|8bR;KzV3Rseyfw<>u1@gvAb|T^#&|(qg1!(P-}4|HBqB@ zOLCA`Xx!uPq)$Qs?|6wOM*TZ@qrKaC2ic+|v;^9O9qTX;4?6;uiA01Ww2IZ3lZ=|x$ zod3x<`FOOv=e#7J(@f?Av1wL)q!VSpu?~8T>rtnzJwU7&2S--QJNzE(a0$f9{L4fL zmX(#F17U2niK>UOu1K?i&16I-x)ZC@FONXEa~F!Ftdi}Nc&1*kf~R}=fSZWCBGSaS=Dd@T)U{}mG29%3f&N1k)lDgSZS?H!2 zlRx4VhZ*R81_$60CkW&a=;|Rlk2uzrE=h&#xyWtPif%hNZiOWytl9wr8mBJ-=!4^} ziDhz>qCbA8EVW=^S3o@bu11u}I3d}4R&*%|Kp+S#_z$fhJ8b6hD2AEu)6r!FGWanE z4=ATDz(#rB7I;=T1v5vh1uXrAeuhp^%%{v1Ry*WNY3sbuYg4}2cJoi) z-DZqtTU=v7nE9c-{on|0Q(ssWh@b1f3OjMqFca8vHWShAjs0l|JZpq4z_RQ*(IX3I z8HmtTVPlqMCweu^|A{~J$h#X46xWEt`2z$mB5548(=o3c!x2&gX-MI$Y!qLYLR%S~ z!fUw1lIEAngTD0U9c?9^AXb|R#5rFe@YT_boWXQ=eQ4)n%`*MXR`B#SIrP6=Nv# z_zvz0%#=|uNO(EHN7s)(tR?+X2ID{ZoWB^?W~psS$1a~|f~$_>5_~Ew=?aAUaoM>X z;X-4bC%I(>UPf=n9gZig$ljN%Flw`&fJAh|m4Zk}4=K-i#hK$&iuakt?=S}3nPwV; zXN~D(+T&jk8*^#%F*sN{D`VnR%_@2v(xunBa^0DZZU{Z(H@3{qgG;(YrvC!lZ4c@8 zM=yC{4vVpiVtmn@70Tf$S@}AE@22-Zo8S2LtPF zE*tA^f4LALJB8n!vkYIqmiE_{|JC>Rzo5Piglk1dzsXwd-_0e>|Jwy1YUSc;WpCp8 zU(cChRrC~=z8x>koA%q0ZUV6Kr3r0{R1KNBb$pvJ}V)9(0rWG51s?KY^^^Gg}5 zeeWQ?Dc>H@S!;l3KY+!_44=gfpIyMm#~-{to0`zOM$`kSys*1F+CJx>XkjaL>>D>s ztg9UISlxf0WSL~2Ta^gD0|X}&h8PQ{bOln0xzIY?-?K|BX5ISto?DQhL|jc{CZ50f z<53F??^gIOiBMj3o@FNdv~I?&`w*Rk32iF(?vfI*DW+@s^(-?XXdfk0$gR!yv&|sT zw_>u`z3f_6SZU;zj3MD9mCV&DDxjKXd&%|Ph#El@14AF4o`of3p(B>jg?w|3NNrQ& zA?X`w+>zI2I#xw+PiwN^gz|C{xq&_wd)8L?lmL z-=L8BqCoax_0%OuD)Ch?K(tTDl$V%VT({iXcY4fry`^tWV{=rGqn^{Cd$rco{3~xQ z-@rS|t>nFsUSroQ)^ukPelg&#oj4Yt$wY9V6dKPYhPjX_Az&HuuNUVDYDw9EZ%q&B zyUF-B7Q+Ai9{$M#m8+_!u_%bxvso{Q4JsrkYB@|@s;Z2N1SC{YTaKRq(Qt9dt~Y2b z>$3rRzibe~0Lfo5`$-v}tSD4UhrhX)xxJp*@ng}mNO;f=?NG8RLy$U( zksPh07!XXwP*Xi-sNJuJrnIHPSiHwCVyd*Upk)k6P7wuL*eb2X=vptM!|Sq}CTf?R zP8;=9rvYn^<|XI-F3xy#HO3vROb9tn2VQHqS=45J@2xl2Kjd9%e$hPsHdkk%rQ>iVi)rTCiRN}=#2Dv{EPhgbxE71}n?Aby+vY}PCl|ZZ zQK}yV&c(CBP=62@osAc`(x`&0D##2jzOl#%%?7#&+j)#->SI=v4c0Kh=z~CQfN{+j z8rJU<)l7`RHgAAsh+eSl)o~ENQeVPrg6Hcd=Q@@be#WlwozT-$5`)#ibodiP1) zagBFWOsgQ@qUju}XsLjfC3V<7`>^Wdq#5zDy76!`e|Zl-3LWO_xkc^>!-UaUf&Sl! z`UjVz&A&!Y!8;<@-3o|Pi^S{p8|Oz@uZ|ZeYZBU^b0X{q5cvnpTGymNJ!cfNyjV(U z8XtjuK}Vx;h@7I1Bz#!B;*>ufAPGp2`;bzeh`8a|7YzJjl!oTk0Y*O&15XtdNxLyz zACR589~@+fMDMGg@QSWD|JE60&~7go?$_3tgeXc_;Sm#DY7tF&2CwsaF$g4O@rHQM zwbTbD$EXZJ!kr_CUf505cSnZYI_yJxrOp&R#odaW{tNzTzUR#a{`(7feaqU+|BaIP z4{h7Z$kpsWn(kE9|J1kB^*ZT;3JQvXv|BN*f)GYDg;finq(BhJc#~wylbKD|?c6OI zKWY4g4*pk4EN?TRv0OR;;grXPem959!u&sKBQ@{0$jj{+B1V#i%R7Psfu6TPx#>A# zSX(K0?iS`-+BvJZ2I-SzR8%h#;p5vkT=)cDb?b(5Y=h>RO-+v2>C4t5RpU$vHkdtOX&i5J zb4>Kk;dXH$#OphP-Z&Ko|0(2bx$UPGqDm~Esv3y z2aK<2)-aTv0YVGXu@H_@ou@z{dq@#bNoPoHC9XU)QLG9*b`4Z+4<}SGxJmsBjblqu5!( zY|*btxfK6@Ihp{OME`m4U0jV3e*7T&|G%z(OmUYQxW{<@<{sp)vQa>pZEL%2ud3dzQ=~=SZ*Fm*W81DU7%d~4(badX3e&fn+xL6Ck7M^;_ zf5m)){I&MoSziZA*Sj0z>R#Rb{i?UK{^k1>9{_~eXTW;Z=dgeqg4={`A4oOpOic?{ zo~Xf8JRpAvn&0vQ&b>G;D$SiLm#0$P7S8B z`(R$1L+2|K$bGTaOLyK9Hn{@AQ$2)A$J1~7)N^UrzRp~d>Fe(1yw%Wrp6=<(4o_viu*&9@AGe&4F zaUxrNwppj5e4iAgy7M5i$KmD{pNn)FqJ${Flkvc}ndr=>ELy+;ku5EV+LDnsToJ-f z38=K;BT)ml^X|nIRF}{QJE(4m@w~YTQG2Mx)rve^(%xIr$K#@aCNUe;#b67K7`~GC zHfSur+5|I6frl>pDatm6yHJDb-SloK=k zo8@JIwX&%BSAI4g+`xM=bG0+9WZVomE}NP9afjZ+&S7!|Zueq^(O`f~ zf9<8Jn}wnpwf@RIbj5Mo@3Ug;OTB#O4w5QQwNV?-TsMUceii3qO;f*)DgypVxnUgc zogtm6h#WU<#sJms+C#h#{b4h_fIca_+k-u88W2nnKi4wrA4I8=-Q|Zr;H5hPO8&6j zt={wVE8bJ!5gd@<1&oZ!-iKKg9-{bG9kM)80Ul46lHz!r(xYZe4G@~CYC+0_aIi^J zgK%(4V}oLa#@FhYym=R7s?LSZ@J&Y_!rX9xi-N@sw-5?gEwkoMZ-^3)h#pmmg8IM}zBZ)C4EfITybh zQt;fJ!LFBAYug6fYx&KV%tfsRvL1ue^WDjF3?C0En3Yj5xO&4jS8W?xnANV=KJ>Nf zX^Mh(CEI^0iNI#r@_)oEm{J^U4Y;@y$ANozII44U{Pv9BS0MGyb%Rv|MmAr zk8sH5_>Vd*(vk#Va4wk*)7d&QRyuLyB%p94$94@$Vf&7a;GXVZ*_WWFJ}&|heuYvX zg;E#dr0E4KQCnRDglx=~0epW{@%{w6tp{ku^XY~E?d878vNa>}0lMF-Q{ehT*5Xj; zVr!!x~DV|!&~qcz}mE7BFTLT0P^K~X1=`hjgXsokcC;F#CNRl+!{a7@vNhv7(* z7sWb?IqKs?IX6X7h75U3Sh(creh$qZS03(tpwUPb7GUyds8V)Ly#eLKoZKCSgX+E% zesm!ox|%7(RxI9v)~QO4w(xsGO)~vvx}@G%1{N<1^I|Qe)?zVT;O9hxgAU-&Zm3Ow zo}l!_TbW0qIJs3fYtO+7ei_~!uAFZk2SX7Y!+h?lBXqsU9-O8xq;9uZp2xTw7RI`N ze!oxPWcm3D6|hT2p~B)ek0U4vsKw7mKVFqaw0H;$sfp=B{}^T!24+MFC)fvxp;E$K z#F`OC`Ag&wmmpWB2i-m=UcrlFTiG!U?y1`^m-GW)V-3&+PNY1^Te+u|BI_DaHxF^R zIco(=V}~L>B%qb%kZklizjhy|?&oYm4g4WfP*s8~F8{H6l9q>7o54}uuaFVH3EKA< z@&I|0FY!1q+5KA}!DQA3f=yw@wz+j^%xT_C*c@9j+l6h;<82&3HlYXY56QHX$O=}4 zrL>bU5!yeM-oySbc6L{`idTzl$SOJ#$O}q;AU)VCS>gL zpKX_#js}j}cc?frGb2`(BuuKlV@+#V38Us^EMY40oaGNTrl@xBq#W`^wkgx;&?151 zpt`G#vA|wii}v`ys;O5THHNvPsM#ATjj;(TGKRvbSyPT4NWi{< zGc2SUvYSsDSO@E`*doT;R?uOt>n+3X4H_8ssjGBNvZ=Y??bcgt$DJ0kFLsM20Co4R zw0@rMd35nnTdd!GExwEFQSyTRHg5HTy$6Ds3(&MFZ>*|_U+!9 zz|7FB+wF}`bBK7m->6y~mTUPv7Tr$_Wn;^Uk8s86a$VKX6Hxv{wFu*QmZf~DrL9ZN zu;Uh4YthUvLft_azgS4&^>|1_O>De}UpN4j077hjB4Y>{SQNBoGZz8J%50cmSe*B* z4cM!ZiDo(NRJ$t9z)i&LvYgkB#X@>%3pkAXq98iB&9(VED1Ngs*e1pt0_I;R)#sR{ zx%`*0Zus9#3cb9n_oy58z@uzeX*kHL-f1B{NkCa)P*u<@+tOWV7(?S^1E;nER+|c}q^;5o_4|UyX|HSn=Tnmfc(O5OdCXwgg4g(;U||xY z1rYKmkp#P52`xR&;4?K>a#_=2Yv1xK-ja2vc*o=+#A()pN;sRTvD?7}S@ST&-PG+K zmW%hr09q@4&&<46tT6*%*&XGi<`!m2mIadzk_Llwk}w(47!Rt z2B^=>G2{}5)31=UfEdg{1x%Ch*B6r^lNXT4T7!fgesO0HX)Vryms^gm3{TMCNXp?u zDP@`(kz7$+5TQpjPqVJM?oFnXi#NRME~FR(y3VK$@Kk52in`)AW@NwWkkXJYkoOJ9 zW^j6p&FJcp3{)Lci-sgGN9Ww*nL4s3-JI&8T`@9!F&IqE;~CvT9>YO;0B9rmkqzZa zZc*F_X<9JNF`Z4a+TLU?Lmoy6r_uxp4=EH&WCTcmxGSrd6O zULgKi(&$|0iJ|-+wWz=UI_-%`#s1LK1l8+&dP&j}Cq51}6RG!M-UgO2aZ4%8a#!;$o=H)WjLdlifk-Xx&AGrNV=v zB=%BoRZ*@c zkeW*je+!g2Q8inc`w@q_*Fmw;ZKj#L1WF{6lp~$A#eVqinNnCcvq=4LTZike;c}R4 zv*e?`Q=_UWgy?}@VeJlE)e2^*DmBy_#PL)q_}hXtz;9ZrH@a|#zO!-%zmp2SgW~z; zYe{&|H}?qoDW`6jHJii28yLhT@6@}|2PcMYWW^ZD6OW}$Znj}6<9dh{bBuD#daPgd zt`{5z2ivye$xuZE76e%1bK7+;RU+kJ&TVs9MP^j@*2naI-70N7>^!RcqD|;3U)T5$ zA2wOA$v+)$M(k%nQGF7QNmcuWMJf(|!7HC}m%5?xEu-K^VP6q&6i zSUNV}vdB8d7dGSP1kCW&aQvlu#*!7!FtTOgM3Y{@VYGN)uiJ=?-Gs%6N3~DjKDU2a zS^T{^JMVLzugEBh4d$ND7Tl}N&xGcI)_96s)dX?yXF~r2@{m!0IL#D-V_>y9TAvuN z5Vfg^Qq+WTv~%o{ALr5?j4;thTv+ISK9W8z7q3Lpe(xPYM8tOD9T1InnDdvpC~4}B z^kl{|3i8Caw+yJ3j$ zLgiWmtewGE2*|})3a9Y=JQ;vq;X`-O$%NK|wc(;=I{f1Yk_Q1xg#zYGdCD7XF(97F zccy+1<0HdQDy7~L!$9R7r{+MX=8)&*r8g;s7%7WM0Vh3TD~?KcL}lt)Y^x8AS1v&O zp7TJG(aB^9lT?978&U-Gq;sYVNw+jC9$kGOn@!!z1#k9F-CI+xeJd6@lL6t`XaVP1 zg|=U_6;v`w`t@v25i7UL4Q5R*l~Xw8pd&fr>EJc$MD9R}TYk46*E#CdpnyOw;9t56 zkiJSFtM7^f3-#j%{eP$8{PU7i{Xe@5=J60C0ETtc(ozvPB%W#wn_^0Jned#j)%mbD zGZ(2yawcOY&_(^-ayb9{MsJAmY7_tKUc7UM0h96BsdZ4%w;R|e7z z`7EmB!mo>r?uKDKnVU=D;Yl4ca5gbSbOy#Vs!jA8WIuncJ8pB8pfFRHUqrOlSi21 z+F5k-4Sg;Pj5IDg$M0K|jyxLin)t+&_m^^!PLGid&0$mh&$OfH?s1f?X;!~{Eu)fB z65FYD+r^n=kWPo4Z?2ed@@&?ZAID7_gytcUo34@+A}z}YnxY_J#eFVze%4yJhn|m{ z!k1%mv)BAj>b_m-+5P$w$FK zV;q6d0Bpt9pxs0P2~44}gao`79=bbLkpoWIkZ+CS7tVWvNs*?$X*8#CPw^$WgrHQWrfS1BJ4R29tAP-tY0xJ*NV3Z%;6?bAE zIqw1xr4;rO(8G&5rBmgO+mZ|b2(i!Sm*b7J&3>WPr@k42pMzt100?sGP> zh~|lBmV|?CS#G%@9)bl!_MrWAixL8a7`<+-eAC^*r(;g$UXa#cv_Tn#O#sxghs?3w z(?4$a{jxH@&K*3sbnT9hii1k1_pzwSmIEVBXd?q(GkRt?LZR5gV~23=9sSB z6>eo~0)@~J7GB~fT7O{T-M~p?de+YjMHI~k`LfnO`vfkX%l#}S_ zq$N!ENBGo}6WuK(b(^W_`1mX}1NX)q{?B2OiHxwz`jEugrKsu03 z;7YaG=OVViecKyJ!bUXJ)V`Ox)9SwfeWm@vnx%?w06wl3Pf}s|?CFH4>lGVVO%g$6 zLtRx~jOyuGwPQ7TwwzdX4^BuO9nJ!my8EEZwe;08@oam0aE2DU_D#>u3tE|4`2^Cp zBMVn6HL(TZP+YF|2z(7xRLroqew003?tH;QvT=1(BP)64d7^%!GnDfI*x;4Eqam6X zLjiAjj$0gu8n#O}5_!CuLvMN>W}!_fg1UXlu+C!B9`7qS5q@0-8}jO;U!>fVXjCkL0c8XEK#eV7rVxqRHZ z=LpnAN$yFEZKYx-_2DZPOxGGqgTwxhZ(i}pKs%)l=h7rx3yf@-hPpUtBW;G`b%ewL|&&-F8MUED5eS&PD@E4Nzqa&<8K)Gpt4=pj~YB_LL zPbqg72!w*Zoi2gAY$iJ>#MH-xM|vd2Kp85AspM*TeX_R(W%Bpcau%$tgpdbA!tJ7k zRkMVLCoVX}TWVVp&wyPoK1E@`Afk6khyLk!8EVRSC`$xE2i5@Ws;Y$6C|sTTDiFs# zeCZsC2m{^+#NVWp%jiJ?6HCZnMxG2`D98-Y5B1OFi=nL=jCuJ)d~6G09djd=IOraB zF=)v33BLU485|E5fInHUKb_h#uEwlRuKzeeSk(y_q7v202}fAQYW0L_=i=wz z^Ds`Fd3BGDNqz20gk91$h4_aTq%(ze0m4B|9>3Ac?$&kr#S2~WXEX_i{6Za!jY1{S zr;;!!8v?BjYa=g{y}a9J-nyo;jWYtwP0{bUV-O{Jli?Y~tXA23qxR?||H{>?7bW8I zgZ}Y@AN{|xyKuMo&ket(jjOI2+UHF7&D0V3bg>*yVQ5L3`vaj$Nf|d)Xc>ZJ7Qd}^ zE9F}DxS6vNz7f6^G_<83p<(Gnb-~LXl2}5D@PeY1P@x!V3df6X=f^R@-{QJb`b&?Q zo=ka`t?;#!A#kz$j%6F5< z^E^=wf_;-5{a(7l!R9*|4-o`;u!-GUyx%3ny5CQTeV-5(&c=&8^a>b{SKCa{IDg~hBDU|}^KBm$9Gh(?)lgPmxKt-L)* z1#axv#2AqzF)H7Wb-MNn7%A73Y1-Ox=TAVrsB7_hxBleW>K5s1X6F+OIC;{ha&h5b zcQ9|mQB4D1WFl!C9Mg2Zw)J)nR&R@m|LaAQ*sMG=ZWDIMyJZrW!4xRzdGeDxd(!KA zX5wcz)22aV9M6cOZlBpga9mPx>{_e2H6gLWH5 z^FfFadwz&$V45B}!N~?bj%8FGM8risS%!EbAPx0{;Ga)O%-*#>iSW_Bnsj_0KHmvdYCy(@N! zFwk$P$A);Arc^WMbSIt;LA)$B@Lw{1&GAA}(!@9&*p}5b6LVrMGMq*=X%{DQu)t%B zdSq7PMqlQ9#bHDD0#7Nh&=a&_J=}(~bWxcMacTk6W{2>S0oOKk`_=c}c%8R;CKE1e zfOwihySgXEA!GA8*<`wkxs)e5aXz*Fr&?o(N-(gWIY0fDN)!vw(?a;{u0@q|8WK;}GH7!mI4mmZ4Oc2AZ{I7wSqQGQmnCG{)r%WDz&_ zaRdIPtL+acb-wMQxN;_XG;&b&4}n%`Lf^8B&LJu(Z1~HJA@x9*op5QVFD-9zFSS8d z-r_?@|C&QB++|1EQJ3%NVFgB7#4M&|ZvQ9@*Y1eJkVjPho^w^*QK;^cLkmx>-Y>4n z`)^3vD8pS8u0FD5#_I|U?PiwL;v|e#x6LYq?&ijZ(j7&XWOy&7xc+2{8YlcPQ~CNYof(-xEF^O+{K--l(!#7^TCs=b z#i`ay_t^TYcVTfH_i1tMjcTJj&YCpmSwi0)cuXWqUX_RQ5m*l;!+u$ec&@D16S+{E zQj%?*sDauBmBSQT|D=XZpMgu$`D_ylZeNGT~;>X|7WI@e=w8+<uvwe3rxSCWbN?bO(sb{95Uk+}Uf|IGc?dKVbJd}+yG}O5D*agP6(&=ufd}oXPX+tny=rU{9K);-f z@EFHP&#{xB>xG+WnZ?95Ml+qKb`d|vNOJ+=R;X@uzjFWOvZmyr9j=_cR9PaJ|1{T< zc{p2Y*Q~N@x1N(2mHF`&f0rH+{Y!=)k|O}U;o?yvN@m5E&7-OVTeuB4uDU~e(68T1 ze)8L#dSC@O-s7r3rQg>SQxRn0eW3w@#=TBWsUkY#%$9`!?1CN)7>zKF=D_tay@saxEA8@ zT?|E_1o>`6=!Z2$&f?ExzqW(v;HA(X(Tc2GC^EtO)%=W8u&bvN7NaS6n0LNNHCtk! zsqU@K@m9n79c#Aa_GHwRi{gZKOd9d5u7`i-jXb~;$E8%?HMq(gY0VB!S0Yy=zW1Z^ zSf$o5xF-5d#Q&M}q!Tb#6sWid`cyTTfdLmYaC&}62LqNdxOOZ@^lKEUnqbto*yib| z(5n}u0>NvWd2ur1NQ&0c1eLoz#2em=Pc9tEInRvO9!&wknrlbm*R2IIz9<jE&_Suk0%x|Z%TPX5NLK0tOa(E*|1)_0>v}~q+0zfhVW|mU6tzV%h^#!x^?~;8# zyfUF6we9feJyemHY6FS}rRXiW6+KB5{+85&ZxX-S0lRNhr@prnOkIfR!y^1jTdZCS zlUOF<(sPOnO5Sl^Hx!OenW62Oo!URaWshPbAK6U)CfJ%^hg(l zwq?}LY+<@(kl`_|YeauR_i$GHR$0~Ax>B^-Jr|-g?fCV`@3mUpxs8XD+4 zG;|L03f-NwMEow_$Km}zrw~qoyFgmmDbYw-wfv&t{$ev53m!)ZU_s3|Ek6RC;bHfF znD@#9nIr2OFkQ{SRB5%NDKrq>Feo+8R@}ryy8GT;g8lYPmGX&IJ9O&}2q@DjD%v`pefE5R2(AqWl*f>I0r*4*`G_Fr@zh)qXe&y;vPNAihM!ZIx0}Jo1Ao z%}|N;mA&sb@X)LHAyzu|l^^d$OY#fy~ zw+F9H!diflN+g`6?{p5kEiX_x9x*pd;Ao1w!1Ln+2I?SholeCs>+{*xsu@zx)GOQU zsx;vD@~Dne!$}oK0*Qp=Q_u@-Q?UfP!92dEg-88n_^plNsJ{#7R8-M^Q)vk{9IH)0 zYdSYdxMW zmFARl=@qRxdSYnDx3Gr5cShlC5*$TM(FhR?iCAgDeZOPh}5(kG@vzH{OC)_d!_7aGY z(Vcb*oAj<(O`)-{n<-7>XapBj&{_XL3;pbOFdJ2!*cb0QOFieDX<5@sq_R$U9vhEp zro(Y@xTM-yk_Lcbk(`+1Y5ruNR~#yIKNZCYPQg=Gtv3>x;xbblAT9N!t39;Lo*6zN z=II4y%%^;l>B@1L6Nn zn*TFwbp!P-6aRZX_VT@ur}+O(n!-jd|2TS5*HIx6`_8@#l*Kd$3Iq#O8{C0~nr!IO zhk^lyiNX5?ha*N_9F(ayQ#?czt)=D?Qyc;-t(E3VLlH@wEH1)wXO#&Y!Nm*Y<;6=Q zJX}1hsT(X|v60ANSr{GO;EU`iqBA#m<976HW3ey)%D1+p7q-^zYTD3;C?|i-=voW# z>YvL`ja#F|cn?72`W%k}>JdtUJrw}jr-%O}&zY4-2vR@6?b*XqZ!9IDe(5PHm z;cDYavUP+5|osYG9sz*5#HB-RLf1yMacWEH|QCB!*po6bvYrC4|KWqU+c)E$&{r-*yI84 zP$0X3Hft<~MD>A$SH*^Vf{8W>r$&QQbg%pZb<6foK_5(0Et015&2pP`lGL_o1Ted_ zi4|khx&X#0y_vm}EL8W;EkOIZX$?=@orbMGb!5^&K*iW|)EagaI4%I(exH5tNTxiM zd=EG`>uE>Ess6PN1(IS{v91vYCmOtm^jp=#X^~G6kNS3~?=nA+rH^&K1C&tlQST%?VI|rvKVWI+;K#4s1;1~W-&tP=YTU`37)?h;JWNA0AYAygZPhDA^V94|r{P>$8}?4sNO zs(}QTa|ixy=%?fh{P$V8cq0AQZ{WNOIlTQ}h`h)$`1*J3@Yk?|p>D3}aIwnwtk6kR zBeVAwp$LlQN62xE&SB>nkKYm>W(GXy?39$_4gO!zHrFPYAnZcUGcx`Olj*Bwp=wIzYHUK*eXeH^^*pY>RH#aX{0dg@bu2MQ z;^L(B=S7`kLmS3{Q4ROFmvAqf1$jiTh=q3N1W@D3g=!Sodl_{G&eQH;Gf#t1t-^Xb zH@4Qfjmny$hVwVN(iIY{ZGY~FWGKzSvX1AjCwiQ9j8i{yf|>4h=7@s*Bj9!Oc>hM$ zI!pn>OFSCn6XSv5C46(Pbpz#+mK5;7acF;?Kjw&~2IDnKHQYhc@qjg0VEAcgw>DGW zO@y6KF)coG_pAu_b8(NAabo1HQ(C$(m}KLnT-)G6N%q${563aM+JNf-%utQO^jiNo z27F8zIOqVIF_eo*F8Wy$FXc*3Xso-;moy@0=;=451Y7HE*XV`sd^uY~v@bd?0`Ecw zRNP@xvRWcl1Jh-e2-z3d4R=t=TE0_54 zE2rfR7iQ_N*QkKs65l_Mk2T=K_NCWn^&dH6unIHzMM*JEzHo-u3YaqpXcf1+y$n5) ze8B%lr~Yfe6{JU5&jjq$o4`)}KMu0~meGG1WLf{i@eif0l_qEoRJ0;06z3a@sz{F zEI&RosX5QVfC!Kdw&WNG{dk&)7BE<(J_Qxd2rfOsSq!kV9~TNrjFXj4NW!U!wJxGO;FKUAWL1qxa)8l$ov-+&bq%wWkFnfvs3lFN8V;!>DySKh ztn@2Gprfm%Qq#DnipUPKY^mLl*g$k}+SI5;5fE6{j;#^sh|FO>slaTT0PB2(< zONsu-`bKHZTBKpXx8>7o)+UjKN0(h%*hBAN zI4!>WlpTUs-iaY`-;*Ypm1mm*ya6uSBJ47v=LCi-xaSs1e7UwLa40H2yl2|OVeD^d zTZe|jA6GhsbRb%OR~`bg+DXlOJq-SQ*@M93>*!@XMpkzS%wIDTfV1}q*H}8ps|ii8 z|H0xlCjgY&=oXtXS1jJ$H~I* zne3}OqJyPLNzm>p23<2lpDL_61}Hu9W)_r=ItpaVOE{E{H&s{4WhVI;+wEg3C{uy` zwP2^bIy++s4aU<<(e;;dZ|iwLsO!OK?WFJR$lWbt-vMqPHz0*5e2;D~HFrwTJ5)3K ziRwfc^bU_5;Fgekr|gN;{6vBDs|(!LZ)dlTyVgm)xl~?3ocr26SRJdo{o)BZVR|2N zYO0aprNjJwcnxnDC$Pm`jYR0_m?MjwM^fM!-ng6itxtL-Yz&Qv9$gJW5#{5*2zRQ6A1?xc6DjY1XCn2F+vmSeq^$oc_BPo`(-`1>&dq6kx2-4vnUiZo z{4Urqqma+D@J;Gn+B6s2i-;rT(H|9;jdKifRfTr)6mNol*67vb(BMB@7k| z_I)mt8EVXS`~`@;X|fb{PTL4oY*%Bavg>f|7J-qNYwMN3`O$0-qVFgyX^PG?d79ZZvHtJvs5+ttWgG@Opl74-gY_BvWkxL!#>TUiT}%Iu)W9npII`^Ny9y6qv<#$;FPkGBn^ECCth;Vl1wxo1re zbnOuPpsFjX>M^bh(5s-Pd>kz?dtnH{6i7u<#s@RiE3U*0yof;i$H%JMGjV4_I@@DS_EO^{2I-wjg*Ln2i_yNZ3r? zAR&KP3T*w|9Tk;t49ZZT{3GXA$ofM!XQb(;GP+I7RVlSKLH2Oo*1Oz^a5mac1l$6iC(ew!i&8mtgq zplsZA6E|IhlHd>I*d8FTPZeOK==ZJg3Rqb+fH`*s)*{ekb7AR6c8eH28hH;E&eq znAVyf*#+~)G97)CnklCFtuf{u6B9J*HIZ_b)HtHMPH}|ECBe^~lz68bu^%d_2#Tpl5K)K9DWW(*^S!8hb%G3ICz{@WHqhc8p}RZL8CFy#22aiu44LoIGGT ze*p7l{zu&{u%!RFo${)VJBW zgLnEXf;lBQi+3gA+oQ8;p2fbqKy!C(i>UJS#775l3iQ)6qb(sU0wfl!*MnxP|t}b*(1Q% z$l*k$R$Ea58*1`yI;hq}laBQ5qP8h+{!pgC{0R1b1+pZz8xvFNsC@ARAIK+iM5wR- z>GhT*$o7p)fYGTq%{tzAaj~p<-?w~l^F1-EIR~0nIuxJFNKsraaV9w?Ba^77NbFIq z(Giy|sn%rO1))Qg1LmWN76DS7(M$#%5~jH6xy(*I47Id{g?70hWWT-!RAegFD=qBx zhJ*%Arh$hUos9`8oZ0jfjv(eREX1@avSlu#4Obd&Nio6XrMf65BG%-r$#&2S_Zr=A zDgi*NmSNX97wKIK=&Of0t#Sa2KMItCDx9;cR*{s==3tCB|=JvVO*!-!!|?APag4{_57tOIDLp>vHPg7IRDIDn49 z*%}Q;itgl3FaoQXjb80{l|u7dx=pUBiv}WwPCDia7dCWB(|8QmXsja{ig@LYIMAOb zb&@}NJ|=~Q#CWphkCe;0+%_3~GzE;(IWgtag0rSEK)I-R_0zr-o`1|eI5@pTV!NC2 zPFz$cs;$9BYfG*>+##S(slgB!A;2A7JjFe@-9M@_5lWLyS`OFjow=)R5|Nw}g%RLR zA7r&~+Kf)DC|QBNlGaB<-o!?~bNt%Z4u!#c^Mld;wVJO4aal@J3*wgd^Mv6l1dn3h zWnyZkBpN_IIk4{rSzgg!$L;cllET_iy|4Zx&|mHP_LairO$`BY&mDBP-L>U4BB^R$ zI1>r5BYmCnc(LhzsO}>;e02l)CzpGnCYpawo{O)qBltVlM`ZW$6PC(slVLVAi^`DV zu>K7@e9p)>AL<&P{of=tM~W1}5^M9VY4iBdWQ@)7ZBBv)+rOz5zO`|F;WaHnDc@PE zOJXrlM94XL!c%wk##hAus!@mUe#h@@9q5Fz(xiAXrXC`}Ay1-_9 zFf_rb4!x&(SihLp6zelcK0&Kzt^Hv>P_jF9?Td(rFb1y=P{8S0+|LAa+TWejQ#@?d zxDIOzkaeN z9YGSu+<;wQjbFqbnW{7-3^XV0_8?Zt?z*xm_%8SItV9p%qLp(EEA&rw4}T+EB~{H& zgJl#U>5{{Fl=~8Uh^|t$ZMjNwY*+Rq972ev1-u>Q-XFR%9B$5=!VriK=7d1A*JyF* zJBGOu-^-`b?GO^YGYNhvE!HD0o=AHIz1U-8ECnqD^r#g1Va5?*II1kr{x=^~if}v5 z9!&x@>!D<_1^PtqJ@tbvCO-=@zk!Y&c!KK^iZ^6>Kt5#0HdvKij}Xg%$8C^A`--X$ zn5^m(WV(2p{jVVRm%>Ly5)7>wobt*be zubH?o20&?f!j&1Y*RBk-CKpdlif8(6sC;jDqAK5(R)ddfz>Xa0vx~X)zMYCj?vdr` z4H*N1R@lRQ(Iq>A(e%VI?2(=^vIo~&en@ZlsC%5S@G^$0MJj}gO_?PTywM4FO2vF5 zvmqJAa*tt>lw+cLTM4|{RrJh)3new>K{Z#(`p#&lU7c?&uAQ3-LWw8l1@epP<|1mdd^ z`B-K2KxXm~^=Bi%zV?i|%r3VoBCadCx~{C9G)VA_3cN}nb67Z|a#2*ZK=2PDA#c0> zq|>G}>_JFbH&NuM52 zV8!8Je1eluOt)(7`4pn$&?G(W)}hEJdMEY7EG0d~NM~Ic)ZRVwwr^oBbSkNte<-Eh zM$=1$qK!9*y)-d-vyb<%LOAp@HcbF(O~;HXl}_7JYT8jc+i9LY=KSAy=F-6w^Lm*| zSj?flIv}bTLkM^(TjEm{>jyDlU6fo%d^Y+^SRImS-C>WQj^wtes)?TT|Fz4reL-CU z0Y2RSJ^E&8_h0Cn^Sl_^2V2*O+9ByrZK?uRT1;+7qy})Q`1JUl20K)M1UWO0T_V1D zuU=~%__OByAeHIYt!G76_bWd?upJCvH^v|cO!gh_mXn?2 zz-rxt8igMA{9t}0j)>P3A2bFG$Q@FKPb z#hCc1s_>>myNvj>D4_2fKDMP7Ax!;E*Cq79Qe&JgNVR2T_|jV4vND=FTVKiHcop2A zr0qpoKla%R4ct$|)EOV;jPIrUOkky8=ul6_bY4u!A8vED<%g$3%FjA4ww7A`P%fDY zKON0!;^4BW>W|h-BrAmTWg;M+;kR%h{ZF;9JMqy)0|P`8@Q?4cnr6D^YzE`lXmMcD zGw~H8zm8hY%NqGdd>UFXpJMc>rHc4#ExWlShzqwEM)At*CkDO}bPbz>HuD$df)>jk$M!t$!|NUroR>-% zG$D%`s`%E*OB{YXuZdAf;a*F81^|AjLRu3LjpbCKhyP5a9b=F|z~7Vh0c5SNXW*i#ow zFEGXlgwbiY^>=!Za%Y~WjREAiF!mfAFiNt1)VFdGuW!ZlVg5naue#h};>T33>4y8M zi#mtBimOo=V2=$x!X!X`2=M_q${!-UmOI2C zC?Sg}`WApftTH1lxL7(m|3OR_mrR*cs2|+%lU~8Bg4n9!Wt?Sv52=IKC+IuPo>h86 z1pE6B2wg&q1`)f9JLfx}M7`{lXlJm9{7{eIZ`vYxB}LiJsq!B1N29u@2>!sU?U8_4#8L7Xw(8xaM-+fNVGgoQBj^ci2JM-W9@EW;%$Rh`UJo6*%>FH;|PN&;=Vo6i&gY&FQZ3w}$@ z+`xdMU5B8WX|~g1(yl%ztS>woFMz9957ygS!k=$%4kcznn&)EHk> zt%c@96ecVn<*MpKXL+nm)kgHpQkvYSotA17KwcT}dr;q9c0FD#?%L9FYyLI3!jr}7 z`zEf&$Tv%FfbfX-ZEL0t6BkpMEu2lZznNd0E5$ zikwmPzy@ltu3_4UoQll%-kfk^DHs{_c0kduiab#1pbOgklCspEc52_B`9BaTp7pok z!PSr_R=K!Lef7^Zkq+=kY?O(ROY?W5&S)=Ku5;Rc zfP{W!GI!7WwoZ)NUWa2XJQ zO68voJx;UnCU$+9iUap7jc5KINmj0Y;DHvs?J>}pVeYRI!oe(EY@tmyymeP}naN(X z?-E;URb_|G!XI}YFf>}ikLanbyld+#U3FtT=zh*ilr6+_x9EhwFS}3I@+gBZ&~ZTY zqq!y6XU|9HsEiFFwuifmsEM_(ga)V#C7Mh_ARtES$$_#9wK+v1@)y2`dW*ct*Vc%w zoCp>Z60bx4sO$d5H7{^JD0ccVbU1R}_x=77Z)Z+GAM9NjUdPVhNAcIHiC{gVE@l)Q zWhEefOGo|^@isN|&O1q&5$898`g9RV(vNtGqwMp;vceccQ_9{58kR+99=lYxHtV_yiYfNXfg3d27kU zP}@4exPV@>0QGD4i+kF%+#5~>?eYdr`uuw_@n;FGr0}%i_AfOfLU$7Oo%9}AK@QpB z-ouZjyA3GFF|lteuFMebK0c_1T8BsQhf>aykNN-g=7}OV{QB?HxBqqX{BP*nKL!92 zb%vTyg@ui3YZ{7R1)Em%Y5}uC!ck6&fs_w{**4LJ)^!=HHZVZ?W-c^5m3Z|*IecUT zDhnNzk~Y!Wv3%sSpUR1i( zSE*U4gU+9GJIg&=eSBxjuV~-0>E>q~TDnWP%|#N0k4bv8KMQ z*VJ4r4Fr7oc{7-JW!rzlZ=-#fPjfx#L9RAX|BBy!@2IE(@f+M<{C0r?#BaEN@mv1` z)0^hXfcn?2rfc}IuT3=G5w@_~C#E>t+AWdaG2E^dcGmoGWQ9E8bHvG7x^&mhzZ2cb z-#h?L9`qkO&(a9tc&{Cusd*YPCG(zuCeLrEQe$Qbl_cV;y|X#EK>Hx!;n9RJsDh%t zW8QD_tG?S35;^zKaXlQmV|IZWY$kyand1n*3yi7n+s*wqs5qp@ed;PVd$#8h)%Chv zjAayVwQ}$Af^#Mj4EK*42iCom|NlcuTZ{Kfc+A((?36K;BTCwe~+Smk#zhV{t=D$VRytOt~{G z`41{X?nm4lNu29MPGhx#$>1@eh1$}2~RwIk8peS$v4rRP8iDk%%%Ae5KvajAe; z3Rp+{!Cu17CyyuxK$0xP7-L-3EwMNO5U<2FZ#z-$sUkR&nYQd3-ghha7#G{epY#YBH4?& zKs8;l(HY*`|7%H`mVDIa{@e7;((a%1%~~5s-+aZPDoK;MIsVc&M;b6ej#@_&8ztjk zYA_tEiNW`664tfR4NB_qy_Tp)dqb`II*&%pvh%T!QLETQ~?=y0Ww_o{RVAFxu zWEoNf{umfDxA~OURGM?fZqDcP*)|B?K(QcR&*~Q;0PzS>DN$kXQY=DWNWHJg!CT-r zJd?l?BKnl0fNLYY{(yqapVjG(T8yyz1eFkq$~h$sMmEcX%`i(r!+ze>zhbFIz0d4HM9Sz;y7D8 zsOJI@zKP6@m`zq_{g$CH9ik;Mhp|vxAJdGl-lAT|YPRt&s;wX5B*|ns7Xff&<;bRk zfp+C%AP0h~=7P}ojal6pEjW7ra_*)7{-|!vgwetVcw`*r0eN&bCOW#cCOu*eza^;UMGLx9b zY0(;{jQ>I3MjaBJqIA~c4EfAS@0j%ev=A7k_Qw*z-wxhJk6|!QpqfNIhc)oV$TGny67qGvP>+(^%j%%Vf`TP`e3+ zMJgI+0|E6}4*`$Tpvf7dQ1!X!w0)`jF+k%?m&^jsokIogqUG7e1R}Ni9(1<5aIVRm zobXUHqfS-XfSKBW(iAt&t-_&C1^CIyklt)8&c4pbp)l&rlg40<-yd$frRg3LSrb9( zX~UG|ZuGl^j5)t*0o;=eNZ+o1Mf2e;+^~v(BfRqm(axMJm4dpfCa)lID{i=?K9WQy zxN-eHvQQDH$uin=Y!6gj>ea{Z?oowb1|qobxoc~D`~8CG?U7xOH??mr#R%^9wR5g- z2ow;PpP+AtKCfNex8+^G_8$QuKF4gVKCk~C=NdbxoA$e%CwiD{A5f!1Sc zb3aTuy({+VpG1a*LysAH;xAefKGU?d6BBx-Dy>4I*V_O~&MQ(O`Nd-!C(KOP7+;(z z8=i+&yu;9#y$ytsVBvFT{1%k`$JkupL`jRf0!S64M+#hOISmU^aXxg~eo6cq9{hEz zfNg6vB{@9~E7&L$6%WHy_B6Q8isovowN>#@P)hJEMj=bJgBP$`p{?VQ#%f!nw4NE4 z#enG~g;Y&lU5UXObpD;q(C$X7{51!8U(7E0(aQp$VF%3Vo4dxRJHm^{MYMkmr=A`R zkYC{SK8?H|su>UJF*gW}QAYeHdn;Oap6nXEK~mQz*L6T}s}ynxli$_+Jr>v>KZLz9 zJbb!-OixVq0HwIYuAGghG$z|+RnAFx+2vnWc#}Ong!>-X%lc%dmdpCAj&!Ug=m5d` zrsYp?Q`O5o|9dn4p(5`ObK#IQh#u?fvnqoDJxjE%j=QoTqWh)zm(CK=RPi%qF?gq8 z@edHZLk;N{#v2dFv3gmiZyYPaGG@LM~eek14~p|~Bh5WW)-|6p&Tw-A!;8>+6TvIK8nlg69u zvwhs}W2`C>g%}NDahdmBRi}*J-8wrqE+JnRQ1G^sTtILi$SbHivt_Alits9h3YvH0ch?~#@1r#b4&%E((*)6Zy7cPiTqi^*nHPjAR=+95Cd+Lyh$mmx2^-u8_L zXr5+;^+Ad49zHeKlmZ3|-k=`z&0?qP!!?zMCr%nJ#$^H|< z@|Dvk@1>qN$u`o3x6u8rG4Xcs10m84k^CbvEFi2)M=bKOl;>GeOa$Wt?Js=O+XDl_ zH@c*!xH^=W6mRt5N4BJ=M<9Gt0Kzw&_AP4HZOU?H{Zbi^VN9~05vE$SKHAt_-AVrC zusgx~6>l1U;oAe{!{+yvYLw1uny0Qge{Y_-;D5q5h}TUBI|d+p%WpwRRjwcW1M?s7 z?eb7(m~)~v!F|jc>J9T>n|#${vt$DhzL5bnJCy&Ex>m{F)z0Lf<~Ew{>mq0$Lv9rs zYE}tsRI>i5DB&rjFw&ZPv2$i1#d4sHj6Z#B`DNHgeAaKNh%FB z1-#+#aBx1gnzEgu@O2I4+Gu$6j3<>rG(KfU|9BX9Z6_~rLCsz^Vei2PS^FfS^vOjm zBWpQjhJvvoTDQ0NHiz;KyEv1&bCwp>{VJ+QUc(9NH2mP!x_ulj`#Uk|W83+evo1@I zqRMJ+zST`v?{-a6Yez~PIM`LeJMzaCy@&N=%HLy;&mcgGS1z$C&|XCDxplD znk}9bHH*SfgM7q43@u$kKk^ROE4MM>W02zd$$~`&S!^z!>Ch_~Mrs^;DrL9rD&W-o zR542Tp6rBe((Cq5x{~RnjNwym>H_fL+n;p$Fl_J+1>e#3?^&3dY7oFPG+V-y zd%(zLuUnD<_wUmOx$$Qeud{-$ZUO@cS>M2`epTgyP-~+Kef5$@2o7Fm7q|+aM69sSX5XmPxwvz#RQBOVot#4hnX%r zYaiX;?l4e7qI7x1yrga7IF;Y6@@Wn_mqrcpNj8dZCdSApc1fM|QeqEL1^pVD6Xf^% zMhaO9f*d3gi{B;Bsmm2|?7@FTo~TI612nt&<#vm(OCCyO<2<5_n>;X5JrpZZ6oRy! zoYca3TE5_?RGJLPBKWi%(T9`S(#5HsktH=J^RY&oM;a*7tc25TP=I(6d}no z#cTmX2s`7JGlwvpKaEd?Q$bqsJE$e5V-|6u;KAr3T>OXvIdVL?BfR~LlrK$x&w&)v zT*mA;li+7_DDy-E3|Y-|rocLP?|>fP_Fxd5l5PR4ZbzaVzVR&?dq_ydeM7{KQTXI)y(@OO6p{NiL`h;qtLh z|4CQ>SFJb6GN1?X?_zglSKI$9cK?YpHYkS&9;6_lq>O@R1$Dh6V+PouU9?CYuJ z#+J%%+lI|7QzHhTu8~OCHXCxtrJ=GNG)?tlJo$(6qYrvjgGqC_1zjbC;d~PQiS761 z{ESPp3hlT*n~;Js01`WWsUHz=1%DDLcA^EEOh`8$TIQ7^4jVC@#M@|3*ddXj@RSQ2 zy-4Hk^{D;ESrzvsAiHieXhte;TAPkaqid0EOBv2FX->}t+SgfvJ`#$MgVyZl6lV?( zW(FN%BmZzak~;k1Z-HHbjI9bfRZo1exeK}_RzBTw_4gKL_rV?{f}X^WR#=a#TwHlf z7UNcBMZ*!q^R!aUn2jQY+f8?LV+b?Gxa53-=Li11lY?RK`16E8ixJsf(!6R4h5S#n zu)Z2Px@RPY>&=)Edgz`HUF7|VK)5ts?F$ctEYZ|(y+-QhF2Q9C6GxcBDpT*W8I{=A zZ+i%9LfCkzCt4f)gq?^w!AVQXLI9%oL^1bb9wQtTAbKart`n##%{zb~E}vsyV?<$g z93fTsR*<{Ckx^VXIhsuw^_!Fd6XLyI;1DYN;gD$k#SQB`UERC3!yoodN2cdZ3CT>B zviRvh3HEeif^PtqV8|hpDFX{E^OwsD0!P?YX%IG6C)|_#i)qP``Df3{8ATGJeDonk z*;TAZSIl|hqOe_2h&LjeJg;|itZ&G|9rKB0c^%>rSl}1SuM%Wj!+!kF9vX?dTOu+@ zcV>_t=XpPP6R)X>R1am7Kg=DY>jYj|9H~x0eMRDGZ%b-UBP5fbK|Gvapi?vk4oN96 z?Qi*D+qft#{MVk$&CR?hJNWAGi(d#Jk?SX(HAU(uDMqj-$lC(u7uc?bfapEwZTnvv zscn>C6eAG5ulzgW*T3cTU(ow2kbR+j*rC*U{6qXIi_9u;ykun!Aw`SC0+nk^N!!S0 z$F^R>vKo~k+si-rnYWns&28wV9;kk8&bXEma>+Eg-0!&L+~$1T_(%S3|yR?_i9tTu~1Qi z8@BxQo#|)04sA9c*%4RKAwEAK05B&IW2tI2VSOH|p;|-s%T@e*WyRBXct#g7sPhao zj++hD~N>Y*F|a2Q_| z0qxOGPt)TW>@+U8o`%MWm#{AZqJf9ZsS_?V;h`ru(I(nRSW;|cSmKi7EGLQB7R7Zn z&8Do@=dHeo9%B(U&lwS;(Zi%4l~l8+<14o0sg0Xv`Ch0{<5EFX7^!ki`c+|UO3Wek z#a5+`NccnszRV^Xf-1X?r zGu92wYT;oIfuXXtxj9Q;o>hpbeY$NJPi@6r>YO-f?_xkq=l22CQ0pzN=#n3m=jxK-(siF;h0?^hRn=L+YbIvkWX zgFDbHCX|x2z(?0r2zjli$(OX1S(E}lZSMflRqR-CN835AKxhDi$QTGO#otHFM)i(0 z4hTTE={0z~&GQbXJbbw(;v*Ju^bRH^x2VTtXOxtSO$RApm~;&4u>MIWaiOPkBrXv8 zN9k?z`rRpA!=qf@A_l%3`fc*VYv4)^jQ<7g`mjGi*%hwqH1tD3cyGigH9M;=6v~dU z@GiU>nzGyQSY1T_6`GqZt!O;wb$G&HDd&h`h*Rv6!M`>9ztZ=?FCz0lfxZ0S(!T!V z^7;Qq-vh;3O9aufW;NN0GX_oA>sM)?s)4gVd~W5g*L=L3dT16D9D4n!!%n`dyDqOk zG5mghAUnKozl>p;#T^mcU4R+b_=G!M!HBFhR+OhR4%~rcjeZ6BKVw|I10YY^DoBFR zv!W4`Tme<-sRJA?+9Id%*f`|;q9Q#FOU4=gO zbUJEKXxO+OuXV=~xjAC{v&x+BoHY3?WebDmiVKWtR4M>rbmKC1x-C^{@Dqh;gzh|8%>aK-;YU7Ox)XYt_+=!i-6q z{462qp=V{NZVMAyiSORr(lz!dj^5q4N=Cca9D-^oWHUIEv|Il2@NC%`Kn)ARoKeBo zhE%a})tS6_YZPHnlqcuh5BTr7Sj3=djAb=NB?N9c%ud&_H zZj{;Q22*#O>_Oh{8*%=E9X+n`u4gZ$|7YzaRt#5E2GUq+yvHhWrqLt=~8m+lyC zb_J?B5tDL0{_!1tq^rU8LhqB&EvFK9N&>UiE)uFZ z9E1^W1lUQc!3f>Y@;Ci2tU&QAPu`FqhO64{9{C?53oy2)O6ff5mz>8$ZQ?TlA#4I-YW2)_>$vth`DuEWSRW^HD$I5a3gr2go5#=* zq(R2KxqA(Vt*>VT+ z_x68B{R%AVfAaTNZM;QnU&QwFbGm-ASOYexfZ3r#VpJI+G-4dkV`?I{f`*j^Hx2Uf zfz8#6bD3{zo~2ECWj7e*MVO6^d9_+|)S)~vf0}vZ{jR!|U$a*Yb4;;OP7)mGwx8{e zUb`N->pwrP)b+uR2hAPNg~@}j4B4i7RUFiM^I_^7MPkE_hb{ z?nsSsLO8?xKC3}JLg!Hn!Qi0_(O=k20SF<;Qv;-W=N(C}ze7xm+*Ig%X!Y8o@n8-U zbRZ4ebx?+NC-HTVhNPhDC|rjJsH1m~D{xzBd8!G>Y9<>Jj-L+ZDc(0l8y-bsE4#yi zkeQ}MQ$6EMh6h5kQ_S$Vsqq(EYi^EN^n18-S;IkzM5iWVp+S*xlSA|k-!iMrbC_uj zwPa+lohx%;7dr4&np3#0Cwk>zxtYnn)VR~wo3nKODmI6&G%#zhT%G{F7PNC6abR%o z#IyvY)tIM`qpPxkhqC&H8S)0Mvk}Mz+{4Eva0B)P+G9m08>Tcu+Qp^8JZ#*4x@L(XUuWvVk7Sc zu332pkP1Tybvi2MR7s>lB@K>z4(O3ZpQF$nXLJ@T3<=m;OvP(TdBTdXNY2yS99?Ic zJi4#=epX0(4bzVo>_QLM@6u5+9d%Uc$Sy5D&<|R9mf+V_qxPi}WGPyQL?W=^*KggB z*~}eBIn2~81thbo*u}c%NXZQ27mxm>aDr+z6vHGNMFE4)CDA=ku8ZFr6_Dkd%N1IC zIb)pS3cT>Z>>GE@1i3-H52p5GVO#>{J>nT=1Pt><@i z8uyTwdP=MdV{Gn?9e$-hTd*0+thc3Y0qQU?(@ww0!flFdHO>!}P)Qf$Zh(7|*2BQR zs%}IRpg-={^z?p0u7J{lNT9dm;ur~^i!cXIwh`7|bZqjmOBb)rKjVFm9lj9JY`1JI z2Y_Lsca-kA1EhP`skeGV!=;^OuD$#Z9=J9t`Q}vLKuAEF{UATax;n4g-t_bpZp7ch ze2#H|W_)y8$~Sx;6+86Qd_DTA-cV|$M)O>#7G+cU;Sk=WHP&}UBo6DsZrK&BJ->cW zH%tT4!+Dv062jiwzz2Z7Yoi+cj69!6chDV_<8shM$SHZk8B>>m4{fX#&lm= zAZI{XX~j0fYMB(QS8F%Zlg=uEG8dso(|D@LIv7eZ+TY(*|I^cUkc1;xdIiXVVri;U z28fKYsKtSwQZW$;+eDsmCW|bLrochFKyoDFl>N#|Z@hE96}|S%6_O>xC#rfXBUkpH zq}E{}WbQzWfwY*q&q3^BvnEjIH=4vPZD1?M9FFoBRvu`kw(-blQ+jxwOcD29?i)2c z(YNk#4L<1Q)%FRemacMfy-MSC6w~wM__e)n=DTts3~wilVsluBcuS&km~I02pJGj5O zjkk?)L8v2YVudZ`>;hdoMK+M4FuaJe@NrGuryS;xG1nRSV$Z{DNDbxXoLC+Y)}HHd zzN?q-|ITwF8cFfsLLtdspd82(7W*l;F<6^@?+f>Bpp`(}f&F1{Mz+d}=X;3v(c!5c z-XU2qPX_4l6MG~wKTT)mp=lBZ3r((LEnY&Xcvm?wffOe}zTloL$euf7_%Zh++ ziRHN9tL}fqQQ}xv!9O3rLmyHT!)||2Y+%ZfCz72@6=TdSRog@mTt={MFA&}_;jAN* z6;GV>HK(}$-1ct^A5d|^wv8}~+k)RAI&MKmwFz}X@Mz`=yTHp4!Os@_o_#2vaa034 zoexjXa6_+jO2!kKLgfmQ00^qXq#fY|OR zB^sijOjww>^+s%7X0(Mlm*_37K7vp-#`P9du}(X}KBqKUy4-_m3Va zY#%t>k~w`GhvxaD&tOtj=t*xG1BmYp(VjstEY(Jj3)FvP!iol%)>0guU4ysp-t1`| zQ>2^Hw%$g*M^W*dAGP(0Z&`+Q>~E?gcg?Lnb6QmiS!#SMYc;8BGwtgd(ChO8fouw& z{!YMPe7uK8QPcL;30}1Em)F=QS2$_ ziLvY2&Xb>6Aws+5D!1s8DHAbqmq#zSUJc}E{%|GpcefSCXTioH!iz2)aIap?;@(EL5%d>wkhuQAzuGYEt>^5!FZg?^t$A9H|y%+=L5krR2Kyur*;}C z*ANuER6|EYuV5sN4kCF&4Ynv;GYHLUr;Q-8g&J-{XpY zB&@@+K&1OgRpxHDleSbCOF}xjuMp2dDM3wX34ERg$iiz=TCWTt2s{4qhC9R_jN8V( zN{}{lf(uspzQNLhud#c!h>h`Kp}uj2WW)P-oJQ@csrohE>uw~L#Nfq@l0=T5OdVWC zFJuod%VjLm$@>%koRy{=4V6ekAUuzaSRoq`0wb1yJqLwbh|~2tcm!#d$cP>lWl({b zKkr?tAhy@&I1~O^`aemP{vOWe$=J0a@NhJNheP~-J{&O_d3mLO9FLFUI`DXqc*iTX zsK3DglV74hWNo~MEb@L9Q-|SfVN%u>r*u)?^!`1ade~WTdY{x)UgzJLX&0}< zXShF<@s1@7RR7w#T5F^f!^3CbG-08b_o4T43ODFjG7VI9L{G*x{j}cx8JK*reU~gO zv2?q(Vd4bK&sMg#)(>|4qW>4wV&ZvbbsAL z&+vvz%Q~~Qo&?k4n0zP^oz7`&S2x-NH~sy(o9+ma%b^po+MLmLjE!3X=y&<;)yp(xY1}XmR8iYQ+R8*Qu{Sl#Fqx?~ft1S6iCZ zC_3TjA33kNKRMnRpGAQT8x8ZsOvx;9U!2}RzN|u7hO*`dsi?8kuZ{w6C5h>D`cVAb zOdek`lYnFJ+F{PHy!a7$OMMhclbVXm`ta^s=zo3q<{c^lih#dacpysqzhn}L+1MMp z{BP41;DVHX{Si~HWW_Y7S;7^h7R{UeVr7J8|3Jz~T5O#pV#*m=FzB53SL7%u!UUr}4XBa_>rN?fV-{`LY8M@{kg4VlbYoVX1htu&2-PB5 z^we;)ggyI>H>-M)o>Gsmd=U3bXtLQ}Leil-SEU&G#ApZgRMF>@&xV&NtwL|KQA#Xc z)g}bfQL2Ka+yn%}gJrBRT{H6K_MYq-v@8_6Q-Cu;}=U&EeFuxe;BCc%B2H z?A_u_loi;E<_Eis<7M4*O}7gJj@o0TB2KF=z4dreUZP(Ja7q#f-7j^eFHUgCT07M)YCEH!gLMxj2cdWG{g1OEIb-Lp=Cj4kRzD4 z;v7RkC-ED*yN2wbN3(j@9wS#$>5Jpu6Bo_h9mx_)b+hz6BkDo=&Q~>t3M7L#hrrfe z3pxC*MS<=`SB{mgK!kqYq(K)4N8@9p*q}z|NtLjc?2qKZ`&sMIRnHfXAh2 zW?BzVWXj$&MXxYa?uzrbL{}Ou3^wH8Qijo2yG-$~g@NCRXvFSZQ2WaC+n50Mv!_g8 z=n4&wqeNGZKe^vwqp8k%fRG4Z6$w&?>oq{gy1zQYqa%rZl{}gL*K~%XQ*|YY9&P7C zu#x>34Xmhg0zE8<_HsSETQkv!>CfE}>;ZNct(Ycu5k{?ojJrS9xo7dFq`lKjt7bfI zE`iC&AoSr?E{eyIl+O?&dU%znnVy|=8 zI2AbCu?pNmnW=VOQI#eKBCh_e?g&B}=GP;>O9w@%-f9_vbY(*15txJz5gg3w6Y=rf zg2{O`zXGw5+WdnSHg0d#p3?%lJa1l3Up`(`)hq^{_Ec77^+|O4m%BEUORGciD1UT# z`%Sro%>ZL+%{gFQ0mFjVN^Eo1M7h684*p_O5B_57t+NvnWa1-J87i$PR>$u=H$Tbv zWFKB5>-#)VdU17&$nIa%D*cpc;xyxsZ`n)7TOdqQLiC#$QKTyfN4C_m(mxmX1jqOg zv+=gV>}LjFHquLmPGp8Y*QUn>s?3A{D}23-IiFqw^!Ue5iPPmaDHO?h`>MBWnKUkQ zIq-RVF%0kAW*0jcwu2HeU)i0cz#Z1EFQLQAA*Nqt`lBW!wy`WLvjGjjD@}E_ZX6hl(CInh9cOlm7yK+`M@DV5zJPM0VWLonmxwV zzH$pQ;L9UE{5KInQa-`fC}3=1{ZDM+Z0nMMTHEh>Qydsu$QUdl^c5XoASr3vg2C}3 z8AnKok*=ub=e%_|cQ$b{##}x|&|X$ho8(ax)&c+RQIf@vy&foCC17d+X-+^Q%bC$- z>+8y}sg4fuh>}v`{>j=;E<562pjU9~Oa;OvYRg-^e1h5k1*;G96~uQ4aS7Dw3C8V` zJ-?%irM-`){b$fci4?Vx2&;0fqhHDjQ%G22?`lSm3t!mCtvQ$iX(Sjbr*z2YX*4TX zBP0<%jVF$9+PKTfZW`VEUKWG$E$()fe6J$}JJQ+FwjQsfOV1TD876Jo&oKd?zdIXN z`bVBMgP#^V@=as)!3MEUhhdh)a27KDFuBDaB@RmZ0DYdk<^0!=i|QMu#wy@DM;};Q z_+N^$|9I#8+cDig#TK%3VbFD!XLI0@aEdkAi4t_!=rQThz`b~}5sZkELt>#7?aSr_ zZo3FKy%vtGJ)p(r|Est#K_>Kr%_TYBh2iD;%;$dHJoD)#^A9eEEoja4vjj(+sn`f^ z=dQ3PAXJne6WJ6puWnb4(oAhiMAO;sa=OnOrt;17_I8l4OjF*hI|L zsvL2|J+xsMSlJZHuU!a69BB2NG?$T_PiY4t`y$wl==*k&VahFax}{YMXr8cl(kaB zS~uc&!9F${tLiR`((G4+KQ#I>ZFc5w5*$EXl4Hc!l9(rlpUzc6lHb!A z1CwonZHZ6IM6JahlVyT#O~b-dUFp1-wd>kC4NJ2`I%I6YlNG)KPNRvx(HZ@|>(WS7 zsJT$(CkOERh*bd0(?A<@xv*vCs@}?U8*vG4an+-)Yi2VA!t=^llDQ^Tzy|#mTdeKo zIp5Ji-#GenB>1Q_0LnLe6^qAnB{{&`IOuRCxK4AuhyJvBvsP;|lnk>m(FT#sG6pZQ zd!zUErpl`qjl;>(1;5~WuHq74K-q&--7C%=ZB9D0{IHsd5x2%ct)5>fQXsQMrV6;1 z)big>3n3qIj(zL_zexzpYvVJ90|NefqH&!r#vHF_f}><$C192`D^aXcaO zuuM9%V->UJE|gwNA6nSVAZ%8+RV!Yuwj5(qjkeo_hfXSDu zQ}M9%oGwD`*t54wq@j>*L5QZtEa8jbSk1Nt-#xGTV4aLLzye(`HW$6N34LJw7b6rn z*ufDy59rg)&mn!`9q#2m3o|xI&-$)x_5}$dCz(0s+d~Qi@C)HOwz}6DdV>ux5uL&A1K6Jj;tM-Oo?y#b z3JnewLnptliTAKl*+544N~(WTozmY7X!s83FH~-52ouN*ZBMg8=nHb2s;-QfPly$( zy+fPQ@1@p8zY^D75<~s$Ry<|v+?oga3Gt|()8mHGf`O1vnAQ-2cvXnjY$}r5d5Z#q zDaz4vj~k<~hC$h+MNDpogLt)zbd@~7%UyPrQgVy>&JerN^w}4S2o~8{G{XyA@grw zv_V3SG%n4ZntPpY?D~7~oDyo_iedRe#Mo$JG{s|-F4n5c5t)X~YJ-v_ zLg4P@Kyd#iq)g==nf=OX#}SvQ)&W*`pegt=MYwyF3}zL;Mli;m?6%(~IMLej@{-x{ z3S)>Z!9h{Q7aCP*yyX`Wbv4;di9w+CI132eV}lVdU)Jm~=a|20;wU-01l{WBK7Na)l>u7Y2i_PDf1K)3#wMmHM8Qp{dy)TzMM7@AM&Wl37)2m zR8m-8tleMy4NkRIFLtaT$HkH}+*^Yq&#iP7aK9!kvtW$h$Xln_pGx_@wtlH*25dH< zy2rKQX~tJMBkDw?Xpg~HLI;O-dK*Ly=+@^cWD@tO6Y3wyd2W3`)mpgVA zrKh_YQ*>kqGlSuZ=j}(?%l5+umK@Sxo7NBVBgoh9NWdFrwNds11G21`#BZgK3MO~- z#^gTzs+&jaH#ez*Dw$9GlyH9+!#&+8FIVRRM!VxB{V)Q;7)#&<-fAx|$kPYQkj)`H zox!KcY#pA%&rA>E*WAYK+j*m3Wqj-(=R4T?{m4&YiqNQTGc&^|O)g{OV(lbiH27I8VjQJ?U5EhV}bHzl-b$NC*{ot}G` z`aILw-W?)DU|5-MS-QSYGDCjbp^(bB_T}63&Un>LQiEY5@V0S$B}?QA}YJ>PT*$ z-re?{RJjLN45XA#9f5EQT3?~x5QHlX;V!lqP{PM{e7q-%nj9mCPX+QSgd$(S_PTp1 zq8Tzs=@oH4X1k4XcNm|qo!d^XcuYcF-lL=SR!>A3qFqTZOd_WXZ2bwFwZdP)|MfXV zhH%;`2R0l0fz5{hH~{(YWP^vhss{SU+S2;yF1gSbWx$uzS~LI>%cSw7kR>#+sf8w4 zu=3e8i-)W}uS^=bke*e(6{dBVYp#_Bds!WO9Eaa@LbIQGewm)_+H>N^q2TdD(Z+$s zYx;vefH)cKO2+A;*!hUUAOlZ`;SqzFF{3Qp*@7$)JLobo_z?#$!wYXp0m$&Vz-$9o-sVL$i@3CKlsSu;o!s!?!B}e>T~br+ zk~ucGig1{DsXw{%+)-JdU)+)@8O+kEUEob~P1Dy=EGCpUGn-E7{^xLiWZK_a*-T?J zvGXdjw6m@T)^IpganVx!JT-mmaZ=UTdta@q`(%ZQE2p=oO3+%T0$0$yt*d=z&9xii znK?0L+RW@qgw2qX)U8sY(fCuWd4$Mr5ga zb7EI&jE%_i+dQfUG96~s(pQ-wVqpkkVQS^LmLzq}&7O^&w5tqFS;;J#1>A&^R86--M(!^E9dJu=Nq{YWS9lBZT$WN6nUQQb< zYh}@P-`FFSW4}`NcIpTVEg_9tW>JzX^vHO)99ac)a8-@0n6aj1)o_mXa}rC0tYc@$ zuq_J7PHtnI%g472yW_Tb+2EgiEeYX?>YkLYNl&k>@%v>oGl?u0kmFoEdCsN7^@cgxTz zWNR(lfvTsfP0Pq_rPOoaacU{Ibe9^4x-`KT>{4KVl^syRo4NX?TY;23JI3Sdm*@gq z6;dwbhL(5Zh@HK%##^|8(T{(qs1j9B>UMsGs2Fib~*~$0woMpvmx*$-Wy!NeX-)#R@J=KE^dEr_l)ku^+jN z{Gt~Kr@>|IO?{fx@;r*yaYNA)MQ?JNz+>;xlYZ;6{m?kTmw@d)_Y36 z$luh+aHC%HLGz3Q#}b7jqx`(`^y_H)eI(lI<7^q1MN?Z;nN#UygneHfs#}p#NA!lp zIn82m&D1l?xPNGMz(^H(XrUSBog8mKPL1&tLVl7UC3AE}3T6oW)T5}MznY%2PZN=p zU{mp4*$F=MX{`}2xIE06F@A{rY0Fh)bz9k6)2VzD^VmRu?XiU&j9k&OUBCAVwr1}^EeS{HJZc1r-JjxZj8Ymhzgh%(XQ@p!V?QF9DzgGgJfgQxBF+jd^y}cF=y(~alf&Bg$<}@9BsX^ViqNuaI9Hb>O z$|!XQ)dl?stoOxgK9DB^a2;i+-LkQQVIdH3WP`3P${T~JaNJuH4p03nw~_54WRfuN!M+iDe1GI;VEgLB79SA+~6(^^8D(KI|z7VAELWGn!) zkuurA19*5=(}KD5!yd||w(-Y+wzNAda-^|*?{_G3S|&NDgfTZXZ52&J{`K3hLsvJJ zV3&9`TTH}W8B=c{``coo$v^PX7W%@J(jG`Y8)45yZR6pBbx?hk;2HEPvgP3g|0cP4 zsW|R*yHt>f>$0TjS%SN!*u0~fyI%>q>e8|6DYv?^|7{d4OqbPbcc_jP2|E00e6K=u zgBc7U-+E+yBSyLmHQvOJOc;a7>OZpLUl(zf-2-JG6>$8M3}0*}IvX`=;pJX9@iLU9 z6);z+x63hud%$v1UT;pBQ$Iw&WiH}!=5Pb+h`gH-zw+x}N~+Y|NR@CYtNpXQY_OZ6-uE-e?<3iM?C+DztF z8#l4qkz0X!$V!O|N2PNvz(}wXJY+I zU-P>(?=`~gHT~kFtKlQhtx#3@m5x;YyH@}E&Mju{tlBfRhJRNWKOGSYTwE_fyvRDh+aCJ5jY{^AyHDA? zeerBmE!NdkZ7i-tSW}Mn|SBr3vJYIE7_TT$R%`>e%%*PX#~CINxcX+crWmM)yN1dOz}4mS4}AI2hDNI z!gSo9>b~PS#t}TW`aHs)^0I6U(n2qmc;AjI>nnY6{!qr4Me?aeBi#2E*bo3r{(GcM zYYQ~`19%*J3k2Bz(?P%=x>elL&dBD!?z{c**M-pD6ZEJ1<5ELONbdaW^O0JAz-Xe| z)WJL}0>OOsnf`esdd$t^&U$ULyra4j$T1t%4;p%l^wLWaMAwGjbnyr_!9289QJHrA@(-S z`g6N3c4sg(3?*hw)V2mZW?imL9ep|rARTI3s=$-JGqW{}lA|$B@2FG` zx8n3qyPRZDz1KoGc5UeLeX7E#?augHszrFL3J=~RC@$or_Hyp}zjf8~o~K$U zVlJe9O9R4X$M2~;e~RX**gwOmeD@pA`f#?i$TCZ%EW1^Eim3Eg)(L~ViYet;wf6+0 zS|J36cFw-cRLw)+#H&C_+s)L73Kut#`^p*$C*xrCB>{pU2)00a(myOIdyY(=dmPF; z(obqZ(YKgo5BMkUVqRJTs4Sb@y%<~jPO~?{sN~gwKciC}Hl4lL&ox*{46lG<;!HsWwM%j2Re0!K1XiHh| zGJRGG7vd*osv{E}I5I;q?bA_85~6Yh6izXBYNb(*&o&N9bAKmkfx60X zio){j0Om(8Q2S5A?Y{_QOJ~b}JW1;R4Y1qn{sGuUMRETE*sFw4 z;o<+rKka>8qL7JQYg~$ZRC&cL8waamou2>#EaseBDMI+>EC<*6nGa6uV;di@SCl^x zH$@%5Wt6BMWxLj>UNs|)!As7a6oj>fDZ-V!3A!)`X*5+F41^i{u@b&$x;=P<;kqMr ziSRj91VZI3hASS(Xqd}%Ex^=+Kc`VDe*>fVmY^p+lrlT)S!@=k%sU-jG$vk_V@_qb zB*+K_h6z98kO1}?azyXfSzMP3dwT$;yJe!Iry&rm|)e3C^}lLJrF-x)=*q0FsCh7y4Y^H=*HMG2kx1V@-y z{5oam^*51rHTl!^wo6xSETmD^>hWSEy!!EDjYj=ye+le;{}9-*CFJ_hfEOlFZNRs)Kt$y5L%`2~$-^BMraJzq zQ*%?1qu1bFBj7+gn9o3=U7HYYjImAB#*5G%?yiA}X{Or{R)nw4!BXH7W$LV|Q;obS zYNR6gljc4q$MKnEjt%L@FUrdjY_WvQX?B1oVa=csfrqrcAvz+}8xl4jNkgY)^^w>U zmeKItj*qTMXij4USb-%gAc|QeGBZVhjCsH%qV+^JJ#(3Pb@Tg=PiN#uFQkZxYeVr~ z%~D?e9Lpgc?3JUhYjy`wN!}UW4rcX%TL=6#g3}3H)mXjhGZ@I-loESsmu#Q++`?Z0 zm#`Iv0~CqS9_!y~eu&wBpvA6|?nZ9Yy2|{!;%MJT$^2&p<(%o=%$D6y z|LE&-EJx1L(%D4DmYu~%KM{sSY^>tuMRR=$%jUKr+mdx>y1r6avt(Zx_?5?3f2Dt3 zBiwV^kb=pk7JhU3yS{$+kU8?!W2lIS)~CyH_D0 z3XTc}8ggWnX46-!Kz&`2?XpRTXVo(`G{$vgf%HLU`@&9+HbRgsZH+d`tlX(H*}{Poh~Yr?^x>2j$GEBhe7 z6E#aJ3zoUxhlnu_ZQ@~ajeKZ@PpkQso3Nj~fcz6sUnl&#zOD|`*CTRQ?B-&PXo37w zw}=*PJ)gO$WI`*T?hL4}qwB5hUhN3bAKhXUICOP$r*SG)0~ZHbD(#FwQ$ZTu5w}1N zpuL zBale3sepcCeZ7$y7XRsk+L(cl_wr{0h1ofcD>+!Gm?k^}q&n{GAtK;bDig_Z^RW7C zKEzaIUaRxz6u5y>0Ng+^BUR|q(msnlJB57r^R01ujs|X^#4Nz&A27YzxI*C$Ad;6~ zC#UavjYWwd3Mx2eJkdL}4W^FiIVg-0e&O3E25>e~kZut$IR4?+QD>Bh5dQG%$UuJm zTL)?VB^Z!jC$yy_uuYSrKgjG}DA3BwF|<{vS5U+`?Y@4@ORLE{hZJQ5ydG6KF@H-|9K7#e5U`g zfTE#0hyIQ*y2=Ze7@%QHgc3r*^Z6?Zd=&yRF3F6cpJtcp^nqz39Orry>v~fxl1^q- zJ7I*l&WyTk$#kunFlj)w_KVK^@x_~4itoWFle`%~u^A(6K;XsY{{3FS?fovZ;|;X) zGhJX6;_qXmE)ye!$jCSZ4GA7N_}^a!2cNE(_`ddBM#rwq7=by652WR#&0PgVX3pip z3ocxJ<=4Jc-CSnwx| zoN_&`*8t$4&vToAw?6MEJiyDdK6gdyXSnJrC(()JsyD)WoU~LLucotctvp|A`<(DY z;B>j2{;7VM95Q$R557))Q;Y2)%ZVMqYdLd)If<}>hJ{oZamq`K#Fdg!kWfZiZym9x ztdhWKB+s447L|uu@U8pSril>=^u1IL(={ru?d$AlQiv99N>%JsQQFyKeJbadnOfA! z{n7v-`@A%_<=e4CQHCAOx0m8a4U?(yY#30pZRU~mNa=j;4f&RY)VzTy;`5+=6m@an zHi}M0Vn;OAKDx?MUh5F2dcV(vYwVwW6m=)XVls=C~+ z1lc2%eNJJS=FT+2KX4pIODg^ZNFU}>qD-%~{MO57?J-rl9kDMKoFE4Sb-K|ixhPBK zRFsw*YP|qIHFAxt(yQBDCCiyOM0pz!-YO)Mhtrt*Va<-0$zw{c$~}6VB^Lij$*a)fk0h>!8iB_lasAw9%7G;7C^ zlD2fll^4D>D2&I~WV7KhJ@NtG`-cgHKwOXBV_PRa?rlAPAPMIsBp&MFQc_#3-o;w5 z9y=K5_g)DbAx~pm_OP18yFNkU=fA|IXmIR z_gf*no~C!#neR99XywC) z1P6ooKw{PjM*EZiA0skbreX@hJ6}Q#aCyto7HLS2L?C2@t_N0_>EQi%N#idh_hytk z1x~7}S`~I%VxB#zE~G;jzvI)uc*CP%Ct_FjkD@uEUe9=l^m?HO0U%{8P;2y`!@Vg~ zf6f+BSL7zpwl_s@6}Qp-s>`gLl^FWTW{$azCLnB0kSORl7O$$E%3fnKpH_i(^%=I& z3RQiP&e<&EvTWr#0x`Eha7027C2HiS5E;YUy8d3u&+E>$zyx_7Yd{G4@AF?MKe( zHh=Z@J4^-=DvXo{_&J1M)#>o8lNg#^a{d=fjk;$@aTa0VEB@BoXRl$Fw##Z3Hw*9h zSc@|S^hO8ht9)!xkxBDt%?H2MTWK+rwcu@M?l|Q3UEY`TBOeS)b9|g8SnRZrV4WR4 z?NmR9C?w+vpJ+i}Y9JzEtTo+~c?XMu}L3u0oAAt~wdE$R2nUPPGW z2-;GRv690y-M(|Fsm11SokT|+j~GUpkA{P$utM7cPC?bfM`T}JNjHh?N0jPEaj$Zc z%0(A!b`2wEcKa58{?a1`DZ^pUYs(ST14lQH{PVReWlS~65Tt$unT8N}TJ9VwMt4cn3q( z5E-uni9&~cjW+fpq@lhPWmbiL&CBWhu6^nS3G1U!vV2oMc`q#$Z`nt(ccVXvM2}{v z*k8}*^_>wJVLuQzs`4B)5I`+cC!Tzv-+i_>?Iq(>fzFs`^!yB~lsiO2p-i<$=uIBs!e*JKd!?i!F@roX&|xo?=!f_AEZ2>zz_KU(+uwa zS7FCsf!@+j*{)YS?G5>XLM4@w0M_D?+9|@4yycjC$_0BpJjk+ZK85)^4%_`eg85Ay zVRPAXWb^L`d&9-MC;TRauN!lS2PW(#zOI`?PnR4f_bBFN+g^h%-FAH#?_-z#Mf0HH zP@fbXucqT=u8^-7Zh9%qUkJNr?Qbv&&RQiz&oe!A`;oO@F-H+pzqTU7>)HsRYlhm- zVRmtr2NgT5adNls6kS_Fx28&IUlHdf!r?{v)YT(Us? z0*4C+GGy~Q)=mPfnwqaP1!8@vU_CVcJ&=1`D*a+q~ z9?(!X_a=5Y`L;~$@SOAxl}2cLfa7MU&HUobk*!(Ym+ z`QSk40tt`Uv8jP5CsS~l8We_cJg1<|l=GOV-X>8-+VSK7b+`8sV%^X;-FoW0h~xJ( z{J5?9F7P%P57{x?d1>1Ad=zwPUvomW|J!ucFVUZ@ ziThY-V|Bz!_r7@i4U7*?E|3N{BFxY)BSMSh#OeEEl$g6sOCrI7?n#}N)7~_*Tq4Kt za1I!lgMs*%Wu-SV%X@>E2k*VTVPngwV=969a~gyEJfgnKN4ZQZ5><$azXjy!Obv){ z4vk&=i}1U>dWeJ)KC@6WUJ6RSqvK6;umhk;tazxlsilLJggL?Mq~$X16EDdvL$0x( zO$TKf!&ywwl}zR~^0Y9bI#*^I8oNo#3%J4CQ=13;%kk0w#<16X12XJ*D}NYvMC6;% zr>J02B{_a`=EQ59@z|Ex^xcIwywLCe_Tfz6Zc)qlf1Shqj|{sSmajZ8g!tccxW6&% z#(oqyIA2zXLgz=!fO9y`7RMUu|L+{`7m#898^y5Kn%eR4w4?Beurp!{CSm*6WpY4^ zBUlyQ0E~LuUPeH8fDKyiBOieAS8OH4R(pUq95kI+<@s447RMzKAjo}JFO0dj3Ojc^ zF=IR-pn}{6_O?kubqqaebVXiE$+}p5Dn3O?-BndXjRTimWkMMU!RRAFG2$d_F+hW< z%;kCqJicQ=fIGA_U!<&D^^u z`7?*pF8Jk6!j!b3mRS^9q`ilssi0*(*ue8ex$bv!U9S5jZn=?8vzpm;9Na~+k6eWZ zo0UxSPHhaW)2M5m{(X$+cK#s`V6Wx(;3ajxC8-ZU&API-%EVZ&n-fI(qBI?Ah3f1$ zg>dv8-fu9+tbT-$9>-M11y{ZSi*-Vkb)~YlkKoT7PQ4a4vcMd^f*VBwIEQOW1kT~U z{+YumMZJzZB5cS!3in54DMO!zRKd1Yt4z#E^<%BaRg#K8p19w9sLE_AC*Hz_XH?9? ztawPcEh=LVRHg6g2S!t}w+xd+rmr})do1H(L6bxfQ z`XufA7|k^Khz?Ka$e%`DyMYe7NF4nx;Wd~q`_AB%N^`&Bg$ahoXs*x^eChV*d_s~i ziCqkCIgU3fLQ?4_-_|H=StY!N=x#P!ic4m*L1<;xA8h-L<oQNI7k6 zKHqnDoR*ueHy+uZEAo*-zp7!pt5Kp|=VN@PoWE#hKg|8}k_t}NK!~W|pd!>6>^Z$7a_n#Sd z9BCQrvbfqD2+5^C47){_Xs#|Wg7}t2nQF3_a%#qr>iJtp5%V^QHe>JlNplK?A-%guzdHcN593Sz3=Sotupf_P@8hyI6Q z-vf^s0W$2gf97!bf97y%f6d{}#Kpi@s+E8LZ4T!n@n3Ve30vsD&EZ7U^!{rO$Dl(m zcD5~r6<@D|_-791gKgIcWY~doxH-SSG3*h7f}En1NGibE3Q>i#ubTec{MJhLiPvyR zm269=*H)*J9DQy>wttq)Ppard@sfg(??i;sEevfbdI+EkB~w5MruYv8v*ii=6m#&P zQoL!B8R^6XxxY*?$XhPvQ<)RX}cb`aYQDprBL>`I^ci?2370Zc#U8GqNBx8 zU$Tdcq-ky*+8OZCo;{+M1vXQ@aoJ~m-IDP(HC*e4-~hLKBkTqrPw&p^?28SkaIz_; zK}uo(^CSFv{{Y3f^9X+}>r$}giT(bMF`Oyr3PyWRWZ=aeOCTYgZdZ7q-YVwcTVfYk zjz*=2uHVT@epx$=nLeuiM1nr%hwTihL_OtA^F<2kR?(Xz%Ghk@D%^*6O!d;&4X|=^ z$N^tFp;>DxxatVnZEdt68lV3isErK$E`sCD~J%VT$z8bAlmE#9rhBR z>a(`ajRkvkeL<&xb=V)Xd({MLV!bP#91ph+sK;45z{@c`YMNlrL!Lx}2l#p~Ar`)p zt@e?Zt1o=AH|Bxq70!!y70A)^_j-5h#mF$s`}vp~u{zbiG&?=4g4GfvDHr)q4!g|N zZ^={rPzBs3ku+{&Dx?)nw836%!0DI9?hY7W1<{uROiB*ilB+Z)RICtc8Row$h+DRQ zRS?F@2HJ`~MZDO)(ilGV{hzkU|45CN`tM+Qz%krD5Q6zn*Foj}+hKq4LIXN5 z3M?%(OR_{-U&suZG=G_*)+j;`LzqCDF~@+NfY_djnRCXDZJv+1x?=d>`5U*Xmtxpv ztF07A%?j@w6iN>9k8jIKZghkWwCk zo3ETCm!zZ{rW=vb>Ne zAKsqL=o6d3Ih@*6TIe@noBkW&R}jWRcjh07yU)NCyh3;OU|Vpw!cxTz{%OGN1fTUm{`j#(c=}_@2L9IqEWD*7N+G-$_e<@ z2TjAX_25sfSqjY!(n0v|8ja*6@+bYRmlKTgEghZXPNtgTqGP>w)!_%IJ<0kHjX5$C zGQ3O3w|ENTDYA)%`ip;B5ev)=qwD(7b28}>X&MccxTl`fZmBy>VXx0gT9-(ujK&E4 zC{d0~tzj`R1}OCF%5ToooCm=sSer#y!Gf1rvv50O9AU-CI)^*CEVzD+Q8-HW2a{0{ z!8|`;%8~i`TI9bYEY^D@tS`uKsS_IrM~@N(<*XOw$J)e#aL=abwFxnWQ$RDu&bjga?)VnIC3-_(TRZrrwlrGU@L^fY-;+jd5Q9bE(YB4)Q%itE zYBZk(V2AG17m)EQ8gmiJ4($B!LmOTq7I+m%1L0eX^^_{NwN#R&j2kS%9l8~G6bi_d ze>{((dHHU}Lj^hQX>a z07&w+0o)ukds>_pe?EY@uIDw_GNLyD1UD;%KE{ShR_-EGqrKpTdw}LTM;0zx1>oVz z4}f@Oj(&q9&}|C;{d0@-Lot*|YK2rbAnGb4RKe6=0Clm|a|XF_hj_s3${ZJBKI14o z8sr9)B{Z|02mc{zk#wRUywio-~KpQc~(pKDE3HR!L-FR^w zT4K{-GiHiA1k0~p{BhU?BYW~IPq*mO)i@2;Q05s`jkh1_AxpNi)wi#NNbm0okSxfH9B|XL3NtxD9x(wXF{9a4kU6@%&9G4$340AVCW5|;EG1}6-BERKzYR@7_v?@kh!X6=7lhjVozm?gw8-fCW-|Mh;si9CSmy-Y{vtU?7HU! zuU&!HL*@E~&d=mFKZwv_z|mjeH7hQH21-zPA`-T(|N3dT(y6K!+O~sFXBdBalluM@Jc3$e-s4-xcn zd`7i-&6jK!90@jC;oIm+6aL+Ma}&`Vt^*F>Xn~h^DE}9ikWZFm(4@Sv!D>X}dr{oduh#il#+nCcS-)0qjFC{#@ z&55BEi@Ui;Pg{Xs?0X&?$xkO2z&-d&8f_XDO}Uu`YbSTTE?gQEca=V!#6weiwJ8_F8 zzr929mAx(>PUjRcWekaKL(MUGYfg{OJMG1KF~_3oAB0Y8`;kS7=|V}}q-54CqxWZ` zX8CDfk}hKH@v(F8HOoc3P!nblKmmc@ullnxM5y4$;6b}prEY$(vSmQc@>r}ipQT_h zD{TovDLjzswZUc|4uB!y?ugue$ZXo}_%V?wROPAxROeXm2?7YB>sf4!PJXTAPcqL- zTy_w96gOv^2gi_?2>2+?qZgHL(PSu9<{DvUT0EjXNSPOE;}jQCKgx>WXVN}?=^<$U z+Q^sVZmv8sH`3xWt&_^frX#GL7E*0H!5(m-6PX-QoPM_&q=h}E5x4E$%2FBu*{D?= zu*L`a8&X|+$uXe8^4kVxyMG0{_fLCt@84Tq4&dud>X54}Cs3{je5 z*tlLqH<+k`rn}DtM9e0q`i902o_psybCoe($m%fERQ1`#JUGYTf`*x-JVg0aW_F+T zM7y_z2=8yv{7)}Ihfjw;6VKQv@2?HJ63V3HO)wFLZ-i19~ZH%c(ia ztwFQ@Boh7kz`utK9)kfs`EUP+>#u+G)_=YJ`i}srDM~kfit+Fm5Mqy;jLVVdy!`#tXuUUy>? zoG=c%t~!U6ZhTlb+ucgdPENo^O;x}PM|+>!{&QOdT!L*{XK0`rtRU2`hD+8?vrvKZ z8p>dK5U;eMrwHV+*7TbCRKe;!b@Cb%|B?NWWa>*a6Yj-vyHI%%=n*Ve8Ij^tGvkYo z*wt7B({W=pWt-mKz4O4Dd@bQgr|olg_=?nn_Hjcu*RhZXh@I)GiA4YOnyhELc9C6c zoY(Z7>7*ke0QzReQ5w1LErf5{E8ib~K~&BdAA;tvlAxi>Yo}9%pES22KgU9|!lLT) zn(x5ulUs@_-cgo_h;S?`Y3LLYH~wvm=H@hICF~-h?lZ>mNAB#w3S&HGsxqP-EZRyl z>L6DLl!YVT7ZZi&F|tc_N?tWiq-7%%kK&@SUFi#8P;QYYsY?o0kmnH^6a3sqUJzU! ze}>5K+s>Qr4#sJrq+$7=2I~6%W9%H*Gu_%WT@^bO+qP}nwrx9A(Hp0Nif!ArZM$L{ z70kQ)`?_a(&oR5_M?Ckl*0rwtJh#d6>M-0_#mmOu3v`x*R(nA!v@|`gzrn-GK?>~0 z1tk%&c%JVB26(>)puxvc%TS1|1S~hI3CSnHZRo~=GNNjX$Ck&Z$Dd}DFixx(Fv9#0b*n^ zwug*i;V~Od340Zf!0wlN%8$RTP*Q25mOK81mihm{Jp2oY{NI|e|H3>VK&P5ATLqv~ z19dvH0G;ARZKQAGWTmx4D-d&m%Lzu30D1F0Q;3q+1&{B7-vg>EtI$OMH{?MVMLC(_ zsLsNm+acTK@ant&Cuk4d8>Pt?kNj9h8YB9MFNgV>g354I&`+ek*u7>2WWMAiQGb|C zO-*TuX^f=$j*#(3j{7zl@guk>Ot5H}Z#_wyewtMl*C+Y7kU%^lom7+o#cU(hw= z?9t9fVYt+Mi%W^C_+~5J)J(5h0d^8 zN8A9iyeE7Ve2*C`O|+kyiHIR;(A$h97hidtn^ad8*pwuOr{n zsTnv&rc+JN6|IY=5*0g7J>tqKVC69wL%wZ&+eOnI%n5Yp4bNhqp`jE^sHqYJ66%Ph z;odF83(5J!;sQCGLUPX}I}CjmiecbvBS^C(8PR?jF>QPZurljf_QLlU&a+M_rB|OJ zG-}TQIqh9~#F~3)dgKr%E*+;fqEV8j9I0VF74qLWPSK3cOrQ)GexdNRgnH5L3(0RAsnPf{#MXaJU`w^+)Vl8Pp zgT2LyuKib!aqLXT@oin%1~?LjFz(xpd)IA3F^F(*CkmioHd-krscZ8Xh8t0CYkeDs zNWb*k6Wo6IAt=`dsdX%mTi*O>03}l3T=xgm8H(F$Unz>*`b=b;)D)!(y_Wux!oG_q zL$F?ZmpP_QoQ3ygif!xfmG!-dEaYb~D#5j~wvD-wcf~w-Y>BJB9h)|IhuL-(!D?J4 ziYdIxz=om(d6Nux>dC0XsW`owp}s)-oGLQsiRwDMquR zAs%OD0|byXO;>CAz_1=?Mz3*$Altv?PtTdw2zch;Onn57JR2OZhIVl4i`*CMQ3@ou z!`EU&npC5=_b0IYIo943^H5NL#}Kz}(G+2*v_4LUyWHQf{(c?7+7u~MO-Zuz6@N~L z<`6BzLG>2@0C7VC6>H%59ZwKFU$ZZCB<>L=+6$Xt_8HSgv#`GNP~)A9xS-e&lLQP1 zi?ZYp=Qoiw#^pGsAd=ugu1^ict8gNJW!>IusrRC zdL0tO;KaaN^%v=|c>jH-xFok=mnIH@AWcW8JXf04 zu~Xm{WYlcE_C2si(%oK%u_JkZ z)kb$MAnR=7>3(Pv9YnldH))?| z_dJ~MJWs=idBf-J4~&X}DXaM3lg3usb~}(3D7*%T1xX}FY?3CyubO^6st@8iNx9v; z%!20wx9}=dXi@`8N3Ji%LVs>zqWsEiV!(^swWt?a;q+&`;m=-|>RPwe_k?bVbeZPq zy%NiktqQa;{`j^B&B4^s@4$@w5h3%2tq0l>c|!ROGG0CRuRg8r-FM=qYt+B`wADv3 zOVdS3Fw;2~xWsH+YXSV*y_%2^9(Yj!%L2MA7wVYz~~CQ5{O7MOLgyD;Ce z1QW5vpb$$-iK4m7Yih$fp|k^*ATC%cSj6SfHEzC&mkL9HUDLMI9Vq241?o!!IkR}{R6UricIjcE~7a{>-zETD6l>(Wh_|9P3Sco)w zXuA#DXemJeXGjAdU&do-Q?Xs!`^?A8XZ{HX$%|T&X^Bq^-<|p47LUiJ#}!BZ_S>Hz znlJ3T$wIb2R3g<)18wWDb>Su0qQG}UQF>a=e}dl-cA*a}-@tRdnSbNwE!Z{Ux(fqH zu=$oF&^YS$o49EP4x8Np!NmhAbza*2EZALmqEHm9yf{5lRx#rCz-GwYo$~=WJOe~6 zb}Ev7* zGc_tzslVccbP{QNBn?En+RluQs(FT58`CiX1l~^~Dqoz`XyryyV+>C|1rOxo0-cQ; zysZBW65O|4nS`~zX1Z?DpM%Sm8xjsi`apF;5)M0tY&b;9zXfa+B5YiE}~v{^d9oNQm{)DHTCKEXlOM=&`j^J0Dlr3ZC2@|0yF(F>r^fa3v6pHVrdwh3 zCqJ2%vg`hAf@PvWf+tN+g_@kCW~`_hOlV8{*Yxb=v6Ob~c4Rr%N4Uo{N{Ww)-%?MMcu_;yW^_x>gbi*Kq${GeoVH<2A>R$_N-XAM zn)?qS7a6sm@ID72C(oJQQ15=P%Ixfym`I5-SbQE*2C7~zexNl+SDZQa*DCjKPQ>Cb z$JjAE<=5w{dP)%6Bk%ywXbe)rBV_|%k}%r>ew__W;L8-rzrz_cNNM3+w5SZL+s-qA zF)u8AhJu16sZv@8q)V4OQE=G|xN4$q#w0kW4T+zqsL&>wf;hK^7tWZj%5nP#6WDm_ z4TJgT?ZV?ZC+kcmzsW!(* zWh~#|Z!J3t4U7GDl?~9sN@E!YLr*j=*+-^_t;k4^k!*hPEXq`!m@vg0qYQc$;@6+A zdx?FLle##2#;Ss0b^^!GJ@x3MTlTV-K^(pfowzq&tTQ2%POr-YclF)0y-N0dr<250MgH+k z@o3p@t}HbGLK%-X!l}5b9qLGijNZ#|4}MWM(vhl5h{&WDsX>d3yHMiUXX7eKCLKq; z%Geg)woe^r=be)(LvUxP8CW~DuDEZ1)2)(cxmA5gSqP8S|w$Exnjgvp#)~q4Zju`D0V4!rT*)v3h9uAr)kUfZ@E##vu{-G&lsLI*W z64D#q!eSjjIi2ZgJv<$^Ea{WboLhFrl6lGEbVFw|b{2AQYyZhIK2pn;n>MaV?+D@a z;+Bx}98n_hJ9`%XhM_E^!RA`Q_qMir69d7Dt0L+_-q1xgL2zov7j_o@#l`X$pPBl> zEP|Cv3mY~Rc$K`bvv=wh+`UyWpijjz4K=~F74(ncHO0!8OBKIHZfk+>tzQxI5r%7V zf8I!JCTNBi&x94Hew|i41H17~(xzYvJ1X%>Tbm{UV6-?#urTb3Wu2O343>L}g`9uu zuS<7-Lh_4NUVO%_Cl}0wUr~@@PRSnrdKi)|@M7zaSk}N_>X!A@=VzPBfp+ zGBu6`wgoeuc!20gzqq`^Ki_o_PiZ3;{zQT=*N<6msmE>h3T{hv`r5PnM-KL%LN8lNV$C5XtF1=VjDf!UEI3TrQHNEzBW;s27Urv%w2sv%9DyRQkCY|SzyI)V72(&z zEqXI)r*gL=cGm)cdrprwMaI znK@-I0G zl@0-7%c9<0g>U|-b><`39H+WV8xcgCGfrvO)Z&g*Ex)s7H?oDs3Z%xg9$>^l#3n3L z{(QPg?J%*s`lk1IP*P1$V4F`v6nA9J+43${%sTF-c+cxub4bj%*i=olcB!05zDIK? zk9$>-cqPw!4yWYwc?(~Jl@FFZ>! z=g3Pw8BZ(tNiPTHre?{f&<-`kD9EhG=G`Tc#?vrYDE#@GzQI81SR^6~G=TZ1Qwb~v zrXk4HymzgP3l)?QpX>}wOdPxLSxI2}hDZTx;`zGRy~llX@ANE81xWU#2(~1Lh z2&FkQU%{CR$9$s}#1jmPZ5qja5?%N{i@s|eRDC0g_1$702?pdold)OxZ=050c%lk` z*~DLDRCB!~ULZe5V?!07quh&)yv;E{`NDpUTtf%&!-vT>ZNwA^NzoBYrxBN4sg|Y= z!D&%TvuP)KSl@Eoe@5@}9T%F~Vg!As8eE_mFu*r{jQWUr7o@7=hWW*9nAAr*Q3X=S zttl$yLTB50M!MZIY}EfQYXQTzxi?$gxzVpErP8h;*)eoYKUH~YR3IASbH-SR4M9XX zp)RpF2l;DG^n^Slm9Sjl+Dl_`j;OUOc7bjJ{EU|$HeQ@S?$zam3gYimOOrc5z!W&O zf`Dl;>HqubRZ3n|OjAMpU*jrK-42MtMD-_!Uag^}g9j_Hl%o+N?3v@RO-CKmVZ{~% zuYK-Xg)?%j$MN8hd>B@_hRZ`Tf^0Px=?40^FC_)Ya5K8_7q&O~~&b z@4&LQ$4)t>U=gSS<7SNi1f^#gaiA<*y>nJVe+Mo)cTHFYs;@yWB#bjP_f0MgJi(}% zey=jzICYhgSnn_lzfxRJS_HmtQT(18cy_rb03O_cF_^A+Aa;VsUW$Vn&^qWgPnC?f zK7Ud`dLEzUA#DY)6kqAfCLg#S;TAF7qH+e5dl}swShx0UvgI4j%JW{O02=LEg#DK} zNIsIT($r ztsjwHA~T=ARIz^dZaQuM-F?5g53sRkYH(I*4 zl2|s)=oml&cM3hAa!nR=sOlG38Sl7iNSP5S@{1;^K>eGH*FKkr8$;}(%{Q3K_%Fm~ zx8L4BZ`+ku4vQ4RW-duZXic;qm@N&^$kwmR>6JKVD{P&HYwFb2CmKc;8cgjLOWVk& zVYG0y`Tn3NtM|iqRPNBg8|>r2JM7T`x(* z#mhS{yKVgDd@&9>_1bgBGWPW4>M>MzEcb-FaAbv@?!nNqIcfM^Z6!8E z(Oxw@wnK82pUXR!oK2IgavN6}-9SdmRNAXVo0KXxr(X}kj--M*dmL;qg`fA>IQk`Z zOObP;pAW?7XI|J;W?^m=3|sxk5(b>zZq1iPFD)Qg%FZRrvm~_VLmDzzA`wG-e?AkTAHQxJQ6L{Y_8ILzu)o&_k&4$3JG z52Ojf({8ru^)a zf|AH?T?Qe6v?TOq9BL)jV42h?%OY>}r6-%b1DWBXMkK3dknG)48Aa&aDMus1CVvp| zbjFMxiUzyBl>@lNBk9C;y77h#04ZL?8^&&$KBkSka>)k~isa8-m5EOpqtaKw1>do- za|Y%pyK%tedk+~G#w*g&Q%fV#O5?#->PpJYrp%toaQIU1tYcO;p%CnrU=3DV>=ohB zo}=f$-8UCZ($60scs_+D=+b;!I5xbNtnUq+?u9y4DkxbL7vkKuo@;LSMQ9Jn5KT&R zz>FA;8lr26qoIA5)Jzkbe)koCs{YQ&<#aesxzgmgOnd3oH(z72qw`lK1XN(B3qSz> z7W@Z-v#jhb4f_zCjHFHH&?A9=8%A4()5!X z`#_vrHX z7$VoIRwcxE@v;u;#|0Ut*^yzCW|3FQLKxGgt{%>SqW%2C7x`H)N%(#IgJ=}t;yApF zF+w4LKMI6uJoXn~g;H;i6^`O3T4e)983rGiQwE5lOY&&o>4$WDB*ql6_oF}xOv5rN zbV^INaP}B%M@=@t>{L-1c&yFrF*bEL`T)fFL_zgCU>ZTT9-D}gcmh@q2_)_Dh`Lxx zCkc%2PbS%i@P)ny4}2ve3%5y&<;h@~v7DIsIg494k>C)gjxpj8f26J{lk{vJE0|^- zNquOqX8nk*PccDr{~gca+hO#j`J(<7>Bbabd;pm!ojRnAN$|TC~E7#YZT@uo@8B3 zunbj$VQ*34Vz5jzni$Tv18o)16@Yl}nN%I1GJSJ`Tn3B4wPN41beUrN4d~GuII@2onuNxyN#N)x@PH;N_<+MfrHuC&&GC@Xc%mV8Sd(CW#i{+WEB& zf`+}5sUVwD<4hXw!bA5@=1lYml$>~+({iY z&4orX6b`g){FN$7reJ_{YC7lCWUsLXvrS<@?{BcF%pcRwGX6q2#Z&TgZm7bHXRq?9 zOXyUj&3Duoav4@ksj;OdMC0d>MF}jbVg9Smc=Mz4I40pog_#vvs=U@VMMsOu49Ki1 zlx!(diG1a!(K(XeH^ulQXPhJW`4$P7WU!Y$Ma-9(Jtl3n!0TPqa`?;+@U$NJ(lRVX zHP@0J>#j^KVvdZHzIcy4bXK(5ma?CepuI)J!|lUOS+}5wV^`OcQ>ZHR;~3c*s^pwm zD)o7yIMf2eJG94jkm-+XY^V(Tc-4UVXtop!lp|S%oxb>^R{h^w=q6D$<0;(R5>elI zOWs}}kY=gG?0qGJ_eN0~jahRO z2ByQFg>hF*bJs!Z)51VTOrcZ2d^5qRO^D*pfU_?O5T%6hF_%Jb6LPQ+t$?x##2c_> zN4YOY*QBM27a~;Du<6crzc&eN-qGmjoV_PYsdxSc!^hCqXWug$3KC;=ks;JMXR zkYc(xQ10)~D6dbsc5o1=9)!o2xpY+s^jBp>n>61Olq{%A~@O_KHu$aXw)@fikj}-;n6{ zsa{z|W%;5hSvbV(;qu=MiYR#iPF3k0sXb`c&&Q@%Rmvoj={BB#=7|%bu0{L!5nRR% zNXhK8RFYfON{Fxo>X(^gP4HrmboFalxjiIfsA&SR0t^OKwrUiGSu41N(~h%!gYQrm zf1fg1M4Zt-fm4PWc=7p%xJC+iqySEtzqwHKR8TZ9{Sj78`|EtmRwQd{2eBF3R_2z( zWnf|Kej)07i&S^giW#?Fk>kMW(Qb7t63D~Q=h;u}gjTk&uJ)cKSUhnN795jUqngP0 z`Q$fMf3@tT@bPxGy!|DkKP|unll7Xbi;*#Smz;GkK4x4l%%lK1|5+K{;AG^i#ki+O z4b%ivqXrr;fxGJDc})-pBH>wlQU#*OX|o5;zT~1ALYvWsFeIJZ{^VJn&}sIr(^8Qz z=_C_rh8`SNImd0{?m*PbN*^%)=o=6pQ`=Ju%i?}q>L*ts7O5{KIGxZ}4+~yBQicCTyGb&f_n-F)lM+Hn53Fr`PBpqm)e^D=RX0_$$7Gk;(# zq<>tQk75}X0ix$dL`-bEg?7A!;hx%Gb3vU$;b?i@)oH5(zV4COI+vLH8d0!LaOE`a zjxYJWS6TiS32-&wiuuP9)Dua03@tU5!WlCh2Po}r)U)!aT7chOW+~p9Q>3sLm6h0`lAM*eCY%V{?1L;~3%k?#GDxY3D1BFz23#Ol%TlNFi+aNPzO6lb=wruKAO9)C9%PM02wH_IXVUuI}i zIP13%tmw~Czq%?G5}q(mCK5Aor7JfmsW`Y@P;^<_$D5h6otdeXk*nFivuHIvmvvE8zVwwO5}M!TRw&Xvbau6P z&DN4sgV00-ZY4wnKSa!5Tar9b!F8DHejQpQVn80l9i&S5f9u!cvD+MYVDudF@wn== zc-p96KI{H?c}M&r!^z4NQV7RCSQh|E&fhNrnvLUWx&w;>ku*nX)oyHzW{{@Yui@Ql zPaF=9lCZ*BW42Q(XuSv=Tylo2$pWa7w%~4LjJ5kO)(rMbs-A>4@ZLopAQJ2dX2&NF ztt?1*UUpy0_avxo%i1VN7WH1KH>^k8b(pj_poN!{=uU8{+7DXY=^nA;j@Cu(GFqBn zducXQGN~nYlrD)Y=t-;y1}XvCqln?qVQD>mJyFBB?1keQNMd=23`_}A3S-slX~MW( z0oA_A!YgJ~_S_9UVR-s@sw#!B%f9CFiDUhn(FW^HiK5ILI zB;ZUWlF2}14Pp%B(NS*;x6yz{UA{nuuakjM{Xy42+=;W~{5IUycDSpa+w_cWvJiV9 zH@}I`?h}aWb&cGf zXlG0aqsLVwEsM+?sY$Gj)i%qWqBdyN)8g;z0WHHWjQW zKZoO}oaKjwS^cz!BPzu7Jxf!(zyLgdxXMhn64d4dZndM?Jj|NuF5Di5yg1`H)1n*x zKD9)7@(&x&Z1rPvqa=vW$qv$s*S%v0Osw8)%uTwFh=IG!z46UbSE~!~sA3CAg`aMQ z=-eZ&hE7Ypo>Vs=qs4dO;Q5|_Cw*rD!$8yoJ)V9)A6|Rn$yo`AulaOI*DU!&3JxX+ zNJ&T@T>QK1*+kx736Qx4uOvpCGelwvw@qiqF^7j+LcnUpHij}UQ_qgnPF5)<&_-Oc za_D63Yvu;;q-7eo1=KXp4fkFIGpin%^{-O)xEwAd(LLHne17Sz|4Auk!%F{6uz@@D z_mVDGG@RF84@H$g&&mG~Xt-Pa`@__<^3+ws@}IW#NLpu?OPf0pMxi8=nwKYLOO&FL zYm*diTUoGHN=hHS_}xcql*BH^PKQQCkE%pTU1tTRrJPhiHUnZ@9%dUjhH$#i@Oez| z;Dzct-Re=4V2WIVILY;tap$-F()F}Hb^R8qfcteON&uWw7h|BfM#JAv4`CUS%U1+} z!*T^fl+7MaxSxZ~SEj!?=o!I&4!FxVDddaM9Lxy04ZHiNMGc|Ziu0Q=n*H)M=;(_` z>vJ4{=l7e5lut3BJuoG-oo!=(xKjx11_fSVkr7jr?h-Z+;%WTMpFR|lmA7O!mvCx8 zUjRSpPVpP$P1wSoQme1*=}QYPppcsc8v(eRybBGV53=>3J|+ouj92%=LipD~!&1$sr>0*tDwp7l8^F*FIk6qS-?~qo%Pd!~3#=bq1%N?wd`k$$1oPi>&@O^R7OLdur-w^vL=x7fmL! ziW4@n*6>4HK*}n^6Xl{!;p^Qm4hI-Fut2e-3IIiGOnPhHnfYSk>3Zh4Cd^Z8UTrSM zwaYfR|AeyA7i4taGszOeI!A>Exe@_@mwztym*-Fi9AVMvQx9+ zSR`dh!oZ2Iyb;m_b~}5K3eFvL1O^7w1pe90<~w*bM{?`GL?>#+@_&g=Vx%A$-V46& z1-I6Kl}R^|-8-oS=A@=Z;f?17%_XcpuYgf{PuL;c!%i@8t@UZv$D3RA9J5PrE>K%R z-ODP@OFvQ&hr4bNo9YeOv!IMpuZ2xA&z2Jm6VABa(h8BFk5k-1tC<%8GFs^+RrQQL zN_+Dp0JeRs$}j#rG0men*aT05yzc|4oux<0EJ-AxHaS)u4ST5#{S)RGY;1AnN}H%u zPIElRGcZaU6V@Xqc^SH{B4E3EMpHj={9`LOyUfkO~|&TN$*Pb-M%7ZljZdW7aGqh(?enu05A0 z0yV%rW}XU06Atks=F3L_o9+oa?2<8_1(D0I1?7$T>l=X_t5G6P4p#h74Y-^6=4utu z&WCx2PEfV{nI;Z|{_bjizCghRoJRyZOI~Sz#e>2(*j%N_$9MQQkY#<^pf@RuJTI0T zs|jJ)Lu2si=|ihu#g)>{Y8~5=+>afJzZg;OR-d`8`e}om_!SCqG1_W`HdlVH5O%X0 zqu8xZI3ONP-#Eaw;iaT;$)q<@2Lo~u(ROc2FBZvaEX6dNl1#GI0e*H`kJHH}+4_L@ z4(4<2rd`-3l>u;8$v{sTL${oFB&aJ!bJ5OAbc22))fx>;qYFlJ7+k^7w=z^$6SW0` z$96=Mq{zrqOF~>u1U^To`uqT)XI{FlklintmK9a-fB3+eTW9&7XY&`kKuh3!Hesc{ zV-~kCL1oJ1cJ=Ek*coMa73vIBI>y_Z2Ay5b!A`r z{Rj04)&&W&eB$I>XEy$&1eHv<06Q)fRNO)_ zzw-tF33h2qqO9QnTf@PtA&C~NNCg{a6gcOLs_DUl;TiOR)3#F-el@-b5-VmXLh7Mp z*_9H0L?!tHg}y^+@?n)F&~NLTRK{B@s9EK_s|p^dL0yoC`&+M4)*{I8Ly}YXSAF0Q z>3Nyxz4ck>{Bp~9xO|*pY_{ad>!fwFJjk^TI{bB6t2h($<~Qf_`w&0yEw48Uk}FQ% zAcD?ItDA-3Mx%*sn+AAeA=UH~gW0`mUN&q@F-IRx#S4lPD0F95Z=n8G5) zl4{M9_v0YsOq%R}3ikvx4A% z_WHpI$!G1CSjQF$vPYhT)`YzaFL_>$)NPfHz3o9raYeD5ny`oHwtLabbQ`fC?$N4$ zz_Yy1s^xRK{DC^9J`0_lVT>Z{C&Uv-66d4-!VB9Ve)16%3<36nJo35jo||^SJya2E z!__W5HY#8>)b07_$O{TzxTuP;b`cQC{ln;r=pZdXEfCWJVQAE^&N zcb}Z8Q&YafwRlq_DSzvPM!Yuto@w3`=-&+;6oCi|?VSn3y@k1b7aMspISSUVFJFnD zcG7mXoO7q8PPJEqiqIlU6i~Ir^`+yX39A66l=50Iv;37f*r7rCXW>5oTSVXlc>RUv(^K}!hJ^vvBQ>?5nz|y(%BLtz) z9ZK$}FfOiyt^|f|IsGSfSJ5G#Z{&~YF_9+|^)zQ?YEnJLT)g{2050=va-Cng-T2&JX` zg%8Cgg~N#e@%|w<914b9-lVrDFF*qBMy%9S(KO7|1qW`{%t40{u1b}w{=agrH^9zs}SMgpE<{) zO2zdigmt)r;$Y`e+;-6?<#8l8IuZwwXMdpMB=^+@UL*O1XIgtl<{+MkB%xj};*yZ5 z^mOuv(K*2m)&xm1;H!0$-<${Z;_f< zw9%x^YYe3fU#m(c)`X=BtQ8!Vr|m8@J@dw&YiMI-C^qqI+S|4NP#dfN0_Hm7Unn;^R8yvI-*sg_0Y~5y8Zmchak0>0K zvsMDzTJmy-I4EDj#k5pRc+Xh|*6r|41+ClYl85hA&_@l1Gfs=NK)BE`1xOJ_@G;}2 z?Ov3ptux{|@V%8$m=zy}Z0cJhHw0(o-9RcnnwHfOgDD0HKgR3i$EYV`gIpl?FHu-bjdZM?$g^QeeFBkHXRmc;vo6Dv^E%`R|A` zUo8cHW?*JL4}2N>=gj*5R-CB)`Kx01VUJwrMSE)*Kx|B+j6?Wuv?9d66(=V38~;+A zG~6iMbX&TShh^4&oD&>8Uy$=k3%h2lw)$LoWnX3ZTpE1;{DA!Nv^(SoJ?=#LtlFE& z8K@Z!^ha+wYY1ugdZU^=(*maBlxlK!vI1@4-Zf(k=l>iTuk^iR;_u{oObmY;c8}sV z@Z{!!=~AwtnK|NLg4Zylw&4+zt3DN%vR11vxj3n+vRf`UYnA`H{$iy>*kP(&)K2?N zcIq4|cm&*2)Oa!G!A6=0pi7d0R zPQP4=xhg95%5}Oq7(QM5tU(_!9Hi9y~r(t!$ZD=>PwgNFmy4nqHbh> zU6m{g+#@#~rEDH2m`oRw@i>!j#+_Gqxb@y-PV)BDDw@^Ys^S6Ya&#VsHAnbPx>Om1 zIFvp^{M5|He=dJU-==!jA4JSi{(2sMCBm_PyO8O4prtKN(EH6T+*P*O3fo$rSS)leSG`nFWAwxnUmzv|U& z&MkQXqU>~CK>{J%^QI&&YA`-Tp=_}OCzX>;?N#5W>Fh0*CARwjAztmrBemu0WT-YD)L|TfmnOQAd^E%AruuxOoaY z7P5GmeSx-SM6CGS@zK}g?}5j@K)PajsYbd;T*yK4@it!%PCuv|+!Q<0!T_U(oUSHR zr>L(7FKcN5g5wPu$rttKwxRd$X-ZARpFeo62ct;(EnI_2xB7AGGzDYkbTs?@qy>7o z@#E8vNLL^0eS;|vAg=MNkn=e0iNdkg&W9yZ$S5QyzbL01wFzyp31bwv3sl{`BhKna zbP3!8>57_O?Klk-migdslf7tr9_1kfCNR&pyQ{6_;#nSI-y6uBY?-3gt93-d2ndqz z>Z+kByN&KOL=B$7d6-)JVljL~MdT+RIF zSo#00IKlo)aq<-)OHL-2zN9@rhn@IsE&S049?4EQ&k~=SDeYB=p+8YJ1Z#)fIb+G%afT2hf~1m15_+M-2{bWZKU;RZLm8*ddT^P`9Sb9 z*Vqy|^z~Gds^5RqhJ)}k0k<{J&dS0$mt?v5XZXcH&mj0w!a@aPC{SCvb|Kkx*bD2c z&x>^G%b9;Y%C~E2oUvX98Qg({qb2Gv%r~ee3fOoOh7#X4HfHZ$1 zZUO}-y9z+TNt*^2vl<%I1X@rJ;vD23uw?f)2(2RS&DhL_vqC!CeHu^$oo5I$P#*NI z!K#p}d*;Y1I^AA1SRYTySikpN(ku-|uc+;Zi@C=yAFXk{52`0_CvNMNy$h;6O9h{2 zk%z3Fr-bpMcdV=|`2v~2w(P5H0XV6g9YCU%Xu0(b*=JXqKi~jG)L;9BC`p|C*UYPwjx+rmuh{-M5e4riSt(4z} zW7XGQ!rYxMU$z;Wp5gZO2C{4qW7wz3lVPWg&hd!3SY|^~FO2J8Ba7*al1!$Nw%bwD zS)4;TT?P~hn?@;P!&%@~gB(Kypm!v!{fYl#D^mT%R!mn!Y5)PU6^pfwFxjwlhu1NS z{bBArLmP`?``bMk#^SG%a}=;>Ykt{g{~SE|Gm(6j+&BSbE7nEo6=Y>dkA-9nMutIe zJbQ?eh6CA(<#7~-y#4oAm_FDy&&=7h5*7Y;wxYO!aeE~2Yn}qK z6-oZ_q3HhS^hl=s*O1K}is_4i3uc4Te&T2;Zb^l#2-0nP0g8=@53FGX(N12B@?6Es$#uWp_i-y+f zqCW_Z#!@M1tUj1fdEEEZr=Zg0`|3~Q|D*V~qtf6!bym0uO;+?i7jN}mC{qXFGDDfC z)Zn<&`9QhT{SKSe2pLyL)!zT2>>a}^?bdYdif!Arjfz#VZQHKQ*tTsusTdX8R)rN? zm3;Gkd-dLHtv+^le?Rm8&KP5!&vjoX0)~+CO=)Bsc%f)&(FUmYw3FOZa7`MpY}&xP zREN2Yev1uaG26r|wb-T&7g6FK2T;4`^6DjE_gh|&q2spHM@*QGWrF2BX%?z3FhH zz(j&|d8Rh$QM48G`ITL1_Pl_AU#D{86qyg`1zg6Wi)9Gtmgj(L>AbS-a8qRL=`SlM z@A%*SID_V?Jky_3xb@nqm1=i|>byn{1JT1R#>QBIlI^NCz-Hhr!pK?pmba>Xw!MxsE}3Lh)nm}l=yLZ z26Y9YNV$B~-|9jW?FT&pi*y%GYXx75GDpS%>)DW+19S5N&}x#M&2NF5OCOr?XG*g& zuO(W`OY(`9kz<+@j*$FIC~d40dtdx{g8A8j`*yag+k~c9BR+D*0kIPNx@N;Z?om8N z|D5*4mpAf?&_CSyMnv$A$n}jP|0AF38oOQn|ErBG;|L4Hu(K%m?gY zP{7Cmp}J<=;k{*hbYA&$MQGmMF>S47dn5@$Z~UPDyRzX^X1AS2%@D^stSD&3r^M!J z^_70sssjYBe@xq&_cFB(LT|qV$2(%L#!YOf$4zc14|4RF+8o7f2dNkCM}Q6OcG}oW zQQz_sUTj~jCS+umI!!h5Su8_E`+TFT`(~Xk;vc;` zPpUGNse~_?p$|3kacoR>2z_WB|%kw$I9iW!#EDayE_W_0xzXOq2*n zox&EZe-5(IVH@hJv-mu@J$e(ROK{Y)i?6^SQaogU+&QfV{WK;p-Dlqu*b@`PA>XkP zCz>4+%tKS0Owm&%!Si(04Y+=n+9@N;X^?xQOpXvQk5~oOR)z~_%1=>%n`KSN3YQ}% z+p^vvT^D><5~C*GpJ5SHLs$h#F^T-$tvWlBhuX6 zOFG$sFye_zB1uFzl;+FuhvVRI!3lmg0rtw7I6!wgw6evgrk8B?XC%TqqtI9{K)OH$!1FfdBY#fV>-$UH*FYO&ea4z=>z{U zhc_e!-=YCP#3aohV&@r}@jZ2iYGy9yuO3B-8`2K_ft4FpF9WJ4y3qr*RZj#8-2w5g zud!UxFPI)oZy>y~<*~|-WMWYwrbE<5a6N%Wx#kg}DElNbo7p5Q(nL{d zzR>}xdP7Cev|v3!c@|~V6k@+&?~mC1JgZZqX^**i(pV@y4T`3gsu5qfAyB#I*qqS# zsKg|FQ9Ky-FOR?t9xirtOKQnDzuHjxnr8QNSyx=vV&sy*ITT-ElCG%a5xc@^(>6pn zSqPNh02%USF;GUCZI;lI_WIc7a2d;l%N{|mqdY(D?j61D$y2#2c1SA|=4 zw<$uGdxD68=+roS1sH{nP@D%ZxvMOIzi7QGtC|DThKHV<;x!OBKjporFFX8?6Ma8# z9`FKax>)8yoY2tf5#(^wSsiD@Ko4j(SKa{;R{ZQY&~XnB5fRwta=Lxh03I*}oJ1Gp zF~eMv)CmV*626O|f3}4_MH8WFV_;`@10(UC-UqSH))|P`8kh;s$9Y8y$ua7fHO+u{ zO?B^~llbPsY{tV*|6sbd-e$g~O;ih}xN(A3GvQ4A7l*9UP>&-vrOU-5AOG&7gl1Lv z+WW0e&&UO=My)$m?e$obC~|lh6%eoGDDt%;;z7=J|Gq?Vxb{2VQH^RWE(5 zas%Qusu}Np;I$7-ahv#H)fQ$$Y(X&opqGsj`+!WlytZYaD{uhDwc;%a9{8xGB@7k> z$+V19+&oB8F3;{z(dMXQsrD(j3Qt^2`fWRHD3Yap@>3jahMs@KEEMW{%WfTbq9%N7R6k`$R)37aBqr#5q8mAl-nqjnO)pQbh;Q&H9xX(RzG3=0Bom^?kv2MT^>Gx4e{O0qB}f0gTJJl@^mjLYNYQkp2&u9EM}=q z@-2whMkL}tX#YyrXyQJ9vi$Gp+TZ$sf38jYf77)>;h<2;0z2x+f|(af#Lc zi>}4F%HwG`}H8!uwNVSzfNb} zvDohG!#~~hZD1OQo6t#Qh2>7Q2G*gUL*IAAxea2ZkFU#1FW7V{rD{*nTm9&)veAjr z7Oz$5clXyuhi1x1J!OOgPh#5uE#UCcfQ2{xCVPapROs4zZyb`iS8aEaVzmos2$=!&_V}%Zyfba_HW@-yt;ARhMx5KJ*m&22h-P;V zrhrgjcX+c5xq_@q-Zq!%RMMPwLhJ&mhUqBKU2-cW&{4L_TY3nVKR#L!v>^W~SG8@8CVC=yxo<j#Y%f|F9~NpHoq0R!0_{4rMq#vKDvakjefzZ z+K>xhv^GoMX-G%9=Sc5A`C57r;rsvPYf329K)!Z`j(s+o6zK$%2y284nE>!UsdVTA z7=iMVoT+8n2`>#^`pV{mRr!6bj07t9ALHaV+)5lZ;6fM_$r$OhkJn;f#lBiMhxp#?EH7rmpda2x=x)XD67J+6airY4Gx91@>7s*7Ot8 zYL_yUO*@t)yXh_e_UgELB?vCTZhW5_HeInn{aj8AA4;5AKQ+_xcs|*J81Sz7`3^ft z#kaF87gnAljEk7t9WE@DzB4|0Vl($FvJRh(0&~AmY^WYVQKxaJg&CXYIn<(K15m}v z&R9}utZ6h6x?8^}T**7=A~spl+hRTO=UoemNg@W5A6fe|V*KISLGe2EV2Xy?<87z z`9zvCi1he=0)%V2UR8MaR3}B^f5Ej|kE&rJ76{)z*XO!uWHUb+h1Tb}AF+R4tO$?> z#>Fjot+jzC?=Zj86*9dC=d|mR{et++6sLc=qe|b$KRCtiF>Mk11o_vKM;WsKD)4^? z*Zw|v{%dB_#swGXO+o}owdMRLxEAwh41{Z>O6*uE*f9?iMm!WcMrm59lpj^<7E&YA z=~uw&fMfdN5|ZIqw#kmJm1Dnc52Wv(cgF@_1c$)T7TP&to7(O65ic{Jv7g|5je+8l z{$PO4U*ZxKG;v?^eV5IZ6$~_(>P**nk+)>{YMf_fVidt-d1^<$Eu^znTWJf0HHJ3( z6;tYq3;C*oWlNyA1OxnUafvvf;ru;T*mtRp25{f`yJAYdsWQ+`3s`#AeaY9@KnA9< z^ETkwe%uOC-H270UtwJfujcxoTK{+H8RvIYiIxLSk(H*?mOXEmplNV)4XAOH_B20% z*MWEF9AP<>p{;`I`4=@W4xO;8;Myyix@po2)T}DJ(`iFawy?A{+k<>#A2No&v@PLnY#t zbN1J7_;u}W3ZXf>?2s~XW+pWUJQHM%=p5CNJ23byM`xG{_nodPRBg;baG2A3U4X`r zx|A4QCNe~&NU!Q_?CkrN|!!;pyk zqBV~ADk1V(tPl=Pd8+J`HPlFxMZfFo&P_$2Qn15!=fnf-J2Mk*?Z$e3du^dTIkpRg z8Q14a&KB;GJU=bjEbGYh9=kfg7VU=ey)i*(ojXgLRxqr=Cw%eV6U)3dDuR(d zw=-QgQb^MHN!fRKlxTkvMHH|~Stv|7&1{M^fbk*yB{7qhgnMx@KGYZk$qDTNT03DG z^B;9*D>sZk%kzftw@rX`XE%a^;f?686elW}ck~o1$0kN+f_H~+W(-a%k7Ju6Sf^BF ziOvNUH>Ad>T>3HL=t}raz8tsU(H7~iU=3v`aZI8gBBSof<_X^`yg#9@#9X0UuMinc z-VOfs|9N;mv^Nrn)~5bvwDy0cC4Yd@5)6NbM-$F07-`IKaA-&jc-c{53Sll}8Rm*A z>3ncYQJo7@wl(<6X?qI`5^9F^O11ae^5OL>>q>Qm8cMNT$v3>2g`Kx6cE78wsZLu; ztA%6f%N(D}m(Js^1cwo9na%I(B#Y);ux^1qc3BOR9LYn!QF=u_Dr2JFpOj!>UM*^j;GIFeA5cl@H7hPekG)15Pqg|(ykTbAe%#RLazk#49m7eLO}=g7bfwZA4nNEFg!KLt>b!p<7@G?pdn@*@vp>g zp3|NCrWMo*E-6|8<|?aB;M1TW$pud!jgPwe36uuEr7Hk^C2BhQf6l+6rOHkswcAf$ zv~!Ec@hW%F14b?o)it3$U3CYuaJT1gIQ>iZI%7SU%MVtZSED>2+^g3&F6q+RV|NS{ zi9dGItFj)ttJtd4^AFmd33O&?&8iNbo`YhA_7!kHst*LZ>P#TPO*SG=P%X7m`3b)F zL*WW0kG)74-i;$j8Xik?jqK07LUM4ev1E*8=ojdW7~C`nKdl3`!3~@s6651?^e|ki zHKwpa34>+4i2L!ek~i2O4c415B^Z&Ch6^$w1D-XyG_vt6E)ut5YUqRJf(l)5=cl^< zJoLRx;{oOo94o2cE4IodL1MP%h|m*Qzj2TQBqF27XSK=9Q{%%uI}%9%j6`9Hf7F^6 zRO~Mc&p#{gtQJ-0QFha8J*9UrG3ZwnG|x(In5WhbCjvbt5=UB|)NOfEfG*2{aE=Qa z@ve-(SZ%TAT^cUkd9RJSxAh>Sg{*yn?F_kbgl}7+RFoxq#oul0A7F%g`OoHB^wpMj zP3xbhH^w9_2zVRdY)={qZW$`?vbGWqxY~KeI(PA7W<-aEkIR4bs}T^!vrp%T5?A?u z*HTvYaSUjRXQm2UZ0uXH4tJTk%z~ao^zh@@M3uhZ67jSp)TU<*dws?)bg|rB!M_)~ z#tbyE>vylQA&fDgVf#u8UCA?0mk73z?{vQJjfCLpG9oOH<52vT!%m-bb|9brz9SZB zLU!8C9sT840)Njw9#xM!(t(CfzzoiNs4%&EXN{>H*{Xs6yAUEsd*`vt8gkm0hQj)iry)oK?|abI%@*8h zU)0+=A9Tdp>Q6F&ptAPbXP{j_#3qF!LD8*E60wgC=_00f;?`uZSUZC+-d@J=cZj#v z){x`+L&{ddjITbVo2WdV!clgzVyZG^cpNnh^hvx;&tKo%#9Ft9wwjDnb07A}f6m1! z@s}P2@XpcL?mm;_IzrTV!e%{ZnZf1meKg5Ex+HrO#_dAL=ndm^Nv*9=|3Rg^X0`1K zq}Lxd>ykg^qV|f}bf`FyLoJI_d>ULtr9AeAfQxzm28)Sey($(N*Br}yjxqr zhzTB!2d0ZIYI<$0^0>?p4uAbJYIlK%vO?2W$YsSQ-KSZQj?0QgThQO!JtGfqn|XVe zVV3=p?|)ucUIdGNQT=*LMs4C>N4Wir`TEY6^xDDuEGnlA?F6~5h?_>sr&MvymNG7= zsJ2!iQ?C5n-#GqZRI+!$#MM>r(4>=p!Bja;r&}Sfe{6;z(ZMUq+S;=A)+ocRifS+T z)BmQ(xhz?-LgmO~X0J#4^sUO7s1~0Z{%#sf>7WQ+vxd)Qb+7Ik(F40bc|S9t;b0Y1&o{>SLrUxmc~O4bz6zM_AG z>bL5(DJy;zYlgOlOA&`y#HS;8$K5;T7>L zJRo((y~V}EDU6y=)d!FyPusEtYh(l2nn7x-<`dq=z}Bw~w?z0mohnR+JQrD6R|oRUP(F}Mx16#n~!E&Nv&Fq-H|)#@Ljj);^%Nt zd-|O#{$W0woA+662{Xw%w|e%=9eNxSv3e9-5??Uoi+7j#jIPpYi?5_dFx^vRSfX5+ zE_DhkOn+6wtXw3I%c%Mksi{$Xj49pd;+S^KQ=B6rC%7*>VN1+>naclkx3cmaH_UDO z83P1sx6~p#AM~PKq*a_xzG>I;PZHFZx385BI#JyWwA5cPMm!iAXqVaf z@(T=iacWs6|0odQFy_qThwg0)GA`_T1Wz!lyXuX6yI6u&7&U?5f*U5PatfL0DBGw`3qBGhHw`8uY z#VkV;1zWkJ;FESwT!}`H9`sh)uH7IK-BbG=VJX2s0;Hkf9jc~5oaQ-PmX&D>R4$1o zlP%BI zLuQ2%BVRNeQ@B;`Na+sWGtNnej?07!t_&I8?CgkB>F`NzkC5lS*2Ju;;@xn%^0qjU zUrmRi6B<(&G4gUoH*B5aL_A~q%kgehT_vLuNb;SSZ z_5NqJrmmxbrh)!}2qTvc4_mGoDb=h!IAv*5BQ66G!ig)QgDhruVagpkXG)nP_r<4o zZNh};4eqt_Li_8Pc8h}FN!;0)2PKL!bVC)A3!~5XH|NJ~iy5HmXU8AP5m)HsWY`oP zx0JmLM;zKgJYmR;Lw7EIo_Q-$RO)xQp~T3Br{u_w^d;u9qbeCk{lUOc8(d5`S7X+| z-_||AobUF+ji7zS1r+bP(V1$BQeY}%#a^p$)Ax;LUqBJf$^_+PpcTV}SpPI-s@#s; zWC|<~k9ffEEk34MR%^aiAxh-Pry00lrcPO9Lx*v9)6V1>DTnFFUs9)X9jO^vTmm{5 zRYUc?wpb9&fk*bV_G#0*BEtG&qaxFs5`aO;W2Juf9hQV?p3)^SUJC zc$Y~o)n#&cO|nU)e{TIv&F*C3k-gJ=DPuuXAg`k=d}@kwIz;}|GLU)SB||BeC~-Gj zJ>Hi(r?t~WaCia0HLeLX-=Il8sq2xZx6V%#JNV*2lQp)PkXjem>MldfYJFJgHqUEg zL9oE&^)SuVVSH376U(vQH;$ch)?`1{^eefhl3KeWG-4?sf6KqZIaR9%27m@eKrKELBDIX& zZYdodeIltmjZ%w88guUXSE6bd*-Tj`rr3U%jzpIDPP!XI3ig-b3bFxr?Psq`0Cm?b z0Eb27K|ioR*v!`qJqy>}eK()3bf;5{Ykp_l7nF%?9NS@yi4-2hxz##SC>v?HW36Pf z7k~+h+~Y@dN#(IKb~MVDH?{r zz>j#e27eF~Tf-wUra~5>aSaV!k)u?OEX*hf%cm6M8N$f_~7G`I~opw7}_J z{A%Aeaww4;l0e2~cyzRY#PL4wquV9ZL7y;=o*DkH%IIT>fw;MzRf5Z<(B-*5!461< zxK_;&O=D0&nQ6010=r_u9nsXa$Vnnd9YI6A;}+7HagjKaMj#qaaCgc2!jLP}-Ov%g z>z#rKj?9s>K{?=A#e%SnO^e#Yjk1elIHTEG%T7i>$-ZZ9BD=ZoQ*_DMhwk8X4a}J< z&dghf<8C-2)+t4izrst5npsvEX;j@6!$qsRJbS-!tFWj`iwCGbrvmy}bH6KcCFo%c zmHx!+x^al{P19Sgj_wggoh2+d)@T-lckV55&Ss2*GN<&kX!>+`ddqw)! z6LReqVI&mz&~W)519ty$LjG3<_*W?(b~jq0t!8TI{Jb`uowkyw)Q_beO_)N6APKPY zujqgF>oe?upCEX$w*HbAyNJXINCF=n{M%c>oD}Sj&t?`!UfnZ(=DB~~@`OG?uSxvp zSYkE~xTAoLq5HY9aZUzbyN6-LCwQ7f4`{eeW~=^6}@FR ziP>GW42Qc`x=wpF6VH`0znE7sweQdS@42)i$rEtYNa9Nbvy^4B!{sSzUu^E&D%~vc zOtahxs8nl7R&Sf}Cm-@~SS>5J^YUkb7tPFXr|aen*90r21^Hu>0mjTy<4zkSe+TUT zgvd$`YJeX79kAQnQc*`v@phCYZZhR~12RCKR=S{Bp%}oe$(k;lo{g1Q>LIlH6r7)I z4UbY3mA76e4h0vG0YU;9U}z)&$N)nt0dBU?n=eAHaM^qzLVzf$Q~kY^3OWPB*7LXA zybT9r-RWCx(~L7+D%(SRE(=4n9qZ@mzNd1lRn}{;R)lwKKlAG(0A0JaA(Rjt`u4jr z>Wob~|IaH=b2FDOpXZ~y=h!#u0xL(q+iyw*;?CEyH@|mVNq8*z(?>Wg&eq`k@tkkm zGT$*a6ixH@ms)gyCny8zi;;C8Zm@|JUcT;VO68pcBX;C^*78xBoTgF0h#e9zVn=!H z?Gu}(;;60;X{v*;vk`ZsMuY2!Ca37c2XJKw&6|ibJn)p-$CW&1x8TT=aDC~Jb)ISK z`-kOCKqJ;r!9UsR)Ts0(0ZV(E89!fa1C#wN1j)W6}LQN zj+vGxy1^r6?%f5~Jv%Ih1?izD$4Hq+*5X3LQw8LFe_IdGoc^{RKmn}>XnmsAwj!2_ z2gFeV>Nn6_bI}D13+wrQv2K**V+Xw3!`Kf=>4QGi98sj(BT!DM%DussX(RlyH1{u> z>E3<5!~R-Px@2#zMg>vlNh;DEvrV2TEHQX>5@Mm$v06R2KA`Jta(@6r3+W?iBoriB zPMJFI6%xNCJ*i4X>-{{Oa1Q?6X97l9P%a6~>bIc&PZ2xd|MY)udH>c>UBUPyz&@IW z$MOo7kYgT_j1uX zL83IfzU*IVb11v9R62un5%xjw?fdcMp?i9V;iSzaQ@nnLVd3| zRFW3hraivbAmfV_HvwG8RtdADdag&5bcc$c5YU!OdN{76G@pQ3mJQ zJ4BwAGSg4^PafEWyNz6a()O3*0BIHIIFS3>aWL_hJ5u*eZeP6vw!8=|V|&YOnO?vFABK}skqF_@)0GF2TW>USs6t3!wJjgSO@ ztijk*Aje2UAUp%hI58E`+=l=``gnL=ox%>cgs$2!xv!X4R%C!W2Lj90NFzDmyD0U` zw3+>N-H9qpU({MHHvvHEz7QJiWJ_&^bcm`df{(7rBLp5>c07$l za+*{b)@#qrA#%&cb4mh-Jv)S}lV?Hy{Jk)s@Ia4uJ7v)PUDI%|FvOu-2h{Z){0F#p zt+=k{&lLvwYMGGOJmbU?tAyy~8^Y;RPa-1AXT=f8XU&lrX2#iiLtPm9thnr0nrt-9 zZby(pvP*a6Y6xiqg;GQ=3R3}_6_ywUaBJCl{gE~d3Crvc zb7R}sjz&WYqh8{$3!F=z#Ky>-DP89rRuwl1z_#Yx{b zXnddq?-ahWUwbi0$|E(Gbb2et@G!YS$0AyA+Ef*%brS!QxDKeADNbrFS6DRp7V3EE zEf!4k3pM+!PJP?=O=*IpEeSQ1n8aJw@t5JFE&g4dYfzluASwcOt1}|08=0$PN9>ID zNp2-Sy6iSb;WN;*D8l`i4Xea8e*%Mj983C7gsIc37b+v7(Yk5LtdPZ!C6G@t3zwR} zhA?<_-Y|zzb3vPl*+BJrg3-kB6?=?cEh8TL!?OvX-k6M`37KIZbiA~Kc98Qgy>Xw# zrNzmwwA`|xnqQ#3ILpQH%8WMxy8T`{zr~XkF06humT@k&9IytGHO-!hutB#?60-)$ z_)iI6!RksF7L&}Dw@ekDMRrmOgMe^^J*Q|GBCy9Y=(gPka^`LZGOt^SB$wJL%UtYA z$KP+QPB%opwYk(~yI@@QX;3rCO%vW%>a=!@Pt@hKCrR8_@Ik!3Tve?ec9sM+v8(IE znz5DV-uj}dZfwB57kGVOUz^n381+u2`Lz-G)F8cG2%hs^W{Qet6YB;0lxdGM71YA_ z9^F+wpq}BBTWkx5q{BWXdqB1~1wPal7VM*{!)6Z)zPRNOzf{&5iaN%&0lw>L2f1VP zk1+`%xk6^*Hse%tdQ;A=mMn$ZU%)SI&0?sBhiU{%(Xiya@BhI7G(mIdi5(=TD}UKS z09!G^ek$+K)oMn1z$X%=$Q_d5AKpJ+R;@>2=I3;E44-0AW4@mnN;M9y*~%J48GkRow_F)N_66 zK_5~yzClogp|{ldB^ef3u_UDVdT;arz~IwNki4V=5%SOLl<@X57bh5sLjoH1iM2 z*Ozo%71vAl4#5IEL5eGJ=RAw3A79|uT&L;f>#4Jcn6A0~BP<+Cz9f6&54D}RPk>RJ z0$A0*$izX7e?>}OAf%4-%-_5M7^ej|*Mp-2M4wx;EdEqFe{oB5S{OB}CP3j>YSQG2 zdKjj8!Ylb4=Wo#vJu9CQpw+k<_R!5w>nOR7rw{R2uO(&Pns+8N(plT>-H?(h;=+9P9VV$hR4tU=IsC#TpXB~EKRpn=$LhZKce#Rd69-k6|Yl>7)6hFE4<;-#*bs;blJdgcAKv|{P&%k z#lLkd99>wRLP6_;a=(5ZI>r-=FsY^=KAYT?{Qy0#5Jl3CJcGVeaS7~3(3G8Pxp%^w zt5lK-ZHRVBzSHKs$3pq|x8D}>rESPqz<8h`kigLYcTxYpoCFdM&SDO3#{CLM3_!`Jg_? zF#`){H{#-Xn$B^V$>`?m{d|9i*kw5Lss{eP8x~&bD~KV(8-f3Q%o?v0MWXZFYsyfs z9IwBAApvmmbFgPtqw7k=tgcA+(!hmbikCrqPB<3(jG#myENqhinkqSWA+CgItgi#! zeu-;N{WU1EqF_w1+N0-I=kl5VI$OWkY?mX(;ZpYeO;1oR`@@9!myo}QIJPuKv~5?t zlkvf#L`*@em}AGV`X~p{qsU%t`S`Ve$+S-_opg}oyxHoBG}Eq9_36pf8{~~NxQMI= zE0UieRi=EYSx-Zm@kL}Sdl3WuRaBPX~_ahm(K2((FtGA zE%{aeVlUddl?{d%X?w;8M(FKkelVU5&eqh0y2TGHL86#WTI;H3BEWd`yju4vfiLEq z@Dw)p5bl=lou;8VYA{~2Gc{|ZO$^H_k__u8S}oNuD@+samEqmaI_KE z^v}`B&O7Oc0B_m|5vu_i47t`IJ!S87M~3q~6j*A?^%B)LR4s3iA5M?f(dUKFC=vKn z`%5r(wDx8Dlsh`f)}KuZ!V23ck2#+ly3O4`U^Qls+O_?q`}( z{87zM>~jnUTBv#duJ>|B-xK<4z;uBF_TMh1{$s!-{xW(0$AdRU`J2OnFmlefO>Trg z0j)40pwd>d3+VN`kTuQ~%$26j(o1hY^|B!>R@STOUYoeZ_xq4`B}vRz)O)0gDd)3p z&a<94XI}1}Rvf-~LQTOY?}h+60iggtEd5SL;e6L=h!s1~;iEAQkYkZ;X3#7Utm-60 zZ#lg={ph3`;ZXLT(#Wi1-;)zTB2;d~;z<6?+4`BTU`c(Z!-k`QALwnje0L?cCejZ$ z0Bg2q35NK&{~BZ=%QGNdfd)?DR3i}-NKFy z!`>no1g>b9h)y!QgWa5Ix@kp_OZ?>Z>xT*XylS2gaTCtZli(`+z^C4^N}(3B-5uHw zRyC5l4j6YI&W<*7BLrP)yj5mJwtCs!6a6`+<0<`^zBe91BJu{TiSUCBq!R*_+`8gf zxy@(;7(jPLvSvDM?NzBfUeZETI${_>g@??Wcq$ z8k&|(SsS{0oBi4nSzKg}M}C>30u6#S(Oi!-aem_+ai_V48u)ydnlrSy0xg|B*cBBs z{~nN1bln;x1kRTxaK8TA0)c1cZ}BoRWgk_?E}hF@FHl7;-Os|OQ$Q424|M2HaFvkR6-`$O;0`7_Z5 zxEWlCfpD0<*Id#KobT4?`1+w9u42L^sSHcg=u7vOFqG9=R=gu>NhcBV0ywE)B zHLeGpdGx#6`I~^CO!O5qHm7LqOF0p&ha5-fz%fxRG+9MpLyQKeyj@qTk{R{3i50wH z4f{CQpfu}Yf-BDQ+dP~nx(#mg%PeM3xUl7^d|E*kTwycj9?X{ITmq~ghK)vZiPrh_ z{?T?;D8lL}-YRRo!)`!9aoL&pCI9r>Z1D1cThK{j@fmGc)SzKUJBd=sWzDJs zrW>BwqwA{$^g&z269Z`c;wU~5z;kM$(DL5-p$VpOf{}_{o5E==tI?dXWARY%B1rWu z@C9^Hy)7>t|n=a;8=YJ2GXPh?(&pB>6b~)~F>HaoNPEZGHgyioGA76-_U-;d?;$Iz8KdhnKo9-2Zc$lhXH%58oc+Oc~E+L z4`wHB$~Y9PkBkR84|?n=k81e-rqs)S9NtrXP~t@EaxA6Q4l+M8C6uC`s%O5#hYpkx z0ZyothrzJI7-Mzuf0VNvYF@ca`CRlsL#mo_>s8(X4rJzcBjB;sH&~;+60Lb( zPep;7{>BAn%$cErStuJ^A$D(hN1SsQ$!20>9>00iI#$p;@cF`2VxS>eLs zn|?5+FLXB076>GV>32r4L5CX=?CN-;xO zyD|rIi;48Kk_OC2vp{H>A2B@xY>&2vKQek>#svB?DG?qvm`8{H0A+7IT-YfGI9k1ap-8mztZVGt?9G&1*Otx=LFG=~P*5qTF|a9Q(^P3MNBY&%dYyUmD`Z za#d1uxb1ajNH4nUJI%V9Zo*udF12+KgVwUNbMmixMvPuu#gpE$Z~R^R66;s= zr$YifhUG2IF#Lp0y;NONkosH+M)WBj1@;dymevaqF{=GEgGx0IIXliwzi6);&uxFY zE5D}(8)ZGIT;PBYIu9XSd64cQQI9CTz7Jw4(YN5XuX;G`?#~XRh`xYG(_LHDsuNnt zCsN0i%+#+Y0QQ}X%jXtGFmJ@Ejf#cV!+5eN>}JJEV&$WmGUsdEO{_hx?8Z=a(QC9C z%qKGKLI+=3XB?R|Jm87S92<#3^vl7dx$=jvf^RY6s-0jDbvU&a*9 z>XsrmUJn;!lqsogm@c?+7;AI^uKgjYP4pM#A;qom@CP%ExEuHcwQX}`8BI+TN2If+ zLKQy$AmsR}oLMVt2Bg=7MtL55LZ13s(ek58$`p!qweqMc;r{k?I}^=SJkvb%Rj6a> znoY2P{Mv7_&Gc^W+=BEe!=hvK!;#2C)rH3g(!;HS6z$J5qJRB*uua*Q(}VQo%N)sn zukrb3>sX4WjiKs_Q|?@?t40+aG8||`nWrpW1Vl6fv{oxJDke0HjTt@;nr};|ww~ri zipQ04eNL%UL{7VV=FeE2g-$0+yhQd`43l7VlYD8rGWo2dxgqH+UJ|6^ImuUlu1l`t zTJ<;3L?WM=3?3KrY5tjPzdy%$-BMy%&%tpY0}7(|W$X@9PC)$mqzcPFz%$RVPyCzJo+2-5j~%V-KS3alfwx; zE-0Lo;g+z!>Ja_OiTmiuZk-c)KCw7#7XA61z6=xRvm(27;(4a=!NB*vj`bre?nCeA z=k?t8@n-|UR|lL=$)eAtlh60&h@SnTyoqN{!DtnGQ_QXH*>*9LxS#qlvJD}?^&*qJ zpNIaBF5(a^4f_2l{E~t$9JQ7Ch`Ad5Ah}F=5Vsjn5bfFrA^vAa;q;fK#H7b7J_1>9YsR17tC2>5(> zk=mY~+K>n@Pppu_GL9bbSz>Vt(m@(y(-Y(Z>-lZP4?>e<(&gZ8S9F5xh%+{}_~J*( ztIFyGkg%YU?meC^r^VGD?hlJ-oATxRT=WWPMj}H7O2NErTv%#` zb4QUI#STq|qpg==epTnOCy!=nS&$3lcB~J_sf3HKy*yNBJ;#p4=$Gh$n@eUlA+sh3 zj|_$x9EOsQ&I+)O#1~2JE2&=#@b{)5FT=Cn{B zV!)-Eaf7glj*`}H{jyc>2$(NoIrw-3WUED4ee@S*N`5O7I#2{x^BY_e;I-g3iu2Hl zC@u;FSn2s0X>y=PvFGK~$Cpb)S5!F0iRtaNx)Pe3J|YnGYo(vLV7ISsURdeoPUF~` zDO(8{mbGZ=3^5>xCnGn(`u$-NxytQf#n{QK|4Bc>YrchaS6jiIFp_C*VrXwZ}$VJI1GFQy{rfD|)m{CEi-!$8yR#JI6b#Ox;s6`y;X^ z+$TU1wvQaFm2QT^w)`XW(Tcj+XN0NkfHqQ{8}$nqCn&sEeFL`wmsBKG@rokf1icz# zU8Pj!Fb98G{8mFc4u>)KH|xph<7a<0lt!r`Im)Jp%&25unqMfQ(2$PmY++Uv7(@y@ zxI4kJ$C*$X_IE83#;16*DohkzJ)Z373-d&+DiPspW@9BqWlU)ZBGfDTq-JOL!~&Fg zOlMswZKqv1I=`AkkJVqH(N2BYM4i?Ab;&%=Tzf(sLa-9gNas3ofPWDYNR}T$l*%KP zxV{R!hRah~s+R2(?ksl%d7LH_>)Pe;dd#WtZYdz0xMunr9BlFKP(G=u^{C6_`hdn^ zc(`th7WE-)(6}|F#tzwniEXDdkPBjczmAtzDhN6y0Ew;qCPJpDOZ@EuotZL!CWZ7m zy)WzT-83dyO)%SuW1-Nu>0(Gg$hMF}GbK`%PVsovjZEFQ z-G0)Mw1~J|CCzCoJSV{G$vJNNtsM(-U(hvHs{vYCX^+hUbESxVn$<7OCXycQQh9lH ztkfyy>9_BXXZJAH@G1D>3BfEFdQQrB1pQStA*(zn;t_eo^bJ+JoSE>}`VtU2g2HZR^rd1D$W1|F0Zdh`z4i0#z6#dUyav1qi zk|1Rp=fAkiX--E}FJWkXH40NnPErg**V!IiI{ zOr;{q_F!b>=iwo`7ZVL7>*v%J+U%mU!+N)!j3NtbmGB$gA!GiR=cIF_kPL2Q>!=#z z#HHn?e0sPPTinZ}IWtq3tS%YzmUeKdRGV*&r0&$x@~rRId3!c{%4#Y1H+>GDm?}!R z(DLxr1FBbv*Z$$;>BiA+cpI$V8YsvokAc#k&)m*1Pls-}JK4%KT=^z5y1Z~_b^EjX zbvfhNBh1_dBX00?h<@t37etZk?(Cd*Twg-dSz}!p^Jfo^{A7IEw6s<8wzf<$fBC9+ z+NH!#P>x$$fQ4udZImaRbHwqTnTPFkJ3;)S&0r~_qNV*cY;W!+Z+{HoY=YhTNVyaQ z@}7W?acSq-7ha;f$;jioDNXi%2AQoV&Zftx@rz!9_jfqS<_g9{l<~23526vEsle>@ zD@Sk7YLWR%%V+Ob>r7{wU)}!miA~I4MKEuIG^@1A)@OB{&E3#>#tI|G>vi3aMn9sV zK|vhSSuJZ2zl%1cZMl6mX!}1HN^A}$`w!jPNjR&TeZ5I@P~R-2@tBptEi4IUjK`b^ ziCQy0EeanS=n9VgBA5v^a=Y8W&z%cZgf-Upv>z6#I5o#&em!z6YEp4T<@}f?HB&A> zdSbi-@vQNMfRxFLITZ>o3)DAPMi8t|^y=pd5^AB&sASuGow52c>b%bh2`R@L2QQ2{ zs}GYo#l2%@x9kJ-&nFr|eQzPVV#(8$l#>zNxp$D=i3Wcj&s_eP+@F;QNxyQ6=P@0M z_@1`eq|F#gTDEECwZ}nwN~LMPXTImAjeQRt&jQ6rDsHpv7dE>i2bO4g2V`M{90`c( z&+ug8RWC92$f+G$<{ncTv>?q|Rl@X0-I?lNZ19T%uBD0cyxtgyk8=yh^Xa`g^NL>j z*BjTB`$h7JXW9qR8Pbc-T$&ZBv49Rc7gaW!2F}bV2dMjX@w@UQbw#Y2qOHCiiJ--A zDC=NlmmxP#eeQ3D3HsiNj|)+*eA=1*W`{qFl2EiI;ufF`y)>yzt|oAN4CFU-%zFVD zQ1iKENXADYlX3G7#_F(RWhbK0fd7xMcZ$yR+qS(ctgvF+wq3Dp+qRR6-q^Nnqmoo? z+qUgWzW0CDZs(k}Py5!sd+wf#*~Vyd^w~%MF^~9%<@_R*^|UbbDm^DkJ%cKP@qIM( zg3*Ug+{6@hf>@|&mr+$#;fYuLEklQ=FzmjmAJ6~L&y^;g^o`eIuHKy^cvMUF#^}`Y zuD4bdMbQ4Hz}+JPUB85^5&5WW<~MA@kHGnST1mDC)&c?T1cMkum;Fyv(gMYavR$dF zDG42J8b=}9G&BxyN2D;8#sSr!R~9Mw&Y7QNj-8Z~59vJ(grG>b9JwDT0f&}55L!$~ zW#ivyC_pQjd(OgpGT`dmi0kOVs&b(g%zpvGe=>rx%7JGv|GXD55Qc=3?G(ZJu3sP< zbZqIo1CU$`ET&6z+4t-?RT66lQwZF-aat&pTF472<599!E9|Zk4_OUcYk+7zG*d}+ zTS(!nvuB|7qBt89R7o*ZRloYa2&UnQNbZ5*^Y@CDgim;y_}k_I#=MQp`(7a1c&5NN23|HD4DW)OLbw{gLkKH)x%o-52i9Dh zP8r*GgU3AX1U}K$2X2{-13>cZlt_Lc<}+22y%UQ5dxgRgk+ zpExp0p*O*#v?71M!%~`Hhvj@wk6e#%CF0DHQ#xqM!TJ*rfX*vcs zodw5@>seDppGhxI)pfc_bWZM}s&pV<%_&XlS{WjHg~R^nK${>J*uXy$^kX zK9dfe9xemfra**}FG%o9OCjV`&r{fpQd*2fc7h^1po|y+!`u#<5D4G!2j9KJ6ndk| zy3+)IEJb4WkGf@Z@Xnr^!Fh1(5X#NVouz0G=I`ZyLsg$~))CGKKioozi((x9}-qcv&94XvaSd>=4B0!?$T1UDo+G5 zRhfSj&EJp}?LwG@N(p=P1*sO$jWp-vjWzqqp2}#ivL;$|MNhI=F7YLrZQEthp@R(W ztNQ2{9zH;$DK>7><`GeWqhCxGJ^%3IVKpI4+ol zlG|gLJ8B9MjfbF7W?Hdj%rM64V0|~~S!xU!i)ZbS25-2&{JvajpcE0;@a#FIzn|NE z)v8(LUncGEY{Xy+Gw?Es+#@nz+a=Aqi~`I?7mZkw_L#LqCpkZ?tdt6SwBx}*$0;3n zuwYVV*~AcMMIOl2{zO(@!>Ij<-4rN-e?WKp3#C2~EBF^>O;SJ=(h}?t67xp%4aH%l z#H`k!MR`(BRctMjX-;n-5wjdFTfaO0=?Lo-_LTBf`N5-h9%KG;^m}r7*j2i`urex_ z^gAC0t4e)i)0T5@EjiEyV`?==Yg?BQ!izz@P&6C*!)hkTn~3*B)DZz8r9de3`?!N- z$oFDbDEZkNO{?UXW4+C}_NeU|N=ZJbXD*^LLm`_qm z+?)|;H4d1iVVdMN{hg?y6=^i7xrD#C{+z(f@&iPIS&60Idm%U9vKe8oU5WCf#q(B| zx%zK&_2=3@{$F4{bT%58f~Mb;9b{z?2t&2e?e=5Be}j$eqoA;$!<3=klrQCKzTh@O zOq@>gq3CO14s?bX`dIGKpumE@dJAHj+=FtJHc=VSiougHY9I(TudS^rjRH2T=9qP2 zc~>`*B=erXi;;Na7J&|WCB4){6+dgVJMR3}uFgw`$k&!mb=$jG#f`U8_XM4va@vvr9gFSpQRF+nfA*Sh|OtGdSy6r-}k1O(bqzq$`8i-rU^3vxzSb zzeD>PLC1H#;MLM65tOU)-b&~zteZI6YcsvR*3wgUFMPdP%wo>Yu%E^YrIK~kPUo8b zBje|D-W`qUzNR|Wcx1XqzsqdwUG+wSYB;k^;i-o>8 zeVPE{X_PdUKr;$mBs=fB?I)c3Cb8x$LwWr0K!xyEk*|wneJ3RDCNF(P}#$WCyA z^HaA|KK* zaW193u%o+5ODW&Lt?cK)g^sCxig%%$+makAt*_GfaX2P1jao^=4pvD9usMm4v`UIp zVQJyWW05sBvd@mcN;T}e-t|BJ40(3RITt1oMz#SQHN`ix*6zeP`GcihE?_Mj)5V{L z@aSNyfJq0TZq?bPoO|YN^$h4vqVfY;DH(T_`V2w72a2brKfAqe^u zcZ3`?^kZRQ+Guuj0?BBV&05ODAVM934gI^$Fm*HFN~s01W}t-0&sJwUjn6k+3;iB{ zh_(acNty&IcKG>~ZkX^XU%(Mac~x$x_zUyg)p+2J-Af#34INmN746DBl@n41vmKS4 zb+2F=_%Zr1(2zd{V!2~lXd7yGq#c{SsMrX1X&`{~aOiF77r9_h)NSb(!Lq+)c^HlW zE1c|edD@O68m&a zsWpPEwdxR^cve~qtDDtVTY{~H&#X z5Q1nTQ6*X{rSBw+Eh6G`Z?%|B`1xrYpZFl>NW*nEnF=wtV2W-hIy@+n$F_e_^*r;3 zRl3aNAzeG!3u=LWuL>F)$Xh1rc1{PKQI<+c6Rsk2J;V;4KO@xU)tcBVD!`5zcROb{ zxWv0fSeH@;(HDrO!xlR8Tw(xqt+ab<2EqIhJ|$YoSY3`W=aIFgl7TfT+f`{0Yk|}g z-hhZWJkCl@1~M?Sk5lk2tt9jc9r%z6J~5DMpzZ??@}}|yAa6lNs3F4a5rC2%5>7r; z(f3D^jYZN+{{V}3Wu2}Sva_cWI;AnfkQKeq>(BYN0^?~p_$?rD%p-z-1?QPrCG*vQ{?4COl0`pUBl zJLhAGhll#vomAHqF*Vwq5Y%{(7xuT={?-*Y?V4`S04elMZMGSJ0osO-K^e#&Ash^r zMjm4pmR*^2XFc!}+c*smprRoKu?m zRb{>2vqoCk`j_KS(sUnR?1M%&7G7|Xrm;fwmXU*Ff9F?!4r&s zRn0t5^a6LQF1)aLCJiobYo=DOdCaFx)j!X3x4*tQ;Iu?UYDnziZNvj%tJf7_yL=Fl z)O#9neF+h!oeYC$R9lJi1S5v3y;`6c3YdeT)Ox*M$f7osD=G>~OyLa?&^G14ZM6p4 zy%cb_rsMU2|G>nLh=i+lmLU2yR-Fd^9{I_7sOmLJCQ|=~Y$b4-PHs@Af)u2NbOpIa)<~T?!I67(>bHn z+Pe=5s6M8dOQ_J!?+VTr4op31YxqxG8qG7Me3z~#W>Iy_%pr%bbwAp zB}&EIua!J2wLUoJ~L>zluUa6|Pe-=*=X-o^2$+~x6M$Q|*6U(ka6 zDn(*l5PZ+pdgfi0_6}SgHRra89&7>-7*OVQUG^EJ;ncvvMfM}vq0$*H=O!jnkX9dX z_R*|tJclr#bWntfW9tpO0xHc31fLtmdg3V5d9e#M=vH@ZkIj2ajHM{9%uHM=t6`BQiDP4yI{C z{2k?$v5xj|57`XbGiW6fbtQ8kY~)u*?^vn!fZ4#636Sc7d&{#*qf|i3?-~`JZ7g$D zcl$dujeEeQk|EhX1izH$B zDe67ZJ-cVh{rXFqS@IJ3X>Omm#0TvE8Lz%+B=en4XwT?^eEX&dw1*J>hwGfY_`mvw ztD>xYAMnk7T)n>f7iF1GSs7Dac(i#GxzvNwVcu@{PdaNU>AXYtQ_=A6{=WpqIJLvb z3BA)+eeaLDRyS*}W97e@bE?Gm03vYjsM+hu+}B}XU?Skcop)ir0*SP$n=y@+^_Gn! zGW(jZg8bI8`R`hUC6*qzH!N@S_;CPmz<48zH_f-;HC?%*V zPbrPXduw9&R9r6hf92Qt@B_vY_7<)qVn@hW1R#T!;5q27z|~&7-~kH+moe{rfq2s; z<`wKFs#wx?_u)k1J$sW1f?0l2HBf1tKL*jbv68S3Uv7C;tToiA$c5^lq6_%R6!hRB zlq9Az7{c-LtFk+Zm<3$|SH`nMxROTXE&m{6Gi$(RHxl{(-@)|%91VHDfwckfXf%OG zL;N3)hKccChvWOd52sn>C#s4FDjOtgz0;6Ife3@@59}*g6%A!AP`W?T3{t!=U>}M@ zt05MkRtPM=?YBBP8vnldcz;*u?T*1zb2W)utOvMyuFb^F*Nr0bpw-+=WA+t01ra%& zHroP<$aLP>PuMPd`Ruuai|A?xvmptfJaYq){?Gv6AjWZAWVVj0YiyrHhixGHZbPiK z9?!=llLf6kyZm;PhXqHy4oK{{H!joI7zop?Jn@afU{bk@t{FuCPQgOby-SS{xg0vt zT0*kYS^)i9LW2D*0eGnJ7e=u}U)xb|g92kH)3E%cr3)97aJMbotZgVOXYkf=;RI{& z&%c`@_g#y`i&PSUkp-)d-vMwS_wV^(lz*9CLwXLRP%%!;l+2O}Lu(DDE2@;Es2D>z zMU92>I*Y)OCFC-v!|@7m1-t|;qHZwbOR>T`6Gr4Lwvio+>Z-ElBmUe${GZbCRZr$i z9gk?q2io@ffScd{@qE-pmHu@;)oQl?pi2DgKv(ND3FpwY){JHfh^S}{nx*2@!=vHz z-~$(yGfi;NIewp;^-C{S`1kwoaAZ055wgVOe(c0L@wz865hT z@>0034W~<^re}O2e_qm}M{mYvo@frgh9#lRWoP3kjxpUlN$;-Cz!X(PSDNRPdF;lv zxCCp}QiW4vS8*DBxkv$>F#n?&HwfDneZo1lGShT@26_LKuX|cGC=-HhpSweoP}G0a z72GY8MC^qF`-d8WbH?Railq5whh?fRU|oHOY2=ui3|bR%kotE~3ehgHDEWGy8p9gE zN&_OBIQ~dHqcDY6v$;Zu-leREaj|`@Iq-;{D@2w?lik^RqlF3|wa|j@m#I3(LX)18 zBc`%(2dsb{y5@+rSF96r&8UXFMUU!)SY@G)Qj+<*o&!W+~%wbYsscM?S`k-y09?h zsf20v$$~67%&a>(!9ls%(S%qYlKgO-u}r4SI)_$tg(BNrJsoFVTnz?;&k#TKXz@NJ zW`ao^t(w&sA$lenXciE>&_o6f=F%Tt(xy@ErX{<4Ca1z1uT$u>4`GDlQhmnddR|F1 zx(Ifa*4L$WJfJVq=9yZBlB7Fh+BvYqQp%Pk8Wt%t)ihEp9@Eya=JcdTZ7X+Cf3_<_&iP^B@7u5&< zM|e&1JDLkY?Vw+f3t(po+6(dkMyRQBMn&+N>dnVs*ra9K7_AHmGeBe?+>1$)Vjgn{ zF1uT>4x#QB2cHp>*DDseQ7y?C680eHb+m0La@&P z++QAZsOteEv`&@YZ}lwQuy58&7+nD2i8;@}0v@$Y>6f;9AL=t;@4sWidUuc}q%)i=8!rrV~wg)>o7_4ftt2=J{RbMRwkJCNgH7(Iav>qt7B8z7RVt#ePj}wjPH@ zl|eqoNPhwIi1}}g>y>jMZWC>ME*<}Y_eNdYCRDnJF{()gtloNUc#1tYdv$c08_f+C zeV0fA>KWYWeBW6(@?kAkfV>hJ^2#CHqOxShZ@TfiEPx~#<(+=8wMTv}k|O0jaBP6> zSB_Gr+j4u%FMAybyJTRDN}d2r{_QE&I(kD}dTMLNUuiD>M5Y@jPR{D74tG_4(mH2# z*5kI~j9o^vv0CYC0eHV8?8?-F_OF8Y@`%X}Q`)%aFROdbSiE&`0v=M%7lMz|-GmWv zs((+B=nnmr5{5Q~(6=tz!GYg43N`d=vtp+GA>P(u2UF=MgMXcF2?5N~sTkyw_kaOL zNbvb7+!O+G9rSyru{p1-%L>TnYo$q8CwAZg>8bj-fP01$jQ$Xe9Ec9voYzXIT9*JD z;TgL0F;qzWc$*Sfx>x!|p6!sNOcgiJ)%!bF%tp#unASiEpTg49%nf_w3DdRk34o63 zQ_=Mk*cmrSsl8!`01-fOjGealz(rXoj)sm*b6gz^C38#@htBeHCudv z?Fz|05VgQgj5}FS6zfeEGj8(&@wXo`0rX1_hA;rI3zlmTN8 zz`zlKge@LXJ4^Wlp3%cKj%^el;0<&Cdr?w>hPnh-1Q%Gy#rlJQOR^es@S%lk$_?#m zJ7X&CU}i6;%bJ3Y*p&&{pDki~k|ohSeUi!Z?8&jNPGPTEz&I+ISfL-d?J*Pz&#DcM z`IpIVh^atNz!i*Qw{f-RG}Qz^9gXydE~2)01JD?|MU3 z0Cktn2rl@0?m&&1x-#MZ1b`_#2Ww|ZAsCCT1f`eq@VUg9MsEdo&8fvO4)>dbbiXY; zRlrP?sdlnvz>)}@#YV?)u%@Y1XxAmU-Tdr$1okRrQHC~{8bQUH z6p1Kk<}{wAAe^~VI%!&q7318tG+cNyN()Ay;5byZ4wxD4lPt%%2jL1wp|f~Ea-k`` zMFVoUdB)`Rz#C)C)h%q}g`9Y56J`g}(G#)KxzQUZ{7lxkghNhLX@}V_dApcWJoMH^ zGsp%DTg*D_=$?rMSq7bg(p3hVeeA(S-_GgJ1Oroa?@r1;(=?arJ_(qAr?gv}(NL#l zSAt$r(b$hi9M2tyko3*9(9(LlxgB}Xn?dK5;!aOyjHjF>-UtrXQqD3??NkAH42y2b zBb|g^Sv%2&AlV^0V${f@T5o?uE?Q_A{-L?~_@WBzIVIOb<6PGW_H1#Sa)F?yZLirE zyRu~(fm)AhKXw(d%dDBmL>i9?zQx~Kz7fPk#)4mEbjnL4=zC5w{5Lj4-$2FZa_62!*c z40=h=M2B;Nf1QZEMO}7~{dVk-qf!Q0fSN2C;A9IV1z7(@qoaEPtESoiLs-Y{=<#C+ zEGR$#3H3KhfUp``e&})#aSc59Z@&-dzwa~bkYEu!h%Qev=*y7Jo5{E0G^J~MjqfKb zE5dE9jS)4o)E$*4yEPHYw>|u8;~la(6&^@zJnh~yI49U0*Mm~jbY5r$_GP?WBz6l4aS(q9iy4hI{AQ5xs+XMwzx;z<203HU4pLAnp3~KQ1HR8rO zsJm6;llyqESvOSUmFvrlB!w~ThU;jbuUUO!y|LBms7K$ix$8=H`FIs>6$ZJ5UFO_u z{3D36u(ga8%r@zIXObK7>QR&K`m_&Ui5AOzSXh(@!TSk}`73|{UaIHnehjZEMyWSg=PQw+0G_{CH50ikRwsx8< zI$<+JVEz@CAJv#UvqHtY!RDM zTm|eks>TANpk5334IK0FS3JWv=JX&YKnvQsQyYQDBy#5V{$NNC6NPqL4zc`N1dd0E zMxCkr$>Z8qocdW?@Hn;K%qPqSHBerxcsgX`A44nw;>A!0&_b^&ob=je0rFkO68 zcF~XTH4zOlr8g=nE-z3n7o&&BPp&%sg#C|SPeznU?x^RaF6(R!~_$GdVm?qc3P;C-Z=^ zE4)yOKzcS;zc!u6RX6TSl0ePeFbT*j24rGw=F)RGvxRw+H~^tnZ-yus zM5~bPNLMFcRY0xh`%9b~`FnQLNE#nm7v*hEmk6bjsD~q|4>(z{0HUT5mgxhOB7md} z?$6E|G@3D~{kRBnJc?=bCyNw!mO{)7zL?M}3ZJE@H*}ZSSHx7)oSRT_E<=ev!_xq> za_vmBU)a@F4z*WkEtSSdy3n>ld+MyFyb}mt6FmvQ z`=rp`7Pm5K%{R&z-q|Bev?|dAERz+&{Wgr3w?y+MnAc{Gb_XhMc$)Q=RAT*sjzFA) zzt1(08ZsdE6qWc7j)Y;DLhL>WfC)6jCpd11vKnO6P658}pcKsbfkudE+R`Ut+DcKN zGKgM&{~&hPnVY6*`y!`A$j4C!%ETh|ZFV=Fnx5ZYYv`qZsC<~xuInaInn1Bu->q;NQodK9XPH8o;p!447BN^dEG$|J|tm-@vMCWnf^HA9{ig=Z{xa ztvaRowKecLy?8C_pJTC*I5<7nCM5(Mqf_;4?UAXlhtn1MfooXj<^pfux%e}S*n~2$ zOaZ>`+qu*47v7lypKp&G-)J}+n4*&%eE_Pk&$d1>-_g74Zf5#Y*=egt<5`?0hq${N z>I1}&+AK+8=x~%*TI!5<>wg;QzH_f3V#=DwfE)1$szS@4H`hLCKQ_Q1n6`^e$}p9U zti{7Mb3G^3pJ;GEWCaqG*VR;LiB&j1sJ1C5PL>or$}vIlsk56LYP7>&wjZX{h_1Ou z23*@_dvLrLE~lwvS48ymKfJb8L=}0jSS>7QnT5&zOoBlF?!|l5K*#qMMM>=I0FqJl zk@rzyP1rbrNHO~Uq6-O~wExGM1tyt!(EH-0W zGd+__TJu+o2Dy#93F}=~VqQqE2XAUzp!mlag9;gQbn*&JV9GB#8e5_PRI+?1*Y=RHuXS#;+EBc+k>x-V~WC4u_Z5 zULbNeS5=4|yA$7U_B66Er#HMF2=>*Mtn4F)R{T}8j$3Yt?^`< z-K8U?Up3leg6i@Nx(dPxyYAc3zt8AlnbXibfmCy^nFV;o31Qryctn<`FV2f%$QJw< zn1niNBj=Wi9@ojP0jt>&mNFa)(IbD6e;cIYNEP?*9f9&2KGAzJAJDrr-M)~eaK@=M zBTt$BZp?%if^1)A9MthtZ|L%FJ&`; zh1c>oc|^RoFhW!OzK_!VwI8K7z$PYl|7Bd%HbpYIWM6lxDzk&KAre)uxzSgeosrp9 zGjYZ43&FfGOqp8JvU2KFJC9p@D^E+;%JgbQD4+Tp|C*Ux<5(Xnd{2(zj^2{v4j-M; zcCF-vgd?{8&woQpSYP)U#Q>#OTlD|@>iMrS{g*nCtzqq=qK@`C)p<2>04XFyDeNgj zJwpmkt^vob0JCc-EbIZUXjR^ou0x(WHSYPdY*EY7c2K=sqsWRtO_pE|poDB8Wmz4l zcG;i8UnQOT=cCYnp?RTMXAp>vW|2vdq|~j-@}365q=9g(=hr7JJ-@HjV-VUQZxE52 zL&6Nl-#+gVxK9vV8$_WzY(8vZS{rD=fZuFBvi+Zg-7)qSFC_qbh+EqY<$zO8g#hnc zJTBaYn@yO79e()GqNC@7gF3+b;r+YP(|2sYlVLDH@Y_5m7GdbCNGj>ZiZLuxXnKdP~tpum!(drPHJ{|LcyVDE30GhS=RgZxY%zAv5 zg!;XBI}&>lDbo~anHH%kSM&r>)yv;en_{GfMSJm2F4S`QRK-G-^C2pFc^x7)vK=$Y zj8rGucC(pipjU4IThN@vQ@ak%_D!jDybzPr^i3-T1?o~`F{8}*=7SB*hXZfgAq@OW zH}zhrgX$zFvqihp*x?H{W6=1wTN0fmg((4dIkA)IRiZF99VWM72p#fgd#uKKq~EJD zuHpxFA)U!`>E)ax;Z=t|w)PVfBWzlaTIDhUPxQGxcmW2AWTl3`?%UECj_j;_%l~A* zRu%uMRbXa+0`;59%9t@aHNo&j3QPHJ2p_6OB8N$SCn#Jr(na(RL46PDAY{m zm+eas%czz&A5j|RLnQ709JQo7&Qezs7ePmWGZbN37@|(LXQsYXXJq^bj)ab; zGl~pyHl5zY_=sCN2X51_Q&dT1Sjk6ccu^yNA0_Ne18~T_1Wci)5ZtL0Tc2kNLs8l- z>PbN8w*kGUFxzYY;cMxkHjKy)OWRf^WO<(9jbu6Kik&m>2s3@o8I8f(S+;-dAvw%2 zLxYITG05!~&H|b!hVO`HeoKS*XK0-C9C}1%*#1VFD+@7==@Sbht(7X1d|~t^2j7fV zM(HSkLNs{Qyzex77o1~~E|TA5JKtiW5Ny}*PV&qMS_trWQ4{4KX1 zgM#uADy$BB#+6ojvK}_JhdMRxxt4m;m0vP(!b+VDb&eiq%Muxx zJ|GaTx5V}FQ`}Lc5d9b==yy_M8-;45g)OohL4Q-`A67_0(`{W4YVdzc#d<~Qgsx_) z*EIHfDLn%A|?MTt2SvI8ZSX>W*yB|7cVa9UG?ZAt=HNT*n zp!zZ;!CVp>xN?;9U5Gl7vWn+UMm?_^xwl-uxY?vn}#?Wq<+(Z0V(7$MILYVD~6M8`Z(jaL__bXbO-jY_V5H-cQAuEIbD3tA1g)ONkKSOxkq+nVcl(c0Sf%VaZdsY5 zb82Irp*6ykuYzk!JQrbh`@3Rzs758iKFmiLm3Q!|Z<13iQ@A{%syOC$BI7Np*Gw#F z%$^!P_f7^Z@UO+c=fbbYktqw;U0Tr)}Vmws;){-8^dh_D?mAGLf>2G}D=;*(xJ z+BBRKWwNWjA7@^tgLrL7xg={GT%7CdaBGZRD}NuF?I7a`yUpz}#C4 zTc@5FR<6qHD=uaD0UosxFq9vq0oI^|Kh%ME+dCZ_{_Aa7Ycu8mpy@u+xo4 z@+_|mjByyybJ8%uWM{M!YJ?ue!WCnJo5|EJZV)*H(4moTDa0j04Db=D!lhAmrVM{HR`WeQs zE58yduy6Mvy4#)zr4x#+DuqpEtr5thsR!L)n05b+P9`)&_oE{!M`FV_P27iUh#BlX z602=;va<+lAqR|zenRKudBL~!zl>xS-zKJ)zK#_SQm)e5Z)zwhEwIIU4vB?J-) zUu4rFM;zOzB{=K%t1j~&Lz?GZYUD2f)#phB(Zo7`*z2PDkGyWj zZ}4C{8Unu{6l1BsP19COmV8m>g9vs*-}z6vl~JIcI4|n(>yp{rApmaW{k|~%d}fBf zBJ{m5APjtYFm9Q{Ob4~HT_E~xkm@i7<1p$i zMNAfjx|uSyX)-2PU|NJ%(*{k`j3A&A3H-W9Q%j_AJE+W#(P6UVKu_kiO+-XZln(n- zu)^UX>Lyd1y%rYcmL9{>Ha8j}Gke&8#G|=-T{RxF(Y+(It0LVg3a?{CZ6!*F9x*Nz zLAhshY6O;2t*TOOM6-VBkT)PH^@@!FmuKn1G{sQAPeZ)lpA$baV}#!&$B9%5@1=cTYb;}lLT~pX!Z~3^a^OM{vT9$Ah!J9 zRCtwjB@{tKK4JRIDx50}q}j4EXmE5?1rWuayxDIE`DJ^K4Kg9UCLDI`anpUNc}is0 zeo3a&e9SX%FQJl(VfCJ5#JCIA+^yh|`6R}6F zI>VKNdWG*i=EIX|2vHNgt)2-Za|_)@GjXj7cshY8UaP?!bL&%xW(ow0ezK_in%Po4 zEKVo;Q9n9WR2g)nA@l5ym3@4U*2=>I4H+|p7b1?x4f+?4Kv$>&Mt>scR(=@oX%(i* z92Xlyj516X*>{&oWK&a;CUTutMcQ$N$6_f#v{wqrDd*x`K>Ya6aY5_2xTbN{Si=LD zBe*xNqJjV|TfNqq>W_drUR5jG45Exgh)i$QR4CadaS4LEJ+H-=UtAfI6h!K6?Mf7i(hvb;1=RTr-Dr@|C}Q8(g8~vg^jnEI$%33?SS~ zV)z!hcCZ+aqb^Vt-Wqa?u-b65N4t|HPO$*z8Xk_O!gv{rp7=JF|1UWm9!uk*(A(6a z;bIXb*3ls|+YrGTEK6dift8w^`s}9PZ@Botn~zukCz)1#cRZu6Eja*F(5XS>zF}0_ zrSxxjBo{R1ZPri*kt_7{gjJ-i_qJ=yf_YckMlYWiaYvpX2qPOYskfzhf?1L9K7~Q9 zr@=Q4NfuYlgW?|jQ3eq&yFFO$7!DTRL2Z%L@P_UCZoD@Y-@)JQ~K&$vD zjk_R|e<5vw?%ReMi7gtZa+6pu#HwOf<-AJA^gpHC~~b;5RA=)po>%c z+`SFwHX2TCF|ONn#AlM>`ihXCP2hLJ1=6Su!Pu;zpKou|7FiXZt&EEDTatxVJBYtmapC1f)2C3N8y+ONfV%dPh zk%{ad@l~*xPUtrDwn}jLazWln9v#EN<9^L|eav;7^S-?O=L=*9tc^*h&j=Uhin`*<`r{qB}(xa zR^D8?R~@DW;%%c;$sK~cp+$MYMD>hJxH3c3R=h)Ly-Iz2h#JD?xYRS?CeSkds^BLh zS2G!+R=qo1KtPoxR=ZcR0cU1|0=X%r^?5@l8QwfkYdBG>$r5-HoNQVHZJrtJxP#hQ zgBr_-L{4;`0>^lbrKRRNblm>xY+L#dmqa9W{s{??}`L(A3`% z8^;Z_k*LTM#*Dvda;2+js|yG!ORX7qweg@O|E3#WTIfBvE;rHjQzLWJO*(hv^x8F#u1F)MTtrj19DEa%ngmj2QvN_=QIxwXPxo_yW^Z} zzKsY`b7uTowjy~&Nn|aRTMk=@Uo*2r?O-j+v@kgGT$}z&}~bE z{Y^01SwMG${cU#Owd%(=YP6mhOz%SF0O!)mA|p6^sue}Uo${W^FJ+nlSlu2BOg`;O z)0VrK-MZYl7-OFBF64W;6SojDVjji#=wM4S6;wqfgNl_w76?-`G!1GSdn|F)LDDVd zTM}G|PR8ZWbzehH`?6WXmy&8zcDY~zT5OxFT_EX{M?_}Xfs`V!N1}nQmv&66M79)| z5Rh!Eo6wv!>+tusR}N0|bJwV$GIQY1B#D?2miOgeLnV*RL(Vrt&FY38(8kfJO14Eo zEA_aDAagt>M9`yYCSosYoLq_Q02a~!zlwuN7~|2=2uJjgLL7X@%OyR+MY5@hJLomD z(fa33O`dD`V@^Distg*s>?4Xqz8OyZ8+ET%)UDlj$8ZGH>soi;Hdx2$_pwK2dF-#i zukKUdu+$B3hY;%(3Ls6i#!C7dbmuvak+gEe7_nqK|6Er34hSg7NBB<1NbwIuWX_+t zJ=1BNyde;)V>%YJE|fUCsqYZ8tK>!uOUkv&)8a?y`uYCe1jhhKAnDX@&j&?#gU!*9 z{sHFpvVO*ttT{Ha77L_qpq1_FYd)$0?KN~G;2N0E!(U)6^9W)VH;C($wABNHuk?nb zcAK`rYd%^6j2aW6;`n9lk&8I!h$AU8YbL-y#L6XT1sT^zyUcc1gVI6R{O%Z3>mT-{ z^zb*HAa#CdY<;tcPY{Q0#~z*@>CZad2c~jQDuVnpE7L~-%^L|_I7y`82A3@ol8=+h zRAo1L-UlFFAIbL=<6`9iSvco@bHcdy%Rd78)A%4}PuflTf#Ba|V}aIYk^rD=?DRj& z#{cR(|F3LptqybtZgP;OGs3|a7dNX}RF#1Um9B**6KXM9u>QgpDJOU^bAne+Os^U( z8t`8{)5;&OA>00|NdKH2%#MsrpntW``@DKQkuC7~aq0f;dWYwKp-%7rg*x>>c!uk= zHyHf66%0h3R-OF?L$Q9D84z!)tvx44UST|asehuvTw%Ql|4ICz$erKwm_V>#vzjy& z(2!|2Su>`v_WROI*LJQ4!EXwPHqf1hUj`p)fb|Bwh5u6lK1a5!ynZPr3h}!^snn$7 zfTy9G;$Ns!kDV6qssbSDG{54%P^XHi)hPvTudGp0>(!w(=BYllEo40RL0clI>Uf7x z8{^%$etw3*a)c#edshltryoU~71jN%!pqM|>c+`#0nb9iin4Ttt@(j@!-13uDPol+ zmHw0jlE3M@GIW1{EmzJe%$S({WWprf^_pb&c5o$CxNnyp?mm{o52e1d{6HOD;VeW3u0Bywj7Zg zdSpg-y9o(kpz{f72+)PqH=gd}@JjD(ML%+w?BmCWqWv3MPnwtN;HmWMe0?xG0Q5*q zd^xCi`Otse@p5>{>m6fGXAiKzheYFW{WoCo{F8ob5e}Car1YAkk z@Bt>S@FLHiv`GFfNx~q3-n3PacDB zD>ON!SG59g`k6KtuXDT;+jH%%oA;@BH9Q$TrkY|A0Ix~)Y;1?U3Lhj1qu?F5fmt>o z-55rzQJel&Ys3ySKvk6N#$Tcv=>3T?kdk9m75TDXy>Ywrx-hoJg842kAsg#%Nbklp z8#1Vjkw|lEE^(nBK8@W@lEq_2i3nUk85!ULTnMJ^>w~`JWDr0JDk%$M|EMh6gH!>x zjsCw-r;z`kPIr?9l9_0z+13%*C=kKmX|uqz8#%jZDUDd~TH1_IXef9cEW@7g%SXnbG^YRHkjn7GEnEqiO_aNxTGL(vXl zy#d%TEAUBxqFPa{NV2(_4y{#A+nlL)&C;v>D-vHWQcL`}B7?J9S^xXz$LCKT+3BcY z>foMYA0xBlsf~8`E575YD}k@S%YVQ)1E>2qqgD|MJa{L_tBDGGmv(}YdTbCiSHBqH zFxnY;4-Rj9F!^<67%9jw*m(1I#a&mT39LODe(>BRePN+~9-N$V^9Grl-e&=%v2zfN z5%i$;MLUaj4+-_+_tn`tOLiX_f9y;$UiE1Ueu%s|YqHy0zmx-Lu>DGQk3l~L$JysV z-{NdDugaU^$J4pk9%ML-$?5u8AexBQ?|`O|Ug}Itwj*$CakJ=qF>9FR(fsMm`V&Eu0Y_yZ zxq#GcLSsp4u;gBCt!aU88!#!P)UJpf@xtj#|q&?FhLKBf@7)lpb~dSRQFR)*6f2Nl0l3zmxq) z$)5BE^=7}-A=FVCwS0jXruzRFy9YMSwk!+O8Me&~+cq+6W!Sdu4BJ+QZQHhO+eSv* z?^Jc)=ho@I)%7Fx-fO>W%sJ+%N%J^8TGAJ}8jE6y{0PJbL6=R8>H@u(j_ZJ3lg;@v zn|Y}XlRYU;w4(-#T5F;SC5D4rs&_f7q;K7cvr@!YH6I}nYERiw(UIL%cZFf^K{@yP zQVnAc=O2W4sA?Q(0S~9Df?YxQDoTcMc}kHJ`88w3e&!S=@cg-*zH!~D+M0xV1E+x& z%ch!aWfa;^$Yfe7NGEH+?=}hYt2AF_YNM^vO+1f5da6``O#hTA3{+NP9QjF}at+?Q zZmXS%$C9*LJe;F;k{;D$&`@QfwVQj&kwX#=0wPd=JGXt_T*TJzu|n_DM9xcW6@xz$ z78FJ<%a{FXQQYqA`tTaaC?0WdSq{0;p7z<%bhxv(a06h`pZtO7D+)omz^*?P%d;2B z&D#dvf1PvL9Q46zhm&N&0u7%+uAHMpAECl*NFD{b-h|fR>!isfcC_ zh&Y4v`+|4t)t;Jf3M!BsJe!J2@DD7-YiO3HA}0z$XKX5A;K!d>DYfjwk+8SO51Iai ze$iC$aE~dYWF}h2{v8-UMHX6=^kX#mZsViil`D0+G9Lp=w*=kpK`Zx2`?yUP_s6&^ z%{vp9Od>Z@Igd-D+zel>SRp0k(W&jR9>m~mgO@#>+|;la7f9WYG9%~gk}Nvw#{k%c z^nK%OQ8`)zJdEC3p(}hT2kufYD#bY7HkK<}oplxe@We6^=4rRdIF#)cV}ZuuXMmyN z5I)C%>b&cW_Cn&^{sM4zm~B;Td&Ldj()6d=JKO?Yq>C408!zT{XIReyQ$x`X@(*h% zNc!ApIjNcbu8bR*# z#U7j}J8$@zdYW>tAc>20)x!ZCqkn!ZTnKg7IaHiFXEY5D9>Gsn3m8ZtaRr+(a@B1zFoPm43 zL|9g?(Sjazf235eW0)@{+a1GQTxJY!wKVa%b#SjV@Z&3n!8L-msua0=jz4R7sUk&g z2p9>{%=9L_pqr$wis_Ygso-7cv($HnTDu`vx`l7kPaE+6-X9((#Xoho=nSoQ3)7(A z2c*t7*-QUWncQOIc0}WD?VL5=0MjYGv1M145M&t=Oyz+QM^f_lzTA?`A|1Se2Sp>g zt^_GW+CPynIs~UeBHG6&oM%AKrJ%_QOw-Md+k<=a^KX*5g=2dh>mhHG`QTHM736S= zOe$-Qeu+`i8o9{pqnWjZwHMj!^(2~EsST+n+t)CSE+>9*!<0*vSm&{CE1bO0b*V}} zcM-kD$9v`DHScIzz2mYr%Wn_SFyD|kXCyM{;{_F<1>Uz*$CriJ2rY^X&LWP7#Kc`H zcH+9u2hr zuwy!cdmx1i=0zI$ssqDfd5O`o%PC1-glcGNme2Jh zG`tCE4e|FfMe46>|FzJ@s8Wel}jLy z3LFTGm+wpbDuYQf?vzL2Cq<4na0>+GCZ3BOtC5pH@rtkyJ_@=j28ltG)k8}{se+4D z_9h!^+$5n>ePxgs-hy#ZAgA$6g!Yi5K7(t1`5h0Zn)vo~!v&U&N0wNW@>mxsOoU4y zTAybI3pS4Jv?8JFJRtP%EnpvM?a1{1A+`Y?qJNKV;u-%7wpk_r7q5pN_RR150ktFa7qiC$@zdU}g{BfH6U+yU5j z{{O+YP=#YDIa8(HW#}eQkB#ECUFb+t>*A7A&?Z+D33-ORSG?B{%NSd zW0`R$3{d~kQ5nCPx=vcKx4I%y$45y)E*y_s=`n#Nwuif-tJJMl5!hDJda@iNv%ZCK zV2h!;f_kkbOI8WdJ1fwjHgy*nsqr-x`^g3}jqpfflIH?KyJDF`j+53=Iv&4YnrN7p zbcz(vk2p6&@@W)N}UfGvTncRSV(x1;es;Rfcaipd#RM46@j#jFlEyK~Z=}me5^DGi?Db zR#u%g#Bj?}CfC6NMq3ok;om1SM5J8LWs7m=ryuVmH<^yw-PCht)wZg*gS8ktm*h1` zyRO_-8N*$by&}jf|)4lUN~X!nqOi%C+|Z*C&e*kNIPADN19KHo(dyiNVJk* zh^R|8pQU7;ku}y#dC-u(YW;lCBM+wevSVDGmD$@qLL#m`uqki3IAg=vBo|PtR#DgU zOEY?3Ot&I#q!J(etdGWo1iC-QMB+_~k}Hti#)=o9SH3d|sy%oJ=ZWZ>g^Ou%aiotu zLo_ux1+~sUSZUv?$hr@9%!FrBnOvj|I3kbN{>$1@+M7|U?*6n4K!iUpxAScTwH$jN zat$R6b?LlpqNehprkOU$biSF8$QG39dmCGgFcu!YQ_ z6eJK7eunr4W(Yw}U$BV@2OA1+O!Vc@N5rYfZzZB};k@xaZZ$~tWOHq})vpWutr(kQ zyXTuLoK`Aa=MOJS5|*NZC;MAg;&14$O+$Zt(-qi=ze2fhIwOhu#hj8#`FdFDQbnTY z&eev5B>DTG@e?wRX;yFQz5FQ--|XPNBj$0~5Qj;uT?~?^5|N8fep5`{ZIj$`{(({8 zLT_{Ph%l=Y-o^j4@1UXP-k#AwZk7-AQTZvd=h`0j-4OcaZnyLl99-8)*tdkz-iR@h zyO@^9UoK|MeQ+5xahvf0Cy)M*KVF8G(MWw)2Is{4*EGYFZ{&u)*Pu;U*Wa>V7`Z&^ z|Eab8y*xy@nc5_t{dC0I=zzanW2wxrVqL zPO@mcix{V!$?ma}J~HIpMze>?Jr_E?>}X70L{$##9B$Qy+9E}+zr=hkMy!n zn4-5yCV!fySEcMu_px;`+!O{?%QFHpu$C7ZFQW9!FH3#6Rq8AKAby2k49TDm$aHeh zcHZp`yz)I8Cnp{?4{aZ|+*N;8(kJ-6=Q6?N@Py;deg~VFJQOD+R-$L11L0u51`#TB zQ#)mHWrYfyQAO$iIV9peLmg&9k+srL+C*vWDg{pfZ@AIb8oZ(t)> zUq$!;`J;9C_|Db~N?MI4Ij#TldI1S&WDxI|!f;8KW95hCedQT+KMLu2WU^&*?d7d? z1HBCl!PMDJj|}u|>03O*+EJbm_2UR?n421Ikg}Wp@?V)#igZTu?MUO^;>p&a15%!bLG#tSXz6C7PfAZ=PRSZQFlg8$ls%>T0W_uH&`qHSE{Nv+fuC4gormSpeG5CI`;$O$)fDkVXLD zvRwtTDw7%M15*c7{ly_Ig5Z}U_LS6EMKBEZ?~*~b58-|}%UUeCB_40?FJe`>#XWN# z;t*57itt?n(Dhp{Q2u@xY#rEL(pDe!erjx;C_Mz7m>pTI_^A%UkZemgrG8E9ttC74 z9FrLbSWSWN4!HB&`Z%ivoE@ zNXrtBkf1T@2LNZYS?8xjLVRcsRbzXzse48e7pqVYxa~)y@M$a;St4W8L1Lbs+--R^@dKV96vhb@ zqo{ksD(2++(-1ddy>UrUtr4319X+u6;iWJYdYMz3DO0uK)+{#ABGuhKNz;qe%OaXJ zTt>Q79h!1CLw6*9;{ae_2sS!oM&vLpF4OX{wMIHO^?xYaO405C(`%sY*a(V~HQHaw zX1TlYpw7HIZjs6>4_~yW;(cKX1#BCoVGkkXqU zyEuITsj_(aNp{TaXKl%jP zWGSV>6rei>h>3V}le;P_QqCPD2tx@6G^Ejbtp6-(w}vM^!oX>W#< z(*-ZOiU7@2)CS*;O*dmbqUY42RSF_eD(Q+Mm*kJML{@T|6oiGuULV0hIjPv0m2?NP zG48h>8*i)P$>kSyL|gdWB3t8>tz-eKpF=5iGP>0VR`ikfavQJi-*K0lvnBz6?IM-) zoLZzK4mesPkn{vPs|j>1KbXJ&yxsbrz~*)q2>`Yrn~uW1zkuy}`(*KFxFi7B+S*^I zD*AjH0(iBW1o`-#ed?(lI#SBt#R1izmr{tAd-Nq5J7v%zBvH&YG1pTFb}a;yrJEdm z>f&ArLKdyP5EgL9{dmg?y!>mgi)MxtTb?MrrKDyuxgvq<7)a{EwB6{r!yxQPsTOb- z3tTDk&q+)uIsU)Wg+9i(YhbX4U8=avauT?o2_oqhZYivJBAv=oxCQTqXZvfT9}GF7 zEvB3|+OyyK5C^CRSK5zy`f2hGbB!?`r8O{@(9Y-)iQQhsv zyO|BT@u*33G5qJ}{jws4qxw5>TCeI4CR`S-=cFR)P<`288r*7a)KeO_t zP`u^ZdRutzeGS7Can}-aSDuPGY#XRJ)g$Mtr^e)-c1V$GVlMil(@p!#;L!#jjCqSH zm13?alV}Y22=ZDmIeqHigYY~dn=K>qJaVNN4B;4*kI48#e1i-z7ZrWKMtx@XCRgORU%`>h&#Ul^=uEM3@%OLY?QyUS}zcP@f2$Fsk-G~!b20)hx%*$+W zI-__N2@Cx}?y3#!JjCkC``C^@ru)$pw#8GHbL;()6ML>5G$u7RhI82w)bry#TfrH| z`Er5@WIEg?1(mFVIlFGpTZSFBgQ;sFev^%R;-cToo*s1J#(4?z=Q^gkS}g+qsTSCq-mMztL= ziHWXtnB^osEoB|PKQBB&ptb4MYWHbppC03}VbG-#Q``lGD&+H6nVJqy-!;SRWOQ0` zV=f)qX6W`a%M)LtMJ}jQwp8Ok^lk+N!A%GKN^TkP>o>AQHk8$OpqA~;YQqp z6+oZlor&!?dMjA%k}q9Qj_Ia6-LgA^Z+yS~tF0Zl#DaVS+=TxJNj5+`_`lkkwGyBa z$|t(3Ac~TJA{0_Z#9E3l=a(cB7>!2dO)f06S4VJ$!x|Z@%&+Vq-Eey5x6SW|Eh#D$ zU|2&_PR};B*_)=%Y|s10DYoBCZbiku7dc7yFEHfmtMr55AxRJ3$e(fohC63&A!8Pw z!lDt;ogavb^(Fe1f=}Wb3Y#YoB@3Ie`Yfla^iHTHHLYG6>+G7{S|yYH@(gd!*jx+O znxph><=8NZ&gYJl!SnUj`?SmNM1^8sb!P3U$*LVKCLx_p62umzQ{{%|3J@;Pe&!9HbxdVGA&(6P zw`g>q{lPwd9e7x(h+vAjN2*@idfYB-R>%FrCtG#ddtKw~YQBz0-oa}b`CL&ueb(ly z!GUEaW*qxBmRs~WtmD_bbZ{?oTR+3b=7e(Fm-a4Glq;)IgSr4 z$P+$R#hx^ zL}obRtYj5>g8dz|2*1Ac`y97qnk!{%h}MJXEysuOuP7UY*AR) zuNZ2QsAQ0;J#6zLx`+gk5j0%`CN^2C?>v$}OApX63)N#=zkQ%^iqyKl7?XTzPZ@l)d zgrbW2Ne8vcgD!{~t4N~J0OW63*@%uv3dzPAXn`2f7?WWvgPySIRFCvl{Lnq)LMEP7 z`lr=5albj^Mp~LXi`()0@$X~Tsn+LgSKm*t9p<0;93h&F6sf9q+`-?j25NpKveQzv zl5=hZR!aSXbr-zJeGHxBGB!w|Tj?Aa8 zC)E|NB&qbjE*CTDVAt4-dvokxTi`jitTNXORwi&*L!Or6Zd155LsfwtHYcE-3vAP( zbqmE`8vz`HHL*B;06q!p4OFa0EFJH)?R;=B|CsiT6Q_6mT7GH}a-gsu1 zWG746_LpRxkvWISxnnN1wBk9lsVdI4S}PCbT()}8!TKr#(C_7L{?}=l@IB>j;ong| zO7_S}>J0Eff+z{P2SbChyU)I(y`xfGhZR7eo43t9?^0=fEZvrwi-rw5-RiLrl1W z$}&Kb?Q9qYZ-&EbecF|dQfp_~KH3-MecAH+mH$S)E@#YLfjyd5JGm>Nj{SVleWnnf zJ&H)q#u;&T-3!LG5aqLbb6ACW; z)qQY%H!+j8*u$aZ-R0Qn&BO=JgVn{}PhVR+?*!+}s{HJBuBi#oZGt97TwLY&c8>St zBR}_zqwSMQ%C;4|cWl=l?>Xiz-7${()IO@N0T2iN0lb5e&R`LqJ=?Zu9{hJgK9N8# zD!P58J(PZD5(dF6-UW_PoX>q8QpU+w{Oc;j=5K|bc1hhnv2XFX%U<0VWb#{Aj%WK) zi^6{*y1@Kh^MUY9T0A~-M@zZDb((^FYI^N+7zf0o&4n@95ga&=q=3gPVL$ywsQEQR zsFa*uskbL!P5wB;W3-C8a|<)j5!~Vb>&II<{7csVKZ~*b^QH4YY0X*{MGUY3ie$Bh z1`wFF@cU_wL(tM?2^9q$%x3+oDYs$4C(UZe*orBmeiiMl+KZFi?IH7S^ZVjFcSa6n z@6UH*ovU-lY){vx$?fV-U^_GfnaKS&2HCyfu#6VtJ+>>%m$9BazYZvnWhJ$__hmBZ z7Rx>StMl`^Fi;g~2d^COaBvkkul`s_e0y@e7PTj@-~H#B8!2yFXhQ*bs|Gca zRER!83nU1_PRf0{rS~dQu`at5krou^em`ASm260@CLTOc@>*6<;QO;K-gl2pAx!q{g%O1It#qyfDS(pbiz2T;i?7BRk-B-|^7Z!pzJYw6y;ieg>{PS825) zzbLjEjf@!OiOu4W5Gpel;A>$*E_j^-`g3Umcw3HKOWDd|f>2HrpQ{)Ok~2M;Bl^)! zsqh0o?O{=~Z{93QG7Ev!p|Mp4VZ6mEBmtr164UH{(^XH*2o3U5i?4GC_|`j zehgA4HS^mydgVh3wh_s?w0cx}d8gwC$n%525BcVhJ32$y8E+mf@V~1q}S}CxEzG_-Bg=t>b5-NpB z^oPRahI0`>Yq~q*09w-o&>Evu;e_*R>i2fJSBK3ybo1q>(biYCIk_0pL+Rl?>j4t< zPdp^bANC`6fWvYpgaNDAe+fLofd4^W^PDyUIYO~BnB9Bf?{#GPD6*po19XR7!SPE zNEn=`ILu-)#k*>dF;XgGw)SY!de6x1>ySY}Mp37?PUU1~wZ-D(ww^$@F4k)w1BaWJ z9sz8H0QPLRR6m~ZH_xOx!nL%M>vV-fuY_GFWml5ZxXL7SMu%}(YTRJ?!z$W}uPv_5 z*(zo}dDn60J-|VqI$C_l`)O%ZwzE1GNie=t7GOcDzdnfu1Kj)JMPgH=I4C`<@N2Es zW)hh>?&qx14RU=&YocMA4Wp{;Nkb%0L=|^VQ`8q447tg)MybDWe_z*{L0RTt(M5wL zdN}^%p(R>O9q>@N&6#*5hEPCbMZfG?`lB?rvTq$SO4<<@FYMaSrjx=7MosjPE1_;( zrZa{bxnZ4$b2rEkF!)dYU+88C0wq1xjWgK#&uR5MOYbGe#+?JN?rn+U#>^Ew_~fBg zftO@XAHXTBABw}=FG<0==WZDJTw4hNe%p{2I^E@$cej=M-jUD`Tk-8}QrB%@&1awh zPum8-T7U*#_b?isZ_jtu4~gOITfb<8UakwR`X*4e%BkPcB3ei*T`mboEyqaoQVXWQ z0cpGmb|vJ~4?cl0U~lZ;9DWBwpqO~$X_`njWg4Rs+DZN7mL~lD3*{FGJ2lbdHy}h_ ze~fAdmUoamG3ex3gQxh~!HzFsS<;W%y{odLZ^X|{G zRS!E$%qP~*T7_*_y+I7d8fXUw;U&@Pi5MatL}Tb6PP_c7fKA4`@D7HZ68-G$?B&S9 z=>!s&W(>HtKsMkd9gs5Hs{rN2YV(_sHZ74l*A%^19g_}){t+(9NzGjYf<3HL^=_s| zFZYI&e?RHeoe^B9B*YUI`^D##V95NOfGJFD&K$j?xqy-XfOw^<0{=QSz6yeQL@Jsn zMt~GNK&pwo^zE~nFu`zMMc()ulM+YJ%QALK%v zp?#~{$xpb~sT?Bj@;jt3m>#^sgB-`E zh1KsEC96;qKQe|~KN6!$iNMxu^oA$kE;5lSX~)HB_NUe^x~e~}DI8VD8l$$nRFei_ z@t^NIM#VPG!`k+}*^%0o=I_~AHUiN#N%A<2^WKc=Qo(+BB0*Txx-Zk#xXbu4i6RDL z&f16^wq++Dwsiw9d168AfK>4xa9-{k{-#&hqXK$?;^a3L<=?;j?1TJ5+wNy|o)1a& zW4JnM%T9lzlIN56yE6hC6GcmuQ#Ei*=Wmq%^eODed*Ie0scH9?T`#2K5uvg>c)ze@ z@ZtzI7Pg#!G)}T>q1oV?0KXOLW+jurNqpTxzN$*v(oVV%Km37|{89#^u#b{j=~x`7 z-3zVS+7DTTx{OOBGMy9hKv%Xk(YD!d)foR2Z$orRyg9n`&SdcSLD1^j6gpVI3H%f= z2uk+4RQp#t({!8%wCk{Sji?wS_u-`*T)FHKFRa}2*2xvk*wfUw~z(wbss|>n8@(|ANL9wO~ zR8+O08h@Zt2CZs*l8khp+K)T~hiT!ci(Fc4y^T0leW}X|n!|Zpbyrczg)4loo?39l z&REl9ea!kixE<@9btQB2W`SeQulfUA%HFbL*@b73jmblk3f8kQoXgVI8M|lz!F(xC z{v8e*Ivw{^=>cNYtQ$`ZxR@_MyT|8xu2%jJLLuLVS6;_RVw)ciA#dQ@NL6E(~rDp?{eNpUSB;Hs8mr2SbKNa7Qn)}^By%mnYa6}Oq9xCWBnkLXN zd+to0;!f3v!br&aMHs@+oXqJ6)*lS@Id)a;(+6ePvQu7} z0)h)#LT|i&6fs&s*|e^)Hm+I?{tV`WpS7jDa^kuo7N5!3eAqX5x+efWxma z1J0n>3rDydL$SWjq-xTxf3O^@*1R2WIGwbJozP^qh&@VkUP~0Wh19z2M$}x=Pr$6U zI$W|qxvyM$SI;fh@;Fu<$z+Fd;zed$QWwh+b)i3i%+Gn-NM_h9xSRON5G6A;5FvH# z>z!&%hg>S03R7!4+|wxK=UDgC>uLyiH80QmYOdktke-_5~2;=RiW#&no3T# zu?33^Fuc|o75A6l&P5{$t!OoidIkrkfONSQg~@03Qb6I^LyR1NJK+cuaLJ6KbL82G z!DKHYJq|SYf0Ls*`lG_c+gX_?@Fj{0a&hL1t~Q*b74kP)X^?Ka;5oO5>;zd`iQjh} z8o@dL3ICxX8{L7Uv|-z@L^EA}C_;i|SevQzs>bg;^=u^;C{~sQpcceKQT*I964+5G*#(V z#A;oh9$v#HTb|%NU@t8O?_Sg-vF5a*wqU0#1|)(9ujTg}55cI*MW!k`|72dF4+QRa zG?Nz`8XiI|$xO3U9h#N;FtHb%-RIYVrk-v92{i6T`ZZKnVZ~u>Niom#PexAApPy;8?BIiMWF{C;BU5; z?b<;MEo$;xZk<6pcbwHu+GM9pvSbCn->$YzLun7&yprQ^1VgQ0w&@YOJvO9yy=jDa zP5_ysOI&V|aU3WtYbX(CtsqEYE48_h~B2l^jd1!Jm5D+Uc}K5QgR126 zODN!0?Ik_fiC5^!LH5fj3ehWh5)&KJ&p7|VdUC$3-s=%S+f>FqC<|8=>!Y7>?D|DLb7J8Xm)Q46SRh5+8{aNgxEs_40Hy;R@RC4_W@riB;>Oo zjDA`g#HGJ)psmP-1F@5OV3G(0U=r0bzTpj-|DN?5FL;98S0lJJErpvk#hBWKvmo_7 zz9k#x;q2^D5_(wii*U=(sT0obhio)znrhR~&ePIs_}%BqHJlb!t3d|b(ANwHt73m5 z(a+-QS)6$2|7UIYkxg&dqKDvtv3xN65a&4)tzvffZ0Q0G#DJhPxRqyNDtsl3?_uD`n z0k$&IJ4+0*`%Pio0K((lV7}=BB;7etz-zW=mRPnoK!JGU968<$_pnDkJtoasTpKL(A5sX7jJ_sgA_NEYx~ zrKk3*RbEy)pInQEW;>KkiVsuXj$5R~YVp`c8|v3?Ce%aglm}x~b}=r=t1R|EEVE)k zZDbbHzhK$4IPrK|+Y30=k;@A!Pj&#kGL{YgeIBu#j}l5r)UuwU$hA#fZFPP#IrGSS zsH~Q&z0)=0@KgJrfjI_(@PdM+l9RE&0&`W8)dnemz#P?Ifw>EXhra@Iof4n)aG(}l zPUe^|!aev}_vJrvxc?ED(_{xzgpqAr@7;~pyfs>ZjW*1=5m!-7|6R{OAf?k!^*Y(3 zKwWiZ8&2Dn%dD9?zzJh&wJz)QPFmNnzbwwUc$#iis7-IJ50Z`Z($|+lfwj_bgVsdjRT(0|Q)7pvN@KN;<7fGG z$ALTng^Ta+>fPQl3-Y-)8Ye*$xAmOS!5;NBa}f>k9{*X~K4If-K_4%C`wl;`>^CUAej=cNu2DgZU|(`{D<>2 z75dMV27xE+)ShCDl_UGj^8thZpWGc=i%h*R0iVzBjGG z)+3o6x8K|u^*foVn;~vvBCyr=(xQFeFgP^u0GpqFqJ|E~s;s^Ka)Fv^)kVti49;cS zlUu5Tuq;-(f5l*EdX7uSwCT^ZHY|XPC}$Hr-lvaU=|aqAs(a^f?vW>Gilc}PQ-WRf zvrF3snaMk1B>WJVjKS=etJpK$=%Ghy;HjitD^@p+Ooy%7lYv#>4>#`^)EJH(TyH@) z9(9(QDTg&t8?#D@X}uLW8h05_Kd!h;sCNYHiRVmCbi|8DK}u-(d=3>y8YCQ7C6eoL z3%=b(^u)e>p)v~i$Be2^k%tdPwU`rR(=|Mu{tNxTV*T^`?mGnFSfhsVZ`+#yzY725 zdTQ=&KNr!yHjS8UnXtqmfx@hi6oinm5FxPP!~6U( z#n$m~l`r>AEz0$S1I=^17GL$9(@UeD6`L=Ax8BFCb~$8v^!47y3rP22U&Bj+?)i1M zH-*(ptj4(B>>&*qr7`=A@w!wz0b{UAOIhSk9khi{rbEIYZ-UN{gefMtq! zaNCPFHNj#e&m{cv>g`M`cvZW}dIk!Pieaf;&&`h$@* zrNwYSmDDyxG7_rhZSWos`hI6u5f4%idOu^t{4c{M^cGeKb~er84HE>+|AhMhO{ax1 z4E}VxQRaE+4VRvAjXH6EKO@j3&pvGY=m>M z+hoLLig$?FNY__?vJ}S1#*h*zO3$IUPP>JK0+Tpuh2QuwHsQvUfUd?7bp`AbLRz>B zRYfei29yWLXy$5Cqej8S{fa54=_zD{QZaXv$KdGNX}rW;b19Be8J}|?#;IkT;y4^_tbgaGsfmlH%$o^u*gD{|{EUkWmG`&*ANHdQa1>x~zl!UPOUWUux!?!t2 zGGU=Pf#gmzhw7_=(Uf{x?asfe04cAOg@;mp%T#pFgUfJsjr@3%9gmx~GUP{^8d(|g zXuNrTQrZt#rhy*{LNTWCOu^q8k0cX$}K zEB+9uNBmYIfP=zK$lzZ~)$0!)6u7LZ2vlyi3_V*2LS_Vqh&jh(b~fpt`&E3?5NrzU zhQ2+2qxEgt(DPf!;Y>INXfm4I#&|sx%`c2v?VSR1%1zP|8*Is>JzzH|dgeme1Owm~ zJ{_nX2sh<^OQ4SE@8CDYT6FT73n8G<+!j@^JHsy$-}ncoSvm;)Jd}1;_OKGF`PL;7 zU(^Ia$X%9#E(aRodo#{y=KA!P6H)gUQpVwWv#b=nrnjC7Ko81+e;L zIfhY3!GS)YyOT4!XK&WO`Ghow=UecrEczFUh;!vb@m5lxW8~(5l-`qyOtqujh7wT} z1>vEQ#6fmz^Uz?TP|-~NQ2myokFZOBRrv~@F3kiC6K0<5Nx=JZF z*fB53FtVqQk&xmX#LYB){EYERE5>GQmAY5v!q+(8Y(ZJNn;2NmHma0XPs$JhD6Cb; zSB)9{nJT-20mXtQUs68V*lax_c@aQ^I;d08RjxPnTreR8w^(Ac>ul2G ztJlt0IMuqAcJ`sY#a2-45w+WZJYX`2q6#C^$fxNFo#}?^$c0wf7yW6qn$Pm|qkcR8 ze0E;Qir{>G{m`$tOS{Y22WZ#&PVZh?6U%jnnt4e$vc;=l*0NwOI01QSu{mq|325zi zG;sP<%lcV%L+HH!jJyX2jo?&>S|4MK%NRRb^gcqKl7j#I46qA$-OL^Pnw3xel><)# zwL2ZhJI)XR_5d*j=ZG@Y+9AGa?0Nuyet<4e26EO5qA4K?Fl9la&_|Nmkh}Iiha|!S z3-2oP@N%6w@RlE=*k*H6w%CH+Aq_brFu0>?EVV!MOJ+1o08RGdQCsl z84jC04`zCf0P;$T;l;nQ+Ks%S`+vphd{$*I*pr$bsae)MTfvZ@NLUUN}{EgBkA$z#Ta!v3a(*!djZD zTdxw*Ab#}XC+jx)!wxsOS#~CDz-TsTT)IzGEsEBf;YNtBNojOz#rg&J?lHl}6+$tACIzaeyjqOOe{}3^5M^roT2uzI@w?QMMXXv^-2Le^Vv*_SBd*LJo>O?{J{^K<>I0v+V2?0ScPz%I3eS}~o#sA`sP(BjV?p%$3ss!nV& z1^RzwQBao$^Txb_OP0&7vPBOuvf%ZNlCD%y92pcwh3X|m^yr4_HAdvr zA@)+z`9CT_6-Zb;tBH>E=&Xb^&-`jm%iGRX@Rdvb>QVauROx1G9eMCt(tXD}K#4Tj z?J>MA&^qOs6i*564%}{M$c)M+UFAYeb}lJCW#ECp!51CXpRmw70PLne*9_Ru$7NZq z)w7M(ld)OU@JuXomQN~osMx(2reYm;W@e2&Hz|&6tT_VqJ$$p}`Qn!Q^5c9VN8i)k zsdjew#Id~n?(mU<_U^m#%QfvQnt&v*Zc5$hj)`T?eL82LQMPt{2esX&c%{-6t-WU7 zNB?4q+9idHw`a`?VTbnTmk8E<^s=&jhivbjtzSi;UXxyu#QKqN;7xn#xYbda3g216 z>QQcxV^~|4J?go`{wYgJfLj)X`)z3{Lf*i5s))lPmWZRipVC4un6+&iuxi;WmDSI! zb3)m3^J9mf_Iz3QR#O=q*}o1e*be_*%u(pMNA;h0vxMxw@#a7Adk&2#5J02?2@AAP z@R;#8vC<}P8x-7A{2WD z&||A(=6Y7ebh*L8c;W81cE+qP{dJx+RT+qP}nwr$%x-yCxv%(1RD*8BtQsH)y- zcRl#3fxQ`>4mSD@salq)a#m?tzkn>3!_Y~&i!rOVlhL=bYse556drvSP%HG_781*z z1&HPIrLk-qG*(Zfj=HxqXCuXSl%L^*RC?-F*FA}^v<>cMycd~HWW9mzhRLVvMhaMk zo3r22a^TNO-3WY%42H)dm?Y`Two6$!Mc)C7iic#R)IdMpHc!USPIdQ>U8YQYrUZ4Y zUu+)XS`+@WaYn$pD%=H`UQE-KCfzN3HXHg7pu8G*pT+-|38(-eYri}B!&0q)SMfdr zTu1I6^6mv0q(cb28bMCkqOO5AaJ~R!jPxX#fOZEeEFw`fP&r$=ZsNk1wb+qunt+Cg z9fT%o(}X|y2_lV{^=Wmqlm?>u*?M?3Yp^&({pdqfD36RZD?UZS6=ZnC`V8hK-W%z) z``_fx!sitn{r?Nz{Ij3_oj0rgBiV!jml6vKBU4iXPFx!ql((!DmO%Dpp$50+*Qm%N zjmJlqGTAdCSb6_!t^Ne@8vIj9=3Kc}%<~}X{Ky$U!W_`prs6>J?&*>3)zvx8b3fhv zk24z3J1h@0p582F?nVX*zXMYc@VfNK3wj5e=oWw{VvlgbB1Q32x{$4U z*AP0}Of4i2C?0-8;j>tNVmqbUn&LJ}pjy{-4w#1hc2)f>X$hv_G8{_~COC z>LRmK80e>xhE-YN7z9ijbb`|tQnaC%lkg0l%H1)uN4Cdr^?zRhr+ih2F3iPC6wRqs9 z#mR?)<_CrG%w-AEtvdW~@#RIz57x`_>6*YgTZ2JN*P1dza>KNg1&2nX?ewFsD%L+@ z3kxkFKOPhbb>!$tZBwu5KMC)dngN>7OzSeWQZD(JeQ`m=0QvO$=|d)S<4LFG{uNoM zArWb>8cPjRYwDtJ#?@P|B)DTAN~`pum=_Ep`LpWH7pY4Ll@1hr8*IgYmS%(FIqF8} z8=(7&`iJU*-Z}2bPV9CdoUU8(_Zw~PoHO{UqX=-!Zk!T!EXmGYLy y!A|rq0#u>CzW|bJG4t5Ho zgirkNW-plr>fWY#Mdo3Dx*GmW(}U(8-dyUa7c7<&Nm2jDx2nj<7Z_^smVmxXm?ims z1Nm_dcx$*M%TA)pNs8im7*nqHwF7&Tv(!G0))ZFP3I)%3mmR`!BCr1m$@I5JZCD@P#TM<;-#%QZS_}IWrc*IU8KxjfTUC+nt_~ z(FHmOO4g3t@Gb-gD8)(H4wxu%2N!~O&{N=*T{O!B@aFU@JD?O{Xj4x2ur1hPI-45j zwkuw1B-#~ReaKHnD*xUVUz!GbK}~>L(xLDHStEX})aAR@sG@=%w>_XuSOc9{&~JJn z1H%T+2*-_l=w-1>w>Ns(Jz6_vp>*t)kmVMN*Y1Gj)uWM9<1383K^Cpv^oBV6Hv*9s z2@}hT@>%6E5D`m!&+<L!*p<=#Gq!0R;>p#Z>k<1RGqO9%Px4hbq{^l0=hjyCL{AE&t0Dq4_0EIcGT0;2B0v>g#kcYft`J!xbx?x+&g$WC$M#mTNOlm?b z`umR|-?VQn%(sK<-Yj`I7e3hC`pfgl%z?a>n(ds3SRlFoY&K=4{Mqlc#N7jO% zv^uXL_II1sk!mD#O8nZ&S{RgOmlS&qq*>C#)=vBBnlx;vowz*;8#orv#hL6I%dvdh zSNC&|-}fF*fDOXHt{w^#rkcmfS2#Q=oxEh-lvT`7TWPO2%5se#Sd`$L(BOy_6l_(T z52FtZ&6f*Q8^`r`+?9Xcx}_e--2I8Ps%HtlQGWw6>$WjLb0O7$E`Lxgt}PKB2DB|~ zVgi=B6)#=x*sWH&GgyI$(#v^LSxQn*r{cyG=IAGWs_c{yt3I<1qG&ItQcLYb1S+yw z=NRo0dRpnN_CctC*nHwvfyiLlOk#DAU!s5&b z;VYwvsNxMBdwLTQcq@Pn1r|9~_;Wr`M{!Q|Uvg4x#oTRm$b+aJb&ZqQN?XvcDDUFC zH+@WfzqnaHWLUx&> zM#WYJ*rQa2e;w1KgY41&BqDAVYx_C=BqEw*8H^Z$c4i_82sZ&v+p-aKlwWyBBz2Z+Ez=odu9fCtB9 zg&-(lhLA8L*3jgY>L!IVwZzH@SFzZ(P?e-&y;-^E*FIwbIONs8xwg2rdFAu){?_>U zLz@ZyMVlui>FPG#uR4!2Jg%-?uGn34+^%a$0jT=h@hkd_jm`QF@mW;OUOIv(nLY6X ze$hbgh1Y#6fawmY3cOHM#r0PQ?hNZd>n1a9&)78vS&q?TE?c~quovxX`bvI5Jc>qD zQ4T-0@UyU5w21dl;!nhqpzNTmZea8^+yC%q|CQoh7^_1G#6E$I^4%`D zGrQEEdY%_wxU*Y5cAUO`3>Y)El3h-WuJT>@($5mE;T)?VCagI-wSGg4AP|Z5iG(aT zu;yZgAZHVSX_Lqbj3X#Y^8qPtQ!ukjaz;~$R*3n=8S^6GGLAoI?GIuCvV(+`r=Gff zYCnahh6;w6eBzsXH~`33(1D|j)ZJ-3VZX0{ciDC>xE0%+ z5@(jty8hj$P$Hg}$|Ht3W_R~On3@5MMykfcRKy0W;1b0xvXD6-@r;Knu4Y)7GiM8jQ!x|fS8$6})FjE8+y&%|tP9VsIM?KoGHHX}gYIZ;zUe-s${ z<*8?sr;{R!xa(V(z?hT?LaY#n&73ccSG2xxodkdhos zVa`XXpajMj5kUUT?*54*GK>v=_*ZH<^^P?R(_YE5CjTnS+E4>^xK52aZRE3 zguG7@1=2ZprZm|jM2gsn?3*SA#Sk8yTC=_x#n%YzZf*wGp`muiRNkA%?YZXpkg@*v z-$_x8M-`VimR5vd@1Hw_NUhuKX;fJHoyo`=nQafxsw3@G)2MqzO*RIcwlV z%3+ZE#2pg}9QS#MO7(tGh)zlvQJhAsvg0+L;wkK<7X9{z;msur9y8)nSfhY} z#44w8O*;uQ!zg`kJ8En}go*mQ!S@MQkcI+ZpiZ%QF~R}R#0dcFhsM!1N$QxkT=4oOOPHuBYW`Mlk^03x)V%eVh)l#)-dxUQW!@zI|Z{hDW_cg0ke7iaN9VZHvRUcnW1!lqMrE3pbB#lFQ+G7mSjcnIcO`pWD-c8H4Y5%{3~dn>Q#Ms^ zW9@M_YTP~b`q}<@@$Mpb5jguSG+>G7L88$?BuNevxMG7EdqAI?3P0H5U8kD6ae*;h zJ;m3_wy~zb{CPIomEf~rytBdln;&~pFykSXhum^>cTS?Nz9s_Z?2yh#03W#dA)OCk zQl0Lq9T+hv2WIwJ28xvqYXsjlTtnFo-q;x^JcbZ^_iuQcC)oaN&He0RW*Gf3MGH(` z=oe+&gXGH|Pcy4!`;IKkjNU|Ovi%^%77fv4jh{wYA1||ElO{#BZ+WMjfKMc+yzr;7 znGk9=$;-OBosJIEQ)_B`iBS@RHL%s5oiRG5;%E;B61sqciTCGH4&f<C@VX*tdt=ZB5i@j=fp8=XsL9hk4kS zIopaJ+*cxSUX<~JRBytQZYDSh`1I<76mQ8EA`P&E)z!O|lS)#Js zjU^jXL<+vd2)v_?^T|H;0Mj9|MwzToA*Rt=Y<3f^xiw&&dUMMASH`2fMg zShdHgpwQq?XGH^-FYgeLQXnWbAw1z8oN@_6?}2SlWfwBaF<=xHvZ^B#fu|nY7ip!) z{<{QYsBcT3>I^xNS&+>fj5NA4`0(%dt)8v;Vd*+du0B{~uzeWpFIX)Q#8E9|G86qZ|vXuE1aZkI_9Yz=~&XjxhS{=bD`vyWD0C z*M8isxZ|XnjWA^n%0Y)0f@)>%>^6Cz4jTsvwI)kiJGNNgndjd-XT0>#-+fpgn391x zM?N)x--ScpYSr8Q7FdbhF!0Dcfm0;E-N-S}I?m1Q0~Ie+I0vhS$^VpV4bmJ1=t2Pr z586cE|5lTC1p@9k>&D9ya8%Ctt61}*nWKlVK&DoRuujh^kkFnWX?-{b?a8I3B`McKO!i!@Q84uu?55V)!gRj}GfR@(56rPGZCe|Sgd_(>zf{FL4u7s8G!`rAL zA~uyE@jLTHah>kCH#|yRU&80V{Q@31*$GkqurT7EOC!;LP#o#oDVn<*|M#U)`yV_n!E#14jjDveS993@d_B|a*QRWit~^?M(q5NQ?%irplN0+h=613 z;VN1%;C*l!K}j2V2q;r)(JE#Nw^M7iR|e+U3eB3a^d!@AXG!!j+jVANwWUXiBIQPi z^|^{QV#e*7ap!e<)k=%0FZm*K@@FId9g~jr2D3$K$9#F}aVLUyL++-Dy2FHtit_|8 zMC8QqYMX`2ItMXpxhrLmt0$pGi1s=1Gzz3jZBY*5Bz+pk&JNWWCA{;HKQtNg{_`mvcFVkwK4 zxgL}*1hk74P$1oXOz3Tl7np6!m)^gD9X*%d^X7o4Q)bw{p;whU2j=7wvO8mS$E`XS zqb&7N$Qux>NzsPPML6}DDZYSV7zMtfWacX@dWOS2RTwn6RCFF)Uhra>uy$U9Nr!vj zt~rLrMp&ZmlE&pK@yv;`!q8E^%W#IqwB2}OV~{M*0YdA{jOW%wGF>&9u6dGippH~o z?De&M?GC)fP$K)GO@d;|KZPQm>2j0PQ7(kGVYyR0`@(%hm=2%t@e#aS`%e&>ixlev z-B0dQE_tGhXF!0WZTBc4HDuY^$Q9V_--^v1k?K{-hZmT-2KDAYVK za)VEUwYN7rQIyDyA>Z5eJg+e`bGX9jvL#ba(P<^V0W_f0@u-??n2OgVG4f<~$iqmi*Q4kkeW{ z1l@oJD%94G1oP5O>)GVUqoOlHjjRQLO<*Qo-N`L#nWfCnuy8Z?s#Gd;(m5HPg(?-O zjU(Vj4~k&u9ktI4-1`%ltExFK^;Z||jT>{RQL2YHs4J+6%dL+gR@1k7DuZ6=CRb@v zdcJZv@J%}T+Od{! z(`(U*Fo`Lvy`S6iS^HHxjIK;Ot%yNr)x0@j$33vQ(8p}MjJUsowo}8r6>GjSThfTQ z^R&zY11F%%=*3?Cym(Y|_BJ)asEp+yP)jAPz-aonv~V6;&=2lsUQu0CaI!6)6#NsJ z4L>Ez(?U$y$9;x-D(_c2&YsSI_q#G=&bh}~?DSq$j0cU!s42)Wx?L>k%VcN%|1})Hfs6CXmpF!ZOyfW$Ra+BPjcK=W0;z%M9*YIQAE&H(ok^bM8 zw4AxA;?I)SH~sG+TB!!M@w@UTqcTYOvfq+2QN670=ag0}!SbUa%5 z4g9lNqIy;qIjsDF(Op~1yK073sjNshY`i)kMi=t5TmGb(_<GB(O*he7b{UBz zF^N%-4VC;t!JN2VnPYs=_(H!hLb63kODS+%$wCi0{g%pV{19SVr6n(bQaR;R`Gq5K z=@u)nyasz{|(_O+0Gx$7O77qyq6DrJLfL4L!_fZ*)_B8T=^`woOmOE(-tl{ z5@K7aFCA&C;tpl2?50V9V`5#oE7hfBV}cr@nW95+m!*h+aYAWZe^n5GI}bECvZsbe zPd&CL1gAFA*L4QW36p;u=Y-_}pnq%PfZZmYzCUZV49I~-rB#%p0>3D2f|=jaXaGVK z2;b6d1PT^}4v0}Q|42?PdeGW`blqNww+K{#!}~IHH6;L?*uMxK!;D!#iY{nOV%#LBV}9EozNV|H3nfVg*wQid4+A!5eBEl&52X zitU5VZ$KsN=7#7GOA%6dGuq-_mm<4SVqeh@`6Ipn4c&mhA@8|od0=D@?t z3`j6P0?H+daRCe(Y$Lsc1EWhMuZ3i>1$7+ zh#!ct1m@;1d$L=p(G>UzY3B1Ut03PLVFDb~8|BO5!N%L?xtO)UhFv}a-^w^19X%#> z0l*MunMg}~q>T4D$_oIiPcoPJExG2~4kKbC6IlBonuMt5JrRnMV|1L4a>2+C?sMM_G*&VanH2q@8ah_syqZvh=$6CsefQ7D0Z5p!0`|tM12_= zFp*-^IGjWv3>QXUr0s*ZD|RAJm?V=W4T3PAfO1{?Rv1K_n*g;eU!HA#+AtoYv71rl zO4ZK^J?~2D_+$bf-m*)*s9*%gS<0-uICpVcP~&8NwWn*cFsZft5)8`&T=Ku_#~y0T(CfV6se zW1#fe)0105wy%?k2C%iAOs_K@3y7;#!$D0V>Rd*F1%)hAf`>wPXTv##Xg)O&gDK@k zIAu{(x?DYNsMRpbn+4}6m5=$ $`3`zP}Gh8n`ftRtC|86!_W@~R+4gfQT2w(ypx zOF6%NWQk0~J~C?7KCqqnYFDS|90JRWiQ%-Qui3mK8`kWD_QF-Kk8ZAXi)i21k>Nu{5}))oi^ObfAPQCCwzMVbFkh3&4@k zNjb!JVeO=RpuV-1o`aO00_%QT{;hpaKj}-skjuSh+?w$+N74Cw{^qUS1QtDmx7J!- zu)SwlC`OZ`Ud1T4Bt+o$kkdXU%@tsB^O0bj?jlp0SU!d@c;Rek+xHbS#CHB|XN0!4 ztFV6z<(>^=-4F~@csib>3i8FE(4V!`2pB@ z@?@=RVBz!JVh?b*C^)Vv`_K2FDMzR&l#+gZu$CYEUok_`H~DO*GOCg(M~6j6CNDZD z?-H#AB6PNLRBKB4x52N37`#K#jIR(yfdvA-NB%b+;fjS?<-vS0%8IlrAbAaJBpFLD zP@Qp|24zMSw}>C|fzuWr^xg`)ju_d+sCu4L4zUizrU6*7b@y)BV@C{*&}aO!#ZtK- z6knJez6_P4skTu<0(qn?vvcfNR`tT@qj&PcSMo^4@PjgMpy-3D$sQ(r_8Rog*O_Pb z+=6?*f~L$WX=NogAD}&HM%=LL6qDEc&9?ghr`Af_Mx9i(HNi;LM-jkJS-8T1>TENWav1+A$ML}? zbiD~aig@Vw1;U2NmFA3L#_s)W;P|fszgofd%1@<~{(cLQO!1r}4onJp?b z2$YTO6xum$j@(BavLna*0HtT5@jh9|et(F3|UV>>l zc+P~7U?if$QY%aT?9)*+138->r!MP;z8huiEr(tPT=wzYMu}`ma6u+fOujBYL|7Be zRagsGQ&I2-9jo2)ad?bg;|~@Y9I&2 z`rK*TXdmvvMwf}bCc~r!sSjQ{RGOeY=d9r?#8V!>6ok>_gm_Z?6i|4c2vu-QJSUKa zZ3+y}F%rS;_*Fj0->DG381r|iUm1B`Nbqz-PSp_*DH(|Vgt&n=?yGSE^^SG5F9~L= znQtNZ#!}+LM)H~2b(E!n#-st2K7v@{3qZHl!qGwp#l1u5h0g=0Qbt25iBN$T8>tX| zZXi=6Q>~%}x=hbzI0ixc_|Cd6%UPM!otN5~d+D!I>pMn9MII9+bmV6T1;|~3APU;v znmj&rhGC+fxlvT}Wz97RlYtZ(E4Vi4m{TP)d@~756D3v;!^&_gT2aNgS*B6y8Ebdy;5-1AC zK9MfXDioFsiX5VBdMYoB(@1S-Rot?2TlY%G>Ytd1SXGc!$&=0X6Zd%Llsw1!i;=qQ zE`=l7vWt}GE1e99gOolJBWClYB^nvlsQaA%j`6Y4lBq)XE&4gI0*qVZsbiWofij){>V^4h5X@iW9W(x`GpKg$ zcjuKaAued^6wp0n%$s>cbYG*czo$V;L))S!^ug>k8^r9SH7>P2 z5lPxP>K>$KZo2)(WnQQ3qX!Bx^xy|tCHyNbNSJ0TYF6y1n&5(#cosbqXHK5`5}B+(ft_XMfZh)`d%ZB z@WU`z_xS52r)L4y$@Y`t7Oy^lTNKC=PbLK;9N z!jyhBO5g$Ep7VSm{Ll_ifx+cKRG&Nl24oJ!+UXXMHf-3W`<>>Ir<3gc9|)=D`MwWLge@nsFiIEHtveP%P+wTV6jqA^uh4yG#lH0N-mW?xz(!h(8c~6jC5L1) zLo=oee74#k-PSmSRx*?}A6z|VVE*{zUM)4%LMI>^>oMJ^qqhfqZI*PVTWZpr3~Q!4 z9>6!rZwJB)AKy^3&jSHal?He_Fl&3+?P#f*s$_%U6Yv1V{q|Cn0!OwkXOXu3! zu%R2?$Iyvy;1pzSSGGpro|J6Y*+^T|)&cmX(`$7f^w3q@AF`SN+?H>Kg>V7O@%iss zz?({tJ0tAsmME`C%olLIPJhQT{eAI*1aCk2-f;rh z_wvdaROA@qU54T9w&BG&`6S7p&n?jlDe{{L1@Xm{&}Wp^eo}wSntAMWp&0z9f5LtA z1JTtuojc0H2U^)U$Hw&T*5CK}7TlpvOIFqIe|tI@ipNpB`%#bO{Fs&h1Izm_@Zn&r zZ*6X4`Y)CjsjR8EE`r8wiD?805QLyYy0hlwJ>^Xn$4vviUvpUVGW+^Zp(8R}r2wls<_s zD_waJJ9PG0NMPKwlMEw=je*W6ihE6nriF$=oiTBHWl=^DC&$}uE5b2z#R=L#L{K}1 z3iHPNO(0)iyzz25m4qnY1mpnj${cvJeDf)!)B&yrXOemQ(TAb#)&tl1t;Kvb>nHPe z{L)hc-*bd^{U2jOI<^uKRa+6YX&Ms}n>(#}Wkt*d;<^+gI`wET41K&7`IU!+=r>09u+1rek zK1gVBondc*sZb?S&A~hB`k4w&2ae8BGA9suHkd{Ycd~Jw+h$|lMgBI42u?vH`5<8zs$zfV@v;^hZQ>FYj@*TqjFCuEa(^G!%koN2uxQ>~Pb6cl9>^?&^R3?fsXS?oLA$$$pd%T}tSQ`B44=S4;C=d%A24 zq`8T(vu7XjJ3j?DF0&PxZ9P{k&=0dLgA3{qBsm#)ssl>GrZLmjHyoSd>CYy8de>zN zb$-lYjUaw ztFq0b;kNgJw6mrvT<#`}R)Jk{2{fFhX+waqt@W7IuB+TR=oy7eZba45&xP$ z{y{Wi`GpCCtf6;yjzIpJx5GUr0=Uv$T0B8+l9l_;j{ zQaT^I!V^(Dt9ED#Rl+TL!0w)%SKS!y@T{oB_4zyvS8=`E+iRis7R+RUW&XwBL@pj&mjb0l)wD^z=XD0LN6nZPZ`3AW9uY?# zRsNE5XyEHE;dt}w6LXcDNR=e9_8sjfGY3{cna{5z?Axz%-)IQ*CKv0v33=TwVp8hK|Wu0{9(Mr;5s)p=A59She%IE~n*2NiVp{8}tQy?JtopxuUs!7AM; zNSEki>br=7Ou%M`jYNwrr{5hw>c%PJoHnBe$U1CrCc;S*9kkT4Ye&P6n?@$2{CxIQ z5Ktx!i0TJ8tj`}|#0a$y_`G{XgO_osPM#8c133+pUBh%kzC*86WCgzZ0n9{D6+4YV%0QN` z{WUY2MQBOwgcC&tHDNv@K-P?g>rH{}=0DwpGp$vaD?LR)1-{e;OdZTOFyuh%uGL}R zIYAA+;8F}5ZFTtQB;UCh{bt%Mrzz{H_eBP1ZjCO?Nf zEkiciQ2wZ-2j_-c50n@2?joyHppd4xGpT(Lr&dBRS$l^R`tTZ>`gG!-Qf}#fQ4SB8 zwO6UNJBr)imHMS44{_q%r&ZfZmOc>C=hUjFHO26g|+ zzM=o;jyO1#^&$QlqY<#beo_2~9U&|#tfnihrli0xsPwOmQK|awh^dPFRYm0S6XE_R zV+fx~I#Etac)b#Im$}X+m-7uY}%VP*Z>F?ks4}daIHU{ai@V zn+&Wh)f;h;Y=vAydEwvc$eem52~?#gd@{md$;c_2Eq>Dh`s5?!CxO(*Q4JJiEoFVe z^1j6eq$qu4qsPqbRO9->y%uQ8$et>Lf-f0LUmanzJ0R?y@-Vs(`{UX#5M30X-qO9x z%2<7VPM5ITayQb*@}A|G+e6_B$HXAp@;BPPSDpriI~ap`rz?P;rIc^X+$N`gXjt}J zi!w2h!H%a-vrJnOuB%2Qv&;vz2u&*n$_UDwqf(nrGf}``U?#x>_GFgePNY+t=+(z) zF|I1Q%+Tk8<7X?`H<=GR+R*N3(uI8m$%iKgAN;aftp*@WXqwa*sHYnY=R@m5!;=wE zO^b*3>z#_&iL2%AOiGfMK~O-`p(j0i!_;M_3aS&jELv*wsxiw`B4Df+`S|FjGNHjG zDvTohhCAe!^w*$7cTSe{1&6|7;X@j&@XI~it(uf-phVV-XnH;AN#V@lq?4%5GYr=% z#0>3VZTu$M4&(nUsqKEneH^YMvsZdBIhR%}*{fG#v-e}VIMz=IjZPIj3i#UBcKHiJ z|01tH5r?6c|8ZhHG)jQ<{=O<6w8#~S+$H&}J=zqzW&|N*baz+keFknbnaHU0AWj}4 zr0%sc=?bKfqSm2jV0h{olW#L`lF*u`2IpGRU^^Abw3o0MN}>A9lb%dHCTui9b^8$U_H3Ogzg?J2lQaIIm5?dZ!YlmboAh0RdRUE6K6v)DVub>tzwGs zvSrI|>?a#55XU_Up7aYhlfS|Gd3$qZSaun$yASB&)*;|y|8t}kNMI`!D|^Q(1H*56 z>x4r{sU!L?mf!VT5hNyZhM%WHSS@t-P-oJZ<*rj z6A{mk2~Spa>_lh7AO(Ny3T9gb_VkOy{k^%6b%xW&72C@P6J~4Z>DQMpOf3u6-FE&p zzMRPSd5CB^rOz--DhPQ?AZSWBrfUKm=kYTAOm}$%tn%a7iuO=2De+4PaJO%l(|4A~ z{caS+hM@5Uw{VNzA~$xz?ahH0dhBvdX0(^bmca5z7Q<7uuw@jCm}u{BQC9n!2+@?r zU*#!%NcBy)!dm_Qix<>-fA`Upj>>{pG4%crOiX$YnA<1W%FfeBj}uMpqJJqS=6y23 z7u8H8p1US}qfXw$A98K=V%h8w&{l(dKu*8@u$sf`U0N9gU^8tu$hkYdMvmok%khXS zq^vv2Kg~L|?G@&aJ!;r| zbd>PVu>JlSwlx3CVXMn8CH4Og+*g->25uIy$H6TTi_ImIbvzlQk`8hP3$(r1S;?O> zf>nGL8!U{uTBg|H2$O_Vd8^>eB(LBkVyeKJ0$)Ix^$~k%$Zk6}P?&EJAJCi5*aKk) zBu|*(VcRW_=dJ6jYwpj_ZQE|holjR)VO|=Rd|2H0{z8e93>$sk5 z_ourRc)HNWcJ^SF0Rgr{_fuEL#l zz%<zrE6+X`X->%`v8#%!_aMo)D}>ISbd?R}x*>rMDFV%T@OhTcc)(n>{~ohr~WUs-nuD{B55Di zI>$}Yh(=?lfQ-yE>8%~e z%^4M=GbE3~{gEC3wjD$Yn8EMhU|3sSBS7l?xDNI4Q)0a`#SflYAOiuzZ7ggVOx(IC zi^g>$t$do)$}`7Plg@Q1)e91`$lswBf`w0^9kDoE>?J##fRl(OOG}@AlDl0PhG(IW z1LvyC?#>a@6X@pV9{qj#Z7u}XiR-&&%sn^Cn{gZ#NVa@QWsBg&J4KQ3iX5Bj^8lX; z%|WDgO8fjQIlEvusyEuu+x_DcTkPs;{g7SbD!6qOC$}dp*3=@Pd2XAFj=#fK!q2MU z7pccnSs6Yjbao<62SkULEoAq)^LF1~eEo1OUf`=3br($o{BSK_z`ZqmS2lzJ+si;? z>dP5}DCy)ZjU}4jM6g&*jq-?4s5T=pEJ=}0F}#CR4{scyyL+I0J0Cffa+*~}QAUbT z8_BD7lV1}3bS+ije@ev-J&U9r2DZ$Llfb9_8L@$Ec40E0!O`p1+4 zG+8PeG>yNTdTUrAa(o0h%Vbi zUnNJIkt-yc&M{>G3x68f#IZN)9M{X9U9m7#A=(a=@so(#Xp}74p9GNP=45DaA8fW&mKNH@ z-5lm_+x!|gnbZb#>FZ#823#aG0bz$M&_{C*Bv|8nVk1{V)i*Qs%%C`}t#-%WoK+CB zlt8K@EbDBl(e##yyeG$FNXPRSS=xkO z)R^H_q-QKH@}>15Sje|E8ilXP7={d!yv{kracXP?o?F&>KNVn;OB6MYT;E;ZQD~fP z06aA~94*>ygCmFKN<$x@EFE9I_1S0t*mOmr(*>C=Jb7j|65ElfkTxcoQa#;MZ%l)F zg-n~_tn%Ho;DU8^ps=&>kCeyl$7@07`ZDG;dZ6cNZt(kK2{gAZ{x!sML4XK=+9xy{ zLgML3R6*#qhq|LY2opSlrzB+4FgJfUszz(0=Za^Fqi+{A%+(bnI^ae@ zZR2CQ`z`5Y1D=RpW*D}oUztk|W(o>t?}g3q6WHegOPI7D(O3s0g`QXv{BEJh*aX26$p4e1Yu{~$Xuwe)4)#kQ&k#Pf3KjDGS zyiO#mP`VJK6_A+B&%fN&$sO6FsJ%UWMRTBwAdg%4!nQuyUJo5KS;2t3B*F{Iuc_3V z#_re+<#@0slkE8^UWXyG$8rd}z+T({tN<-cF6WC`C1JqT9XVJH=vrEWG3mBTsk%Y9 zfm{4in5)Tcu|90YX;msoXBnam^G0i0VFqKaD)^KrMg$nWp|?lpcPpy;Z>v9(B4<>Z zDEb1!=r%f$QqRHWhCqsnX8{P-e)SSBB&(ElK1FLmVjfism_im{9oAmT zOSop2RQ45v`QXR?MwO%_ey5E`o2X)dW7-d0q1w&;fvU!ksFx<=6euQA+v!xUZi>&Yqq0DOhTr_$89Y7z3aLg+z*4#Hx(fskC(Zi|8 z%q1r!E}wbIN+VnyJGT0n zuWdF?td$NOF%A}EOlhsK3hQ5HPZy6AG+eRQ@uxVvba)rH_#q|Ux>6rgW9Kuj&Qqi* z=sT=2M@?>V$l&ioqz(E}A%|$%%vz+!xu@eQB1LD`05)3M*;+EUrc=rfIW;;g`MVix zMrk(NMVX^%o(;Mhokk^zn=RoK1q^y6ExYG{jM)c<@)iMN^m>caDnaw4eHN#+?Am#u zWQE-D1I`Mq79ut14fwRxM-Rlc5~jpc&UBP)$SSuGco5&(U}2re@u;{4kq69N(fg6? zr*}Q{R-)vF+PZ8?tI|r2>2UaRSMCFgOquzr<93TZx1C?=N9zL_hm=3j(^GDf18eW1 zgB{U*Df>G^ZpeQVP5P@b1+HKLY8+X9P19o663x@Qn_rR8!!LY}iEq}+X|l4WMnTrl zN!Yl2`%`}GK97#m)xC&oV69c?(r!3~t0w>4|BGFl(R!8OPjG7qXlb=x%z8=~nXHUW z|2+>m`|>2+I*M(?5gIEr)s>BlATaT6qxh;ws)~_bpugCE zv=Sj+b8fgWuGE~Y&#!#|7y?Y_U%8PGu4agI&!fdUEw<3|tk=v146*}W^-tlC2h6_)|T@pwKYoIZ| zU7IWq2uNxKp*dg`n(+q`L|7*&DK32GLsx?p@kTkz%~B63$0%1(@Y|GF;pe>~{wX=@ zMzMDO3ODN>6O--gc3n4}uFB^bQIm)RiOGlNM}7r&qks6PM)Egq@XpPvuY}y2^){Cz zXZ&0uQn%t{M#)R3DYaToM@btuTZ@p*)d9! zqRM6E;2O^I8EqXvhH6|(SK0^AhR&@;OrG;NZql`*to@Y^-}Ver=RC4)em$=6 zZ}C?Oe!l+V1PKNHD475mPMrJu%yg#Ug)Gy%1lX4Gam)Z$lwn+Ff@hhuIAYT>P|QKE z+$hLRMvJebUPxc1y|lm_dyAql$jFPxtXN9&C)&a!&9FvC+ptTbG_+V+h-mkev41j= z^`+E9wNr47%BICTNfTL;PfDj*mfbANZ*h+yg*C;pEj5fjEmi2HL_-` zTQXehD@sKSYE%REcHh)FirGs?6k@gOJJa!w*Qy*!sFUJNq8?%#%AEO6keZAmh)kxD z^t#q{zMZ?S&5wD$)ouAF^t+H$k{41Iq7ax`$(>p!@_y4r{kBW#LhlawtLJ#?^iZ6h z+G}N!01eMi=L_p560YaFsytRas>Zg04YPh5<T;xLTVUf8u#Ug~|eUfISo?kKALxUk!Uy70Q1aWEq#rsu`;($*a2o>8dD zYNPcc+8Z)9?RukAD^^mC_7jNiA!wb^o@kws6R_I@<}f$3FCc{Tcba3*32Lp2eRC|< zZLuc{3kp)S;WEo%k(Nwmx~U>0u!PKbT81<%Sz5XM-Dx%0cZ<~qUqrH4}@7Lv<2sVyg4JsV7P zt^SlSe88F0rgc;awbA{8#!7eHh6MqY$8y=Jk*YUUo>UDVT8lfFW}hBoQ8JF`v5`S8 z`iHA@#@1niu!xc!&L$zguOj)7^2%XB@#`FXnIBgQCj5BOsLh`zn=Rhc)H&(W%`q;{ zdjiIYkzw>zCeM;`!wrL8Thc#QqUaX|Vc133`2sxRp#<0!F4;+I@{>n+^r9(cDJhK5 zG|{YTfr#h@!>mP0irK2D7(d7LAI_Wln*Ph%z9YEUee(v%aj4vHs z#Gu5Ms&og@Cr?5&%|Yp4?bjm8=?OmCm2xIaP!!=u8VN2GLE=AFd4?Ff&8yyltj03n zq+9U?ql|wq!~n&aar*!*exR|z`ZdGTGK<|7A%swq=i&7o)Lm2QJH&kdEu)oQAYXl0 z2}bb`gr=r=09XbbbVJAhRL{c4)T(3>?q)q9C)&4=xzmL0A3;qnv@HkF z`7sH1hOqm}!Duo<;=%Aj;x%qQe}2ZW0UKXk zhWp|GTpoai@lo*AH6X*Vp8I=BA5JJPo#nm5)8a?ioZkE3ya}nF(`*ZLMpr`2J2AbW0H6Srk{h_ zTM&QKh7hx-!Yq|X#ZBJJ)x!wAo<*fUx5KZn*J&!H&386t^aG-Kw(+KAnI5f@+5dH~ zG>BF*f8m<9gaIfBh%&H@O7#C94OIVo&xgCRykfr)lHY`8y_ztZnt<9O*6iRgs{|@J z1qJ$7tZUiwx{{`%rr!)*RCl{yZzZuU;pV{2zOw#w-0tN3`TpzA9n3DCBn`@{TD+*i z5Jt<9gQT#qa6GXyG2Wgbi&$kdhv?<-l@7eMD3=B+mK)toyr-Q=1Hjhh=M+Ej3@g*| z^H5m6S(}04iKxR9y&sjgHziUFBVL#LN);AiAg)|Nt8F~_1QhlY7;(!1R4x-bjl9m9WsJXQBx#yJR zVM01#5b+WMr@aG|#)~%spjmL8|M?v}Hdu5<&3?3kC1o>|5`YI9$teoq{6^hQVsy`g zrg>VVYLRq6d1F*-tX-T2-AAH=)=V96Z$niFc&vJjSVbN$U*Hs@2vb1vNl&B-MR}#Z z(j*CiUlu*F8{K*NUn2G2v&nODy)p!zjXLmbi2vi+Nc|&F`~AO%Q>-QgtdsoD;YcJ# zK*2v3t7)m~sOb2o8YFq$LcJ75HHKJ$of8~ib$?@Z^msmaxWey+c~U2Z8c>!gWuDN&f96}^x19+AH%L0yTW)5=2IL?+d< zBAS5jqB&u&6ZuhGe^T_2nZ6N6 zNb{TpM=|-oz&_!Xs7Hjy(UFKNXF@|`4nSI)l?lVcgshnoP{hBcR*LxK9$^ph8@E+9yyhr`V)6x0oOEX*ZAJfi=FxXfnNDRe8un5IA zKbjUyieZyvtgKW~tq_V4x)c(u(>T(@6Jb&-eyw{~y3>CTaO_@&3Q8{iHP!s6c$(SM zE1CK(2X-u>Lw1ZIR`fP57e=|oB_ggzI-M7TO74=T;jNM^#= zgt6Ig4@wPc4cY-BDkL!o+DYDv7H94xZ`ENT&%y<4OM#!ZCe3GQ3SraFo+>u2;I181 zVOv0E*_W{qW@Z;)P~fR1k9K=)p&!%IT{gz=w7w zw*{L}-}^Eos*FI1wkDa2D^Po@AYywb4B~~S^)o4{o}#gTu*+t%$eTx-E-k?iSS1t7 z{BEhBG;bY6(X+^BfZ? zPmqwGIiv&lxo?^wwTd->LDOS%M-GyJDX7T0{I!un_k{>GP)4ccL?;G!(89Zd8T{mzwVx4el zi@+viB7pnxikNo9qOiDGYo@I>yDV~$Lb+x7g*%Ciw1IoKu-|gWondU-fIUVqH@o@( z)6dXTw+QyMiMrB+Q~&XuU>lZ42)P9&ep*?xZc2xu@Uo`MF&j^{*b*PI%^@GK?bKUP z0kqrr;86uniUMF+tuVm?i8_@SitqS()_BFK5^ z6$OC&4F>fm)UQXxXs2K(HS}TotCcU>6F*EVXSBRS9(6~|F(PwpNGYaP;r@S%Y zCW{uMEHO8$KX^L=o~T?~s|!u>vtwyQxQsUz^o1O)Z-}laFGm&k$+Sgza4o2l5F>5| z5ZsiLJVM*9-Q7a{LJ%aZg7}``9gQ5DkU9xqmtAA)Ye6w->qcaOgZAA7YK%co%Rx0M z!tdw8sOGbOe&v~9FRFfBf4ZElu?&4$3_yH%a9cfTk20`PME>&S5!D#r^)2JY6`;Ts z*7(>S!x<+A%^iH^k~y?8nD6L>xO2egUqDFJLmDW^6x-tj`rS03d6(p3kbF6o3U`Pk zbI=^4AH8IkX2B`CdY8xN2QJ5VJ_rl6*M-6B19v%OpUq1K=2!4~+m5udkY-i#NrbrS zti;kS)uax~h;m*D{*xkGE3ViNnPtS}y5;5}9n5b!IjAuO*Num~H=<$6jPp9YLsG&I zJ^vE|{s!pToGy8z|BC>vtn$w$oBs~ba3M0~fmCI_Wo68H;n9{+6f%7I62zOBWx&gm z*1yGPn}1%O8vnfb`an5F@gpc#*+`KV8Y5iXH|3L5l8hknBe&ctVD}fhBnr1X@lc2s zP}n`J-?YlL_UU2}muD;-Yb5>@*)Gl7|1}twujIN}(+AbL#Tp%OibHRxl~Wh}cBNm~ zpVyN8(feFx+@t2ROX{Gtd5xn!LYXY{jld)fI+Lg9ic!4!`o&?R6U_yX#>SmMhfjl{ zcIb{)S8gIZ!V$saQ8y9yZYcuTCtM0Y;3iB^bn!p}cu{h(wI5~nebK;9@wd|4yqLp9 zAk^0pqgFT9#+SaI;up>M@p_~r7W|R&Tg1qPID)wYI|=(}w2yYxRS7sT_k}5vOWaQ+ zuM-QJj+ohAfYc+*<@YT_K6Ae*l_D$3s}KomzMt%1{GiKj7VHb{zeQ+O-!HE}fazd* z7~oox|8Rd&Q&Cltlm6!sMXO(^0(X*pZgp}57kA3&7Ns<`q#>QQG%M*)(Gyv*@sT0h zm&Z=HAjooN*8L3F?b(a+_e{G_u1LhlcK)qq`MVHrMjr(lRHbLr=eoV;LEvN?m=6A< z2-2z?kq2cuE(}7*v4<}w$rKbz@`44dtx8Ro!m1!=4QOFC zy5vR*4Rsb_9NufQV#I62h3|q1#K!d|Bz=hVrNZVgl{wTKMPS$b{N|Inr$ggt$!@Uw zmN5$}p`m-ku)@NgrVN-(GseE=3KGOHG=Y{o(u5GT9j_bIV{Rq6gg0akUx**S*wou^ zA)Z0E=+$S%162M3XhZ6_y=)+-!UbX$;`A=Zb`=CHIEHOzHIXw+!{SppA+T#5mywd9OQ>O|gYG|$bm1P0ie z_*Q5%UJx|>fknG`baMNEpEY&Wk1a>{D)ektS|J4!vkOcyqxoTlyVOC;T;!71Q?*ra zTjU7c^Ynn*GzSftl_ZV^49`f8Ay&OwJMY9yR&iNETO)J5j^w9#UenS37ByN=)eKE% zh%VR^b-n>jn(kpvc>7MB^$9X(i6qLYc$=-DtNRs3 z(dzT}A>Y*OOd|82>8llLa+E>NMP^FkR}vmVA+b7Us^H|KO???V*NPmTEFrkJ(PeYi z(ToRNQroo2yk>2EY_?X%Y7~bR@k3e)-Q-p|c1mObZHX*+Dz2n0zVk+%7Ib7$Zelx6 zn{kOLs};c;JAC-uMZGJ7LtH_N<zQLEKu~A;L_vr#+nNb5M|*y^hkbl29eGNjen_lI_Sr+$OT8y+SX4iMT8B z-|x5Yx0qm!=ue32C;>FCn0+z3n|<<5P}$)|M-*y+hR7e^xtMMt$9H#BR(g#z^P+Bp zb{VbC5xXUdqK>=PK6YmBp{Du2g)GOku&hIkpWg0J7dA#cyr141V_p%2F)(<;w@7Zc z7-QqcF&g%59KA}u`5$RtL(Rn&CNKYdl>CakPeASzT=xE`FW9RZh4MxWp3iX$%|UPg zB^Un)?%z-5!E|MAk8GTv+y5RGzf2rOtdNxZ(se2s=!zhR!11=#v8+hZFHi`SEZDZ9( z?}Q)sI9ojhXT_RNq{%FK$TO3EEDX3y?5Mou;MO4*Pq!lFp z@eWMZu+>DB#PX+rS*xoGq@r3fXVF!0e#V*!E*j`H8k(K%ne&`&ne**r>i7H3K@Uh)UoFkPl>4rH z=Tgjo)Ckj3fc2!*1UtkE`ea2IaiK_GRs?QZYF8%gwnjZ19IQk6Y0?!h?6&$`^jEdZ zK6?m21VPau(Tf@Sq1vHwMO#o{gaDmKZ@+$Wq<>kjh7wv>j8@E%AsRHL!X%v7@Kz`F zYyjlS_8 zggWO9s??F_h8<{|#w?W~Os=-6lu@UF;;LTc+06*WR`@<~QKAi!Fm<}Mjw=j0*c^@& zc|}Gjt*&#CWRFuKC+`ymAGi|1WAx+Av53S>78&yMDORb)>9T#@mYkOHZnPIf;s(aq zB_7%|;;zg_%XYPyJTp}PqVzI7epMFN0nAYJHn1z~%3Un~p~8nT43>@d8=_~62G6Oi zLjoP1Df5gjifil^c)t0JI2UYoRIH0;VG1qfgc(I z*Wj28*W2N^2Kg#%_&s>fTQ3se5aT`RDDbK zTz&g4w?7cAUQoV5AliFEfGb?lRf7XOj-Bl0S+0Dy+D#XvWVvI`AgU%C7Jt#czGkGq zlTi-y&1j{B=kMWiH-A!}E@4Mf**+oa%Xq(c@fVdo?iFsWL`??i#eJ0CC)8(tCkslf^D4*#$yw$ zt<~ofG@#4TORt@B&Rg;@S0b%gWV)oKQY2Pn?2$sJI+c8HE_-=%n6JuVZS;h5tM}YvEJ6pI zdyI8wkcKHi%`#D1D1V|!xkvz27is&hXTVo~JG$Z}De>nJnc!q;*zacH(ec3>IJ`5v zkS4?`k>Aq_&?7I=ki%pKhnkgOPI!#<==Tvw;+D=J$m^htuU|dLSCIDGKLIrXf=Jgf zRr&PbcU0I0qHMuE`Eul>%?@@vYZ^C*;*g3pd45h1Hip63f^+p?S93}5A_vzxX>Vrh3f$%=9&hZ9?qkAYa zSa2TR$+5KuSc4#*1O+3=Pl_aRZeJDZG4m}c$$6*N@O&(tQuwcXN}?~b4>|%b0MdLp_iSD(wIUui*ew`sgdy*UdFp%LxEcwwNJ2_L}w z?JQU38tcY;46%LC-%BBL>FNT%G{<)=QMJc(hXkPWwFU+SRdO`}PDI<%ld}3jOa0Hm zCU1+q5%kr5=F%XlL=sQVxsdMS)HakkN8bhTifnl~JEhAQ?kY&wwyS0TT+ExtVs`mF z&^Mi|;jCC2I?XrwoC`RFeal3b;h;j>Qlg3or}#6_yqdVE)}nbQpquQv<;&F*`tt?f(FEtE$WY2RNl#NL0TDD0N%G*e4pv1ggTr@Nt;|fIp7|y43{96r%5W zU%Nih-}al#A-7+Z;djPC^>~v&r%toBT>ngqd%vFVoPaPK#6&w!uD3z2^pnG-()-$z z2CdWLgwPw94DXlmkj|^ZDxY+-%qp;r1loh z#4qYtjouKy!o=U~O$afE;#6*|+uxPkbW!3o18c>{<5Y?y)S(0OQViV&Sq{+c;RJaZ zd>3u_F1F%Gxxz!si~kg&N)^o@_f4!xldF`8nNMfBpuFJFhq!7!&yl2P<|R%lPbd}O z;HrH+&N1?>jV&WFrk(0E7lP(}w~7&Abh&k4H~h5#a~$_2;(Mrb8YP4=N3dH0XR2PD zWr-*8$G$Oqs8nP4EhQd>`x_)z;LKP^I(l`~gCKCl%FNX7==xIIWK3Hq*M}v`QNC4i zkNIE8WiMH1p|CvHy~~qtpLI>i)Mmm0~Roxskt5GazaCAEfTTAJo6g zDgPsN+sFo(1k8h=i!#Dfk)-J4b4GKNudC1-?3%I2|F_glbKY3{0cI$?0;{%+lIwdb z+5UT)>@lOn_Lb>ma_t@HbU$4AJx%_@>V^Ya-2!qWdBze08$hdD0baVZ67bA6X#=#n zx&B(+m_zR`L=j(^YmJ4cR+9G$0I_VoRmRj24sw!zt!}Y9YJtdo*G)Cqb&oxbq?#g@ zNvlztI&hso=GJdur55^(mZJ%54=KtGt`Zt=^IBy|u*>kEHJ&#SLK|sUs~>GbRL&mC zM0{r!X9cdy|FpUx{#xDq7eK3 zt?BY$I|rT{pO(UX!A|kECE?1eB$_DB4q`f6Eexg%)$TJphDMY0?gHnlP9d{Ny2XhS zzwK%6`C6~&X-+>)3=5hV?q#!7XP)TkpXf797Ix}t$QUEs&5bc!3g@H5Gn*V^nTWWw zI1U$7d1V6uw6i}?0CFqcYfH{xe|Tit3i}q60k!!B2MihzZ_i`Rfz@Yip^V@RAGyA8 zcy@7O0i?VS{CyKi&3qcant%%@ec2U1_DgilD#~M0X@}ab2HkzbYuS9O4{$eEZmVN% zoFtv!yJ8x#7VkiS_mcQOCZ7}IR8LZO4`uc%Qq%U~JV}jOo}s8?fF-ngzllUUaWMXH zp1?lLC^LNpxn~DzM=^txh2;#w$7)- zg&&zSD6<o0PRDOwT=uGtBl_dn22dUD{$l(w;O!A(~O#S#tO#FN9;SHGZsmTnBZputvO=W_wU4Qh@pj1saz1jVSeM7f@(u`QY{E2W?>69M!_xI zLC!iLz;BbWnPH6)NBJJGj^yV7Tyri{edvl@>pYpbAiK#f7K`5&#d%6D`zRizB5QF` zRoP0L1Q)fW`eL!dZeh}y5)M**Yab-4j_0?^6wA*(2pHk$Y~CXMT&i(EllX1~W2NYH z>DeR*tI~7k5>IRh*OK0A#-zAsKCNsMn^?hW_4)OG(ujVV$^5ePeNX=vddfe3hW{Hq z1(>e_>M2qU#@evO#m&`C^#urV9+h2MFcNSGI7bD5g45E0W2_d6pXpN!yI!A?)ziT- z+3`0YR3j&LBDgTSM-%OxD<^*2uCE6(JN{6&klk#|0No(2=nabN-J0-Lr8XTe~aslW3qCQ95|lD6Dz zJ=6j729X)=%~D?kUrLDLa$Na2fzzvpCCd;AaQ^BrQFQG&I^j5(G+K$7?lQwEzLLu; z=DX*9nrZN@n<+okpG45v)RvO-BU*D{G0q#7R2CKR0rlAJzh-}3jt?A-0kLlB3jHJK z%?bRkU9&(xnKDjE%&C>J*sWW`U}<@;@GsWA1jM@cyPGb`N>b)Fl!RxEg_5MD3)L5r z1rs*rYh-IJG=W&R4G`GV+tSs|=kwt04Y zqP}KEWBn`|;0~WaJ7pRVXs5gm=*>LTsi_k&DZ>KolxsU=?d3aMpq*kzYh$L*Xlsm4 zi2E|wUs{Hx)p&~p1XLCVxCC1N3+vv6`zO}Tj7RVTh;?_a1F`PAzgV}xE$MdV@^>qb zks!aZj|F{;R=#Fb+5HzksN+MQ#FQzDzgYLm_wCYuVBL>SK&-o*sT+uOuUz25ow2$| zkuk}oMr;tJpsyK$TDtzlx?%oe-QU5d?|ozK>}hX3A9>YhziuF*CUt52#kwh7DeIhX zx%SZos!e?h*ZBM%+hts)GJEg%JUJ};xY@g)?t&x14ffMwLXa7pns^}F!os zoosURh3rI~iBLN{|?4|;Ra|FHH*K~UC1dV4yboV zfS!jkhp%W|K;L`euk6!>IKX%(51`4WU*9vcdgs30Z#7tO+rB?O%Hfj6#*!WBjqHCL zWUKk)Ct(qD?!8Z5MPrg$% zUu_wD@pqi|GG9{k{5K$YuT`8{H`^uqlp%Jg2l;$8`<`Ky0XGwSTEy2welCAUJ!oFy z;Rwdskw%W3MEYX0+YI}@xVWqI1;G-y7&8-i7$XxX->PD!?AHZK9=$PRrkIu?#U8*w zl%nyK?@~i3$}U?LtqUd&)CDfU=BVGgLRf;|Z#>rFpgFaM4rJ7GmkpcfOzI7@lNPp)bYE>*Wq7v zttW2^2F5Fp?72T$c@igUht(Y&No2$|4(I6=5AZJFq{XyJ0$M(=kKL zIOEUmV;~(Z(YkEuSk03Wqd2)FgRNpD8m;~QQ_-7{wc@Zh>%vsSjtlTs!Vv=HuxFx) ze>p9=7UQZ3ZE~{(rypD3PNF<|(@iBMvg|!I2v+tH^pYSaMs#z`<7i>L%W= zA5N5v!3jH=jz z*BxxEfjT#X_5i`oo)t!M;8_ZvGwIcu@QPac`Sj71q3BKQ=wx2Pc5x|GQl|l^VY3x| z+LDh>ADrs@vFupZ!T4_0kI5q9Dpl~)n*?T{wE2vsA-l@ySHfg8!b=q{f|c8WHxuu& ze@4X+(zr4J{MceS76U`8$AU>&7lG774Ie9yN4;ArOuPSphC(r!pvJ`%%@RGT{f=)K zy>oY%arSDMo7=AUTfmSi=5?hC#ALK#fv(YOQ_>lInGKFi<6!$8ZRcnLk9@fE0r~Is z=u~InK5IPJ{#^ir!o9qSQ_A{@!4X%s-17fK4pVJXcQ|*0HbMCXvrb6qShNTjg4Kmhg7T*rxev zWo#X}Ts*X&k=$}oq3hA?8__ClizsjTo2)=bJaGr+G4%XdmaZ{A$)Gkbx500Ghqdyx zkoK!>J>unh1I&G^7IM!TDmRC-JN+|+rREfsShN@7x6=0U0g3hz?8Alw!_`4X&jgey zt35ii?L-k7?1S27-{}jRnSPEiI#rFJJ_Td1lXNwlQ-*2Gsc?kv>W)xD z+q>we2qTVSbM!9BxmSO>G&~=hPlp9VNN=}%T`)e<^{M2 zNZ#-V_%iD)I_Nr{C_wzw?WX(=?+PAMXy`wMPL{i}d&@=P#g7mn+5UqhmL>4j(s>B! zvY_wG*&x{T^nT*$*OSdx(C+yaf#7jS^OlE~w%~ds2sC0k!_nI_?cQTF-*O13#)F6<4Rgo)^Mi->k#epxb?77k;@g!afE* zndy-_DAB|387MhI@TE*)ly~NQAwO^7WA@o%_Ov%+SxZ(5k~+{Qyhacnw-}Q4;F3wrB_C8SH)NHv|#an-ifoWD`pUb%kTj*Z^`$W2|ZClGp)7B zuYW}N15y}6)DspIr^DkKT@4$Re)5N`&!y)tXX7@9{7&pmr_yIpRuO=aDRn7m3Tjbf z>2_o}A(PpnI`N8G?vY9|D5zza{fOlI9zxY6Q?rup;TU%z@BL1H-z|CFdKT-+^urT* zkbE=Z*%^y)7y9GJ&pS!feE!FNcWON}6@^`tCa1gNy@s*=;+g)VvXBX6xYV|giEcW* zw+1KVMUps~VUEU5km3WaMb4NZzk)&52=c-<8dkTJ@cM6hL2CTDzDKZ3hum#@h zfoql`x49rMmLj7mQp90w%*#CFh65Yjf_hNudyw;c4hVh+X+O`fDZ-cX?F}BZg|PMC z`?^pjX#@|1r3pme`w&K{S)dhnOgOCR^IS0W`*)Na!{0NAP~(`K>{E*VQ~*3{d#&{Y z>MTn5Pn*_!VU;Gq5xUC#@QR%@d(FpHrj*sx!B;rW|Dbo@#IBFb&V(>RlAx zHD>M@Z)=rr?7#qIUJ=}uCnMSKey%4!Jt@Bb{^5B(c1kMQw?EOlrFB7s+NI|1J zAZ3=1JT3vwmTkhCiucht&}xZF2n!0tJ4;Gv2%~XBARnUZ^#h{hv2&S&QxGPc7T{HU zy+euHY=HZN$Sk#d2P!ez8>=zp8{kha6CAEjV~CIGO=q`Q&*4?MH_ENNt1Z-; z3M|uVjIAM7n5)Js?33L*!#7_A7@4i^6h0fS&(~WtSzqVfuJb>nR^+xSns&j>ApNuf zLS3Uw%SZJh8_DgWF(^B=%z8Hwl8k;c)eRI+&w2wbZR+`7Y;4&KHyH0S>X7x0V;0ba zDwb*YXS)LPkK$WGJOTvxE(WxLOaP&2v&1VPZ_b$Wj*;r-7?S2w0w#(v1e%L zJ!!KsWYFzqsPCePE9}dhf-f;hF02U z>TbWE2k)809XBOLFt;bA+&-buVW5F1Zq z!3XwVK@zcpGLKAed&|=^n(3~qGQ7K{FGzA4tlA=+JI9#rU{^BK1WAiWc(7Yiw(J(H zFDFR~Fbf=h_x@=ObNH#YCnkt8^Hf!BW!Z4{Ha|_U@_=FAiQV73wM zW(KU3t7js%1#*^I7Oaj&_S!}Mej9xvdquIi#|DEJ%#X##6JEC`N=uzTkiVEolZz1) zH3aF2{H11lPb(4{ck_p;-tgmt)jjN1Sl}0lAew<(D;5CCir*Rf7g7)#N$k)r@|)*K z;DJj;030I~$SzpJ;|O+N)sAi3!E=93(Ef%v<7I&(0h7Fk5o}PK3~&Q9b2JPYQxMs= z5-UHPRzC@Zi!{sinqQ2v4Jz|hMcBk{*!@n5hi`^q(@_sY1=gUXULCU^!OF}s#j)U{sxv4!+ zJxAAxOsKgy{e=40AI+bU4~Iu!SNI#SHGuB_^rMkC_WIYVl(s6WFwzGAFS82wT2zdU z#^y(FAec=rmC-EK$QLwqKEw$%tkugB4&2e1{?t5`Vqm6dB1gW6B;2xMo`2zFW#n?j zF<0PA+7K_ke%d@Ug6cpNKpP>}O^gYBz;x)uZPHYD1eH&H#ga#4^z?MF06>l~gIypy zAMc++oP@Ju4qe*;6_{_U&tuZ4vWdBkxYH^<|MOaBbNeVq?HRyyNjT0TY9`OKX~A?K zL)xjHnK}_TSnj?;o7Op4Eoz)%*0)(JwUHXre&sVyZFhUFwc92YAf~PbZxJ~G`rO{G zO1);{?E5Dy*JRqr&v1WVGL!ejzVxNlpKKj!sr}ieEl6~e0HTo z6kb$wu43ZZm1-Mx(pkMx%bt+_gzU33?mW%$lv+%Mn6 zJMNZy>jK|)Z&IR+ei!Hp8p*CqII~32b`(AM+8~*C>XEM$qh_z>)Y=ir8Q}x#% z=7^_rr~szEQV&v^4r)P{8*4c0etfuV-Lu=N5aFB^5!xxG%_0KRkBh-{94u|>lUOrM zVgbQdP<%_=Ge%tgdbuaC_%2s5pJp5rRQ5sJ*)z3)X1N!z{I0#*vJ`>PyO2N-V^}0u zuc(h-hpl5e}%OMS2kNv@%|t?xQg%kx5`16c->{oG6%0E5~a9`MWb45Sj&CyK~0LteYw0b@T( znZ@)d^(p-hppZWi|Mlxx6CzKb0*2{P=>NF4MfJbJ^xv=NUrSKxw*N@DqBOMAh0KGA z4@!t3Aayja>$ER{vXQ*eL_PPx(-&2!vpiqfmj;>@B?XN+g5 zbGT?|qQ_t0Z4D!CwbFn{f|pNgAj`9$n{Y~zZO~yKO|y}Vt4$|gX=;*RVXhX@SfU7~50cH@$)1g;oVs+XN|!(1&B z?H+}XS_$g`-gJ%4!`u?T!|Jv&px;A{l)fTcts}`f6-d67u|S>m=&G$#t^MYU$r#Co zv(aQ5<4~1hK&i)5bBZtnjMtsz`^??O!X0F~L#V!$yS(T-2Tr|K!d?>zyfq1nhZO%3f~k4;X(b#vk!Oc^4nfnZ06}5c4RYyMS4cDf5a5 z3@m^}^?}VPr^>JDzfVzKrJ=a3iGcN26Ve^~^cIIc4V1x$Fk08KI8X!W3+#Ksmb2jD z5Fd0ykaU5Eh9}oWi7G5I*Kr$%&nj(*jcmwZJ+BdDC3x7^`Pj2*D9jw}K_DinkNTq# zu(okx=hpLRtX*C=2*x`0py_dsSFS$L&f2x;CPpNeCfJ!a$2#{ScK+BL5AI#wV4V_9 zPB|grgAqe1IoO=(6QBl_m0qmcR9#?t$_v9Cfsba`Po(G0X1x)w`4`OJ&vN)eore@ZTH64A`+PO7>X~3tyZAdu8yPBU z7}Ddv;*RB@z9K{KDuOqXr1FSK?IQ7m37Y;sMZ++oZOo6KjkgOnyv7u>#0l#fEitfU zl(fJ!%=bJ2NrimFInA}%b@}pTlikUj;}P~QJAO7Tfv0zAMGWLd{b=Fz&w|*Rn13X% zQOG0P&a^t<4P?P9OuI_?4LPM=K_Mjzj~bWhWzt(&Ksd?8&+F?AtGKgo6d8qeBW5eJ zxI$b8Xc5zZdvqtRoPOMTSDUm$*prIyZ^zfRERCAZt`~#TB`TIFg*GKh9zFclk*+tUmS&6~8uVGQ)GU>%KjHf<*}myXL=84HlKRdy>diOf zO5&P|5S8fKwX`-@23S4_ttcHi`mkSCXdOq7^;!j)htyTM!8})rfSw!5nbI7L>TlQb=|Jp!IZ7L?iP3tRI8^QMBR64GqCsxH1$5la6g>2iVMLV%FB z=s?>NMyb)Xt%FMifUzs;pF;{zXb;%>=h)cZ~KBDfX15IhRrLzjV5`9K{W4K)Gm(6}mD(ux3CqiC~ z?!^iC?uNgWp38k(V5P^J4dx8OHz#TL?f+rz9NX&(*LK|oZO}BfZQHhO+qO018QZqa z#%|2UX5%zzxMvsk!h5`HuMhh-JjWRKc+PR%=jG;DSs8N2KeW78yzIa6Fu!^RxAEZU zOvU4hAzAR~+sOQaEH$nbQ%x+M{&x2o&efg2r(#b8Mp?<=1wX1>vc`@yXWk%A!XTHL zpiP=0t7-c2hzVR%BGm7hCJr~Cv2BiYjSuQY_|@UyPMUofLbMi7YTEFzeG-99PEWdP zywxoumW>~Q_nl|R^YEzm9?YK2y^puv0pdC&3`1`>BRb^4F}`wY41)=Fzv&x&z~==YVW=&oTU@eh-pa?1%Y?)ho?6;X@{Lm%te84O))@JI zIL;o~&2mIh>{n(? zFA?L3B9T*SMREeJQIx+C{UX`D_8i3(YAIx4e{^4bUh^O=5U%rWb%ZYT|EJx|KmTmE zHJs*1K+p0gP+{AD3g-U(8{%*GeD!B{lttY4^>bsk%)k%`BEh)Og8h0!LC?`o1rUi? znGn)oq*hUh-|H~fvsb64;smUHzg4PPuS zF=GxaAz$@A^1FO+<=+~~{(a=n_lzCj`>2Z3d1XS}14cY$C?za5;DSRoWjSh z19y~(Frw@<%1Q{Y{S^oZ0@#ntf~gBLexe65_`spFOb*Wiqa1DeN#V9;uQoVZw#P7X zQf?o1Jw8oK5l@dD+|J#=I(aA#NMq?OT}1-aA&G=jZ1*AV4A>((N|0 zTE0x9%m8dB8cFmURT~+tCZU^vS?qKeOlPZ1VB_%%8N1AS{ zFfsEAD_P9UVC7R@hn54(O2~eM=D%DaJy#6o=K4^{q9>J0c(W6V~M(`ld*Z8zHDS^NWDOGrBjHIXGBY$qM!rI zysLCxq+z@Iq!Y^#44AfcQ_h|3^mt}-jyOFAr#`?#JMRqM5rY9m08Q4GhLRYieSsmH ziDx8Pim9hEXsd_E4;?lc;{q0e#|gIXsa)0FC$q}msVpmDWlUItF)A~~V)IQQ^4=0N zi?!%BqWsouIB_;(O*>a>`p_!dWSbbQ=IpB*JMBa>MtN}nCmxaOpy1n~XE9h%N> z`KFYk3M0E}hc&BDZT#cf&7D$o{R8c&;-X4xiu$B4*}Di;Icy`AnaMwYF9#@X92tg|fm`B&F^; z>UXC9v2NbH`Oc1MlMxzy8E4VgXN+$jKH$r?jh0HUM3>fQ)4y?F(Kw%R_FwVfl?*Ol zA(2tIqt*8JSufmhac*u;br$uAp00T)Z;HG;$Fa}WdLl}Gm}Z>*dV98yF~UH~ZWN%2 zOlXe{+#?Kn{ObN$a&H8p8h)p8omcuT0pg>w29Jfjjm@z zEw1#tHKI00^F&A|bub1VnJGy-ofQHWCg4!Wg|lN$`qIvJ3Wwfk^@Vy*7a0=bm7gOU*ivxAv8K1LQ7>qW%vq6?A-7Mm)PR`cbPM@za|K9V90vWIL-;*v&yMHBJ+R7l;|67XQ z+;CDR0a1afA}k6Kl^%uK$0)USo?eE%J*1_EI6E%O$QK%4N-E}?i&!U~Otq3Zub)Ln z=Q(HYbYmdW_3cAipEWXX8WSVjS}!g*^QVi9a6_sPMq`M6`NSRUd8))3_a3IS-1-oLe5?e2 z$w0zxB6*#6h@4m6hzRp}UAKnPyr(;)3sASS=p#-e&D(J?Pbgz;4BLIoe6`TetYg&n z&JSnGv6T-3UFNeib9m=axrR;K&pEk%t>(svXM`{)u!y>bO-3xf1t!>#tVjb2z1 zm#dV_BvM3B`XJ6_c@G!5KkqCy7)O)}r&-dixo4UI$_ngdCO~x!rxUJyGqUB$NGmu`)4gYVDg#-oP30Q&10Za-9E9Bjr@8!s+>>PNA@ zRxsWR~NK+j%F;0C?`t@Iq748wI&`7J2F_j({SNBY5RdQFNx1_H z6#UgY28Eg|S!DFS*ODjP}NE`cs90+u87*@JrV z_2geVCz;~2DKgc9awEwpC@3lz>MOU)Baz0bXzVdVtBt@eE6cXcA)At4YSlvmrpgx- zKa^CPF3Ce8&`7jQ8!*k+QxcU><7OKR7*MPx8Q_r8&VxibdD-K9bf=41>+Y}V*Wu{P zH}I*6xEmyIwor%>qz@U8X7;VC$9^XRUd8=Lu#+TfwdW?-<(hv1T@L>Sy5jzTE<+IL zDor)D{wwIh{=cB>I|y`n{uAhG{gRO@;WjorreZJarLDZ9%ZIK0zk#l8vH`^!CFNqip-o`M(L#Are$u{&<1j}a-}lOmrO z2y}tuXiOl`75Nv?#r_Y_mE2|hSI{N*H_-LimSqM4T_b-$m(0ueH}1);YO%kAt}_gu zO%UjU`v>UaqdQUsfv&gSnal8=t3RMCK92wPUqF}9{{&t6`!CwQ|263P%PZ%9g0AO3 zgq@e2WVMbaWNuE2+P13fwov}ux3Al@KE_5c{{wVkjb{DB97tobk^apg^23OEx&#fxE4OIW=1c5HKN=4;L5a`ms0UIL)fv%ul2{$`!f}Y(= zs=&L9v+@|V?t&m%&2mMi=0om=7Fuxit^z}}7XDA>_yxLN2y6vy=wX`1ZZw8n-TiKl?J~e$Q_VYqNfI*Mnv3`Raqz{uiF(bjS%4t2dEUrP0_bJyrj<}>;lK=X zI$3c1#Ht-ue)(oi&%7*!R4)f%g692&4qqJFPE~n8wE+V7P*_9DkDIJwLQNPfwN-ma zpq8s^D!Ck%tL{-NL(DDO)NWQ_La6*zc_1t7*$P72y*~yz2X}x_AZ#h7{^9m{g@8c` z1_*SmZV~rH?=6EsS9-^p>0@JE#Pk;*J5XW=&PMYQX$|e-%oIysvn`q$&F!kWxA~nV zBjgTbIFNSm?QB|A7COw`J-ePe`*!g>cAow9ubitH{~w%7EEvSOR{wCW4iM+sKM{dY zQZ>B8iz~IMwI?Z(e@c=t$f7fCm7~FKm;#$J{oD`-^E#zT!2ZG9vR%BT8U6A3X=nT_ z)h-%ZN{=}yXW*)FmCy#~TZ%`r{ZABpTR#G?H_yo1$?3o|$W5DzAaA2Lgk45>#;!qD zOlZ)qWaT0wh;v2!jdNLmIM+LBeJDAGCGndDuX~dpS+j=6C*vc8^72`>$=__lKR}$z z*oP6sxw50KF^S#7BZ&XOxjut9m;Q%8h>Ori{yV(Hp12yJBW^O3R^SraPv$CSF!c{1F;G^J>fet52KJ4f~y>$I58VG^RZhBmMilhZC+7TKhFW6K`Q{i}2w{4tz#- zNsN$|-|!GA3pmVvOYK8r$TR%LHi{+2%MI(2A9YJQ{PeDS|Hh^lag1QIWvkPF{BH^k zXCo^}vi}-$S=#*_bG^EwsAKxs{rW5BQWDCi36G0~gQbQ0`AG>{*2W5n3R*%dS&nVM z?D%4mi(49O(J%0|CK=6Ey0XdYyJ9R>6PYM*DR&>|{bTHV^y7LbzMUJn0ngYb_rq(? z{~L9{>!T)0=ama_4+QbJsg|(RfD8)Rl!b(Kn(ZIVWuA#J#OyT2LI|r( z2LxmRE=B%eEnRO@IgABy%Ymj+&1!{F zv}TLI*|FwcR{SW28yDkFdrI~NIuw3G86yG0@i`YI0?+B>jLdDBY35$&dDDp;);y*% z6*ncZ^PB^%Cmmw&X4Mde8<5M6%Wyw>p(=e;^JLJaZ`>nA_eSW*1%>74dy+swqngL? zRrH_t5-Jlb+4R0r`a5h;r`>9+o)(QG`?;`?PK zZ8=P>W53iIK;6O8vt|jjCSllSQ2KQdeNVu@`YK-LN0u8LwdyqGkx2i!afD^21B~4` zPK$|WRqfrC9%WS_y0_~cLCqeuhr)L6`+O=`wtFTX$4Qro1BH&v-MgVhnGk~#<8YARRoB^L*vl5aK zIdZcclZ1k#?p}wD1EAKCh@TWKzQpxfSt*tN6}uhB{FC-N{Q{R#dEArwod#p=(5jA5 zf$dP7s>o?oVQ)zE3{3&bbyk7q`F3iJu0?NZJ$BZ-)9AHz{*?I?PYrYq9+9p?tQ)^@NF{veM-U_ObE(nkPRnFkvx- zLs!9@NB2zP|MA1oObD!x7h2{uPRkzT(t1=?b!zygC=WpT5zCeA8m7AI)92dxaqJ|o z#C7*EPwd(o+B-D5+e=*FF!duR$cGzfu3-*;yF7+iBWPteNI*?4vdV@tR4B7+>J%oi z7Ypr$Kde5+EBm|P6LOUmRP+@UF(x6w+R3$}*g`J_ygB8#oB?Fg zg)r4(uxj>t%Tux+r6Lo3rt6K--Q|Oi-tBKvZs&bAnDpG}#S%r*p6Ly-> z_`&v_ZRF;!(EHK}f7bREl6tszypJm(A_mWZ=ASt@#`cOzn<+_on7|E$3IQw=K z*G{*}qD#SxuTg&C%I3~L#8mjfs{goGDb__Uc7q$sS~9v%r8-bhY(Id?mPUL6mRoV=Ois=Z7|Iv|g9?*DaeQs#Eowp1}8h{@v?z zSt;Q>t4YbjY){Yz;AQ$4zW?hDb%45q`Upx&80afw;VK14q2^L$Bs;K%wZyu<5Z~(r zc1htSJ@SdowJR!G%kCt6j~A|$)xL%7Lu<^&s*f| zL=FsP4Spxtf@DQ56D18xP&U)acAv# z>o&BA5!xQ%FJO;H;yM8EkY8#M5!CV=7Lh1&U^Z)(aOSV6*cvk%dlGvz;Z47@g9I=#+Vd`t?xHN6`4*Kj@{rCTn>Bg!)v`9GWLQ_c45s`;~`pJZ@8fN54A!)>?Q=#XY__m52U(`6TlJU?>Mzq=neKBd17 z4yeWRNFxfXDBZZuT!d( z_EeHTRsD<>2+9FyDi3$m6DFq3;-&O55JIX_wHvEqp8sC3%!D8VtK~(P$lWn_D~xLO z-u8xHi0Q1TIWA`&WM3Hm?nj?`0nE3WyfRD+dKx5Hz})Bfo-9cv@%d02f!`6S?-pl| zkU1g>C#L0tUN5Oq{!^q9%nSL!Cy86w*<8ekZFZ608*=YoWnaScU6AYX@eH?j+@7v~ zc^H|#LPflyeImaj0~}^DWMCXtMZ_nsAnrcT*Gp@4gwBqr#Lmd@6shOAu6Oi@!Fxax zd!%L_9=;TsEXv!5qbv4RwemrF}YtA`4m4Qd3W?&flvZh?|X2bAYvd z<1i=g2ovM>*=SIKNvF2-mXQ59g%5?d8yRX19k|(Hw}go@kNX8Y_i;pZFFm&1&bhRd zeLX; zB;kmcq+itWLZ8m^o!R#7Uq1Zn(d?^qhW{7RX3)|6UupA%TtBFhc%X7lO#sE`O5ra? zZC0^Dq`iV&p?*IK)ZwI*q?F6K7$kJR0DRLd%y6>2sok7(=jraFsmYO}HSPf9^GXQ= zS<$basG#e3r!1s43=Bks@?pbvG=WxiGqjPq-ZGg)=45l`PM+~I?GHzeL{{D?F56RP z$DH{3ylF8A()nBOPq<2z_SnKp%v^-dV24=MQHQeG0`*;e3|~C@1Q&gZjIB5peqzGW z@9Tn&Cz4=FM~~3PU$Wvs?^yd6D>5@l=vckNi9%gbURhyCLy#>lciHjE5T4*pE;KZi zzey=D)n(9^={6X_Lbu&gnY`%$oDE8ezd;-8ZU|dZU7(q8Zd@b$5)aBLOQQ>iD*Sc} zi5iTEZqPU(_X{w@_~i<9U$aCN4>^HGXiqU0hJ$Y66J?S4(2)-AP8n`D0h6Fh(8|0{ zRD(@??s$*3#i)TOmwB<@J?uYD=TB@SjJ>Bz6x0^o`=84%ROE#O|K?K5R{xt1(j+mo z&X%!wK|`ug4X(CiNj+4EkXA8C2~{MYuOoz%hAz{kQQN|n+RFFa)-*hDZ-xFk$QGjW$^$|^PoEB4)dMNm z$&)Iml#wLW(33|k)cmR{@<@e8KX4CGMKE^TELw#yc$NwSBT-sy2$WA@IAB5)#Y1OD5go1$k6lD&o9$-7*CAJ;7V0|J3c>zuo>{d`VC#<%G)0sGoSs#4| ziO7CrP8=`gU8LP~g{!PPHWDaoWjZ%V^46LU$CzwRl8N)oJIQi$ID`5YQwj?H= z$YdOQ(-zkJVMs^GD~-FeJ{l75SWJ%mB*|-e;>PB=FcOdDa{CP7R#m7k zkoIy(cp099y5foJ{V^DQUCpS{o6C4#m)SfKgXw3NxIWQ@#HYKWfRtMuq^ELcC3Fl; zS3_qed(vv1B@+hkdu2c(&12VC!RRT4j(h+R$Ox%wIbSE1dTzzrV#^Um)2+ZD!BZz# z+$o(T34Ul7m|;Z1v)q)xTeEX)H$^DRT3pN$qwBnq+R&#U(d{(zQ<}25>oSv#0|HT@*A*dVwmUUzj-9)+NFz8?8T_g$T zxL#o-HNYZg$q|FDnl$-fxg(aDuFdgsWGj?F{CoJ)7aC)Mw3%2JLPB|V-EY)&zA;joQkm(}zi?oLeGHS z;N~ZP2i8P7Lk;>PrZ06mLD5^@>vHtT~E~v1iIgssyg|Mz4lU zYSh%HX3#dkS34Ii9cI2_KwUkDM+@nA(g1_q!RJ-F(QgPL=Y2Z%dz53k5;l7{gYpny zZQ0f<;(U=CI?Y7j7roi*p*ElQ6Z+qU2+4ob5i>+>#~X}Hm3UHsZ50+<9`Oi$Y@$j< zR5QzO0T|^Q$fS#lB7Co(;h_TOz(+x^d71Y)83n5V6lmSzu)SVjpY;lgpy^tYK=6D@ z&aKy;jph0>lT3c{n*fiX23L3=PmCP+`+`qqitAheoD|hbY&OKKLqEcMHi#E0p(l6F zyZl>XZl}93KB4F8E+nZ7Jn3VgfX0p={`+MqEx*w&%Ymwr-q_d+k7L-u>gf>)UT`750|LHcp#oEY!eXuu%GK4FseW zm8Hr0)PSJzUeg6VEddvariJf@-jxjjnPEobI7Zck&`(7gWnLK~5no37Q%R5q+Jze! zb3XEfI{HLt8|=dR(+{)8WVV@kVjaGXeawV9*Ppa|C}_%&c}SC;u!S^qretH&u1uR zEpiA~5p(8z;%9owj&}XPKgu|+Ysgz10gH6Vp8{H@I85k-Z)FFZ(2Jkk*!%V?R zL%wkHYsC83vdCD-*~r*R54r$(hk39H;cwKqBLj+uo z)B*J2ueSBSjXvlrU7MonLvL3K;tVgV`0$1k_m54DU%{c$Ln02bXI{~E;S@ExtN3US zxr31BaMbUr_=WkJpOe%a97f#fZ8^*H%wjzDX1ntpl_To6Unnfb;ZD*~;~jM4KjWZW z)xO2&7XEv*M5&;x|qEdIW>Hv&{Mm zz?(i*BjY>dDj~%{_5Rt{UT!V+19kR?D7(H3s*9arMPIWy>Us4sMTc2Sh0l5pE6uIu z0AcJ%9yz&{(ek`WIxxzE(n1@MOqg~J@etA86pzj#^(-Up)!>dSfC8Z|lGePXnx_he zLF_E2&Y%ijnm%E^r^#)6(;{}~IS_$PV&6ri6m_iz7P5gVNbWf+D>*+ZM~}S>pq+4- zGPxJ(+ukX+wc_iNN*~ox59zAvyfj=-Di)B(D!rR*Ai`e`D!Hlw+E=rjtfWkwY5OX7 zG6A@cuTB$1sd+1oZYRMH)r+iE-J;VVM8xx7&}bor!oXTbSRDm+-ZyaOJ{7wjyVXtPhZ0lb@uF{ zp6khD8GX)R!-0$+#MyI;IjO5^t+Ng*jdt)7b9Wh;Ju=>*L}G5iUb>Qgks7LHefjiF z^_m+FwykpKp>1N&GYtzp{#8)WVgRk*4TTRQ%Ec9SN}Y9GSW;LM!xt`!N^iL?gulU* z&qtH-)$}I}xWCd&C$Fr~DO?X^Lru+!daZ;mq+n0&j%?Quk%4P<)+mHgB>sk9GUdwc z_NQrd+T}^i%CGQxbt3WM6Vhv->ms6_T!F*Tf!@wyu4X0t3?9k@b%o&eD2JQ90?SGhf!-_4y*phd^pKR;hq z<^D{HAjr`8DZHsFT#<6$n?a_)IDII+wQoJh9bT1^EHZwsBfgs{7@w45>=7$sRUHAi z_(9EE0t_1v-Yuk5Ix0$GqSVMVzw0miiO~R!LQ;`Y2#5c3W|7k^$90&6VdF}Ow#_b?1+?bby?QL;tO$ZEt+ixC^(GFv^&*cXN z$`tl;REaTzfW3&&h8}&Pg!~J>4}zClm9O(MrcO_dB+jo%s95+y(^Q5dA*fb{mb5(>@CsJeO}U`p|(Y=d>)Fg8J45$06>}eGyC< zKbnv-j7iTa49V3=o9>=dX1EYQ2kmOB--qQ4JM!>_f!9_j0}8H>KJ1N}>~$&p9u&3K;&m@QJ7 z>=BBgL(ch&`GxpPXKEn6@d{4s`j?}a3fc#c@pohAM|x4->V_bpRP zbt*Nr&b+|ZeqztG%hm-t8@Zn8S22mBzkHOcSlbr$C?WmHYfoIP2W}jwhovWl)A1tp z$S;RpVPGesr8V_>#npQhsY`wbCr3&tAG9u<|`Jl ze})g9Q|=%K9mApUamfP4-ihg5s>H1#n*^NkApg)J1R3eqRCR|o863J9>`*p70bHV| zR0{IByFnx%Ll_Y;Q3z9qm@yHHXn~@Ty9@Me#D$a0H&8U zlY9-MxEZFr+AurN$v`BP^4 zn}yfKywPg>ft>&GhW9k@5%&b|(K@J7@#w)H!hyaftPa~;#fJ?7ehOkXcsdku82P5K zGVQCdC{Z*?$jfAzM0pyVU~s;XTqvdxaTNlVL6`$tI)VRGXq>3r9Fo$H_`Kf?Q^tCy z8V-o6I#Btz=3pL5z+CukSfxGmgI(!_hY6~lg5B~DIbRJw_f+g!f8-%{&rLq74a2y_ zrs~1$Z@scbMTgd#mqP^ADcMk!nBNUng_EAU{Ah7v+?2#rqhd2EeS*%fvLpT3bX_!x zq$PHEck4xSiD7nLByIlnJ-`riN^m&(B7=k?>}L@H3lJj(W1_TF)J-UtEfl2%xr4LZ z9Il-X5!HNY6wzk@P@D-tYo|%%y|UoU*|$Gs^~@wj#g@%3Gg4=Syf@Zj{IG@kKiL+RvqM{yI^k*OQKiKZN^+z4$M z$*7`y?zvxU>hWIUKWIeObNpR)uN;& zQxS@?51gXPtu=h>u<17+x3q~(z+H)a>-4qSf#TTE0IHG$6`wGK%3Ne>x)}NCh!Emf z&mFw4Hv;`M2QF7h@eMBI^x(b*v@0{78#q(i5kJh5V}G!oCuL8Q4b~9~BW<5=(7IZ$ z1b2s&G>aBHmi|rxT<&s$dfp#6S>KARPT z^z6aEzZ~_X)iGZ{*6S5UFVh-8qsZObXvi~Mg|a@Fh5CmEcdzK>2G@-t-3A+pO=k}M z@)36_lpe1nPyC|OS}I$!q_ZbS&Vtb{$e1k;r`aINTtqb;oGHjylJ(h-H*rUc@J>v8 zDP^akr@ew{bTTzS0j`=DVfA_!{>`1y#}X zwC_J|=gd>vThSzNx3sK(_i=EajdVI~`_a*{zj;>#uD4E|K1f!FVO;E!uiNVhi!QUaqkoag6MlXLJ1E|yBijf zF7yj!qH#DD5ft^gODVCWhBCF@65pQi0+rfV#36OSg>vdF!{VuOj)vHtN~J%dY?Q?X ztO!_jPq-H@t!OsA;a23Z0g1=nAwO|ngV=jlK||v_p85nIGtp zAyOJYoz%c!#e?FnkXpPPP7cMF>ujNO<7g6fFNG|By%h$8-+Tf}ncA-Fsxe$t4jZ(a zHr*1gFFARCTqE8|;ZB*^*)i+x4&Y9mp`7Iphp3IFGagZyn931df9>I>j}lF9E)CF~ z>;WZ#f6HXd2urSq>J4sOW00;Z&M`k0Nv9STP0XI)7wNNO2HqeJZ-^(?pnVx(mw@;c7w(X0?O=R#8MEc$`Qr*YKHSon!9W;6zt8UdcUoBCy@cpmYjBkGxRJcg$P@t?aX|G-+8hqJdtPyrbp{ii(3G!Ha%#fRuf25oqjAhoCndi# z&zb1`pr_DKxvB~xMXhez9&lFDOBrausVTGQyQS*53JO9;?Hb(1ot53dGj#}kO2Jr1 zUM(`Ph(0Sxb7Z3K*V?E4WSOzI3c;SzRy^UVdD5bO&l8M943mxhMQf5Jl-2OcM{h>V zp2xbFm}dmJk|<$*J|0z^-DV@2W@_p_B?b^7W)luEpm5BEZ;yxTvfj>9qk& zWq|fXQ}bDShL=4D>s^VF8bgPER*_cAKE*OVDs?a(K8EQm=XxJaWj=AvGJ^m zY1{|N5Hr`^XbO1T(+sV$vFCy580CVzQZlDPOZ7-bTB(7*%LIbwvH^Hm3s*1}Enrl0 z*HQ?K^JtcamJ&hAPu3=qfirPnW2$gmODtV5!h&MK5mf*jJ!JtN&lN13DrN{^wCaGW zjm63!E8Oxmm#}m)3bTt!(=E{Q471+;R>ah~uf>I#SI#ya1#RFYD(EQ{yN}mJ;TIWe zbo$p4iYO&iI4ciu`|%wB0JO^{GslgEClJ1~u)&=J#!(>FSvuPozjEFau;KZdL7}I3 zs-&)^(ApYmq>3GcwXC-b5WWHy3L9FCqRexCM|SzlMtM=VRv1lqv%SHvzlEYc(V8y~ z&^VJ@GB0EQMH8%Aj|zc#w;(oCKKfiF;W$E}+n<Ziz`1|Sxix&~S+rPD*7Gd9$W-Wpd@RVxyh6_t>;4$D#4M5=&;TB@BkVG+ z2~PY{Uq-(T_vkjG5hnlkOup^t>I*Ev#c?Oxc63)i%n4}aS?h3trJZP<>FA1SIs64C z_XWj4{$rr6i;t+dW&h{>f_9%{MV7foTHm>Eh1r*s()G|W4ZrXc1N7@6^$qvf zy#oA3ydS|Y_vDQpVexk^jT)2`p!dY_pLPa#PWlJ)5td!E`UpI+1ON?u60R6vBbraV zyJWv6Ji=*s_Km;7Cr(L6XL~@zCP0$5W%+P&i*o=*EH2167g)p~J-bv8L?orUTKRPZ zj?@;NT2c|bqxBlNdL`oF&}qdTdMLyl`6ZKobsUQ;(!ace|0@7QiU=Z20o^3V;Qym1 z?5{7jzow)A9;~!>z*YxcCYt7?DqUZ~4^-G(ao8nMxoeQH<_y9Fkh!ZPk&g_)zShsEbt3SptQ$@o1C8^NVO>TPP8rYDsJl%IM&xQJ%su{oEdvVVJ+X8+;nSvfyyP zZ=$BeH9PNoaGY|VYJcE9((Ar^a;E>V5}Ar($AUQzqcigoPJkxx!EyGO$qXzm9?Ad> zU~y=IlNhg(M?^Z16t6ZD86BiLN>yj@+&)Ih0rX@{Mfd@0bP*EN>7aC+jWmjn3gAQQ z-FR%Pdxpk*pg0aa9O&Nl&V|oKG0H^>mWQpM9&OVf2Iuop>x+5JivH~T*ro9rgyFAf z*hW9*A<>t5jfx46ye>&Cc#Pf36cL`XWYp+IVpgn_k{)v<)z5$&??}xkq`;Jc*{U?G z>jju$NVjRc>A{6ne>5w-8u}gaRQ!|#B6MwiUp^vM{ zNaM{i0Wea@NWGUvk$e>AmzXJe{n`9iI9@utwsx6nP>(U z3$e=Kj$)_{*U95do^b9oZ;h{-dg_a{PV~Nfdxgua90H!0@er{k_R%P_r^9-gM2{H5 zGV;ajsxXhlj5HCIK~A`yKb zQ-rfL@{)^P-5o@4xhnY?O)8D@fX{|hVs^xI&t`w#ma?A7Y`pApR+l_6c-IYBLBPL# zBs0_;a$6U&&`vj`YT2=+lG{T>jooEDv_O_Cf5hsc+XLMSBSus~ZSm>YaNUGW>c(gb zvd5UH=2uXLvca^e*C>X0l2Tl~-c>4E)bZ`EL($>4+6BK`A}_bAA}@EnnWoAg`TX*B z!ye-!@%Or-VX5eLEm5bpHBmijDmSrvUEsGc>omDFk~=DvgNe>H zsz>mlPVbS+czlZlX?KU@ubEN#N9du}-teE0J1WiK4`kLPPFSN}Cqka36wZggC=1KL zJwzP)OKRwnhe2W6&$qu>j`dg@^2F*x=TTE_Mz|x|c_6zg4M$P=#cW+ov2NR~QV&Fm zkXv;nH28pXkmgv&bLTq)KM%}yaPBcvg;r5_l?yie?9mA3EH*sfL#{D8C)s99~CUE|$=yW`e?zg*XUcdz|wJ83Bz8RvzT|H3jm|M__=8G z#m8vqUMs#<$^oVOJ1G=x2_JvOv_-o~eNZjj^0%Ri)JW~9ThmJNk9t4CJwEhMM)Qyh< zPv_eQ(nK`i;e~(JT-O2YNkTrr`iVAjQW7)cdf$h>E z%6k3QjyCGKNb3(+^maAE{ifb}^Sl;nr+DAKOlT+gCPm;V@6;JA-no5 zZH0JYPN?hPvNcuh!3SR3u$jv+$=gg3i?{^&(C*RzVcfplKT(-CAFMvZq!ADX?F3l6 z4ZZ;DqU&MZ_sxX8Li`1fx=qeQhSpw*Pzwntj&p&m6iO&Oab*!!(p3FkXxd3mBH3(g zFL9HUA9g@ml>y^b1iBY+q2AFI*dqQM=!v60h8|0uQEd86(V|ZKB>YCL{*H8$O6xVP z_u5T-S=-yNyIM)FH+ag!rMa0Z?w|opNm_+BOC1O)^rny(K!-l1=X*f3hBJoCG@(!A&R31+MVHY;MJk1O~?7&1XnOkD<}M$_P6l-2_GKY6&W-(BPhiq z@6Y_eA@#}J!2o8^2d|#b##hJ-LHt(#gDq`;>Bnl=Qt8?t*`?b$kkyn^v={9Gurh?g;l@r#M@*%IGw zkCWYPhPEyHfCQ(Qpy0gxv-A0CXZz6*&j`a5)=MzFe=py87(D%+q0CiQ?rZeUhcl`q zGw#ohP;^{F*Z6~m@n|j`NuLVX$DBeQuJS64-l5j$cY9JDx74M+mA7bf+5*w$0mROn zl!v(6w;d-wq}N??1>L72o2p3^+IySNxYnp5V($W^<SAl|@Dxd++@V^H+{|t8( zR|gwYVS8H#LnlKQd#At6cvV}__7jSq8}k3*>z!jP3%jk~>axAd=(26wwr$($vTfVy zvTYk(wr$s~^L^*$=A7id$xhb4J1d!UJ3%tZ+Ik~=WSuV7NK%W^jubRD9aKu0MhyXfhN3xiN9Vtp= zb=8~T@ge}28z2d=k%;&`*PJC<+(jawnZHI4!t_#X^5f(bhjbXT>VP}jmJFL(ufe4j z4=Yh)JU5Qi4a^rZS#NR|ow}H6~ z6hslsq>br{2jicHR?NrX#|k&%oH=ZH8Lol=UE9SZ?7y;cQ9e`kZQKSouvT|=+54nD zVS|iRDBRVk)S}rxtYjQ}ESIAY2DwY3w$R1$peej`H|-+eZf!Va+ip6G#-1cAW#$m$ z0J`;oSxo&G!)40q-NkAQO^;6Yp*CEJSzAzKfSA@2V}J`w)j_XM4GT|SW`LcP**mNv z2>Tm?1PVp0iO;;lzsm&IDF&Bg1$f&vQzS-oU=KDr)qDG1Nsdj!x`GMGxwP|?15VFK zCoa^Q(4;*&iA`vqZ51)wzG6g9M8MfmrH=^eEi<+#XgBdj3ga}iCrDhgN}W@g=Nva6au+YHI)&tk$B(-l#Fb#d=xk4q9qIe+ zj%jb-e}w@l%h~QX(E~34r$L#`=epco0~&ZBe1Q*9(fdJ^U>BIsn}?&pTf>B*oZ$F( zqS#4KkekGIVY6h1G;a|;|2&ab+~XL}59%F7u}DH#Vxmaz&ZH~!zHG-wlh%3u=2s0+ zK5@-{2LWlh*9+ey)^E3ZKLh@)sd!PhhPUf5ZZTz15;yW`5sL91L-iTHfnGdA&!pmm zkzfu|-t^zoto=d|*wV9e6o}3B(NW2VOxsA0ynX$PJ)wZ6TNz{au#t z5E*O(Bj7ag>b6GG`rT%4d5nB}sijSq{ziGO?gAH%=>Lv}70BGnm8FpU1{lW|MrY$b zUH^`?>eLbkC-Hb;jc?r&*`qBK`N+%pL>a^Jdw2JGMm$hYYsJycJOjnU;~xEvPM_r) zWn`KR+bN{K2j_)(QFC;>5sb2=n8XG-s)UA_fh25P>WbXN5NhI^j<(m0pa3zXSxJwLvX-rciTC~an5H}Dw(dw4gt`_ zk|15XeOCx}r~S^GV&ncWfo`&w)G(M?{sXOB9>6)*dq$wIuSu|fzF zrmH>>te-;Y3xw!Wj|gSPx!W;ldoQ@auag+9j2-{06+&n6Qjb1i-w`s>sqQ~AaRcD4 z*sav%_OJE0b@3J&NYu@Wu=8|)?k1_+Wc2Pxn%zc9x*mV>!ukeeqpY}>$(#0SD3;%Kx6;QbVjrj|#iwc#o@Ek;x+H@F|TPMF26D~stT2pN_LJL@F zD>Dpq@^M#xFbwO~UOAr_@ri2DHMGz-7l{G9CB3`Po*n~!$h-4ai=d~L>q4(s9!=Zq zg)3UX5YCK1n7YSqGz`0&0+MhNdM#F_VDV67EQlb1%`zyYU*H@#u~Ws>j?ERLTX+Nd zg?0Tn$iB9jd`-m)Q*Rzv<>E9?K%vL1T89=!g4X2`W2{tu2cQeqvyrD!rU@) zNUkO^l2Lv4Q?qe$&H~W8ahL^cY2ENwT&{2#NE}4@9%Ev_;5;Pv8vt|;z>!?bfIYEI zWO8t;mX5HE)u0$@Zn(t@iHL+Sq`^}NQS%h^*oKO-)t*$%A3&pd3o@1LwN}muGgF=F zn$fF}C^x~{wqjiJp0ept!?@k>dcoJ0y%`~MYhRc!MzQnDTSyc{N42_k!I10kc5jiZ zY%{7kFT%aErGIv=B{jk&%oycKF3@dwf?#mL(HhK72lWB*e~OSI@n&scGZpkEMIT8h zsK@e5D{C!C0}`#PHHvrDplLW5(S;0Rq&( z{f3^1$pWGKG^q!jl&PGIS$X?k$Ox2uPTOq|5pkr{?!uTO8 z2F&GbEgBoLef|QXJ9{hkAu*8a@&V?zbj$XU-KQb{&H3f;cgBulX5p?d(gS+9{es{m zFh4W)r)RmW7^Hdm%pK}mMk>zW-No2hyuwBUJSHaf78*vVJZ2yYtJt0;{D3O@X4JcL$?w<=z5Lh!-U0rFS7=TfoEE||MPjd_oVT{XSXwGr98@s9u(ujkT z^-?&QvDch;9QPg9m9C3L3P!^aV*WBDa?YRf$Agyu)l{p_ z)#>-{HKYUx`8LUqW=mDhPr1QD{9JIS&3gdF=7qZ^b>?deH$h`nz9@=xG&pcsvbmWq zG{$8bt|lV1h=Wk-fOWrPob_NE8R8aU&)P!sT14J|(CH2P2$&5~r$L4X9eijrKL@pY zdYC5HL1P*vhUxko8zs*$^?8qzPBz1xvlRYk4l2_(YD;FeqGJ|~vk;^1$OVF%dK}r*a!#xEY zrOWav+>ttXd!7i+%>J!`SeUkb#RE}RX?+X7!`mq%_T9KHSrqf%iudGL;p`zLLW}Cb zjq&P)OdSQDmYU34p&mFhrAHv0P(}=d=EEKKlm#cESx?fl&|LIU*{xaoEUiFZ_5@G6 zgxoQ1`(WBp=(oCxzWAAQQWCF>#Yl?OZkVP>g{a`*Czyd{`*x66w=q)Lmm_D3{|vd| zP_)L(m~w)0E0OPRAe;LN*qrxmt;tep=Qo99X*9n`k_f^OalC6oD(M|gVJP_GRf6ou z#}gj%iKEFa2Hp1#Bye0Cavn&zNB<4I79BY--wU;pm=qM#B%(R8s&a|V)dy?x)E&XQ zv%fe0yuBoLd&O)QROr?KV4t+J&V(qPp@!HzB((vt!|L9qW=q*4C6+X+%07zNwY@&w z%c7fc-S8XudCOs1%JM-plkPB!0Wl7;>ktn&e_Ua7z zH#Ql4k)Dd1nB5|Wtc%_hL5L1g1X6oRJ}@1Q2sY`R#jg^EP6ya(?4Y7VVzy>xvRzHl z`bK*qnVw9FJyPafQ!Gyer9qmcFV%6!%v+`(XxZh!%ivehffL`??2}VdqU-nPPi^K7RC7%sFev&{V8?f@m+7cS)}zf|9;mq zL@mcy%M&$6NI7s93V+d2J2CRnhudiUq>jpd;iwn8q%v>?_*--Az= zI&F=%MctEjcLj_cDJFgWua!eL`2-oOAAODL=gdm{pKhwkD)Q1I|9xit(bj%$r(d$b z(rK_@O0>4F$hup%W4cYTWxz!Qx z|4JFU$q8x4ri`Z~QN;TRZ_qq#gYO^}0y50lk^}TZJ0!iEIwKv}X8$b#x~72E(A6Q{BiW zV+HJ7)@72i?(SMjjcQba<_+uu3pGnmir)u)sw}eN_5w?YSDmc+594Rb3!o%7+;PD` zADBZeRJ(g-!pNDle#E8ys6C+UB)SAZEMT5jXKhnaT=2@!b%0x!gl4GRYlGd_iH>Ua zYSZI*a;_NAtQeZqG|*bvI3OIGAS10r<~5SHo*`*CH?Eq*gf`n?8wV({TqX#ZI(h54 zEmp*;U=x}d-N^yN8mO^5Svy|cK`&lOaPG4U>D$HR`)!hIY&_yXuXvY+JixOtB7XMvfaM8G(^ z73BMu>$JTz=(YKDd=4~f6qdDE?TUVY*99F~!lV@(oTUl0A|8vewRu7S4zIQo%GKeP zfSj?v_8fENQTdre`;8&%-APp@r_=Rgl`7fLwN zh2EKf{l(z|bj>OIsitfnXv}{TnkuPU8SJxlw3j6mLd=3G-Rs1DRLlPy+S#gvseV8z z`Fj%0RR=9NUTVy#MJF8ugvX^xyWL@!WPO{K#I4}Q_?tcf>681MOC3k zi%7tkE}&`wkthqHCkE~jb?K+!LJ#2fVckGEABe_$UytuIai*4#xK%g;TTww~XYx6( zYbulZ_yXHc#Kc%ekOVOX#G{s3-KLNZk+#GQyro<}hx3C<_vj+Q#&1rZUuy^`oC0=( z6=Uk1GKt*TNhNG>AJqAaru;`IZP9EQ0_7+myJ~NcAO|ae?OH7GTtY$JC3@unm~J6I z$X+DE%y3~iH7kr7gP=c!OhT%-MlPN1fBWIRAyQ8=7Pnz-2l_gMjdHGd?G$v6!q3)& z#0}DI)k@)Gve{ezuvyUXzx$&=EReyMjP-Hp3rQtqbxa*l&r-|#Kp8q;?rbc^up*Lq zzJ-NDE?TGu49bab5&0VG_EJtUUW%NIHwS$6{4d}xebZA;=})sN{b^PN|DjnaO33{8 z98LAVIsL#I_11040mT8zb-|$OWliM2)dc2%ngl2x+lklz{{0u72^UwC!E1&>Py7ev->&x7y>8C4<5t#BM)aW?P~nS!2%9MuLnH^sB$d0G2rk;e z-?E)9a^2x~=oDpPhJ`-}2Os&+U{GzEwu8{B0B+@*_z2aia`V0s=w+2gBhfx1K(4;f z4|u+!03?@X&&fMq@$kRk{rZr?q%uLq6N^r&qTAH!&2}7bYlX#!Q1$6vrjq>EwFgqz z-TAcu$7T+OxK&nHxJ}lqsGJ9$i7|QltY;pJw*Vr6dxVM$ta981iSv2gz838fXGUjX zy&YczI+G*f=>Er)M5-6ymKq@9yqxd@fHkzaZC|N+!lZ_%ZNoS&Rc3?B`jmQ5v!pycPVwVNbrN19nh zpEU>G!}qp@#t|xaGAaSdiLtEojI<2--bt}U)nSH@0I}VC%HL|?EAlLrnd_#Kd+?Xh zv(m+j{WS(sLy1dE#8YJwEjz`NB2t_f$-!LIugniS05A(U(j3cC{mA?Z<-ERx(DhZM z)^9&a#lYw)ezW2|j;+GRF)DkbzI*N36{cwBu`Yqvm}g7-9m`Pzv4NFUg&m90z%iVf zYID$GE0hzhMrke|$_;nhQW}n!)2Ve)Ub`cc7ih;-lOIp*qs9!=Q>4p^hr|_td*N@W z57^IAHWAN7Ukh~i!VU1Yl{>DRyHBk-*HVTY%noY!;h9vW83khNb1sAWgZ79S<3y0H zu4bmSm&bV(f52X33ia&S(V2jDGVzj1hD5Vn+oC_aTTF|_HqBo*Qed<7#M-FM_+MYi(+r57oR3@gu7pOjlMZ# z#M+6yEngUV9@n`Gad*trW~d44I7rkUD))b!biq9~_>#gM`c1=#m(X(wN`p;izKKX< zBxcUl{}>n%77d2R6|rHqwqEAL4Eey*DX?8MjV=#T6OYo!gvf?`i{flO2tVf$mRfOv zvwOtQF6fSviZi&VI6ZeIFi*aI7 zHi?mITEjA8a4)n~<az^tt#yjZbe~AYni@Uqi zKLm(Stp7-XP&RRQmbNo6Hu-POpjz$S3v~(8m#pQVbqWvxC{iDoMTf+1DmH+>GD?_e z{}6FBxM$`jE^ARvS(e>bL1+QU}bYt^_prl0GU{|v3gCjy6YdW+`G`aUyzzb(L!Bscr@Q>g3 z_8Awj4JR~20f-GdKc&ZV5C!Tf-Fv6RgWIj6&$xAmjTwF(C_grYYIt zMy}4xn>Azlz)N$g{uC78GBm6~gezwegGM(zOP)~IxcDb|5oe%4f1+36G7W9U+?12| zU^OJ2w8Ws)086$dU`_o!ETCPNs+dZhTR-jX>%(2JCA3Lc6n*$$EXur@Ag8O3(`<5- zqP6c@n$XS|XCMmUd?fqV1dM>p`Lr)QxjL>D-E}vAMoPmvBSjvLqP(B<#DPZ-rG7W8 zl;vy*Y5wiCJ%UYGG+OYQ9cR1EaYIbbniF@q*^u$`&G=nU?NUlRvPl{30g5rA8MA8kz8K}UQ^_{gOv{f_F+ ztYcREXYXeHXRh}3envg4}G?8SFjo#Vu!|-qaKD@w1J+s%bv_NXE#gE_H5syly{qTo!R->P&Z6xo{6 zT(UKIl+M(Th>)CS=_yLBNg$~&qGv?5Z&V)QC|m+V-)&g=lgX$yJQ_$b6NAZCy=|$? zV>cf7wEu*#&BvaPb(ZzFr<=|oQ3ocN@Mq3mz6cQUBu!yVva*(fYbn-~91Nxf{Md0g zD&1r!N3BacDfI7^qh4sK2zX#kpmCbSXj~(jQkjjyG4CitMn{L7cp9W$5c_0WIRdcj zB!-3{e^O(SezIBtJuy3MFE*e*Yr$Yt6V1brTfNsOV*f1r`iZ#2Kzl|q1uqehI)hsP zRplFkZ6G9nI6omG#GY51O~ld-xPKKDhKO?uyo-Tl9veoMQ1hUhSHU8{Wm9u?8!q=Dw$zGv{Qm@7zqU9We9GbLR=TvPk- z$?*J&TG>2u``e1~rR!}l?7h6-j>5N=?d@}3X}Lw6;>QDVbC+kd8+P4!6#e`h#`d_!#XrjE<@k)z zU&xOP-dQ~#WD+H;%^KKh?tJg)qA<-hg|nDdmu20PdX~|rP*s!0qdqb7Kx@AyAIUkV z_-irDma=xkrL*~#hl}>xZdZwJ1fZMhDlK#>Kb>E)&uuxAf8}9OpZz^W3_<~BKty48 zs5HpZ4}Wo>(nsrzLIbHM2olflHlnd8o$q^G!BnP3x>*{cmUp#IA3W>JyE2vM;A}*L zKd@k)A!zKV4?a~$6Ioh!K=|*Eh_g4+Bilsf`4nf+iyzasT|WByt1E=+d?brIpF#`6 z=z4LemV%r#nV4OnAC&{mh}UQx_yp#I?QkQ&;6flbs%Sm(0RHQYef}pA=j7Q?GEQo6 zv7WinN^N5PlgS)&(>gj!@-sVCy?1ujB+L zb^O-iJSkyUc_teI);_|Wp^6Bd?A#X;V@|iEE^$1xbj>+s)o|e;&oGB?=cMHcyP}9A zN&Hh%%e};bjL$Ku>O0DLc)?%=UD%>gR8U0hEF`RnOt5?o@f8KS zo}R!f=3WlTwv$QHNIKdQxt9O<9P~IHqFajh@i@8JZCypuCI4X)AW4t-3i#D@ocwr4 zxj5??I*duqBU`SmS^SmI;o`4*Q`8;9hMwpvyQx<@zs%$WM@vyw9Ip?zJ*#aQJ$5b9 zGiUM^`Thdg%^0QWzLI+{*I%4Fcxr~6-^(qXX3*C>fkSs+KAn8gC76SvrE|O-f$+Lq zWIkV=!B#h{1@5IMI4D$EM!eB8?wr`}fNM+653k2f!ikK4!$9iIn!{`yfFvAlHn;3l zc45Dx+jk< z{85j{gACXkska6iio-L&J2}M(MLV!oq9*i);Iu4Y;03@NHvGTMlI^|X9tA&sit!&m z#eYaz`9EHY|Bs86{NFALT8|%wEOAkiq}6(30bEdh-U}LilYGG<>f&$KZtqp=nu(uC z)yeM06Sr@KY4-5(C|~Dc{6qJvkm0}bEa_uBUNb&9PB}c!8y$XMkLPwks<%xMm{qbn zE}i*?aCVgLQhn=`it1B4xE9+g3d8LYaGhkO0I+9eb9cQyAFxcy`c+ki;ED*T7yiCb zXjhDR?0zJHU;+hgc71W%Hk#?UYN*ibkW13%r|1%y9PcR>xcAC+OxqRh znfd4#13bqrBI+(oXfDKqf?fZ zy08-anR6?{BG$Pln({+@b)FkxP-D2F|zD)U7h*{Exn_9W+OqtQWYvad~3~h9iiBQ4`Og> z8er7IOfUq@7PcqvAvNb>xw1Gmva z00Ni0Ul#(7$5b`sHAmlbC@YnEi9X)BQCHR`0l@~LS#&{%ks?OLE9Al4@2z6r@tDQ! zU|Mp?4*%S2J@DK{_vlR#TFcxSbq*W+&1s-Kg?_Pne+Hwr{SUk1YXhbu1ilhKxQNRi zesl33ULKP~yioTuoCqy%=gwqFv}~v(9`2J9nagh0x${bRS|l5l(D0ssGHSYY8a79G zVssk@X}Dr4SD#xnYuAwUt+yn1>FgxhyG zeuUZ)*2NDHiWjX9FvA$WFPP!`VhibxhoI%62&bL%BBphbx3tQhXNCn?-2TgHQo3p> zo@pQ11!FG@#5B>`#_yCh^Ehdxw<>qtA>3DSH-tV7XoMzKg9-Q8Ma%+Q5~wF4i9uTZ zHl4^r)mdjLa`&jA3R%8LSF=6h@o&LpifG|8x%w3j0oH+s^*jfjX}mEiBC(D$Pji7M z9ueJ;DIIc;6xqwA(y9gXk;Sxkx9n`8dqofq^F^OkeZ%f>T6z{T6xBvt z3efq%#6`rUvaqrsO-hj0X0P%|;|XF3QahzLRFkm#h>Qa_P^K?rA5bAY{~noLGy2r` zJJ!37|8;GybDh@u`F_In>a0@I8Zm^o*vqj#13Ie-L)=iQ%fpNmpeHC*Z)#B0ieLC+ zwi2f@0vJkpaLZ>`c+yS~x5=;t?aJGC$IV6WQjA$2Hyh0TRo+Z=R8>{G^1B{QB3-urq zb8h#+dxefB>2Z7V7_gr0_?r8_R;Z>zXj06iAU-HoY_8HgIc}&&hz%DJxKBT$C`74R z*>*Cp<${zJihE4;+hXjcBSf};d+g65!j%))_IN0@Z7I@?iofSeOB&FQ*K*3W$ElFeDCbeeookg*3i@wyq)uZO>oN6XN$}1j0_*h{cS6=~?G?cT3Q9ns*yV?L*3*{QSaWXVcKt)h4Qr!EtL-XtHUO9(C? z=chUtPNxEw9;{*@EgLKx$-M8b1(z2c->@yZrRRdS%uAW&soGG=6bkG8&utX4*DILz z5uaRF9E>r2RpU~gM^Ws{hl7Cz*M-McG15vLIt44SA49zdZN*Pm5}c1wZ}F2zt8&a? zI{?f5{5$X(rAikd;^H#1dPXRcF+UJU4BBptrp z%qO5~#y`3?{`%EcF|{nU%$;<7c*vKFEvEtH`d|10Z;q}BDh$Crtw+jd=1ZJs@=?i& z)Gi3|7uf&o9l;8G9)O=DNqPATZ19C>^T-%MT;fc(wx0j-m#!!;LoI+a*0?_SM1FCy}M)KN-yIfX0 zxd(~Ba`#~f1p}P~6r@{gE{AbemZJ^hr9qfO@{h`f?VC6P8z>e=&8UViAxy`MOfYN8 zG6YI5rNDl?qc<#e-F-&gFxz)W5+pQzISeaHtP>xXKG319b8g7pYBwOh%JhFgFBc4+ zRk|vnohWBZs%58@RUYR3QV0&$Qq0uV84+dW@^ zjpUD8h}5&=*||rHo*@5r_I)}_J*NTIB(-_~nH+fFvT8bw$iEqCR(ymPCJXw2zq_c< ze9M^3Xd{_w8q^%=pIRKBU)^{(c&JW}gtLA0G5H#5lM@PYg1f8!(p2{)7~}I2v+6gD zBfqey)(?X&y?jrGc8lwr{C9U|NEB4U2Ltz1RqDTw&An_jASXwM=Cknwd+E zXFoKpaWu_kuCO%NSPt2*fUGVMabV^|Jje|ZewVJXo`w}L`Nco6s1H@GwE)r)m(qfC zX-OpjuPSAJtyocttL+#>OoH=>2SJ550i}SscPfhnNe$f;a6D0S1&4eNji> zNd&#pi)e`3q`%c5i?xqDJb6RPVlVez?r7UnA;5(bct{>dCEc6v$64Q+Z#Pt@;5X3X zAChdI<&A{!tV#0c6=^t@Yg#P?NA5s{h7A zYBS-l)fsoHgZ5zOk5TjkNc7`Exhte}3KU9i6)Fql3_MOVtohe5Dv6U3bBlnJudCan z+)`GJ>ZTxVAU18*nzP^V#AFua_ChR9G%m|rUp~)0(f!Y+O08(6cM15JLNfyaG5m+7 z>TJrc^y6JOF#9pG+q#-K{;aG0-yYkg3FWQ4w8Tf2K9&iKP#^aP1Uy6}i0-y{jJ+80C?nOjq}S+Z5NIG-PCuCeK% zZPt5ly8E2*vD>0fCr+vTym?M(d){!G<~jCDdH-R~dtcBa(d<2EJHv5hWlcMAO5f<;6)Fx!s!s%QZTX*k>yNqLN`r3!z zqF_!PA2osR*50QY#Ds6tIo*+Cf1$i}3njmk$iG0e;S4eftvW@l)u93JCVj6j6pJ<0 zgMU~(l==F1nEJ8sqJ-tAxo?$D`cQNCwCK@cxI4;9>L$cKue5u4?`m_W_5kKd>AjWK zLMWgx#9Y@>-9vZMk}^5nN}LWJY25|4o5TF1v%9rrPp*js`}_bZ$OFg&Dqdbc+0?|3 zV34tG9rX%tAv_tv0sr9n3S+>BEcXbNXAKhC$;=g|BvP?tMPxK3tJ2g`Q(LdEYw9jc z2|uq3wSY3fPB$ z9+mKuXZpjN{3w1oyl0^TpK1_Khz;;=1k2}~O!ijI)MI}4m%0`8-t&`c@j14cg?(nG zfS`(nXz0ogc=g}XD*mu?W9jl63Ag%bGWT~r*TB=xO-S8_pKFke! zIPtUK^`np9j{h$@&`jPpvR_zC45F)(_4@@x`n z>dJAn@Mra}bcV+AXKVGxpZ`)Jse!Hf>rsJ(K%}t&B`>BqSOwUYOSRmHEa5_yf`mzQ z*wPZ(JK{&CQbfZ>~DSH+J9ZYC5Ehm}T z1a-9CzGwwoYFUU)V;O8f_aJGHImS}GIp9+^LW>m#{;i9gKxrXtD!c-;{2w5SSf&vs zD+Q^VXZ3IBP?-?v&)9@qa^L6)*`c&@kcmSeF^lqt-flPeQa*y=G$AVaI%>5q$_Aq_ z(SuTaYgstd4=nX!lVT>#o+?Rf;wITDeH*pmpCOx0&r?Ji!9&zrFx<9YAwj{OaW)17 z+*B}{c%>SsAdi!A#KO`r=_I|ZW@To|8g9TP;;@Gy+E7uFPY|$L*GXmmR5Go)ps=T* zR#M}bZ(h7m{j)KXJNMe)oh_SNV#V^6_3_dvo0~s#=FPL6cN@*QJ;-+MjOzn$IvOXT zKi8B$m4A~Gr{z*~RpTxY0`vt;I0qETh3FFc7ATPIc8ku^Epn{&%9?F=yt&iCda%6) zao+k#kju4ksjALi$AP+W6ca9DT*rn1Yj^v;;V;j;a}N`~KB|wkoqvzFRk~`U` z)dx#9#ESr^t7c-sz0k+s>DU{AYzPnWV*_7i+y2FAk>inb?&jB)`dO zPL-C{yG$%UsbjMf7G+VyR+Smp#6Q8A76z%9!n_)E*1J0j3nTA&^^DgRjySg>^ z4IelcJ^jY}Fv1<3NV(e(P#;;t+sm(9ADzQ=Cm)1<=*SCAq9|R>)wX?#|CXMazq<3z zAjI+a7ge%S6;U65;#JDi9pD;+R$lSv7`4RoZmry z6${6wp4obbE6(lyHDc#=)Adj1;+H#kt`J@Ecs8g5E_?FoZMbAEe)sbPrh;uvq=NxzrD8UWJx2oC0gd z1P@9)U+P_N&GW(HxtC)bB1PWvj)8S z;W5e)RLQ(V?J01-u~i*PM9pwx-1M>e#3!-Zij~d%<;K!L!TG?rt=+>*A69vTR^!xs z&I;9hD*Ta)N4GC85rz{sxfu?&6EIR@%sPdp9vDgpHqQg$nT9Vi^?S3v;_|Dfl<0bJ zGl35|98ROpgF1@P6kkB%G@;KQK4jS<@5a~k2guE#3S9^m~+_D-`vJd&pLC> zKD6rjolS};E1px0v%_T=9d}l3e+ymI$spA!Or7@x;c@WyHr}n+r#PV5T)$)XY0Ry- z7&GZD%n~p4B___h6OazW7(5q7(D0x(gpy$yl;ng6h*4+PwafNm z-pk;Z>tI7xJU?DqT47zZt}3^y*8Qibq}5Po^O><0y)-3rZmFeto|(zITwP^XRy}s3 z!q8W2Y1}}z8K2MOIHJj&-(&+bn$9`WZ25u8*xq={n2ezjGo9X`qUaOmhm^(^BUZK0 z1YYw&=>`|DYWH=Z_wXgs0)BSKrGEJow+cHawb(NhZh#<(39>D>IcCJRIM7fw+d8m=y z;D>EI6^*PRf|-;CHy21YTJ-TSiMY#%vPjWG20kch-jqc{zwLC`f=fy7#IR$Ah}r_n z-~)wYoXm6;v0qxB17L=bqiByqW=-a?v(LNWQ)wo1>KNS|-Q64MTEiK}w*e>Ep$C`f z*f8ogOM&%AJ(Sjfpuk307pPB7hEtrTw}2SC&C~3_ex@)U@!@Dxmqs};rYuq_3z9A6 zqOZWtGDo;nd0^k&rB!s5XWa^u&GhB}Y_j34vDPkbJ=)>KU*Xo~GNImf^7{&Tzjxd4 z3v*mS=L|LPPqf&ZDQas>kS$VC7F}xmW0~|q5dx{y*|{m22AXm1WN+2-GV%DooS zGI0V7Gy-Gp3=Gd}QjTVHoKoyM*X5k+xJAp!|15Ne+Md-diO?(hV}W?Xg5&pvdf|bD z<}zkk5XzhAb7J9$fY z1ozMTU3`L0XV3W%KEW!jWK;5fr*MNHL-&;af!y_~*wx(rK@JKIe(v@h#+q4}B)J)M zX~UT(GW(s}jPJubu!iXgpGWa1pq?qXt7Cq*z5^$AzKXhCa{p6|ZbL3G)com#b5T7(uRWHu%@N#UtGYnIM&j$$vQM@N zDA9z`V^8S-vuM6P0XwwJ8vkKXonsk0;V81G^$ed+7-pI3GSu9l7<`kZxMPAhuRx{u z1(v4Y2L%Tg2t$ZDu^)1XS5qA6z@MC^5JP1S4fIg8*AY9f1tf_}z=6V)cHq;o9r;^F zEEg!QM$q?#cry@(?N8Pdd;MBNn#V7VOF-`<%RxgfL26et_tUR3z9L-fLM`>eb&g!u z``gA)vpL-yvG*6^_C&Qgoz9S}BbXDsr+MF)`7Up`&5@ObPj76VUbi_QK5>LQBcLkK zKAI$mnuVPJgP?)^*-r&eSgv;O-e5CYcNb$=+MQDRUuV!Di)6oiA}MVoP8u=~oSx<%K*0fR6Slwik44C8Op#<7ujo?n5osG0!(P2CeVS}f1WXiE|`C-OnU20^*uro8VBQ;`GOPi zw);&}%yEhqp5*=e+cpQ=227j|TX6y#8+B}iPXm%U-ub*oOr(DMA&kTI>9hVeLPp#P zFf;8v#&fqq@^N{ccz-3#yO(mQQ!E9aPmN&-KJ86~|oToUj77U#X-xm=)1P>F5i-vYCO-;L}c(KoM z@#IXaz|Ja9L9)E_&P>>w_r_X={PGb)sXepr`Rkpr;@fD)Jh48|`;rk0D2cid=RIYYZrFX zQ>gR`Q9s954zH>+P<5efYHpO=gI4*`H0zd3LOGi(4nHaz-g(_8C3;UkurXItgsVgI zjUNBeBVvlbKd2H&g`ie=wgPFA@du|Mx)p7|UA>87w9F(d^Kex=yE7{$b>YwiOjkBO zmFY&2mM-#^Q42^L{$|f70yf7&!^9**<*W>7?PylxB9B!63@3;GvPua;t^kH3w!QTa zu(r4WZ2=^@5+A)A1-_UXklV_Qb358rPr4bY22GpA%dgK_Yb;$wRVzWzBCTlu=TsS- z4=^K+z56?}v1yP{^JMEn&3Oel${EBZ)wOs_>Frzv3o0&Q7jddem>I01w>tw)?AZR; zvFcxJ=>%En24tu{YZFw@{;tc;gVBrB0Yr zyCTf76JrDS28oGfiRhJ|Ibe22g|D7q@s@?BVR!=JxR8~?8(+d217`o{7aE3UH7O8D zS*=jPqAQYxy=*?1##{yFVAWVfM`w*&&>ZwmDiL;LA@4z}6Ev-XZUrX(IC&_0XHXfJ%GVYUBqlBYc3~C7lU@7jR3-Cn3{Nd zXhTv(yxZ*V=exMFpvtRv`GoqX2$;(k#eyB?H~kZh?yBvjS4W#&yx}a#`?cDG)dCv2 zes&eLzz72){)RLUmF?O-uly5fD_u6TJ*e#_*;GAx2P78kwZ z85oF6Yi;8G)NdQ_Fj^d9JKYs}m#Y$rBl%R=NBt9c)bmydZ`0m4n2!CUtGY=Vl*|wc zdV1ASJ?5B+`nu||RYNYo{>?(D8AAI4??kYaIbriM1d$Fbk%)c~2Bj_^;2!?KKimHQ zVeKu0B3+lP(Qatm-KBxX-JQnW-5YmzZ`|FT!rfgPcXxMpceu6BITN$zd~+kdnR|b{ zK^0L^KUQXC=JRCcTH8i@hg8MEeEeRXXgAwr)BN5VSoUDBjCgqGtHqV9cdesG+Rby| z;%(H2_|;>ce;98sV2!<_H`vBKGtCZ&iG4R>zPQ_R=h(73Ok@a8>Iq51KUQtgn^RYc{c z-XFTT^QLPa&(^KJkF(OINiMfuK((`}(Ki+9w02D}8Wg4#+B|WCOyVIYz~pDl}A#=WA!<%+==@ zrRSK(^;0e$?{w+MNp8yG<(%Iv^(WrH_gjiwK8m(~LW#fWRy+&u&cD$NYTW7ubacXa zyd$!F{^)y&mb=vz9QOfZ`Ghq8C@X%pc_Xm+_*!-_ibdOS;PZtFb3rXiSK-TkZ|?&C zN|M+%M=6RAQ%N5eH3$4%U37lp6T(rKbk?X;mVFFP_tiN`kaIX7_I(!XgT_B@mFep3 zoLoX;t>E^}13Zh1@#<}tocj~v+=Fw{$9U`c8kYN$6MAyc4pQ>)Sr7OD;O7U)xuc2d z3a$+q86j78*eCT`=k`)Fw^IA_t~}8JgDQWph<(Ijb3vli#YbOE5l2U`gwy`8dO<8> z5jv}L_Clq8A7IU3kH_L%vi5S)sVELTK-Gz={iN4ca}9KNMw?C~=R{Df4yjb2`~G}4 z7D@|YwB}o%7%i9w<8EXZRqHGADjuaJC6B9n0a*L|TSM+Ic`9Ypkd0d4o$)8|Vf=6I zj@`{{8SUu={`&LRB(u1+t&^jOjlGqgBMHMlFOMT5c%XZJA_i`nGGl`lHq=!k+l7kC z!K#G>>ij}UenWKWH74Gfrsn<~jLf#A!5-}v4Z*Z|`tAw>x!q=~?fUJq7&|kBoDNK{ zw)1fQgv8pItZYC?Lbq8R{cQwlw?axn`H*oPCcdOD97zx0IO5tLFI7z`tG@PO@c_dF z14a16MG?4>&|WDl4XMFS)grb0omNO6^7b>edk&fFNoEnck>hCDajJ>8{&X3~T$BFk z-&CN&`o8SNfPDF)3T#U%{Lc^&&;Y_BWoB)pU}RupZ>V7O*O)iSUpOf`IvE@P$Mt%J zl9en9Kce@;ioIyI^?A-3DDs>m-I*s|@RSc%^w=~t4pM*1`W&oaB{y8O00QZ6UgM!~OjH{B=PUcQrj^h5MTPE`~BX-x#oY4WHNKPdaf z#406^<^<#l$*hk%kZ+F(m5!sDg52YT_~KYq;$6eKNfzkGuXL*d_2%{@h*d2qq1TiZ zt_*NaVJANmL-h7ATsvI>NB!1mva_e?6R+aW&-~+Sdwm4D@Z%aSwCq0+nV1ruJp}TP zyE&(3`yFq^!+U2IC0@U;xE*{GGo1M4Mbh6xU%@l*gsf^R-H030!ve}273zEi{*Fu? zH-XWoQ?O$q=J^T-Fh#6vY+r%YNbgJdjz7qpX;zN>1Dz~Y6B!jcBP&_rh1BWs@U z`m0BEjchXnIM;00$*&@M|I;9@K%72(EE;JIST^Zr*d|EJFD@&}<#F{ZG{2IYa+H7O z3da4>9!sTPe3~?f*dvjT*!n60_{=0m-QQw8XIOhh`OogrUlG8YX8|G_ILd>M`sItr z{}2BEueQC~s}HIw+UJC^3mI4H)D-;&Vfn+SKah z<%tQ9ZqbU0Xaxc=GqjiA<`(AR_2kRHx_Ld7`d|FJju2kLyPo}h{9kyOPK}{Z#CV?s zFL)j&S`M=YA2S@-7C+t}xV|iG%VHyA%G*f^Lh6P9z;qL~B`G`0zlRMPUZbH!TZHHd zq3bGN2Y|2Za+hpdJhjSgBA%oR@cfvwgOlCW#O{o|G<~AYlD8A+1<61Sx-`R;8NN|h z=OycXWO@5i#FEi{Yd;HH=lfvIf#uSa;*dCurs_$;Spw~(IG3CEgX*9pcICpF{8yj z8BGnvvYyfSh3A4jTf-eCWYelyseN!tQycr54JS`-fggvq^T^NxcGQp48{;Lsdd#uL z)rTxs%b^R^YJK-csA?e-uv248(s}&qTE!=ioR~F>W`9FyxT)HMU+A(+V%;n}T4D7v zJsjk;J!|j3x3L!u)K|TkjSl(~{KB#kQz*P`$1_3FJa%Yi{iHK}3KtXuw-qH1 z>YKH@(_DY}mNt}H>Lg3iNUe!QF|MRC!p6)DccGT?=fDO{(=t-t`UlD@@hK)n!G()~ zt@?SgF3+Ve%@c{Ahoa(6lVQ!SpDtkyQeVZTxx%3EQF=6Xp&BTtxBYUb+Udw5SPhRd zIMCwM?LY1N=u=s4N;AwdGORGr!i80E1F`F5uR5rrHugK+8BEVzaB%R-4zHqFw)?YTC;SeElhsdy zZF9yNFa26vgSp_IvvJEC&E~`v1`E$3X3kE&xIy3@mQU4Ad=OKDa==MzfKI8sVi&rS z(lpgs5-roG#DM2w{u=S&6f*{AVHhs^4>;Q8kt+s}oJx1Fs?VuwKpC&gw_vxaS$7cP zDKqRZc9hCv%mgdYch1_Qd4>RdG#{nvduOk)fE3yd)UgVse4fOc(ya?9As6KLD8;ch zdZb9zi=m=2~v|KU$qEz{nqzj`3S@0)X`@DG1C&IAi3^&&s+|$ zpG9xebY>`U68wkEn=p!NuG&?a_Auz~vlHo#Cl3^9fTB+&HZbN9wAtOg4zO8nlJ2$d z-_?1}b=HorC*e$uQ4d_D1N&p8ZFJ;B4Y)XZ@{^!7=&{BjaiYiM!P$rdPP{H^uxg)B z;gS|7)BYUM(gm^f-tb2CadGDR|0qNd2P-r7)1}^r;p^6=xIy`C6r>~;7Qt3UHCc|h zF}89%PMy6mM!7v+?!GaWdoW(EongX|-l&hWXeCviDfcW1>PpN2|HBRp(76mfAkP$J z>n9kXIn8T?dxB{$KKL>Khy+CC%hY)o$(1Sm0X;ZDD*4r4c zRa8^?!E7DDEF4f0@XXznQ$;a@xa&T_et|0(?->f*0N@gqL#fhg-prF|E~8#2!_dBq zFgt7pbztX>pqI`RyhYDJ7`2Uof2G#Gf{bLY#(6&q3I(OkBV`=w?%Aad7-Wc0)XEgb zuiBO8{wc2MsrrkH^Goy#Dt>-jz{#G~cZrqg%p(D~eJ|`bavrG?JfM0(h%JN`)zV;> z^_^{gI96c{SzhB=XA|X%XouHgo93h2hzr#dth){O<+JC_bKciO!5NN`@14n<^Y&Gk z7BCGGUxcYTRL9_T~wQPhV=3Bd3v*B0dfcZ)K@QytB5yY7_LJIL^l_J+b>g zFr(iD-za>@==hq1JjbhzSCTqg<%Dr{vF;6We^xW({p#hY@{^ zmK)@5mq#9yegR5~D~z3;s;^LrG?7*LVG0t$C9T75kB<}yVR04&fTE#vs-IKk!+sxj zlKj2YN}YryTRSTIJ0#yf?*gdr`BZksRlzMU+j@lD638ks&JeFUw_aTpo=(7uopOz# zs3v4v_jHxh7B-8`dRf!XHh$Sd5oB84tfZ$)@eBI;?J%JV<%Mm8_OkDci`InQ3S2a8 z4XQysi|WEyLHf(E4oxr|ZCdj9YGu8Keczm*KuM-9_ zoc+T)cGz;5(c$3W^ZEXP@&`IAaqHJWcQ3Lr!{L@Z6bH96{pD7ue-Z32@~U9KT*c43 z*V;+?hSsW`Lo2iqt7DOs)uf*IG{BQW;7GgvkgA0TY@o`Xc#s{;nwUFYiHk5<+V6a)`zI9X4l|v$QYtHgGvvgw0(uNj z&&sSH$Ix_P`Wq-UNjf_cS&V0WzKr95X6zqoku3mislapw3D|((_+mU3pMu_Y=uo2=ZuS$( zQU*T6-HaiYQ1IK*dMfTZ{Z^X=k!H<=;9cyZ2VN;gL2p3AhV&9H8GRm{3Uw|6XIlxEg|17}`EcF~5%#6(p^c>A>tm(w9frDGddIm=PPKIVi)&@o- zQf5|Wjz)%Z_GZp{j{j1dPmGb31Z6<%*L|@$*8q-9x+3$3>A+^e1m;2p`ssIv7cpm6 zDmvHwNMA=d>_*rMMm9kzFUCNK$GSP_cpF-M9i{qBg)6iaITc-9HB~945!bW*+&^n+ zMVY6#i_C+NrOl`ia_DwUvu;l*DXNGuR25Mm-ePET*BHT^Yn0?x%!<%k5u+#adl}c#) zo{iB_;;>hBii;f8jlOGLLyChc^FgvYLv`e%0EdQ(tswnuHm_g`pixA@|Qy+adoigs-ia)Hi_0ngbE$`ClOXU!j%hmJsBJ&9G`%<+Zpcxg+tr zJK7ocFUo}kr1T!?B_FEctO8w$Z#}CT^?1R=!mMXrHO-9#t)Rip&0CsoQRbeZo}zCf7wI7QH`jcM zu-XIyW(ow3`+ozRlfI>y!N0<$jI<;uKdhITQ<3)Rp!k*2x8v#Va17ag1#-wRzsF@W z4aGtxBbNb!!)}7DXrM!$ck95C(2>x&UGCnVWE za4L8BKCpzbfTM1HDm6PKaK`o?ebX;knxQ6eJqv5R;o!BpVYk4uJhGum)fh{^c5HZ> zl=FG4FGT2Q?>sGn;nCt;9Zh5qdq}yhB7i>TG3G!coBSFsOVF!Zt3z%nE{^!8` z3o-do2~d8-@?i@pt$0ipb8EkE1Y0rT_zAzi1^mhV`r~nVm?X7274CJ&#|@bqs0o#@ z)CSJaeU95&>QPz-c)dM+2mOhVi5<3d$i8LjfXjF#Z@R#^06Q_Ifd}u2CCs_Qu3y|{ zY^Kk%EXE#%MVZ%k8%bnuQXcvW+h2FfAZUgtHI%=NT}BXJP}GZY{H$cw;v4s&7DPJE z)~~?Lh9o!%*OMAURdS$FJ|P+tRIq;ULds9&%Ak_33bB>~JJklvep zDr(W}X>cwD^%B~*GBM9uZ%cBa$ZiOfrEHTH4If_$yt;9HR;F)rJ|3O1vh({^X7Gwc zsPH!nrV<(k?1!^es}h0DQK;1J{J=)dB>0-PH2$HfzD5i;^?erEXx|zw!#2EHLhV%L z?|d|W{rH*LvCy~wG6B#WeE;{RP;}JOH?uTzbR!XVwy|_Jvo`q;H|_=+1?a}W465vb zB1Kx_x&W_8`c%x%@(FS4^*f zNp*%&Yg8;b`nZ!~%lsHSnpo2wL5e|keeY?7Ey(U&?v((?C;p>l7w1xtS>CS-vM4+$ zFY`K(l@Eporg6j8{mHY!_GdG+(TYu^mtIGrfTl^Qgo=LGaaY0QMcNkjHJIK~!*0u? z)<)xCMju(`gJ7pI4iuTs$}u$x?T3Yt1mT^+!?XwiSII+lIDZd+T3LVx1ylXl_uAv5 zzv;05g&q&>`PaW9fCUh}|3T>dPxStJW%(aj>0cpW_9O}z0;IXo_}v}v_PXQifuXZ{ zi-$|vV#KlVYy$d&1>xh1PXfEYDZ<0w!K<{*pPoIkz$H(Zomxn`?5pV+?h~y%G<3r* zK?Z`SFHV~)0WRDbZkwKf| zmHNpSyMOqdNc4_4Du56Yz2mUoYCVp0|A*{VmD?4>Q(hmO-(?O4hO^0X=6Y)MWRuGm zlcjk~vj~<(v{H3T98tD2aXlYwMiz%t(cT3~Vmw!x2ma zjsPnj*t1q6X+Zw+?(BW>t1W~p(a?CSe*Yrrt?~9$HYWPp`}b?8E!Yamo27^9UA;xY zc)*(>GL;BG|H@;axEUvmL_Rc_M`D9E*A^hXrI!t0R)dmUTru_Njo~|i#e!ZxO3iKP zWIgfzU{tu?RaiU{RR|D7vKUnD_RBm9*ls82|BAu%oh)q9@lQc|*`Nk=8F8=2K&dNr zzP+GR@%bs5pcCS_LCHnE#oQYmM`~E$RSkIdHnW9KP%G6Tfc~owrgUOGY_fVA?#DTk z$N6O|7A@lkCQ(4{?F?tytBv*TvkX)v+6)>Ko@FG)Hw5|m|Zxt zOsuG{T8*}^Xrol4K;uf~+uzLam#|khh)_2<@R;Ne_uuN@80tCd(eWD?I9WMa0;^MJ zBLTgCMahn23l|h&%)!ri81ba3g#-|LrhOu-Flv`+m1;J6wcSFq3ytsn&^=gc7 zx-Se*UtYi`kkC959QHa0Ms}Iv#{;W7+H(&L>$q;;d~U}aQl&rNFE4$*WC#@NeQUDE z9ALYtjq5(AzAzQrUZw6_jk>gB3-`dfqAtZ|<}f$ZlhBOqey29a4w9SvGjsK&3~LFC z88gUpTYXIjJ`}sjNh-plI7{&m?`Hi5s_hbkgxP(#1mRL10@=>W{hw7hUFh3QgQ2-f=Ce9XozCCFihk}2xc z$UpWGKjGwgYm;>CI(`37NI3){5OX&}S5)6II5T?&1MCp0LZ=c>&u456&1ZNSwa& zh?zO`8ApPs(yC9wgK%NqOO^`5=q1Wwm6X)TV(cX{)};z?)AL4K2h)9mL5k@TMqUw- zE)5B*&O{^OJeAq233D{LIx7bgWt_)}8JnJ6=ebAsz!t9v)`KGo{W@>eA&+e1mKfJO zv@vuxwrmRH2(z0jOp!jgH~!a&nboqcq$9+&t+^AbE{QF5nbSXI9eKSW8YNz+FSQh- zrx_wORs!De&Y~t(?npEI-$9R{>QIb0ef_2e$vd7zf8O4~=+QYFAgp)EUS56Uc_-l# zURSL9BIsc_an$mYHP|+#QuN3y@F4zqNsYInDoT47T+(X_of-u+a3 zquf01Cb0d%nvq-P4TY0fM(r2SP~F$Qc%B_cQO~d0F^QSf$i&;+b2CF9#sig6;yk{J$qMCohof})6Tx3waQL6iDc8sMHO6x05&v& z-4h>PNS(ee>T?+rlJ>YP0CbAjhb+AUtB4o4m&MAo%yvxj6(`-pTz=v~xs;q-d+`b zsnv`5fx?wd6#oKNvUiE;39w&Uzvpe;P@6 zRiD`1MLwGXXcZmucBUyEct(`x@M@92qibdUrU`TdGK1dH`vH;>Zbm~=_!Y$N!>NSg>NU} zld8w`IcH(0CsgIe_jx++%a=TZ?hUB57{{%MkBon7#aw+QaJXT}qlvV>=&Dm@Sl2!{ z!wNJH!q%k3B6hzve?YeR&%@zgS#>Xzin11%RV!itTMC7Lm>HN!Mf41S`+9EwG;x8- zsv?Ri=4WP1wVwLVU9?2(_8^P;isIiGKh36pC*;tqtdeq22U|POj)MbM%B&hXmr{8f z5WG(6WixA5z7|Q90srwKpXqJFCJ@3uCqwEnk#+m-y}9qX>GF{6^Z5kxhkvb{A9GhP zL)Gr9P0)3c9~9bmEhSrVAaBY>cN-;YJDJJlXmBNA0bxZheky%4E?F=A9(zHq43RMA zto@5Gh+@Brq0*U<2)|0dvtho7;<-{1|CF6D>w2A!KI_HoBvzn9KhDU)>D1BwiE@6_ZDH;ceHq z22i;rtX4d|!)G)fQ6-upo`2?Up97lsiKxe`J3Un^R~43TURvguW!vhJY{hn?;xcRe z_QR<8c+w9zRF{&z#SaxC$If)Dh;NP}@rbdp9s+*dwY+ydw5leAUiv zpzcGC;`Mjt_;Hj>Wu-Wi3fAMNWmz)d2SQ-tgMsEc74OsBT8=H+>*i(_rv}X-IA=Aogb_BWfna`VsYkTaj=>PEL*m;5A}U+2r0gXV^Z~PH09D*!X;AcaXprnRR&V2c_-jpToZmSe zWc-|XlK07vyeJ-GMs5@uZ2b8vj?SDUr12$gf`Z+!8}($()T7!T>kMhFUfQ#TE0#*8a;nQvXlr`M z4mEd*r=aaCn1eZJ%{T`ofcj@GfOyDm@Q3JQf7V$QO`^s{p@g5(2%HfcC<~lIHPI-x z0ZrKkElo`=ml>LPi9~8W-lqb)N_R?l*>|cn?S{FeSA3=NVDWFQu0H%!)=lQ)+@;JWDVV%)iHH4mDXSKD;5$Y~HY;)&%F5#bUBT7et>b z&ylRbK7~x1JIs*)p^aeXW#Mw9D{O?^(P?1C#1-999>wSItl)c?Lf1}X1@-HGg#&ye!nS(!Hy#IvyM9|5W^LuirOWYQJ6jM zJv|yE2ao9`+@3`0wP=H3Kh*lMG0HNtaCQL#x733-nIi}~F z3MJG-Ae8w;+y=cXd1E}+`EjgzMcHBT6k2{lR0!!!B88zAbHS$X$L1VQ#L&%v6SYW-~$z9hr2Y=g;^E-nCO3cJa#xXP`gaUkDue|jhF)4RKLHj2ivXBi_Pstc|o=GAB{i-4%_ zC>0l$D7Q}3zlm&VD&AgF>)Oz?#_yeRUCt3M6l<6;K=MB6whcQa?-dg3#p0;t7I9wo z{I}HpSIo!LhUzW?$_+6CSF}a{&jam0W4@X7KW_IGjSP((NCf`=?jOlsLC@viHOTrk z+1jCveC z-S6za6zo)o`m{_3T&{x%GzX%N1UA!4ZPN=U;S|zm_6*x3rzi2is^d>fJGz8fHeH3` zTgMW+r>XU>x^o0ZZ?I6B)347uIi|PP!acUD*QrQt8y=`1t4WP6$*o2^VH;LZ-prJ#k7dIX-jr6z zp1#Z(9PHq>>sAQKDU5bm!73byV-g$F#~%XFh%QDMl}0*wu*=?{Xcs{pB9(j??LD<%)ngo}oB!kJy z$*Fj{&`im~uQxh5SZCD#I)DU#O@o!LtNB~2uUw`)=5>#zthjuStShd)qBKFf(l3ae z->>Mzt<;=Z_F8UIFd{=vnW1*vI^d~7Z5FlEw6X%OX2UcQ(PQ~ACW}jnS}r5mPIW-s zSTbN4ZpT6rh3+6Hbjw@xaRVwsz_pYERlneeQ!6>LjkRSy7gA{|dP&s6l$WHcy$Us| zO=VH|w#;}O*)v!E(N$7Oci{oJZ`|k41S6AL`ciYF68ume`EKLIO}!75d)V+<=Dy8_ z7mwDUr#2BE*?B}Su2Me@s+aLoWqxU{A&dD3jfM`Dip5g5@esCtiRVr;)>D5}Cv@h1 zF)S2hiwOKZv0YP*Sxm}?&9tv7+>zhtM)p=ouH)32L#ryiq9N04*FsiS3~OpD3oD>i zvI)ZG5&~3$OVgKW0gzy$a>L9oGEF%V$UdpG<_EW~prQK9=#7zS*~x_|9LiJz^U^3$ z88zc2iRJZS4pBlxCX;(SOaiE-J@caWLN0X@9+GAUZ8G|Z@!cb01`diRwac0zpo^60 zL{^j1yM^&X6eKHCTqrJR-L0QvbySpUXd=qhQ4@GE*BXqALM{RzkzUo5c7U?_6UG|` zs->Y&`X#lIWdQ=I#wt3%GgN5cG_`S2KEJm)SriJ_CHu3O3EzwsN~kq$pr=*#FE3ZC z;vg)OQpaC9!roxPgzrWJ0^Qg7U54s)^Axn-$)MP%+eZ^zA|^ zXdC*BcI;uOYQkpTV>d*xiyRf~HR>s2H{4y6&4eAAJ4sQAYn#I+vKw1t>#<9r%0pD_ zlAr^KzjCpmgPr6`f0fkwUdi0AoYbtLWe{r7&r;YtfxH-wQ*8P@NpwIQ!9ibW29-@? zXK;s9am_MRvyT&|tR^U;AaJ2 z1~pr0^+uAG7?0}u5DqQ{rrUZ&KQ1uZsMmZKmyG#puE^e&dvvcFerWW|(p)oL|E6$8 z@pGL*MhFd-zJ~NU(|hhbps^}k8f7|nq-lM@G|Cf(DZ+?8jDL_viy_q@tYiTBt||18 zFG^EGu=guEG+XoO3xUQ{Lhxqo!(qXtw9eg68mv4@-0waTOoH-+os?@z9a`z+=3Js_ zr$?=2+`qyGY3zP->xGfgTqit%!Q3+M*K{N6ki;d@p6N89X^-p$ zG5BO_iNVe9?LFU0yQ*NPdPv^G8I0GazQ$#`Z8v4;he>jkZx>S5E^OBiO~$&mzJ@M) zQ5NxvjBtoh`pMR=_}jQqrzp%`o+JSlCjx)V76u=AMNe+P zA2?`Yxtu)sf>T;wy+Lc1bKYokU84N-Sq7|$kH>d&mFk+eUTJQQb-S^cJlxr5lR#J&lyues zFnzxDo&$mcB$}a|sXWTzWz5K6tMEfR>4S&>m1XJE-v-D2@b=$080-^MCgJMyc-Ig~ zvl&o;eo7CoN`!Z;n0-}s zMpq~Oz-<@9y!mDJ<;eQzdXZ%tJ?f#(-P+rg9@Au)EN(yeOCM$x6c!$|b)P`;kqNF@);TN#|eJJeQKDCB9QeN)be|iienK+aJf$-Pmnx__5PmObq?ohSn#%_amo{ z!tq%N4792@RaRJ0B}1+g*G{LQ=q!23)G}H`ZHKwxHH5KnP7cYE*pP~|@ufAxU(?fe zF(cU12&Z7#?3?vK2wTXeFB@Lglf!M_rM-N;bq%}d8aL;Nj@XUQYdoS4A6vT>f>h$l5JxI$`V4!6R(X5ImCq#cfP4{Gu=*;C|2f&BT{TAB_94*^W&r4-9<#j?Q%OHxH&* z6~70fYm5-(AEFFjElsdGarKy4@ld81I$2>~ZoHzQ)D$Wcz1WOkraepQM2BjPiT(n6 z-pW`Q+jy;z)qVXnisWDET0x?F>PvK!1By4e6(e6tEGTZO;hSbC^4T>@U@Hc#RuG#E z{H;=uSLxw*ylgEnR+S2t2V!f-@s> zEa+s&nKbDe*D@;)0r|hWUd4n@Il4UPg7B_VD;%Y0haChYbbGgn8YhC|?yrWx_eRJ; z7^6@OiGzk==%s<2xvXZ5IH3)CBm}43<5Gop)!)*q)nOXGY1$So%pk7`$IMKrXs|Fl zvW;^vJ0kF_*LNbWLNbOvNUepR+alY<+YtbFCk63XBlEG{uqzO&^dUV4e%a4#qYPIC zC1aJ2$uC7A8L10%r*%?ix8T9&CHh*~4yh=_+Yk@%1hh*ZZI9Yp2Ac5Qh+BX`KtqQw z^*`iRk8L|x3TW%+CTE#;1Ha0+?BfJE!~3%1B*e@X{t z8v>ccQ|7R8Te#GhGm(ki@y%>q}2E)&$ z^E$`fXI{;|dY>ct4-q(ld+(C~9joUkMlg)MuXgwRx>p&T}-GJI%@6hg8KW9&6=+N;ok-Dbq;oc`106ww=ssyrJ zq{xL&LtL#e2RIsO;*HGlYH5QR`}o=EB4tyuX4epsVQ=VNZ}?sFQNvP)w3p4aj@Inn zoVo4kZUXp-+)85AB24ejY4)yM?vnp17u=>}dC%svmiQmR)bO?p9HtNeFYE_{%q zo8z*;DH!9v!$?OxA-v&`$xO(Y6&X}Q<y}2nh{>gZSk(P*gY3N~mO{GrEo(c2 zk>^+FuiTsoSg0crRNSZ|I;P#umnRbt`grG$sl10eO30$kYnroQ+kU7$W)X7x9n~;B zS^`Vqvu<`8?osLDEfsimYFW##^rAaZN>fS`TG*b(I0yS`I+N(O zFK>*#?@{Gec`C!k1o!`XwumJI5}WRE>1VuEq>yZ)nG8MlF+O*)t9lcf?+54ddz0+k z7}SV(6D^GLg~8o0)u?;ZHV)JEz28V)|9*>KBL1YtPu2}3+u!VCy1}{jdsAtr?LrCO z-|Y9eA=p8Amc1@f*nu$$>7opNDjJD{HyW`Xq41zgzN2b_C6C4hP;*E)#TzI^;TE}% zgq9{}6nfexjUP3hiDan58qP~UdsFL!#nw{ zRRoQAnWuP4p^cC!#d7C|?Ms#lbry&WyC|2u$wej+lrh{mu&8*K42)jRi;t9y%>QKJ zhg05F_Dw4en^#gUEu3C-WDy^itDC20$sQ;$kK&XaSC(&(>z^lTP@z#tF8y%=-XMji zWKx!_B-c?=I`>*C;#DAUqNzOkESe#zJQbcmMgWM|>d_lv^nkSAy-e`M=f~0V!!}JL zjz-=sR$Ur()khw0V9MJ1&?CSvXkke}dO+DjLLkJ6-ace*3O41HiP6L@YysUXp}oc* z=Mj$)<&+C4O36AC(R%Yc9?ru2iRI2z@iL7wiD)QaI6l2hn_jx;KH}AGfMjkfcBvNR za8@FI?xbNVaRr+8o)erN{DOp!9W1<%S8C_ZP(Sq_)p`k%?RPvcNRTQfB zmYhG7dt@qn?yLN0PUn1Lny;oZCS9x!?oxg!xnz?(t8+=!6*NC-?YxO=BB6;!(mwJd zOIx3|q=V8Rpy3LNI!#+humvE=S-Knc=*!tsOWhUwB}!v#ToTcP$9+NUbD6~1R zd#v@zwjA75F=Trw5|1+OyAn!JsWED4BDqQ#y-I~h>dSW-`Q%?&t*BIUMj@r6b{(zQ zF2$!c>e}I4^G?mW>&0ccbz*ET3PyLF8-#FiNeV2^R2zMotpnEmRgiJ{pXk~I6s4J$ z)Ava8>~j(%<{92|S3f9A0eyD4xp8lckgM=js}%!!mCo?`veW$BVtCuRkxRfXUtNIohcTdgTH0)=+wICj`}@)CeyjjrB9339Gc@Rb^BL0&tjfQ`8%!C~m& zm^zKANdO3RtVJ+E48AFe((l7%@We(!Xv+8yd`N@ zD|3v85*2O${vRrHwhaY@LhQoxMuv6AZIvU7queib{5TW*=W{eGw(6MTxE~rgX1KHi zHjeUA1LCQl^r@eCRv*2FA1;pJPV8(O;4tC#k9 zDLAkfIY%5taEC7~>E!#?rQ{6!F5iE;TysxUciUeWf_nZqzbx307OC%;JWs^avc_UE8LBqr+njQ~Oj~>>e9X-M>1;-F2&L_Fd zq9jd+!EV50nuX%If zdtG$fB=^vib7kz>9U>K{r;c+tY9$+eEGINK>$-a3?wS^_@H0YS1xO1;5Jd*53uK?^ zzVc6y1bH}_+=$tTbD1njwIfjp%FZOSp!~QGi9zgVIQj2wPN(~XVd;*iiNa=<9UmNj zgb3c$nm%M&!BlsnUw9)3qd(~~BW7^wWEC4dMjF>AJ|R|V_NY$KJ2qqQy=50(LDeroYd3-oJEDi&3F7a0zn{LxmOoOcUv>?@rz(eBmVvsc2#{>d2%xJ(p6qF&S~^Qe0R~s@+IR zxVM&7zdT5|uas85R9JP_MZ{~=@{6Za%L})mX>CD!83c;*dUytp z&hgm=zTm6kz-yVLG*8=Z8~6HaMp~{}@BPY)4W|*&wPXR;Pe=OWHGp`=98~zO@kHH8 z^!nTJ5yR8!bI!FY>gi9MH`IEfS(krQLBU+=2hhQ73QmpEym{; z-lz-n`%eS zy_>3(ffDxlltA?XkTJF-{m>d}X_PXBG_}N;z9tH3DI&p|7+w`21W%vX#THH|of&1S zZsv+71&wD~e+?^WZKye8xw3f9%)L#rffF%;6R?*x%umC zw`Hnyf|zf}kcjH=42CFv5}@F)q=Vyw9M{iQhzdFBoGrA;%X z%0kj1YSRn&It>jAc^mhgr4%+&F3S>_K|n)=xC&BBL4vgBk;*Xu$$7jMFBdxoA~U*7uOokvVEuW z=;&L!NakeMmIc}6_PNnf&o?WA#b-0rMVM`HKDt9R0mFjmuHAiQtojJm*C$sxpG8@e zr*CxGDf~|D1EXZmW{lZ{RV;o4ajJRI>2J)j>F>A)=yXjl&Etbe-@8;=WghI?lDRx_ z@9n7ljctp@G|2oM-C9;<>)2_3Mz=4FTKKoKzL>$;AkQhj%?%YvoC3*VE=MDn_MMX2Cz2M?>1^(=Ie2*&We4p$vL|;5#6F*OI)8RzxIYs;f6)8W z9Y_lOku10;Hq%b(PQHz1dLr`-Y_Gh#QO0uQ)}?t`6gw@_=~TgNmnqb`+{XzR!~ts_ z9%X|`Z-V@P_FWwr#sk**s<2=2YM5HGO;5>R$Pl&woZn zX2iF*eN|c;WEns5?ulNnEJyFwq#MLyAHqDnEHM^_g!qNrK!-F-hpv{Jv{hJoKX2iP zSC6+on9ZTydNv<>h99t+v4KFRAC0R+MYroj)z0Kg)bwTImrws>-{XCd-u5(00TWV9 z?C6m0)84VYo3Ol@S36y^p>K5&^Ac+0SZ8|lxjD-wYaj2@7Vb5eMu)S7tg7f$W@zV; zs*KJ!i?uJ!1}?nl0kzN-?;eXGP!42w2rBE7>cY58?Sv|9XH9zdato|#ROLB{jImT( zKPz+b(9tP}4%em(F%s@_%TrX^3adn0n8mAsTVN+T^)gM3E{P3(l9%E10#r=y%%Ntz zv&Wwpu?J(s3?)T%^#&;}CV?7XNl{r@r8+=EtV%0&QWvmt|N~1i$YE(#ihPl5n0KXxK>PCiOzvy;gvG}S`ms~6q-dcGvGNqE9-B*G<2fH_xkIVL z2kCL$#v00XwD@k)?aGS14*Zh19ALzJMMh9)&t5H^vY-X-R*k|J?Hdro z-o+}S1xH#aM<(oA7;YaTGO*qdF(9Yb;z!HWsMrSabhOd zcWasYILAl%MBU)eMPAg0E=iaL+q&kHsa%p*8UP?R%1!t^_i0nHc&@_f@7PYk`-8vi zVuOP<OaO zIA4dPKpI}2rA;3GmF+ErfBLbG07S;f86rv^1|fY3pfm095dKGAFEg;$UKu2VrIa%^ zmTit4em>P@Rg~Qfh0LwRAl9AL0oCQTSJAXvon z3ah08&vX8=zC%xJ zUn)2zamh=2M_UjFS+&gk5svT+>VQ&vf)yiXRR;wbS$GtYk|xrnMLc+x@hu_sxa9Bv zk3|^YxE=Oa;BN&T(B-FL;M`9lqV7vg&zXGoGuohUIehF#P9MWt@lUrP{$u&#C-|VB z@jR&ueK5b#pVXC;`vvzOMt2N&_EW3(iG2E_kbPu92L(j6M+e(!1>!|LNbbNRW8{JY z021ghu2Qcz>7c&-I#@VIM51-y8KfZi10Z+h1wqqWlHALqAFS>$eWp|52n=9zN~KHH z2?dLZBUM{rkK|PazV*ul%2;*bl7}uZGso0|&%k6>irjP?tM*9+OoCb%#*e+AB34F> z9;{J%49oVx+Gq?Mtu;L85!e*StxB4doI_e@Bi?#);h5mgWr~!fW}`neD|m3CuqjpD zl2PL`Se7g@0x<==p%X*Tj=A-$Xo4a!$_KE&$qE>kaFNE(FD+KnD#$`cdpreXMA~JN zwKKq|Z%XiXHs&Q0R#CmM>miY(V{*!oDeiQ*b+gAgr*2EC#p<6soQKLGIN$rZA!vs) zfi!-O&Z0e7wN?iS!Kd#!{)Nt=c=on@E|-M(7WG;q)R@Vlt@7NV&er)h9!KRvoN~J; zmysH7fbgIs?xpnzgu(zZ*KafP{9%CDT)f@MMzNa&g8oYqG9!x)T)7z`)4Amn(EW-z{|e~t`EQ3&NN4lk^YZUZbA zYZ54P{)XiR0Cs0dkZN5jA(E}#HQI%&YGPfzv#qu{sfIp%+o7~}^9VPjHX)nhh4yHa ziugvZlK-C=@R26iNyC=;+C8iDQF%V-HsGN+GLTx*g8uv#H1&_Ff1l6}F zi9@uHrcO|A)1#!45u20JTFfc2*G(W;V#ZUdT9&4l#n02(N1!Jpx+dsM7NhgDBo``c zEWOdP2GL40O3P~Y@V#WuT2ZTln~i%>#mch1e>oUWc|dH^KS6w~_c1t4;!N|*l*|^p zv#T{?tN!FsXC8D{B-1^+gc`|AbDLslZ3?orYPaFRzYKO7)ws@vH~vN}nOR91qPD`- ztzE*+5p}s~a|N!vdt04gg?kn2XXo6_ z0^Hg6w*&2-2K<4!oyEPi@wap9cLVqi0q#Y<)rE8C-tEG@h3kLe*!2S3Iq~-beBs>9 z@&CZR{qlF?+yw{tSpejRaBl|5Hs56V}`zmLM?m5N(G$nnQ_ z>z6n9PI?5#rpr$-FUlSJkU!;2EM_5aM1MxysZq>b8Ajc18ZVJLoxN`=j1ZZf(`SK$ zLg^0ae>xsI~IJIWC!*-{i zZQ0VZs?POe@~8-6qJ#=HcE;!(X_l+gjlE0g_rxTLPsoW-r~i}E;imXp$>SC~ z`A?kK?L=QV$!)$|HG-3~eugUk~@W;@3(tR`IvEA#~nOFqk-DMv{8jD_OJ8KNu0 zbRJVh%xr;&D@v}osi4=Xp!H#Q^5maXmbmC6JH-%awBaLIwZM{FL`|=>^^>kqg&ot9 zZRKnQ3rb3^ZYyz~~9EOH(O$m3kNOF};f;m|(5YK_^kxjb~WZ+&99CJ*BMRUn;S`nE?zgD1(bS!Ns5ul2HhumI%J7 zgrC{|42z+2NW$ivpoE9c!70-kXEKj@aR@|EHUaWW3aM)4;7u|io{Z!{+of)Q!!Ru# z>3YIeBvFD}dh4z(gtWKNH_m(KT32cw0e!+fM$#V;O-Zz=BmJTy_R|FJ^rEdDG^|QL z5%LzaqHL^kthK3~eZ(uMKjI0=I-t~%?| zYqLsGbt^L3e=#@g)@I-|k9y&b_{m$~&|8quJ@~B+QOCsBw8GiiF|RAZuPXuVu2>sp zM7a@;eb7~zW|XU-m%%Nra}e(!wD7H`3futJ4XT|ZYG-Kw0j5AmDA@hXIWrAwtLosK#OcExw4nw?P(Q~fK_ zVYNK}K##gLc`#Ps=CPl>y75C&@q{(Ja0t&JUk{C)K}@9FEdaWGfPfu`%*o9$bvL4r zm}Z(y9)+5SGJskr;}7^H_0H3x@S=W(YSOQ1FUgm-f@nOPUlwD-ua}1@Bj#4w3e+CQ zf9MX;Ek&nrU;OPP4HfxuOY6&!)lvVTKh8G`-mhmqm(5Dp$`Gw6>4`+f zJwm1cM*l3}O}S!y0VJW?92dwtw^JG>NVm6|lcTQK77b14%m`=&zf#J(HegCT==iYs zu%~qZr16L_Pas$a8oiOdBSJnB*Dzcm8hPaMw8>058Sa=-AqlG+Fd<_j2UE_Lg(%cm zy#8Uqxk)o@tphr6+3kj`?%R8G2M+x~8{Gl?4oknIqZN}e56%l(L`tNQ6UL&T`ZlNict`QP@5bX zJ?jKHX6OMcx9zq&&Wt>_>Q*Q0M2?63;0-mGycrOgB>o407zEmA>c~HKBWTuitjXop zFk&ce4|{2SPVwJU+0^IMKv~?et^7YpaXYv1$xVPNRok|Os}&84>7*6Lh01uflICOR z08k;z0NE1DE*Aih2e*a6s})QG;Ip89IW3a9k<5bgt8O23T&s@-hX|2UvV>%sb(025(As0+Q&00Vv|JIlJCyU2ieGfHn7kt*?-=PL7rvm~L)$5E zu58{N&U@xbVvbMA^3RR-Dy2smt3tQ`>EVkOT_pWrJG|bueJ_ySddy(}CD>QCgASF8n0QSDn`Y}HmFOLV_1Tt?0U{H2 zpJwN*$n)n$zV~Ogocm6CMZrIo((y(%0c)BWK7f-JVlTgfNrDk}*xtwnZ;mocASIzd2gl{9 zCNcXGm~jk5WNA##`lRfH^{yS5apvmRo~HF)p?Yf-l;PXeSCdBc)vR_a4Vb$reU*8{ z5>*s20Ag1ZWb36SPA^(xJAX{0s7XH}#mkvXgVnD1if8Iz;TYvjHI}qhAs#(|m=G=o z#>@=XMWDnm>qB32QAUT^(<`w+A=}4o_2wvR7#u;Dg^ZrPc)z(0+Org7J^5fz!Jf{! zS$NWN>Oo`lG%&@(3(|C2^3>A|_z?!mHUV^dBOA!Vo6b37P=U7_}~uv5s3Tc87e9<*T$cC0}9l+1weGr`YvMe{Sfj#yc@vLH=5Ij_Iq&J62ad z`^Jh}KqoZi9yDst7AI7^7Qi(i!;>e{d&_eYj}3lqX>P)zrB}I1nkt?n{1%?;HP;E? z^e}g#N|njt){4DZ5-jOU+&gj&|DsoV<;gW1Y=173-4>uD#8iqffTu6AtuM z_fYy~-q|%c^dZ3yb_dvj#Gz zQ2tr|ylGmEbJPDX9rszIi`7E9(R_TPkfui1wXVS~i^{!rdCFR64UjQzvJ;f@(XgU3 zRi&;-7-Q)|bK$*m8CIz(hca}}jM^rbw!?)pCv~JY z^hi$~YTE4LFzDsu(j_I1k-s1blAZb$LWd;@=n_R{rK5T0M9I1(3~tfAc>0Qr(qp7+Obitt)ySek*^?g6*WCfA(wiIsY~ub zXMz~2ChA+pM7Q*ZfdYh2QYjR{Fbh@z6s0_<1ypZHsu@fc# za8qgNgW@MOT~_9|-+#~G#f}i*U;b*0XMWkI|H};G|3oqVucT#;s?Be%9mThlew}O` zt)zfL)Vx3-lcE5Kh)hgM%EhX2mHMr_F)_&2PF!sIWkk?CKa z*KxF~cqBL&p5hLAcPCA#GRarvo;|H=hi2i0Kj}F2Y=KI(-+ReOBcTtCb`Si!*qxd1%q!g(eT)#U2bpnz}J z^{%>nKa$qVxl|i*kI$Q7=^g$v(zFD8I^FBsH%Qlu+f=xQp5%=WC`Q(G)}=p;fLpOyUUE%hnJZ8Uc#} z%fr+VK~2^>&_;H=WDcjU%3Sv=Y42se<877U8Kn(fXP?N?J0Qp+Z5SdZmyschbFr#% zU9#H+59R)0W(|d|qX%>*?F5<=j(EmA22Tx0qEbR)dXCdS5UD#auUw1W<+HgpZ93*0 z2u&ktXE@#J)soIUnYduqIA(*q!W_a(e4P9vohFvWL7(8eqX9mf@F4j{c2oF>FOT1( zax_+f2ET_xW)Gzz_;_iPnY3iiBW7jBAZlf1p*F-2<9}IaFfxc38hWS0-7Rb*X17)H zjod9=RI!$vU+OY2yU(*t+z!oxSMW{rWkwN<;Yk}XgX%kWiztSH6gzJl3)u+d2aKt`aF@zF z5*ut3Ek9q27P|W+T=-(4b=+QN`Q0fB8-YkD?w&nk6Lk~s@{ys6Qs*z}Eo3$^# zEB*Pua4!E5;A)CK7ftys81}yfgYJJGTK-qT_;2yuoa7yKY*Cc4K4COA$Oe^s6@l}> zmQawjLWtG?iWr3=N>3<9IsC9`gi^XD&Ma7zAAlcJ{R`z^!RpPoBHoRR8Wy{N-{IcO zWsVpB%S?{f!wbLPC!_&pTw{AEDt6u(uAG~oiY zVAwSF4mXm`QtNgXJcpM6m99cMEw8Im`qmC~&^^rv!kXaoyVbdmtx2AiZAA{-?@c7$$6{? zH#vv%<0r>t6W5TP<13m+=;y@@Q;^eH(I^?4C5)Awre!8?=8w--zfY0u#vNY`V{SJK z`FoTVBvwv@_fw!I*{aY@^o>}7N{mvR=?w*poJ&f~ow=}xPt8s5o+@LM+i9Qn&w@c` zlSHqyK3z@p%775~rTPYpLGod|qOU(=Vn;3Ds6&uiCz|c(OkR8opLAb;e-fP1Eb?~~ z@Anzr&x_FyZ1G>bzve$@aMdKcv4AqpF-HE|vPKh!ya8uX3L|h}N#*;b6s^tQ6G1g(X+E@YJ5OQ%XFihb=f6FU*WIn`bQ03aZ7SEB0^dm0v-Lx#{CQm@;)Q z4PNU)P7V_Y^BgKw*jG?Qfz>3sq~2Y)G>ar&B6(Zjughg6O@J9$PpW=-Jm_gnX5 z_lc#8q>%GYz=>MxD_zzx_NrP=G-7k1S^Ea7K_&+@0BEP_!|BrE#qjDwp9~@BN zl7B0e;o(q`;Tl@}8u7z%OX&o`#i1hmShBC|TX0AF&Th=y(nRow!mmT%w*}E|WwOyN zf)jD;O7op7Zx8COSZ-8H(wMA>ya`^5 zUJx})+Xe;xzrxVJiEt(PmO^Byikf9Llo2(p$fBhoWk9=~xrpX$MH|rsrhi(c6;hyj zu;}m)xLn0^e?Dr$$M}*9HYT`=3l(Z$lHC{VeB7F&JHJ1k8=!2QG-5m>uK%Hv6t^6% zeDC9o%i)mojeMI4v+ZohdPT7K?Bjcw@r})}_2u;&1u&7Y=5S&1feL14C|oJoYP#%R zYRNsc!fHPFj0vm}T_CGK7|S09iEZtG`!lo)2Q02^x5=*#{i`aMRU$ z%1wfren06LxF*-(v4QC6+&)6^^rbK3<>vIA*Y^Q^r>laljMb>3AfyWbCUuu89O|oA z^y?Ey0T^SVhNXdNjQly)H(UX!y?k0oqx1ay#uz5KD z5p;Ak{7(u*j@mybY<1l4*yaVPNSvb9lfq2%E&OB4x)n<0achT;LeLi|KVkdbW^7l` zHwS8Xy`FXa(r#XS%$~mc{2E{k_s|dFw>B%bK~|xKoZjz49O5insHE&CRqPBiPMlNjP;&za8Yqpan(Vz&ef*bCX>`^Y}K=!-VT8Fn*|mnMk=~W z59BhBLQw-=$U%r6#$;DfAapU%!r?Yzuker|Uam7x=ESH(rSmI+D$z+duMh-X!Ds%z4f#r z5GpgJF=)yt3?6?ahw9VT>&~W42e#9$ro8$*s|U2}DVLmO+vRH1tgBSH_E0HAZaM90 z^_6s!{lTKTgU|e~5Zbz+0>hZD?kk1bHEwa$V(oO`icMOohL6@@WSFaHstaM7=P8c! z)DBZM?s}jbo-fuYQ4N-kvbU^-s^wGa)0wP_nqhy}&p4}g@f7#aNe>euwHqz17N6J| zadx7?*(KxCLN@5LfMr%X7Z@g=;_mEoIg_V6$BRnHx_~0?q>9w))IqWg)En zO845IyHnh1H(auGa2;{f^p4e51V!-VPwIfGk9xWd^N`q1MymsAT$li=gPzQ*wSuni`JUO|3VTjp|L}897qRHBG!}LL zrG!Lo?-&DL=U==vu@qhy0Iu+FNM>|->yLigB3Cb6{D(8>++Ab;K^~GzVl-89MFP+j zwuOFVQ^!Q_Byz|kGOHHh74%W0am}ilz$e^-&w&@olf$hXjf#cc3z*WHhVQz3c&fk; z`3MB}ThDqZ!iGWp7tF@IARq5UoFqh!^|)&5bT+Zex^HTcU~Ph{MDb3jL{u9>XVo+R zs4MlL_2S%I3#>#v1jlX`cK%z?#ILAf zik;Fc&n#aE^X4u(N-|f<{>s#V=K;^T0fg5HevjGbJ3lUHXhSb{P&)ilcSe+p)ehlm zDb-1l*hr+zv9n+&)Ro=l^k&$cGCK;Bk+?Cu^>G$~$;c~at$#QsUE_V0f2gA!k-ad= zeM$R#DVk$-Ux=1pWBiJx2{`t>1J_ti-{eO6bPPe9SY^xMnlpgh0qWqDi7a7^bSC-L zm0>h%7smGX%Z8_hA;Yj$J$%siaRKiEZmvVp*|maba~SVg=+PxcRH{K^E8RFlDY_7U zwkXbw`0*R%xe>kOh3C1UZ1IE8WIc*9`r`|zchKp6Rou12U$>$#kRoLfrp5~K3JQ!? zbqWK^VGd$YoSNV=-(t5Leh^g1F(JE4L>&F>6*Of{u#@4h*qPn}AvQ_s(E;&UnjHZk~P*Cuuu+D69n`I_nHnv@Q;({ia09rLIM z>ObS=e*?JScc4eSsKcp3aB3^}(56WV7B@DUF4KSn~p!L4eu9y*2KcQjshpUVs_(>FXMWO7sM7~1SI|Kx7C^+aD=0S(+ zHOa%tzy`K&ST=>FB}~GP$|o@d71^WJN~KHX6^|GNV}{$Plq)AZfYVelP->=Yb@M!K z(i3jYwwPa}iL>LMS}kq!ES(@WG!JBLA$JfrUp2mwS5klZk1n;d`({(TRJLY;Q?aEY zh%0k(eb)@Hxvp1}_>sIDobcnS>vy^i{iZ>- zi39wG>SjLzR!M&3E2yQU)VvwIMYLVl)rEeX4krunQ5KL95mw2dCP(Y{~0y}tVkzY>hzk@@NI>oNTH zXdl$~@7f5IJV(plwI;UHkz~WU8_)HSx~pg36W~7X*Qt&cJ+V7S6N6WG!2I+}@@fQ+ zwZEo7>rTVkq()}F%h)KK(u@e#wL4qWO!!$DSj!W?>^~x*>(+jDPXRGs5RttFlz*TS z=hJOh`V`#WXM{KJSWZYBaup98WL^`0pmxw-d8#Hb2RR%48p&f%+$(Z!B7V^S>vjJh zACK#?itJp7zqol)00650rMLWl9a8F!zdUcp|6F?2fc927O5s15A#r2sh)0kh5HN5j zM@|F~>L&q$M1U1?C+VhsCUZ)Hkulvj2@PARRMoC&Y2`aFYF4Wht6F|FCol@xxUSu} zvTd{UsZ;ycX;rl%?$%P%`17#Eog{-H_!e5ZP*_b_*3;=LtJ=SdtjQ?%oC2LC8dZ)#_{sPRs(`C&)j zBkG65<+pNnWvwN9bNW~-=Xa+B*z;RC)|+$r=FWuaWs6^mKLq@LsJ4FGPyKpW0{HG1 z#va*ozOM`X+)t4?Zcs%0yPP?+|S5Pbx!vp@Ny4JjvhnP$ek3~ z7#WXHR?c>iPhm}6K15g%(d+FlQck|X_l7BX{-y#?+m}}$c3~sCKa>3P`hv-9d*P| zZ1OvOc?(A|k$IZBl@m$~#a_6b8n23uv!hlZAV@}wVTy$fma4UL7F;IUVxeB8yXv_` zOSgewDHuMC@-)?A;IC$)&n|>=g33{?)%u?nDpwi_O-_`q5$i3R5Mb05zA9Ea8I3*H z@-8FP-lS+qP4~nq(RL~>P~hw9-M1Bg602#t(u^}wMJo)=$p_U^yK7kb?oyguQvxIl zqyUgK4usN)vjE`KLZ)tR#w2R^5n;eVKGVzVu>6zwAYAV3RjPX#IRqyW&6i3h>7y_u zFbcEVB(uu$M9_G^(;lGWG6T*%uz%7U=rH7nS=Dr87psm#gC8K&7h z8-V#bBEz7?nRj}_!y5$dCL8}j7COyIWY}L}Gn5B2Z zyd`vaFoM=7EC>rQpul_`C*ELohh%4&hz`B25=&<9Mup-*?02R(BibF~sw|@y2J{wGtgNMQDtV>or(q7mf&OHobWDOvAW~mq^-Y!>G$g=u7`m6WzZ+Ck1Xg02JR+9 zYO)}+oXWiy2!fYf6m+GN!JYO;tPHfSp|n1`5L##e<{>AQ%wYqH zoqsE!Jl(s@6C%icZR3Imp$r0hi1$!u3oV$I!d5~*PZskV7It<_llqFJFxO8xO8#l( zAv|)VA|>`H#1sr@@QlH`GK^JZU{}diONl2@i^yi2OKfHb<$6?j0*b{&q>!;puQo1B zNGTHSraUQV)L6zYFq?1{WE0G-Tz7MpD%>?bxF6R2nA$!l81hi zXNCLWN^9Ahg|`GtqMOk)yu~u~m|C`Hfu?J!dD`u`{Ibph30vaTg)N2RBffK`CSo;- zY>IY{0l^w0>>3}AXsnXsQm^j>BhK6@z{!Hn>2{!!Szd&Zgjp=*w6dQ8z(fZ}I`f6-8P=(a) zRh;l>IYo9cP>ZIs?kT8+k>u_#o9%22G!Sx}F|@K@7At^je60M3kZK~JDZzqiE=dOL z+m6G#WWUn4RrPCh8qQ+(tY_0a3s+pB$RSx>@=dx&iQX7T!1Ds1(MDW{SXX)cOc`|I zj`Dav4m`}G{-_kF*l7A5zsRRQPzQuhsTNI9)I2sDfUc1KKsqTZQ9j5UAkNKb`^t_y zmhyBlFv|f%M=LJjORI2lLfX3cdokbr;UB{V@onK`pGQHadKSwXD~tK!AQoHOgdc{O z>oOd4-Q)e-1H0KPStiCDo>O`997~FV)k+**ZBju^6qlt-7j9;q{!c^E;3Bd)+tAS5}jT~{EEETI_xb%1gg6}bxwzLlzS$?*mh%RC^-KFL)BH8{LJ(VbCi7zVmBtCUKs z#75e}i&M{RPONx#19nw5lp72i;tm6zLebcSScQRB$uRS7Ttyn_q<0iJ7EIu;T)r#BZ%gDO8F5)b0p&V5ff^V7VqYb zw3AsuIFqR(B0GV~+Jw^nk$($~rY0+RDWwsk9gw358)f7snCK~`C6&bJiaS(ujaoypn(Y#p`jUPZ~ zA2b0vaBndlzH%?()x|u%Qox;poniDIX!tLy3V|&+`81%+GXcgae(S#fO5>7FB)3W=Ry#6T5xT0*fB1wT z0VIK;GoFdgHPDmh^TYNXks=rPrw2Tl34j~h>V*rdne|8z>AOK{K<_;NxEp>o=4V8tbJ00^7IqFUx1@+sg{ zm8!#vAh+%>zGL)bea^%>`x##1jzcpJi{w{c`J=qE${rGL#SF0t2KF&qvx9k{X81F5 zGpMR&dQr$co#?+XSI`YmbNqq`n$rlm$uy)E#dJ$tCnQ@K)8lP+Cz6434x#xC3$=kuP`r={-rsM~r zX9ZU219H?W^1dbF=DHp+d?ex)#~$S$DR}`!f0#V7dO$Xibg=TBRP8&4* zeh8{+G8=bQLQ8z01hn2qj~iiJaXJ^-L|OJV(pFluina*-EA*agrQkQxPWCg_PTu*w zJ>5oIY2!lk)@wZefGhpHp1O41N;_U9?V!*0?lkn(40X3A@6M&t`So_~%Z03?ZhB~T z*affD+egD+lsB_Lpf>ae^Y4n?lR0%jS0~j_UY9)K2Q;7qoA6&&y2LjvvW|d{ii9)M zntu947o(fd+ylS+6EGAs^Xj^VuMkgu8w8zfs)raRJ0L*$f-$oG*$J4}v0sP7q<;Fc zd=?5L>(=F*&uE_4yjii=9>kfa4W*#FG`@{0ikEu_KRbCxZ%te)^_f@uS{^rDM{iMF z@0|Pzgm*1}8$(BLQ{2o|89h)g-@RAjq$z&OqjFU;L*K)RDcLHpk4Z%wp65bd7XhJZ z3sO_u28wh#pUl;McnXqk+iD|=33^Kv;nq)K>_Nc38fBhg11 z@9(zohLeOPeE~F2M8;<%spvGpOVq=`G{bK7Geu8a#Wygc=a)yEuK+#a=3Y@dBJMBD zf(JIU1#X_y?IVjvytIXn?sUFguJ`!W1$__Rub8%l9S`!aAh$*G?!4HC5{mjd!((lY z1MamEw}vn;&M3bLJ-*SRS4h0+Sw}ejS&vVwxT7`?oS%@f2XP+3$z7r8p`;cZx&^Wa z6dNk}#p7FN8^YN+v1N!^RaeIjv|Xs0q!SB8z5J__#b#;u0#{P{ zh1!#m4M4rZt(h{b7snP?e)?s&JI5=B+U49`uB+8le8NCQ zPt?kLa^M9t0uMeh&AA~<)&8?N5m48-#IQQa>uZ_oYxzsdDuE-6uV`ml|H>B!6DZw_ zVy<`2`z-OO!oeTVnZJdTEk2c6Zu78OVU@dEkL_wT=UXtIveZJOJW?4O$Q0B^NXw*) z^Zbi_61Kyakjd8Wx|aA=G#2RT?e)0Zc{2CD1mhpdlG}b6hhO?h-9J$3?LOyygxYXA zvv}_>huoDSTgmMX@XCjTOmNDp!pR^0Qf=8Sj-LL;Doe1AenND$6o2r`K9V1G#gV7(+0h@f#9SEfA#o&jhcdsNo) z;7R+)yEOy>!MT*>Etl1gZx+{Or>pT}d z@5gYK4*-KK*JTjoq+f4*QiFSQ~cpk3m*I7wo3-t{!k6!>uIn3ugNcq}{$U z7+!c#uKOG$CAUGKg;~q3JAM2hm z^0(n&p6qKGWSH#hMzCG`#|_M94|3xzBKhzEw8L&czx@M#hx>f+591vvvOe1#DzZN7 z9V@av`yDLu4$B=a@($Y_F7gg*LT8WJ{>N|QqT}5^5~lOH4oUa)oojgcgw=(2T&9SGgKbst$m{vk+)p8kn6$-N^{ zemrg<$0iykL5Kul$CraanV;x~oKFly` zr<`FJfl!Wi4p%g4rkr6J!6?I^zL#9yujXKc>th>nSNpfVsncHs-H~n(>EupYGr~L~ zqAbHSVj`S<5llXeBAk5^>|<>WX+-rxUs#zwdJ_rDxCJ@8OAJ>zMIe(sD+=jGaa0&B z?mMH)cXBSHJmokzxer7~=N#$C_>b~9JPO8v5xF-wX5Wbk%7IdQbTzbn3IPwq*z|5` z{|p_4*nm*zGuXe6@9-|n)WZ_18_R=&;Bwg^EdM47D3YEpcz6m`m=Inq3BI@?h3LD|bNAlnI1==?BZ1X;RNru{K6k@62H z%UGb{hB8C1@Jr$cq3DgTkDevA+!Q8P6i60A#RCy3 z(%aY0LD(jQUE{c-ya3HjF@Hgsc@p{2L6`Nsz@yhQ*jF^bz@@fqMwxRqOZ%j1rsh7F zr{hFHv9*1M8*L!_;z6A;Ha=UCWv8h>N&DbV{Y0)V{iX&c23IC35HPoBkvf=f9yoBj zgLNsJ`s|)zOI=$%h=D;Y4>LEvYz6xs%tzh53WZoBVvf1;CKyZJhqo;ZLGIcTFIaC| zv>?BXX?R(3K%fRktjPOOhA`-NGg$ePWl=}IByD5M$w+4JQS?co;we}{QxzS=hfwV( zSilCifjT9!Nz(8;R4cbyuNGR?5L%O!F)TQ?>iHeo(ozMS*O-Bw49A9stm8s+Ij9Je z5*F0w+j4*SxNVz!#5+sEIbA~2e_ZBaDPfceOq&;5s5eu`vd~s*4%>+PY|U56i`!5r z^_R}8#4$iaIw4yTqe)jK;T*Tuvo#azhKMa-aKM^Xb>gVgp80yeiXH;g z(&rGIPTz-e-Jk}`P(=!x97LxU;bb0GD&`}pkT)Es0SFF#n_+Yc!Ff^(N~yJ9y+0YDaG z&9GzxO(bxf(|)0po$g=pEGnTJj2@^z>_gZvT3JzQ+WK33YvMH8@RHij!9Y0jRXu{i z(dj5f8!Ga8Hxp9yH)6PfHz4vp6obU{ooSvim+fY%?o{+I?^TXHQQax9n~N89rLqie zM9W3uR^J^g-F{0XD}0=k2*vq$TgW|#A1PmjJ0tfEYeMV70fqi~C~UXxk-N4m2K^2a zRkqEINtQsl3YYFnvGteiY;52S?->RCoB?2%PsZcxv8~Gvmv==*#l(tjmq#Gw_*)M` zfS9lH7oMr*vepxKUM^!n6IkaO=pBp|qf$AC08tl0@Z+49{|{sD0HjM0ENadf+qP}n zwv98kJ!fp&wr$(CZQIMJG!UE z&H$A1J=uis;x`@5h8d18s1Zy9*>fCf4IMQO%5D`Aljoc_&r6Zl|0%mrd}Wn0T@1Vd zw)s@CsaM%)fm$O6QU1X!#t%Aa)s)lG7=)LdIIRt4h~n4UE!#`w^DQhdN=iE^qbr__ zPCut2C%a%StnE`%BFVCo9G27kO4TNI^O^e{TB->T?SZ;bj*j67c`}NlU6D3Ip%*)$ ztCHEdtO%Wc_^_<3@s9;!yZU&FJhG`FYlY&^l@Vac(HF)S?61K{090%aCc|tv&5Q37Wg2a2D`Z7p4 zXSY&A+5}_cu>y!aqevs9lQ?%Bo1*6g!?{b^52HW9rS!lgS`sAN@BBiw^RS!@IEp7(+@kgy3*s`%jNOSot zZa#3U`0?vh)5T(4uxHH@Ahy#Z;4~U$_6GS$O(OB8q|(v4hG8mPAbKOzHl`c?Rtz^d zsa@qzte{m0F;y?WS|rHVEK9&%mWW;)J5%VQk#O+z%bbpHt)(k&tHsMkRE=A_Xv=fZ_2vW{Cen^;q!ff+yZd=A&tw=n^*&H)WOQ`5q z)8U9IZZ3}|*RC+`KJc_CGM+-MpOmcli=lKjC}UCC>6C|9z-S>5nf5*LA@C?MFBs;F4i_EvoD&`ng(fLz--;6z=|~Fem-Y{B1lNVljYeyiYuN( zOh1+}ohH>TRS8ek)*7Uodi_vRVb}s?DdOAKvlMhuvjuWkkNcvJBJ{FSTG|Fw7{#lk%~%FtO;Pq?A%n6hZmCmGk}lgt;u0Lwd1M*c3#V=9`jc`ygBlVv z;2c{8yNEHNVmfYIHZEZ!g9p-wYFD1T@e^&gDdMDG=-C3i#F}j<2efrkjctQAbzlTC zZOX(Daw%a39$o3CHR?x*a-n1}bu%Q-5hc*}y(F;VC1tu&X{||#pqwlf<0*3CKX1@k2rs#4K9%XN6Uqo4$^|SQmsJpr2%sZMRT^re zKTf4ORIV^IF-J7eT52pkG7>1Ur z4{~yvjoMeAa7w1)B-E5P*Sq)i3c8Q#n_&!%)pzRRj+X?OHEaVH!4TiTIAr|5h5EDNvCT(pfvsfA8{PkXzJ ztUM2@peM+%h4!d=$2fDGXMc~%=qXd*-ofacv9s1->iV*5SlOgjrlr*1zxh0KDqQY` zDM+*sz1Ejcwh{zwMr@nSca%+DCv>hd!_c(NIO~Vu*v?#GHZgr=QtU_H*zpGP7{ay( z18@pKaf{js1AZef)7zo^lT-TsPuYj>mwr<21F!NEd+rlm{u6W_QgDu^@O4o7PNw|B zT>g!*6eGOESMrXg@I+Vky;AzFru@TM{>@hIgQW74eC`ud?j!c(BfQj8^3GRq{*(XS z^SkwQ9M zJ?MEJYBez5Cb%DYpRFj63`y$%VkR=aFJE_?1VlD){u&}TM5Bq+EpaWImLRF$oE1bG z@U)3kE^(-3Gq5C}S_3jU#C`^iUwp3>>;+*S!sH~YxKZbC0MoQ5qz9_;6)0Q3QY<}9 zhMve6PZ%}sUxu-+#KZ3R!|w3I?)1a%;KT0Z!*1B(e#TUY2?;K?I940dGc|sA)&AdF zCmx%SY`s^C-%j!N1jM;W2jWO8@er-_xUW+(t3XJHXi8xSS#Euz4S-g}ut?$cngr&U zs7R+v;A?9ij}zH}f%^oSs%5;2a?2r?8vqhQ@lTgVIyK`|I@JfqcrjL>S}t|0iYub) z@z$FMp9EvgovPw4Ig5~8G*pl+nU$KTBASZIQYwWcOBQ($DI1~R(?(qhY0AB0%G)B9 zHK1ghGE&}f#C=_}V3>Qz>OcW|yy&v@c+2q;RRMtNa_1WW4f;gd&>5p{CVtDvrmIihZ%qL;*#BIWnm^t;NZQ9N@noS8sOBqoN=6e{a~tCZ=EIA=p8RwIyQ zHvcl}V@_%Lk?GrFUi2m~h`jB9ANh!4J*K6%-IS`!wnQ3r|PNNFinh)p-zlP4R+Q$Fxe|>xD%#PC5al)QBDn#D03oa zDRUy#6f@E?_(`f6pd{Ueeb<+z(crvt0Fc~T1?C`nnBE;6O34q+I1J5H4&uuBt5&yt zBqv{7H_kg`bm*D4ZFxzWx9!Q&(XefONre%;lqq9gFO)KG0hKznf+?Ok!R0efdvPYw zx`HO%UH27bad_a4@4_KOkDuyrKBr)EqR16sIyU;T@AtALCT)VIiTop|@I*6`TNo}f zj*lxgGA)L&4It1kckdt&3xGDe;Z`ZU*_s4usuFBmk}2sGDmG}j6= z*K;)2Oj?_Unwy4Nn{PDN4Vs$=zfEh?6fbEh{NRVc=hg4I08bx;Wn0o#HSWl=(NAdru&INNW~igF^R47>(QiPNCfcJ0yOpt~yvhd;Y2I5X7NK2?>ow zz`-KF2Le$MsUp2MF5T^klm2lk*HY{tT>H-@jz|+8Y>Ye)(K95f6ZQ_@F=TY8opalM zIX(+R4TNY3X|#oZ=i=Ze>8PxX6Kh}Y;$R@DxSY)85^z~r7=_F%)ly>Q2FZa`y8$iQx_r3A_5)@KKTsb#SDrtjF+14%;QYIUCo5udO zNG=H1=6>Yi1MnEjpa*oczzEgU@mA5+Wk``ouHjP+?RCY+E&3=~rDx=fAoQ_kBSWCo zL0C~%dxuuE@Cl_y9eS@w3I(i!XygVIlgr=-gkg3iYy@uFp*AeZ*!z&R$Rvk&HP+!I zM6|vGCG8FhFMMsOFgU}(l~LTWOc1h{)jpil)(KmP#y$`+J?n4#0Oi@BME+!`T?_7pl1p7$W#aBYKD>`**?UHeDe&^-Mr_P9DT zI!|yNBL2I%xB=%3)!f2=!{j6!d(#=WH0)|STmss~Q5_l* z=-&-KnI}fJ<$)bZuHiNEfXG!&OAmzFsY3Q%!;6&XhUzvj9@s^cBrAw)1bL77HQ28-PGVl5p$!A zzGdd=Uw8zUV*quSAfxnkS$8h=@#*x507c{r-jlJrgY#rq^rhr@ zUMg$x7USO%nm>Uz|4g8SzXPU=u>&~ zbKT|W^Wg>yy)twL=Jfya7q)w6C>q`v5d(L87wkzjB6Amfu-867?mOQK)|t^eV&$H$ zk_)g+wI=eeNJJ9G?-ZM`Ys%!YhuapYEHsi!jWvprkGf$A+&eUrRhy{67VAG{X(lW! zG<20F?$c4i_gI>^-#O`@SizgkwAG~)Yvj4mkb{KEWd&LV@{j}2$P-nK^s%VprABHm zP&78(MT(6lemu_I5BiCc??Y8v=e?4aydtN+lpdM2o@rat-_xJPcPu;3Ek}>0Nd?sA z^K}F$4k;mSd_Rag@s=f8v*#ItxpGzdb?o^218%t>eB>r(ZdD*dq4mTF2o{>=>KS!{ zrBetNj7Oa?{{JG2`CJ;UQ1L|U8KY4<^#3ASN)l;G2%KU{{BKV;Hfvk(6epQMtW`d> zi>U3x=@0x6!LCH(F+;n6=n$r3C-)e*5uxm)oZoe3fZs+9*yW1z6Ew;vt)C^WpQ8=F z%^A-VIyC4+4=;g|)Dppsh99_-wg*iPN0cA)220YmnQC>f5L#NAQ z)O}hwB}b+EI7FEo&)49;|91!bjpkp5@rp!*nt4H)wM*M(cZThwamp z=9R6j7QC53$4U1YKHaqnFs6Z}tvs#O&Py!UsgV~F3Bgj%${&b=Y|0;yq2%0jiafP* ze90dOw-FAB_KLy7NhjI-Y!C}EfnfW&oXgYbpW^%HWR`6AFawW6TX;&Jkw1VP4x1wH zblYp9ATpPiou`BW|D|H>VFF5>kYWQ=NZm2O|<Hnsl(e8h}(5oG!XVtqClwP5Fu{Pb&20j*a-VadsT_Y(k9)5XiQ5gLJqJLA_vR- zb}k7DXx9V>3|xF;de8K6y26nf-x{+rH*O)^nb;Ag!#imA{}g>lc>aMD#Y-K!1#@Q; z#F&mAVBG(w_#;Yt2`Nn$Q*ehk?*?Js8OW?Fh)G8fl@dEFWrlG;GMf>3r+>dF_P59l zdpT5RmiW0>_Kx@xRj%*P9WytLH(bad)NS4?ktcz!Ki$y8ZIT;EH)>5k+dj*yYA5VY z1RK3XXV}?3%_ZITu-`roHr;lV+eu!it2eqtP7m}yGEbam*;n5EED8OaKwF}14*LD9 z1ETFUzHzP+(N0GT=$lGWJ+y6iQ?Z-Rd-iP(Jh~sg$7!0amudE+EY~alrC;T3ySpkU zxm^5f9q08}dME36l6{6+a=w_2} zZ~6KkSr$`@1)!=0t9n6Ntx7~dmL}Y4)l(plCY_t+1z1R<(ymYypJ!ecsGbGTE|oRD zXBCXUZR5h877bAM3aDXxa@d(B>hnsyeJ(5Th$Y8QO{XL8veRDHi}S7Q)#ZVkDa|{L z;=af|*YouDIM<7?`w8w*mR6#xjnej;8Q6_>ai?R^{WMpS>vl<}W5NA2&+j1eo6Ukw z$C#tEmx_WXAWBEwiuvQ>S~^KFV!B4eJ7>xl-JK#eo6~hFL-+OJ)a+$BNVn9=zZ?*5 z>L$akl5S}Kw}3^mDICs8&107lxFMkcjYlMaab0v@``BrG3qb?4@F8q6=qfq-$Qy@o zCkm0aO*T)|A~E`i&d3uieS>rsHn9ut4!poc$h6EwV&oxaG~BKrSnMk7vC0aOhAUJ& z5_^Ud=C({NPAZfxkgWnZ(UfKv;X<%kH0AfvB=tApn1HUr|4NgEyLc+3OB4Rt^69pD z86c!7=r*nyoOcd&pky7zPFM>t$5LwSpn8z~iqbq4%@62VjQjj5RLl$cu2hZ)_eo{Y zbibb$C`ar~`X+SdF?x@wDRm)e#_If(sReJu6i{7En!e0-X-S{PM@rzBobLyHYA2x5 zPv28&MNt^kMU@QbUu_nTyI3J6stg29WV{lrbA;3(%b zF%?&(&`(6Cbm9~Om1^%_f)vI!!W13_i^1DLtgJb$tj&XqqTG+bu3*-eq}G=~ZdVd^ zmLK>M8*p7sC4Wm@0ER7Qdfl#8=*75@=R|KpS%XCzGX3e(CjI0@b%1lMCOF0#_BIUCykB z2ZV1l01S~lc6FN{=Xa2ZC}ktamthG7-5|A^$+4K&pJZf`hq3Pj&V}zVKCQ};O>!n> zUJjw_WR6R`9MU`bUsz9u=9D+J>o2iYUq}v(K6CkBSoe)Ty%_6o3U}tj&+>e^x)^)A zh(mq8i`MZ0-=98eid?T0kGE{F=IgNEYJ8kK5}gefq>oEDvvRJ2?YLki=hdZcpu)%P zkTt4~RU~I^V)_ys zB?$WsTODl|Cbeg8Yo7<^uO|}|w4*LJ`n^#N@WZQ$Rx^q}$QNqpnD+;hAk+V>LJ9+;=K8Dc?~@Q zJeec3bU`E9ad70p1oyE!ECbfJFdN)>)T=fhRosF~vcEx%7r7BVC+C<4-I8`{tTA3p zV{*u7QoiH}yx04GK+f+;c8XNJSPRFQM;F%~kJfkNd!j-QH{zsy)IYN@NS=FYNMhO{?A&N%hxFtT%C32lsHauvx##~sjc|Y)M2tec2%j%vvB8y& zTOm5ky<>L_HLRF1*wE@Mx&u@HRCiVExydI|2FqNoSue+X?Dk0+3)?Ix zVB)GU5$^C6Su5WMjeCmNE>cJF&4!L!NZFfbcvO)!E!m*4yw~gT$xnmo0M$2&>3GFV^s;mdc{G149EE;*d!>8pB ztt3M{PIxYoQYM_BC#3Oyj|^+ZdO~zU;XylT*8Um6Im4U5ynA)V> zf%mD~KyRwc72AM-cgoi&Z?MkNAH?nX--qbq(rtk^+xp|b*{Yoga}T2I;+np^_b+xy zy@-=j8FnGPkd(8FI@KNDh{q_mq;3ec^C>sDE+76K|Ba`r)pEW-xl_0U$M=F)|8Ml} zLOxm^)z!J*G~u z8xSC|zNb}jYf_t#YzN?@>C?v3xwO?@mcX?;5RZwEMi@OsGC3};p!fOUx({;t+`;P+ zFr({F-0C4W&Vb++R?B75^rRQ(gw0yml|2tgkd_0{q&=$^*ho84>sFu95%cJN-cOk1 z0=fNs=UCAwAFCj=ja?E;Ewv}F6}0EE>l|RZmOd(+?&SqIMP_$Q=r0Vr)1KneAL^k1 zq1XyOv?uqOv{neb#BR^vPf}=eqip0*7kzX}ka$ol4MYj4%JT~F6*QY*W(p#o8LJJL8Ukd^)r&(PZBKhH0n)WRM#k~3)M%(}kC&_X4`cv;o><`k&`8i(bodm^H5y$oLA z2&v57wfJh7aq;In7chw~I5mW<6cG-a+~%u%NLOfwU0DV+Jlw-qz5kwE2J=A)1pWsA zVE(uB{_jGYj7{{Nt(*i59Gx8W4V``&2E@Nz*x1V4`ag2b{=bAHc^Rnze&kG-oguBt>Wyq()<*}|)+;Q)I@tJ7?+Qo@-;#3=S z5-XF+G_yU$R@9CxYQbMJ#znWyss70j{bcve~C;f~{sBtBC9$H!6S~J(09DU_Mf=FT1yV8zgQI z8?7FjA`!S|cs&B0jCQ+@Pt+8;wZrV647mfN=U5T#Va~ij%h-?bty#AJUB}I`tB>5j z`X%`VL@@o&bu40I^#9@_k`(?!vG86C#V}Q6%})jE8Tk4Bptk;mW)<=jx{KQ-XY194 z8Sz%1Sl_7srP<8F?MHbum+i?7wwSD~nVJ_kfDtn_g~9X)7&IC|9)-GNyWD+YvE#fI zl3AF%*Jqi)E&|Xa5k$$h$(@0518aQoqoEGVPtigagYA#huhk|#?1E*Cb((} zXjYP{y_~9p_2jK*%H979iYLT;Er;jV7*xN;VELaLL&-_s;Xh{neD1<@;y9u5@UH|~ut6G5)t2IpM~Tx zZuuUlOB6DNPA(pRd@vo+JT7L`8rPIub3j}Uu{Q^QHQL>T>?O+MiiX`SCCPF%dY|C( z-4dzE3$6{p_I;Rw^^e7)IQ*lWb+k5y93%9SJ%F?F;o!U^B#Vg8D}Vt5nG?Blup9a< zL_??sa>s~Aj@{*(-H;eQY5+NpRBoWwjaVBjifU`(fa5$TQ|GWMEJOZ~-;y;?_TT>; zve1ZRS%?4tC%+&y>i@Al{?iOq8W5h!Pgy@-$DZlcHYbYy$(S>Z?=rS%WmiKP7MX_#OSJWe}5Ft zL(-)*xo2!30y-O$M*eG90?>-=^3R;L>tK)DQzNdA2_n$zRSMR87kP5*^cM939 z@LD#pkRLVF5dgH>tq}(4m^^@J+vtRnTX`>vA0O@W9dq{e0V1!A3De=<`u-x_?6HPi zwCpOoGhOaa6Zex(G=;HT$MXRFBr-0aFjRxRmo(YB;*& ztp;OuvBbOVektT?6}PgK^;d1WCWovP+QW8F-QfwZPJcBI1}&1+nTI=zYP3bmz>X$e zn_W?iW%oBvoBMl@?%si^`>L0FP@B7Z(e1+9Al-Au$?+=47Iz9G+I5s}@x3W&cbK`q zUCUF2tRe*-1~TeWq))^NB{W4OitGw*P)Y=(mQQx%h>~Yv&raNxOdJCY+EkX&B<+@s zt*EDSmVkBQ#HcdMyM0dFskBfI$2n@9}xQs5Z;*kHS;M(1d@T$)0lQHCZtY<)zG7WEAMvN#XnXjbxxN z%piA%rZ$=?8Zl?VS1QAT20ZvnA>_46xyp;?Awrwk$!6~}6QQlCwIHCo#kIJRVcCNF z`&TM6XB}3tC5wf9#kEc?-~)zA>E|m+lrU(kzDU&x>IGVQxg{p>rJ08Hfl-%eB4pQV z-}k4RWn`wNvZ7?BV63H)`2U#YmrZH1S8*ZO3G~vzi3pV?(VGncvIknqP=h-pr00Dn zI~ieJR#_wcle3BLgU@IZ7GGX2rdmSsuSMbD(5!GZKeMxVYBr+G0||7kJ(0xR_eDQ+ zu@JDMr+RT#!ru#1`iqoXF`$qBNjyjo{+lB?$V@U3EMo^sgM!quXD!V%B7)D$C^+Bk zCb}-dVdtH$mZ*IA%pNcUun@m_a!`;ofz(9y41a#@cddhRF{F9T zuf765I338&uIHaCFgVf_xQz?sB^6s6Ji60r&2^~W48WU7M_W%C87sORQ{+KBh7Q$& z-8fJ`TrAyeorsYz58~?gmn_02wQMf%mz=;Xv#ZE(J35KObSI5F5jQxJWKNUG4KV;e z-o@m{JiLI|~!5wpqfjnh4pLa@GTe*@ZiN8LW z8wIK!+2DG=zC=vwIaT7108~J`5fCrnPXiLJlKJs7xEDY~wTQ3C91;YS1o0ZrJTohW zEhVBJmSm$Dg3hG{bt>t)TTuL<9U3c=2Y0f#wd7#?Ues)*Wl6Fx;5p?m;qdT>AzfR`{W1CeFkJ0^mp}uRlk48Dv{lE#EHI{;f}Hxj@`Ndqz237 zVDX@oVahQSW!f6d>uid3>g*Iu>P$(A@8DD=^JA(P;iMV1r;dQ@@Kf5g%$1fv@qlW#%1an@^BXvUD6_6tF!~KK?9Ux?B(-vr+V`0BV|yKBg)Ya zNt9&5`-RXHvJsNkDrfjSMYk%HH;1}OE{Zk%UX*%ME99P4<11m^ydx`2SzjfC&HWw9 z-G2JvE5ZXs2Tv%qxI2ZnQi+f`f|miSnSGg1c9eW+_>Am>Ofgr`(e6NGq4SOMj!4KSOO}zFDLtFpaJ* zm6no=`x{Lrsv!C%gLP%bzdyM|M=%#u87*CY>A$(Pb6D3QK2~Z<<6%2qYFqkKU0R2( z0ekhVzkj<8O0z`tBz+;mQ>AYEC@!lkVosCppi7IM)w5frKiIiD#?+AD)PBbv4=9~f zkQ_xbrwXaf3dy@L8wte6saD`=#Q=q0zP28G9Q+(SJ-p<`Svhrgj1CG~ta0FKURo(H zm2meA&fmX~w=Q>e!JTLHn|egvk0n_xLd(z2-(sP>V~&fC>pDI9t+B2!u-Zle&GyEz zP>LsD1FNSJ9sO!)G4tR7)6+iQR=^3ONRO`@RjT4^yK5P45P>JliN-mZvdc+dicR|3Lr z!|+cKkU?<14%t5di3*%GghXB!P1t#vC(#*GRThe16GxZlSDj1oKOtC#OVR{b1u$I% zsg<<`#J;hV7~5ELc@t|nFB_3f>*zCwOE5mywmfOax)ywe9lLa)uzN@?Ik5NfYs{Zm zDhsQZk7PfROSfnOgUVdqh@4|WVv*-~KPLa=2%8t=YXM@eDR{5E`WoFT>t**8_Jf93 zjBo{;4Dlq_$Z1qW8rjnuocL?%XaT1%N&tc{%~}+W=`eD(7W^8Q_$@Y$Jb~y9vt)AC z42MU+=Z>RuoG$y*!DKKc%ow((=_fZZ5(gpqO4K-+OO;l`kuQqxxr zwe!|`dA)j@-vcDxnNA-XHBOXxl1rm9@;FCWu6zfVq06N;39^=-8g&x-~?;w~0XV z0%fkYh-IX0Gd@A4O^JCUkSK%_&Sqy^elvbUJ+Ou;RDE=}N$Yz+vbo8cf2!%qy8bP^ z37jc*ln-3Eu07sK_@{?ZC+ADVE2puI_$&Myy=ziy3{K;VYAO@BPRfb4ZoDakE^h(; zEBF+73JmxUw+y==lK#beM%wSa4eEv80%UvNFHyWElg}if0^=1y95K0woli))hMYsK zr?h@amd!Ufzu?Q1Ph|*#47m$2x3SiEmj83L&n|{Y5E&_J+ti~MD+Xd7@@I{y-Ijy; zVswU`T|6ms^DTy%b#j&K<3=hg+4SGgxrEqL6^X{7c9bm(d^ri*)HNT&0n_8>NAs#3mcMAOrFADJ?ZC&J03k1K4?*lfllS7Cx4IIW`Z@sZIhZuvJrl}mlSqX*@>K0bb^@=@xo;}Y)!v#$`vQ!N_qM3z<|Ne&t=_6xehz6!L!f%9`RWZK11sm; z^_Ei)QrskOVyDz#Ga)_qA{a#;5bRt{D7i#Q&R!Y5Y2+Fs`ggpT&zJqm(cjn>F zgf?GsT=)b7Km8ug3~3Ab?vXkBh9-k$Zb~Gac->x6ZMgjGi&5-d^`DrZwR_no*8k<0 zEHH&y!_}h95lr6uEd+>nBVaU!W;vm=Veej%Kc1Q(T86ta3$lI1in*&onCeo_MXz7 z_{G&tj!PJ<70x`WJ?d0^R{2o_9&rhuD8I+dB9g(pu<3H_1FCn@L*>ngo)+B9ITByv zd!Y_+N9J;wlnZAilYc#fYUhlK75k6+ZF%eM8k=2yKo$$ ztx>%==es**;#U=SoC99gxSSM95YhIxOBV==beL|4r5!KxH%-WyD8>AcdW8;HN}3g> z*wu??gt6v|`xA}zR69OuHZQca?!P$y5{H6*nH?r-jdp4r@yJ*oRQBZvz2qCFkCO&j zGdT~pVTM6*?3;J1cB<0xVOa5jm2WH;BiKN0`tpL=Wuw!LG8;d3(vd7xyee-zbo%Ju z#^h1Xn8(3C(LyWcG=mN+Sg>(Ga=5EmYD2kbLOLw%ZGq$o*A|ZZ5#i1g+Qq?L`@f9saYPyw(!Z5KxecnaaXD=hOVH?6Xfbd7IU}ik?`V?(RHq3Q^wXV z=BnvKq3-(+s2w3(>1YNYqvw;(1b31AC)VnK5H#)w>3u2CHwReT<47yRWfQH@vUHcO zb=zj4P~%an@Ov_m6NBl{oA4b#v=XjfST=3iQD6dfJ?+~KE!38?I)s#CNa+E+jzcId zhink~SJG{-l+Nr6FnS?k(^37UkwN>e3VuD(lrde_u-XE4_cH!58^XFD(@Jbc)LIW= z4`<|B&w`Jy0@_#N_kw{jimF5LD^(Ye)7$B!Gf(K%c)0WsY3{7g9~!LD?fTd)qb3$| zJ+;L)N$ruc!oeWw4fjmQNr^Q8>mVng!Z$M80nxvWPHb@*w!vtZ#f;r39B0kcXO!0a zNvQT|Uz4M(HiykE3P0MV96#0@RZD*{Um{8kLj6O`tIF;O=HV7N#LA$)H@#9lq7}g# zu0qU5m_)nLy|=T1pxsy_TLX*lx0o|-{tWA(bN|E}5ZfY>=@RQZ^}n5l^`92rpGkTm zRe%0~{dYIl3GuAI*{{n{9`x7s_c$3QjTBjqi_&1$i;fZMXks%E@@yc*xVoVLBymb~)UW;LA@r%CKe#{!b6nOUN& zgXT=^JJ(0V-Vp?)Qujx9p0>l6FaKQeHkRE+op^;TS-X1YBeIi*%tC!r^*=Q^(leL` zt5F(R742{c=TWuHRkbPkhb?uN>zD7f&z-P?e4a9!_gtpynItc-+zl}3zTdPbd^VYp zbF-25*0M!uJ{1vB=rxe3lc8{ekbex^gW0vX3`U>K2m5m37SJ4ettkyB<90te?>Pil zCv0c67Un$umdO!YV|3KHz6b&J&^|QQx!I)ONoPR)GL9Za}a-0Oykay?7 z^Vo(M<0kyEGx62nMUsA<3OA7~xEksdo5VJY5eU6Bno)D(Zng=P8#dLhqwNt@Yp41M zUdFPICBojK&kr1P3#-{!JTl)Yb4*6YU*VJr3Pg_K@BWgm;`zh<`GU-KVAX9R#az_U z@LwK52|8y|fw$u#QB2VP$j_Bl9{TY9`-EklGK^XOg$#54LWcSN&lC2)z}CdV|G7@| zU%L{&OGEljw*Q49tWx`Dr>%nXJFuZRhGR{0cm~p;N))^gdbo8|{`ZmOeTc=LZNN}!Wy%w97Fof$U zcTB$rJKr0YfzN4hf`gcn9#m)?mi)EUztBH1F~eIOUCk~9Z%O<^At(|rX_UOyYgFEvTERXtF13!0$!h?E3kTmsRsW zb9B2>PZ$zg8tqD7#r%eczAo}mSgNfl!C%L2%ec)8?aBE~Y)gicQ{xOY3q(MJ*gcPp za=DYZOpH|-z?x`NxoR<%YlSpfE|w6SR2w#_!CgXs4L7(L%<%RQ{#BwEsE9x-;Y}7l zUz~NVdt>yl8i+I-E+3^%)|4j6JquvlMB>=P6UP$tBR*_~%EP70@jut{qW$!$hH5MMy1ir0eD zbj*yW*JU9ocx|#$CxqK<5k5bGbY*pXNH)Vn*r=j5f4HLEKn9oOI$ ztvjp={4&K(L^wbp7VoT}IBvFm_+!=|Re7z5rKhHg%@-Vt1ZCr!qKw?SJvUM)=3ph( z^{FzV&)bw$69s2Uw#ruht0~dBtyhBo`t4p5L$83ZcctyA`S2R4o?8AKnv$hWU(eH zLem~|lPi10%vQCN99pT98e+SJ?$eIs0K^JrPMEv&MS)vIjE$TWU?iZI9cHH|gL6el z7AaMUZ!vTf+5Tq?xHCu(wOOKooKQYjm^3H#M0(UliB^ilz}U$VC3P?QcjCu01EieJ zl&0n+sS(A@{R4fv3!`QJIlD1Q`~rhD{a!i7c)i`+6q0TEZo`lyAu_2W!8u}V4W4m0 zZj;3Lrdk|k0#jmoc4=M2Hf}R8FNlM16 zk;BeV8x4ATF!*#o3i5&-*w(`BXdhLI@x4S`z|BYW2oAQ^?!hbh_#HrxB<>*O^RxG5GVzz`Ee%tuGxF7$7D zB(5mU8ikQO9Nh?+TRxe>h|5Z3f_h9zc)klte*M}>etUbH7?%RpZ*f@ToD*5xs$s9Q zYouGI$W(jMVkMFQ(M+L`;@3-#un5IHW=c^azDd3*0hGvd)4giQteoE#Xp8UEeK559 z>qf{HLGf8C;5fp>n7sggoX&m5R-MeZL8UTPm_d}g|E&nK)QiFF5_nv@9@Sumc?CX| z|2FjN)g7R^>MqkLwug*^KdI-abgVfSrTdxl{4*&o|(6+eLgR0j&2(~EIM(Z zpUHO~$B-X}gxOEmTiB2?`i)rz_m-XX7O2oOfUKE)s5$+Ri@c{~ z@d?P;Z+T_$9x>|~x%e%#>>uF<C0lGR(cnROWa*H3Etv$?R zf&Xeraehq!pt(V4--AQQk~TM5Zu`>$)LJg3cBy9xiF4=lt>>O)gd|8OFYXEY7B2%e ze}2ER=7hWdM>qO@7m0m;Qp~=F36Dr=8oo>$;b0N;Ta?ds_WtIQ<|ntsXMUsS^gzbr zlZ|zkZ$Zb$D>QQ4I;ivnHyZiP+my&#>_QYB%LweXO7fudjDo>}^+^{}V3}|@ zy-b7Eg3|(Tu&a)w5?cdIJ5MryK2vT!TO1!(E_!~5d||br4bf!7XNV{@?I}a$%5M)6 zbGbDG6u3!EJm3u}`^8uOV#kyfq6+jjk#FM-}4KM!Ai?BAh5(_<2jZ$nWTu< zxCla%%&D3ue{@-%%-ecWl&MqgpVft8-%4(g`zD9=zW zBb$agZKosnrMHS_!7DbrKfr{vrb8s7{>rj%Q&$}yCT%*`xMR50X;}iK{OnO-ezHWT zP7Bt}&~gLvNGgmUAPa?UIRM8F(u9>NiczjyI^vcs4 z$%<{;wrzXG`9~|ZZD+-{ZQIFBopW~8{dU#4wX5#Ke3*~3<``qn?*4T**PnDrT=29u z5xpUm8HLqWp*rdciBefpL*%T$d~M41;A-0wu6pnyHph{;Z_tYJlKHUx#>SQw7G=*3 zS5ybdd|`>k%G6d>w{V2Q2EDcNV6|B}+;G_hq)fHrjV_R$FgF*A7bI@Bivd4oy1(Va z&5xSMKJb5s8w7bvc41%g!x#+XgWi|r&r-+87~p)lB5EvNsJoj@99&D$M8fY$qBC_X zxMY_*vJPP;QOzQM1rw3)7dw+j)L45EY|q@tcIFZX4hZ^4540dJ-{AYG3m&BM&)@K6 zE#LTFAnrlp_(go;`2FhoQ)@eJCudFMKiMflq0dfG{bym44;X7yV(o92KfhQ zrzGHPZB9_7uG&L?qGYd?R=qlW^`e^-8j3{RCk0&74*gXOg0|L`|q_ z>!c)VN)brnVRN!Gdli=S9QO?}UDMtM?tT&HS0Ejck+F6Ms>ktG71C{2j_;yZ+v(LL z%~?)t6#BRodM;(R*e{uuSxFTQzUMm^>Av=!>b30e)ARq;dTlHdLUl7n^oWlRN4N^5 z)?z!B&DVB@bB?FW+(vX>l!gv9)_p)J&$sUtaDon8R`6An+8(y2Rhg{v5 z2zMopDV|r8xgn$ zP`)l5zgOIH$6&QIm$!G77`isRL2s~YAi@}yK!0)E^kp$IQ3H5gJ#6!t|5p+^nWwe~ zbzY=9af>_pf_kx3b^A9Izi@>`U31)(dy!Eh1hR zj2jTwRX=B*AXsx?dmO(036$E7Z^X8ce2I#XeD&;9rn61qaXGQDfq)u!D;)mcgjf~CSFUZkMk5zH{`9P-?7I$83@b@>hp_uW0- z77((v{s96Y_SplICZ5F&`}?tt93mZ}u*@T!GI|WmUX%c2<+S=(7`?7bRvG>-6J+f~+Sv}Hk-i6=M_x}5iACU5tz*)+yU;HSmE>VESq{SB`{y3o7 z3-KsVjND0_2kEuUEn1OvNWSi8-i$HlPgGD&>L`V>N2ebJ<|TwSq{xWolrQYJZiFB9 zos?_C(pnpri@Kn65?qu;w$jTQ`U)s#{l_w_f12k)~8i zY+mqOb`~G)xp|1zAi_4?g`e%sC*prX|ECR8a|k3LAUpyfpx^)B-|+u6`}tqM|9{;m z8M+JG+d3Hjzp#I{I;>CfV#@d5>r8+>^J_pSxgk*?bO<8=E+iZRE&@eJiV-Xn=t8E4;q11EQ2$R~&6vD7Qj zVX(tR(M|T=qWg11BcG`7cSmDpdCY=Cb|3dkzjOyq_CJAwe(wz;>Qeseftl^Weo25e zeQ_83S_%ETQv&iCPWye@KYddlo%a{^LBGC}BtrcsqAnEMLr%_*YTWt#UjzOY z3)?ev4faNe=@-8Y{uUf4zwH52>hHGafB40rn;!n1G|&=q!XWpt7}|0h2kSr3 z@%qBE<-F(1blRBI3nx-ZY6)=)v6>!yQO~Kxa*oTqE{-%2q8&dn9UBU_LtzyZ99P)TuNI!-ofypN=qRWUYo%Or+Wz!gS z-5QXLJflpcS}aIaMvQFAw7clYI=@y-QODRawpT4gX*5Rf+%FzPok6?(GjepZYOIih z>ff|HGE@$w;u=)h=Mf;d4IzlD3%y#p?8EPCcc`p3y&TgerpZaMgry2xJgx}#^$s)q z#|hWf=h1&wsf~Y*L{CnC3Qf*prky;Gny%; zmk5`DCwfq79EdlHvMYLx)LyBn*j(Z0K4vB3?y9+`{}0A6NV%5tXEuuy%Gr0{DbBl9 zbYX5jmCO&vX+O+@89#m)g{s2Ylojp#U}5^PVyjL#*F09(Ww(h86gzd)k|w z+8-bH@@sPN+mRV_~-$Wr+q=qK0fm2T3kr~>nT0)2}IF*7ne7|aC6L=O*Pd00xvuC#iRPZISBmpJ^ZEhIuvIq8@b4B>} zl1Wk(1yZX?vD&Ai+?F&uDiyTqd!w8L8qd`3-0jRb)51tr)ef}?r748eoHdfECuC>Z zjQBa$5UrTWtBa^0WZBBHsNpiq=3MSFRw85z0K-*3W2Y+<7BbqWZ5Cq@=;w2@ zDy_>nspMY(dn4FpoW|b5P?x~S3WughTPLSH$;ZZ#LAut(!XcN0lNEzK92+tj0_1j| zlQvC&#Nf&wGNHC)dUi`}U89*I^0~&111)_+g(RO!G{H#-UkQFSz(#-T670XXjA%Du z@B6Hw{C!iKr&kvY`X#(ZjxsRMDp~+%_hKO%RK|js^-%FRwEBS)t0RP@r&os&w7obd zj?BfASrlnjj40^V!IT5_0;$mICzGltm!d+=@8IQJ8W}C|b9gHbOHf-@izE#nGW9Td z<{)8dHgbW7PMacIVb6cfqmOj69^6CpwLx7Wcf>Id>xYpd4(*eeP_kGOf9+Z@qaV~lDt}OkCaC<2)mYp?pOQnB-=liEe|14fqAip;BZ*5qu7vL^ z%#oQ3;hOP?ap)76L*-l+iy37vuOue7g^U-4|9+|L%YL3IR5Rh z^)lrHZ#~61)+}h2#d=gfak7MQ(KHBe?nJL>zpoG|&(mV`0e1~Y&I!D+-x+7+w%@@? ze_X_}t;3>c<-8?Y&%BJtrW|MJCuje;Lld@bfN9hS}nY=S!sk$Up3b!R$zbEbl@bO6w_RT<-lk6Ha9h-pJ*(S-YO<&LQ*wzr&cZ$*y07j`_inDoOR#HNfN4f$xI{5$IKwhOtHm5r2Z>w zD-WmBg5lVtIq8QRu_C7voK@eixCt3|yZA`XUWYI?n~OqtU}lNQ!$~`gK)*V3HW^Nx zBaJ%T!E@XAJSki@I9v!feOs=?_pR^(nJelE=rmPo3*2F!g?pl*vf@?qK@a2j=UNyE zki+Ni{DOJChKhPavKTjIFf8W!bU?gKwc-q`pLB*`dN?$CcS zw|O^Qc{7-g@7*UHOeRhDN>8a0CiSl7wBBHrF*-b7;k#E5zTYhgEiHhZmB2J4TPy%z z^yLplD1pY;NIQ_y4biz!5#%0v{J}|4{;=_2@i`Gy$%+lHze1)AYi(zVI!{ozl_E)T zb<~GXWsp7twFgSdY^{D9##q*w}Tf6X(ufGx5r;0~IeG zx7~|@VM+#A&}F%BMV-RP6u6!xR>$NTR?*ab)E-e=pV7&(+rpPNd%3-?v@oj{<9!$mH;PjuBZMd^i>fw$0`C>-uf1GEVCDoj+(z zH~PJBfb-6q-pky}5QzWM{63PAtZ?zxoPu*Ft6{O`V_}{% z8xw<9U0^)NIS2Xb+=)WMa8zsz5Y_FKRHY^P_!G;fsIfZO9p3 z&!L{K^wa4*+i{J2anUvxpx;1)f(!my3!qQwLt{8#2Y6l42WuSsM1H1JFS&xT8{VcpJiJ zfCbk<3ub*=bNkXN0f^Z*-sOA^khNzDgi&O9h(z+J4S0C=EyfHO z^}N&k$x*OFn1s?}!_5!ZVFVwsGu;_l?T!!qd2C zsUM&=ur-;6B~>FOwv>ex3PS3}FuI@&y3s`oec3D_stb`#9~HMSXZ>KJ)tAK-jku?hRey7qP(sSg8 zAGO|!w|-MRnt93T3vbUY&)>g^y;XOo=*^Hm1HCf=j8}j@#$=fk;}l|7`u6M|IH=>=96u+yF`V&m6nl zK)?^-dvMbq$NQe&Cob>wVnFUH&YSbg{;s1wK2PfD*vcoTj%B4n*5gke;9@@XgMdGV z`ssp~w||EA608UE{T96W4y}g)5-TG%T`!KIUX*;!iwc366$J}J7i`3iCe@AbguR$)^E;skn6#LvY=Z_Tj(Bbov^ob>o)kPax zz71)Ow?tqu*+*+gp1+9kN0B!z>0@=M_BvDSy!*;C$8%P1qG0eaf)W_!af2L$ZE?FLGipG1IY8 zDrbz@70#z4(!z4Ni9;n&y>HTn6@A+%Rm7K7Dn2tk1DBIeIEf(g4Cnnykg`)qBsk2|!O?dufTC!52;Hd#mMap6XKA4b*D?VEVfxbtG zIS8t|^$q%;9D^Ku+Jy0sMga7~F_8ZcIEMeD5@OJRleWGdB4|!|AR5Y}n*p)4l;jiwiCr^FD;#n{YvJuTNTsm$uvZL!ByhjQe_5?YEO4>i&2irau>acrjOBmbr^o@T6}5pi0{L4y z$eDw4GOzjq$6rCfP&yo~`eN-P$H!k&?5`a9_@d4BCWPc4&QLy^=2QE< ziS#Rp;n@6A@Pn^AUxMLK68%+>D8L^oAN_A>uK+decOm(AB>|o<`Zr4N@`4~kX*mCh zKIn@V&~|ArN3g|k4^{QH67F{w%y(eg;sZF_+ceP^?041gPkHn&%+;d( zg_i4zA_1tKaKViL!o%GVke41Jh}-Y8PIZtMM|3-t>k8EsG6ga)Jvb=MP`>^G0=>jr zA;0^!k6tJ!{ogO3Bt$cf!`zlfMysT4Diho$Ih)xrB@>MR*<$dRS&@FJ>o@=)CI$IM zoCUQ_L*_RgCX{KV%R;tGr2(M~yU&**opqE@6B)my2xykmvZP@WEwsHfeKezy zsUW7AxudpjoE5WOoJ*l@w6b6e+!<-3qDd(dpXCk2rl~+PVZb}0M*+j((u_+Y=&R{S z_Fignr**H39gm=)TuxF94_Oya^H;0)oJv27s0W%*8IzKs2|0XLL)~pfmoGj>u%^hz zz+(QPEXJm?N2nLGYP)FAmZ?XxIE{<(6DbNFj0=<61|Jd*BwWncD<10g9ZSc_pEc?% zcr%)c5zXgk2V)XjfP!T=X6v$k5fzvmWU9SKA3CXfkULWVC{ulLrbLbBQ=P7&Y?%G+ zguPdlsY_h+>f(N>DGS2{1L$Z>_Gq#MRO$2webN5jm&Z0vSK zvQ^}Co473*f!u4`j{)i?V8p4HGuc=g+|H0hj#QQH1Tu@M!PR&=@<+S*9=1T+Y^aYt z1ew!Zg}Yu@*fQF9h^S8{EXT8^%)1_^lWnNjR?;^E8FyuW;tiEor_IUgPr16q zFWij<8Dlfc;4>;`hE!4L-^9-1oGKvG-X&inI=eldt+(7A4rAU}h_UNYF*nJ3@eI?1 z?B5+tMNDIg#>!}l%aKF#aw88I8?(4qnw;ussYh{yZXB4?a_XPNR2NHWD%HB8Y65v2 zN&tr$cs?@k4*SVYL?XWMzQ8fHt+vQ?f&fDm%=NGFM?%qV+g z+8>y3W5>&gp_+NoJ0<@0UKGR+J5j;8M%h$>xkA-wF?wK9X*#t4ogElcC^r#)NJQU? z(l?y+O~Eo-!Sw7Vej7XrSW!CoWha7VRN)kGE+S#DQ^F>Rt5*~n^U*jdO0t@l8a_D4 z;g#F#!8Y`m*=Hfwu%;%Gy`m=BTp%;SVs@6Mj6F2*gu29~DvtW5s?)L2zSW+nb$JwW z2`gPskSg0VP;N-GESt@niI8ZH8ZxI zDKjh!b0u&J_omm>M5Dd}Du+ksS`-SB!tju7T1c6T2au(hE@xbPtn>4XHRZ{pX@o&I z(~lQNT|L}uPvsuWZJZlk5W?Xvb^lYXRUTI_+U)~+oEfD_j=exfGZB&pvMn}9!qmCI zhbG0vT`AqIRSv9Po&aj1yo!*QmE^I`qk{iDoDQ54uX3{Iqw!efDvJDjE-t@!*;=K* zr(uXhuP?B>#1Nk$6BVgb#cM~kDLAd!4DTQETwFOUo*h|sXz=~Mmmg0Cr&-zVeQe>z zME1mk#YHFF>}ea=swa5PI{kn*bocN>Dyq*?5J-}I<=e8!^ z0pxD1rfrJPB||C|aZWWw74)>j+7sUN+iLOhsvOI*8GT=?HiUG=viqkfE)DG5sq&br zHs>@TO9Z|ud6}@KE!PTP!76!yo9}Y1bt%vYRKoFrybxdenq0K6>Z}l?0z68GqEuk$ zRqjOCcC>s$xZ;msYD|cgIbPWMLzZ{STazz+>&M62H~{1smb-I>JJIgkT$hjio;`v)&qX-?6Pxq}p|~y8u{)+iwIkPZ z^6rG#>3|0hx_JYY;(NgDL7s?WHs8nX-*E_7o=N{4rzW<%e`(7xpu^48n|l z^011vO-$4u$YCiu-^dsYn2m}xQH@I?l>+_ZHB)3Yc9potXHIIy1twiLFG{{c?6LP_ za4*IUWw=Rg$?>DH-4_!1Sx2jPRAZk?VAl?AP~hc0v+MS2X0=Ce+!>@N{y2vo4wR>i z&o6okRMxQ1v5pFrat+zHk45{iPfY2%-jawb7S@|%u5S?MUQBEUeWUgd=KBYA)-~Ik zQy;?ucxFJ^i8-iX+MDW;?Pzw_3tOB@ZS$@z-s>h~DF3$q)*uzc09X*3zQT&hjm896(^aMTpB0ys8q+`k!u<>;vc8%3Oge)6S*c9e;S# zTeGJMt0!iy8*qa0t7yumt`)cc-b&@29M@< z9N%B?rA4v)_?w$&pWG<|>~_x_Z`Y87m`92nJ3T>9>t{R}A8E6%3!a4MQw=$=*}6LJ zfs}8W264*2a3B4mV0Q(7dl*6y_V>rLBNIiz*r27H92M27W1Sqc-3F8nAtU!Q%?9k} z=U`^4rXRM1K#`JVG#-SP&VYVQG8=r-vO#P~2fx*U^SCxWCNNi_wiHU^4}-QlG&yd+ zbKaeSWpm78cWy(EATqi}CVog_o_gi_n5CxBwA$os2ix^|L|8AB`ECg=YYyLwb`Di< zqw-DXb2m1dHk$))y2DTDe_D-$FH^X75L*+O(&3mwRn;RPYsSm11`w>CEZwLPd=nE~ z=YcF)`X{fY**GLtjS6!QA9l^?6;ZeJ4!qQ&8eY$R;?%a$By0TgccIU9H&;I=dD9pR zUuxu!S=4QRbdVZ1+jRG-DCA3&E&VfDno4K!r^+J1dZy3)ug>`%e8YC;(!X8HY8Z}v z(ic9Gax)jcd{|$8v7^^_-;coHdvu(&>1RRhvQAfKa69ysRU9eah(%X)4){fVj-9R` z*%mg>WiWcms6UkCc=j6%ubIA`W8>y>R3<@b?3?3yMn&&#v>jNvK^!e0kOzMREuR zTwuh^@OP>ab{zY)+o|AcYJ@$KuLujFmC*RQ1*ta=5trePl_1EJvUZB!WCsv)R1~owVg%`Xf-(lI!9!~VG^7-{?T{8= zga%G+hQIiOqh77M41!p7;&vi$gTP}!- zJyQJ74Co`ujR*$_6JyyDqqvUo8=>BZP+Qc`yL@+MyhOt{e>%bACA35chK%pvTYuRP zquwF#(p(ST-^IRU@dl?# zi;kv|pj4Zl|b_twWwkX9Cy0D_DJ7Dn%QYm*vQ7cD$KpOk$WX`y7 z@fp64?7DS{REn){R5;b!J@UD=hY!!I#$a`Z~vYmo7*i9m~sJ zM^yVg{RpK}!X%%3H5Q2XS!nmYnEVZ(nH5Z=-P_ejjvwfg{gW{rwp7fufaL}{Te=z-f*P~cTPGqHu z_PpNWxz6#P>SDNl-=5Y7DoltK66O90F+x+^KwUB-yxI>83n#ggpp73YqrhRtNyYN| z^lCCJhQ zo88X6KksZj)Fyb=jThW@!W$n~^Qbag#7bSquo#W|8!U#lK(=1zTi%*+0%PeGo=0(h zm3S1GbqS>kK0baQbkc7yMRzxb@NBaS*+)Q}Mw~I-9yd)DI>>h(Cj23Q0v(OE`APj; z**SjGNn<XvKnu~+G7kTOvxSfjFO8EvJ9^f+l*T#SIvH%Ed1Rye!;`slvT9e ziQMQ*8j~#nodjvq?Obnmt#POe$4W8-_0Sd0SR_1<*s$ts#!+%0BhI>YveuJH$3 z8uE1j`A~TB9-;>xPDd&5YfNI_B!Hw6*JPH5MhtD|5VTX{40pjDE4fxbZ_W+ulJSAC z?XL=mK;AAU@wZ($5>FM<0G-Sz4C{Apf6zmqgOX90cH5Vw5}q9qw)LJA#OpZlfP~Tw zw@@MaPGO|fL9!GWOHdTBl|OU9S}u!FNQMl4aFhL~$^IZ!oD`?4Yt+x+j-qe=tTLv=Cb zYq>2xX#Da87)|!B?{9QXJt~2QZ?htgK}!c=+wZ%8y5s#Rd!;@KBK^k(1TsAoO}h#Q#;Nxp z!B=9`T|89V;3=KWUocU>%kO&*jgjR}Zmxk51Qxo8$$X_AUlj?ygp0b1dA=?8Tu}uO z3{v=AQ>fdd09F0+FgMDbf~SQ$L_KpYlGh8U-FEuvYY^zVQwYspt7Aa19j&yIWXnhkY1~} zp$ie?U+FSyl7hKaOa#*40~@Fw1r^@h;PYC0k=1Vo?zP4hy4(0y<%kKAzS{EcgL-4Q zvrTTSIV+BuQ5=O_T?r@k>nP21=?hM>H){sI^3-@`%h~X5QSNsfAst42D)K>SfB934 zH01(~a}+!?C{*J*4NhUk#-i;c$Sw?$<7uW)Top->H)e$}erf)(_Sf`c!K+aRo~*md zQOQUcSj4oAISu|StJ*a=1nJ9am<3>dj`!C&ScT@6pH&GomHFzCn`jwK+oberZ6%%vsqy?H~hxb{XTUQ~(&bvVsx6Sn?>R z+CqiZ?d%%3k_+8uLh-urDB-6PcE4>eRvg}K<%2^e;HVkrPpR#N8U`yW2?Zxizd#Tt zaf>+c$r0#lZC!HHlcv!61;a+RH|t^`GgWVpI#eX=FiHsf6*PvYlp0%~ybt)zh-MY7 zzh(Ta(2&{yaPzvK?9nUQ^#lv8Tp^AF~pDo50oVsQ! z&c=EW0&Mn4J(AeW;5&w2@HPy2oP~oVV_OQ?yDK3eEgB;H9@e{G}`Oa>a0$ z6x}&77ZZ7)f4O!54RL(N(kPsb+_^t$-jd>CrgXk+b!mdTZjgpwYLvg2dvCc9?-8M; z@kB6(>=ngekIVQUQnEhsyp4K7l)6ipaUM)YYk%YoOz60eG|V8gX%^YODabC@-tZ$| zoeHNFVuMm%XsS#yi88q+GXNve2p}YXI3_x{@MaJ*r-Xhx6_xzPS;lQ~pu(t?FSi?W zk^a|l_{79?7r|$!A5&#Sg{sbhuDMLT+ftIMo66#CAFL%*o6JBGAaq&*Qxq{#Tc{(* zTty}E9-?v_C&6nO&%P!DFpvzLFrwz8D`rW>MpaFnNOhJeE40~@J{oCDok*cpKGIaG z;$d4!1(sGz(AY$c3u*idV9M7RiF}})#G$4wOW~G2lI|jdbxEg!rQ?#5tG zcIq}b9j>j&b+ExOXyRdmhnnJorMjreKcnDDm5P(|igipaL1-yVQ9IU{z1Y zSlBdV`w&2^As4G(zAZE=ji1;#euK&pBd#a>F{(UUs2-UmW#En3paQ&cG#}o&hM`Jw zFCG_iD`P+toxEx2j?6W*ch1bt9{-}sH`Hd#gOaB1KT=~m4iKiF&`hX=2sAZVU>R;C zQFi+K2ilMik7j}co4hR(f4f;kV?DV{m8*0vS76hY&ApJKE#=a3#MbSj@5+wnA5`gh zV6A6~P1~x7!*njjaz)I#yCly%$Rnz*((Z1l+%+1(hpv zH$0CIp%-8q$I>vrt2v^%{$V`N^3j#zZ&*Y_yFef5{rVA2Bo&3FE)WRGKXtjt=KJRi z7;AA3QogAKQBU{P<>tIZ^ZwV}+A@E|Jj=*-OoMmzt(BElod z%uW?P^&`QmJowSA_{Sld6IZo^8R8xuE@RIAbm`2w&L>tU}U)8G3x0$&gm2`9PR>Am|ob3R^FTeJ;GjysG3 zq@l&dPplewUz8kU#;&d#F9JuK;xTZ}V0>c&zjLUDxilT*{3?Ttyt;5B89Uo*JAafT zv+G6HMUm-I+$+I=$|6G4Ih3rjp*p)s{%K}DX@~FHa!RCgJzi<1Wl3 z-8*rXZhFdXj~a$!Q6(B~w2dgNR#OJAhvC1`b0 zKXMYo-h|N(^3-EpkE2L+{c%zs=d?8F>C`D6g#i%LtU^~Oa<@g4NWV0NePF8V<`2ikZs-{pYK=^| z92!`O&b;;Pe)T4Cs zrbJXQWq8&rF7O$4+}40IeZ4+2N22MtG^wSRl8HZIVGCu3+^F=#bCu7%M^wES_Kd+?P1F19j~1=vqB$QgZ)3El)!E!0 zHjkpTe6^37#xqjxpu8X^#%n?pZiI1u;B&-)%R()JOU@|;FKwe%J_YF^* zkHE1T9Y>?*rt64R)QYdax8*z-|F(uD=VAHA;B0z=NVaTPGpp;Tgjd5A=NPQ9phpMV znpVr?0bl!le z*$BJ66D2v6rc9_YYTg(sbl|`kTdtFFGb-`{m92+8Kh(4Nv)LMze*MRunA-c~K-{r_ zH!Sl2$}Yk8BX1>q{gCJ?_so5+y5j)J>yYtjgy?C^{U)7P^jIL)_^xvQ;gOcoeNNMz zo7jgYuo(Di$k;C69c%0qXc>#=id$FRvgABUK=L)eLSqe`o1nTUHBv5>?5}W>mrR$# z&GqEgMl9g!SP{+Q^p3k|{M^I&kiVNQ2{TFslPfOrj+gonyfX3H{4ip1fNQT>#_86E zyz$=SxYxmw21gP$G!HT&M6hzKFbY^_A^$I^Mpx=@OIkQSN(fAAt?UpbET{PqiF`*X zIYSGM#E_LCxvLp3mPDn->ejZxJ}UE~FXaa4Yj$WWX#fWQs1nCHZPdc$X_TL2a0|J6 zimIAc{Ti-MMkN>kv{8URlv&36S+wSv@^zY&jQI4CcDiv|?`W z6j|2el``q398&5(p-Sq&2LH2pmpf%=#T%eS(Uv@RUg!~BcAFNH+cHaZ%^0*&HrlcB z>zbJIx}ot4;%^!v78xP4kl%FU3-S|=AMY5!x#K`E1sp>h7QavY0|U;!V}E+Ub4(C+6UGVLD$J# z&XFr4m^@Z#dCc)%;THG!r+@ijg<_t5ySL5iU8A|y-qokzLu>S8Qi=sF97o->_Q1I9 z1DpH(L5I5F;I%0y?EA~jcKkD>1^1+I5ej?%1@xc5b>*baNgWyp$onTjnd*NAT>lTf zrT(t0a*XZAzhb(2m>U5_k3$yKZu9$>hA|LBZN8D30x(@&DqiZolMBPcg{Hl1M&TlB zo#k9K(=028H?gg7d1-0gtccS3n%p|Tt$2R?;MncolRTwww{t4Ep&(%Oesk;mzI*$< zYvXlWoC9cK=h^PX@wXp>mqKU+tRf;_#7K7f#~|OyusUxRwx>jI^4~&U&VznK#5>1d zh~kd4i25mD;)rRn>txs(PY0YD?;D7zR)?if4sfE}#zCVD z5j>%_P!9JSd2;KNI6(ty-7r!oFIrlmxMf?HLq?d>GXT{{|kDk**}3#2Y8A zBD~^YXaOU0HQ^QBmQs%*ILwEvCrq0<@@(Bvn;>z&JJnIs@fVD>teMveu<@*6j0z;(?3#)ARb6D01N6x@}(he0J$tHAR<_!0|a)1^KlheliVm5n;j-0XymJXQN3)7nos!Jp?m0yr2fg#P5L{MFrzu{U0N=)^I-38B_Z4y}~R?Fr28L z?P!3i;nN-7bv{15&ABC?UxkmE6Izl*IALS7+O6(lprLS&4sIfS=VKS zl?#~jQ{@2I1Z-$?D3N!n6>3bOkypO=elZ1NPRZNjOuB2w>>(s&gU^k_Nn#S~mE*}& z8OcpFMUyZD;@LsNwQ7QVIABLH;yx7_C)Ud{-~zqcFkAn4=sPQNv9GCC;?yvws9uOc zVEjS|XXYS_`3k>)F}KufQ3YIg&0+lF5W$Yjpb`RlSjevh0oV+%$Yx0m z7XGV#n_KljX-Rr~jx(Wx;DmdaFjM>e>Ad4TV%Ud7RmeAT(G@|&Iw}s(@G$hoE1@(n z)q5GRgjB8c9R5^_nd>%Kjs>)6krY_G>p)={_iaubjD;o%j^%dc+A_wHX|rMciE<87m@NF;L~R`H?6D=NIbjji`n zk94Un1DmIo5RSydu;QQ-WRaW~$H=YYMTa@K+)$!TSp=w0F{s1au#+#F+zKqVvKzbZB)RnWXSqOx)YRX~A4zkjlVsvjn`fLihKIp5hM<(u> zn(`^AB^1}x;0a=`mf<|FA~$R@WsA*~Ni}=)fl8OGYd_-;UA+!9Vu;mdzwh|XxY2jv z8m|*9z5@pnXAV7e5B3BX^A&EwrCZhV0V}T)ro_e!B%fViHF(}S3TisjZ-?I$*dx1? zZo7Iwk@y?;$N)2HHtzL==q;Oa*sf9pv#ndi=h+apovveiM18_vx7mQ^GCSU?ai)OQ z(ta$BLTPJNMGzL!?T4A)S}Z0x=#KlK7qeV?mArY73mDEuSv8xz*x8cHupD@R6LfN@ zti{{*lSU9RcEd;*eRiLoj*1!1&a5%PCuRelVdgd@8FkGY{e|yheb%ZT3FuWIHqoZqgG@r{F z3j9kp!Xd=Ic>}U7Hwg!%$#f)_L^W6TUG;|cF4(y4Tb=ju)I759od_n0;OVI@J>syi zJ}Z`)Vwh|8+2cl62S6DMccsH>7nps3_)-zn&r-R(!WFxR^;>PA#%-s6Qp`8*l{dg$ z+o-N|2-ITHj^UQtuE}l+p?H7YwA-&;RU*}1n-E-^Y{`Bj`3#h5@Jh5wS7l{I_w3le zPz*1=kX{*8Q=*aOFX%-}g-np*{^h=7yg;3QF@ow2urJjsZm&`n(O(e?VADk;;z}pJu$|?Gu8iQ z-RhTYek!>|*c$A(*xNSl@kSW1ITP4_Lx8Y&_w-^G(Wjr!jDE?Qyy@ZUiGIG-XFn~l zndf^^IN0x`9Q;GsXX@ptNIOVY7j!Rmwd~|icl@}aKm1vhB@kGV54t?4+)CeXQ()Tu z-0;R=jJ`dNb6EKQkoL|&mPO0nV0GEHZQHhO+qP}nwr!uXUDah*mt9@#{_4)1r~Bs3 zi;0OiJ7WL4_Q_l;SLP3+HMJuiOT*H0?M;Qb$6y-~3@*1WOYSn<*3A}OtrmT=GITHb z5M{;i3PNdRV!x^`RRnQ;YBemc-Z#2~9np`8MPskH8yBdx15F^c&g!c9_1D%Cl>vXM zQjMX>=wK=gTBlIWhIC&W4wGUgx}&FJ5HXf?@oR+ZOq#X_wF=Z$T{j_6o!gVq@K^HY zo4~dHRrC)H)2hcE1PG|}TlA0kf2hvHt!&L?zbA12EX%^w>{M{nQ2p(ZHyUmCfc+XW z)h$yLyRBu^f;F?Fl{-??*|qn~=_kpyyRXdIanSw-4+#DZET~~IbQ?P#j~9^kyCFA( zrka(#;dA)SHP7|?`1Cg5&nwKJ&gQUExVKRpIGMH5BQDERuLE?<`;pZRP&cjF4YR3{Z$COi%sN0z zqQ}T3kI_5+pmW}Ca@`~Q6jx2*G?ww;7liWlHFggiHG2x56yv8b<6g@x+nIV6$=F(q z6BN6Jgn;((xZyE%H&uTw`2G%ok75HHVGv8?z(>!x8Pb7SBROUyYT({}>e0iyh{8l5 z=KM&;dY#sYx0<@>HO-UPYkY{CJId7(n{(10qOh;kd1MIXfk%6kAGjQIw%_lRJBAgM z5(UWU{gcLdh`McSGf+?LjZSKE`Nlc3p4MQTW>CVP;?Wok0gqpNv9F>^=X<)3#E-%0 zv5=X9Q3t%%ZA({mdt^E@!H7aS z^$T87`DJHye8voXK!qIar)_m1J)_#)OM8Xxw#9{eETp=~=rptbUQV%3YTAgA#3{q^ zJ^}HhKShb)zNe9Z#&T*@$UMUYwoc=u@dVXS&nArOy;;quw*Ec#K5s!850kO8qd6?D0$R}nLyF*K#oYG z#<3nv4U{}A=`G3YFmDtr{(A7Kb5OS;*g;Sd*ue+-tVz34IN@%H1MGR(fzTN-U1DV; zj2+GBRB5Ofsnh;mnMaA@@~-)e_QZSg0%O5_3Rx9QR8R2Bxf43mfU!3NrYDLki7m`I zE*(zMOCx(7&IeQpmDiKKv%eXcCct-l2dL(fjj=daHALAK#VWK8(R#8;{#f)lMv!wJ zlNLWy@r*H$5vK8mn)Z@D;6xOX!Q;rluneLo^OASXJ7)c_|793>6JgVF_Ko>400jaf z{U5Sc%Kls3P{qvD%;leXo2RU&up)@c7t&Zp0~}mXxIZ4Ug zTVTiF*2rVGX*D!(8X_=AVAe|qGj;O-BuDOYo5gjO?^7}G_wy6X5QhP`mdBHHzq~Y9 zVd_*QC}XydM2)h_WHRzWHhIha5;Z0j%FMx0wBHV9rH6u_Onsc(R}*B)|C>3KMAd9=Y0x2Ud7JpxYtdoFk?&FsMuXf z7iLnO`QxG!1py_E8mA&p(kyGx`Y~V}Z6bW9A0AFa`sd}`aU4Pe{crCgNuP>bJ%Kru z0{+Y9N3wZM=?_%Jmu^)w&As`!RgnkTA{M?#%k|8S>c!duNKOM~J<920uz5^wH~Fit zuTGc&-o=Q`hK*)ww@Tm5SrECgiZRKm>;0@j-;T%<;T%On)m-%&O312FPGem;8jJsQaDUuP}}zU@tdCd*IEPt%wR+ex}}2~==-(%;==mpCr$ z3+eax{O}qb26m{|t|KR840*`OLF)|CnmgmHVbTs!F?vMFjQMuFxLwEWiV9;oMn~t; zHZ}#Hu66E-R&Sj0$t(HjZMg3q|8_v<*%;ZO!$7!0Md;Wb_^+4mM#>tCzhmT#8Z<(gX_L2NML64A^-p`xlf*6Tfc-}Kys zy|^>Z#(?v?QQ^ME5CK_!X&VUI3v%c0-sAi;uQ{okc>{w15YO=R5N2!k^1F%Q<`80Q zbamPlFzQkilC}m=^=>6x&j-=>h$)_;K^#evzIb-(o2nq`P2dx{B~yJURGpj;;Icf+ z!LSy+rHkvk&W1yTW&-E(%{X7Io9WDE&s^Fp5ou6%`l6p{{ZWT1<~oq6T@xPU!Wud` z88UUYZwH#S%Q4|i4enGbO|J;cDI{6aL1leK?5?%k+-0Gf_w8#*N#VqKnwlt$yW^?S z08D!c8?@U{W+_yUrw0c%|Cez1iTN6dZuruM_6QK=ZDEtxmG;It>Ag~=%w%}bO%1UN zte+K$Gge0mPFrDc<6&XZf?okgUUKin9zoAEe&9h}w!He3Zu@2oIT-QA62@DA6 z76u52@BeL=u{W|6F*30<`-d4E>A%1I8%pQ@-hR||98uIT|H!AfZe4$Kon*RD(P@** zQytNQw8NGxZI__?1eHSVoRjJ8-H*&w=yz{__wfG;Y3RKRWNAg?A>kd0?Ql{Z!^l&z zcpBRJ<>`m?HD6vz1=j`;nah9tyZ83@@5g+=pOoM9{g3p4Zk~0x{#WUXq4~~F#SOq; z8ziPaqt1e3{M+>5Jfv3{juoLG$naB&g_D(og|Jv5!1yVM3?LbaiA)4iMA+4WK;4+| zQk^p7F5$7LGteu5^~0dUXkMyi}*ea38Gx z&H=lK+;Jj%Em%vuJt%;55qElbOmVk_LT+|)GuKEDNz!JdIYo%BZZzDJv5q3dX?8KV z?`P{!N892br)*s9}j#8qFE!%KsmZQ@$rZh=tBZ6w-Ce8BwD zn2+eGlv%3C`6pI(7A!K|4(d%G6pK)#nb#@N&c0qcN^`S$?~!C|1>O#`ULn2gY#uPG zvrtwMHNjq4EDNR4^09kQkr{I?p^o`@gHtxy12LRc7X5D^-aqAju(9T`!&S2pC^z)6 zYk)dX5k38_MLcR%j@KM>euSc@R~ zq;-#yKJ&Ar$W_p79o*lv0wrYu`_Od_t3JnCYsY7gS3^y!SLop;Rmz9YYIxmZQZq0( zUJx(S!#3ylT>g|+$c)rmnCS4#wkV+@= z($aZ9hB6mhjDYDgr{rtvhzq1&{xHH$$rWw;G1WbVz?}>=vJ;#r568*L z_s@Ak8BRTexhB?Pc8sI#MTGRcHwq=#-O^t~YkgLSA}y%31<5+lZ#n8(2x=O!t&+3^ z#vDhyxEw3DA7-RKS_K~=H4S0Of=*TLa>Fx>K>4#X@N`jmw zK#nR43z!?dVO_=;JzyWj6}@mDd#Y|26FpGs=Y=c5}=#{7x53#`$qHohKIc3>ge`ntf6g0+p%tid0CFa-7Vy!w;qT$ zOS3wnY@Sbx+Dt2%(uKYx(xP#OZMxifDJ+b5V7tLKl&~l2&>C{ayd$l2Z%#eld5whj z#f6n4g&Ly5jUU8;V=BGmMyfKzKo8gCfEhSBT)g%9kowiTtms1b=)%t3-}$LdU2psqrukcmtFlrx69&N;Hql2qV# zTR9`I1`whX*ToULohhMUhP{KI#+tkdK1Va@x`R=8@4zQBhHVko|j*s?CEVBWPjUjdN<*b{rwlDtJCty zk0odzAO-aQr55?m$H_krkF;-jE|pchKk^1r%Q}=+F-yhe^@cdmLkA|%!DX%Vnv^z0 z+8m=>$LxKX$L3v8sCdoicT5I3%p9pq5uC34N}I|~hpx-B=lM5PKJ&9(5m#U7erMaP za$AZ$FBy;gfB*gtDEJuGQQ!sJS5?xFc5zCHn4QJ19iemfwpo05!7!j6mAGf(uz%cl zzq;E$>J-U;4}~Zgk;k}s5_FN@JNXVcY4)Czr{%D%czix;36JSh7WylvBYcGFrViNO zhb_%NDLigy0Oy?s@%$7y5L0)A_|6AL2=c>p_PoR&5@s!XapR_cJ9(W1gE47G5yMY@ z6_PR&ODJ?YPJ~hLX-3Y=cq>e@M4Zemo4o?(Qvq6Di*5Ulhh3cCyrPb=5c*>qCQiY+ zUeu~WF!VepL(E>YaSSq!U%Kz$5c;DWy6)_PwCKDm_aGZqNQP3AR#eoL1!2aH#GA>g zx7v+hR5!$zR!Vlnm)PrG!}qCwG-((1jz(`$@#SBvl~|Qv&J`e1#%d$|Fz88(1$_!7 zC4kgw!c>6hL}5Bg66`_KqTYv^oONe0FPCJdR778fSi@|BYsY5--EO*ST#$a0l=~q6 za^^GC4pQ@$zLl&%K3CPCw)(|{>~el^M@=&aI3%8Xe?m0-N$luMA0LVZ73#&&$e$O9 zZ8wLdUmsavVm)~yow-4|TYn=}ijUAjUe4f7+cd$1@%1Zcm+V9=*^^!JCz#D$MrhWZ)&BiWS!MvTSi;=Fk&Hi}(E z+vkO}IHr^md(9*y9L6!qmVRX+j*Y&z_JHS=o*`+KI4yZNwpDC~pBh{5H9!3J2!j4o z5kTXq782A5Bd~O5@+n&AEGscPS=wKgr@elsV(WBZf$bk}VEIbcZlf8)4tK3tQiP%h zOc4j>nYb4BYq1^`K!M}(v?0a(D!YXdt=*3F@~kCA01^{5gu%i7p-k-|Riy87pn>fl zx6USdhXaR~e2s%V$lWUeu+F_5c3}I2-vW)#$uw0jpS9E!*8po_T)l!hcQP12gZp6p z#1BZE%>jP&l_D`E1GuLoKy74PaC910_?mB5#NJ+(z@b}l1`AlfqWq~nV0f*830S`K z_!I#M+%v%a9SdL`aXP5{Ge&`1{?25XCrfH>E6buh_1D@{2HIT}$&?>e7!cvuCWC5O zjJUi4XIQ~RZmFJ&9xfmMbPq@8=#ckH2X3XuP?XN9R<%*n5iKfZ=^CQ(58{5x) zAg!GHNPTMEMzK%6E{lWCpZ;J`G;0C$^h_>EydXcmJlp45HCbsG26zYx!smIxOq>_@ z*E%3Q4H6SXoRZGXdPFNdPEc0C75l(-!g9R>b8?WIrP#9la=a8ZH z3PhAx=)Ec#b%|4+7A0+hN-=6Z6D?qqyoZb%WlpgQ*F|iWj)wI7q(u1uJ-Dr()f%DQgH06SIV z$J`ZKJbSWb9&P6_<(y7wD5CF!5kZz{G9lh)nJU-ElY1=Skt+Qs!gv-mWoeov0d2Fv zT+c$nW%?WPdO23_HuUlmM~8>AR=?NT{Kk8F)rHbWbVd!8(Xv5}nI)lZM!&7z&zJ|? z$2EC4eY}+jQOfP)d(-#)C1umn8tB4xs(Fja$fKjvU5ZZephtJRCVyXl7~|qxWubk& ze%&?(WJVQTW8WLZOip-cQk4_=QEb&?&?lqW3HrJ059*pQALEtFiqlT!3Utyc<1D5@ zz<5M}^42e+IL=U(_UTF`2B*CU9`wjbGuXpr(Rci6t<|YEPhI~WFFRjc?Y-u@`si2m zZtvC~-1<6axCQuWRZ6vy@fs7Z)M#+u8Ce&>=LjC{M7}$nbecQ-;n?(9m^_~b8@8ns zN&Euy37~w!fG?iV*?z~A&KF_RX|RF3X@e;U@i`DmZGX49j5xqDVJQFxwn!R8j-MmX zF}pj&T`}ew5Z5Tgr@J#r3*JAg($3$El$lJ1WidRKtgwvb+C3MOjB@=kj1`D+-^y!{ z1jdOJrUA8(F64x7t~{GuGT)VPqLOq#A^~3j{B|>S_jCN?A;u-_FX0q8(Pu>AGi-@3 zA`86nDI&G{`=x#5o~n=CXSvYKgHK;bnRql|7^b5(X_?&CpmSzqe|uC^zB zT~8o7UmQDMjJmG;TFDgACG85DRwWIy;#s0TK^W~Z^&S?yWm8Pz$t;PeavRD?gyPOo z9$)Mcyo=EoWW0qduHXs2>%?0y(hSug*w4D?JAB@&TdqLtn-%C^A(aQ_axa4~nk@~k zyHKlcK-~ zN{|n#fLMlK8Vjk5?hNr3K1^+XQ07WLZn{>*r*q{|AuGBFZS6_kTO)rl3!B#z~Ra@~UwQnJd zkB4rd-f(9-*Qs|?UY?YnMwQ2|ML5;E9y7;h6ybg7Kn}Q_JNYS``+iQeMgA=6OzSv) ze3yp44b_oOTBb1%zgVS*cAurK#__U>Nw6+2)jUxQZ%EEo#CSq z3*uI`bOP~uIb;1t8|_zXt}A;v+fTRyJ$VmwK)+tLNym}HA7V5>vN;=HwUsXI?eT?P z>k|FV&g{-k>=d7Uiv?y$$+uSXM8cNZp0}=)3Ko^lH>Y zjMzk`hDxJd@Eelb52bL3%w2p@G<@QO2p8T`^63+5?|ML4RNpoYr7xpQooLY z-`kMYHtvj1%AORKh9fg7P3Mb|?Yufq zM7xAJ=A4f*@Pg2TF*vP~yjpk3 zPUJIu(Z?fBqXA_9*x-UE{^H8riNFj*P@2Q8+F-E%EHwk9nZ~z(z4@p1yTkJO{V@D5 zotA&&Lj9xH@~=d&SB<)0y>3^cNF0%deS%IzjC?~`5o{`FJWY9LekV5*&LImRH4Of{SI^xMl0G#PT;8%;}p9ruhG1;s5p#{>8}WzZ1ez zW&S}38~@uu$CAf9Jdtw<(gtmpraVI7u$j5S+FWRKaMK=pooG}pUlH0+Y%mD-%?Ps$ zUhG`>HSA!@>%lFkA{|T>lht*`@BjO+z?rH4*ViYQAT|w>WTyJBw0OaxSj?T)=ifVDLwn@Ew>c~+p@V7Lh3xMKndwXm%sFhf^+sv?^p}PQ=ebq9n@#wT>=xRc$;@8$WgA!;JBA6Z*1f}1q%g=P zmPk^OYz|t0p6P_VyKIaDV<__rx!3J1PP(JUN**vnwC&T$H8KZBr%B{S0W8!OY?AO; zyRN*%glWGDUK0v7ma6oNdm4}Tkh$jNh6xLgyxV2-PBbFjdBf#4DNRVq8(lv@g}h^` zs_igc-}&s20V~+uNrv=bg;=C>*d|SUGdKl!GhOI0XvYc*2G+2Vr(vz@hgCyk&SK&rRx+Mfk6dB=$loB9T!baJ8pm zgIIJmSCO=)(nDx_7Ty6)xnNJVQMwV#*0oed*w*Q&ShMglcs9n7c|D#6q^i>ri$8;y zz|OpPvyPBtaoHjK-uKX@=c+Elt8lxN~|(Uf&EFM4xUC%S z*nRa`-{6*TU`dk=h^|HaVYr^)i}v;JLU%;?g9sAuQ(gyhZX&<+nMdKapKc1p?`|{i zjR|R4Z%MYRUldZ#979U$BL6(iGrVG_kvP=_0?4VRnoz+!LQ&yV13MF@mhg>)rAZM6 zHz@*on~pL`zD2rpG7<%sNVcElcLAxt<%;H8@I%A7bjz`##FgIV6=CT<|C12*KN2XM zyoUA=76=F&=bKUa|388LdEKtlgz{8fMf)>P>os}&g9wojk~tUFK#qk1*HonEi5Zl|tI`#fNm zHY*XTtib>6wEttS@AuW&z}ZO?$L(MEKya=S$c%k?1nwp|3xm}I8}6s`J2$+W2Sc9z zgM+Ia@EiKWX+C_x29VQ;l*6I}J7=qnaTO zSg-9tx0o9Z4iK=u>mks*`$Md}zlK!tcs(T{a#jK0@aTt$&|amO35QG|2|kJ;6w&v5 zJ4)%;{SNyO@1I6Ur-zO30uPR!%2w&T!2w8OKhH;nFU zEs%LE&)pF6wic;02W2QZ&mzo$kso;eWeo^a0|qr}3RDAoo_9K*@)0E6+`ookdoA~W zE1g6aYzHfD=m*oK!#3^(QnSYM_G5 zX!JA6Qr56tlvur%>o{S&u5)mC5jT$XQ)mga{w~?H!%*9Xbxq~=_SvJZ?ZPL7@A(IF z)ve@9cr6$&l6}kY;zs!iRDNPQD$$yW+X$@;W06~4K;Hm=7G(HETJw_ho$TV2p~6Uk z3Z_+M7vdnty1+=dce&7Y<@g5 zf7`eCDE?(p`#io?7Y7y8(rpfh7PrB$T=V$Bl;Swd2gY=|?LEyeRgoDhWoSiX;edF2nanu#n_^t~9L_Zd!q}MP4)l-K|ygCv@>f zx>*;zllG#$T++;i*K|6(4!pS6#xHdbazJoXR%Y3Lh}b4hi|uhCCNnnAHaGHmk`(LY zF9kgGO|wZ+d)ta*Pw5s$3i^0MvgnH;~y8K~!{ESbOE9~j?>6>86@6;Li6!=e!DneX*= ztq++K-MPADT}#GA5&c0LGG8h5*_=@7i4)JL98vxb+f1G8f$MYJga0u(l*46fLduc- zkrfjOK&|MM^$ln3C4hZ;MG92FKV&Zxghjb*3i`cD*)}~**)m*HwH2)uWK@&ll9$&~ zOuZwYJE5mu6vmC3b(my~QZPsn#Xnx>GZY&`0_~L+WohQYh+J9PL37kMq*$rUVo^_3 z9?N7}OLbHln#GN}gf=NjjM?5?tNP$6A}yCySgrT^l2#o>V-+vE(z#pd1UG4_QZCs= zrDB#%gjO!Q4m6vXB|GMOVl925I1_=vP13lcAOms{RIkWt56AJBl#0nLJy#Y8pG}EH zIe`jIrTQ3E#9SpnE{{vuPCCvL?mkPoCbhe0W#Y#**`y(5;q}xL(I~L{+o|Q6rqz^g z6^B_qYbLWnM9Q(Mio6OB&4X3-y#kq+YC8g+&a;BbsUVvCbaI-Is>1nit39H?3(}WcP_L66R z;mrt}ehKyC*e|l%lO{`rSXt1~gEVZzIMhA!_U z5p1zz+ntuv2poAdk>`_%Dn2I2ELix$Tp%lbNrq( zDD9#^bd~3cvE{WTyAm=MhY$hrGp?iRr`(HlUa#bNSeDdf|2uQ6^ty3Y?v#gyL@2(r zIxJv=aH&7)Q+-6}E992WZuxpu%$V1$Q<)Gw{e6_%!CKoYtdI&74jj=ChV832qnHNQ_^2PcK1}Y<^vo{AQ}?d12d^;D zzSt)Bu;lv`nZX6d-0_C!`66)n5_tY$Eq6A#aeIbP{xL9j#NJ-k$gBJ?M|b?$hl4Llvs+y_W7z9vt^m)oMFiWRh;MQi85Pvg)1tM{T!)H=yUC}JkCMd~ zQ%#2hDW9Ae3^dNl7UJcPooLUMkPzhS+Ax%9h$D(h0%ib>L3I+Z*7 zEN~ffLi!5Bz8TX;4w#%1Q(Zc>PJL^~8c(N+nNfJF!7pY4gefdf(YX;Lt5!YVA9!!} zvSSc;LfoD#Ip2KjeJXT?y2<@Z+0WX zC>dhID|eI6aD;^ zW=~4`%{{uqaiptHa_bbTAB;mDmO~z!MIKU*I>e9K8qyh3-WIJ0fL$vlFW5ywN%qaEWuh3K1FP2Y@kWX4i`ES*boJ74uo_EUR+3IutMO;+g}51KPt zxm5H(54AlhiBL?X@&LkFp=JQl*clhqSW4O`hnvgVu{EfOED0DwG~`t9Nb8aY) z&a*KQEaAbxp4!opwN%BWqGZ5~g68$X7hs&y+Lh7x?NpKDrB?X;lggN^0X93`322jf zkJrx#K6=E_>u-JFYDD`r4mKq{)X}myn=DCY83W=cVL)GD!LV{bjOMnDdgJrx7OZLG zqlf)GS36CpS0cmJdhdPwQ>b;BU|Krk_fej-{*Z-$`i)2gv@d7P&rxoik!txpAobg# zsTZ}H40{4;oVcQO=aAJSV|TmpozP~0vUsQUi0WIftr=^_6uVzJ(T%=7K_X7of!wwe z577-g>x70To6e$WtHU?4k4zwDgS=$C(emsa)XE7l5iJ`#(9 z+ZasER}_=Fmx*)F-LJn%;{Nt0a=@}=U&!EJdZjZbSA33l{hM><^>_aXCH0R>)W@@M zzWp~vi3Rn4mQeb?P*UHM$$#rSe79fzeKzTXdV=O}(K1P!BP0x&lnDeQx>3uN!-^SA z2>Ii}xHDA{QJW!U$B3n|r)4K=bVS+;O|=wN$BK-OSewrJM^`3jDVk5o>Nn!W=X!MQ z?3{tpZy;;ZT#B8De4Zu4VeW+Qtap!Vum5>qT_0j0)PtfBEFpT@@7vo; zCjm^;fa!ToU*C8q>yLtw9|Zg-<3hX#;}!Q<-=L3J5r5fX|DwNI0{jsbBPWfX*L0hH zCsSzmPhL)M!2}X_t%}^*(H4%fy7_1X`6>5e0_8_uSPZXsWCG@go>cm-rUu9g3>Z9( zh$!s+w5jr`#aI+(xep|vU;MqqHx;8Nu?QBUU%FKKVg^+2-yog9MmPP9!)Y)9eSnxi ze{jQ;r`{MDRAQm1G8Nr&wxXO03z_bKFb5qUTAO-hdbe(z1T9^Owz#-(R+X76+hUQK z?2e>ebZsJGT{>gas$^Drb~E>is5*}5Rgz*V;~_Gzoj5tZ`m%Ma42(paY4Vbhf7E4_ zXXk7rLvh&am@RivopQ<3GKrCUzqEEjgh?p!Tt-oH@^)cWle^5Urgp4kyY!tzn0QY9 z3LU*DX)5{Z4V$)eO4S$}RKMv>lW9}_QXSa{X6Tzp&l(n#~adRIq3lR%t_R_3704k3ua^KBmfkI5~>&NV$ zX6;`{$)V8WB5O-12Sdfs1XS05+;zGiE)0E^LVj?dE$H%k$Dz|{k=~tns$kX9TzL!k z;44(Avx^YSf`!$elu5=h{a^_;iTvrM;W4%|B{L)cjCptrZbvUeI>h zOQ0extj5UDs7{i!P>lXNp@aaN(6sG4dohH2-GI7vwg@@$nVT){T}GPRkM*(H!Eken%LbEjvuyuJLXccTLDQrUa@3 z?sY4U)?<@eZ^2Ldha((u1+=DTaJheI7^dYV`E{*rwW!zl^`wF~&wE9-SyFbCs`*bY=*j6Ib*zlHNW&1%Bn zRcQY}f~nV};Ey{KFelf^*QnrvLwqm+z&zR~unZY3o<+Bf#Q7s}p@DQJ*GfF`v7)YA zORXLo?fxT3y2Ga$r4iVsynjiBBorKQsEs6P8LOC@eUr zpI`xL_Z(mjN4Ly>hVVG4u|PqU{dYNC$d3x3|Acv%e(DSb+%ZmSNiJK>TpRN!MWq<` zk`Xl61+ ztxHmty>ddk-f)K7&`vTMUu(x(X#193t;b2m$8gp+B+tMlU&|0_vlaDlH<#H^O;qc_f2 zxk?N(5DB(4QLEQtpLg&Zcui9iaCLI4^{TtnFVRal&GxA4g*)n9c($1NP$}|$e(1m3 zj#EM%d5aaT%|BJJfN$mZtMshfJpjU$lUExfnmHMC&9=p;wOeX)Xi}_dBycxar=_Yw z`bFg`q?CKM5<(Df;?bc_yDVep`KKMz!_OEbYOzcdz{xqw-OP_lKu2TM5}10)wmIA1 zwxilYp_+Es&U%{WS7OGP`heH7)lKR7M6X=jRROg47Z?wTbVuN{L(2Gv4j2;lv`0J@ zDPJPtCDFelMhcs6aS31YmPh(Ke*T_ZIRQFy!`nk2oTm753gtz!k0&^ZwRPpaRT#0( zb5L>!@m9(iyb%OFz zj1`DWb!A-7W9NV*Q{{)jG3{rD(x{D>l|ZUapW5#9orc%E$zhOWB{yf_Ww>v|FTAH? zGk@}hJyPI&;Q+NKR)`#=5k1Hw`|V&KTU#ROIit*+t{b#JE|faDsfSiD?MwX zCy4u4vti2;_vccGHO+e#MVJHs;8WVpQ1YpG-t|eGWFmbFTRJRvz z*>?xxkUkaLCP*B*w1v)&H}v9E|J;hehF;Gtbxyi=WXA$^P3a>bi@+aYy*kW)#T40z z19$e}f7vhuPHiwH=E)Myzf=UU*0*zYEPz#ZCE``txjXV|Dv1l)m-ov(#$@(~edaiM zYtDQ?UxDqU|D0pXr_0?a!%h}fy+A#pycivc$?PiQWen8H=wy5YHka8KIeGjhmI$0~ zlc{F~Ya=Y*8(PAI{I0P_r*pnuu8sncSbtP%#XV96Sq~+N zCUZ>D-5DU;I|Efk_?ew-SI|Y^lwjatE(;fD>UyPL(vhJEmSOoID%S*isw*c9R*L$k zo(kh|N=<)Da~utD^ojc4MPAB|UJS~Hg?%ewE)mEx@u1|BeSTZwwVZsPCeY~#-5lMFoFn&<1nwO(QUo>2l^WZ2`{7& zB*7EK=e7RDl{`&Ti5>sVSs~KJh4Fi+b;US31kcIQW%rJW7U9W=yE{oKt5DLMq5N&g z8V|DR^1>5d^xTj$-rWHBrL5d^me6lo9=*A54~ln*#*O-Xh>qk-;_dU^J@EN|g_NP1 z2dU_M!`msoM@VGATUesZx8@~k*!)Qq+VKI7Y-~1Vb0bqdY$)7Vq>x_W zRRnVok(=~p70ll>8f1(=Kmn=u?Tw7hjZ(K+KF&8B9FNam_fY%5ALz;(aZ~!}X|&a$ z#)N)k+x<@q}-(lpD&q&60Q!qwMsWJL>V+Wesm zAm;*6LDTYGmt3)EuxI{8lAbsOW zyc#iRoW>(eYj3GJ5sHFL;Sa_zf-xyd>?#d&iLWx82U*LRjeH7pkc(BZaOR|qSo5|{ z#7lVu|CM|||B(Jx_5H6rd_Vt_GU>mS)hZ6ouK&c3|NBf+2>yohdIb&+1nZGt7>;;ID42!0~tN^Y3jQ;W2$NDq~UA*VVB8%gdWUk z-`fD>NZ^S6dZroGtRA=~5hI!hoIgmVw%z5q>xxP*MCBFYv=A`F2s{<_{s#ZAYrZh7 zSvC8P#`tgC{QriT|Bp3`nwcB9*}DEWS5g%jhh;&;KWp+AtO(iZ?26E=!Re=~*+9tA z%4kA`)+>?t$j^gzDYQ{E>+tb$uVRh|!(gBgBUuDtnQe87CuNZ#d`hKbw(bYUUPUNn?)>#FL6YG3z_?LpK_fR28iv0`fEng%XG5b?8yibA)>VK>nB(K z{k;AK2yum5=Zs6YX|(v!+4Aeo;*a3p?&OKpu-fAtzxt)^sb!~eGK;y-C=DzWx0wNJ zxrA5`k&sBYy!OhII9+340rPGtI(%QXeni~%Y=6UpekR97k%Vg&Abk7?5GB{^;s7&( zE~mW#h`iC*5JH77J&9AiQEQy6Jith>&~n7(kLoEryjiRJ<=bSw4-nzb6T#9k|X*8w-3*ppHP=1j}A*%ATB^u z8aJBv-H=6O50*1T>G-s7cjN)xLg-Oh4s%L8P1{`oBK^ws?m;QfPu5t=qecxNQZBxw zDLx{k(7=|Bqn0|TH;7tkmr_|G9J>u0L`#;Q;LMrxT+ns}#Uva2aIH5UPuLp;F}Mjz ztTGR4meDLKGM6I8`+)hc@oTn(5|01P2gQN}0+Rb*cGUlf-+#XS>o8tXN#tMTS=FYM zQG`(W@lyZ#Px$a8BqAv`7agn(<$`*I^)oL~SHl#4=vI<{z-CZOHI<{I;OLx8xin@55cVn8 zOf8S0gG@7Z&PHn|l#HS{wm)$T(U20O z%-%b8@NAxHbAHfrQlZMq+K>US(BDRVSCD1)r0&(Xy2)gr7UVHb+-u59ZkzxDZ|fj- z%gSx9KHrdM8R?cU`qEdK#^%5Qr>ZVUT{@i94nwkIbg?$eA_4Db>oZDCMN@lPUw}%5 z8WJ2soSWv1NyMa^oG{iJ7Qv#?OmhPNLdKt*IA8)?P&uVIVQHAg9)OD!`Wavb=dp(q zEdwrKkM!1Nt^0=;S5t~dZm5NO_}0`2)xt8QrSopuP>1F{ehy`6wisZSYx8ZUpT?nG z9arS6@L!h}0=O<*n;E4u+6&~7RSdbv`2+p5R%-yg8 z@U-5AXLGUVW5z8-`eS!S+ps$ioubS8Mx4p{4 z$3B}lDq0kKADa*#e8l*#iTJhck<`{DWjn27&2)c5jLeMb5_ceCX; zdHRp3K2d035NQw80b&f)_k2!n7;TV{TIPjJYn$YT$WW(gw+=)5GO4(jxGF7OBpI<7 zc`JAoy{zr!srI?DZgaYtuaLzd!8RGLoZ!nOVEprx*B|wCkmli;eh2Lbvv1!};E>GQnuXe8&#A^<~_f=5e%V_Cq-lTy1u&V_82;!p$ z2%%lGn-&AYS+On_vcAE{l42Rr0P1eD-=WbIb`wJ??6KoWIL&5~A_=hIRTXNVgkS=w zqYJLpq%5CKfP<>Cbqey|h(=ust8=HwEfHw2vRu{EGKQ(F2Ig8gC#(J8dIa z&h+l=YGF@-EXN3lt<(hy2&$gBLxuRHrE~^L2fxE|?ZJ^zZx0R>uNpPYkdbxM4`g6gJ4d_$@ssX>6{-l2unpUKZo=9U~MuBr6!9X~*B z7S_;=_-NOaHM(+q=M*^$QFKENX62^3rp6CHa`PjlYvuiwPaaTQsS4(4ReEcLf9#x_ z@v)$A(nEvh>eTEEA(qH0fEVEW=>94jE}B|Lh8KgxQl;C4E~vDJW9tcdTZRG?LCJ2= zKI`_oS?~?+bBy$eUTf5#{-kH$jI>s-Wxv{~gt*AX;e(y5eBt2av&sF@aaAHHHbOk} z!JBTXv}$hAsJ@3xoCy=N#-vryl%=TYy17p{)&$PUPrn|QL;FXvJ-3E=|MwW59AlT? zy`_|~dH)3*dsHxZ`1rXE^$AIbLTn}LJ9gIV$m_LqheX431of9u!DTKToVU!x_3tQJ z`{7@pf5|N+;6kezXxAW3-o1@&WM**Qq+aK*NxG0!*ej;h&N!3<6`-AM`a%zw48)~@Dl}yTIhqT8G8P^9shRAP zlOH0usG9!1e`$84T--TAo*~7`+^0_ADS$|zpf^LE^r*UtCckG)V$eJ@DE4_mlu_n{ zY$FXX$WAp!9$CbF?HlT}JtFTD($s(jC4YY1=Q8OGNE5)dTnNLe;Tl|A4lNEh?_{$b zHix@(N(zs0iVBC@2f=WKlj{^6HDLRM3s`roG5i7{KoTdba=`u*m7wNZe4z3v0~oj` zfD>YT9dN!TmAAU$Tyl&VY!A%ia8LA}eMpodVWGmN7aQL>lU!ZBRdKVr-i*T$L_+@g|0yxgAO+a!9P! zUnGP8{%9dz@aJzU(C2%(H<`Sv0p&&qtokb-emaeEt;w+k`CdeB$0!y}Z`R}n zV*uHN2b9=Xjjfln@B{EOHGj%JX0_?1sEfaoHfA&Lz5_HP)Qx6e4JrS~V(+w^^#5V( z9fKqb(5>C>>avY4+qP}nwz6z=*|u%lwr$(C`>Q!8=Ej-1XTEzQc1GmRKN%UZckUN! zJ&O_*WR4nmQ&mU<;Yuk{V4q*gZdjYOsD{if_`Acr#>9IbdhA1NM`35U|20KIiS>dN zf=K^q8fr@NY@Pf2Lx*;~abe%aVy}a&@<}p2-oqXGN&r(`J{m#9zk?67po)EGo62cA zrVh@&A@?`3nbN>3I%9xRXl<9@vM{rZ~Z^ZhG)&m*eVEkixBNc;@)zq_TR*m&0`n}<$EB24H zUvd^rBzC{TYcoVLLr}1)%Ux^e@u0IP=g1aBeCC>dQ~G-D$QXgYa#Ptpyn+UOmYo7N znEpBcZ@~sZ^dmV~{g<^*P+r>T`Iw>o^odgx=;3noz-2nvYD1u{9;1#xf{sYU4uAWO zaA8+Co}nc*oFpLc20n(&_KuPO6e;B?hRY||2>4alK2PYfIncol(y0ZLlG~pOH~K)m z6Kwndu6*EYe}kQ+m2)@&?P4pvO_4NN?Nvz3_#)T>QMQPf(lxqZ!|V*Kff2)@+zO7E z*)U|5p%O2yh%GcUBZlxFo&i*zectcLiZ`??t*_KaQO!WbH3`UqJ@gU0AQXKU(g=+t zG!(3C^77b2S${f*8X8O}Y3RaqPjBG{TS z%|Ck#2RG?QS}a7->cF4#sxVg{H_@lNGu%OC@E-7*k&rlJmqW#wJqI;5d`B%WiN;|w za$7iJx1G;U(;`Y6gXQEDHUb40+vJsW6*eH=W5JO$(XayEV=YLL_QPAS$*%at8}~8P z%y3258h*F?7c214Yz@CD>tj}#%%{S5R1BCcxu7duSvrs-)th6XDxj8q#|4t3CC?IB zl48Fmhi->n-a?p3K#m+~~Va>}Nun`Td(Mwa=L8qyW%=%(jK8I&IaM0C*tu_)5 z=QyxTo%N*3FT~MIu;52+204r+#KC%&!Vr#AT|`IJ*HdYeG4|v2%j-_D#g8RKln&V? z5le=0O0n$KeOCN^)pE{Y3x|?MN{(e8`r^nUoNa|zt(XTQYk==Jrx%$;jL4Tbm~AD? zDNK<$)(2fQz@hFZ(D1!CM^RD4rYQntGiUy7hMjgoa8*Vd5WHjvr9F{l+2iC00my-A z3C(CpXW7%YVCrayi=XUY>io&t*T*0|uzU;CbRw-~JD~dw;N0De6k)jta~D@ZpdAh4 z4ODw`tB8JCn>g3pYczmTq=Qs6XlRnqW8f19<_QR$rA8Aer3#Z*NABl$c$T;E9f4Or zQHUU-y&D~#U5hIKPCMA=AP98h9wtCT;Xtnzz~~fDzj8+(`vOda6Wfy>sg5@xL0POk zL(OZqi|r2PX%KGCYtIrT$Khd;10U5VND*&7!uw&dhDUDD)DsnXCJ|?y8!O;es4w}h zPPtHm52|PMNk*8}*)pjUr&S8t}Ag{bJ zKqViDsbyITtd~C5uQW*-m%L5%SaZITqZQvO7sMllUi=F`WN>dnt`}S~8_Ji)R z-SzeCs&%J;wZX3q;dbLFmAYRaLOxXn5Y;9Tn({);c2!oQ7@JWbj$mb{G7IzC8HCS0 zqR;j`7d@AbxHviQx8U49+zUgC7yK8dX_pV+Sw)*{Q!li0Ivo?Y&-B=MdBf)Djf$X7 zHB_4k(wQH%e?oZ2+h3qw=q7VXOER+ZI5-5(OM|Nl3>^u%$M0~8u^+g&#b|35hEM5r zcx6ZY&6kM=dS+$OCEaj*I|Ge{X*2sZjD;}67{f(!1B z=v4xSI_*A|*t{vFPQwqnv(37a@2J$xVs-LajUz?TWJwB2iO2e|F6*6bHSzC1)rK8d z`j_=s&oC7F%Axf0ptlIQ5C8mYJCt-p8m95RqEAKn|27Z*KXZwHF6b>?kry$%NmnO{ z;&)L6*5C(-fmf)=B@qJ_a*&Y}l1WN{$GUJ=MHM6M zrUV@asUM%-Q)Smy4O&!!<}cV%_RAwX2<46IXj(HUsElnXwCs)gvS&pT*;Lpr+$?u*ZXRWQJn4a8OtE3^ZimdwG7#w&Fv} zD4xHn$6KiOVZJx=L5L9Co2c^)7~F*VGA<0(XR*$c3xL@?d_`GH69#I1mG&;h@u7(( zBd$VrOGZZbwHX7=2ge~X5|-dKVzF~#DN9KbhN2|8X0zG~o;+6s*1Yt((v+>-z|k$* z+$P42*a>Cycau#&SMalwpR2YdndA(Cc2?1LzGtvUe z$28W_s1s=-RBUC`5MynTW&(`$=Gp=d{CSJ}lnd^R2;Z4v+To7rX|@QFn>73oPOY*}IQ7drr&Q!6dStzZ5@z)8C$3gXjcd9qFi9sh`z4mZkHsPZl~fa) zwJyLnT7S$uOAEO*8mLpHXJpHVa?`a0B6$~hy1;FgoSg!t5$DmdHcis#1)@jDxv`f5 zT|<4>7;t6|I9vzV@6g?5uo}S$^}GH6(TuxV%Pxfe3hiJ&+mQv*eXmJoFd1Q zCDjyPr0yMBygN8l7MeYg>&5jziy~UNlk3I%=1T1WXcfX^pSLvSS3yUwATdHygA(hm)T5;tb{yrx~S2!iZmeMVE$2VdCR;7a6t(2x(95UBN zGKJec!E)ZbvOAu>WbeI2bO=tV!ymWoo3mW{0gce22=swWQnEF&3b_4AR@u`zdo{^3)_MtW;tMq})t9VP}T}2R~Tk=u} z|4#X#G4$R2qpG818XUT;knTTITnT4K@Rq#fue>6CO?==!1A`FMS2|gIQMj(cTYL!* z=_-G~fDuupAtpq0m%o60jmA(3*~~kIR}j%-o-RBXwgeB!J(=xpx)UH?EE0?wQItHN z?pCnDbN7ZU``b}B%10aZyB9B27yCD3&Q`^ufI69Y)de@_XCk5=llc0h9LdO8Va`%< z>#I(*mCn@82|1}W41Rq~sp^(sOrdL^hY&N@QADgnTz#Y{f)@)4^dpZ8Yv(I_kGm4@ ztMnZ#8ZVLUi0A-CuzlvrnLiI>{i!ONs%vRlMWZ+$Qj?>FETdY$S$Sa?(%CHQ>qwat zD~zY29m4sfDdvdw5Y^xnnX~3@2RB!iwdq*zql(t-B??1~OxK;EHc&L;h)iLf<+&yi zG12BSehTNT1FS8r*;|m9U3Q!tG?Vx}TBO3bMJ zI|wW8e2~u~wscReUGLaDL0D!!E~Tk^R<_QkQHqTR_1?04Y4GFlB+DVj1nqJg>Tdu# zLBrVqfoSWtVBJdTkeIVDYT)T&`0!VJwZfEOT#~`^1bAT%cvnR8!&D&@=@Y*2y6Mz| z%`GW- zq9q0sUv91|Z;@M`g?%I?Q(nN(uLvoGK)5y%1&HMun*TnAYup!dgJ;ht>1!<`LgUP4 zWR(6UC*VwpmKfdsDAMlhutr&iG5j*7QPLiGq(VHFcR;l8sW%1hsx&Y>6{gbLxUFwO zgfd^R{Lpd)nIB*Z82qKy_yC#qYkkTP!u%7lfuCyk2odcm_eDc@b=H(#`-kfL9#~+p zm~MGYgELcini+_#511W%{S6uSBf9?q9iRuP;aptf4q1vTvXd<+*fxW6x+T{-POz9Y zAULY!i?h!*Y|cA$*NDw~MfP_rBvgqtH(~oO0T3c69cNEyx&@Ay+Tr^oI5VO4$h zD_|54SjJl}kgSQ(Qjn?^kd?qqWWsv*LHOxWat~sClq8*@dTb{!dnNTJ9VK?*9<-9N zaHZWsJ0d%~x=WisPo$skLTx~S~mRNZ!)_jMiu#7kr4?2FYSmEInA?BT=q zS=1w=l*RioCK~upV>I|Jksv0or}RGr*?$>3l+evCEp&9%Ss zF-oQ;zGM6382YQ;L@yc`!`7ocvrjzl$AhHYd&;G}{#5yy-nw{+&AbjaY>0b``P}NY zKW}A==!;`yvBGyx<=*3fDqKu0dYlt;r$3~NQ>b{p2gR|%UsY}?&uu2?JyfKUK&j9zy7s2eO90CA^OMC?AygE{vVcR|JyJB zUq1U+Xj-MRrtJ5^?5hY3HK|bvxNxV?BGUS2GNev88S;#UzX&vF2!tT^c$#zbDgffP z7k4k8w(I3bXqZ_Z&wKs?<2s6EX#r8u=|rl->y(p8_urQX5I#tD6#NtHhKQ0A0SN^( z5@P2nnkI{Mh;n zzo~JqZjy&O`I&6{;B$v0c3qY*;Vup2`GJ(ZKqibx%%WW>I_N>oCJ8D!V1^~d-kC&3 zDSB1vO33$ECH<2rbvGTYohJNg97a4rLv%C0B&UE$>~Q;;x+3xmsaRDCzXVf_NYW-a z3geQB+p&xlTlA0H0mMOrwqT^&BV%_ZYP3MA2x^I?Hrq1#X%A({K{-|1bW?M+X z=W*C5Uv{K08}YuUy>#NyGe7K=1QDLZE~vu<&sOS52!hT-nlMf5H3e}h#0_c(jqqkN zJp6@t*I#suh)Y==39mC$Bk4B_N-sJTveXVE&6~C=N47>y+9gJp9f`Rz57}cvIpl># z6KtuiWBThMDal%QxJ92~BcH6%ckd=&I-o$vM0EOVW4rC?I3i4N6XtU=S5tvrXCWwC zCn<({Ea{S~rqWLuHdGtFU>0&cR16W2 zAf`g9IM3i8pI*2K8~m94mSn!H;4j-(eod`mlukae+I(pkaJCrysw`AL;okprd4eeK zdHd}3lA?*%A*6SY%b)26-Pq{}8?YJ19nU;iaX0Fe4u>@Ew z;g>K6+zf{5=YaYX~fDyKJ@-MY<^8bA*9>#b8UJy%J!Z1UvI_1Rsm1IH)M(c?LTWy{+HYC zpDQvbM=WKGud6N-Mo#IuB%ZlBc!}I}D}gixNYX&QbUCrnUt;FgC{A^$B$_Oo<5yHn za!Sa^+trXwcQ@k5KnbdA4Sbr&$iTvGlwmnOI1euzT~Wt!-JFb^t5z)ZVAisx*Bc&x zum4{C?U*Os;<=&oA?Z*L67dutgxtv45kc|Zz{Ay5snHY$_ZYiD*+A}{Y$5j3yj=t@ zyKMyTin19QW<~Lol-VxQV~emUFBxz-c)z6JzYfs@b{Rte8M9Lbb}_6D^RTVXXqWB% zynFsx>aNtw6Y3cp_8mJF{p;?L=ar2N7p~2H{P61^XgxuSyf_QwVB$Umc6XwP>#=P@ z$gbiYd5~xbK0Zs?!U!o#g=v$>ik>|h#>IFUqIh?kIkS0cugtA&00zb;10ztPI$``w ze0qcwi+&Lezy=8(6WF9Nt+b-BxI)^U!4!D+0|Cg`U#m7ELHJ-S>hkT--Ec~R2$K_0 z-4Z6*1M!o5bIcgpHz@Crkz+#YPkz?Y{x1*CLLz4{iDMAe2~b?hpcEFN8Pe?)w(%$BDM>8~X|zXS`(Gcx_3XHY`xg*c0G z&>E9kV|%OXAV(tM)5Oi@;!$nQtB$2+UqXn2r(CfpgLCt^ngYQ{t2m5?7+z1a&2b6@ z>ms8<6o66XY4!X3Mvxsh(mS%^1g#evP}*s9^RrGo!za@SD!9Yt_mNxwDy$`OlO zCXW*dph{^H5DN*z!%Vzc&roBa;RK%K6BIQDYCk@@ET)Hu2q|s{^XS6$S@=zgTz~ar z<(VR1t38Q+mKx}5@BT9;(8&q8Xl~4jv74Q16bubbJ>TNw}2?FXSPOAoMEd0F*@fC6d~1C`$A*DU&nu15&MRZ>C#LGhDPRvewN%!;Kr_u}@- zX8UHyIu|sKu~Y4o4!3t{?3pE#Gx}}Df^orHszK!O&E`C8xV?K^ z$B{7Uc@qBxt4A}VJJ`d8Py8qocYQe(0Ji8cqINbzAi+av{&x&Y#Ks9y>MCaGOx6M4 z!0X&P^)_&VwR=Vqwn0fE6OEx!XT6{bDa2u4&_PRPl!BEOJ7gNH$qIue(b76eGQe#=BmW3B5cCc8q>kO~ApZf{Y;Jl9^er`oj_1%$k%4Zk(C@DVFaNMx8o~v}mjxuiQ;b5b2E| zh^&Pxq$&AJY!D8#`Nvub}n8~!+$)L<@2&Jn^zk{nzKg$E9D}R5u{T(sPx4xWMQ)I2o zC642F)esnyR$yjD+tTOzV@_v#4@|l#&M4z%OT{P+KvB8+XXiB2d^}ey z4*D3$N($!#KH%b{GCyf%a_ee=$GeqLP2Wu@n76sF{VqW!@JPZ=`qIE z`kIK^8D|X67A&zS)s_lo|Q0(uA_=G{cr0<=l_k;ino zu9V=506@=>PMl@6q~JHt{xo+;jHyX}wK%Wp>I?qv-tJ`=IUgjMUs&yr%kj5X`sNe*{ZndUKZV{I^(}NfflwEFKrWZnniWbK`jb+6vC*mIqk! zN0aTodH~xxXvQ^otB-vH&M~5bK=v>*#3#+yn06TqwgdeqNXjunJ+|?RoKLBH3dM2( zK8q_@45O)g5qmb_lI1hrW0UjLyC~NYzDH4g4=~R(ur^~%$G6+8z1X8q`M~C$BV;Ct zqZV##iy$Bz3jU7NVMSMLHJq!t%86ZRBmBlNu=Ht7Mreg_7^d1SXj#l7UfPT?^8^ppPS&TpQ7F8|YB`!8X(e(H zYXXri;*O@Yymyw_^P$n;Y?*F`cVha$(mK%zSW^f{Ar==B$pETpjmd)hccUxomf?KM zK>hq_Ri;v@6*Nj9iG7q4_x|ZVaRtqdPRF#cd9F`Z zFCA^I3dL4Y2_Y;I2au}vdhiP<5Jv)>Odhzrt-zZ^6v-_$h98j_Vq01YamiGyjy8o5 z1-zIBPWfym(DxrZdR@oMAp73dIpb(Abe<29w;z>7(?09wv7G2WglZw2wa6~BrY`Xd zI@80Ftu{SR^vZ6|R(wF~;>CV!$!p%z$1RNU2NQUvM$UyV(?V;7++*yeZp7@=$ZDF- z8hYM2Cd<-n_)WOVleukW@Z`}3vvhrvxn(oCuRoP}slUJQ&?AHqlbP*jxVB*CfY{F` zr0XB5I0Haj1CRY1w{$WC_(NwunRXH0si>LGfv&otF`{G+-0EuB6lycXTh~|)C9XXw zQwvn(vi;Brjh$xAo#|5V1~je(N}UAqtlndBCWAx zJwoZ&^wZNDV6DSt)`n5~o@(-e#xZ&GB$NT!UkJ^kCymLuYa2xE0{e28*ba>!Y}LIT z?a-;P@$p-*$?E|r*)k|8_pBBvfn@TxNZ37BmqNDZldvC;Fei_yyHuzoDwi6vjRr5q zy2WmSPliW8<8tremKyR=N8S0lcD){@^&=0zoZ@L&qiCLh2Y-du96G zflj`WY<_8SoVdKeZ}kn|czB}z(F^;C{rN!}w3HqCfaY!Yky!qxfBW^>`a=->L-OU; zW?g*Q02O}j zl-#=vT47|GgH$uTx}&`}rL28O{jvjc5?~rEIfj^NziI1t9($a)EW@cd2|4+JX$+=c zbxhz#^jMaE$eo44IzoAe1+r5*DROd<-yTDlnggB+3cB#;(MQ|w^m)AP;AkG~@(ztK zMqZh%bM}2BoZn2!YQQT%erzHEH9}-+G<{}xugU_#=^;B>z0vIS<18l>a~e^^fMIhz zmOo+2I!D&g9v5^r$l1C1uI8rQscOj?7)ym?bUeY+?ZMWL_ta$fl`W6iN-xvV{U!}%+ms+D^6}iCi3-3MkC)t7T+|Fd&$}I-L>PE29Jm62p#1SY^6-X8F2|A%W znmaR#tCQjiv*1OD#ZGz@^#upJ1=Z~HoQ!gd0t=<}i|@E?(Q`-Zp+?XtdSOr9nm%~s zpj11F^N5+Rn!DdgsrvV zQauwJgPC+orseRimPsn)v+v{(Yh$xs_HGvd>IXBz6^q* z(<<5W%`{`9^0X^|Ik4KsffBxrcJz#~Ehze?HYoi?=*=r!zmAj41bxX0P#0fgCBvHD zcKO25Om*xaA7AShxVxd4z(SnETDV~R$!^T%^n>FO{FU&rGyzlzgqVlF3XOwVg1DJ4 zzg_$o@Tuv~%olqNIeil_c=%0`0t_q6tI^OR0au3;fJ8%!vJ6&r zaDgT~PyStG5sdS!8nJ4ojL)5O`2`C&7T{g|tWINd(I%Lpi=}=_IQv=(P11LSHvr+_hA^jd*HWV(Zcoz5LH``K^(!DXXO6^w`)T3*VOjHm6V#Pgjo-m% znBnM?Z5Jj|*{K3?OPrQ^@eIF9FlkSl+70E57Hb1<{89&YkW3SEU406B!DbQ%b~Mamqsg|aw>GBU|~&Z*oMwD*FWj=%!oJ5jCm)iX8I)ddr7 z%WmNRGLJBf%NmExMT^b3g66e#>=nBGQFJqzCpZiqSF zOy_QzJuo0(kC0($DZTIM(ZzQqVI~r2i&5;m^^FRJT z{=54fR9ScUCiMT!Bq25jSCcRg^otBpQ@~sCW2I2gfSeVG6#(hTA{jP1HE~&mg74VH zzIFwB^d+_F_m2Xi$Uk}6qsxcuI(gZ7NoMmlbxF4xBY7OGW#sm_)Arsz^xSrtI_&O# zh3$!a!Se+qXv{!J)3j+tpu-O8lgvO&aKEd%y zSQSKNjv0{xm8L{gd431gLH1vEU?`yGe0=nWqAZ2bLfTCM$xHkwgN<8W((` zW#Gih0e@_LD0B5eSmX^Bf85MnYKa@*UP>+~L4#VKLMs!#NLc=A5|*)s6T(t#nJV4# z30M_-7W%q6RDa4RkNk-w2#Nczold`koAF9H33{HOdlrMoB3bI{X#z=Lt0;+dkvOS8 zQU*bS_(O<%dx@N#7G_!|pSrv~hsERg5Yv$9U%{KdZDM`#Ox1W7*m*+5JXKiv@C4lD zCmHF~g7V65gZFMhmQ~>88Voex^%!lxN94TVXL5w30u5p23F6Q5q#39z$%J8#lzOF? zwdcghmTC-E!;-$^IerGU7(|{IQQz5XrUGMgm-lJb@+Q~`1AftuR#}qUIaq8Q0X`qF zVyc)lw`4_lgs52yA>zwSg%B*Z;&5D@PkfA+)F*8_{LK`Y(sc2qfL7d@NTJm+RG_P4 zBMC_Y2WHg6nU7ZH^$<2|DGiI0q#eXlik5#Ad2s7-EwGevEjX0%YJp7sab>fcQ6`kc^5W?IZ$CsRs-Y+gPo2l?q5a1Ql@-6| z7}lH$@y;>~6@ZfSCo$z<{7tzG4m`3B6PDdxjhmsamP>f#?%G_}MqNLSRCULWAV`ac zX$C6QfQ?~>=4!?YOS6hOHGa#^$mskdVAFBFMpAzcR$!-!ysMJe1QJ*%MrLw_twb@E zJIm6FGw8QeByY;kDy4krPne)99~8>O{WQl2=$A_o!g7zKj-eY9L>v6{fpu5XP%a_F z>pdkz5M)EP%mEKq-cX&2BAm3RI~iFr6_p2Mwc&G*^p2BR=it` zL8MlLf);PUygXE}xc#y(59!e<$5X_G?KoCiYqK(iebxDu5f zlJ@jq#|W^~^}4Lmo!AtgPTRtc|ZSo@n{71vf51rBryfqmLpPGfHpceZ&84H(P0Bd9WIr5|*e1&4J zexzYEu1Y1VH9olMnQJ{7(m;3cll$ z)Rcs?Ws`iP#jlVCZp?G&*C|xiY-z6u56Hx~qrt=&za{Q5kn9SEsqup~7tR3#_r{s7 z_D0$!$pAdubJMt3g~Kh}8MogmZ<0)z!;8hWsm`%m*R$f;aJ5s#9>A@f{)IT!c7aVV zTuB)s4i8obR*`kIE6NBLWso9MDq6bY#2}v1ySfGDFW)dF;<5?c(h6_Cz;W9fsHmM~ zKC?p!Gh?ocU=h_a+%KxKtw$+NPz1L>7*vfXY& zoUa^+zoe6A-9S7KuRKC#ag>gp4_rgNV_GN9(Z52x0*!Td(xx}ZDzfH5{Zi~BOVzHj z;PWX?3ZOv#{#)L#*^Gs&$+rWq?>q8{@jn#Oisk@QD`O=mTdV)7rIiH!wS-Pg_^-9g z@ZWxbdK-;GtLx9+Fy$s;H916FK16sP5{Mlyn)CBzXRTEBOo^ahBG~6B^bm@+c zyW9xVX37Ntc-*yB_TxX^#|)evALrw&Kk8E?2_rLLFrV{_!OW$48e^{cdqp7QSuk)l zQtIfedks(I4ODsI67=+J(Ayr9tPV`j?JfD$9$hn4n*vmmm2KiJuTToh-trbI-VH)k zkHx$35*Nauk*}$kV-@-a{b^pcZQ z8o^s(1wc)e$q{JYJ7(bz?rCD#U4F|dYif4X%6KuN1G3s!lshn-?vL{f#1^c`EPA6t z@=Id*<81D+uU@s3JChHsM5Cc~PjfVw_?_oaxPiP-+SC zeW5=#R~!QzY*D;m8PFYea+dod1grZ9gLda1s?#LK(_Cvc5uB)h4brXi$BDB8yxSH_ z(x=BnR6sb!y?9+IS@Myiu_Ja^`WODdWa7`WLvY;G=j);WWle^Su&AVu&~nuh1gt-V za}a@c`i#uM)b!edXgeC3Y4MRD!NfR|zZu&`c6>fOBp}BYRnxhFtFQ^E_Ob=k$`;W; z&vawAKSR7A_|r!S+e7b?R6;k?W#nYd{C;(Z0;AjNW!Ua#;>qx7GdwUY`FGWZ{C872O}YuH5JUqKgb-P5K1q#_uZP0A6jidk06Bcx+Fl$jZnLJ{ z7YK+4;pc6j4Hhr^1l!q21ZphCZ9VSih3Q}D+l@(C6RGPh?&pl{*UkI2kH_un zKR-_TOd(YWkOZOX7+jHqkpp7F4FrO!_=a}Lq813E{eXTIMdS&Ti5jcznJ@S8i(f=7 z?y->!+UKEC$c(WS>X9?ZcaUTi589K2)$hHwMX(xmkndB28W))M>?R$hp!##@p>Lc` zS~g8u2MMz@J5uwGQEReBe*W@;wu0;T)0wZqp0pKd^bB3j`4~p;YwL%CrUaFyOr8tV zF!3u6bEipM#G+M-2l!pEl)VVhp2T8-<)XuQgz0p{z@sU?N%j2R9mOlpUPBE;ven z(JHI9(WY<(?AJDMJu4B18-T9ek zinY)EZZ8SnAwylR`{bB5weD@45MbT(fI}aQQ;doxkK4~WWF}1hjl(;^2rD1Mrf+?F zkG0Go+7c^$E1s2$nF`8kpK!haaNp)m-4(`18l8jSXGKQ^dm#p(Xlo9w54^Sr+Zmgs zi956$NZmuH5=#un7Gjn*dlG7uC-XN#aGGc^l<%A;E*c+XrjQiwm@^q2yYbeB*X62bW&!s?fv}RV zg+0ee+6GxsWx(p}p`0z!e6|SE=eUJd4({J)%-sv7s)t`b@$}QQtL6uO;AA2wf)TEM zxj^T*-t4P%8BX%y8m?J7=2m>7o7&I&aAUbj>WFYr&JeARcvjNt5e_MMkb@psslv>3n_$2Cl}9t>?R0KKj?{6vNWBPE%$BClR48QW#XzOy zvnm~8*F-5cIPlt11Q3u=iD*!?wtvQxn(Z-7Tv_|slM1eSu+@s%TBU3@Lo+Zh!e zB(HoVlFW%*BI{h};5ukz-sggONSHA52tGdL+gF{84aUk81*tils6 zQF6-zN)8ff>0_wxkBaE2Hn6cY*g(Ci;0>6->(Hs z(k$g4)J8O>)~JC{BKj;|8hl{b~;(8_tDkmun-qpU>V8JvL@blU9 z*~braI78i`8qnZ1lX8o3;*$+L#hZNHy(>MIq^NHfblw!cFhGpGD;E3Kehn@jY5B`D6%FPro<`J93K+T}A+Y0Ga5@Avrk%h~YKqoOz zA)21!^#bcy_mpJotE=Z`4{S?ULk6-wdv@Ls`qiIomN#ht+BPdks_hyYVCnLOqK$4( zTQdeTcr3IBwGjKp;3+)SRdm%UQP6D{N;=wWF^7aCIN>t|IVw{?FW`6vZpyn@74S>v zWIXQJt6|u7Ml$7KZKKgrk?NXN-A_&a@9H`tKm0*P^iOf`~Vc${}k zcG(%3QYqnEjtY#V-I6_&lq4h5X*97mo~RLaaH7J<{_Q$;HKX--L!uStaVsSe0`BqH*KL~BU4TeqAW ziHh+^&DN;wd6pVqfrodO7+vxILy-e=4FvrGNB7`z!Fq@B)?)PV?@v8$N6YCdroj>Mqs5{3JEy5BT!byN z?%=eVTQ--s9c3b`RH-|R^`hkV<`4wkdqn-^I1%G`TCoK~^uzXeco3_@fa>&S_Aon3 zlnW^Umgf?&&<@n~HV)Hh99=0Y$=9h#U#cJe81~qRjPv%M&6ZfHy^dNhM^!teDS{$} z_(a;vMR`h8yKR2lL7jim;e$4e7@^1QHtDRi_$d@9(wdtXjloa)uKvacaD5s>ZW^a~ z_ahVGgH#@w7seM_iqyTJ%$eE~mGH-`RNzb6Fhln+Qwcx=lOq%3$~Q={XjA<0r`nwt z@+KUa4yKpyx=E@<%qXsv$P)U3*@*ZCFi7|j%FgM4HZ0yaO2yX%Y4RhTw_wvd=M^1j z*&)txEyrOA*fU$4$|SsnISaS08k~|M8oPJlu|t$*o>OvG5rwz&6Zf@?W+nWB_{0Oc z7WIP2RqvS5w;ZLX6YqiHNUp^gu-VnuYvwm9Q_s6#Ks}+kH;R?~cajY+S+XAF_xOMN z{~{s#*9-Pv&i>iI5dAOAh3{TQ9{x)Ps(rmq;E==*Ery_z%o;)1ia*y!36nN=x3^hP zeVw>4e7$nF`UTgOz7&rHQOfg96yw+mjoKJSx?gabossi?a^lrQ=FjJsGc=xW1ukTf zR|D6L;9vxtm+Q$e;RNL7KF!2#XhgwF65@DAjc&}X(hy^iOnIB;9J)wdkb20ib{4Hg zt#&8;`W|ZZ;lMDF|F@gbyt<4NnyMd9UAPlbYDCQlU5!;ra$SSuoLFZ>Ep3##-$;(} zu+F4NvO{5(0@|VK#?;flN(@;`;?#IkLr*O5yTr4hb1aCYE^8nyPZp+5PfOiT&wJXc zh&?WZ0h|dqCt&A^1y!-y%!{n<9SAB{Hl*{_{0x$~JB-ZK23WBB>sMZuP5A_crpWYD zgfS6c3MM9#z5B{Jr~f&qMWjVDsAg?$>_~*}MncI}WF!;TAz!*4!9=vJ+A4Uy7&NA$ z_wj9(6^Dv~NN1ha@zc*Ec4%95>XbWDSuCK4xIbkzn_~g-yI3fMui=~hJQrDhs% z(ghdnUXP@kXu`WV?44LV?LL?}fR`v&GmDaS~P_$p9iCR}3cWH%C`qyo^GE zw`fe~icKG9-7V-j4{R1&#;u!x#HwsSn=ikNz)~IJK?65z6NMQeGP|v^IVnqLP_RkycpTe(kPMn{Fa)ofjB}>yc*eAcUv|%oC#supE z)hf;>RU`(~eM1v@*tpHq++((BacjX2^oX-MNhycm7*XqxlMQER=hJq_iDtdoTZx~kszm_(A)S5Kgj2KZV;Y)YVUdET_kR_JsM3In}m68$cV+Ub+EKCy2!!v|GW*-2&|7g%L>mWuEpm6|&X(&y*I7G!69Xzh(8zIJAIL^3&1N<1(7)-I zKIf$vV7YQ+BhjY^1-$S!dC<*MuYnOBdd<7;-i;l>;6-EIje3$hPU`D0)>ia)q~9#Y z%?MpW19;M>vR>IW_Q6EeRDxi!ajVBJ9wCSA!`sK#qCTlTu#>58W&KQ<%q4z;X;WGC z9=6{u`8g3#cJCraZDtB7=_ZS3jR_H;K)>J4!0-bvx*}wlL7RbQKk>H*hKe&tksUbb ze2N%DAj|5P(F?5NXI^=K@)|#YSW$tZ^m^hUClYJ0q~C zFZgAoZ?i?4jzUtU{)YEuW#F&I3vWh^ZJg5&`T=SLNVMb@YWe}gU77*uQG^I=f+#gi z1L zsN@DvfiJ}0%>$vYh#+H9l0oC(ijY(ob~K6=uf6)ZWwaKGj<|)Tbd+ z1ffM%P3qK;%78Up{37bqf}2;-z-bclTBEW(qdt4w%RNdKLVpO+sp zKBgtARWBPqz(m~Svzi$%Ni&M<8B zL>F>#>I7>~cBq<}CO8)3Dy(&MCBv~JWzrcfmsXqL&cS3RmlUC<&KB@MI5@mYrL+>H z=`E~;+;rr;E!MPSv}05feVZo`CfG15m(C5?1c|n%I<5Vk3r*J)Q%o#i!|&=aPOfpo$RUd9WvDjHU8d_VRiGE9HP-1d>nH_ z35ljQ3lUQ_PjEP-#Oe0FKY+H20dc4qgT1R!_Bf=O3cizFAp(?TGimZF$sk~>Jt|sB z+-N0clqs_-H6bwW@8r}ADHoi`0u{x-LLeB4i9F(uRwpc^nre;JTM%RE7R-f>=y0q} z+b?BV%-Or9WH<+EN~VC?@8-ou?Qnqv|IB2s#TZ#(|lX4{R#=$&`wAl{U;rs zNEA$!_o%D4wKT5|OkoyGHSnQFd-uvWmogfb{moqEiXY(6AD`$E+YEvX)#?gK;F?RWnzw8*Y{M888y zz_EOoA=e|K$*%FlpdD2&Weaq*0g8@bN--6MjMHqWWzNr`C*s$H!h|XhVHA{3^xWu` zU(X0BBCOEQgs0=38NS(sXx$_5Y6++#EX?>bk4z{`|A(=2V9%`KvUJ6^t%_~CVkb{* z+jhlP#jM!2ZQHhOPTuamrf0sM?rVO-+2@>%wbm`M3Yck!IP@?CiDH=bzAdD^R;<6z zCA9B@%sjTGIEV{BIJQ-ZZo38tJ(E> zyA=_LOnhMWb=6ErP=fTr`qB^L3D^@1KIsb)kg?g(A5nD7#Yjwu5|EWayv0?WQJ#w1 zs6ONxX|t2DqbDuS8XuwJEW9yHMEOYD9=0MJ@zr23dnha(XW61+l^zDDDQGt!9MCx# z+E)}+*RqKr9HM%H%iapdI5asOZ>+(d@V)f?Y-QS-39(2sKBNPF2QJT%^J{Pln^e#` z8)rwv8e;s;p}95Ypvq{0R+eb88GwzpJ@u`|r0W`RY7c^LpZIWPM7dHpeN*$}mp|2o z{6ug&B+&|qu@BhSFH&=hWO^Ed^)-w?=q14pYIrd{uhZreEe-{))9ZRo_~V{K6CFGfU+;)>d)E zCddkTlKFsmJ)HiUZKjh0i!u>@`lX2*4~l zIuD%>ZtM^EF;O0J!J)?F8UG8+bf*=(b`kBf!iYgGM7te>+l@$WJc9Pv`<}hPQH&eX zS$L8y`&t-XSr4k7FSJUn*EzbAl8<|(=&XIukOxB>$DsS2_t()d-W2|s2i|cB4Q1kB=v%9-5xKoErdyqY(ti!rgtbxN%YcMH+P zMwD^FT*RtPWckCiR1gdc28~l6Qs1~cl6&DIwlrF+_BkaRxFQ(;d|VsUono*poUTH^ zUB=4Ld3&n<#sEWekmu`1rW1510}o*)!#*}_GKHuF?%uvf!st;dolUn_3lheNj2d0j zHnRgx+P+J9JoipK{f+5rGE)3owR)#NT?-8RD7#(6y3Cgko^HJDuFj>mSZ5H#{@OCx z(>8>u8aXE`ZT<3ip5f4N|`z147_J$g4c>+rwXNq{ichwMx81oR@GCbi8*|?+e-eSX#p1CEx z5A7$r32*j8+*%^NM8U>4<;!mt(p`>m|ys2r1!pPK~6kJb>eLbD#}sg|u;H!;l(+mh0>F_ykw zuMm2<-pi3nTQM%CS>Kl*kIb0w-WsR{h4obQLLh#}|3kHh^p(EZsC|j!0J}Wz+z>)& zk#fdqS0KgXk7{%L#HZ^GujyuR*8xw~y|WwUnMCT?g0kI+kxw4s)4wi(h?7>3)xY(h z@!zs7^8cS5?SGsARV@LiM_4|5%im`O1%(^nw9s&zh1gDizL7bAZ&4)?{r9`14FzV_v09dD~1 zUrfN&^pNB@LtbpI4&1k{I=tSRzB*sfe>*>0f-3O*M@`tlAyWI^ z79WJ+`R8ThzBu6a69sXD5aGh_iGyzSH|@W!L3ak^-9GO)cwr3fAH^Z!(+&~ypa;G+ z9R%llX??rZ0^fPC29*@Mp9;c@&~TG;WA;)RdY=^k%+;(Jdi?pNmjzPWI%#$fvO>0ka0>OEAahgmcI#in5`}rBEV+Gb5*W_x1x_WTQ`U zi*oTrCxOPvk|ZuWuOBr4R?D~E3%wll!ATdFbmc0L>NwWr(jbk%C>~L5pA)~BDW|!T z-G01$EVK?=^w!3EOLJ{tb8fg?+dQP@c5Ng@29_f0!#_%@*h4G=hVuy#w;Z=WO$-6! zu2}Ih>Gdf8lnlT?WnNA~pJNA}>AfVGtFV71PTC3ItiY|BLEV=e~a!ooZTN*{Og3itM18r7ggq;(=dKv#`YQ z#x&~D4;C1VsOF9Dw`?5fNk`~_KZ=E%oSV886g#zR1HJ4I>9WV>1?GG#7~y#cEkfPc zhHyl2>D=QL=9rk*Qbwe5~owUs=l|67%uGOEs% zde@ViVI|f{a%!H)6QL#Qj*yv?mGv{bNF- zPo;j~uiPW0Mrenuz^O^GT#K#@MyGENl|;F^Xq;Hgh3&3?J;+~dFftw0r3xlmT~Bq5 zA3eNcEnJ;Hx)xG1uFM&zQ#q9*`Wd6=o^1tO2iZ=a07*k3=pYwTULcCZ{Y$h$nlz3D7~ew(O+lu=T6Sd%_~(1NK6SZz2?!` zZ3dFNU15N@+5_=AuBw`wsHzzmy)pFimuvCG`*K-|kf%R%T>tIcvgfQoB*v&qa(j>a&Om}e5vilEdMrTm`Rf)0}45NZDkxp%|pv}4cHy;V$>MfknVM3qAs zi}>}*R0EHE0rkS1a8)3f%RT8jS=StRbWJB$6?f@9>)l<4An|~7@;UP2eCa#>A;
F{ltBsSp2X*t}(tCfZcfMn# z=xuyU$!yEFI)oD|w&xh4{9YzTlTDO)CcX`$C!C>EGE)bC;DhH+4^m17)B{*5V*gTs z9uxm>Z;_OueSP~wh@6-4sCGtmZlM}d&ndZ~%f#f^q031R^*~TC($u1?>;gJoL`=~U zbM_UO{8+`I&yoZKXwuAdAzwlFIA1gj1=l3T;A4^g$=v;M%;a5SOTs6zNmz^zp!I{r z1I1f0ZO1XrL3hPLd)kD5?N0Up731&U_f#a9|3RVjZx`f$i+K3ofUF=DO)C^JRNnYy z9e(s)5+Ga(%$5+KQ{r{&w4>)I#P#uk@%r3@PlqfWwCW0~yuyPrh7xhYhGZLbVO(A%Ld>2- z5pMPaapKr$*)ZH|}0B0H^;8<@c-W^{*=;W6asej75Fk5VcV3sFhfo@dJ zqtEK^sIBUe(I+aR=R8;o!&p&%fN)k*s}LGx9*sVx(U=!wJSJMxtVHdERBxrJvB?;H z))GDs9hjF&vT9GO;v8u}W1cvgR>7IHu^>1&5j+5E%?i;MUf`)je&?~WQyEH%>sJ6b z2M-t%$;8K>XjFxv;bv%4RbW3ModuQw2cyrcR{{)g9E^zTDHuirhqu9aM`~42z0~vX zu1=tKC>Lot*G5-Z7SM&=_P=CO@$>Xe!0j|m^m;(-0ITQeS;#Js^(GA#ac~Fn?CLF+ zuDW|C6S>oP)#~8<%nxT(E8As&1rr0{iYSOc6#`3%U=0D5i;L~#jokvcpq${-9FmcS z!uQ<~;R_TwnQZ0(moF0f~vRj_*Tfzl17EEPx5F0&UMqWgHYD7kHFv<+8cNISrb zNWz;^u&eaqrM&nfKH7cX^!M;Mn2{{jK=x}bE3tV!zR*1u4L_<&ApVM3I|R8~OezETXf;ohH|G84U$m7R*$ z6WNz$U0c0=q9Dq(M?#uppY+y)i#dv;11r;cKFm(%5M zjoCLr9g>{Cj5fRPW~thoA)!%L6$8PuJsV zVR%M**4)0AcM>fcGIxvFt|L;o!1`8FvGP~4mR0j>&eD@Rn1}l1Pvo#F{%Q?!83+HE zGK$I_{XXjuqohn^-S+v#JJhb&Qb9o8l(Ojbh(ZOY9^PWx?S_nJ~cdkiq8AEtL)3e{*L0m6tuy z9Mc%uFVThcuh7Ob1qjczUBPJeZPzg#wYPFvY;T%g*1kpBbmx|Lw&4)He$_`*hpa@% z^T=^3;A_Ps3lRNp6FM zi1?RU(|lnCtht)rOdb-v9y;r~F^_Z2!>*M0UbDER>B1}_0qhoST7yr0KeNK5_1*W8 z-)Y7)vZ@HS_8_Dke6TR={1B`R~hqG!ifJ2nzFftdP&KC>?khcjlRj+JJPAd90&Nntb{-V~1(V)h6 zy9Xez7cOdCXrc{=cF$GxZClOFUY*UDXjPY1nR z)#%|#dJx0&2CCB^hBUgcFm^VUtVo zbERxtMFv}2w)m#{NY?xUm{x*Gegtw_Flx`bLo=;0^ASegEN0O?%8A+CfS~U&Hiyv4 z{IJ;O;pq0)6>mZ2%r4lN_>gB*JLw$ApqXeLbYbQ8V(CfdLieB>9o|$ygLFH8QF>Y; ztL+lGixUc&-u@f(2@4w6WK5>$V09~+Tb{Krw3lZcWiI_OE0c$=zSl2yh=w>8u{6&* z35SK@l&#C`DQxEA)8N;9D>@I0;Pt{|=VsM-H0>l6e9pLZPM2fw*h4D}PDUeR2%5R! zNUCPa)k8x21G9B3v^7qA#c3{fEin8&liWn}#Oe(5)Ug-}vl7mtBE5X2R+{TAYix8Z z`UtLwNGmUbl1Gektscfd?3&e*&xv!;#BWn%4;cf;;}AFCT{7BIcntSW;X0ZcVX4F8 z0^oNeRqbPuwTRMw?2`gEo5bWdypSD6GqCG^=u`l zhpGIen$#C6X|UvVdz=&V5Or|llXGa3VE1j%8eCg?ND?7~_22P;H^q1MP~IC37J2j@ zTE)M>Yr&XH2pG<>)f8sgpz_1<&y{OX+5qdpyi~0MV9Bh*W@sH8<-}3!CEIq0{K_}X z?>wR3F^Qz#xQ6eX#GVPqFuuzx`5W}fI!B=a+v~<~!Sx*Cusgwj;0f2BXujPbRG*>z zCC|)U%HOO*WqaJd@xT?xRl~H^z(55=U!@z@jPLtnd*548T7bN_*Z{wMQ@HPrl8~X6 z2)t`ab@vK0lRY|$L8v3uWlsmyWgj_!CseM~P0fXe__gw_D_LYnW;7ke{0g4~ z+e$p+dq&*1`?|nuWGdxgfwVMD!*#t=SXhY_HuqDUgrSQ-KAlM|0!q!Va9(h`|?$9qJ>ly7#e>^rbCyRvQfo-iphei7C#J7QRl+ zL!Dv9<2~0FC8&!xoGobglwN9n7sp0l9k=DZ74M8E%c?)&DBCV>Hnc_}jeN0&QVJ5@ zo*Vu;La%{csOHpD$6wcblJ9sAY}Ws+O~vu!FoxB)>i1gr2fj~Gf%lGTR|VGz3^Yn>yscT2Ql1nak6XJ4Y_Hxqf)Y5GId@Ia7A{~L*$P&K$c1eQFE!@_mPJeq4eyFNlYdRxnS`g@ zgOjp^vBnYfGPw&jtJV4VSIr6<#+x#3BBF5GE1_e6J~^@xpJ?<+!9^d5RY!KCQEAKO zlV6-4H;+6}m%Cv#I_OxKKebYfI(i$VKf93b2r>9t5H>c17;S>}hD8Lw3^ip@WcWwH5T(MdG6v60NMQ2oiNiv)9GN3Zz;$F7DTz&Bur>6E=}XfEPbz}@^TTb z6&4G=U8;x9_UTkP+0G67j-#`2>u_yK}AB62ti-r%~)$ ze-s7t;GKidppIXBg8SG+|7!2FLd((Lx7wL8|7t(&hxKwyjDvJ9r&6srIJT%+kTMVR zh`L;mwc_-~b=2A=#JfTI3I)t^0EzL*F@3?F1l4iL%6ZAASM8(;x!)j%$#w#aLtup) zGoiXxQa6(4J?c|dV_XtfnItTT4N{uS-0#dnX5$kU&vp*&EYA(4v~^6g2fanXNd>IN zmD;>CNikM?I!ng`((}@eHe5Snq^q0kRbUekhWO8vs0AKhm67~4t$0z;#?WHL0FT1) z$~fpYbdJF$zcIITO&{%iHZXbsQD#{{{%~I|(0~gm=$6!2AbJSx|@XWHmwG zkap+Bf&G4&%cXE+UIFrG_Gv0pu^kkZmcK{7Q8jI4kI?tdc49$%DlwHBz_euTq-0RX*@tZ7#57ta#R3*LmETE`3&Vb75}d~YEf~gg|%;Mnd5gDHPwH! zqX(E<8#_AbTig9#ZA_GkrlXQF>ZgkhxkQY$xVhBWfPtkIUul{*0(ET{?lqPQq2k$Yr?#Ccm$Lf$OviegQy z>tQ5*)C$sN?xJu6jXQLF7JWm_MLnU0=mQkeaEgQq@s-BQO0yN}vIi!7)P!^}?TnWI zi!6nkLfAICJ~J<1V8Jy?sCx|DrUb@PQF~dFaM>2=3s^Zxm4XCHjshhotI;>njmO&A z#?d5lZ9k(OznkgdW$;8Yp5)f~2zM2ACP2&Ra3p?ueDK+XWZ#~Th5)^U?H?)>xI{gR z-;z1@LqDIR{gvZX&kod2WU=cS0Yn+Zb&M$}Pcr($>2(&sAu0it8U>N&ZlWf)%5LVe zi)D>Je|9vI8BrwSr4VBRThv)3qqZ{=;MYjA*Q-Jamze8%$`U%Pma1*4la5iVmn+gA z(U_-$lH3BX%!>-KN#4T6krTM5LX}B}s`tfn4$PaV(ELM z)chWR^T$V<<7W;J34*g*nRiv@-yGB6x4rx_#bwq_JzV{D6(R^KthjR(y=| zw4%OV5V*g`)e4L9GdsT@)@smLxn65jvrTzVYg&!d!!tYWi)_+A(1y;csK!3gH9xrc zg39v1<%G3|guM!P;-ZJ1wYIyzPMG0zkDrlsYq(9o4PD)f*y={v=;Ca9xEt_G1eWRj z1T5lpMiHzukSZ1e!+N13!Ugajt+nmI@6`4D(?N064y0D5Ro^tANZ7Axx1ha@iA5Wz zs0+JiD)4N|BP^(<+(o({=S4*<&fPyi2DI|exXSI9PN$bl2!;*zR2r%Zf@9(4)`c>w zx_g#}#;`=D>-PqpZRr^&kta>hC(>T?a5sP^_o1AYC=38nU)ls zWjI3S(QZvQiNB8gc>RH*%`chikw)cS*+X@`FvK`Sz4W02a36w!qjxb)1>>9222GLn zlUGmrA`oF{&p1r&&INsuRDD9End>XHPaOh)cbR zLVE?(9uimhqy%@h%T$reY2S@QG>;7qIc z@#q*@4_`tN_ikWn2Cut<9EtI3N)G0iTeR;_o4k%x({Ckk);%}HAkSK=+#f9GJ(jm~ z*g77m5l)|}IXbB^Mrp$gIo+fyb@;8@ch7xq8yki*^LB|b=JRf%^}Q9y7jhKZ&aBxv zpRPeU#!=;N-OM7LY2Bz9*U&iL*RJiml)eFzh3ID&0Woq9pP^{eWAR5mVYeRJghZ9) zv0sl04kPp?pSu53+>k^gva|KofKu#}q@K;`YL)N}=N zhos1T?loS5014;OG6W7CNkfZ_NV&Fd$f<6iyMAHyc4Z+DOkd3=n&M10W&yKCBk#?Xp zG6!>n7fa$oGPp#NB$S6vXy;N2pDLy)2+4~Z%3HncGS;PBHZ-{ma0Uy8W=qy7XI7xI zVN79-+rq>WZo#FNEtJKwHk5OYT%3su1Z%j@0^gsr-8qfFWq~KYG%p^VTc%=@?|>~A zdFw7bz@}kEB?lA#>!<1^QTxevgrh{uGCFS6yl0;aD;QF+Vw1I$FFCBd6oD~$Xmq;i z*=IL{mA~t(DxR@Xn>ld-@7gS95S?1tqfiNG)M(D&q)C${bz+(za?5SUZHY7`-gEhc zNy>YMSsqtyC4zFfMyJ*S7sIa8ctm};oWz(aov^RAC04~7akTMxb;gF&lIVPv!<>=M zlBlR>Z(^B*=E}$4OXwMU4YGwoB-*v{(XY{E&;x2mo(-9T0onIE7DiZO@<|5A=uY}o zK|}|%>#8`(hs6L{id-9|JaZOyWTX##iz zh%vLnEchsC!ko(KfvX4h7O*sbc|MjPjXAane?obU(t_Whvx@2d z;HVuN>xG(ZZT5~$1mOhCe8bQF%zP!xe4MZQu*GvsE~sujSYH%Fj;_M#h2~)RE?1W? z&mWVVPwRDWQ(eCAVB3&<2qb|uh;(8M(M>`r@^Mjwp%Q4S0IjHP>vj~s@C>h3GTSBP z_Ojg$$i^K}$g8o30K^onB4V?+EDuQcmtkdTv1$l*;^i0|DrhPbR8Sao8(C_Ol$uQc zK}qSqI~pStX6|GuY$r+5K3ONdF6a=bEEvt=n#MgenbgP9PW(|ydi|5}508=hH;hdj z1DO(R#(#b$)i|5z7)KLrQTP5nOEIqP%t|$Ol&))@u9P-{EF7s<&*Fx5dBcuZ2ea-s zI&c_NiVe%9Hmg{WY@r=l+)pktX`vIJ*+$&^q?!n{gvuM9qCIZ^3A^@Z(x|p#WMml{8jGN#j8?Azj3b3Ni47J3rHcMIEE>_XNedagPlte=L==q5 zLe^TP$R3In(A*18@3_N$YC|6i#l~-dAvLZKSeN`IJT=$^-%El3cWM&E_oqQy?C1!t zMYUo9FY^y7^rNx{sr3Ywae#*Fa;;U5kzs=JxFXr!WrDf9nCM*6zy~I$hgea|K<1Jd_?73PNrIYKZU}Ca8|f3mmNj3j`k9#_Y`P$Nx1ZG+9$^BSL-=+5!XkD z$_!6JcMaEb>8c#B&2P19j41bdqj5GlP{C2|senq|WcN_O!aU6XGM3X!-8rxp_-jR- zpw8JU>aD_#t+`0-!DzV}|5&G*rdbFtXX&}CoMg6&<6do6xZa+;H9t%<(Ee8YDR?d_ zdzEfK-#z4pPoZYGO@eD-NV(L|xXpakPRk}XlN2LrhehSpw{1FYC{M*64nC%F3hc!| zE;bmx4Vu@y5`BYmynNVd5&)<57teQJPMLa@)`#k*E^KpwI&ZC)S{PfW z$W1`*k@ic>3hsivvRV+ITy+V(;B@t4U*_E?(g-|BPZ@g6yFMNJ3jX)6g0Y)XPblgOAJ6r1g)}Xc05k zP}XD1FWJ1$h&fx??~(C&BVuMR=yChP^^p&}#gAS&yArY^#^+=k_2?jiq9hM`& z(wN$&e+MKCfV#|nP#8PGD&UI2l8@}79&rE0hs`Ar#vD9r{>^@TWrsl3)Td5xtwRWgMKr`t zBs~u-yVDu$a;T*>olsIMum6P%q&Z5TCjZTVAN%e&X#S_~x1zr5e*tVOj!6nLBJvtq z8lJs1%K2 z;|1a9FYIpb_V3Q^M%FJGa-Ty`oOC*>6tC!{DJM;EdAfvZ(9|`O{EJftRYq`tBQ6tWL`nk*emc_cY8g+zYl^Znbc>f=<`2X~Qg$BrSL*GxP@B7L8KmM%d)^=9(^5&-hK%$M^oPPcX z_}yEv$L8B{>wU&yZve{(mR1i6Igk$pQJy6zo@ieHT~s2zCMAbsdSKPiKr%$R6S55} z4oNhRu$_lk2ZRJwlIUW3I5F8iGd?+`yG0LlzMUH7%wkbJT^`qvvr`!KhFwp?Q$206 zd>edage3Rsb!zOsdqWvR-kUz!4}(F>=fRH#J9xq;{9s&eL=~Y5TVQh)*aKV3$h%0A z8XoZSBEB!?l|NjEZp^4k&XbJ&<&166GaWwp4EBU3hw?>;Gr2dCg=&F&=$_Q2R~@p# z6El!C5A`6N3aYLVDsrP`8b>Z|m^fjSb}zy>GKh|^x$JNaniYU;1tB+MUmZo)ceoIQ zdq1sjrquTN*TvgnDfvk0pvuC78&=QmaSpKWw_BH*!q`l`+H)-357yUr=8lQ|1y+8j z(Xwq(X|B1kY3?L~IZ=QM=f4rEzn6wdJvPL2VDZ9zuifx@xwCftV2{{pt>H&i0qU6U zA-ziM1plOH70!0lv_WczBzgZUnF=jD;#hV4U zK@v6|xdU?kbGC2!$HhK<|B=5eko6DvB_fF4F8+v^ll5o=i$hr#(0?{r|CEC5gI_z~ zn`BJL^W(?Q|N4^tODX(&i&dZj<%Rn2;A>mkR<{Bh%dL(~Xb_u06ABI);s6RoMnsAW z&ZAZ_}Q?y+Lx!IS<#T(=$~tiMEV=EF(>M>vTD6d)d^)xYV zLzWJeyC7%cbH}~u{jqsF4B@&d^NTs*p#^t8it6cEd?z->Y0=xsFOqIshVSyAQ)@g( zdGkC^3w~q>J&}ZOc09%hOT-veiSsfK*zwiM{pWsMf|w*e$n7#!jN^Wd4F1}IePuF# z=Fs1ryY_cq9lotG47Zm$olg0KDfbs*0_D4-1N$1D&g}#GS2Gx2i39q#VjakeBdLA+ zlHaY;!!10?m4{W&-OFfHz6)Z^FOu$Bq%l<^l~rY44arqEH|) zjiXyPGZOFG^uBU|4js5{J!4x_dj!`}R9DwCikp-%>7d;c#C6)Y&7{e;2L#ae{G$g1 zcQ*7}hX=fTotam|dmTK*w|py&TVoKQpPBAnvA)DHZTt33$p|BF_$!XK_*+9yV|zeI z;{j9zkGH~ntKHlOZ~BS8rMDz}tCKOW6#X$!OEE_`Q_oQD?^*}s#ut={e^p*EbUq7` zeGd+F-;NDFqj-F%IkFU9)D!>S-ClXsC;C?166}u?D8oNL%e|Yvxak~p74v8wbd|pQ zC+ptdO7Z!mejOgD0(_JSFHps+pmM6lj&N>YTobndF9dw8u#G=R-<9&;=;^gd0764w zkGGmsT@|<7(*5}jqIn($_CHgkPbCW9^Rpl!NG$JvG2up3+GzLokEIUMM2p0i=&>}A zVPZ{h7<~>&bH=IBD=BNz%-}!{2Jm`&WRov$#cTkT-v{S)_%*6sX^~ysqD4%esx5E! z0+t&&n^`eKL2H&1CH1DX=)XeDwLHeuGpRL`8nX@$zWn)8&$U)s4UEr9ZRopT#9!Gu zOcx%TC-*|MZyiysSpULUw0 zx&q|DRI04f+kUJ$3Z?%r^dB(@U_pfhQ|d9IN`ibm^&`b}O*Q1ij0@NcH0h2J z(ac4Jw#(wdF7DK$K#zc6f^gas&{H3?gZyz3L2h8r8wVY+D{0N+WO7(TO}`QkIiw`j zhqR({FT(OWsiV<3^k?Rr0v!v$|yrSDgWl2W+_$EY18PW(b)w?uSws++6Q3;^qye zIE|tg1yDBlMW5b7*>s`6ylsJ>V0X52O{YkO4%|J1pTBFET+JL!8IU4>88J8YiuTPU zS>K2=;0`=-nStq~aZzJJLBPpvAV&{9$gApdY+^}2=#MhfyJ_v@rTJDTCObS4bd{wOPrP*)I^1nTu%gw%b;h&TG?Vx_|@^M z2F3$X!DB5U+Ls461m&}+Qe)KoP}Lw>UFAf%6*w|hUH7M-GUXqAy|V^j-&!fd~QSI|-7))11fL7QxcDyN&-gTiwu z3W=3ch5x0r-95gWgIEut;csvA^Wo!mISq> z;<@}WB_3Y6Ps~p+vif|I#*s=iEwp0#=5gCiQa3VR;R;p8T1hl@2^}UlQhr!pzJ)V| zp-)=&R|ulxo+4zL!XRsGs5r6aUp#|b()VADr{ElOc}2KzCb*dyyh}(@v{7)sgdbz| zLmIf!a6_f;#jFY6Dn{!`;~0d1gy+EE-FO&gWp=EkC7ZEb%uCkK8C$*cSIJXghVIv0O~U9n8PP+QQ#rbRE_}eMVcd% zw6ZLFMXqto=Rj;%h?{3@T!>a%5|Gnv%Q}sKQ_9+jD|~kF42JDT!WyHH=bIeaR-NCmP7fR(ro{g0IW}hR66>z4nW50i;R+#q#FA)6_Ry+Q?~>_8uzC)bjJqfXS!%0In+y z@)C)h%0iS|Pj3kRsy!>z8RsT-lxc3fIj5|jg-G|{iS_fhjekOB^USgSYY9)*^s{gf zAc$2&!GYx!-rIElR=6#U!+Ht)|LF0(S&7jX+4PB`^=%b3xn;hIW3;n0g9AR~d& zJvvc23NC3VmjbhHN7yijZW-3`oO=Wh;2=9}5gawH6S zm7D$u>QPSk2?!{UvJO;5VmzZ^f2d`nS4P`}WujCdr$lye|JnMV21 zz#r;Q#A5Aewa0QMR7i=ED3PW(z;Jvc6+lP{NDTkJ>Xw#&-*y3xiKV!Cl<|p%3ciJP zotUl1#}YCnDurgF_M?EPtRf5+x?Kb1l9>dJ0$HU~G-YLMmI8%Ta8ET#b{4a8=)<3G z0McyAxf2QH)EvqoYUWfE?_Tbd!6FA7X>VtMK}rr$Q=R z0PZT4D~*+xQfl7J+r3{Yi{+@6aaQ5DD!SRIgaJ=l2UlPa&gxu3d}5G!K>ecC0{L7& zNfQ=}aa^q$q*g8^?Wm$Mr-)J#RAR-b!~Ulg_fmoq+IHn@@h)=7;jyID`WRv$Vc;iK z-7?w&=7IErile6TyjjW5GPVKaMd))gdn-cZ(%yv;u0lhmE(=1+p$R%sMc78XtYRk$ zYL?3FQKxcj9fsc1VDW}alg5GE0l}6WfPik>&E1StIx=xPsh~Yb%mM3#xp^hSLnY(kkeJt>5cK@ybfd#&El_wdjH{IHRJJ zKUuPRhr*zGre&2{59+Xz`(jj688e;sAz7T9QW+U@X=W3gk&mx(JY^J%QmYGr7VD!z z%S-G~D>YLah>cc~BIa5zn7Gb%9A!$B@m4;|OoumU8fR#-I@HYijBzbS`ixA8ais#4 z)^9Afrv@3>pJNPUqJjgZ}AU%5~O(pHDQfxhm~j8)Y~he0|O3E6LX? zIs@N{8R|?x6INqBsy(9YSaL@@?>}rn(F(_)^EBf26TliA)^x z5c3#G`LTWpk2Ek0C}2mQH>rz z7YY$eb7D`4f+)^33pA4;gYtSpd%!wl@a)rvFNd%X3IQ4>&O9{ntHR8`-N)M?jvHpx#VG+?8mRR@yxtB8GDnWGXD}G+}Uwa57OpcM=uImR? z$CMy7I2fx{SmVI|IQPqDu{c+^_nbY>IO+#id<+I97Gp66A8l!@H%t$P8a$vz#>2AX z&)xk!1s0#WcygQ9!ijG^)YLO$fa{5tTR0=xLz0UmVI$leN1!O=@e4yNmAUsLI@*9J z_YNh?`H?x#kJn^b&OlVRi@#BM>NJhRZCgHsiVmD{P5(^+*pAd54k$xsAGJZ74d_ygAE^N)eMg}#r7T*`3AvK=+>n~g!^{6A~HGvibdxIvs{s*r*8C# z2!1z93s(g<&TyV3LMl{emxR7MfgmgjISqquf2_5PaR>}?b=G1{2yh~%beH%}CIKH?zY*TN^gY~Y zKun}xGHM_H`@MAh2P7wq7e$(_^aK(WF^M}6=l)=!n z)hk*GfZ>z1T!wM@i@7E01LHgg3-Re$kjp1+**HWH3d`!PqJ}^tYXwe|t7qQwfG|_d zv@`v2mZp62k}jj}ZyKX&P0ZnCfjf6eqky+*)`0ttmz}kN`j5jkgjtCVIJ8qft`~W> zBkp2sJ|;XJ9+=KH$gDQ-+^gW+$=?Wt+fhe-PqU|7APz5ZZieOiGqyb~)1H0>y{6aX ze9p+9SrZhp`#;0e4Y8a(g?EIGE#uU*jr#~`HzkzuFg@q`>49z@3RmIfBY-DJ@)bFi ze^g7n{!+YY4{5ji@=}xH^r``t)rAWA3l~>}rX2S5H*X8e|8IOv9p}9#Nx8fJh%du= zwrL#A`zs&s?Uennim#Oe~<}R`3XFGblFHeH*W za`s#_PKTVg}v1#HhIp{Ys0yG%lcMCamn^9r@i z?o-l_Qi2`#iJU=@`XnR_2SyIKf()1|n>qX5?1^p6h|`@1bC{0=a34+zFP?fmU0y?BCIIYgw( zJzS5#jo@WDi2cOS8hVgI2*gPiT!n#8uRf%X-jml#;jYAA;f{Q$U(mj800B+J2EXOY#uxH}N6f`+MADN43fWd!>m`07f*0pK^IW zQf8oOi!`)FC$(WsVoI1={8_I-=G+M%>`@->YMmg|=AF5dYz{Gn$n3?it}I`}s;; zRNX81w7B0qeq$)ujQEfg@}?wmHQ# z*z%K@9-2DBDiNrTyXwnm4k250!=R4bP$>5~Lu9B%kb6c=pQ>Tn!Zkd~?^ME; zBaF~XE`@_JG_U&!Y)@Jyf9@O++Q0lk9dV=%BdQrPWB3^Zt;vJ3hBt)!(o1d=umlcP;_*OA8*Wc=nHN)7pjMnxJB=#I>NU;mK7`7mUpg;VRboRZk5`=fM=T#?k*qT zF$kDQY=M)H*-&70(%BbXe3=k{bbiwjDTJFV_Fmu;zOj!dd{G=I881zxE0O$!R3Uh! z-KGM7>WL6^u-z?!i9zu{2>uZnXxj?CKQaw;rX@`MH7;g9z(Bzt0YJ)M{eZV1$r|+$ zW=WxuIU19rS1r_oD@81*63F5f_qLYTS6AFRLANr!t|{6?{0*|pnc%yOYq2gD7d^BY zpHjlo4yhd_bJ>t4Bf9XI*b+xZY<~5A{R+I?Al-`D+=QAP6(ZnQY2_@n3duM; zA67c@7WJeZu9|B=U45;}Fo$hv#yZ@O50$n5gM?UCACS)^{g77KkD_~!?C`ZdnQ{ICw-#`;39zaI&WQ*A5DD=k z+>IK?7C$Tlakbxgj+Yj8Y%njHA~4JuC3nz~HiCt2(J{lPFW0+iUck8>)|DRe?y2QA z41&CIcai3G)Bfi}?W6R1c;@MY^#-p-E$K%+5Zb{6Y+42_5LCOsYXWWRHkEoSa(WZd zL$+)AwL@f>RejuSIKIb}9S3a)+b+o--N$yMzTOgo$AdWZ&mE#S$cbSOZztmyyA(p^ zq*z_|s1)z4bz@wK!(Rk^-B5mAVoJ|H|0t0=(j-pA!KMRI_2*JItvD-m5=3Hq)(+nF zR2h4l0NM1aSrDj_=Ev*?s0Y7033>aUZ02}f;0uv1}{#ITT_$lNRITnQ@ zn=met56?Ok1)EYnFiQb)VT%x48!<4lrt3J-T4XitHJn=h8j9xG=uSvk5%NJ;HC`)9 z3G*aRk)?9dVdXix_S;M?xga(0L{KJjuv)h>t!2l??FB~4%8nujNl|B+&!uC{`q8u0 zr`{%#uuhs|gr+Y7>+1$vsA4Ekztl$+_2;QGwiS0^^UEHvMpP(8nT9!)=4SBRCt(31 zIRuLz_K&vy{7okbXjHDq6g3&+#PWA4Csgy=%&(fGtW{71alv9`6y5-fNJL?u4f9{& zo{ir*a)Asrcd*Kbh?APV;LZ!~f|73!p4`aVaX@L&p0v!w&L`G7_qGeYCP#VE)ZP?1 zFX7NtlB^rVwXaZUchBnIp{j!$#(}Fon$K!!Am>u$Y=W$Akyxtr`>tGkMNM#0Z5CPE6h)gu_!8?F3rr;P zz6j~S^Xp$a*K*3957)mlAQ1?D{9ylouu_K304pbe4c&j{154N#o4EbcUM_8GU~J;> zA52xFy1SRQn)7FLvOAO9bVf(C(J<1mOM^zjN_2q9aJ%t%GKpBcNd|FhNgCP7V1avr z2KKmhUXfoD{|T^+czQf2W(f)Z?m(Rt=rHo>OkfkTW;(K+ptaisKZ|(MJ8$UA0cjuWhi=)Rl+`g8OZapDEIbdyW=2ViwOGm|eNSe5(B> zNYgtJ7LQ~qEtu2gJD@-H7+dd3z&xYY{Q5+}={4I)~6UQeHrVxMt9aJ7x306O~OD3q7X&Ax2C3S>J zGc)W~RvY#5idc&W4W|Caf~Zyd5s`2Zi@Qw92-MP!heU+FqX@LCSdD{nJ1UK0@aGPT z-U?jCqG8x)Q`y*$Ntk8>m$$A+-wLdCzf1*2WQMX5Xl(irl$$pVa%u`7GD>t-nf-p% z*kR{R*6-1@8dp#q41oB7q{wQ*({>RbQ1Bq@&q&OqZva-2vgf+LORkU@D zU&yVkYNsz((?K+tFut^8H5P2PD)+5+Bq1M`Y<~J#WtbNCY$6vAMI$SvrKaq9n{u!calvUH%%uU*$Z0ItnOxiBW zTF2xXrl6JZJC+KT@UX3%6}DgKez}iI=MT%aTV!ojjKH>F^ahUtw+566H?SR%*4Z$& zt)J98hFO(+NCzESW)M!vO3%Gfk7bUd7Q&ihAGx&~&C6C*r3nm%-`q9`$-2k|pEeo| zXBl&I>JRhKp*YKO)@Bl35(9=nFwn6J+1g6IHu#)xa4)P8IgZB<$7treas!MT{R% z%Oq9h533NV$?J}mDiWFiXx!E8m(U1{r;hV!RvGgti_pm=yZ%JBl8+4uGn&mvI22z^ z()aoy9#kgxWeq7~()R-a8xC_C%L-O0MTQB$LoFECPH(R6qtq;a5|bsl$RI8Lj1?l< zu+|&()Kc)TRxT`_=hFbs&8Eu@WhNTZ7>NfKmvQISZ#V5%9WJSw0J|^P*p|$fL8uON z80YBAFzC6PlqQhBoI1Wa0%+J11QCaiUF$BC1NC2&3nA<)8I!LWjf*=(0b9yfyK4|M2G)~FC)#{ibXF?STf0Rk zO8&25PlvVO)@!odXpZ?JJ(O*TIvRbnTnQ>m^8ROiwf z4M9H8GnFNyhmeQdRUV0To6b6e=_tZe*pc zaw8r>tm+O0Rt@{4nT7()*620+)^(%tq^njt1ufI@GUB-BQ z#VF&AmU;Bmv~gD9*JAyfxDoSgTb%=Va-7+s!We)=Mt-varc|00;iPtq8OMVpI5i*z zRtQ$pA;L*>j^sYs9n?VUrRqa=E#5|xgf}t<-C-micVpaqECN$W#MW0m?E*n4TJzSW zbuoUuus44F0LQ<`)AjuFnBA?S-N@JRtu}e7>Cng;r`^S=*`)!?VIuy!C7$%bzQnpv zk8OzEHJ)K{l`$(5j` ze8K`p=6Tk$N(V*xyyiw@l=)|{7kxp;Lc@nHiXg=k&H`d(h1B^0)@f`&k`=~W{aaR; z$soZRbbx$!&MG2}PLQTbO<}Q18>LH~3T3yl2-;nhLu3$k>g$(tO`Q!|y2xL*}>XJ<+T9)fMt zTG}-8f+Ay5rAOryfVcyf%o`%)c|efJ2D6j-s(@fqDJuXoHaje=et-{q|A3ZDMlO>J zBVz%e^TKQM2_iep+4|ejELtHRi7P1|7}Qc~kxi-TmLd}QP?}JZccI8-PKq+Bc|Bmt zm?i4#264jUGgDP(cWo)Ft;XRlii9f)LRVqIPE~HxB!*t!>>bxnK$u5EkCO5_7!5sJ{FDwSa+6Yj zS)PACisoq=)>=A1Nv$2tRddfXb3YYaeZh9N&Y<|y1JU6xY2w#2*}N0)TcVom=2nTZ z9*^TTW;GlzCphD;it04Pd{R5@(=Q$E0NnGS?O)Q=DlA;T85b!}t}b5aB3M35uJ~7U zd17vcyCkRIZtczopS@!&O734<{^rPJf`D<5w!VT;!f!X{U1Ar?PUTnFARmaGP3P@W zQ=VZ?5z03|Qb)GYFZZ_D9cTn*FD!6JQz7iT1v10p3;UqO?-lL4g4$}z_?(SehQn)p zaM_W~{mwS|Mc`3fNHxzyqSu;D0YQ0BJ+47Q;K(jO?x6k2@YfjV)?)M5M1AQWKDY?d zS9sIKl$_xT#!OL>4?l$oUdgz_NW;t7wh{z+!5d5YD#uh;*u@K?)GcL(a8R*neS_4D zh;+*3V5*SXR7iD)d+a{5S|=KXv0!@@C46i*%yD2i!oI8`i`sX@D%_#)e$d$Y>TYTo zAZOmlUve4#h_JljEIa%O2(a3s`Lx*VA(rWaJ2NH|w2Rrk}6;R-JWYfLS;3$<) zJ2NEO>vp*1W7qAai`%^h70Gtn3NS{Qsvzl+41(oyF?n5FR@@=2W;MmHZFi*#kSCQ; zSf~E_>~0t#$XRQ}@;`^|%+Z~njE*|C!=et!Pbpl59@Y$)|MVLObCh>w2l&rNV6j6R z=_oCdFX6$c(J)K6_SbjKu<;tPf{wEVzb_7vGFfF0%lN<9b*6X96pj zw2b+-wfHnituRLxN6%OLN>~~>_KwjKjxoRnEY)Ww*GDbYWIWDV4U28p4`e4x^zG6N9((Ubh2Ff_W zOPJ5u^`RL4X}B$r77=2?n{hXIu9LjmP|;&F_giiZ%uAAWXTA<6Cyq5UPU||c14?cj z+)m)Kw{#=K%orpmDaB4gxA&+4+zTz@#L;r(M&h;4eS)C6#C$YnUd<=k$6^L!?2akm zvC=G&FeK-QTHMF=4ZWi>jIj%f^eQvRb)N6f$f(adp-)%|mYsS(=xFca{RdDsuYbn% zcuNLDkS^00|9?;0gu;S29rUbmR@f=@8` zTw2;Aj!y8m(&?>j>n8&^p9JloQZ0D7(yi}D=bNsS1oXu$jag06>!E=aPH&Txy4`P= zCT|d~wQ|>J{biF0U9C`EaoUTYyFOjp(scvWP+lFn|5xNNAQ{hn+(F$^!LtROoi5D>d6z>t;jqqme792Ah z9TfLK?p{bbS+dGdbF|^t?7`8@lCnZqt%JeAtr$G4+5%n00{oGEE_BFMRpR$Q4*^yV zq6(h<1X(bmdR&$_=vgk!M0+`=GXUy}CSe{+a!c#I%UGGXg6-hM6;?CCxGYJhLJ`^% zyf6!tmjld*<8A{m02RYm_NuayjnC+~h5B#^bA*ZAcl549Ai8pqV%Ilbk}Fj|X=}`p zaU{Qb5wMkW3^p8Kv*aSnd2{o+D&6b7BDux#*FHL4T^x$~+9eR*f@c?1^>X!36Ru`E zx~t2mZwc;{{k1#A##P+Vdkdh=cn>_hk$mN34NpQus3o%18N!CI!QBzrEG7_%eUt}c zBF{7$@}&2V;BH2_Yn|UIH@3;!FvQ?^xWW$Q!dD=Z_0R0RP9Z5eAHfoNqo((_>Vn)3 z=9LUfl@bYgCt5NExO;O?l=uQlnAzm^tg_p!*E~unWdubA!jo7tryu+s(FsHu%8%>` zzRRn!$(_MZPc@v7%`J%#ppqENj}?IT?tixV>(BcuqzB7c7Inq6+|4ri_sbbfCcJ+7 zL%P~|iEiDj{;T;_b8d9{(t$Nwqr%~HIuKoa=Q zg0iZU(6)}$T-LO1ESIQo33~>jG?YO4Wur-4g#0Q*nn94pm<5xuc`Egc{i!M-TnM)< z6y|DZegc;V+5Ys`(e<)*Jnimo`HAQUOhu9%iVXs&HkD$d{ghP79@-Lpgl658Tms#GpvY~tcL8zSo$}4Lxc0jobaKBDqexPrq;0E(GE=ooFvmx zUTCwIr-G6C-8&`V?QrGi0*0yt>{kpn@8}K|$9o#~Cx8R`3jM&O#Q1Ec$JZXUL45?*DhV@Kpy^HbEQYDLt_$bieAY=SwC7t~;D*nlro@Ks ziQW{IoHy>w)m|BO%eY!6&T3+^tQ}QWy0^`J02RoKFNJb_kF_!zGlyo(n?d%5lHB-! zuyjT-f%F})lcDbe$v5kUkTSZ@(V|WOwJ$#?#9f@(i;Zwpa&MhTK|2pfxfM|+SB)tr z8&*M0AvX=Zfi!`-L$;mO%G2%5b?|eU+?%3$V*CH)!2fuz!a_1JUriGtE?=qlH1hTQ+gfdw0H7$8o)6*K;@J9d>8k&Fe|yXM5kDhz z++!D;d`HmX7l#ib7EQ4At!2<>F)R#yl(&KO(i%(d}|8(#{9!fc?3qw;$vF zLC>;JQ_SuSZ!QwctjhCtYE@_o@PNv%Jlt!pe-wr8-o_eI$_!8<3WVz ztq*A~RYih|by3Mo8cTWTEKh%DM#W9ZuT^ojF<5(KJ^4@iz1|lcn0S*D6G2MnX@uhUOE&hPw>eWS_*^0}A24h~j2m(#a>SIEga_HJk zm$@n<&6mvtLM6NNxBTJbV#RT%87(1aFsP70o!pWUI0UWbhzEWWJ&HhEk(|bTtJP5Z&UL>z%M~kX&t#=;?v1Ecvc?r7IJaT<%bL z2^lO3=@cRZ%^wec1ZbAjeeBotGCG{w$z)6u7 zB^;rHXQ+#aMbP+Bk9lPbN(jWj z54FSk04}O=s=fOcW6{YlCAI3eb!FxIpkn#&9a2sPPXEi?Y%Mo0kL*Kxwq%ot*399q z&LS#IS4k3}0N@;%+(2C`(J?C6#`J%&!$bbox^Pw>0NcYwQ%pjPNncz4* zewR7#`1Usar1vBFni2vg2su1;Id1-jIY@z8+yDbzD$Y?slp141yweJw>VXM|3fg0^ ziBBbyQfLFh{g^RTtLq9(CC0VgCdFX%5`-oK@A@#?5+^j7WDPufF*Fbja%&Cw^Kw1( zE$@2cGV_nTA-(ZzRQjJ+k24|b0J}pM%E!0#{+jn2OcbP$jR@kbB(ELL7tO|csm`d? zKG!Ybi(ssdJ(w6-Cl$9hCLtQzt2-eZ_YGSOS@d7sDwNtkDXm@8H`aIjVt;v~PaavT zz0&f_+gkRa?QjZRB7$iJ;f%pPfjne836CEcAj=o2>vBdm7B9!!CvczQDgH-tqZYZPLLg0JVo zSa{&%tQg+~Invm$J#LzcM?!r?P1kt%^jP#urFh>7fEC>mZYC?VS!lW|)`5zfy>e1L zJSSpDMDimx(*p%d)LkYd%l!d=L=h?p^(Kfg2sajd8AIlaO7nTA3|=^~1Jk_0+AWze z?wjH7vyMO1K)=*ieE~!hlPzxHi_+zy9hu3-vcHP8$-akj;g+Uypu36RMoMIH#>pLk zH+X2qWc9F%Dq>K`VDyjC?~S2Lyc1I`6~I(l@<4MvxWN%kF|5o61si$yh`Cib;j-
iC{%_`*f2L_rRPva_H;u81`#n+rI}7zc*o#SZ2)BROi|LG6?#W|`x?~Ih z^(dBZ!%8|kK|IkaX&uQRVcog-?O#^!c-_Jd@U+!@4VFyKPwyqUQM69|yIEw+ML)_CjJS-7`(Z&!||^C#-NiE3VV)E>>% zJyE);KRfnt**yuIyalK^4?SViWvfcKnf-ij?4D}@yqQni7+$(K+-JRx&;IG&3d1Sf zo9dk+hbo!9)0D1`wqh`UK8Pb<4orj*ZTkV5*NOXaN0GVn{&cX`(SjrDWhXieV!K z4cXxB30h?*>fgv@3)WvNuaA!cO%f2tZ${7PBov+rE~IMM>RqH#%T)&JM@QojPVYDB z?ce(5-;cnQ7M}2(3+%TvJ?U~LYFPT~N_+;9)r%e>%Bh$zz?&gL@Y(EdCi`w>9dm;lQXh}3TG3= zOdN=X%gowsx)N(h%@cXnKiRYbBzp7L8tVLK`8~?15T!^?WRfv#RG%iu)4`=Xc?E?ZV~DsChr-L&EQLm?7*)Zpk0N@DhkSaq;-36lf* zW+T|LPFit0`@h17Cw?}iKx9cDrdl{m|5SUX>EgH>nj5^7DVm&Y@e?PtD)18dNd z=Qz|;0c4(YSkY0>)7`y$us3W)f`N$+<(V;|$cGdq<&b|r!URsLSFP$V?_lNISIGj=LTOTyMT z=*`Q_oyS3{F7*jHbXcr!THM90sak%j+PM(}V600_!(0Kf9pM3P1JFgFcQU>KA%;j- z^!Muw@2}Wx@_80Qw2nu>*?XTKtyj|*Z(ljP6ku0rQ`=XFmTJ?+)~YM%bqUeQkDXMG zFFS({idQBBHByJfTaj8gUL=tq%M&5KymOu^%nC{Z+lX$NW58Qqy;LCUW`9rh6BX0! zK5U7T5hv!+WCp=2Ww`{CWTiZ#PQ(4+`@n35e00X3DbVQpM?1pS;B%DG6)AjvvKgqS z1O+P-Sm5Ue^NA{_SO{4d$NBN~M72??Mv#Dg!WZjrnjmw{|KT-EE()d&Fwy@-8C$Gw z#0+Z+glS7?OVifT8+%aaUo?*W5Jo^{l0?v`SR%iKh2E1(3lm(Cj1pWy-~&*8mLg^8 z8kCNdKMMviMhUQo%`tIyj1vcppwzhlpDmCLfN;rOt1?$J?^6!mVjWlrr59?XI0tJJ z(F8Y)d8isXdZZIz85a=PD!O3wQ|MH&6w0XoK7Xzp02g8%HDzg_GT?$fNj!5}Wt1^m z9kHmH(y*mJb7;{GTcOcPVH!dsWGHUbFwm-|iP0D3um6q+sLnBhRZGBz> zIAs+|`BQav(aho2im^cD!jJbgHDpIWf1Xh2t z0zJhV$U(H_y#-}2K^3?Afe}gBYG0mkgp%mO*VxvRn-jYk5kdt}>O*co;#t4aJZ*UX z6{w4H_{kyV&3i&{M#_Xj_*029^aV@Yf!hipV(uJ&vbmz^a+l^PKEgTG|G;rOZ8qdG zqbS$iqhQ=zZh5V3ZoRGD*6M7XTAiLrN`{#(Oh;g`+|AOlLEsM?tVudXlka)(M%V&B zq2rd;x=Knp#aNidQN#D06ca{+)cdb~thoBB$={`_O`MB+xj|RY5W|hMUw=jG+06?z zi0eP44_BD>!Rxyzk6ol{FGv@+W!f1v>W;vfHsk!u#y44#rv8##A~4Mw@rlP5g=Zpo zD(GfpusE8V2g@>=o!adq;U&vhOts&j$Kvn8xu-C|Ig>+sfJ&h? zwG6~%xt>(Qfe6V#lM`^3HKv%Ql-69IVHh2_Hh&Wkvq_6hw%XWUw=qEG*}jXOmDL+c zP{n8S$NXE_YX3oWbL?k@O*Q>M1m8)eBYJcBN2*>8Ejb^}`^Uz$5#6J}wWl`Vc=z?G zq@wz0gmLC`HV!5C!(`CJS7UgUzJd^f&u}{6-H5?rS|jbY+;M0w9-g5JtQFDc>=Z8` zv;Wq^r*Ynn8|X+PbZuYnFS_H7H^>J4<&pd1H>beN@RJG(F)Gh=Ixvhfej}WnHZ7t1rKuHUyfFI8!5H=XD(_Ep%_q?mICbsIawmRHcLGNpC;3VRlqe$r%~=dN?mKv}DOlDYRtD zxQl5sdhC?z9x)_(+(qZbDyk}sDFB-$FFwvONxgS2Fx$OB-+X2IxPy$_lL5PN$vmO0 z_;(k=b%p$W)jjWdtHE-eh1!z)@clj3mVtgrc+c~UAN|d8qQy8AFEj^Hu7$8DUmIx6 zlGUN|2!MWJsI(@o*c_FKdR&nn^xp^L+&>Vt1ZZCzUNk!tSBwkBn<;z$1=`gVM#$z1 zNarc3F}LjJK!@2=rzXxK8!i&l%rvVBG?UW0ozll3vo*J8VIQj1?nhFeo!_32R%>Ga z$gC#-EP21CGc5c?uyJ|YxP01*5X|v80xUeZrEgvy`a3MJp}lV4mba>Js!zu1?$Til zW9ChKSS*nXqnlGapgf1WcP^f>v1lk3qw%Fw(ssjZExt#PsVOnY#Gc(aXM?#6K5KfPsr*RO}krL z=;d=o`p;PQEbzA)d#l_`T~Ld0 zciBlR4z{M}jsueD6eEn6_b&Smipt-REZwS;8CsMdNk-0k5dIy!f0h6BLvU%RvII!4dhj%a8ZA)pzx);?`^0q4V&<~itN!_GE4xkEUgN;ZL z&9CpaF_}ZotbMaka<4j{kiLuPP4aEUi$yy7L)T2XT^yCIi$%+CI%y|2sGHR>24@)e z=Qn31H5c}ro&}GIF0h+UZV%FW{i4yn5Ix6vVe^@gzO-fz zTi#Op5bzDr9XNMXwPHMX<^vXYXHEx-bZFX^Ms1ss0SFXJmq~~h1Qd&+MbcO)4i}EQ zi&J58o|=>mM}>+(2pYq=PHe0U=$!!sr!>%zSH|*`1avCU(pQ3@t?!n>G5yzZPxY*s zXw|#!SXJ6ulhhBm*{YTZrX^2|>Wk*>VRBijmNpy1@{h2}=DNGjKWP_sHKh zag-)`xGVF8V`fP%38QScIo_(!c)qoZPYS`Un*-i>W;THhozT8w81lBWF55zxD7(L# zqSpAr4K1$TphLtI{MGhB-%)V+F&-W;U-+bDb4UjQp0ScN8ji^PNzCBlexfR`p`C!j z(W}2U*53yyyoxot1pmFJ4>*9lPPs{@<o6v>-u$R5v`&DQ)`gFkMhVP^ZPDWS9i6m>z*iAj;+2h z{}toi`T|Wc{f19GzTuPqOPSUG7y0l%#dTt$0yh5{tQ8f1Ci~4041dkj=(JZQN1)b* z14Eh!pjP7r#=-*^N+b#Y3fs!G31Ku&vrg7f+>%716-C19A={ydG;5NnM4_{GFrAvp z-1^F7I$D34)&t@mf&zyvz8f(b5?%FMbTOTra-i)H3@V8L3`L`OB#kwCvoG!aW9UCBo?8vI=!4 zkH||M*xFS5c`8h#QBwq+MFi?MY~?GRVtI0nHziLtT~7pn{j7DI;J-#d!ZhS^I+%%8x`apI`IdWJO+;kUFM)Ojdz+XGf&adrRV8m5SX| zKnyz&%q-(D2S%NmK^x4loBtz4x9*EU8`^NBD9+7ruPnmM?Q)k@5*q32oVNcMkI@)i z!r6FVt*I@`peD&2Aldps%`70)ts9`)vqrd*Djp=j(~&zj)0^1Hvo#5sE~uLqvL-- zCt0dmN=j<~M7o0aAqap<0ev@v{DlEKHYAA{{Go= zGz=^Jjy|xexIlC|3_mrxX@?<7|7W`c4|0h3$X}aQa#pIVD#Gsb-3@flp_Q6WPvtrt zPt9&IM$+6N?K%O6N(j4cLq{n#mQGr-AlPfC&3H>bRyh`toW-%MCT(g}0|)_?Tw;?m zLrUft!%tEd#<)V8ldI4S8^f<|L4szsdeDl>Q7V_f&;*fyrgD(TeUUVN-zrR0D`8+L z?fVW;w&ZR~gIaD{gIsQEB1D*}lSv1;+dL@W{YWp5)+kKR@*-@ql+WUW)V!**5xjl@ zUu2z>yuCOmIlD|KIeS%Oe)sV`mtQkKfY+CQr@)*E>#OJ4H9W`nL_d`LERhgw?w6Gt z{^e;je=2e}!3|h3_xg3N(LWVVpj2NB9Rh)aRB8cPt?5Q0Y5>Ne6tdYgx?uZwy4;{W zS6p8Q#ESXzVB8d(#WOy5Zb@bpy0de#^E@imhFBs!Q)#g7dZeG5BLJ+Qt zRO<7c_2DW%kd;J#_qs*S7E^0-`ceeq^=<~oq1UYgx~86MB^FYL)Vj0{dJCG~;+lt{ zd%4=Ftu=B&Bo}J~L(LC@x|1G|F~9_VWhAyEQgvX*HSPYajFAI4keDnd2>d7%Y=SNV zC~oYCsSsQN@Q-!e|YM_OxVyciMk zgJ`xi?Sd5G8SbNDcw*BL1qEQ6g|J7%?!iWHL)~@@gLm~Zctz~FO?&{inKxp|+0FId zFiMsw$zB9VpR$v{GU(`fO)?+30&Ns@65Sj90*j76I*h%m>RTQG^(YC82{H^U}mQ0fGkV_`0YFsVsqus6d z38U+>NZLN=g}dhRx~5uSk^+GyP5&cpJMEM+zv=1H_4)h`+QaMs(Lj*ZgFMyPGw=dJ ztvw?*@6lSiDFvZDgo5MHh_YRZCJyb8VeZqFSBPNk9>MTbxY&S|FG*rm0k-Zdc{C)84wND4IZ0alz4g z4S%mSX5Cj>*`jdI z05b)x7%JjP(3*27QmaY6n@+qyBB=BByVJ3U%)SqR9CqwPHL#azOY5mO!&+i2Dc7I? zMbR?B<~fWOBI2$NlffQ__}MHyvC|Z0WX@(M#^oyGlshks)+RHtm+c3N6fu&j-sk;O z0}1YWW&q`(P($~8D=dU zvnm^38YjE27m#(fM^`SZ@%HNm8hS;76MocGKuXqwmx??JPyo}fh1+0$IFu|d#NGx| z#2}nNLr%g}FdC-TY5|FN^@CT=qgnZUOZ3BsxuJH7F+LXHj{P8bOk(fWsBOyTf(KcL?asfs6;3JOD{XmH>l$X7vw`% zX?k4Fh@6U(hf^^gd-@hojAmc8$HHuOA`__0<7uZt8Hlh+jHJ28Jzq%2@f+Qm^8r_; zogP=@==BBNH|j6Nn)x|Vy6fyMMA$D~k-?0dt>V8+_i3U`bQiRlMj=Y6`zlqVN`W%5 z4!;v;=}%oT0GF_}p{7G{MuF-Di+~vC4HLlL;+*N9yU>!kc5@#6s1T%cz6C=sJg%un zmPcN~X|lcvVyb!8hkvO70>vQbPW#?kmET+I-|Z9aM}x3~}lO{s81HfGMJd%S2*(jigzXjP{ta*{~Kphqy)CzHW16 z7c@Tf?qwWZ+in+pgVbCZEBr;XAz5i$Z=0UoZH&4~wKrP{lh-3z&_MmD==zU@z z=mU|MRS!FD>9)iS5ga&o62vvRzauOf3qQRpj5s88*_=1#YWs3+9l5UG&7c#KkJM!T zZo9~&!f3EGyY);OmhXgWW7Utq>%~_=Cu8K7IfV&mJv4#zLrAsl`jqP;_{6DUuVFNC zXK;~RN}0H_Hs+FMK(7{r8F%iaCM7!opQO7A+sQ3ZtQ0>rNMNwkQdtGJF5KLs&hvaS z2p5jqHIWHdM|cSM$Y8H>YS?gb2~i)PAD%#=$6K~HnS;zZW^mdtK`V>YQ1~Zbq2{ps zawSb_!o9<>;oPW9?^96%EvFKEx|_q8k+rO$+a=s zsO?!Mc5D)!NbN?mh01|}JB5WQS3Ustx3@G}x3(8Sd6&8b*F{=I zv95neE6@kXP!k)r}a=iI`) zif3YwWvPs@V(c>^_7u_JH3wfK?7W$m>wNKtvd!Rf87hcPSu_Way9UJeYOferu&3t? z&%WQwp2u&f1;&=vZ?43Tv6+jEd<2;5Lf_ePRbph12H5)e;r2RM^KCu|6l!K2XeJ z$eanA{b2=v6Y`$avFt}UqoqgM#!3CHj$a69L+ij;@ z&6+U6H>Zr7NUp2z$9jVJdhkrRvo_&z21bZ4*5|X2CIHhfH+P3zzg^uH1#!xm+=3h7 zr-eH7&0Ufd9)IU8t@4_b;+g&&rN^Qm;%hUYjzB`NTuY0o6Fyc_Ah(rcJ$sanLe~}3 zh;xXG*q6T5F4Bf^oHQzFPzlC^E36&U#W7ujndDWtRV0P(P#Og5gY?;Tpm}O8aI`0A zK_xxN*>y})2gVxPpJJ@|_-SnZK#P>7m6WCsngV?>q0#rc&y5UyleLw+dP=S0)GMw% z>K`Mi$16&n0uBsqix(~Zq+${14BB~?LHV~G{&gg+(DO+H_C4t>jSA7d&`SaJ)9ac? zu3?s+X}|*uYsGlw<;#PiVj}^n|7}cyI(~W%x~01M4m6guh4iZ6iP_~ zRQDA7s|ctj4D+G-%R>KClsUxMUQu*y1gJ~-xiVf4_f6WZ+apqgGpE2g>mDM?omv|4mH+f3VAG;u%68U&-5f|n zi#n}nPIE}IOzvML3wC4(eRl9aew2PYFaMpkM&7{Tdu;%0ZImocTuqFnOx*u-5LEsn zTv+@TE>Ne>PRxgsXl4$@e!>yxb6cD?RiccSzo&PQ69E zKUKZEPW#8*L>{^hg;=_(JhtR>)Sz5C>Hza5`xnDoI~llM;~^q$OA)r^k6fpBPp5Y} zjqmAbU%6|(_eMxvN4-zp>Q(p8WbgH_@B9p2n-O?_VEAsghj}Y{>%$-QxS!QXp2%?D zGw=l7l@&fGB7C2AkHVkPEN)YL-XXv8c)K6B`FNh`@L2b{lRCQt_HmE^PYyVcn&#kP`p-WLHHgJU2jjl_P??$0Y`&K0;>D8hw$)kAKuS@YX z%26loP1&J%k~FBRv71C$oq3jYSDH!vFveI-{WJy{1LBp~77v^JKNx$*F3qA~OSdX* z+qP}IY1_7KR;B%>ZQEv5+P3XV+c*2%?mPON9(}vN>|e00#hAPFH(KQxbM0a_1`p?Sx{M_2=a&vKRkzR9dqoK3BwYt>7KxVln1(?(G znDV%FS;9{bvU;)Rf7Sg2ofW_Gn`j0j3UML5-`sMh0(LMobfg_f3!0?5jG;33F2Xw* zuxU<(9ThvZ%4CbU5HeY$#VL}%fO{qdFHv<$UwLakTC9`_DEHimq|hlevMGx7Zkekw zrLz+6sB)w+9YoDyyBF!I1}@AP(QcU#05#9JjEPyD zld)SDxmZu75!Rr4L)*>QR0}U1$ADxPmYV{sIZcV~bZdSKJoX{Ab>gT^9SyHD@1RRK zX*WM_`6#DZFsqSi$;zq)I!#(@b^mSND!G(?8)YNLIoL{M$}O&2Jdca0#1oVw(@A%E~|BFt|Ag)#SB#8i_OOe zbq*LEwKTUu;C$%VsxT8z7JTqjh+nJS&*S5jH5@r@P<7w)$7-6AqfRSg4(sW`M*apn zf3U*|R{{a!kOc4TA>SNtTBDxRL@CcApwPMJxk!0@97tgBBm=!NmigLqww5vQ2)oCy8Y>5`}^uq7;1$UeN%j zcy4xo8MIm&pm%NpdSUGh;Rod>QLQ8whwZaJ&l64`{;^%<1F59JW=K9yl-XkR5(p`U zgpqNC%ZA#;i4iYXXO@o-%iNhU0c8g~fOsHA;X++@m@#-f-x3!(9D>z~)+Jlv^w1b* zW^Ld_hqv>QU@q(L;Y~3XSF#FQlkt8~euMV|Ml!4cceZp$cbH{2Eu8uMMOPY#3N5%^t zbxhNfs6}Zz2z}h-;RmR6BQuv+F0FU^h+|KZX2R9C9$~Oj&&3dNA14rD(1=)`kz$`w zxWfU3n76GY){w-#8jbE};6x2_&5&4wKaGyv}BS#Dq2$^og>C=0jWIU z;7-F~Rw8~UEDs)}KsvykrsPy*ZTw{`(x7(S0vQ!Ql|&a$h7sfD+QKr-NK=z$vqrCB zG5v#gJt8bplad{ZmtX^?W)Z_=9*AANW~?;klbXO8F%KbcwDFhJb0N;AcslNUfbF2} zEQD;LVb_ojJzQIz%a8&Td0dvPd@eBS7j@%*NfiaX-O8!7;4Ahxrvcv>8K1GNICl`ZUvZ^cl~=@z$~ zRum|4VoACL{S?m6ZsL47^PU`nA$ka+<5v?WKiyz-%=iR0a4Q3rb=l51w~;`qGS!}~ zxTg{L&_@=q@PoOSgSBBPr^$Qh)dJ0^dUy4t+@tW57m0D3q-0p0%zTBH2wb|hVO{?d6Jt!ji zUFTt)YT&J-!du@y>q{HDYjI!H@l)%0!mICqtA;ye4n02?GOezmF!GFVP=JtBUv9Ai z#a};RX*K2e2A=Uxwq*-KDTKD`bMS-Ak+mCrSQz$FpxX=9bmrIrJWgX%hgZIt14~Md zF|`4Iu43}7QSlUB6pNJVAIE=|f<~jsGvpA>cG=)>*b_z-hw<`=H0GmYvMQRygxIH% zVvTE+_uHLupYqOX|FrK?;Pri8?Ydn#M)?F+(kpbk50{R9EhJ6nlG8Qlp4Ew$1=Ch! zuG(j*7c=$ACe99SIt(vnmhchg&%=dH@7bY4LerwDI4O>tnEiz-+cTVArP&Z>#fyTU z?(f^}oeiFR?VQAG-gA3vh;Esh&d^A`=EG^PWG0Sp4y# zbvURv1$j4-yT~Xr?Ot=%U0aEhoweN(QLP!lME66i3?K+Wgmo|ykd3#_y^jHm-8s4E z>igA$k@xwmI77b;z21ZJCPO(xw(~h%6Bj+0sV*5r78WcuZ4&z zF>czjL+AOS9=s+5*RL&^pHwbn-H})4JF$}uvOGv;UI0BSm=r7P_Z^$=z9XbUp&=dt z$CeWs1pfXtc=rDFGG7?1bxs!{Z4moZOsP?HA(iI7ajhpFc`we>kW&&c-W_>tVUD`3 z)DgI&&sP44O~=C(5qUcXO?T902_i$G$R6T?KcXkV*sVKubV0}HW;k@8O?9|jq}8bu zIIa}Ytr^;{8G^)cK&(FiYFiYQ=_QKKSEWx7H%i(O0(tL5Fq!Du=&WV7Mvt9ZzLo9C zm`zcU5~RXC?IzeoRl4JIf?jA2GYnx8QBs>l<0vE5DfRa46pWFJ8B7jQ8&n%y6W8D> z>^*|kwdFtqcE2S5GnXsi{G;)}k|`6YR#TB6wXhB8kbLe%SL6CN0pU?hS!Rwq0Wpom zBc(E0KGr9tJEy6w<_)8>u+>n}aJ-|Vp~ev~^no6lv7PZa7(_LRy@=Zb;gMkwR9aCq z%l>#&FYU3*|5WIOahy<*0B3QzlqJKOC1dWyFuJR2MCHLBtydwGeZ`j{^={t(3iHbk8Cw$G8Zy@h{H!rAwVC*MNRq~y%8}zs>el6__t^y zX#UnHEB@^U+zU@(3@j&s)l)mCm-UvVhpKkh>xEc9E_Y7`NdDYVh-~1d8<2`0cFjIa z)N?7hhYQrXk<)&mAI^Sm*$3+>bhw)53VY2C+wO-(Q;E80I!9EL*5lYm-Hb2zW`()Jo0EqE~}Tn!_O%iAMJ$ttVtK07>H zeBG*PTg@N~HKo*9q$en#XTS#|?&`p}I6kC|JL;_si)(x>*_yQX{Qb6uF6!3)(VAi= zF}8=ZQQbeeBVh0h;&OVF4t=mDOwk(x5+O3G)gA>>!P&zIXNuC%%_O3+r>TIA-#v_Q zpk0j|Pd&*RDXWi|@{%gKp3|U`O1>~yf!)MW18q5#TDit~sMdikBFI`%$vwLO$2}xn zWpO~lD<3gu&Trc-YrN43juv-(;bRKtkwH7(QAvwG!X|evOV4V^G+&k^bFWDWMp6M_ z+8jM<(nb*kJ)DNyo4H5MQ`Hw+_ft?9)j!nzJ*)i&Q#8bYCBZ*LiGruN37u9kd9WK3V*!m;Al@Kf7-EeKX7wFe|^s zCfvuqvH%sX(VPGZucQazggFpTW|w1)eJ|e>m*QI<{(Spi={-(#Ni3|0jud`20>fV_ zvh}A4=jPHSwpH?w1I6qzo9(C95eRO8L%(whr|dvOMA(QySC0-Z>Sb%jAPIFt=0zidV^VJ#8DZG#B8y6QvSV(TDk%GO|32Qh_{HPeM4dX~ zxgL6r8DZG^OIzcX-mZR0U|UZ~U-e?mWK*fp5~eEJM&-zm;*w_G1(&W`9$b6>xRRi1qiZ0#Mxj#b@1P#P2U6zPc zCKX8VTs$_<*6HEfie^$F;48T?zXIebpNNerzDJfD3b@^=D~wq3P$*~z~8 zGNIqS#0TcU(?+^11#$4B7n|25_wm~l_jb?s`OIzI`dNqb=aDpgIwc#eq6t>pg&_zk(Q7J|gPCjsNgt{Akx7=nb%jhbLjXzIHs_OSGgn;>u>*WVYF^1yK}%prLX zee`TypzExk8ax@Fca3!2L#*?gN)oEmv7Sj1+a!uM0dqC~+N`L|ijrSucPd_>BrMiw zWokQF=V3isrN}fnQbKa{DXS_|oJdk5WVKIREjXJ@zQLe88#G|2qG+7{#@j#*Ze60F z(2Cts_u5R>UL;yE9*we0U4@4Z?&7Rkcv>VvLGrK4jB9~VLy^GFI@MV`qeB~QVry=0 zt%-DFQ7_nS{x2)~Gb=)SxVelluMWxN<^}!5rW|kPS|5zGPY8wAkfd8wAdAR9LNh@@ zTkVLrlqKVClB6Zn{5EqNVhOa2JkucilSgrIT98)l6Lr}OdGH)_R{4(k{tjZxU;-mI zEB=J}sygiX1YUNTR7Zl-dY^~rz(v&J8EU;U%1TPRO(?evZweC}?)d%^3``6&+RaRM zWriR&UuYwb@Si%pn*nRo&B~r7SqqTOxoJhx0b~1!EC4bf}Bs zyf882`D^g4(6ffbE~3dif`OHUE%XmR1C>v6n^lPjbxi5rY8sT7=tR@rEmfl<$p-=b z5@q!B2|Y8BvDT@l5t$|eS#r1tc@;iV3R`c2_*OJop5ghIB#ODu z!h|+6F@2@sC0TK)eunlSMV69MY+R&*VRIzAvnFkrn{ylHQ@JLi2j`A3I_^yzqJ>&l z=(3tGW)RKPED~g~BEgIF!puq?bw7V|E!BLJ+`yS(TTmDfl})T)t}Ubl zjU{$I@sj8Es9dO|B_WQp%@tBO1u93wbq(BPWF`iGe1&YcdA)F};kTW|+&R0iFC8JR z4@F4(ABPy&wJZ4pI4JEBye=vz&3~(QCetSjTy{(@aXFD{^)IP3Y}dzQ{H_qH_w{P3 zvZA*8T|Ixwi56Fzqx`Zm%5F&V>{;GphgraJ5Z_6#h&bjUG?Z?AxzM)K=G8dql{)BY zbgEkI-?Xa4(c^iWg*j)h;;zn#E!n*uiN3wns4!saOdMYw&`SegpxYZBJFU{<%l3+46-iNq)KAUTlH{K5{}6l9 zK=S|nB(V$Oe~#~cH>2rl4=tEtU@-|Z(H`uRIWyqInt$gg(vzDRZ^2ioL0Z$LAG-=b#7 zqtj4d`bH{?5U1=K;S>h(#wwTVp&c;W?#5(fQCSLCnuMF%x~axjhP698O$zj;7tlh_ zJvm+hu35(9|5oZA^hw0WW)GM%SiqaOw_WlStJmb+?m<6(!2sWz-`*D38keb z%U<+)H&b_SY`$?z6>2S3@nr7*(COe2OfIF$g!|^nvg;G0-q%WcGk?(5;Z zkI^z>Oe_So;|YQjcD#Txi=$o)f{eE@UYKHl-sT0a27SzMJOTgv|Lx6DP`@rK>-FPQ2yIahvFf7cdYK5d>J%Jw|skk&cJpM^Cud6a_FI z#Q0+cytlg~RR+;wwxC`Smn7Oe?)x6#mA>d@@fJp)%&XDb_lE$2;WES5F zNkgFA6k8}#$M<0SLk}Tx<924_!u8Y>Wk{3_&eTub{C+ois9TSLRgJ+fQ^de7OX*U1uw7`O0@;R#^-szdvIh?f1>Fu&c z44}DE+qmdcqjTNou4VDOyIRZvZ^Ync(#YXQ+hz^KfkeJ%;;rB}>8Y0B9@fBqEoH(j zwfbVcKiAqbC`_`Te&Akvfd97(D!DTmn0sk7bZNNeS-b$g>76dC+g4gsm&7YQ=UlHYiMT8x(}%&UzGTsI?@%nt2JH=EH)>#@Q5EHHGSkbD6EQfXgKtYrQALoee!w) z&FLB1bIG>eq$<-Lewl}}Mf8xqrH#~4^rxLSiA62a$!E6GJ*Z1RMWHkhuFBXxH-DuP z6PM(gIyua^s_+%c8cXdAHQXGdh4Ov(_OJ>J`y~?pYkcvF3G`3t*snZ60n^1VQnfEs znWbE<$*;b-r{w~WWktp6fYof*0S7}M?!L2J!0EO07v_K7Ey@@!*#N$E#UM~Wez5%? zc8khJ_NESY|7+`2t+xG5jX?XfZ!jgM3z-2C{}yYm*X7%a=`<&OHVLhMZ9jxfd!|`ViP`Ybsp!q^tk-F z%Guohd_JZA&3R1(wAZy10dc6@&F0z~6l0GrkCAa&8(JJP!`E2wWe3`{)$EJoL2Rk? z%N?@jcTu{@o{m7Bu=`YN$Fm0ngj5};(sx>8Y80utjj4|!*xHm#T`&hAXxM{=%y6r$ zAtAv5^_+Dn+vprAl0R1TTdN~4QmO$vy!KxI^2{vNg*tbWi-ateoXl!^TF5@IzoQ6b zJeFwJlC{=77JTN*z-bX=cw7Ilgqdl|xRs)C)g-rWCbTY5(Vwr9pEhnR9d_u@shTO7 z2`EMg4VdY7(*@zhk~{LTyPlkekj`x7(Xc_^6Q0$+&_x=_;@@>Ecq$iAuNS#`&{ROm z7i;m0>~V5NHsEmO=*&As1#9?eHT!Yz9~MvQ3R^SoMe5ylRWzH=SE@2f(=tZo7%129 zeSNN_{M)N=mH8zx3a?c~OLm%49yFGK*l2p8Q0%bjHWp$%2o2~p_D@s(`~GwdLeHKo zWO)0lLNqT%o1Kk94ZQXe^Dwwj{}tg(u;7G!CTtx7xR(|{aDO&}(-xE!V70;Si0B3C zuvZiS;tig_h&KNhZflv@W=Aax?PRSsE3n;R2Xq2EZ8$KX5epgX#Q{^D$W8OFpk8b! zIaf|{`c^v&2>-N|aIt~$a)uMAmOA#mTc>jB{P?uPM*hD5q(wy_?nDJsfTeZ#S5 zOyUkDeE4<#1|EB{Cl@Qf{$AQarbsN+u|I1!yip`&c-kywct*bz{dVZ0q?rGK;% zg#j$YGZ4WmosDR<6pZ!PhQzgE`n@mI<_%5>RiGZEE7o>ow=1j3tZ;?()WiNu=m1-L z((WWyuBYiUkYkCcVfN|5IYgTdZ~!5~P5=WiT=f#YW= z14Cqk-l1ixJ=~;gq{FyB)CKp6%POYaePv+FwTY+P?gv(DY=i2Fn`8q0*q1OQM{zKu z&bo-bWVNRyBmKTBp729jVr;5ZMkQgbqkF))&Rh2|B6OI>Q1`Zzz*dR10bmm}!MiS% zUErS`CGjhO_yS~+SL)|*2wvPXqwd5zvINe5jN)Q za`Pc6LcNYI@iO!00I`Lp-!mLYP|$J8)CF#dS^tqH-bhWaf9?Mrsgs8&z*5M!6Wv0@ z`n&@-@yz%Q}`S)+4-GBS4^OukR*Kw-O5BJ^kA1$P7;dWePq5X2< zyWvycZ#r-{ox=TVMzV?aQ|9Gk8zRixBt*ag3rUYFymv~2gf$~ z*+gB`j8oSrx`xL| zmqT?pzk$6pUovc)ScSaFJkaAy)s8A2pU-d4%Pq%vk`7&i9-U==09%s6fL&svxh7R( zp^}p$mOy!9m18v1x5%fpx!Am`+B{%T7??k`Ias$#>bZa< zY+3H;7)zuv6Ce-HJ5xdw?T*bz?Wm}_Q$S`S-RmVg{>=;$#_NgJFzu??Vb@i?=GijT z{Pq0Tkj2Z_mdnePiWwtjFl=aVGyt@}-V`vD3G@AafV0+Tkv1@cGvh>^f19~Z^j<) zLN44_jYrN=e=Q5X-f>pIEm?kUwScoYI;^$Wtk(ILUQIPa+WzAQAq!jcL-~f2t;A*} z%`;SLFe@W8xS;wqw1C<*wt$7D%*vT>HVQ8%KU0|ORY828Lp6#2JlNGm`F0}kD4w(W1XK45Wg+-dA>$+60u-=PYeRkHtDHh>66PbZk4vS0>7 zA5Jksi-K%MEeZkqJ@gPb;UVuHbty#Qf~&58oO%tB4(*fO z?wVYEq|yS56G8nvsJ;iR;K*a$=aO>%x$Nwrm$d@eV0Ago{#eQaoBg_Tuo79+Mm3BG zc0nlKYgCr@0-;z$RMDAEj#reC2}r9jqawO_xe>$oUuW#pY~N_is0AUcrC^)af@TZ~ z=;R2_U2wPgy*z>I*6VZ*vlMNq#QHn0vuB)5#bacDqyi4XtiZFAu1ifd8hWx*$>bLq zimSkv6~J`(#eH6RVeBW&R-`}N*2KVc;1I(a{`I4f%P^iM z5J{Xk$Vq-S%QOf)uhn--)*1O3LvJ9&h%1@fJJ>e<`=%GJ^fA07+;(1$-ueBekRN_c zp%KRI7Er#t>mhOD+%ijyhy*;dMx;IFC=N|`x7woqmLfyKp}@=)p%MVnF~u2XxCi4d zUX-aRqEYSI%+Y3#5gV?P5lD;!n4?>o=Zk`OS$Vd3qT>)NigSge7j5Q#K7lVY{b&Zp zHO;C7!#!D^i9EEQWbm90pB%&1pdblhW|sK`oDIl&FXglP3Y=hB6l}52k=6r zEq(^fgJ}t?vNbD92ZgMplq6xo@~py*rc7S)tPVbEJWD(i9k1-ox-3k8QpI{uth?d@ zMj`pREm(M6u3WDCF2^sozuq4qf6G8MN*a*|p_UcZXAEu9Ejcnqy?Ms#Md|9mIgyk@{fVscWr{d$z2y2(vJGp*v7tfNnJtPBbj&)@ z(R`FUZI_Sv`ZnQQ=Mq|+qnPCHgAK8+=?@oh^RVPE6>d}hWLlPDUb`v?(Kk-mt-h^2 zP9$7Fsr?wll3C`TM&6*Gnqdrq56$(SyZM#=FU(BMuOjl;lU7mB$NX3IzZTCo?Re^t ze$cmc%?I;J`ssXA#=E>;ruD)nZ`IAjl=6cpmF^C*3&j#uYNf|&#jU#0{CZHK5x)wf zT$x*OIj~To<*$WPVB=VIXJpL0Rk z0*>>G)G<_qJ~>ZMTi_!0ushtrOYLHJCsf0X>q%T|j0wBTrw|QY%9g=G+vm-85Vn|H z6`Tbx2C+YCOb~S6(|X4)I{xtCK?;W`k(vu4!%^iL?)f=jcfn?h9Ac}_3Y5}m>@T=) zS|Hm@A9?6a5F}DRGrdrCeI1%$cIsRrs_H3Q`#Gw24HcqQdanjVjnUFyK4}R1-If5c zEVGdZ-goSpE{erX?wr!&Gz~UQYDS5Ud8rL$Dc%i~MqneS&=dLlVybSfJ&s`8N5rpn zJMay7TLP#-A1Uw2N8W$FwU}1_EXzBH>D02@{X1g>5WoHTI=0nLltl%jgm^2~BHn@( z%yNsFToSn~)UTJ=q5w7U3zQ1clV3uMSXf)LZ>(IRZ%$doZh`U@Ma)89`!A`6UM(>J zeJ4AX#*iY1zPlP^UjS>|Q%vnAylP73qCYh#O!2*D(R(I+U4b4C+2VaZo`f@kh` zm}E&XS2KyR$&)8y?MNHLG8ap8kzW_F)Ek152hhZd>u#w;HB0;N-F1N`FNBG~ zn>2xwV3l}>fT)k9k{+^AmwLo~w99;H%L?M38Z0!;Nydz*BKV4$!dVWyxt3-L$cKaS za@}{m^CAdh;WKm%jO5}_ib;x_nx(z}Xja7`C%HAJJb@!WVTrFe_$J$>YD4FWB5%x| zmXztAOy-<#Th>K9euuVYL=P~|cUC@U$@E>`P5@IWMeh7)zC^`961KD?LD*QlDu~!e~vJGTJ{JP zcWc<&JuXLFNHqPzW967*<^X76&ku=@?1nIyutAcV99|j%-BR6>|7kkFmB~h|JDAkZ zmozG^n>NMZ<6UtmQ4ZnOPmQ|Dn%1GY?KUj6-~E>k-OD9@`u1Cqy!SmhVENzN##RoD z04sYHR|jV!3o|KuM>kh-2WLAY*Z+>nQk8Q+Wk&LwS}w^~w0xnPDQSgh*5fLz07H+b z5kn(aA_q=gTAFlpw%{?scrd)ox&8U>t4aIq|5IIDeEPq552L@XJ)oWXF5|w8}&4@?YVxx&(t0)5ZOo_~G zgOniA+_RNihIOw#iDsg04h<7IBqzUC*=P-O3I6%{xtuwPx{=XE1a;nC4=PQ~s99~; z9{#^Yh#-m1hhPyHsN;z2zmSob1y2nBTmogo#&{(eZKrPH-(&phE&rFJV&&kmqj^*- za11_v<=MK|yQ`(ZT=il(b~nHg zMO+PQMkPS|7lP*T&rS|X+IlbU z)ga!M4qLkqR_R)K#ZRU%2QOb2ZI?)oEtHx!UwqCwTVz@Sr9QchKD%!hS&zyU!=+c} zE9ifI;{}3C9$b8T+C3nD{9ygRIYA0;uK(o-HLLxHzx@+J8)JzC%JU}}7}{7dou|l8 zB`{_(^k{OVm}^J$w|l4Lb8t5eA!Vj= z_s%4_^Vj1gaRh50;)?NDNsZGn);Jwze0&!sW}+D-rgszFpEp0Mi(g?hFF5th|r%H<+2uF z0tW6m&fQXZ8GLHhoHz}Vuj!m-FhRdl>cyK)!pD~KSYdVVq}^Dxq3xUgN$_gzX8s<- zu-9xGor!aTC0Gt9SlRj-R-P6mihvTgq+(^15zsJa|6sLnP6}(pc_zy!#@qT_GMn9W zG);5WthILkXoUI~4X3J3r+&$rF=nFk&aSWnhk{{G?ueTRC%J=gshuWmh{K9;NzN4Y z23L-nS=13XbP^<}RDlp?bjw&dG-IvIT#~AeaN0kEpM%k^;g}K*D-GU*(N6Ps<+N;I zMz;D(WS)sEQ3)){wiaVfAIXaV?V&?;J$c;A%^FY|J6BNS)D0jTjKyjB&?FWi4RKv) zf|F626(w`gmC$(lH4TmQ>Tgkg%8tJ4l(ZfMr=N9nH=8U==(vk{I)meV8fM(j$8H$< z%EmD_m7l&6`+QBXGOXyEGSqQhM<^!*MgAkh325FS*Hplv>M!AfKN&>YS?0yUH>cbg zwc>2;ig{D6=|d;B-ckN+p*g+2?^E{WX!!|}4k~UX8=ueIq=cu3g}6DXE_>O@TFH?` z+3Iv3za`c%6ONz@;N^BGROFe7z_MCzlQqHHGgXGWwTZH^wL|R~!cw!_QlyzMmX% zY*u!uV9|1L5S=Y1Fl_m>r`q~)?#YXhfP|9CsNfbUDFETYN1E&(Q|mE2JL2Xb0~tj~!+ywt9>cw>l1&{u z{+u40a^X&x9=~P_~+#z(2||&2Nk-H!$1$Kd75tA(WdCN%G|A0@S=~G zFIJ}PYe8nXxU4N7O4aUy0guma({vml<^yUk)`+Te8Y|_hXO`{qY$03N=c_c>i`MSd{htZZ0d>9Iyg0z(6QK%PFR%u-e;Y1(o>;MwWb;tcH77ugR9*Am`qq+V`I~;Nl}lw9{TObBANY?hV%TPqANAu zP0L`X*jox2wOZlL^>W-LrA2qzCDEH`2Jj#Vebo_pUi4*z41wZE+AR+YY`AKqHyGhd z)UML*5`($I+3sPDRr`oVQ;AKRJptQqx0Z0q8BOEtUl|JhAzBLkK`~UFy2SyTh7KL| z+Wmw8gN$c%f4d!cREIrI2!>2gSO;3Z_b)a{=x2P?rycze-jq{q5|_aN2lD9c-6x1o z!#$cUvs@jF=FGzO;x5*DcpdmIi#y~;Chx-y+fxR99TSOJcD zwCFy)i+fy?+?x+`{+UCBfXzx96_{>i;_QoHr6CM8+kGvOi_;U_QRQ6?7MUTg0Jx)7 z{^BHCt-(~QPWHpp&1mVK%^b9$y$)qr$8pcD{CT?I<6oq^`7zsDNv6UX4ysQSvdHoLgaL2AFxp3B}=8z7#l;GOr9H^9_L z-F3u%iHzRfUlfiY{9AN>zSdQ^OHN^PYb=c^F5wZ~P{45)0%;l_=T4%4S>E8-)5+fW z-kS%7ak!=bFS~8m){qAnSb-|V7Sp* zk>W7!nxL*s7+;mkk~4kZ)FkZk9b`Tl2L@ln?NYYMayk}G5&eq*hd7=V7v$3b)vp>c zQw+oVr!Q!TWt)HCrrgI6b1$v}&EYu#vI3;hVrihEl4iBKA?b*zQJU*t8dbc=O3QWr zKf6sg+yhOo*lpX0SGwWg;b|>Jd37{;?E{I|N zBpiyz5DFVX6%tfH82$Mf0M9~iOj`Oc`6*7`e>U|MN{8g~LIiMsPc8sBiv4=gFxWFe z)?AZmeZuH?V!7z9-1Os`4g>hO?_$kQ+I#@ z7$%8-bOv=v=BmRqshVY^%rBL`4|WMA%pFn46(ZH=i>I*9_J#LpSUv{;CFcucVF2u- z{3_9<7$NL0#zf4`C(s{J54^|jj-HnW>{#o`1#XxaX?ygIU&+&EdpN7_c79bB7mWmG z!Q!W9YL4mpw`gam%dpDfp=^#Zx>A*35cnW@?NBfX=l($5wGX@|EX|M56xr4)CI$+^u*)MUrYn5>mz`u9dr9DrZ?EIP zcZK`E>J9(j>Zajr<@!IWTeYg4!Z$X+zr{gg3Ihqv3mHs|`tT6uSsFP2L(;?~F=a=h zQJ3?u?m`d9zw3}rg(HY1@KzYzC}CBJ1`no{ahMJ~_RQZc4%OhnJN0ZJ)9bad3LS}5YdxUpx7>4XJk<+T9AbbC z&FNCZyy{xZarAiBPiFJ;T0dCSyVRDprxJ1A<}qcuZP8(F>!0*9;!+>1vzAv2S*bfC zx9aR7iR^3Iy!Ba5=pz>}YP2Rq`Kw&8>6V*nx6W+6;Uu=o8dBenC|DyyZ4E3MkaVJ( zmUM1SFqYh0NEP1Z;|L+8A8QC*#-SqJB^8S_>O@8N4%(~I&#bpW@K~h&xN;d!k}?@~ zIgy@)#Y=~qUluPMQ+W}u`C*o3om4$`j$T;jK_r{rQ+5+XSgKmjuu+a{-B6RDR_=1z zes6p{_TH>vd-$u!T&+3S~i|r$+XwGUVm?>B%Jm4rXL+FH}y;Sa! z{KY+ULxNCk>d91f$Rj&kmNUGIT;UK`VJeG+#RujBY*u|n60``NYzq!W0)_R{53kYnl@_YnMPQ5% z?w7*H|Ae*n{l;2Tp#qF3z8Cx&-#$^E|DVT}o4L7}vze))gUf%oM9ob9TG6Y z_uny1|6kXruFc4ZOU}XZESw?jjK>ZpYokKN5()#?mbQZhSa0rXS(el`BmWtiR0K^% zZiOP5K_R4)RTQBEf<~1BU34%jVa}87YhAj%rjYGk!g_H}DCs?wOgFAcA4+j!3Tlkc~JDhR27FZi44K)fN{=wIddzQX8Lm^oi_G*h**m1#ssRS-xm&!k>r zh%hzJo3zNQ-EhR%eQ5OU;Y7F`<0GY|UrD{p>VDs!9zX3t*5`o0eV81zdN@*jzTsoU z^iMt$u#`P;48QnZXmrXOl=0l1^;GQl5s4;^uIKtljT@Xw{y~Opbw}}D1+Xk?T6!W5 zSgwhp-+ydGw`;@XXz}Dq8q#y3?M@=xe-;b!u9y(8gtOBy)%Hu5uzIsJ%EVg0Q|<)* zE;VSi^Giazh=6K)lQOO=W0JxlEyN}Ti1t&2ls2DOzU83q{!XlKFW6s$|sF6(3T8)oVJWkgBQ@U+A1GZ>!t9cEvb0h zu0WMeDar+qp%PQj%Z;AqHW#VoD8 zHZpQByGj6kbhYR0OeIT@F-q*bX@z?*+`wX-B;*qoKQ+Xvdk3OrIJ$uW<_jvmA)AKX zPmx=GJN6sdgdZfvCDLxBF|J8RS|x-UaV~{aPOCiq(4EX?2o|NjEt$iVglgG3Ho5tm zHJ50!=DJ?6InBc^%2bpOe*vSve65nTSO;OST!(Rv7R`JJ55N7_JHSxN!jz6S2jPj3 zMTPdCql77EHhBhn)c{SvT6t)ivk8O@n1i)u03`PLOdYlkmhF5UC_l~?pf)hq$`kjB z3Tye=x+l z6bh#PDcG-y({Fn=F)2Ln^mj?63+hpp{i$kGSdI~b0o@UO8i zVUWdvuYo_Y%>h4i}b%NGb`mbr~fU^3*z%cF1fxPwX zMtEEvEt06Nd;=oA_+38mR%bLU0h&K~4}ZMXTlNy%p`z^%24q38TV};?olB%KcYhx! zTj-=}`&E!jhU4{UJ#~*xpRvaDl>D*khRm)!q-e5>>y*F$zbrjpgqy7unn^bJ>HYLtPnsrL zYgV3IGe*6tWIC!AR zGpqW*PS~UPkPWbt3pS@?~R~`PBY*_WYj8 z8P#m#a(;saJ_DY%goHMjcunt#S8^^OU2y zyhU=m7ro(?jBU5dcK2)Awv38)C7jf~K|^b5p9>a=X0!q8Q(5{Rv9!bkjJVUq1Mza! zdjWc`#?qbW-0bpcBO_7oxeoz;YberbX=G7D41Js|l z19|_p)2C>^Ps~2uDZI-|GtmT>vLnclGIo?*Y~r(otrQSMgT)pv8>)^D70Zr*Vn5Eh zDhHH@WO!~imhOltycC7K97Ij3Fve!@_Z$~AE%@7Cy2_dUKBvoLp?AJdEN65hy8Wrc z^+4(xkg)v{o1NZl6+W-XOsf5|94S|+f=2NdFtvYpo!KIWD81kGz0d=?IA^E_;IBt& zm!;=TAaj}jv{l<+F+kRCEE?~H-y@Dl!zz%bCa?Xr2T$94oxZ|q6!q#4YCaAi#ejxm zVYmtDN6?x{ock;i`y$gBQ=s=h5mXdQ;e?~{f(Wl9oY9heIhB2LCLxo|zy5fAKk_Sv1_N(W%M=%;p~xg@@v^n1_uc%~u+jb-1l9m4wQNO|)QAsz zx#7r1!imqkgH}SsjE@*>sX$D>KN#Yk)w8dbpQMAL;O44OxEFB%46ki=KMcW&Z?DV6gHTUKD%n^ zPAj_J@}@xnu#j-w4SOJ}6&-enQE}d`2}(-~=T&&6VV^YKKbwaL`kS+1iu%Raq zn~R?o$%s9wQ2Iya>Lio7mE^;|XzLD#JJ5a$UBxsXAVky17#9iB=`t*;7MmofWXtfU zey>MdVhx7*K~gH8uMyhd^2Xm z7ZLCCzItji)UV!Kwt4>>4s_ffLX-dN=n=m<{r~aNGyYep87apj(?^fwd7-i*Pk{#G zox>ma9n`#>7M!HOFF@U~i%eNx^n6$h`$W6>9z+7BaulT(|0zj~3g-wm{0>z^$M?%`1Fe&ddrLpiMRJ%ADsSeB6 zQi`AUz2n-hYmx_}Ez6`_^6Q2n@E z93SXA&d$#KVMpwE@yqs?Uk*3U6VDT^Ba_}AHHcf-d-+G*A-+!B(VH_vx$5?z-Kaw0*N7=2&RI298VOb`dSm~fM zY9t=Wc*;P2j=3&sOsb-n+ssZVX59ZRQ(CGS8<3nXQ=3maRXcSe`|%{RI`4O=XhI_D zjzz|eip1fY#87-TM+PPwj|i2ajy6@amPa!c>pc3P>1jXzP^D2$st&xd>Y=1j&-!^k zRWfhISyAAf)&l$8N};ti?Od$^CZ*MG`JqT$xzTKv&z+gixL!U^bDW)YHF|fgw!_z3 zgcK2|Fs}Xc+>Ei21nJAF{!Dz_c5EK1DlR6E%h-DU%|H!XhxP?#M84h6rw4(fFbAbW z)~;Oh=g9turZg}_2}mPz1Mh4ZS!bABn68l=aOw~*t?t-YY9lsLWb_Qf1GaeNr|}5W zR1ws_t}+*c3yZprML!)FPlnH592$%qz9odp@i$F$q#g{P#TG**#YMABtd%RgVZ@e) zphp5k;S)3vggiQR#^|&Wq8J2|z_4UEQX}^7J&D4n&6nqg1Sxyp*h0@~*$t6#3<1hf zbwTuu4`DS=LbPnUZ!aB^ls9B7{#xJ-JPUX+#j0llHM1LbOW5oj5Y+lEZ*wT(3HTqcR$zM#_`!xKRYWF%tJ0rw&4Z(aBva7$L?Fc3$gkqI`&hm)iJSLQA24-$# z$g=2(RbiF5@%=^wI!~Ko^bRrBsH#bXHQUNdSX=6YJjfg>N5|HM-cf z?5U0DLec9mW?Py|{;aiT$#$inD;8f7^6^m@pA35~F*?mEc^b+zfGGXX^B9KThOS@P zCVh5iR>1W#j{_Nre<)AGvPu);hp30AAuaDggm>?^Y*z3HHjqKv@0gdcC0l9)dMU;? zz*}qdS&N42f9v$IMMhz%eB}qmul&IFpMH~s0S*Qx|ELz^0smYkNQ@m5@B57w^eHx5 z6k1z(U4@PYEJ24GDBLNMMe_3#@3|;CUXm7zk87g!pu!Zg*#>R7z2V+TmHXp7>T92U znUU)1=IsI|JCx*4on)D!DrYv|3k-yZxK$VS%SD2Ff`0_;u>ImS9CijY|02`N_HD$r z6Jbn>H-M;qX{+y8B^Ld8j8cF#tePPmj2EkY6V>bX2j!+@K%sP}x`5+&jKRPB8pe*6 z3@xj_R`drlK{U7$q94imvqyhl=aop^;n+6COIBIwjdDR^Hr&cet7wraMcRw>M)iVc zV-qZ7I;~xe=ANDg1=HLoPe^XTCgG0lB;P5VplGS2QitF^9sSu9qT#Uy9DO6Kt7K(9 zjTVS~fmT>m$xneM)i%iT^Ksp|Q6|HSyJ^=Md|zGeK4xv*AJr6C))cZ{ueVsCkR~#b zdMgV4aB)IXYP}$8!DeI&vVQ-)iS!R|v|U{5Ec!2Bv~rPe-zfjXv-)R(P>0r4+;8MP zoRqxy(&nN-(SzwN#gCAMg%J55`GLjNfc4}oAjFGdi5Uf@hWa!qYshJw3R|LUA~l!f zK$M8lE18(jo3WXny07bCo|;$kYBajvdme72r>Bm98K^Tnew;o}cD^}ldwiA}hnQrY z*v-za?d&MCX-%v$Beah6&~SMCvUavjcUaj+p4f&GVcFQ;NlCZkxHtfFZaIe1DY}+s zi`>>~ZBM65j-}eWhIaK}Og^|xr2SB&)frGRe=}usePUvCiR6A5Ue;T7u5_rLeR9L} zUXap`lK8a8+1%Qh9J^)%K9ON}spV|!>}arToED*c_*ryH?a~2DGc@d;B|O@=FJiVm zXI84OGiO>QcH8Vtzi-BmK|X1_;wBuwf^EFtxcolymc>UDYgSN49GehXd!rzT8`4X4 zkyMX{b(7h%!6dM~mE%3wp&MI^XI!hajZQ!5CgOJ6x8;VkmuYRGOYZ8GF@K9l$9WE@ z;Q2MHlXzU+`h5CUF}N8sVL#)ljo{WZu<39eZsir)wZCIMw&%R&5Ya^?`J9#WScm)O z$;NU`b@SxQ?L`6EL5ccF8vTjUTOBLX9PV9k9legWoo(`_z;=$r9DOwzu#ycc+!dqn zDKhBwD_SQsz^n4wTJtR}-1~HAas939jm!0IW8C_RfZ0o7x4Gci+v=?+mZ7nIN|FNKJ$-WY6BH8cv@vtID ze=tO$psZ1lIHG|W+Nk`L|Iyu~D?~W6fe$@IML&xJ)2uuvq>lOBW_{%Z;Fw&oDqCOo z08;b@lz2~mrdoMZ{g$E{%;Gw~lzbJ8n*L^q_lL}Rcd!VUCNW_hgQuAeZckppD8jbL zTA|w3DhGjG&+x-Ygm_N5wuitfssXf!3Rw-=0+#HXmHtfXk7nPf1E6U4W{?m^WbQ%@ z-!yVxS!Jq<66tqq$tTd&@;K9|Aa$xHeR?dbDiesY9kMY>vlYMd9vgijdi;>UJS@!y zu+Q#D)td2aC@qXNq)?yJ(_l_}ED;}LXm2Qf2@Wuk2I!E;o`<6_q!gx6Zep}jBsBxx zHcB)E(xl%3U-a7)-1%4|Sw0{_4i4u^Glap9ca2rQUM1i95dv7TO&kZ3robG0`x#`o zMv>K*N&1ICTEsycRbC9RrfR6BC$1HgQ~jQ#DcYh!q2pC^Cveao&LaNd5auGbWP!He zmGF{TM0$E^uv4ZH5TKH+u_pBG`ha}GVMR^my+4G+V%x92%ulOz3oX* zDr2yb#Ux4^LCH|U0&3^A>Zy<-#L2B>d5->)tgB;!2Gt5uU=g68I?rkMF%#Wthcvcg z&lV$!1ja?+{=lP8b}9n@IcrT;$&z9cu$BtLSnJH9Fmk2lGcVl|AgHNmuPfHU7Z%SB zR{ONN0tQRX{@NRxcNX&F>Bel^I1DJ@OP$E~<%U_7D@rfCVAa8lC5q*w$|pJ&96T>O zW4vWIA(Bv5=^ZqlBI2uuFDeeFH?^G-9)Wg_4C|ZgFwxy4tHBfxGp>)5-GBWNVt}PD zqOr>9t^cD*)2u{9@U4l-#LSl|>{t5`$$1s*0P4=cK)_<^Z23%)$k_x6G9c=JV=&t=JGLl@5 z8ps+}cz++goz#Ape$rcK|QYWlptUmy)5+!0S8ew(HWljBq-dND#sD_4qQi(AHT|xP@`A>}31t*fYp6Zcr7Goi~fB_Qj zYm7G{Wuw=8de|d6W-OZ?Ut6JpW9ZaBJvr0feN`g9i;csv{K7b(Yj{E$ zBub<3q=;77dlVuCjHxs1BlT$m;=<{Y@rKeFa8K&=x}fIq9j{m|4r&lNwd+Ok?WO@) zZ+!g9WFnU*GG3_|-pmJTV#*Tj{GzDQR#5@!PE3`!tuzIIL=>`K;i6;%l4%xU{)E*C zdSqD3Mz`xI48z)~TCBNg8Vxfiz3+zk;4hi7%?bG z!wYVFC(XKRGvQO-qDy@};*DFsA_Zx}V7jv5<2!||pZca<% zWCLw8V+s?EC!VGIFkA~vJ?IegZ$uKNUO*O7m1s(+ki|_>O8J#!@%y_`0u8vBZrmeh z9z>@ok5jlJI}CSR60t8l*lt{ad5;!RsZnhq{2+Xm`NVB@Nouu#r2fcE7W-Ug(0Q-nd(r1jLa9I@Ls)!`})8j=Cdau7@9{jvcrkY1D4 zUm*_tAb;YmpMNPPk9lQ!iB|i8eAg%~PlY+&?{{FOz5IQF9bszLRCO>#mm+$IZB|tk zrdV6#0i?`bOd&(GsGp%sH!iUN`M6pOP?7&lS0-7Vak$sO^{Ix5b$#SD986ws^6^>^fl*|qpW zm+D>vE({d~*b8VWeWCgxX*=4Sd-~6w7ED*VJtP!BIeyYW#>$s0Ouv7+; zX11UdZZ<#1suhSfTAOCZ)MmYU9{DpAJ+z-IZ&+l@t0R^^NIFXN(z zK9m$WotP!_S2!sTCrHyxH#<0Tu)`;I@T+9YX7_V&i>RoS1p;3Zh7TxX$LD>G25{oIj6%F1QRC%dP~71of%Dk2;LAoX)6&^>@vRdVhQj+H=ZmBsLs$!#G{ zxQM&kYf1y>#^yVnvB8eprIQb`%a-n-?H1m%gWuDC3-&Ua*)#4-s^BCj!)}`TJ{+!0_vV zhN(<=lH3my9na6@133#29?dP~7e<(kRdLhV)&h;YnHkExD;a z&q(YR6v%4H4sx9Z6atKC+rUnC`x*FnY+^i2$uT~4ZGY}ff@Tnp1Bnb|%tcqbSK4Ox zn#^y*`04T~sgDP)zd)z9lw0Lf<0FC`eYLnPP5Cl76%!o?2U9~F$qE3Z`w z=l7hFN_Z%2Y2I1S)#2(%k-`mQ0gR7>1LoC}!$AxIc_{zqn7KM1utpMP6$BaeRO5MEF%f2E8P`R_#uvp6)8a-bA$z5&_&6%&Bbgr%uR91=kdwp3TBDa~N=->9dBjA= z5pNIfS4@S9JC!Fb`#2i654NfrX^JpQS-kF%!2de_?9iK_z!Dgieny}nBUqc>wvI0V zOyT{A({DAoxI`;xQ8r(CI`5@@P82CK7$(?>TZB7xDwf^T{%s`|K@Kd`_m z3}Q+cm|7YCX7md`iTtPyKEUn$mlV zYgckP9X&NOE_7$`66$*wzI()C=kvd^C5JBDvE9m!$_xgs)gs1K{{rq8o(|tkgPv8ssr03Ej~9(Q?>;h@Hv>~J=WSOsFam)t<0Y^zA!`khPXAim z_C;gjmYkVb`z9uHU*#NE+{rbOLVOWlr$O5^pMXV{o@3j68E)x#v`iAjkISNwkNV6- zleOobM{;)XX=>1Ej=}GqE?(fc?B4@AVVGQ!j09Q_|D3Dju77jH++*`ea8GGaap_w!;=uZz`T-L4IX`28da?P<*$3U~86QyR zhp8`F@PfAJ3mz!>N)a}QrntTCuwt85X2Lh#H_V)xwjr`XHU@T6EU*uVEq&%;-UUk< zH#|8hSiT>DHBN^;+?!-cPI{1?h9+RwS`<^_y=~B@+{gym@XNTA0HM=(DG?M@UIDQE zR-mnfU^7+|uL}^~eYU*o3{Oq&nZ6XjubzTo4`tbM0j1ChPn27Fp-tJrPuYH_?+)%c z|G8B^+z(q-5eWMV`93RKSq!Hgj`J|n)om`lefU+f4NMl(%)2dXf54OFUW$xP8rQnfHO!;+Qr{oSA$z~zyKeI2eOeU~w#6K{aqX>Jg z4U#nPD+Wb~rdNc;b!6u>qm?0rSRamHOUo0=;>jJBwL5*Mg>O*P4jz#q5MOiZN^zx? z;NM}*Z(Ci6UBM1}2yTj}+ab2x@YjdpwA)OA8K)xzb;C#W<`~QzDGtPn=WGX8mp2~Y zm`xp_kC|-AzV$(q_TVLuGkoo4eEbFz-HAl(I(V6JI9@(4Bvd?Zb(=fpn7*6)Ygbc< z)TiDgax#SS)zqW3x*@qNj=PGXz9l`WLM&aK)8}LGmm} z6@5$^VgQWgV_Ub?cktC(YR0@+!Y-&fq#^yEh^@oc&qHYicDx%LH+m>Y0U6c1G;i65 zV<$!SLyEYB)uqMoqySkrtZv4mlD^&F#rU=}!6Hun)f+XcdHR%%Gz*Z^s49NP;U?57 z(~8>s^)A20_w7qlRAhhA6*E>9!BjCoX@Ktx&9LKJ$1tHELF*xOkNNp=!6Q z510EVm_!XT&KVa0Bn;88)IEGY?;v{My{|WDxl;Y2js3;wK{h863j0CJX>-ee^2!Iq z%70}qPB~QNbF?Q(AHp{z{5WsnTt`XCD|CWw!4mU-YxX0*>YpQoTq9S@DuFjxPI0ZJ z#(RVY__?d`IVDC(i;f9*NXp8jp;_zp2aXkjXB$Dyg~8D<8oA_PALsU{vTn5lw#8dW ziibGPDKZzO%DE@1#EsoDVziucuGmTChvVMVeZS+-4dyzpxG98Y1h9VtYIKh%r;R7` zi!}L%>j6{p=SI1X)T3s{Gjh0E{4e%09oR1O;@`wHqK$s{7Co#Fn{haIvcQK2s0xLzr9F!4+MZCVf!X+5H$#1_!8SUT{`Pk7_Z$V$t}k7GEjC z?q$=HnFLDA4yemg>Nk1}*sA-Baud$;mYMV8Yi>%x!viMe7z)~*eDk|&vy1Qx9p6z@ z5jzcx5L~=7SSj}CBBeTe@mJV=TsinDukp7WXjHxMG6xs?6s>dYIzuU+jnsRae=}GY zs&7b)TO4#q{Ju2JnY!bp+EOd3#dJ`d@yLqGp(evKheq_-iUr29g^aO_m*dWcNG!i8PW6CR70H70*UdRL^=mi7?>=aXhc_v1fpxufE`(wm$V zx~H;?gns8F`;moMnbrCbabQmsLFbLV{Xn|GLBFy-*7V9ed46otLL+ql#bQQ6XiyUw zr|*EUqvhTB$PxZ#hr29Q`D$7;y%ZE+9634VJAtUJ1F9uLUNH4+t4ycm)-PJh!yMR- zAuPNzVg3DK6^h=Q1hEUU_z_`lON;&Z5h?1!PxembN1v_o6%oa6gm=(s%v>q2Zaqj( zrXKB6cYGy%I!m3O(YtelSB9=e8W)UkiGf3BYy)4YCE~ON{~~!kfT=pOoUCmjg$PH< zZw}M6nG)fB*$%taWop?iSBShBMwS|gG~<)AQhzV-Y(3M#cMOhGJ)6~253zzTXIw!J z@gSK|u`mY(2cMh~l%_cfyD|o5<;X&QmZ+pskTNCIxDYjKQ~&rAxH>G3?Gy)npO;kb zO_-3heU8Vc3Oimp8tqktd|9ZOSCe3u&?V@ggdO-!hW+&fr0i0JskON8pcG8I8P(r% z%|o$UEXXauCNdZWKE=n_PuU&WMOoou+P)<+mX5?|5k)E}AgdJ5YO;VnKUt{fgqeg= z7NL)dtj--b2#n`vo=5`4GXbx7Nd3lYS%7HL#nKQDSU6>BpNdJ=wF?wZ#QBX|)7sJL zN&J(BHPh702`NnC$cQR!^+G54MP1pg)3QfLY%6`*j3H`De)au&f%?)U(c?K{oa!=@ zB6V>gGwf$Ymc$Zt)|cIF>Y*Pc6AibsC{(VfO8{0|0QSxfhZD1^#TvyQHP7z*I_(vX zt!en`Qe|@RRoovY2r5S3fWwU1e9GZ#>-?iDaL{gT$ETnSui{e0y|H%R~o&iw#hQ3#2lg;5%JS~Cn(b=_A##H;_4Bg zUCmMU*dd=i!4e;8TOQ+~O4`NDd2A>bS13?7+1Q)hvDwt}Vn#Tb{+b=Vy5Hb2?e(z65 zxVu`RceRw`Syefj_s~nzi8cAj4XldEqu zikS_fJW#fLf)ew64VpaS<#PrN!g%}ze3It#yA5(Yf6^5(SxSF8#Bv5H9*!l*%FLjL zIH;5sn^d3Vt}4>4M?K7*Pz6O?46>JOn9TFL5v2{6%axjy#6lT3 zn(!Dl?)kNKFXc`vET~ouC!W%ra*n_cHhOWKH%_o;T5UJBBZzlpRPEghE1L2DUGc$_ zwkb{h;rerqHhA(-s;ICu^r$T-w!XD9+A5~JYf?by24@f0QW0#$8S7n7_nBk`Y)pS3 z1`P`H*yf=8+a-mwemC&W%=!7{aFhK7wVk`5p0vhm*ALn~I{Q_MYZkn-{GQN(kC2UO~i2%dAlOL$?*l=UveoZCDC+&jO-zCd>5ZWK4q@f};<(`8lg z(kJ{1QxN%)dSw*i%tP63D*XKpB&-DXhz06egrYpn1$J)V(uG-G>Qb(S_%;dzK0mQ@&Pa$vi^aOb zalbLHDG~65DnBx;tcmcL+~W(hIz%9Fmfb`j2D=}ZvIJgDrB_wNsca@H+1Nr7X+t3* zv~}8a-AC?F-9++JhA^g#a;Eoj4LQHR|+gRZWqGYdPvSHI&b zQOzlxfTvu9H%NP;W_X8AsBGp*%sHPCHg3?D*!|`W_YnT|OK%?zp=NRGPdz(40{`I7 z`~J=j6c6Y{@|GvcB{@8(1Cb=Qb<03LrAyV}=f8;;2%EvxRlimThrZN6r2p@^vw{u4 z;J-%Bfl3+*v$9AZ(y#>q-x|Y8pfKn}=~%;@PZDALh>sLgd&mVinr!(|X6NJv`SQAa zAv0qc^O>}@J+4GJF2wk$b4>_X59>CYU(c;(bh`Mw!EF$t4bO%KtnRks=#xn`W2b$? zdo`tW<+;Cp;%)KRNY*5`k|OZelIThE1GsiFBngTJoP>L|pzGmAIGr8FRd=Mdn*Wm9 zJ7X>J1Z~6t7PQ74l9}^+xVAs_;X)6AJ(vwF6ESJtxg9f#0!-y*%Z?89>S&Mf_3}-w zf!kN@`-)2A3zRHXD{e}e&{e~gr)gFv7w|#7PgY4hwyC5+QE`ppJ+Ft3@pQ}!$}P95 z8Vv0Y6)@1#q6p{yeF)7(i`c^X2`_1uCBVY3zacElqI5Rqw}l*JI^T1Z&g~_#ThTD+ zZS+{vNtvXXuBxb>rR2>@&DXz5h9A#ihExx)ZMpGNS<^rG)v7jm+%#)vzP7CP*d6LO z-{3Y*7pd8VBOM4!GdFA*2>xB3-j4%ZguY;lA_yBIAEy`xSUZXb{%DgetS3)>fNn@n zw(aW&9>I|D0qdr}@T19+gH}hYp*6piS#Y+RY7GN6NDrBZrlCk`Oe(zWmc=U0`32rt zh&3E;=wtQ=G$MM&pFe*v3}DD&ifbd2d^ivAkgLw58agjzz9ps<6-m@vT2(P-m@-+n zap5WEVqI@qjqtreKBhoI1Ol}g8{F0j{Cu54sn2B(vh@?RSKbMILlh1nY+@GtcP#sf z>gFUa*9)BDq^8!PqGF|Htx`R9?#?Eyd;#|9A%NTT5bwM0d_#r1fp5}rJb_jbayhf? z(&VNDL8uOp)}+1k)}$Z4Fn23)glxu9v9 zz)l3j8h1$g#L^w9Zrq@nE}`62lE(KyVo5U?XB}KtxjQc@ZbS`gS_Ar@lsYLh|cwpe}#n zXI{g{>TQfMwjysr<>LlC8E&F%dXq2+Wsr_qah_Sm0ZZiPQdD-&X)$e*KFk6hLS@Gc zWw#BRLOrDl-e8KeECtclt>qHn&@&&~Y2W@^X#Qg*q}v9b;^ZqVcfVxl{~MI)3j-o% z^|defKPk{(ozYbAKG!_qm*J4K11+h)uk6$n!c7II38L})<$l*kndO&QU0!ZWCu(Cl zJ)`C?Z!+_oTQJjj$tf)2EkC@w6P4hfw{PHk_V0Q|EX{UzUeaPshsw&xbnm(xdiUyj zJjnk17~A}Y+f6XL^t}xdbhtd)ASWhBkRcLG9EPdbscI!tq3xM@w4`9WMO$>$d4mgc@SXA0B(8ZL|F$`W)_?#g-A1!0ov=5YqBtD2WjbQk<>syiC) zO_fj$_M8w6X$X8rscW$M4hDo_5!J<|7g0P(t1*+qRB1hKU%3o0c3=7PUSAHPj{bOY zZR(y!4ePP&-i8ydKnpu>;nt~@7W{<^9qnwY$>?Ohj$|>dbUIc+K_r{=_w6o-xrj&f zhRb#ZN)(fE-hjPqPc6oqkKZoH*CwiXCP-?i{e!CpNAVVdvsoP>_DWRMMTC1UToF2? znnGXIVNh^N&{>w6Z-0o6F77G0rc73a$_-;m?JYo`Rr)DPE2|Myr3aZ2d7_m$UgMYL z%1na8TMj+xxgVSfedSrJ)3Rp}4E|(O_{1{!X0=@v z9H0<~u{gIK387T?Z`_^X7N@#@H_a*veMS(D?|ZYkDxgd6vb!t~z-HY#TEudYv=85R zZSSS6>x?Pprr2OoaFlf7TNxH}7MiDVLERS8rCLvRZWAS0OFU^}?2KXgsvn75qAt5K z*m036|95swf`-SfCst`+dc0dE#Xef>%ULnQk{PW01db=BPQrfQ(m1Qbcm#H)^t9?m zoSMr#!A5~f3d+dPQY`4_Jz-3iJgtq2lME(-njKRllj`P9H1~0g>Ow|GLJHiQJfS(u z`6HIh%&%9LQLCbgi>A8s#0rIH`x)q9xc1=(T&Gbv(0j=?(n*BV@8CD(_@3?}aMdV- zlppx=rP&XszCVW?px0_E79V~^OK}GTOL^d{#iKsw_&BDsm~Iu5Y4?tic?GFRp7qnB z@~O1B&s|`;%y*lUX%CTtTBo?6T4TGc_5<2~UTu9cNhbDPrnO+Ufj3prZ=+43qh9`E z^9lt=Mcn!WdYS$uXCLor(O*bvbC4i4*|9ofy!wIKHE=1Jp$z>FVI(DKFxY)5<>A9u zV{GLJvGEsZW(#EjjP<)#kn#p=Y7UbtM?9IA#aO|5Sl^gJIZ}9^;nh3>EMlB;1}(_C z0d1@6Y+dRfkQjP2gX}X3%W5qePd6GW|0Jr#E@6GE)FY+euoQN*wE_=k8}IP_?CIq= zRUuVi1orCAeN-Ekj!mt3nrAd<#M@;_JhT8O8J7<-H>+f3n;eUO#bsH@Rm)*sgL_JQ z_*v+@dv^OPt~r!z%YoOvjW9SAj2WV-F7{?l4PjQ7JSs7Au@)Vibd;y#)n`s2vhO!f z-D0?Db|v99eJ%SLxvq`{#hdv|$`D4Tq^X~<6uzidKMGP6M`&~4w;I(zazL#W+qy$< zZbDrCv}o*keclzE$6)uqqnvBle}oM{h^K9-n$!v&4i)O3Bll8*^^Vi(_^7=)z{HwgI?SEz!VAO25iQior>6(CQL ztYrY`^p-nKH_VZz;P+5)IOdx4VDza~0*&7%{-1B4}aN6A;M6G!%1dD&BnN)XWs(+(;^&g(_QPW!0iVk|kW-*y!&KEX7qP9mBWc?4tY z)!*Z(X&ZUpb^cWkvp5t_BKuX)rhlQ2{z?&-4;fC*e(jyXc^BIQ@(w*fmDFFz-cj#nEsVyC#0;Uh%oCk)GkP+mD zJExM^MXh4qVBfO0QrAT68GOHBBw5YT&+tetQhYJbE+}d*w_K&;dcsA5n%JUcpT5Tx zdwi!JkG^`MgtB8OLf6l9PXJ|Di7UPu#`f`BY z3htxFqoizx!b}@acuaq-#urS^>_yG1B5-r+yTz(B{dSr(nIlkaoz?mcPTNSS93NSx zr0rQ{!PDwV4%f*$>bT-GNIn-f(ynFh_=magfk#5Rk;rjN0j9kQ`kY$t!+Xyx47P3$ zqO+qR0lW0|LLO#s{U)Bsv02`a(R^mI(sC z@Cpmy`&XOm#d)?W6_x$w^c;OmW1V7Zu}T%{5;99U5y2m|jYrK5F_k#EuKG15$1u23 z`r|tXz!?Xmjd_0HV%UCWq3Tn8OI7Lk!HEIag%#jI82`Qs*5{g*^9#q_w#*6V_0Mf^ zFfl@$SBU_hvbL^K3NW>M} zetKVg%>lI;H@3>@VEf+`43s?bX6?5QgDH=q2IYnqJj6djnvlnB_z?ZQ;4K^WFw{{% zu}CgqNyv}Rh=WA)tIUba_4_k|usE80v8t~Z(sJ%;cBrE~go(qP*0g~vn=GNOqf4Or znj~j|F#?}tI~c)4hOzl2)bIuYNSsT!8Dy-<8WtGp7Bj8e*7U-iM6JGAPE&$Vj7!KQ z9q1&HR5iV~%)(+XG*HvH`3=?>N17(!SlPOr0vX#vkRC~zWW6yewltd6pJ4wQn+n5! zff!%Q=zL$O(f@~1{{L3A|J22h{NIQF0dS{bj`|1fgREIfe@?qy$IP6nfwu)_M?(CM zpYQLQxB&ia!#ek+i1jnt`g)X2&+E-2g+BuXThS?Dif5o2+~fp-sH4oJ!raG0oHhXB4(QfU7)=%_ zewBeL0D@En;I~G&-b%*8ED1dPaCjph;c+~MOPdxPTdWr@`Gq1`+S}isi^5oBxs2xu zwm(zxd02vqLXZ11+>BPZic!KSRW(?~?V&Q%B7JtpQK7ltv`j(9jhUiR@^>g4Z5Dfq zejAkdiKa>30z5tt%j#ZXNj7gNqU)AGI+J!mvadbZv9DvQIFY->ob zo@740`M!H=qWjLjL9yO`Oa`H-)^ErP_h`mWpa^(hT7XRievk* z9#FCL&Z%r0ik(c8A3)U-X3je3c<|Sl&1QB*gMG4Tq#fp>rfAUlU*EPl=$=fpuS_ua zl?k~1_twV$|84u9V0ao|z!S7D7@qP1u^qAySi62}6`SC9IciyT@+_3YWH%j@+Hyjw_ewI;S< zI=mC~s97%h`wxGty6r8$z;IS-%{rTbUV4G`mMjj673)k$*7q zF6=ubzZiLrmx|1WL2pcxqX`h-Y5Iga3_{Aj7S{Y6z z#j*ba9--nVFOGRM{EX$g$s1Ee@cA`7?$ecjXXKGjvV)h9V0GvXJm)nl9GhJ38k}*} zicLcApe@eOlr$VXyR---xHait=$nY?O<_zRv?HTrR}dv{B$Hl$eL#g~rhrUKmIzZN zYwf}=W18&ezcKS%dvm!Pz5p8Q@*0|>ilCK`J?L0r0|{+adq!I5j#?M zE#4dizj!7}1S&ZMxs&BkkC;yRd}RBGnR8oozi-p;%}Zy~Uk!zHCa^q^kG``94R?3R zS+ft;C&z1T<@aVZW5ncqM^z3ix8r}=n4B*@UcJ5FxqKdXA1NPtX!YrEYg_c64Zmw6 zdVo8kZOqVu>Y_t;a4*Pwfd4-{WxgHM((5qWjxYzd*iKUIWHZr@UN z`p}g

$SkQBgj0D_<*lNcDV~Yk_VmUeSPWDqrCR;l(?3cO28(^y&6qpSX4BNUg9w z4$`bYwocY)g-Ndb3cYwqP%ywsv1?aXbN;f3OolJkcDCrU;P~2wTpT4~7?r(iVHmro ze8dWx*{M>aN_SkOOqW)}N!btDRrb;S5_RvYf(b9^xX^>%)Ai+g6gtya z-3;RciVKB$NT8D{2W(2>;f5?J6znUNUYi#4XM;k};0G6%&WLz+M{z41Gq*P!X_&@5 z$?xFs#RM${w45f%?BLvatj4(v(7lBblPInt)@vu~Tt?_%->3a6QIgpBF9VJNK&E^# zIfW)82Riz|{o-dwyU#nC`KbYAX*B4LKbvnfJyzlXu@tBfc0PLen~&60bFx2V3LRcy z<oTCIo{qwzB_N@l`vFe^8ob zGo_APZbBVSyF>Sg9vq4=)}9xV;7y7gL=sqSHxwqtl?s;t_6+rwjad2 zfx5`{5h9qe`@D`YXPSV@k1SMv=d&~KZh+*^rh4iOBG@8@=j)GVZjb20@rqChd3l%( z&6R-(Ip)YbgL$p~qILFU@YH;KFf1Jl@%P9uzoGFNu+P*M9w^jhMPp?+wUW=K#1w78!*}(F1-){H2=`dC zydhZ6fD`|uF_XFB*gsN|J7Jb*Bo@OYAr+jDjYVEDcF75M5AtiqirUa;k%w>W5}oP) zYD}q)-EBJTGqUBm`xq%?&-AqBXy`eJv?7gnUr*h(;Pt<1R0-9eDVD1VzlAD>rEj$; z6fcu^R<>0b(wel*B8=0yPOy}cesIoQwAcR?Uq<6l9TU#q6Zi5^LM_}KTkSU@H$D|O zJ~BdJubV4`99N>#3~>rTgd5_r$sIQ&(Q#^|$SS!LOfrGsQpUO(Ug=bxmwKcx_lUyK z@7H!-VcSxtx7}uqXkP!(2r1zG(2z}HeT<&xDI{|Q@kg^5mS0K}x3rEx8Z*x0+3cbG z{#EIjrwR9AHvNEi&jt}hQ8%EaTOBy43TX2dQLUcy*DenvieRs!2nfNcnGddP{_1U$iHDaj0Wp-dFWO3NV4#JDu5wjBWhLbnA99-+%k;z> zspLFyhYaS6#nk(CaB)5$o8%rAW(r5KvPi=xNvEM{aTw6A!AKc*4#gtIu%x#PCC`qr z*!dVeKv(NaCQ-xb3S@7g&9YFu#PbfI4dM!BGkAm$s&_3QZ5Pt_&kB<=q~t;cUkpex z&A~QA(9<2S(dFHh{LUi46nwJ>8VWqfpqqxt@6BWUC^D^x+kdj%#j4VXinro z)Wq{>8$(|DQX&-9!@X=;eShD55{&SorfrR!`b-_w!_g8UjTZo&Z;4=|4)L~> zMbZS0+L(G4hcUKLY1Dt_O21#aU(A7+jRdr)j3Q}&8G?1Mdka2>-5HKA?gq4zoTs(m zUM>`pnZP*eL5>%`&pDC0ZHP=#J<(P`hJ8Z*Yn>hA)AxY#b*nr3s?1RSt<~Hi=dX1C}Ptuc1)L1k-l6o*v^%xu&@G0dz)+bzjG2VtO*=w5~8?l*9o3#@*qN=7uhKdAPat$ z-5bAW##%RX25y+lI6@9K-)Cjb=AH|`<)7cmlzl}>orj8-v;UeH^x`0{eAg0%6P%9s zYhwD%9dr<`UB2{QlS#u0$Q@HksbxkJNp;R@CF+d3GkDf)e+waiPP1%YgV_63%e~vN zWb@QhXBd3u(Mw*RWnDHfrLA};?gxvJ%9?8^*&pr)q0UkpTKO%+O^x4(nP+W`|O=%1!A5oF;#?u z!a6rIdEGb|CTB3h;asg$N*chdbcRNj26dP<-JyxAZXlAhdf1va)))$eQJmL z&1^}-({gQ6z z{itEVMGYmn^!)zX-BETs_W^8~?I_`$a8;Cc!FG>LdgYNH6hb*I-2;RQholVSAr>8Z zN_Z|3p}33(Abe$I4kdRIpb(~r4<%!$*MT_>YzLP;n>9JlQGqKL6fz=NAwc#B`_&Gd=Rau4Mh+Etf~t2b;0pr^-%zv;D32G*g0@8*$eJ1FI+C2VzXRQ`^vh?#i`btMY>?fjD~ie{2fbAh~xuGsh^fk3{GJc6zNCsuPD-WVJt;ny_LL6Eb?daQY0-^O-Ln_2sZz@oKjuPBJWhK=NT~Cm2eJy;*Kb5QlOAcNQ;qQ^3IZPsJA;#dh}l0LuckuIte7{s2bPokSm zx-}!E0r2sb*xpB#f=tXBv+9+4(#4Ux6HhJG)ES7bdFmZvOv)r%d>4O_r~8?DK5=mt=>-|$0! zLXu7pD4<}?Fo8C1Hvm6K07|efWS7$*8ubIS1UAVM;7I>|hk9mOPNAO5E(WQGO1TtT z7fon9lo%jFJxlo@S+)h;N70BzcW;4gWD2PY*(&j(O6cY&DG7FX1-^*!tHq8h$veWz zzg%56_tSSzy=278cZk>E6lP7~66FOZs1NkD1e%R!H+5v5r`4&vP}aS22QC@8qyx(I zknLaV(cC%3x8tvRr17P&e*@0_r^1A7oxjS^-z6qvJI{~cmBG%+Qfw^<9oU%Y9XEiY zni)YUKqy-*k`M4-AF|J)k+tG*A@P><0aA<275Gjt^-jK^F)Smc0=M$}r^)+P#%jyw z$MKuoH@RBlKHc^#j;fM&;a(yfQstuR#J05#oERY*} zNni9JLyeqdja)aYx+y6}C@vAwggMrg;-vg)buan)hEG^d$DBTu$Bx8}Ae3|M49~+x z(EP$FZMf_~un5ysk-@FX4ZW<-VD$s*>;B2pR_*XxK6)_XpA0j7TW?+xCe{t<-yi_C!{pvUk7*WQ z6LXwWkh$QnQmOrnJR>Uuc^%TWMQWoDI{Asa<4V6?F^Z@_m&NU}Kv@}kweF+D+j7Hg z4E%`A`R=U=Hwhyxw3_5{@%#1g&Clp&yz}gU_pz(J(wk?P^gjA37f&l&m=>~>{#7^M z(%#@UzjPz{zpG55R<`<1f0xY<+j)8v9@z#XX#Jwi=M!Gvrhul^#?WH|;?l&DS^@@V zf3_CrvXaK=IVU8(Al+KE*e^r^LH3J6sAbD}xH6tN%G=htV@yoU-T6}s{98CN-6ce_Hb65a8?zk?OqV>#yaDIF+CnsuQ)zK@&#-f@CFJwqF@8^*V*0L<>-3D zE!k2ha?<% zT^{)in#?;Rab{{g@a&A|%g*HfN@OrW z3H6MQ;VuhDdk|V7XHo5x9LY3FbD!QH3$;}{#^^COD}!)ntXQ<%7mODl)2A4nK0T-E zYPidK70Xy2Gmyt)or0{!bcd@0yizbI30(~>%<;~gq}UP5-sT!G&8#mcS@le*4@x!o zdJ&({5N8y=6vJkn606WzAT6DUsZY2=8HfN~J@5Qf4 zmCs|yU0=C^X0$Rd7DUt&1eHj&qM+;PH&dg2Dkf8&{80-CXK4an%5Q!zy5~vV@JIl`RmXl`Xv_n|M@WeWnTZQRH791 z|7v19bdyNv`xfV!6`M}ytjk0CH+RA_{_-QNy|a1SI_oX>QH@aMoHr%U(C_8B7385g z9}0V3H9yp9`;dNb{bD=*A_Pl#|9SaD?oqnnu#>O1-}dvHBSceBolO^-Z4bv|cMFBZ z&S0g4Sz=mES#=E7Zdx*;Kerd7KLA-p59!yDDje=aRVcz3u*PF?r&QskNY_wI=h0z(m zdG-=~IJBT)-#^}napSv>(KOA^QfRp&#VfYHm81l!a3tkckfF8dGhRtWTsq8Rmrbn1 ztc{?tqOzMEgu`oF5m^+S3B*?O3M@S(>&LGl4>{yBMjYT2tDmNN2lL&}YlVJaSE zu&{fPOLLSNmkOjNlTd*Wg(9E|mY37`i)9tygYdRe=+o!d?w~iuplPe{sCe9CZ3}wz z;d|>6xS^JsQ)Z$UwnQTMMe9}%-{-0JVhb zX(3lfW{?I_wg(vJ8?{PZ4=p%nfskASFF~wa5S4ub-=RlBuWZ}x&pQPc`E@X9N-0%6 zF8{^ z*wsN?tN^=X`Vf-nWb1HM_I67{)FPo0oIb~gAIk4~tQK=rYvjTZNUil9@+ZD@@ zQ$MR=38WHoR0cw<-ZYlC!bz6|K2W9t$t1<5%j}j}RrR&~RA?9SB zdFp;#mIFjr4tA)~{MM41YIx>ZwYBDr0nS$R?Pq$(Pq#)99Z=xZb0D`&of9Dbfu z&t$xm8+N6TA3l!-+Byf$!St~ZYLod#W!{ipDt!521z^-jMLn=z>pvT$;NkWy%!#Mk z0f?`%G9-sGrRO!66qA5x0=JPqcP87oHKx$==Wnb?s*8^7;-= z|DcrrO)@H4YRGC|doV%LlwHD5q&j7-Tmm4X#x>Oi0C_qVkkq%lW@qssBW9yHBf^=r zw~yC3{Mi)Vg(izq=c=XS&(dmdQ^rVr4ApUiy6H5p$Ir(Wy5oaupU-FH9&&FOg2*HU z%^Q>0yd z(0T+^0%?MB!HtAEygtNA&{~2e8G{`3=Arg{AAeI|M;deFySrB z5R8nWe8;$xa32H=-#}tW48S`?2H+LZ6ukHoL%M#!0KBgGN^FW-mHQ{n-##K?E%qyHxSFyx7!#}^Rf#^P(H|`Hh`&{eU1Jeiql%i~;=uhV z)&KhD7Xn|c`@7}<5?8Aw4hN>e0{Yy%CwDfm9O?qVRKCkP(R6{Om>pqrvu(&k6Yj#e zwX}*X0#%8|F6=Wdxb7m54?SLAN9WXQ}aE1h`BX;&Ndf*BxEVf!{rEVl4_bd8gY2) z27)mJwH*vkr)c|@3{Qeusk`_3p&m5kxOIKT1)*WfrP>)M)Rd`JV?Uu}}tXDVSBG|pjPo|^4 zIO~;3)oIHi8>%N~1q*-K?Hv&qsd_NjN31q@202jUcg`&59Y)nSk(SxpI6y<_4Fq`P zL=@hADd6!In_cV$>mh-=%@>lCok@wLXY@YK1t@1kUyn4zTf zL1BK(hnI_W5sd3eT${cuuOJvCW@Z&MLp{Y~ht_eZCRVyW4-1jZDv-z{4G1gX*1Z9m z$HbLFKE`?G$F8za0kCYrGvnGPGV5yY&2?4YIuEm>Q^yuR^$7xWyyzHUnr$g#ihwPv zkqno2+GB>}qS!IOX53?@Et;+J#`EOY>3!?a zXAH6AmYH?;#$eL{ukyT_yVck1*5THQ-PVjB%|Ko=0^Rx=G=Kiae0A)f=KU+DG3l!V z|C^oXKLSntF0!3f4sp-=8yNk(1PIs3##LYJuWKs)oR%x?R z8a<$vB3H5W4$`2ExphW)w%W;DJ|v+yUK8?_C^#W|KF(XrWFCfl_`o#Fct-lA2cPRsG5S-yHf2Kf_xA(= zE*l1=D_fX8{hP7yAT3#>1{fC-QM`&>+|M;nFhxCbT-+1yB?T#j?4IaK3>7YhlCSw5 zM^6WNCR}6j*?*)*{H5kmFD09TENd~AiaAg263D}iKfW^5Af5`#sbvQHyjIY#>@d#$ z%(4bJ289EP!91ET&4Rz4PCMA| zC;b2;Jx~En_(=FsSQ-TE-V@FQ?${;Z!6Lc~9iC$**N?PHqWJARQLUem^~7(i7^hvT zQBsHFn1?J&)Y81qGMa%6q*bJk6Q~)r_GeroNRcaUMPTB|Soh2;j8i}$Twc0bZbfU} z3|n1tK~8CE;}l2a6a5MVHD`{(JJ2Q09X^@ok204M6Zx!(on*zTKj8m05q?uo;#umt zwuJohs?NWJ^KW2}|0x`2yMOwbQHlzG9q&J7*IBI5sw_(!GtS#xrkEYUb4vVSzy()9 zrNFjr=4ZCzNC{*TKmM}J0qZFE18lwcB-t6p^MEWOyDJX1?UyOnDIPu_uMhAY1{bHq zIz>^a3eGEinY4DmvNJ0zEH+j;mS*6bn=ZRO0xoiT)_1HmLX@eICLxYS2tqU$HYgsp z{tKjVxk9>q)4(E7%>$X9e2=a2ohUiXc5hs^-LG92##Nd;f4g|XTv(04CJ={s^x zCsk+CtNa9h;f%k1;m7{t!O+JIzZ1k47IaOHxbe!O({|8t&B$Y^W?UMCQL;}W-79K< z0Gc=FMc_h*5kjb#Eg6rzDBUh%TieT}arBGiV8Lm1ykWZE#3O;I_i4a%{1vAD1cl=q zv<-izif*JrJjj$h)8GoZ-=Vbjhd`V)E||^-7qPZ<<02$EYO5b!Lfi;+WQeH7UtwLt-4CjA5=xR^Brnme&reEd3Mi z+VD8=AWHpR%G6rXR0oB{%io--2A(PRUtfjD=u1!k3#VMg#?r>t_3uu3{9jHvO3?Jd z^ni@SvpG34b1lClOkY*IB_fib0V)uZwWc#0R5H_!8}%^wEjc&PI^lzuK`+RCUilUm zF%X(~Z$z7`$rQ((JG0N9%?)6;7*uTcPFlhck&IvxXj|$9P=FSt@@O-E^Q;y0fy+Z3 znY>Ne%(~ngF5M?J7f{tivy&RQH6~RyJi4QV94zQolhZ{9WBg|JvFMnq3LikbX!C;^ zQvHR58}`BdnMD;!jt|O;8Sm%M-|FX~xJ0AXS7MB;217_?{Eh}8{0|J5BFuzasmw;> ztHqgnaleJq3K=vu!{S!Q7nr(I8jWLras>z-G6nT`7OZl{na;r)opeBCA=hNg2t3Bg z?pL>zVAMYj{IagE3k0r2N>-rVr8xu6nC|cdI)(yah~|FU6Wo3WVzKDyILh-Jru>!1 z{X8y6j~J!I{Kl^$Kgs3uy}E2PSG+(5wKUE=?PsnNSs(P79I>;qNo-EqGHNvicY z_~Kp|Na<2PnjjXE7@QYN(LVo{{u;w}M{@MVA&)`(_Ko=ef7X{bH~owA`yY#vCUp-_ z92L}0lY~j)1PN$!^NvD_Q=y3lDT?G;^9p8bW`y{)!WiX7Sr@Vlb2g^^D{5=corn;4 zWzXCNh%SmQts)5_1bLvQuDUJ+1<&qpf9n5~ZU1UmwPNj9ui`WF{>9{Yyy4h&m*qIk z(@^rqaU0|t>y9o~wO~7VR;vI^2n$YcAcjuGo->wC44YXiqV0lw5L?*pWRn~8R{0(& zDw}1gkY+%%R^Uz*<(6s$^sNi(wfwCttkmE*Nvq6Y2Q&3>$6&t)^G5Vjxm!;7lchQZ zG1d(btm{sI!E+Zh6b6~h!_^F)vGc#|&UIXRT(oD3W z!6J{rb0!9F%^oz?hm_iW#NKO>g{OvgH(~HTrEKYQx&N2~l|E(ZHJ90jULbNXEb zG|&*?xa-K%+AH%~VtuyS-^8q7y^iNDba^qJ4uOfAs57gdRmcueI#MypUtR$kz|2-r zlWNF;hFYp%Z>iLJi?dV_=wqVT3k+d;x@L@3M_?L)d!7)lT@XoWMqcVlekp0tge#U` ze6A5a$q%ZOoj^R;4=p{>AOAmw7-525g*dAp0F1_?a|Xe9sS##ZV@e9EiJmLd zyP+NF(Z-(kbwLqx+`mY@sHZSj)*w4b{yV#dleS>}=txl;p`qx6{YI@DGU=28RW>QY zw(yP~bIeubed{jiq&lZXaE-m_kw2#N3sgNTe*X@&=;?SVBI*JJj)6H%nb-HrHuIWc ziD+!WKXzSPh(td_I~-l|+y^3y!CdB}9YKOO>%tNTt(Q%uQ%CLQq{W9L2Yk;HvH;gO zu`|S2z}Ms;GA&HJHtyLIHGt`z@^Q&(EJnqU>f-m-^GOC&aXj9X>yC-igvd)7&8{-z zJLr)>E-hhJl#%>N5(jUpq$;|Mhsmm{W1U!NmWJFl$M>}XW?LHyWsN2QvaHB-p8ChL zG{qJj$g`F3n$yr+sbLc<1^b%lgwzFwAf<_|-!9$Z=D1J#qJtltUC|JMa}dtlfwra3 z#nHCAyD#OEM$P_<#|Rgjv6j`X1RKoG&-vUj8YFNlX zkLCfi6oRFpA>=rM7ap8lG5Dp=dC~a0!xJHQ<8ELy#$YPZn1 z-6-(gefyl=QDjc<;BsfrG~Ok96u8~P_)hPLJ_Zj|=}(-Wks&JYBE!13t^O)S=N0}W zioVSEa7LzJjoqiuT%URSIlyD{>;-!6-VU}>T2StaG*d>-h33l9gjqTIM=fkGKZ5w~ zDx;Tu=PD#!fdI6rOw9BSGm-I2;U@OQ2Q`U0cZFvR`H}a5{%ftL4{H?DwFL_e#cekd zd@cU#4O6V}wrpZiPJR2_4TU55Z6NL6KeIS4`EeC4q&cPery1Qhs#9rT>Rl#(9+fpj^f%>7 z3P!qMLX5=7=vE2BM%N9ZUsgL|rVD(`1d}V*vKcM>P2HaWR!w#q`DQIX3{1SrOZ>!q zbCXZm(sj zGeEM!0m*MEJfYOg8rhY~-BpmXEq#BmG@g%1#el~GVTe#Wnq(yuwH0{%xp-79(8em| zNN`1*=3RYdS=O%N~I1t)utj?Xp+C|c1{QW`0LyGAUr4xm2pG*G` z&j78IOD@j=srM688MdWKPLL<)Y{>($G*wWhP%#Wt$g>Jo%s<9JL5!R|YVBL}$J1%_ zzWbJ|KoMQkeL)yX^Xw3*0is#+t4qBZ=WK*!;Zs&Cn%6>pe8>5WTr$?IFVIDynvBu) z;fe?y1cipgi_0f|pAEiGZiX$|?c^#PzGI;S_^Vy0;pPnrnvA|Z@rpM9ojVYkNF}e_ ze1R&olu4bDJD@1B)eJCBxR`f_NsW*8*(|3&NGzDASNe)Vn=KHq&=emZrKFBJE12$y zs3_Dcq1_k38dfYWQ!2_`a}+df`F`3IwVkC+Y?d(uZKe)?I^U|?tH2!Eflm1@ zZyCLyP-%lj<3C#>*koXR&H%h@I>U=IS2A;(TjTbjVD|ZAh0SdhDqjCi>ySb52SLAH z0w;YBOJR#2Xbn*lF0nH-)FoL_hlEik=-NsIbHarv%)ZIRu(cc5RnJ2!bWNWj%^Q|PWl&ZFX`Y4>B~RE^rTwQqGMimW5yNoQEtH+tXa& zX#?_Z+0zT=cIp5V*l!fW~pd7Dyg7+x;QwPd$cmo2)3Ca1hUeAQso1qndf0q zpqP;ZA3CIG7MB!qm^%n+T>-Xw?1#uG;HMy`a+z?0Fe#MM5Ir4+*goK+<)+x*zn!;C zO9+W8hTQI6{CZqF|9#oi^=rE3&-*j7Z_151i~q^pt~#nVJ8cNE1G`3m%8chpjUPM% z$AH(27sd^dLKGIvQcq7TGbc%r5o~Z=yp)opIi^!#x;+M=Qq<4;m$;+`l8{XNl0)vZ z;nYT)ln8tLsb@RY9xm>=v;`#K{2zKcA!%cpnqy~+@f@2xOiYdvJ+=eGKj$>1UGE^# z!MOW997l<9qs4946?4nRCy4v7Ccz(Wa?#i(omS}kLl}dL9>^Zx9NDXk3ma89MHKD5 zBKug0jMc4yuZD_sfQtGfnHKzMiHfc#VV};^O79UMozfU+Rx6BQ+NP*bEoVjmjVNTY z*kQE{nX+RGZ;^om6j7!M^sXusqWNMne(gim1ec9yvKD2+tsOq%F&bJKgxsmUa#Iua zZOoqCXk0}ld-`6|F^AOs(x+vhx52LVzbvVXONwOi7z<1tB_b50-F@qYH!x63r%p6G z=Vub^@rrn3LUvp&lF=lJ96<;AS|byC@&Oqz@*`;vyEqw4V+}}8@g;`N$CaykHcVp5 z5meU5O>xfYPUGdC6_J)AN)rloF~ zi;q!P^@PFBNeuq9LV)DqN>XmrYYG=-E`!X|7$of^oRLTRKD8EH(AL<<7gHJz{TtZ& zl`j!V{qevJ!hvEOV=01F4M$o+U0@u>qAoRi5}YQDmHm4gNsQ}g$dhtsOlUZ(m6|=K zhi7a49>|4i9dd`VXK0M_{ddXViD>maro{U{gQMb{SCEu%8F=IRRUW^av27^m5LHoh zM)U(iOL39aUSZ-x)q5pp$6e(sQEKc*5<``dv>}#_6w4cOAcix~_C|78bc4rNH5sQ! z=8Ec!xy(zSVL85DF1B_m`~Bu3%3Wit*3&tpw)O~uOzW>Gn!l*&c?FD&X=Ko`%GLX) zT=BIKIoeWHJo7;c-jMAIfavsCj9EVWcia#zTShFMKgW$EqK+DCiMcqcn)he)GEAtX zHtbwLZHBa3W>_DbG$l>GcW}Tl^Hb>@Q6E&=r>Io7ERZ zplfEHl5$s%mZ<3V7lH%>?*KA9BB#$m_o zWCZK(kS-mhIFB)xYd?eC8PTSZ^SX;5wwQW%M6J<`fh^#_mBNmBO?b4zS3Qf+Z(%Hd>&m(B?!aGB2$7(?z zCci;30Z%|g-gIJ8izN>2b8<#>{#2yEfh8vcfe zi)rJ5FJ+Z;&bZ5_pw6a-`U6{jV~yCe2HX?S2`pRt$lddjcSNzU=_}U;TXd1?8qPmO z=bITL#OfaOetoe2u!;h%iJ$v<<3$Zcoh29m${A$rcNl$<-51^?DMosAOfIwg!bQSy zjY?ok^PSH?3-|AV*_>g?J}h73GzrSTjamD$E{&9JMQn_uZS{@*nUvk6W@&|^_BB?! zUe+xe4m$j$AtpILJ*LA1itg{K?v@_Bkgjz4xc{|J&FSOjy61|d7d`D z2J9;y*R8i}t(0#YVWZ!h_uV)=qGKZw)WlUs0t96tF;N{9M}!38r(f~gem;~(PlG9h zg*bcihN`%!JMj?y5-WO2jUIN+1mvR)2dD?k3wQ#_ojm79lLe^9lu+J~cyff+>{QIX zrAL=Id;S<^60nLxeQ;A%^%NrRD%;oM%!W3|m|)&8>y=B@k$@ zJ*<^=1Ov-5TW}O$!fsw1-1n3nzj^L(4}Whe5e_nAoFDv*y6!H;Wt6BSZT1T{&@UOQX%5eCryf1|FiRA zNA}5_JVRy<3{Q)8G_}Hp%f5YyJY}=)qMnz@v2xS7Dz5RoYUelO&$;#c>>xNiX2ao1 zWlz-v7K_1p;AKfzhWGq7G4tqjG8!_^nvzh-2ZC{)Lh7}Zl}35X4W~&vUDm8d@-hXJ z{(7XO>}n8IMw%|oEE&hGvs9n2`y{9o+$@t(%Z@fi$Py)RO#5S`QmH&_G{2P-SaB=l zilV_?UwkLIw%JC;<1|{2F z{RGK6VYm^vV6KJkfV~(Df$;>z1%K4<_UpLe1##U60pE)G`W`wmJX4@=$xx!w=p)yE zggU_*I}qxZavKf=t*ED}9?M6mrHT-FfsEW$v!CgQzoP=1k=zZOnkwnrqFKl_jf!i? z>20>dtuud11cE+bUb8<;mCETlz|#0*JI`d0+(7_tUk)bHu&+n%HaPf$jhMqUD=*xn zV?ULw$YIO`9hTT`fgc7Q%xK zk!2#xt63Rxhwx&oqr&96K3)p?ef;uxyj+@D99NZ-Fwt8g!+LnKnmRY?ljU#jPY3mAfDfE~ZM$^8DM?MfHoKkqYYMMyC2$QOi>6NM_WF zRUWs6@GufDw!V}*Szkc~u3S3TR$Fhjr z!@Gz);=RZv9t(tHz{g+q0FZ8hI7&%GwEi*ZUj`i->9365>V($Y?%Qn*L&7pd=*ji` z^+4JM)-wiqn_`bEO7#=YW*FJ&Awl~|8EB}Q`@cT48{<7_2KN$8PxPOF_V{{1rqT=e zAHpxp<&b*d6`cysiB(fImF@txAaM)OT>6q)O`IN^QK*qa!M93$9@TqqSO1LYlKfx zo;)zS_`;ow%ov9P?wt!`}cb?BFbKC#6TI{f-@cF zJMfc3^apsM8?MWdNi1FE8D5nfQ9GU{Ob~ZFh){5xpW4VswjW2{Wg;I3u|QdmqMBSW zrw=)H7#95k(^ApGm~3HB*N@yPUg%Jd$eH!c zQTokT;2Jd7%@49QE;b*su4fPn&gy2LBn;>A4Y~kx97H^x=-U~T9+tIfEv@Yvs`HHI zaP?=EyEgW8$4}&W?fA_QUB{H|9nyESrf1x$SDxd~+)9m%`P0h_B0f{KJZ!JuD0 z0Ds-pwW&!jHF?}K($!1y!81~ff=WB1;;8*FFq6-?d%1~F zKF4=-&WD=FyKQ%%KSF8d2dobh3kUX>ZN^+$X;c{ksco*qT<*V$!v9SBKNv^gRO&*}E}P5oVf&BIBi%QMo3!98dal_BOdv2I z5uoI74Oo&1bMbz(u}-}bP%7x=++M$mJpb#`(y=qkQhfy>wrmtT%>X|VAUSh4tZme( zYv>7Sd zwHXEbioj$g<=98+1~YfzVnShjN1SnN+Kk2WLBlsP?HZ$0)CPJ}Oh=PhXR6>!tOl1= z#`kZsF-VWW7a)KIi(V8cO7GDa&*!|IYL?N{sAqDzCbMMAWd95;Fy)JQ1!k}L)M}M6 zG`S*?fz*QU*afD|7LiBS`?m}wig$OsUOEV)Di!3kHW5kON6*PADaeH0GcHy>p$g+y z9M#`BF3`*sp8_w{#VnzjlG&`M5+&Mf`e;%Q_lr%(U4^Az9a$F-k}K-aDUHJfcAgVe z?0y*#g{Pl1OxAy(fN^6TFZgTRRvJ|4VdkkN)nCF_y4kE%W^*3AEeQ_dyPR&C`|xMZ zN}&;1qXR?T0!1zkUZ4pw0Bp1>48puiuY1BPjXj2HMM?+++6!9^x3Uy!se%*GCb5(m zEmJJAPuHva_a!KzjZqheqayxLxuqJK&J%)H zmm2QpG!DuvWX@%B*7>5(2Dcm8Dn{?i-d64`-)DJO7(D0P>dBzoQml2Q@sb{1cM}`T zaq6VrnBm+(*t-Q&_%5h&~@N)EorEMZcxEzO(3U1I)ie|Bi^QW0q zglVr%JwVdZ7v_~cU2>y<;;q$NoSYk@)Ii*8k{@q0M_A_lzWS)Y4AY=WcxJagpsMq;iJ=Q&QEIN7&WuvKGvbD3; zP#c2^yMu<$M zvcx}1nL)f9575hlu~)u>-I}7EzvCnS0+D(427FrJaW4Os(B|b_$rUrDeTD3Vv^;Tl zvRD38!dNCao<+HAmv3N2HXZD6VVqeBuuaUwi+`W(>V zZ8AWdnbKD-GITzaUCD{KEz`X?h+|QY*Dr+VC}Fm(vM}|}kDAV49y!lE5jobmh97uh3MQ2!BfMdcKB>vyPqYeK`)Y)7wTgiI>CkqY z583(+Z-om_Z1Wxv^DJB20T9ow(mP>)SkJ4 zPrz@Ie{G^!K+7FgUsmrs%>TBD{$=(4vx!!z{>QafSw?yU@I1Gu@cf7PK&jg1HQ**N zQ0zQWsC?gMTv8znSf=?=;aoP4?)P}!^`({da^8o$?vm*rA5)+NC%&y~l}V19Z|mDz zra3&vH9p=SAUzbG=>0%OD%#Nm{!~#bF;J@qMogIfD$;{?u*)hF!n1&ZnNW?@jnLg< zD?&Fh+3lo)CxaOVvJ7m8$ykgR($rKn1^GIJ{f+QlG@AA<<$Fu;?f*9fGQk z_By>{^!v{JR`YQQ_^M}<({$69SMAvg^irxc_Xgq_1*$8}ASa!t2dAM&89Fi&a?NgG zmihgiwKM+lmF81e^an)=gZsdY%$9JIHTbnrh34V%GhyXJgXRlpVj<;YsngjF_qp6a ztw+Z$)3b>9VP9jdPY%B?dv?iqQ}+bFyJ&lMUSyNDu*`fl=o3M|zwUUx->On@b>3^d z?kzzHX16wh4NP;0X1!BcCzI)y1SYLtv;FW)gi^cd=*wCD9-aCgJETT=NXW+phqc2w8eZ(<-{X*8CN=vHMN%?jtYyvBLs`TYY3aOQynhH0<*&en=_sy#9p}v z4Xe9Hg}26PNQ|(}EROWoK5!8$o%dX9ERox$_j(Q{^c~rSqaSe2YB9ZQ8e02|-6@l?6gb@Ec;wziHr$fL z0W6iF4g)k9=;`&@|3prFDR}w&>dtORrsF$&_~Lh>nK4j()%1XB%e9wW$tE(!(JFun zh=9M;$nPxiDLAN$b?NkNyJBflB8_J2+D6SIIO~r}y*5Y-Zu@aiDN;cL&rQ~m3p2u(N;dfD8)s)yMY~1|!yorgeF`TOlt}bf1j}&FCbC^}RepHE8nX(^& zug&TyC*HoK;afz~pc2huP=F1mz$6=$ zdIV|prGBHX2K8FZ!DFuT%9zs_le-CY)Xvd6saD+QbT(m##@@&>F119P>pFbL=F|McaJ6(2TNef502hO6@>#=`0c@Tz-NFKiq2xE1Ckpp6~~m?$R}(z z*3}Oz#2a5=zx!Y7c81a|=Kq2i_mAP{pTxL-eEE08xTVrxZruw@s%Bnxr~@g9{-7X! zdL&RmU62Wz{U#SCu4!3G&sPzDL~{U?pWnV7_nIh?fKojC-r=1{n@+9lN+0VR^@>$gV^HjN3smW&$(GpCV*I{bMdiDZ-@-LkpiobVw`o{t-#jL zJ`6+Q2VDY&HFrIsQihLUd>vbRl5@UO=V`}sMbb7@`3RogV0^)5-|c&KbM$4OU7914 z47#ILv<-$1{T2F9wFjFB|02eju-P@#w%BD?jsLgAxUh_ADI(Bs-|C_MEivx@Roj0< zj8olK+Ymh4lJx#fGzp(neKb+fs)A2LiVW!1s z4V7~5Odd7{QM>txrv@8oKpn&86nXA8@9PKcuC28Hz-?1%-^UMOgOxg|QTjqy1aoU(;sL1o0fcN64&TCTFT!_s;pk21}9D;wbQmpyy=BEOpNihK#0v192 zxOlxen5lh>jn$4`{O&tV!YFx=7Kr0QsVy{|8654t4V@-!u=djbCv;9d znCw9j6P1Bfn8%Nw~l-TffG%JdKUzi zW-B-t**v3c?6hszYu?Vn_Rf~rdxtU`UN!_?W2Yne3@Hi4$D*uLRmp%##%hzLE)t$4 z@mj5N^dXchzT<5Z3oWAU&|00^iH34|;Kzm%L}$%D5iAiCh_aJF#9|GR9w>ZU--8gAX&H2>J~;_rs7`>kA}j~%w2bn6kf!mT%xQe?%Q1U z^;gRBhG{GW>zw@syX^X?RgqPz=BW;tH&6IPxF@HX&hUtRYphW%E>$MZYnkMZKe8TH zJC7JGJ|EcYtp{_8OVzJ_X!KZkih^>wqCIlUgDGDEw!ih&iQlD*(11<7RG8P&B*4pL120JNT;{XD6wWR$1?dOZE zUR`;^!h)uGxx9PENutlS@C)JH@#hLRW$f~}>HfCx#o1cgU*5yl3GRorr>kF=Z*X0M zSBrcuIR%E83?X_I>TP(h4B$<9hG-0-CX)RabhK__-Iyv)+tRQdWjR5iuo)`Xco}L} z06=R0iyELUD+lwFoQKmtLNSy)GISJfz0mmlPDF1j+X{sB>8oMORHs;07Hn1+Qb~s_ z3l~^F2V(O)LmdpyGZ&6wbNo!<5WrTJUQ3^q#yG!~;Q`r7S`bkV9JgoCZ5B}#R3@&Bz zG|_2%CU_nuHOI+6WpcFITF5xSSxq_>cequB8&C6mFGtrUEk$+Sl7SHlauKyBFGFjt zy+2B7$JsAi!hwB#-!Vc0+kbNt)mr6q+cm*y7p!sAnZBbh{0I1ebqN`RyeZj8UdNb@ z^-9^;0^MxltMR&N|NIJHFQnWTCcM_48B9*@YnTw^0oVlRa(Pw@uEm66n$Hh;N-^7e zWmvg|D1J=>mvLAwm7$^#a!`-5C5sS{J(;6-oob-7f#>5c8Q+Gy7p!LK|>hFRx3*AY`e4lf4$@4nH2Asuj#r#xmRJQFHyN6tGRxf#E|S&m zT=X?_V8kl8`_eK(bBmP7D5A!I}Mf-CjZoE~LhCt?lM_h3;>s zNE2(sy&;gvJhYO?xrN`Wa#3!huKxI?z*udT&4hIrdr% zn1ALcUgH{$r5=t|9rZpr$Z|fK%X2haU-3tuNPP9Q`}Nfyi)QO9pZX60NGy+hUT@ta zxlG^#1cBPsESL`{6Vnc&lWHq*Nw2H&%5p^URPeJ|3puBSZ58q`??A$ehxPsObPT2YAD-sg=hmU0$w|Q3+U^1<3gnM6Jir+ z8(hVmL)542qWUYbUyQu==TW=W{ubD#>h-o3$=#ZSX#3MSr9(=N|Cjst$YdJ8=B-F# z-`iI&3`mYaS)pvI45bbBYs5y}FvR}E>Dl1pT$t#>(iE$v)1)7YxtUNc&CNIsOq{AB z>zYjj>^jid>DatXzk^1%mv|Sf&QN}l#P7^-9Vi23Li7+GQs#7QJZG|-MHidt>iaqo z>v(YJYx9yxpfrV@G24$+DWwKuF(dB~mk$mXyYKMqjEgP9KUsRvuAeNu$i{7?3abQ{kWZGL4Hmp}6NOA#GeWH0i{`mtpos^QOjP4- z{d7ytptCR+nGbAc0Wrl@||fprg$P``F#1r zs!;~3@CUDv&oP9M&6g-U69pBWQ1Uths|8rtWo?B@T(9t*+TiwKTYB`GOV2E6jA(jf7^WJw*U+AG7qds@e;Y{x*!1ouG_C13IuAXg(FxA!#k|@ zPJW>tGVW4G=6#ceOcX|3ez_>sgQQz$S5pX1g$kME7@5*prbcsg05u?J^TN)r^td_r#uF%xs&Hiha_UH-+EO zdrXd%SNs<_<4ZieQju4fy2frB01iv0d=OtjUC>eq!E#LUTzYs0ttm zWjOi^r|;d}vpsvl+-iqqgibsDoDv9~>^~wVRaWLLD!T9z`19UPpY@ACFK7x$l)Mq+ zxf6|BCg+qY-@!Z!FaA0O;(8kgqC3A7{t$;(C9YfNIOPq=`Q(>?MWM@~e7E7=jx6;z zZu|-?@fRn(V#y=)4vPUfZpU&Oin7eWX~V!~8%@ebLs4Iq;tqh*lJqt;K?& zw{4LbDz8}g-=04H&BybPO)vvRoTuv3(iQqyjtKk@J{~JuGoX=ym5hPAi36h$(8j>Q zUDVCa!Nk!KXlo;9XX0SsWb5$1Ej}vNwzH~O{EhDRhT2CwM-4WAn6XAzPr!`A^1)YT zgu2+wO_te!v1@Zu<6+jzvh(kS-c));4M<9zKUX4CTq>-EQu3on=%c+i2{Qlk9lBq% z|N8hlnE9n5C!ZuT_2w=-gv6%!5#vrtz0Vo~0YGyxRNo4lfKDeJ3h*tSg0-3jY6+Tq zUObF@g6(8??S&kIqL0O_XOA_w$vR&1MiS(eN2u1zLo_b4O9zCslnh-A}4!Dd(uQ?^#Hr2}~NZ)uQGl|7M+E;Xy%f zGO(9ado^zmuP}Dmd=(Kw3=PlTY}sX8-`05%sZ3t%C8S9f`8c@ib=WXRwG&nwn~}{o zxa$z}r!V0<=6Rk|tPU+pL$BM)I%<;!O=r4A9trn^wZbCxZgR8HN&AqCQ!gSZ!#Lj0 z-i!|#zyrt%a}cpsZmoz?(2EmU#AXojT;d0zc!R1I4w~c$u?x5b<&(q@o%M5d9}#9D zh&2ROgUN?>B3QzQw4rnGY7sqT^=?oP<|940kT>zPYBZbt41E3JMgBgH2nu#IsiZ>t?$9^AUV# z+!WZM)Ohs#GM14rzNdM@>T+nC@J$;gxM{XQ%sJ3qdJ7)*thVpbCzDk!dY_O=xum}N zTM%aTyhevg!_A4J9m3&KDVa-BH`H~wQdi((5LFFlXr$38*!ONc7Utfkxa?tjP`ME3 zn+!bK-0^{-%oWgZfl~e%TuO?_&Q}$tthf5rr{M+_`@2T&Gp2)KXUY?PGYlR{=j6Q1 z7#n`8PhiYiKDsoz0k>(IB(mZFNqR@nWYDi*(r{}Q$7Fur$<2v0{s{MQiHG6W-(%xw ztd4}!nk<}(2VU`YBULe{L)q-VGcIY9#bHcHRF)=d%5RFdN7*to$ArqxY`y!u_-avd zfx{Ld3GsyzCzC8{WU-puKemxYb6W)VW+&-_^&q_zmEyp};Xe1ek5A@MN!t@hHf2fa zRZ9Zp^>*e$i5D3Veq+4&V%X#BAwYPLzTcew`nNPNug_)mm*lpi;ZKjJ&Zi)-|0iM~ zY+z$+^N%*j8d(3=AR}IJ^&jIoe`+dP?Nm&Iwoo<#x}8ve$)s|d$nPQ&C~B%pTr>Md zhlS``>MJ#LvLd&wZ;w=cWoYpjH}Y0M)5Edzv$NONyEmxc{AbTaIST%Pyed}DO`)FO z8>-G_eGFe~5JPv3J3Gn*q6=>?83mpRodY+Lau zFKcnj#L*^*v0x9$@OpktILpppIioj8%&RVlt9UJ9nU1DlpMc+2=Z^6yvntx)jWysm zr{C3FkBouf?aZ90k!YW*IA7_vkkg?VWG}Ou+d_L`{b+zo@>tq-3lb+Fa0EBEue%Nq ze5JGm%?pru_L2fLKwC9P!nu=`uiX>)@VMQbi8lD!M=avlZAK58?XC>R`p0El0D(oz zL<*1Nc){3&Dk*7=-1yiCK8asW{_lOrIRUN8)c_hCh zb>S>XE>`*ChVJ2qxDbAX3AH~0@|EHQqQ@uZ%~8`+&CVMf;ICvAp8R@-^}18ZZ1>4* z{@qXhkC6fLloVp_(@&oGxhC>Ip@9C|Lm_P7=p+WT`9E7C@p1o8zQN$ftW9fGnRW=3 z8>Vgh7h2sP*5xsVxsu-~%wakZ&Q}l_TE|_a`yzKRulk{WvLVDgetnf6U~T=W1#9tA z<;d-L;961J>GSajqmM#M6N7`Lh~qR?oV1vwZ-7P0BxZ*jN^@b3)uYqgT)5K_LJrzq z$E);&PU9fA%60bX^QFGb!Z?Wr^q=@}ghOgGDg&!>r zLT!Zg+H?x^{ivKX=L*%Gyg~ac5FRZfIUuTIX0-^I9d5 zEZd6GjcygX&=Tq|4dLm1{WzDnyy;;`WpFQ=8^Y`D(Z668xfjuqe?EAIp_2$AR06wc(n;tggyU*r;fHpxOXJclbeo)QW3?`2)CR!` zhPH`&jR!{oi~BDum^Vk!5lUDp(F@VtMzA%x4{3{~=+Sc{CvJX)w@r{o)F5NRVyck~ z`?X(x<`scznO&3*DV6QItU9tM=Zi{sSN(7J^pXB!&-x`8;kn{3p?z)Irb!`C-hyg0 z1JCyn8aW+>I}?4a&*X>d82Ius-Kj;PNClJLzNNRjGslAiN4a3rd}5`@11Q;d;D&6n zF?;Oi!fcadN`d-X#S}t617ZW^8&ZP2#BWP~owJ4KLeD0VtY6Yd?oSUkcG8S=lbjzX*c_4BCBko&M0i^ zZ<2xB=&@MK>~sb-ye_{TY{=FqKPjo$>0BP{*mof&+Vtr2cP(VhByBAD`IQq zY;7a&;B50>;#sc_=cP1{{gGKW{4-W7?@!irwy5B97J6JD#()TcW;y*0m2e?zUZMl> z&z_%tL>JjPHZ&2(PX13AYI;fvz(2C0>-tKCsj{M+>jg>=c8-o0?ZfYfYpER8sLjq7 zi-eSi8IBzv5APk0Za4lqkzbPNK)@#IFTSbYxCk1}o7jIVTG95}H~mE1gw26#9Nmui za?z)Qb8)}B`W4(h66E;}3uG-AV3VA-UX;q?O%$zmxxoAk+~hZJ(R<}@$VqH;mN}N zV#1MSe24omkQw_c)o;nAgTcuCOGgJcT||Ls!%pNZRYnUp$=>h2+mf_soGXW9hRG9d zCv_5vTUiGx(e=?FnUyehI|I;Fz|8jB| zdmk#kDI2O~L>Xrp$n+(b9jG4d)RPkZZOW=(<`2r;yahk`Y-ymF=B!wOBkDPbEe)GflO=r=!P4H?J&ZPf z945muRK4KU1w(q4c0BG!ukX{6t_BgmvBH3UUb~Jr73a7kqdD^|?KVG*A3m!So$t-p z>Rxpmct$3Rtgj1{wPNb`ze2%$X+K;?y}WiL&@4?Ybj?`Wb+htX4&c95vZQs{Gl1)u z_mKcuQ6{5R1aT5f#M8}pI#t5z^X3mj`-34k0<*N)a>QujtH29<$Xo(AAwU{pW9qoU zg@>tPW$7Cg)X)^Okwlg(6xgfcFk8%nuwi4>_DU^CYnUo5aCq0-9Ea0VmS3uz&~xp6 z>z~M1WZe_<+hCp32N$@p#1@?S$xh|IEsNG{Py5W0t70pl#*4BhnDot-%ODFH$#Pdc z9caNkf8Y3St1~Ad#jAJ{lLS8H$R`Lhg72bn-<#vl=Q=Zp?hAHd*c!DQT8dh1qbLn_ zu7wgUX)k10TXQ;0{G8_8*0qCbCn{(ePRBmgb|5J>B+>Ry3fvrm?^Jo>0cDvxpS~wh zw1?$!dq!GAk-)Y8Brf3i_E04aP^$hXXl+2Sb{V+xem$@^XM(q6Y%{MOS7k+zPlpdo zw4fqsw4jq3sIkb~o6E=HNbah>NUk%TDeDSRk*}62f+*;dC)Pe47%}8(87|gZM^9ee z(}*+0b6pvZRdh@o-bHRvF_qMAK+l&ElhvUyifs;WJP^s#xwZqCpRk?Y=ZFl1;Lt(V z*@<5CJLjCiu^CSnur5l}7~xzWJ*$?kS0#uUmLfGV!Izk+qRt4i;t(<^@A=_AxTHPeR4$Ko|( z+61!W<`3sr+>c=%x`&Xg-EYA1x5r3^AMJmlX3~iGTBxEk^Lt9+GN&F4hG4h} z7}5ep-j}zIJT)2+uGuy5G;LOk^M5}fZ>xXbqMuYz34NkFZZtb}aL3Mw?o^EJ?1N=R z2R<2^w$NkQF=m|f@MtVFBaQnznlu>ci_HeREV50q#`Snn#YYjxwzm{#-F8!v(i#pZ z72=%k7lTRt-L`Yez7w)rS&^VKs;k&ybtCFMYfm=u`U4yesH79Q-1YqBLN`EEF`IYfY@hviVB~v|m?|eb;3{O(X~TU6 zoc+({ulT1@w)GK-l74mZ(YS&6$)OWJ3YQJ%WzJ%HO&bJKA7{btonww;{MFyfweMD6 zuK0)cG5lt~F0Wr-+z6Rs@((7FMH%FOQa;w2TPJk085SH%p+ffx77yN_!YWEMH$OVV zhQ#@W#S|3p)tvi>q@z>X$O|1Nu$(%6q#=F3l%qV$@7{25D}E98B^%e!YMYTtqy6v? zQtiXS3*7zc_BbeuuUYhvC@J+)j%7@9nuoFJ54p{hUaGyCWdg}U=kPZ=bnUAl)Ojr=!z}8Z78F>_RyLesQ5RSm;7Q?ZctiwXH_ennK^y=zlmOIU@~TUV-a7OL5j81pBdH z>w%)JS{%9HA^&d92%0v9s^{tq&nKU4BW9e(W3Yo$m4kM7&FS%9FS5FoROmrB4S>?J zle)LeTl-{*8dxJ0B*^>?d^H#+Cd6M5@(J`&9(qAW-0&6VcotoJS3BL$u7Z;0^muAc zzQJ$7%!d6a0P2!&7!X&+$OV(Swq6Hoi74qv4y}eyhj+{XHJ2JOONRoLIpQk7-R161e1riI^o#Ke#IMS;H#FM zdQ%-{ZbJOj&d3>m(Rwh?;0mEtrRD7cEvB_#YPq-N!p96B@%a5mc~YEzu8IAS11%`f z+oIq1)7f8m($irhBfoU#XV&9f(;acdm+{AnHU6fwOZ(am;`aU~XJs_A-mlymIegEQum#NU`D$iJII%OkW3_Ql z+T&lu1$|Bu5WgdzWXPr(O4HZxMkdO?5Tq21YDP@Du&rJF^|TLyOV6qgVnT;i`g_Iwk@`+?UwsC0gU7%&&!mXgJ;0QP6=UJx;6s1w$zil}vqeh3l`<xS9% zp`V`&N;90hOyOop!BDF%sRS8D=W@lE=b|UBBz4hajftB4_}cloaO-tPfj2toQ?AMS zqrEb4H2zfN!!qS~t>Rqkcnm4S)1<7k2Gim1+tF-ADpg|=gg>+m(0&PE=;7;A6`_~-UF(g^lWEwR3r^*4*)2DMB8PToYkv!G8r z(h>t(x&aoP3Z5ZZ8IhCw**+V96Na3748b?Bj-)^jU;mYptO)rxb@tryDclqO`%{p#f-sYiZvA6t}d#sPBoqQn*<@0cPvfZ!WyM_QmqGVPpM z(6bfF+^D@M!3l+0(Im><6G@R;%^ITE`WftEMXSDw`WbH<_YrsC8pcYvFVG$u>>m(y5?*RNKqWxb@*mkuiSF}aI`@rf@ z@6uAu^$cXl&me0Jv4Ma9{ZvRRD=1qw5@dKFwzNMV5fB|HvQtO>^TGQkW})?pQ?93E z>Ik{)OrCX1-s#8G<}>4MT859UX_N^o1t}pkUT3oFVY_GN%ev=gtHek z${#)C5c6&_-9!Y5)#{_eXohZ;SQ~}<0vYgIZ?Ky78Us+qeEC_jXT`lxfTBnFFN7GA zkWdldSjhkoBdwhM&7f2;|sq+0q@ zIr5m|x86i%Bq^Ba_B}TXN^jH8b}N-7Lj`%Dv2SmcB|{0h+tt`hEt_#O_SzfbH( zXSq#<#8PAQ!SI0R(6w@&Ywx%=i<2dFx;Dkc6bfnD%6d*BkDWF%KjhaYWX|ihTj~9h zy(AFxZrUqE5vEv6kqCFy)F<>a^m2xt(#cXx`q(7iblR6-kPkpXQfHlq=5jF`S%qX$ z#`@Qf0Bu;+z?2L=4(T_*B5yJ^=j_w8G2PsYVfNQu+N>hNcA%eCHw92GVjX)J!-13O zB1IM3XnLELKL~NRW@n+K>}g$HJKpe#A~wHWIiEgDALE?Y)Wr!Msl+T+mzu2}H_Hq6 z>64*I)&)_^?wy3?)|uo&7-bN?dg|Wp25Uijq75Lq;p%x_xE(t6@4lhBJS{P7>*}et zh4p!zt<7~e94MdO07bak)q3=?FV>Q{(B55Ms5d_z`I>A!`8xo26(Zwa90ifsIa=m) z)@}FD1=L#PUP7bvSBfgk0hyr&+VGuGQ);Thg*!A9DtwF}y^w7+H(#}_Wj9;G59L=J z2pHlI%*23&J8B;?Te97r85W=19UR>~g!;cSM<@VVH%SRS)$5R|mGfSixW1TF214mV z8TyF~YkoYB#JPjJJ-fmXG`Aq{O+lFW?OGt9v%fUW2F(iUWC#V(T1Vv z%ow6;&A8UyMlI7TGpiv*jp-*ZjQw?j@W!Y>kgXGQly{>}jEt?40>bkn)Wd6Jh#8sx zJ00`v8QJ4E-Nb8gz^|UnH*R)Q(?|#3kj^axz_0#IvfJk+j*3E2`y-n7h{V@u6!uz~ zn_ofup09M1WC)>eT)6OnkG@mX=TUlXlvVMi%6O#?8UhhzQFwd6VV6|<=w@~Q+$jeB zCFIYFNojqwPA^4r0reE6QXP$8w^UM>A-ZU_{PT;Z=GDJEr4zx>ObzRtK6f1^mUS{> zaSRwGb>@g z?~NrlL;*W}OHyA&Z0}nphAS}4>>AuZp-o)d8ITMq3+!!#6E|&6!c(>Wle_xr3y>PhhNlMVjx#{oz{?71K{zC28j9u0H~Ec1`&Z=o=9Ltdybw(oZ8J` z$G0pr3*QqIp7=18x6<(d-+gV?e335>S7psu5qE>iy?Di#q~>?dsRo)_gtR;G{#2V0 z{}qszGg@6OT`66)Oi&=NQ%f+v6vv0OI!Bv0$+>^dx%W_ObxZB~#u4Iy*GpD%)=1r< zvT8f)RoX3sQmW3brTxztRT;Dv*zB+5tI+l%aDcJU5xhIzuv0mLn7fiY(EB~7O|PhA z^sTipAYp;z>hq{#x_yRe>!8W4I_eiS{$9ya>`EB%J>FQ%Vj!M-p_xkt%rT?hZG$!3 z1E#71epFq<8h1d!nbTys^R$KJ_PYR+?7s(4NUa1N5!pj)6R!Y;DP?!!!tIKJ?4Xf$ z*{+N7ej%D!iSQlj1WZozR{O*hHSmA?6PZAcSLTjgK1*V4<<`xy12-LbzqRUT%eaLo z>%Yp4tk0$mSVKWjgW5zv&_lRmFgm2!!gq-qJeS~LPIzDz3dE{M891ccbx5D0Fzs(m zt|B>!72S$uxT3MIZgRhmOLy(^**R11!H8K>_;IKr>&iW=1F_cZ)8e=8v|Dl4C2Q}6 zr8O)C=e{f`>w+dmIKFOMYmH0OvwYcOuL#VjayxFWdLJ#+zrKNOF)!BYYchiwHy}Bp zCUQb(2XHw<98w{*It;X3;@mIYz>JVq@G1WIaCR5i_tbgX0E4U;U8UM=3=kvURO?!g z@ZnWS^T5?_`kIcNk>w9(`~=owhXNI~^T3uqH1#I_L=J_9kFX~ty;EsZ-lK#Wto67lq ziAU6T-V%zV^eZ`(k_(eBh;FA2>hd8i=qFOEn}C{rlEMWn>|01Tz2+~I%*Y!hhza>6 zSfO;eG#LvY|2Ee7=SxAh$IVCmX-oCu5!eak{IjqlBi0<%0+V0|SSOZUv`V zZ!TOF@T^cuq^e-W6ha>o(%=FU*wnPVeqqBi6A3i)*o@Sh#$`BB&Yqoc@|X{OFOtoy zZ6$*aGmXfx-)X-QN6q)v!8M*{;wAfz?VU;1!OZJwa999YI>aW zI|=nDsfvDbk^Yw|=V=XKyHb?MU`56Eo5^Z9hO>AjUsQjHPN}_fbV7-NN~lRI56zSo zt$Qqp4b!x#pg?Ykz3^fbqGuAuygo@X%ez;CRwKcQQfB>c9wie-mJLxZjx$O%R_y@l_K+u1b^V{@N%y0vDY-C1XW8A14BMf z01k3ZKyd#>Ww+NKgKBP%?-2)X^Tt?4J#+8jJXo!>*{aFu-OIf1v|cDOBts$~5pG;E zqnFrM{knyy(uG%(NT8Cp8_RF#f@F`=);fC8D==k$S9^J)Ha>|us7y}Vqa z*cWLS2(e9ccCX|XI?WkU{^d}5!i3KdV!Cx$i-O>gDdH$mzsB#)dPZW$trel`o%zGh z+U7v62ZF_OLenr(6`X`hqhXgfh}i8}0jnjcpo`+hBUYD@z!Cw%70-q*(7K~O$fW1x z(tOECc1Rx&`|QPP5^F4#hznvGI_11;`#Gk=(fId=qLcBVy zb`;R=L{3%#W9&i)IZ=L@Y=b#J#FuHbq#g2o(_l>}Aj~z*!>8Q9xVopeiI%5o>#xAe z1#BHUupDi|)8ruG|GFGxs^E2NAEX%?EM7{iqx4BtW|#BLfsa8Rpdy*jd^tx*6wkuP zYnO)XgM6IbBMIo7R4u=xg^zCUnL;yGjaIRKp{Wy!Y2%92b!8c+2*}gIAcNc_= z$ONJmAV+WC>69{WBp9$m_D%w{a{ECW96!_=E*Y@&qG8x|nD!DB+WxK^(n7X5M*k^J&jIvOh9{&~!WGbt_#Jcvp*n<*# zW)_>_@xi^{^1e^Wp|ZCrAr2Alwvo8A+4JeY(5pJM?~;tI_3vv%6``-KMEt`j;5C#o z@%nEJ_kYIfETGCrg3pn<(Pvr5^`Gdkn1k(SHD_Zi>SknO=k&Q`^RK!tJ#JO;6U+@- z1Z6Ou(!!hXsG8Ua)IX;F-t1%7^$K$t+=oRj_Tta+J=9 z#~gI7?mVGiWs&r%@p2om8n_~fHVLm*jx%6^6*1U7oV3b>2YIS8x}Z{?KaZnrjoo6A z){tT`JA*)1Y;m}z`d~Hc)U2_|UwWMoN=wT+G@-U>Zp&U@oZRBEwE`+m1R@M8U2vr^ zWMRU?i+#}XS}iT`YK9esE!H*oJ1`5S^)8>ut&Xrr9&Hk{j{l`>I@F7tpQM4B6q zrNESs&Tv3Btm;8^&WRp$E`0|T-Vxb@+2OI-Js9=OZq(23w~NiC*gtiDogBe8yw}(5 zp2UAEwO)M85R6p5=!vZnaRxTQYs4_el#X}g*Vj&7z1;vl28qpvSCbfiVF5np1F{i)%< zCMvhMjE^@JloJSs_eca za^VQ#?@N!^PU=|_{e(NMd6#)GID~e{9TXVb&j{Ir^(DqS?NDMtAC@N*hQVUX^)d8Q4ZA309CN55Yz+00 zszSc4#hRDHcv0vHdsvzb{5T0T_Ysi(S{^JX>$ky{nGoqYMR?#H;PXi4mfE3qGZT}T z&{nsX|BYZ=`L-Xzotr3gTbA5@?3LmWjUcc0Z1Pd?iDW6#yV2N;jF)s7dw4>^g+hNe z?7|GpQWf0f?*D3|h*b^YkAB*0&OYa{bpOenlaUiNHvZQ=%U9JlJ2;{IzU{&Zf@}w(^FlV5>mNQM2DNX{%BapyZ|_ z3f`DxkiuzdSl2dQrvad}7w=9B@fcg&VVJaK+D&C{`_pK7jW~J9@|o}{C!D$m2Dl+Y6paa)Gv9AeuDCM8k2VlC`uem91L_*#i_}c}_ii#9T2cEw;whag`hF1YR0F$0U%rwlJPku zGSxB61;V1hV6FN%=td6gZ+0`Pt&Y&A)4rd#I^7Jl^lhku+aI3AomOfduW-ukzk=1j zC40H$ln;xU1t>alyiZwaXDYAP{z`Yyq#T^9+(P z2`$vSX=tA%dIujxggF_2kLVBVULYu(iY-T9G~V4C zbg%Ml+_l0sC-X;)?gdwzjzM-`<9!Xo{dV-lkrJ71BpjbUIUG%U-K`N{Me13OwfX3H zKU)3^b*+Ape#fk{qSK|0y4rj+nZtd7TEZ5kfl_rX_Ic+=8ta;Ywwf|TZdHTfM2(Pp zofD=&|J-{S_DtPdutRIG%h3`J2-oA_l%4=9U;D+KQ( zP>%dqjEyWtr{zvDx-eV6>!M@GsZ4FdUG4kFV#f&!=zM967v(m3puI(6EcQKaTPpQ= zC7l1P-GKNqbQmf0-SK)e}oZq;lSv(TEu8SMGD3^#kx<|$vYKxI1MI6{J8Ju=O8QGYq<=~RfGGqD0S3V=e4AScB&V`)|<<#*!nA2lkT#3G~zeXvC4 z@C_E+Lm^zxBTV8ElV^oN#B%(sWMP0cDZjWYxBCkqQEx!Jq*%jwy)&LajmjXA7@}I;)jW4kuqD>kz34B3XzOF68jpY4)<3 zaIMjBL#Lg?w{c6}FB5SZ__P}Mq_&EyWflF%AmP$Uh?u<><}{Ch+0NZD&8St%BKiPM zRCApCB*(a-S0-8Rptpbfi2PFszMeDjg?);m-~SI$wD~`x=&6LOg8rccy=<(ZnFoyl z?3c16=rXL%r;oz`tDAyWUr|1Bl&ml~?O0nUdh{84Bz{F_nb}&x9_)ORk2thRaB_#v z?m6>L^PITkyWl=N_*W#C)}y2tqiGCRs0Q;|G!rE$$i2FmayBaQ8WAH#L<=2C+V8$1 z%w{dIiY^xc3Du42zz^9yuFq3la;rLN6LSx#?E^?P)-fyxyIc^uXX|HVm) z!d6A$>OyZb5RzkZDxvnX45aJK!Uk;kt>~c%*BT^ZKi@QHsj`dHR@m2@r&8FLIw>xc z*Q7L#8o^)Nug|=}fW(WbYI74+)}7$H%Q4wWE3F;Wf`Z0lvNenX9eAB`7IR6)Q+OytzAy&hb}rl1%uv1C4hZ1EapGB@4J+$QM&*n>ZH*&4;r%?$I5ai7CFw z*cUWP)oDUsQs9(-bKb`qI%3>16SRe56K_pw_1JqW6OEZPH8Xk;y15MjQh%&ccAEHY>F?lFWHg zhhf19c$iT1+ua;LmId@9Hxl2P?Sarh?JjIs8Fg4IF4XU8MYe6_U4II@#6`P#@E~#@ z{jxoONfgkuDC|^xNzPe{4)=!Ts4Cu)hL?Pl>R!w$)Axt%=ytciE%&>$izS|Y?1$c8 zq>2j%Bm`UnUVjq;w7YQO7bD|SX6VP))0XYDFM=uRwFHzI2S+A8OH9X7WjRf$=#RFm zsdVT{(X_uLn8~$23wIn(n$MWo<72x~G*9$9$N2BQzkiIhhPO!i zjKdjPMVIG>--i0T3QLeO5{W&BrkH5?nGqNLjp!}mJGTEN>DVmleK+XH7LRctNZsrCYF z7bV~^BYi5(>Pe`H9-Eqcu6rUy+M#tLt8eWizv~ue0@`<6!945`54*24t4{stLedvd zr;GS!+=5*k%?p0znAUvb)z|v^H-u*Q-K19t764^Sp z0@&h^K8~3Bcsx|C(>_(fFn-ZmF87MgK)Nr-(RL8&O;Ro5%)XXG+hTnMt=T`-jzUrP zX}srjg-KI{?_5iAcjkewj)g?XIdG%f z(1G2*5fH(AVIb{6#`N?zG4mIuuA*mau0wi5ghPp5Oe*>`k}5ro%KPH^H)5J_PhAOB z(!8wv{Fir554U5TA8*%~zZ;SQ_739@`MKtupy+e?46Sa!% z@U}ICEBe|`L;Q`%-3%*%s2e1-DJFopMgxF)VgIm0TU~{zyesMl68JOUIy+l!UTcPd z)*_r4L?48&vr)tTQ_6lKTWy8-^eY@ppDfmA?+JQM0$|;kvWQpCSAK|}o5E=kk<58s zqOdTQ!%^D$WawzuKUcO5sK=k!qr_YPN5qOI5euckqM2KB;K~?r5`3Cz}kViWLqCpoyjLxhxEB)n5_aQt^F|ozP!|r`>2JKC) z5bai2Keea;tu+dY^XHp9NW;wo;<#1E<(?}HR7}9!{?$lR{-=?4=eIxSje{?nZ_BD^ zGTV?)o{{8{mH1`=A@gxN!fcgSBlCEF^v4vW`t!}j8`5T`qSeHEE$V4$mB-zG1`*I& zrS%6vynB#Fo2j8AgeQp*N^d)^h$~1=c}!omzwlAUgn7IeYk(S%xeG@j^-zJi)Hb)7 z>)cy*RKfNxO%S9n{K?DyQ>o(RALeJP;uUzAeDtF7`WxTjL>wSsAk)va0T^=9&EZi( ze5J>?3!3%8l?8cn%mR=rp_yo3fmo>&5}Dq;h8vogE@D7FV=6 zpW7J8FhlcoO+A(vBD=R$W?EH_XGY;HFAZTP(Xt^p@?mN5@?>eZd)7%OV-&TncI6+) zt+G_FRt%X<7u}AWO~ba5)ht7M<^=ck>Rs~l>Y2ywoswf-ltxRZZhFx1&^6jE+MtqY z)3p=n)O43!a2j7>^BIhzeuuiLZV8xy1?$+1Ed8pz!^uw z>Hmz_f3{9`4|s&bto_}Amk|ZwKz}eaniPq8jFlW2>)vE}@XeA{TyMZDM51um3pAFi!&)n% zsKpWqK2}W!G?F%Q%LtY?7mlnq#OgI@6V4+&6JH|Ozr=c*5*g=I|80H%6W5GkUPjzD z_eD7s_Ujf{SlmqwtUdw6z&M=CC{`z3~w?!)YX?IR8*Th=R>pAk|nR zPxV=@to07Q{q6K{TAY-FPf)<~t+F13dsq!5_$IBRVB;vtleH5II0W<4-WrtOYb2*F zTvY}6#iO{_LRR4JhdJr6{Dn`=G?He1@oUA_!nAj@uKo#0WgjuH?1{4a*V`qTD{y!_q|8WugZ)=(AljE!y`ujp~?ver`Se~}El9aR|luS0dxw$zEI2eT_ zOh9d5l_>I1Oq4O~pM;GE@5BvCezPNZGU4RoE41`u!ujLI8ape>-@P*YM!>uK_RDqK zhs<9euW()bkNk)b%UCPfql4XCgt_*3e$&5uxpKO~jEEf^CS%GP{BrHty5iC0zI#0l zNv|+dzXt5_5bPepNO~%E4|AFBD)VQ=To#f)9Ajm4*WS6wc4Q{*VeajUQa{J|CGNUG zRMNQp+-YZ++BvSsP9|+R#crst{pJ6AiDYWYQY)iU<2}gq;EoEm6PnWjs5N@OUp`iq zYX*K$VC~A5&#ie7mEjOJJvU!&AmicG1Kv-*UxlY@zxPw}$Ad z=pS)m(Q$4wFXvQwiHA~jUSHet!*QkR*4OLe_r=8&Vp3jRfpt%~IXq3LuwnVHp)9?0 zl_b`sQupuaMe!aG39^&lC@zD#*5|oN&Uv<{_^LD9@#o*d2l3S&jPaq%PKtU~V0=rP zz{6t_@*C-odsmq0;~x-M<2k?zS4KV)(hPocC|{{6e9c}J?dMm#WA0`3@>OOJCd7*# zi;H_g9|UhHApPV@x$))leoucuZzD!F0$dQ=hjS=>Dyw&%T})* z?Bfr*c-v}i+#(Flz0gx$qO}&}Ix>2OH6%`^J;jDICi9Hzl7IL~+SApH5*3?byRsRr zERb13A>l2s`ZW4AE)6OjlAALvgYy3XxI4~T5Hv<6Q*avo+K64U8Y{7(gSr)g0U_uc zs?)&%A+A&9J34KJTi$?Ao)!dQ@gr6ybXikygv$S7>>Qjcfx;}`v2EM7UToWTI=0cV zZQJPBosNx;jThT^K_}a@GgZ4gRa-TG!@YHJzVkb$iMX(aH*`+Jen1PV5Me9mp_~k9 zyM{cvoW8F=lfEyd7PC*Q2^*yfx4$SRvdM6S2c)%^G3l#Q$TMAcEth{aL|+c5*Z%=i+I%(#Dql?9bDn7-d;Bzm7ER{F!hvl5zr&D z@831LH%8R^N#DD$VnU1!WOkOBLSV*zn4=@8;~NHPb!85+3*7)Sm7?|ftSM6r4^ZJ; zz>xjygkVg46v3DxYkm@AbB1J=A7-$M8dM1xSqB`}EoI97N6$w|uyJ0HMOQ6`VAER&&Ik<=u zcq63S`1EW|Z8l~M3)9zO(VL)?bxhkX8QtinL~8E*0na}CpC+C~N(~V4GuCVzMX)Kh zqw`|u2b|_{nMH?K!>K214~(`W+>Sh6Se{l?wk$=m=+uP2bLU|^8wtG@9?`Im9WyqK zR)&S7T!fXUq-6paolEnr>PH>ika7{7V+p(P{X zP=1gGWYI53PAyL1h_T{yE*(f1@Cn2W$m@qU*ht;zzyIxCbG)ZoxBth+LH_%N@c$3* zH79FVCl6;zX*Fd{6%Ab#$^RPhXgDbU8}S5Quhjd|A$CL~-x7Hc$Vo!WNR5xzMO7zU zhFjb%TCS)qw8^i_d}yauOg69t#_4BztQl$t+W0?Iim7Bz1hy!@h3-+ZiNij z6qe!Yq{kJN_-Vb?cE!g1$4FR{go-QD*Y=Z4_v6u@1rm!=JyL zygmAy7NakOYX{Vwj*rb;GK^U}D4;Td)=3%KjJW+9E^g?k zld!yPk<}M6s^pm1+yjHHNT%1K5b5PdtO;)wPSHHy3^VaF(v_=nbq6=`iyAFvd8=TR z-AFg&_0V#u%)I8>?HyjCXd$yXud(By62-zyfQr@zv4nDX-OxzuO$a^-PF2uVsv?s})ycx;4d z!L*Kh)VS;x+cZoCIwY3};3u$w)Ul=!7Hcm|>K}~9(02^5pvpVUvJdUBS+eM`ewjh0 zlu1ck$Ag5MO3{>QP*5!CuOE5FSpwpqLy|N0JBn-Xy|HPEkVs0FEZn7N@Af`%Tk4}d zJo3t@Hz|et_)aHUvfV7$FH(lJN=XN~fML5C6%vd#^~Thxh<~kg;M7$!cy;@1$#BU1 zB*|3n*sH^xwBmyp3hC@Y{};UWAKKujRa!SY?YD2v#{c7AwD^DS;s4yZ+Xis{8q3SV zCw)Bf)K1L<-x4Jw1ue+N(Vf>IAR)+1rTfFlvcm=}sHYW3bqpFiTj?7#!L%{eJ7VDB zqM)`!>U-*!yIZf@({1bR>?XWs1DrC#z^Ipg?%uqdCGxy%d);)Md>Zi%x=%uS5h$Yh61x7e140!JBqu5y zt_0n;BB40H47mG47d8T0dY|36KjI;vfe+tV@r|APY5m!Vj0ZD)_85>mv5b$}j9Lzy z7Vo@3{OTWqbGuWSJoX^!#LrNq-Ti8(&0C_12WI2X)1s+D??FGt)%UFzu$KUvvBa#k zG0%_TghJQVLI)5IjKaw8`|-I;@2H@!Q9(bS*SRYY)YM-ufxz!WLA*uazxN}JAC3s9 zZ|fo6_PzXH3o1WTt^ZC%2(R7kBI|@j68(vBpiKSNM;ebdNR0`TN2SrHW0M({-s9kD zda}#8Q02TP$^Tu}1{sKn=b0=kAIsP?emqN#g{2^-_vgXOn}FTq8UEa=Q=(cVE5$D@s5 z)ZFowXn&Y(EZ#kI)zcsHBPxBEZ0o7dZN5C z2PMPQ6})@IZ0Ri6Jz1lP@(yPFwRfC7v@NlCo@|TElUHWgFPf~?a=BSEJZ^+$4OwF; za~=QSxF%FGhqtXssFo9!0zo8E1_Vg|Vmgl<=QSLuark$!K&NdnW7u-Eq{Rqf-Od-Zxih8p;l)x}G4 zU$qOACG2ztaFHJ_I&e z8-O5(euD3auc*vIpK&g-k*E}(^iGCdUiB=)Tepgqp)7~-kmBa*1q1$x`62*5MA_q? z4x|+^>RHBoR60`}(8KM)RTi4F1;$#sE}nxc?sUdyhF44_9=5TQ;AqD6ED{j)2OnTT z7T|;3TeEjL?(PtRbTRP4iyhprRnMD^?h~l&%7lF#4l^%xJti=-5n!D~-nH^(0|8?v=el9S*_@o4OKd0S*sHp$>= z??Y&=p$ioJ$|B?mk=drUO~mpIrnuO@U4BY-%*a@nCp6>Gm9cZ})ziQmmO@iuto6!7r3edN zVLRH}z($apR>wu~8!yqPZ;{C(3gk;O{A+iNs2mj6L1tz@Mh%GVn7y&2P;T$6DyBIr z4>UIPa8Jg3F4;zhK)1q>9MN7E$@2^pkH-7@wxAZWm@a8HcZoF6C@h8=m!gY@gv8xw zix|sDg|}n^@fFAa*}a&9FV@Fe+7(vLq`4S^WUAzHw_FK%aqh3#OKtrpJq@m$8V}1; zH>YTN@`&L(?t=~&AOE+%Wm>6%LxGKy8yK38jk7o9GdS>gtm4%utLz((;kE(Gj9rQg z$6D?4e1a}aQ?5=-!s;KQIe>c*LuN!sSR?rIVor@eQU!8a4lBhzg?Ra58fSM7MLV0l zp_<*%z#d;b5@O_~*FZU_juavK7^K3+6O$ zG?jDRlmPrMimgZ(ZFeDQ!fn4S6Bj=yDqZg4N=n?ihitNL=5$s+$|j8Eg}7NJoRN!( zHl3i5VWqU!YGEZSui@U%Y1T!Of?XM){u|LclJ2>!ur7<3Dm`qSD;Wy)U|y)RM@+e> zJ9>E8Eq+j$t)y7WF|R=`Kj^~WDt;#EJBN<;41*qcAJ@l4()T4ugmRkm5+WBFUvpBz zxyUrK@0Ms4_E&sD^`uuCzrGA?@g7dz&O>W0;l3vqhqJa{Q7O{Yj#>;N$XuH z8yrVW@ql|JOJgt3R&CJgxaGO;qM;#8`FIay6&aFeaV*(O$$-w-xDw{W20u113d`oC zkS)ERj9XR`*2um|a1Ui;p$Xsa{J53|zQJ&??Y$S|lTN~hEEA~x7xY2As%^ip!%IRz z(^}JQvIpyrD~>hW9YF`!<1{veI9N~@ur!8K?`l2aXO5|{IZFDO5U|{xquewY*FUb* zN)Iul#I}}`1DTZnBfjvNBw2QiD_2KCOPSdGp9)Hs4OM{4=;j84@CB_UjZWZYq9*9Z85-!yQe|l|@IIQIz&1-eTtuMjKa;wx2osM!z~l zbA$z5BP>4Z5_T&6d=wrXpPIrIEKkgHH z=wM#ukbXx2gm^Y5how!B9lvRDy@}rmqSj7~J|r?H!4adA^|cJ3HZPG>9h-0*gSrbz z>}g->pO>^o2=3&6ZM zd^J~Z{#O0?oNG~6urIqff2=Rw#}ya|!OG#oZ0F_8)6N*qc26o1yDo;2D+Vula2W=N zfFGA;{+%^38IEKY*#%Yh)2dVuf-p1g&eB3jku%? z5}13Q6?iOh*)-p02;G%8rhUt7rTdOq3MT3doxKi?uDq$AfUGKhD4OxPf9SdBySr#TI3VHTTYqOD;B=_l?LBDi^Rd)Xe&`9cSU#BWk|N{rza5rtp;FOIu$UKD!Np zW*H(mBgeM8MFiMk$a6Mn@#&3uG);jA!t5r9Cv9+E^Ku zb3t~KI&$~?c>TU}3lI19M$9v2S7HQdPhJ%b4bsD?b><#yoaL{j79+REKz!wg@aLL` zB_WOD4ZfGas2#ya>I=Q$nx|aZ81BjeMeB@ZK6IY=BO&*ThN&*dZbAFI!Psr>{Mf`P zJx0zs+|t|;A;ooJdFg_P_we9$38rSoywGik<-AjG3MRD}3pk3zkzOwF-YSch-?O9d zQtowJ7%tDMA1Gu^eR*bA{80EX7-G*Wj-JxpXBbY`U{i>`>m9_o_t~=ZS7p;e1$g>x zYc`caX5>N1Q~T*)5RCYSEl@rVGK9pJ356QD&iGI`l1!W?kccD-pp8zqnDZd`V-5b9 zrAxFntOr8o$>3@F{)|xH(^oVIT@PE}tqTJ;LKnt_kBI2DejR$+fn~cw={&M|xo>k6 z;(2->6yLUR7|#2-abQK#n_==-#p`<`n+Or?kU$XU^t54R^u-7Zr%PyOvXj%oFtw{a zD8joefN6EuYFgqHt)qMb*vK;6Qpq+2OQlY4*^0XX#AWh=y~yxTL|l5U+1|lVS-!G1 zjP#kapgTZ@K3};Xyx?L)*XML;fo=J@RWk5?8Dg&V=pOZbM~;)O6y&m#|s^iJe|`OtXeL_W|SfNXLAg(zi{;+A5$_xY_Z#tdOb5Z!h0eZPW&BSoV8 zMK6wMW)5$P3-|8(@ZU6>tyWdwD68&`!GvJTNxTZG{thl#ruq9FT1Oc(dnv0X&h=I= znd7sA7ZDZSiq+PGtSsjY6A=JcmrVl$r5^@g%XMEyl2sj6@k0(^gRzMwRk-I9Pfd)D zVz-aRxIB#9q>jNp_FA)FDXKbgyZYJatpZ8`2VwK4r@ruQz_M zWGPNvTG^X_PmnonhGrCM@=Z!V%8~I2L$=|9$z8H|8Ro^eGTZ(aOUR@+bM-r=lcb7GJ*S&kt*ndFN#C>=m z#BzT8kylx;m6cS~GrBt*Xol@0H4WEOOThQkV<{;cUkrHR3B%tEH9=Z!{Fs-Z9q2l1 zyR4qiQWrt*8L}dUTY?l%OXHfU7|f79bzf& zy861}C$~&yx>OPko%49NNd+H~T<%^UlvRD!r@)geTsm4 z41^W{@L|z-zRLc9UX&)@%LA(v-qWsn}ogHPWiq^wmge zf=F%DxM2JR*)!-8w@<(4p)f3w*frsj%21uV?v_gu)Pw$7y5zbyJtcU5Buc*SY0TlS zxZ>n*r{+j3*s`%1fWuzANK)tyVjrg7WpC_qpE69TnmN_ChqUhT+NI0^NaLnTw3fne zejZKiEmbgE6ih}?(k;<76TUARZbWeHpwi8y+e2Wx2I;D;*e4e~QV_FIp1n|pCiInG z!PrhAAx$PGPlyg4XQWfRB+Ul8s|~W#?wp^O%5;X!ZT1G6{D9LKN6B9pJbXWvA3LKN z7F)oXUhX@JZ|hZ+R!dW$=EL0ga(GgOUkfJ2Q9mN<9^_B4X`JM56&pWbhWN?KUXlDzBl=qN6S06 zdcnp(o>~h=$DNl`R-XySkQ3H3AW-Ol&(8l>XD? zG&NoKSfMT30s*fn{AO01^4I8m!*bm(4z#EYdVzoq$()2L`N*5FejqQm-L z8N>#-F6UjnJqK%(ntuS5h6JjSqeWnd3D(klYomtRB7)_LILMP#UwO(lHUpb~uR#VB z!##idf#o|Ww1xtk_^_K6LUTH&e#_ZL|yvy~4jUNHu8bbDGXZzL}x$?8b*LI!crD9c#Pzp2=_5A0iV zo>)tp-nngH8-Wl&e&|y6M0j1~bQH#e!qXG-GCUvlYsAisj^XYbfO@DrK0J3a-)sTW zSf>{A4nC$oA$y~^4tlxbwJuil@xnO<&V()5eO=cpYg7Ql?K?lvYVS zhKM$vdNEZfg@^H0nK#zhi1rP%!(^I@oFDy_{amY+f+1f%uM|sXwuJ zcUo(Sr@tY2W9F#C)^Qktbnm|0!%l^B-r$fVe$nOG?_D8OjJNRQ&Y9x?lT%!2aJ;5~ zeLt$Wg_H`J)x*noS0S-uKi+>lgu^xT3$EL<y(XI#)DP88ncVMm#*SvDbebB-{v7!F=q`S1;yKzKwh^!6zR})NPBq7l^9} z@L9;#nspb-!k^7Ku+pJ;4aNa1X*!!7Sa2$H z=jp7eN63h^3OHi*INS5glBG8xuwp%C(jC3P1>ayI0N&C zf20DOX%5ek4NEhW(3}z;i%%4-qL;f09z&}3Jxh(rBBTx1!)SPMwi>6$cTovVop zD64KU!?!CNGK({Sq<3D{LL_z=B zh-z<9{0aI7#cz>55&E#OK}HsX!ji}zq`pJHM<$}oP~la7#AJolWCddbrE6_Oa&_W8 zUD)`C)2pH{SjD#zviFUqu?1MATg|-gc+dCgE~$)$h4!%ky1h_^Q7?^!AXJzb7Uv`o zGzMbBGjmA$&SCS#lCJVcw%x7ga&erwouaz%hlL;@1b)W3pS&3=Fh@#Y z6+yEkQ^~JOLv6O0=+f~q8u~O8+qNr=1iC0h>y-XD`RHzo*$jP}V+@JBiu5FAv()X5 znh^Y}0&l$&%(^Jvo*6;>G}F~$pWPtw`>-Z-f$SVn7P1l*N#3XdRc<5Z;*>_k$eTu0 z#v^-LOG&^!!Wuc9Kswi-WrGf}IO+H|PC^FklGGJv+L<&?4%Sq?`ad`=thuK|q_RE{ z6(8Tk*&?~hp2=ssQG;^$qs2tD18WMPxBo~oRqDZ(QvGs&0;Htq=iC#YwGzy`Rn^kI z%OEd>Qo4EZy%6oU!kjPp;*QE}SJv_j5mpPsbS|Sb(L9sOEG04)@u;ej8V6v!s}#ah zOvot>ro!#T)8x9O(&!g!Y{v#vTZYPc&^(uB5RK8!nFsAt#RJpe+7=jpKs08;`PucG zUJr#3xjn z^q!3Rq11l*MLn}x8McnzKs@-Qz4b6cCL;6fEj;?6T=Q_eYc{}K&pw-(H^v^?vNlVv zcM6$MaiyLTc2v+D_x?efw!(xZ$KwP&b~(HHI|5$X=C*#l1}jM>NZC#WZl#p_2qs6y#w z&C<_KBwtGgV=R4F+s*}2Ml%mxg&Y@N&LSw=Sf*bF8HSB!1}yMFzU}-!*K!U{O2UrU z47B7K>b5#WQJZyE863yliW%UXVb+`(py~zaP#sHEKIAv5--}RhBAYe7&XMP5 zA`ir^>PdgXa~{$5f_9Q~ebPp_I@=~LVQyzmbD5BvkuZJBA)5q~|Jm@4PZGt(}ab7iBKy+Ge6mi)!4HqBRjArQ;{`bN|}={B#N zjO!KB*ac({j@IrCteUca5+5(zoIyg^-4W@;-waxUh;>Kb8}WpHyx)qiMUo7OeuNdB zRNe}F#O)oEg=7SU-rTx{U>g?^eI=ORGUO%fnm`4{KHLe^4Fs8e+Z}@?9&KBTI7wAC zrbPWY&~YbanvVTN=%dCSO9Jz&yq+GDnP##QKU{X`Ip3KCKUm_YRPf?v@sL2vGE07O-kyvq?imGGE&BJkz{AX_SbC_^baY55%#;eq^&% zD22~ZRgii}-eRvc!5f0{tE@ib zLfck|W|9aQzi&jgmYbiNA$+OJJ~&QVdthjECQv9Q?c1o@*@qubq%&z96BBa`t@Ga& zOTm#DSgg`Xa{Y~NH>R#Hti&=1g*_ktCCD|7Be;iIJOnu(i|NchjU8Zq#zf+tCpoCw z4>e?7G8Y{!L__^X2nWT7otavWaVd|gar6^z<`KI{XlHYiZnt3_Me*p>pRwzOjh>=D zZxLk0(0PBRz4J)SW_+Bj{4fL2lwaf;;}qQ112hQJg{qLV!jT~RAwiw@h8X@gjMExO zOB;!_0dax-kRJ@K2LvQ?6^>o3VDlOitN+^BdKL);C@B;3tL1;8Iind#6=g(}AdQ@hX`Faw-ZaS`;Zrn@v1lDhNc`Mr`w@qCV5?@#!6Gam_x zdU;V^hu!EDO3?o;DN2kB*lUNJ)xYQ}a+%N=-z*}>LCGa{_SD=_0!}3w{LyK)rtH{* z;|TfFIW@qLh}$`INvdMldvXo)ngp&Vj6r9<{3`-&bRCORM4Ya{fidm-14;--AWnGi zd+}yn$g?u-mj>;K?c?DBy5KP zGygg=9tV;R1yIEu-MEO*9N-(C|LKyU**RxhmdX=S_o7STS0Cx)!dSZSW^^q+_9h<4gl{EYi9O zjabk^z157FWFnUniaq7(pZe~4tcDnNxs^=CJN-!by1{fE62WP*xk3?aPApC~q~;OC zQ)+q<9FF2j+Q`%qMqE-W@rEpja5HA8btp_@Y_Vt^iQs2BnyA!ur zy-j|8%jQBO#MBOS+y7T&&3TZ!FZ2=HN%FgLm;0_r24gdvniwr$82N0mP|WL8HheN$ z1M%UvHy^>=U`sVDDCW;G`=iC9Q(Dtg2>5|)nZVr_ZNppS}?_JZ`tgNT(taie6>}l zs40-7S;Y%f6biso#`YS*Iytmxqw4!__9zYGhEKUiPaCMa#|xM7eMAY%Y_gPI2XI)b zQPdo;xbAf!gC1tSsrAA%RP@>5KH(!l_Yel$zYn8vQ9$Q9a!|-nbxgyGeY?{GkE(Yj z8rsJ0?87r4hIt)9f`zi_lpgSOh4l42aEv)>jLOYLmZ4Rc2nJTf`UgFj)sS~cphWVx zXmxdM*SMM2uJ~ON!Z@CRm@8SInNL=*JL6Hx*YDmbB~rJY1yY^-s=Ka?l&rOzdpuKX z8CpMxnhxC7N6z;*R#xHl50;035Kt9`>U%}^-U4_M-v%3VY?X(y=co)I&3MY9{~daX zq7`c}+am{mi~*vwXR^3d1tDzv-!$f>poUveu~`Of?+B@Scl%%WeO$M2_So;xKg4Xi|asH9s2}^elSSR zIhqGzNs(u8TU`^x=Y9k#|7Ezo0`oy_(x? zK2q*LhWsb>s@7l!D zkB(>L>&^G?aU40Dvw=}d&1lyZIsCR3)~dshcRcLPa}(ozKM3!ATfbacP%l?!`wC?-_LOJgXoe zy}(7(rNAGLN&MnX@)x&@jiQJvRyh0GGjp2hKJz*^v@mWcUX7Z)i!S`kVwhNFd52zr zfcrtzvJk3mx2@DX61EgWH$KcFhmD)a1bA)=QqBTYb6P`Ese5Q@+VvmhGcbrWBUX(w zq@lD_ld}B?_4H4b62u?tlcc1RF$;xG~!bG20lZVn1zDyS&`8+*MIMW|N zU_<{_af$uA<_LBLmY|K#KYz`=NZz>*M?_Tc*%O+Rlr=Pc+-qeyR{dSHD9eua!S_Q0hvq}QZ}k> zNdN!#^#4Zj{X_Wv4;*6mzmuty9CL@UIO*GCxY}*S!6E0h5USF@;YW!;HR(C@P>wZr z|3@+vs_*aJAki<8-;bfQwC?8$7h_6aD!)D63bv`jP`Q@ATQ#3N?-ZW&_5}TX*u43c z0372*-cv(-XT=x`oQ_$yyvvF`fZxGvsYP#)^=3%GZIE^mk8_G06WG6NfTT+BoR(I_ z94yD>b(B)YJ_t!tDec4>aR5rEsw5YU$<8L3uwOQImD6t!cGxffVL6e;UE>{k&MV$b zC=qar(K<5q{{2n7Z2y{-622o%lB$-@mrsBzC>AGQl%Gh=W0`NLr~*DhbC$kB&0tSg zoymGRrrMILv<@4qzKGF{LH`ze(ZZ~;x>c5Doi0^B##{%(-^aOSs(~*;WtLV|k!)N@T|BIz#2Nhr3}LrtQU3 ztEWZ$i@B>Fzuw@yvPc%{U}2(gq&KmdEMj9{yJ?d2&^=q9SFy%jImDs=?Akd?=#$oI z=|gDh6JavK>{NCWXI|~AyC+HRcaCgZjevYAXNf_{HG|w^n-g)ghEtnUnu9-gVYOVm z7h2t5hv-L;SXq(OysCp#ddmhs0bDpskzXZ&G=?$MM`(T;m=HLJPtxXiPl#R_n&5WkGv??6B)eH zUJBcK9N%9Zmjg+Sc8sI73$phOi30ciYK-Y+6D>01;2t^3^c82nLnXZK3FRV3-H_rN_lvSp zZZDI)sUwv8ovMeGr4!G1;q-T%G;i^2f--T=2_FET&;x2tg=(<2)5O$_Fn5MhZ z`$C@cj7>f5TP-08!$wYcMZM^A^LZ(Db*JUsG002wzQN=#0MB=4$dKQhHb&Kph)bMf zJa!TITuz+xh|xSXiTkYiu#>}!wcej-&sgjis(y#`7{~}7hcWM3ko*{&`ZU_6V|*o- zlDqfc3!9tVkI|lg7Cjh=+vIylnRAVfvPM@U#UY`ph&>>6L+v~fg-9Noe}VtMM4!ps z$E_XuKjeY_e|~DaI=Q+1SI~|wod42t{@16~C&yRzmjXB(B(4N(B>wmao(C)~7vL>5jTaR~AR!RN#wN^*>=2d&cm19SJeY2f|oqk8; zrq)})*H;iV$HcejW#t{eub03ZFJmXrai#Gm{13eQ#$Z^XKX)7u$o{921c40^#<(CV zA+=iLGf*2CiqIpF2ecOXHRQB>D?s#{N9Bib6%YgB2^2&Wq=EBUb}$DmwI_}ur1>C= z49*p$Ot}9Tbu5|O3E0a^U zl?_vr0@ybw%mL^su%q;H)@SUq{fO*}kk;=O0HGDj5EQqvB?^pXfNbUj{0Sy;3^3Vk zsm*P9Ob26&;^a*2IJufQyF$JzSW>N6o(d|RK2f9Ar(5)m1Hi6+$2P^#VwoUqyX4;| z<{2E^Xizm9Vw)vz<6k0QAN*9oZdoA4u9f-PZD3*2Vqjk+i+!te?yhTNQp>nt*%a)c zrhE`ptfhIrk8YSjzk$@v$YIgNdEP9>3P4Mc7P}%Cypy2BHr#ELA)SJRLNK3~V8>K+ zgl(%FWY+oOgPeWV$t`MS!*;4g_K48)46D^QBO(fGIF{9?+|b&_waaC|oo~vBBr1-u zv8L}LUs|7_NU`fBZ)MQPmy_D4C7_6=1p8%2;V`elj)j7N{Up#ZQ?ZEmy*9z!y`4cU zIYY(I+{THwp{hDjGau+pix)eWubg(hA;X!H$FiowZ39`mDL&ujQ!L=4mOuI%(Cc-* zgaqzjT9=N9VizYd#Gp`k z=dm=96v1T9mx5QSNg7}A97!resuD9gOtvr0`ImhDnq-f+nbnVK{>WpSqY?W3rN9Dtqt@ey9D4>do9 z&>fu=TMnI~q=xRouZkter-wvgvxj!m7%j?xYf6L()38qXkVkD)j9U zMy!Up;6+8sNed~vnwI=#>g7m#tA+`CMwuZCb3+2H&#Q#B9btMN#~05E^A7HR!)TZ{ zLt<<7K;w)tu=_2oB8Euf@0Ffn;kWk&D_`{fueuFq%0PEpS--U;7OM+4*9JJ{Vmb4> zf(&Qt6sq`&<&t7#d{+zp^dqZS)o2ZCYsLbFI+->aiu@~}yO4W^VmVHirzAQWb?W&* zT8btCChL$wI_x@C749UJq1HliHck3V_T$Cr$cGV8q9K~2>Vk?UBSoc5p;h2qaDHtG zgMvZJ+_ZE6K#EIRpCjkMvJj65!ThQk*$Tnf8nA`$Xxcfpf7asu`Iu!~7Z!J@L!a%j zKi4f;?<*jdUtvs@#`NG~`5ML^K^4rhw1MxaxJxG^=_O^}Q~Lcefr(R1mbTmLSKDsV z%ra{%mi)I=J}a`pc=|G@D)^f27;AgR(aDluDRMQ`;hN7NC3u-PrreC~14 z(`Ee@v2P>8v**BaVb8|s;TGTm$^5i=`O>To`TpC34u9`$aC)v#>Y zX3U(-6{s7_oYQwRx}ahP6JU_(5va>T;J$vEsd#!PpGx1e7`c%OvP4LDb>Vn^JyP4E zg$;aWKaxhAPAXNN4{gM@SB8}T>zZw^Idl}b9%0@c_OZaow6yswk-n~i3Y~bMg-?$A zD9?cWwvOpg8BsRFB0yO(g%!xXarOO|g+QD!KjnF5miyb09Gfu4xzz#--PXCa?%GwR zAn*92M$Yylq`YpN+=Dycc|UC%0nrnX!E#C?J<0$}tNBuslR32(dM)Z`o<)0#tJ@&g z{7wPqAoC(^#6LShrMSn)0Sl>WBd`lhFvR8QvR+^Q-Is~oz&5+IMnl%8bOFzYaAy~C z>*rperRJIKzI+%{uw1IZV7Zp3OxkqMRO%Cn-5c~I5jCqYs6$rTE&}o#p`PVhl6{-4hDu1ooJt&ieW$NUknZz)VkJJl+ zx}d3>_Ny!M4OdMREiYgrts<;b8wn;IVrPnwo%iv4G9csuI(rTi^f!=?5~Yy<&p%R} zsW_AA&ct}28*rK#dDv*Og7>qLF+7bmz$YB+6lM#n*Cfe zK^|YF>_fJ-PCCXQlX`(EJBcgvl=IN+DkgkKkO9kvZ^E89|>!T;nZRggJ~YP{}h!vQSIZ4ZpKMai*Rp3NQvZmx$@J`YVl8zjKeJS<-4GX2628i zKdlX%q`7K&Mfx$qRIUkY*uLr0!Bomz+3BaSHZ8rcba(!$#)LW27hiZs*^{j&6Z_XqJD?jjnu8r968L z?^k=QW@8PDSrj7O_^Vox)%)o}UjvfRGHjx&$6X}!(l;{m$rfP#UR0PlAU+r>Ax+Bp z5&bplt-Q#4|K%m`non_XeS*ZSHojK&vBb``0$C%Bq;h<<$vnELj5czt%7-U#Do^K#m9b>>-UpzMDXrm+v+|-0A1>a>GzlGbz4oJ&w@H;o=Fkz?|gEL1Tz# zHxDi~D5JQ~iyz@uC@v<5(rmSs8Wu87FWB+5X=EE8VaX-yeYVJX98U&mjkLrVX&`j7 z2?ku(xsZ$@7R;5EcKWi(|A}39M^NrJ?}92&XINx9hWrKQ#9}+VRE)-&R#Gr{rDRKf zIz{f$XEr3w4b3ipgf;>d6Zbt90me@iypN)l1tk=vRHD`e<`J=?OT&3}VwviCXpKfR zaZt-(Gt}oW^SqG78xzUa0}YA;d}lv4+TXmt_R<2rz5j#TK6nhHgqcKQb%NyHrr4v* ze@*@HqiU2lK~fP+IU49Hhk_lHPNu&*eApt9otZHsp;stpj+c%N^($QtQ5^?iJWoEk z_1BVgc436Jk4-{G2K_cJADT6xDFBovNGx4kA{Rsvi)2Xi^3s)+t({#~hFsg#7gc>` zhByWuCrU(ZPBz&D+tP&l&2RZiRJK+ln z&z{}6n7x%W~CstkratBHNHhD`=K${7;{r8jq+tZru~_ z>ADsYnOhE?uvolR3iE34gAOjLf)?Vqw0dP9i69_uo{-53YjJqcBfjUr4R0ulB0HRA zX8&r7iR#7exobe~1=R!GFdz*D!GwtKXK-VUh9HL%J0Ml_*(dphB}ujJ&R0Vfm2kvT z_J*^%6(QNi-1M7Ev6<6%9`|tJ!*6UI|GWY>Hqmp>`9`WPAsCEpvYQ^ME}UcWcW>y? z3xi**%;lhbTd}wjdo1dasV=|x*m^?H&s@xoT+}%=Sc_jV?=TwyFRr>2DPrEXJm}d= z8KT~`f2VV+vtp;F(=r;W1&jqO!h;wn9n$XIsKPqlVDvnehL|RTCR{1)eXvKH4*Wv> z>-K0l1XDN!XLh|`S?^ou!m_c$dM0d53`&YTJzz3KL#w&r5X)L5)j_pIf3C2*7dVE} z=-Nzw+^#6kJ*8azG6*&i7oQn-sxEuwtZvrL$TwFn&}`0+Ivmw5;bad2j&$@t)T)t5k~x$ign3Y_x1)c6v_Dyu6t9za=AL?I5+ z3&&{t*{G63KkeMI{ZQ8z(}g$*|IM6==)`eH&tD$2f39|k8}f`f;FG2FH9_#Cn;zhA zV*ZRUajma&Wj8hSbb|TULhqBZ>Ds>lSpxa|#rS#1 z;s;ojH*AwL(UI3#Y)L|90{>yCH}?G#&ilBy8^mY&imzBSzqs#B$&mNn-d9+=S^5b} zV%5!FQltvif?4`0733;yTnVRNOdI}?eT|PtlKS5H6q$f?J?__-?IAg#^gAjy^J)p< z0H?>9a;z!u8dZkS3*!9rQPL-_xq`N&)n~@qg7jLCWVX|*Gj7W*2Kzx_wuAF@wusf= z{9FQ)AwiqpXBe@jKS$eGxjwg|!~dD`JgmIbI2j~b)t|O7aA#iC<=o9OwBb9*4LjM| z6!MFv;$FJULvB~q+tsqt+n{Hl0HjMjNOx0|@u(xsi)k|uu3hw|BkceY;v)7_%meeHf1yHoI=&6lB z#^Rtipfr=XZzd^uiZt!B=M(mG4Iq=)toQ`Y0p2!^e=cQN$y|g@A-Pr{x#|+H#vH{= zK!GSC>a2EIkuxmewPecw&bx9bL=~DTke@L#q91h302ou3C8_^{o4f|E7!k_|q{)ap zwUx0h`H4CCQl3PVDodYfN+9b%?>;8NK^CZEp||F6yu1rh(jA(WC3n&wTPqtXkB9W3*m-i)7o%^ys!gK;Osr%Ltpm|25yPAYF|-_Q05?L^W2Q?N3J>3@c{%H&(@k zcbtUi`%+|ZY7a`}Uz%NbAP>)GiI*bUaNm5yNuT;>unR%(WT2lrQ8LdW6B0Ok14C-2 zqPBbgPuM|JPKG?#cOpAHtDdijqD&AYBEkwy5ixg&?*P%&u(iQhy2v2LDMV}yCc z{4ryxtb$uOoqAcro7KAUr4lmstec=uQy598@0)9>;?A7Rcb$Kq1~75h)m7N05&I)k zS|=7PSM+N_KU=tIs+adve8X8>sxde{K+i2D>P!)N824Fpqi5jr2t^5Sq zFJKanThvGerItA9Ft-{=aKrQ#5tUF_<4ensNS199~XQ%*w^p%HHIE z!`-^IpgqtIvHiZbrWG2oMWC?Y3nikE66tV38stG!!L5R&g@OwrYcA_~>4N|8o)fS& zSXZ^S5ZKjd#_Z{fC(kPpf$Sm2%`b01nf-ffeDdx2nz}G=(st6!!^oZb9DVXXe)Ij} zyxZ)J;{(}&D|np)WYN}RwLLr1+E)!hI{gVk?r;td;`LRkZ$kOKdnL0O>^3cFNitcl9yK8ijS|oXJDtQRxhY`3CrGI=?3X2;9mIwI?OpPiKS_#ld zzD+s_5pZb*GDoNF$G`K_{f-?Q4MC*6Nmsjbf>4wipgc5D$!+5jDA}L^eetyWBX{_zGMf@!m3d|gcOLsK8VdTnP9rs zpd+0_n7d^*n94+zytYW`Fb03jlJCT(8wI}Fvmzr{I?=bwBZ5y`CNF|qg~Vua&Dn)* zfOlO=Oi9&XtZv53;&BdukmAHiS(BEFceWVu6UU8>k3?YD&`RMSEBq4GQ?#UJb*dHE zY%MnITd)dq4P4ujkozYu*>(~szn%)EY27LBe7<6deLPc0*)lB!^Bgs2KBz;Q!D8Hx zc6luY1eKL#4KnD``lamD3Y8rGpr%Qes{mbXsio3f4u|SoP82Rc5Md2E2{1}%jsSba zW>LBq8+a@`F-|WUufN}Sw+qL}@7WbHr0L&DoNHmP*ntI2Ifvs%nHW?1W6aDC#0Dlx z=NQ;CDhGygPZh(lDzF;F5^jb`qiS^B1KprmQ|Cf)ijE$cwNSE%&R{I~+GaK2r4bB%K z|Bhiy6wugKC~8qEP|dY2RlOIbz}Asb?3E!+9TbY8ypJbXOp<05KGrx6+3Md|?z%Y?ptvV9qIXoju`&nRE=G%aBkXi)X%Bc6 z*7VI=%N{ntmTP7LWnnDRW0VD)T_)KVou~d#rRDxs_Ga4C+mfRpczVb<6;m>_5UZZ# z@aASnd96ATDg!W@bO+UyKhyM-?wr%~1Uj(5%Y+sK9CkeeJ0Q9k4>!wqHfX+IgVQ|c z0{)Q_JIa)0#D<)im{43ZR3!ZJ)&W+i%d(W@PBo zg}PDJH;jx1tb${yXXKup z4GIJRhZ9m}e1^Q2B&RZFSp%6FRgFMkxXNv7-rBLN8{CXua!EZPQ!#4gEK&EeOQDU$ zawL3ZNW>?67j`h0;^#L}7ni8Q^m=d{i014$py zk#X6oI$x4ke@J&Uk(9(9yi>SPyc z1^>2cEUc#SK#tZpgCXuaaSt4`kF6xf??RFKyqO4Wvfc~2%>L!Z>^lkKJjR=c2i7;P zzd|co;{_V>4jAUXy;N1bX8n9%HhM!>+mN>JLT{n(cqf?8#=#yZdeIayySp%I_=~$s zE6@#ZX~Wb5-@w6_RK>%UKVuBoaYE4|gy@Cedbx~OZpHUQb02H-YVf+9DY6k*4PGbi zlsc^6Qh|RZ;x1H7LfMe^-mS+gqOi?d&p24`4LZ@sHQ}9!P$aU$@!DKIAJwl({Pj6&98DQH6@HQC7jzrf{>?HNdnu# zh=$^SyP~!`K#q46YzMkeDF4BZ`?9(xc`^R1Ck*C_;MC6LCpLyz0ajEp&mR941 zX1H*Y3alY5$Ik!8d9i1>PbYFsSuOXY6CAZUyt2wSPG3Bc3PuYt!gaD}vLFqv#K);k zenD$?WtJ^&V69I&C*N}G zO>oOPvz4d;{H+E?Rb*`UGT#@y(e<~OM4k!y5|yF<{>4U=>cPCAk*ZVxrJ-P}*hrRr*h*prVrP!35#9p%7% zY~7FYV-0+?;~{Dh=uT5jm5px8z+ft_k;aiPJQ~x3@rD?^v0bAMI{Y?ni}_4J&iIE`aa9UUhwtb;g3J|-fytrU&`e*`2y<>Pafw_CEaJY& z&Wo{t-xjVQp~GwkCw%9?jXpB5Sr zqXt2&xg~T*tX7aEE}2uaM>7hOx>Rxt%@muc_y^~2tIh6UbdPO~L$B>q>5O7(cgMs5 z7$ibPU3Xe@wIz~#YbvJy^wYNfBzmisj6KdjEZL-Ab8<239FqBnn=FuN#O_vCWVfN- z-)AeSnJ!flU2k*J|DA*$#Le*$oaPx<%Z?fjzdrl$m=>s-k#n*m>P6H_v?mcVn}^Z3 z*3tL}_`lZ?k={;i^WWaa+qbvz-;SA9?pCJ%J7$(>oNA(KqJ8m0VY48J0FCu!`K^=c z;Zm02#o!~vRr7<|UGWoFnA}rc%|!~LTHZce2wyc9Rbz6!6ch!Y;?7RHSw-PwiM6;q z9kzX+KCd3HeEAO0$pT)V^kYQc3SCply;{c# zHI}qndGgv<8X0S?6+r6dSgg?9t`>h7FcVd`(<(Gsc$yU$uq{!jC%Tnw z?&LO7;{m~KfBRys%gym&n;d51p&SgM8w{o?ti3a}AgMnHd5!jr1FFVGce^JFFRNuP zPH6R=9Bhd-1P!0J7aKO~s5xlm!%%Nw7vYS)KqC`(SiYJvc4QB4xFZX1Xe89Xg45HF zKcSEp=74PPTej<+jXK7WOS|kJHV<^{pnet}jC+Mr>;;|k~SvZu0^B=EZ~ zrLvG_UB!LVB}a&7cwPAuQFhP-$y;r-YL#}Fz5c`bvXSz=b2GNny#r;81mYc|CG)Z$ z*ip1M%lKSyhY*-;TimtsO%dVv{D&gaXPB&~a!%maJip+s|Jh;*9ws0mN|xMMqL~Q% zEz6T;5>b2qIxyqZZ}nI?!tws~s-@fSs2A=@IT$wIq0B4&!jjIxcd8Wpr#Vm`{x4jM zP!s6~e9Oc$j&WeIGJ`CF#owPC)&GK>67Jy57syE+ixd7tu6r}Mz}B?8fqd=h7A1c`p0cSAXp7EEb4z;p@KrHa18^$YOD*+UgxpE0S~W8c>x z#k?}aSn0^;XM9FxUY`0wIWd~f%$3i>nI?!K#t@aq$7po#@=6L5zCwrey22r~ zClg9q1P zh$FYyFo>$`MADbH)jh-u%b z{~VEHhI9Pu|1sL?TIR-*zRP3ucX?#^@3wH_PHslF|9d>9;r)H``@bMjGD)ZmP$h~3 zI<7=T96}j7{}+rJ8^&8#$VR=p;gvnry%{0v2hsyGl~;X+<>H_c#s_

H3b`^3S*kuMvYOLFNdoY6PXLig%#pxawMEw)@0 z=RZc1#NIOrvjg9ULsV^8kvml3~@6b<*X$kvPpmCF01KTy}93+bY#|nnB18e z7GEwT+5jUK!{RZNXd_jSxN#%b;IP`O<^L2#O;e~Ha1c4%0lhV0tL>Hrb6Z^x?WDJ! z{2$STy5bJ-conK;%di9RRQf&;Xse~{gPfp0yK|#*XJyStVHx9VnC}TCrTULusX5xD z?08>p((9aZNOTr4!Q!0bs$~iPrzU#xw!Gplha+^C>G*{WY4t}tcQTU?FNv3Zhfp^Y zFUUqUnk%gr7hoQBd!<-oS@7s{_0=e=~KTJE84!P{fXEU_JbVW#*saG zM>cGxTR4LX`v`DNHU_?AW@#1|=4Na84D)Rfee-8jtg8WG4o(^b-<6QMR<&i(pIhYv zJxs0puw=&@UT{Ye{`eYWaEbJg0xruNSM|J>(r2yOy)vTmNNUY$wgJqjPE9op>ClWIsGTw^}IQKCUcVu@GSO?TUIR!j}=4dIsDd-3`{ek0YV4qqX?Gy4s77c-)g z^CxRP!t$y*^;|e=!sA+6l73%`%YWL+o)ar0g^+;-ZMnaL*J}+O^#F3g!j+w>P2t3p zpk*S)T>j6K{L`?-0XzFFAP2gFdgtQLfauaVug@EbkG_-CUaYxLRxa2u&H;TyHSMcr zjR*<(#1Yu$EMd|QRQ83HUiemQmR|WH8P<4VJYOkbW!XR&8xX1Xbxx7UnLR*wuQVJ= z6uDZViO*)jwqSRK#*&Zhj5uGzDB?%PMb!{_s6Ry|-yhL|nBESjqMsH3prz@viHLg z&KUyM4GC`hY?E_CZ&GosoWD579ByPs(ixejt4rDv^yYw==|hK7qK{zcLT}MIXIBsQ_95&d!N6o#CIvH|MC3#ai4XF) z|NmjP-`w05uD+#uTowpZeCV^9|o?x(P_KW^&^2uO-^9rIK#!dnB1R zdl(*&-&{42YqxeuJEh*7NaL{TnkO8Pg&-M=h=^c_l;;R#VJNA>t?o!+z%+>JmIa1| zUD?u?pjpbo$m77kN50JZDi$U1rAksP{w;gD`tx+L!T%K7`#6(kNK{DvUL3`9LJ6TJ z{>2dYuUfyMk_{7{kTMd|xO zEpKCcZHbaOwZk;uc1j3l*iDi)S>~D*XFYMNu7@XW<>e-fxS;CGTo{MAbsv{0;B&=< zXqVcTX2_gPI#h zG^y*V(r?bO2@rCC)~G?M2&VU7mbh=tUj$6avKgEe@^{G*3@aGa-oRj99;cAl){ON` zj90;W!mnTk?CDU}p|V#02D)w&v2t>~NHA|p6tqh3Pv15S59@knjcjB9P$l?I5b>Gz zB7d}M<|2+V){90l8LvC2S56L^NR$>3gLxRt_2?GVUZ(v zgy@Rx*6_ue9A^qK&~Vap5GRQjl^zK`ykDuFCpbYt_(#5p zJ`;+Z{VyH^Ze{}Q0u!x@4Sd$UPgAy@Nv>4btA8r+4rU?nzm8?9ATlnH z{jsu32P^=Xn`_!DiP5Co;*DZ77*Qihkt1qU*wLYI=(a*l>KZNm6kGjbNE{WDaD-@^H$pVMojm8LRkKXduXY@l(^D;(1}f3eJz$K?z|MpR zvqronp2{s5mzuf2DNZ7)WbK>nsfeWN>Ot*!VSSR<;Fw9x%v8?Vt;Sib5 zNFP`xSxUSg70$M@Qy@?t!Lf5Xj1Vm{n~ts4$olAi?us z4JXU8lM-;gA?k7gsre;SC#uCoFQ=6NRt5<3(pF5nz7bgl`I6BDs9PyASbxH;m;s%r z(tq)zOOZNsh|uG3k;RsBt60a!3kDDMvggu*FDa$iAQk1~3va8mb44ULUk`Ti9j!}d z(@KQ6r%0e`^VRZ4Z8|E|^1Z`hBKY!5kue`~r}=35~>sx+fCwD^75 zH{w6B5oGY-Ro6ZlD#jKwlblz~y9Q0%e4iqsfR!WAKBovQxFucN!}|nCEV~JR&OwRK zsSd3pOZPy)j^WV~3wMJhYotn4qko^F71q!Ep{jNYo{8BkzOm2A@{JyC>XtMUUZHrLb?5~97Z7|p zw2r6z#gTJW{KwUcaElb~V3FlU6=dPQA;nXb25VrX`VQGk?f6Ggsdee$77wng+&)4M z?nMO-8*QuQQ-cd(JX7L(#NS2Ip?%#?`Ylo>?r*5*JzG5L> zV#ku9^R_OdE|f(WGTw_)?V&+H+QVqV{ZY494qr-63tbRO&zF!9!lf7JPc4Qeq|2~* z_u`SgU*4CH{-rOZPNgbs)S0%-h))vc;=Iqi9f3(Xbb_|q4C;y@0t$*}rrDlw`U$eL zxh3pbY+gSv-OAK58f6TZi~OWl)0ko?pG+!>??;mYm?tW6DSTGgj}$7IkYeyRu&f-Q zB$VD24}X2os%AZ}y|tlj=c+w1cSTkh!xgyu&l|QuAl^+#AnLTT30|gT6(c*7(Xm?Z z-t))ycI*2%cSwR0%c26tyo?Jq-u&wqby&4H!h$|S)^LRM1t)C5&z95-{EGj)e@>r~ z56u{nA**nu>4_GSBdLbzxdf<-AI{uR@g@O{POV)!hnlYHhA(6(v}nM~q-wKD-kY=5 z_`#$(D)Gn0TCUS!8*Vy}m{*JtY*#;#tV3^jp<;m7c6KP~k1}pbQ-~pO{+oPE9C01# z)YxtoM*84peb911svqr?>39sE3laqun_4Lb0#PAnx0IRD+r@gf%;wQdW4z)DRRPoO(rcp@=H`&P|QP&!{Ql$&NFS$l$Y0prbPELkM5=iT0xYroX~ zQ8v&?dA+tkNGR=$&hv63nYpcwmf$ksk1vFP_dDUpuMq-&KIc$tD&R75^rV1BrrlfR zco>@kd3vt8yV1TBR$zH@O=2`INZx}ka2~pImJ$s!y6&07{|AORxeJkH3UR6#Oqm}7 zj(&X8N41no1%M}FOoFu{sWXC_O3-dz`F-~yszLTusY|vVqiEwMdjFh>>B8fjz~SLX zL)j7c1aA=&s)xb=4N%8+1W8lbjzW{^_2ENhHXT8ZMUFROqk3%_|>%_FUe17uAx z;`Uh2h$ikW#Je$c=Uo+@GUZveq?PWSlE%2Ja9<8sld7wr(wA*D2WB=jVisX^Mm{dk zyZ)#ZgbHu&zX%r|MU>%OaH{5$YMUo@#pRa(F)KK%pK;a!Pv75h#lv_b+KN-=&VDBL z5*kMaTMkbV&1Pfk0u7*XQmj z!y1m+IE@SKnU)zd$OA2IlnI@J@Eg(>Zh5D9yXxl2>6KqL7e|&r*Xf^Dpce~xV+jP0 zFeQE9qlt!A3Ow*#Bk}@7XD(n>`vf~*ob)!O@}wIzCfn>&w$+Vh&aKazl6XtIo=zAsqwGgc1%7S1tQuRl}oE}0FFmdZ9yCbyNvZ> zrKdff!aVGOZDohlRiWjf-?t)UMAQ%jsK=^4K z^8{F%xfceS-Kid%g+LX{6`Du7+ZDVL%4esFZwB7D5rw9$LT!Xcci53|2>g_B1 zVs<$Lg5p;`o)i9<(3Xcs=Qr`pGyFognnw_wyoOezc>6wN{$4)+c>83cZPKC`xH#L& z-^qitS&&w+M9t3ALFK6lL96SE8yY-S;g;T2mw1;{E_=*EJ;LkIx4vlxmw)cp&V<*8 z$D*wO9q^mDa#xNmnJ6Vnwz!bx<0v-q?yzAazNSwy*fm0qGoq^&sX&mB6Yk<^B8|M! z*QkJJ82Wt7G1+4)>=0dH!(qQCi`C#yJhKOq%YxnjZo{z2UI~j++<;G7uaq8X!NOi; zg^Pd^x_JdC3U2DqByzc{qXmf8% zSN>FF3kZ+nC!AKY5vf5Yha)`>c;P>iqaMkm)4cju-)0!WA5(E44|S_Heo{6>w_fSB zfqsS2m|(GxN6)HQdV^JoZl zxwCLS$h$#J_=M2m)XUWHe<8ZYst1&#~6#Q>=TB z_SdI-`l;Q{cHAhb@)R}s=oQy`7*(9UV~tLCSXnF%4OFKc>owCG7~q3jD6}b%^^*_| zo!_il#3R(tfkDV0Xs^9tdSBStpDv4gD+PnN^;3*W;eODtpD=Gbbw*WfR*>Dn@!ev6 zR2*HCqDRe#>!%K<(es@-p9cAWqYMsU+i0(lA%*;Qst?+#?`=y{hDaEJ2aNLt^UK24lcXEetooOi=d`(n1`eU>(`~ZBZb{n23QFpc zcL^!=)At%xkyQNA?!rW`r{?!CX6_+-^vBFjRgt%BDJfip?8-6@O{x4STck`M2fd8{ zRQ{Gg@^6FiIM4af*qzx?-R_yku+OhatF{N2I}uOsl2jf^Hv=e(iqVteWKXEllo1t+ z{&?{4_pzQ*_wfy)FOD|Nmu!VoNDr*V#&nM1)To$$F>q-Vs($98cg~h>-O-^)TKCt3 z$vyjP7}4AOon4&^$G>JA#sx0F_pPt4=-SB#LTVG7lszWQ^xKM7vUG+9vF(GJo*+uP z!73b_Nmo^M7#J0DW{RjEf4`2$=PAleL#|}ce`u0PAU)L7F|N6vdNQ8OZzA zk)XSwL3zkfv82rR--4-v!{Gb`g0#Pr~rtBMm>N zYrd{K^WZo0D1?Z*p@gYI55h+fB{LmiatOm1^uiob#S+Y+7T`ZZDCLXq^h8auNEVQ^ z-*aoC7{X4@Ft`h|bX_s|;da@Fj5<$F!-(Zt(AL6EN}}{QlL6cBCxa2bohRo%drvGP zK0C);L=(6+A?}8Lh~gDQ`Fv;Wg1vP2!IO#;3j*MjFEE0|B8h*V?@4hcM-gxOqbcHA zM-;t;j%88Gz&pWx5mbG%X4iL9yJ1)bQ3;#8faADtxXJt%9kfRD{>n|XD`-)=WdzGr z#tU#MlcbWC%`l>fo<-XS!Vkw|zOfi)W%*5*%}! z_tD+HhbSG>kcfWvuL$wQ{tb2&I*uWfPkta0QA25E{S%n>!7uAQIrmb?J7A*;S3TbF3A}Nv$A6QHLoZe zmNB}CR!tPU_0x-R{c@)fDr&uS_JG<)*weY1U-pRQ;3iKFb54I<@8M`hC)Xr=+g^Al zSC-^*#}0xMg5b;{zMvxJ@%i3wd#~O;__jy^`&9;pVu(Ewd;KZ;@i^IMC+a8tktzKd z#EK}|&Fg@66->>J-8i{bLh%v?zu7Do_Flcgg=H6;Ugr7RIWA!oD-6CJ5o2c+clFL~U7z+(kEy zB6#5&f^Q%__w+YAqgJ-2xvqf2-S5tMx@?i=AyZmRI53M-uo5N(QXcj%;NW}+{j35g zu}hEx4|kyK@WPNod+9*1T7M&Q_34^urQ!hK(C)S+8m>#p$07#IM!Qp>{h`O$?=F48e98D zLP!LrvBmDl8t7nG)R2SRPM8rFhH)voYE`8dR0MYN;1s}16%M9urE}I? z!t74H&(JGy|B025DP4gm3VfAvm2tehPR`10blKqpxAQ3mED{w39SGJGxLYytP}Vnx zc{S3Kg#0!LV2j_rRH*Uuj6ka@2Xj?yb4$QlPL~J~&e=~*_9Y((h-HrXr6F0^kFlj# z)w-B~Eq=0`nbG3r6!!Qz+nT=sn7>gSdh>YrGSTPB4?1LP=KpIgJ9qaLeDEdf4meD1 zpws%h?(=(=(BlV~0QJKd@oHhTmToowEn<60k}8554sEbAYnxMe!WIBOYm{`|jbJRa z-9)+`fGD&dI3uq~LKX^{84vnK+@4q4kUZD4o&z1SkD%UOkt_`hX+7TQ7UNQYm8VZg zUMHdE7RA1}BwqN?Pyx^WP|%tA%d@?2R*Xd)RL?CUcvn6NNu`b-hAyUjYEgH0+|Mpu|B`tC(#@pZ;VYlko+#Hn~}+j zIZj>6DQX$VTS3~aO#G@DOGM%ZUN8vBWp4!?1a+9;N6xHnkcYJp3@k_27+c2*Rrc}) zqt;%t{r>R_e#wg*PTx9mVc{F1MpWdSRo5PqgP7DP@Tk?yjbY`5D(8dCQNCX7n;~^{*0CqcN#_fx46vrL1PK$paC$ss|r+jOZlu^vN8^! zY;Do%1vT&%kZ@bhr0{-k4s)y?22HF>AEgnOIr=gqA$3Kt7A;en)DfiB!qqNGW+X z(V-iY)UHBMm!xWx>L}hf(hbtW_;_x$*p1Kkxe>&;AWnGsi;C+&lu@$YY1OagD?7%? z_?3@C`1?8M2}fdeL{xIvdSpwTU7S-LZJWRuoivhGe{=RmMV@SvVKKYM6(UkL6dFy| z)B0@Wu6n#blgAxmkf-HETK1AHuPNmQV?7X(qvW!BBqZ5|H-{w!A1A%lsFT4xQALS} zlN3kW=oDj3C?p7M$c1Lfjy=-VfHu>?(X zB(1%UN=+yzOyYN5RmzuRNB&-Ca@iaDkEy4|C26csK zkw(hQ?je49Q+yEuYr}($$1!g_%Z!u`Q`?Mlr&n>Qb=Z3E;!zE|F^A>yyYqY|dvkW8 zsZB?g^Zlul*3XPZ^#Uodvcm6i*fRc{fz=;dIQ)^qz{q%wTfhpLc*%eu_gWAJ2k9Qm}gJ3yEj$wXCe_+6n?TpE{d{c?Cr~4N`5JNC4^GB=6p~ zuJ4wU#`-4%E1D|ASR)rM8Vgcas75w3GqQy#c|nXO-Jhehv@$9JUi!GoSoP@lAQ~s7 z@CSBF=~3F6aC13{viwSyqktJZQLgmWS%w!oPt%)rYm&&FS#+qnoPHZAuh@9sE1Lat z`4Lu1kdLRKF+2pmv2n8!DF%Tk_F%-<4kk|l>^$TY_D2h?rwb|3 zW)>}sJ;b2XNW`?@bUM_>NJzP76oCSHsRDMkSak{?*uNRq%s$2g$9zrRZvz_TVR+@# z?na8L@5&bxptfuRs(cY5n1~t1h!}H`3>?FQ*C4ZALuhgHJ{R-^3&Z=RtF*cKs&fiD zr5-ALW`MF`)>03R_wkUrUEGk1NVbV~-$>&IhFsR+yFCcit_aHXy;;8BdCyVQ-$>8- z9=d85priDeHdmxW%mRTGU-W#Tfl=kMDG}sbx-`KsD5~G!Ow6n^NoCmHspUNN&{XnJ zl29uyl6QtFX66#l_Hi{?Md5>n*nbUDO+0BYjX zK{}I0{Ym^eUaP!hnGqzMGf5|Z15!05q)~FU(BW}W6|EWPdo~COHf?S3A%8>c@F}Tf z4)Aw6H<>E*2&5ugmP!WsJKx8W7dp|#qt=`4Wn8S!E=-tlpmVz=Gh}lf)J&b1zNdPF zEz;m%CED$vT`VwstMj&@>f>pPG!Q?7*{mHb$T^I;C#&j1-Xn3c6vVn2%ckl>+H*L7 zCEbtqH}W5{#Czr}D3s*f6(c#F%)X6Pyds`|AS! z6zOVYawxhAlEa#^9sD`yi3VhN_u34m@ecOhsZt^JiZZI!^TNV`j2$|&GRxCxg}-LO z?DIa{Vh!hgmKs*`wN8?W`jZqY&gU)7Y*3dRRXbPNO2`E?u0c6}yUBhK2PPLppX>Le zDuQ>(Vmt(+Jy>;-+g22hBO>DS8xGh2ND3L7=xtTbg?q4F&5ebl5Y4nM#8d4#LqsV# z!F}J6IXEJK#iPAaEaTWLNeVkd>;Y$13-Nu0ZUf1h;;LBY`y$DG$UylX{a1@a!yrZK&M$oe^NHL>%*&V>SG2Z-6Fsts zl-;qjYa;!l%tlVxKaL_y8rbmk@g$r^knC;D??$?BF%cjQ>^A(CS1H>ps^dg`}C!junFf7K)9cyh!veR)@t%*YH| z{NC^o3v;f?@?sZp)Q-7pi#> za@-nS0r3`tN{|d4P?fPc@Dac`(Gx@S?nW@;-^xwP&cu?hI_k3g8fZqO5?JP74H}D( z8n-ay@rCV_jaW*)snY7y@N3E2sWeUE^+#&rL8)s{waVi;b#zFTyGCwNcuy(?QqnK?#qmUS!zNTs%=9XPy!P{n zrPE)%&LFX@1+KT$6Kf;*sY$Zi?^|VD!k`a*csWroV6~E;9CCtFncZ|@bqNMxEAX(fKPD*s#Ay*AM&nf4+ebN@O+{z@! zP16+FBj%Y`5BEiS!KF0jp5ghPH1I|~q$~_Q)mg&$CB+-Yh{j9%+t6Z|H(DjH^PnF# zA}{^RN6iEZQcO*Dgv?Twi*CfV5oC-iIeHn^3ugM%S)~kv+@h`opi_2sd^>x=^JkTF z7_VjM^`bDE! z8Y0gNx_36qNaSl!37}T1-cO>h@<@?~jO@}qzJ5$deF?;NUxYl?unF^S^%sMjfDk?B z->G;zw;Go+ps?=P!Uk(;@or2D3-$L92LgDgX)*CbPl4$|K9JM$)tztc)sz%kg95-)0sLeD+aEYQSe&EMwsYvA4uz~|-!7IY3Qlz> z+Ij{1ZKsuwSwd)mgOL;q{*9aywfUA?08vnlBY zu08dbf|1XC#7~8%BB)01`^NdX(+9Mk(O)GmKP5P#c<=q*4-HP``=E8}uBW{XsUvj1 z`j+EwS*&1KE!nhaBDXeiAtC%D02BrkCyQOs|Iw7(A{*;lECtL;#+B(DTn82loA^uZ zI##+=sCYW7Fgd8TZ=D_&z0}Jc!=3G*fZ6vNT+2U!A@S|=Ck4kGZw+0;boiBR^5^^I zP;`9xshu*OpRnt1sS;7@s5CAU@AFKSbsXN7FQTkp%qZ1qvat&gD>f}$WMSn! zY_y>r3`>K`#t|aJHa2S6aSckOY7c{aO#rch0|ki&mlm4Z)%ZTH_&!eP*ubB!C%g3p@b z4oyIN%_^TJ1px;)fPB~{WD|V^O78w4P0IQOiLKk#@=kQOd5a~$@J#M&U;ImqqJOn7 zXVj+v_)kMT7cjua02dq^`nxCWt!H=r8_31;M^+#oP60L)Xs}v2n7_K@`Uc;<+-1sg zc@_4T;`OH`(o^)|Edx;0)_NYiz6tDoFpl5Hotj|6=cbot6Ab2(ZkJK4Dc z57nacVM0*ae2bS>U^_N3IyS$~z6SAGT+S?p&7yA?Q0cuJP;P)h>iPUkWb0wX{Ts8v z!^Z4QVqAZQttQ|#8=>}dt;gtt9P5%jUOz^^T?|Fxt}WJP24i@aAR{eq{*1NBd_`uD zvJ1E4QoHAZ^)h(I*9`^42p?T31kee% zoLfb(cTRrGn~w|xl9;D`HDp7Hu&9Gvy;^}hfNReN%n{XMM|%xtkZ0(JZ{4Lo3W=IN zL1873bph`Sf*6x5_B|H41EtlsXPtN)>`ep2GW!~2*M6hd&5tuKj7@O#W+V%{+J&ok z5fr`hN}{CpO!JO>yKimk*3dfHO^V%fDNXutt{#$xaw^)I`{DD3t?Bs7!t}w~FPWH( zh9W`Y#+0Qm^Uw6vgQv#y3F$p;&9{yd?Z2ag#X(uTDRD(>5N)b4A@k8?fq-6izfAdrfhvMQ4A`AH!OX5lNGYXjiM~P8+Ez^ zo6bzO5tS(}&AMDgK<{7yh(jU`x78Fd%bkgw`uAyI+Lmsx= zuFqqu#&?Zu^0B$TTNw%u(~@9p&J&>g&#{D)f_FUG0u0ZI2X7=G$)q{SdLL=5B^1&d ze)SNWCHlIOnhm~1{d5o515V$LsBO2E$Y>crFR@d(j6V1PnEWURVc)e z%_-FL0&Ik)J)|(jLhA8tvjJnLfboF^mwRNynA1Z-ZsglTI*>xkw87e4%c~-a0doZn zDLO9PJ~$XMHrcV0y;2km24z)?>hz|qf=|*%Z%OzJu>7>=A}L-GkCso_SXxoHAr+ph zul+rTMNQTy_<}c<#P%>t@0LE1>mul&0IF5Jc|o71IT0u5Eo5m~@=pR@+tz^8Tp-H} zkL2mrs8Ce{t+5`rRs^;H0*T^I&oUyNvY230)Y^}YMkSPHgyFNg{P1^XCjjnti>^W& zA@@2wa$ZtX0pgA)C{(*2JE0v_t|(;fOzv6)Cl=2pVA*@Ngk?Y%b$(9THSpAV%`kP| zbfheqjCH+j)sz26$>95=qEi&>CWKW|web#0pyusArIcAY-XRkLzi@UjI!fyhWeIuX z((L^D%YNPAiJC>{hvA^F5Wq0RT%T#-mT4>u>U*(Scji)WVGtn+e0PVjZL4@aPmstCK%o{uib8r9NzRpy)llkZ=_T|Qz8^&S$vbF z4Z3s&{iH)E(cb%uQDDlkasVws*Ep~8rj*vJj^8L%9rtXa}RQ5 zKeU?QP*@9wT&>qA=m;NJZR3!5g^+H@xt0*8Lv+~RQH(|boN{2qxXN@Zi9_P=X+-wn zjX4DEIRg$rU_%^L{k74!_L)XBa?b1i2hpZHleM@s0Qo(Yp)roJ8}-aXcf4Zm%^vLN zM~9+4+)UxHKAmE?BTlDlce9^5$LC=b9Iv~?NBTPFfDq|S)Eki_6`m0f9;i%;ZKWeg zjT3moBqzu!=fzznGl&%KGsbw->(lZm{<0>upgnL_O=f9_dJ-_IVCidgj5sgF&?RUJm?f7U~;IBQiZB5r+;3 zYFmcM{-{9{_euv^J-hR6+(_{ou|3H45qDfWwvXUuP$p<4DNtL7uYHph%+3jY_o6D_ zfyRt+ml#W$>dc)7<&Og2KM1rPMUF%*lN#LxE>Zww%4M`QjZ0064kDUpR3k8eO_T04u$6RS=MsKEnBy0T7%|By39oJE-uh z$QvTMzqWx`_#t=P7SPCZtHUL|hh6uEJ$kN)N1HBN35D29IXxM!{FilYdA3oPtqFcq z3}}tdL{33TSww>Q#^HbBlqit9QQT)eP(5^mJGx%c2{*i1mYhn4xh=wVVG-?S4Zt6I zWs7zmI%7ken}L^}4`+!h<+MkMYRw8H`Xxy+v4UEp?ekRXit|9|PG<(e!+|{}X;>J| zMCL`2%qg|td;FB?3XLyrg>ZgQGDsD)lp7N=SNp9vK$`#_vs`Ab@TdnWGlaQ&dTs=T zk)ZX1?+wFj7l`u?p|)pz>U8$1Uzna^Dw4}SNu=fc4q2z_|S^`v9LkwX({j`e=yUKjC=IVv0UTMKO$YlWJ7F>@;- z7lb5Fcq)jtOML3&IA~=uLoKhDxNXSjU@+53D)<@(vx=!~T2-?v2~*(@u~-OGG4OW< z6fT)}lPAnsS=vwC=2#PUms$V3BiT(?QxnRMSx~SDpT0HjFdH-B9Dm;k-m~&pS>B@nmS(9<%h^R_PHMW)>eS>ZdRy zAEJ{zed$O-c)ccz9m|B0CERx#-^_hu8ipVm6-9T+JoCDs&tD$lYh)V2G(naN@pV!Q zuye4qieioWB0wybaymvDp?a--V{~=7BHC!f@l`_yGDRblvNXrUjuGFeJ(=VmrjBE) zW?g-o%3V&FZ_zlI3J_9M_rYpPdAo9C1Dor0g~!y_T$F4x9dcwt&6?enW_GgK7cRW; zg}6Iq#MNhdpb{9mO|T=Ye9&V&g2II0bRhtX4}8F_&*bLm*fBbN@5Bt5y{aalkQ`LU z=wW&$smPt!LCv^wwi#DQOX*_vxtUY+t)l_ZjCZVfzWVf!q6IgXQd(Q1Y(Vjs&G)_e z>D*)vN0u+@P@-g5=~(caZ-~gJnc+PvW%tq!rFYo?hTFi3Hza^x`x0v>iacr9qU%S+Y@5(9$%s1ngsG;K&i^O z=%_15i7|XsA9*o>)+HaVmGbpKEan1rmNIWmzq+K6HeYb)S{puL!7)DNRircv(Pzi3 zFcDFr@6s|mB-l9?J3GY4+8t4?5QLKuY&aHLw(!@|FQ7zM)K!kW)0|QV#N}TeUC~-u z2MYOG=1~T!4xHIo-2u{4tsc5xVCl}L)97S0a1pzyn%82#K8>Ly-hMY!NFWuIuD|Eq zq+D>}W#Bw!SX}r?8}WJ-APa}$(q{^C#SwIhPivd}cEgoY^w67%ffV2sFStLD60_f! z=S~l@c)yS8Pl4QP?8V^aj4sba1IO)!!4~?{7bNHr2uBY4Qzg_nm7YnTQ*_Wu6eE+Q zZUpI|yAX}c4sFAr{22TZhF)~~kK*0B^huxbYNM~Qxo@8$Yz5f_2f93o4n?m(+ZEq& zS$k7~d1v<}$@A)~nS!-1S0jUgIndl}8VEONS87tpliS9AfcF-Jz{&L z)V25n+gE&clzjQi(y0laSg35^j`HQT?S|JpjW4_eu(yFt&KDE}o;49Y&t3lEe`4-P^D_9&{!b6|HPv{j z2Q|9QD(^Bwc)`samlR>JWPJaS^$CtmZf7o>n z;WGFxbFd{M$n4>gOs!(d98D*=vEsThJJ&k(4en@U_B_)%g!95wBP*d39K(b8p;v!0 zbg3m4Wl6Wjr-u$?t~K0)1=j2y=;hgj%urAss5yd3WL+z2^vKK zyC7stuO%OAEYsl0k_I(j;p9dn!E@AS%m*Y0n`V;8LH2I_ATv2kO>O9zGe9(=%%guj{V zjxa4fj_pX*{pEWDXpWEf#?!Bgp7R8CB7DxMqTA@-M`|zy2m~jU;RC!CzYQ2Nz$1(I zP%@$ohUFW?Q<4SpN5X$lY5)g&Bn?@B=tlZ!Sa7U{-$VvPG)TyM(hQk&2uF7e86m6C z(e2w(Vco(ku4_^ey$0aj#l}O9`T_2W(Dxdsf!(WDu~GebN2 zC6~p4eJHgke?kv9RD$eNHT?5MXrnO=GJ(V#sv2)HA~{ z!=0|BZn)dEG9%6gJlwN52<$L0ci9|}H_7M*$sMRJG1PZScB#Gf-td2+ENxpEcG!oX ziPnNdBhl2um>e=viES~N`_&xMLDJTTpJuJg1AaQR_!vLKr}~YU#CgbmVor^E-(&(Y zrpAbNAwBZkMYCx(W7Q+5ZnbWz)`>S$)&{l?5O4gw1-vQUB4Q3mc9owCp0qkCJLBvk z><)(RUJI8gx={It-*?3;tDhukFg_Ql`~4bF2YhX)2C!L`EXxauvZ%9G$xE#sC6v8n zJeO+?c>#!j6p;ojO8;I@T5?Xq=>_8utmHy>~5#)03o2fCFDi zB*~2KBAFN&wDkT#PpFgKM5qCUDH!hvOQG>W8Q@})C48)3>uBSm>tUZg>&;QZUUPHLS$bLA+j z{-28+zSa1XgX+4#$E zX>~Mrv7FP&uFzfQ;HW0@9C9!EW#^D{CUoOT9R{`M-zi2rYm^=WQ_p{v@9)&8;e!{C zd4n_%gQQJ}tW!j<%{1Y2#z+iu009<`ln+UT)M$F*6lPpmfol#l7E2-=D9t;rs}B5; zHBwTs;wM;V&EG%ZHx@LR^(hK1%2XiMsEWG+<(g2d{P!+A3ZQQUu>al2xkLd`eL#Ve zA;0mLzh5amd&-aZw4L-f?VnllZ6>7p*%^NLuvnU$$QF1^>=#v*4Tgyzn4BnRD=e3D z{9)TZ5t>ZJ8+dabM14%Y$Vyz^^z}8uwhSG@qw*^IUvkx+@kNzHh6GOrSjQ ziB{e`QXMV#(w(^eR@nZyAqs&Q42z=Uy%xfN?8t&AJ^Hw>F&ww2Uy=!yV2rIYOx2*9 zpkI^;|Ie5cm^dUDm6?w8Ny-$5-4Mw%Pa}j&Pmtn-mr@D-LsEH`^|ff zN)M-WK=(;p_3O0(A-WOR3tcK-wkQm=9R5uPf_|>jMYLJ&n|CDN$Th->S!Td5NR~m9 z%x)g|qzwCHWSUaLJ14{0TVUS*@p>$ulw*B!u!Nnl0JI}VPUvI z9{ZLLyDE8OK}LzWW|wCDcAv5P_#~W!BOv%3eWy&2Rleu325iP0Mc7>NwZ&_?Vs~eI z;@Us0Rr<;95MBdz0Ph~Q(wJ_UEf7N!G95lZnT}+uHWSTAboRYyy?^jvmsx}yrPb_4 zzDeJYxX!~%KAFU8;Hp|6R=cC(jAm|GAEUlwo@3TKmcXwmQZu3jcYbl*=FEe{c~ACa z!SU!jV6|!ReX0_*0Y;rd_mrA2>^D5>3G($Q*D<|(h&MJFDq7%59e$bd_aJi~VtGm9 zg&{%uH9&0B>R~q=^EKza$zOiS)QJc&EX#$>8bUd_AG^lVNR^gVQ+sfWip4ckJyqzu zr_rbfV~J!_S1tbRxhbot+TV-Gt|k~@d%ZUD^g<81o>&jeSdudW%boYloV!rn>@(UW zmv$HF3f#r>Kts&mu9yCV?=C>#w5;QJOyEl<8mDh7HQvHny}4MmE0m5m7-}1oMcDYX1j;v+ zMb{i!{|*4|%_^zNMx|(ODY%iFhr4VM4&`n;OsQGTG{z>F=$O)u`p)6ot^3Ko6_jXu z!~ApXYQ-;JS=s}f7H~WjP|gWb6t?}EVyXhNqJf$n;o-98!uxQDMkoGU#D;FBj4Ewh zYHgM^`B>*Q>7Kfgp~&2}uK_0>+u)u@d)ERETN{)*;{IOv0mZk#u=n@0*P*tE0O}PG ze!LY_Ow6-prEahk(4XS?Q3uRA2+vFVWzwJ8S$$G9Ux5IuFdzpQ34Sfv0+YJtF zrd898l&YbGyQw0%%jItL1gt|G1V6}%Z+1MXjurf*dPgNtdU?M8HkyMMR%sc~#d_&y zlO>~-76k#|%qxMWN zw?l5?4>%4kPl9yZwblr#R+Px?W52Gs*93e@O7+k}YvzbPaXp4HnAEhZVGb!Lmv=gZ zouG*`t0FR@6fNh>k01_I=7Qk26lR<{HEfTfe<|TVkZTK@Ts=)804FzzD=;qX{MSNW zF@^}?18-SzLq_k>;9;GOx!ayx4>e|+-OjO$3-E41u#`o%mvMTUxqHY}dRMb$cbgmSn%%;3~QZJ{*3!V^fn0;t2+|R}VjPBSLqo(^4_vhxzs* z_xf;boa8m^fw!aRd_w>Cy8`ny3NrUFU|?2&|FJzQZenZVXyE+6nXL_tS8cQCKvZ=GvqeJxb4Ndv?7|^XFc8nVSBAgzB3u@X5aA$>epKe7?MT%(6R7^3(H$ z=~jB%h)R1h2FKV1XXK#_`J>~$9Hj%m>?5&N*-@tBITZtsKH6ivd9`pjXNKhPPk46M zdGl2R`pLn|hjvUw?7-Q&-wX(4_9g^Bp>&@2o_OE2TqoilxUU9RgBWqAK*)XAnAt8y zh}o#dQ*cR>tvh|3j9xv#|AdgUh9EItf`z`*!MEDaApE2Y+o}88x!W6I%k)%+OTSa5 z?;IDHa)u+Q0^L*WvE6e%t#nsZ2Nz7!L>I+=9?cl~PhK%0-049vNzPpv{T zJz7|>k3l6jnnJH{p?cKo+hAbq{Io>&b6j#wyiA!Q48B+L3vrCwYdQ;UMY{7q+y#qM zU@HZhYG76(rXs0zdVvy5)}~p_%nCP%K9?zh<*z95-QU0Gh>Eycy@4L(!ZQkWioF}E zE7*2kT!VPO#AP6aBkUBtaIHxU+B7!EdR8hRu`^12xe9sFS1JeXlwJR{D4j1+HcJ!H zJz3LnC3p!(x?VusT}qGyVQ>e>qa+)f(doASx=5-qH*`g<+8_};Sxp#}T#+#HG_nW` z`LH4y#yLj)(-gXA^l1c)f~z?xospzyBDEO(3_RK@+9i5(-p{%bSL1Pm72O{P?()_o zED1NK15~dg%FCD|*~@+AQU@dx-PnUQA$@BR#9vSKq?j^ZIt`epqi8QDOHd42C}o}I z2HeQ^Do)pFr3(xhX|rXlWrGuPH8gCZ29+c|M-)|MAJG*W8~DbCiKL%_#+E#`^A@E^ zG9+QPia9oDBQu6*j0QQyal}$K5+twacCxD|OL_3J_D1poQDiH$u^LjLDBS0VrnCUH z?}tT3@&Ri$O>3xs{uEjo1;WuwQ(D868qkb7S|uJ^BhrKgNf~6>iR;k~$LfQ9X}83V zKZ8(m3m~>cW}EV*0Y$gOT?OA!EbE9IcdQaIe^|{h{y1Omh>1q?$81eBP2EUu5aK2S zSYh;MEHGqI8kneYrU^>vxvT^(wI*dCz?IoF0O(8Em$N5Y3mRmiCytr6dzVT(eY)>Q zX&_8PxQdG}TQ8&!Jsuj%zZ=vu8*dP{v?)sue7BUQtZ*||6L0*QJm2@2_15l3LWhNhWPzO}rZ(}0t{#4(d!uWo-t*EQiL*3m z4?#CPdu8(d&293Aq|4+R#D>j3D6jKW8kw_mME52%c*Y?CGS1jUIb^XawXb0+BcstA z4opgKQN|8e!?D?~JGV--@4-b$?jk!F%+}D64NdvmAjaey_G{#c)wd6djeGb zIpWB2)bB_%Uy429UdBxzjyZN0wAFe8BNg%9Ps6=M$s*=41-8WAbpFvN)DNKbkLv@g zT~?V?-|pb9;U}68awEJ&X${tA$8jKly;vqp?U)RR<(g^h)0TYBa%shy>Tv$xRsfFb zajp;`$kwZB1lN3wr~}WVJy?ttHU+41o_{$?6JZcG!~MS}zLcaK1G zS!RRfR2O#kymEb9;h4v&7^vr$j$gfLFaq}8wkwZHpSibWs`-7Snfu`d9m?#vo5L@1 ze^IYgQ7%07_cx&4?&alCl1DhAj@$Bf(XFvKEWNOvlJ%@r2(- zT=9=F-xRPHkaSo-O8#)FMPdfu&jXCrej7)*$wFwp&T-?;Q9qeNc>9%g(;o5}g*|1p zgVC7uk4nc%i23W6b)v{Y*tt1z?QR7{%_^nXcvH8$qG40gDaV~Ax0#&tvCf8VHd@Sa_8M_IOJiw>3=NQb|DMFo@u+pM^bF;A`Lk8ahC;&A)@TA-)W8W<9&Y`3EMzn+*29jD`ZSLc~;dY059r5zzF-%Cn5LNjDkCRPO3RUeFO z31E4HR(a%j$8+QP&>G=5Chs|(RJ@XIkFpCxm~RBxTn|i58}ozq#7oX}$IFD*JD?*R zQeC$q3PCaGr#~uNa*kg(_S_O@ctbew31h^E4+H`I!UdkG6ft{76#}%L*{u@V0fZ`f zSEe##HLci#1Hf-y$a(vLcWZp-M-`!&y`jgR7Vxi9czr`Menn&94pQ}UWT52z>~2@i z&lT7763hMS?bxHj>Z6}#DcZkyC28@OTv)-D-T~?1QrGEb zNN`mKxxc{te$aG~)O&(OG%f1%8p2R0>>LP1VQ-Xyt7O1zJyx@eo+21jx*4YplVyg+I#a5uK`Kus6xW)es_8H<+v7T^z zjtZs(1=t>1cvO3ynf=tKb2fa1+DaNn{q3up9PofZpr`BqMRhq4^mL_vNrYuz5+Td~ zOd=F_H2F%9jw7Dh&qc0jD4jn- z8mw{o0tu14`6q9LpQ!dF0*A8qZtVCIhqyDybn%5sMpjnRxWBMgqsxwZzt4X|IbfT; z@>I@7)M53h$$gOlD`;OGK_a}UaBivE1D{D}F!l%Y3OX``C@yI#Yn@5~0IDu#LhWy_k+}+WwL9+TB=Uz^!MjU{*FB0Zim zfxg4HO#`L4ddrrJ&fy_;7K`<7$vMWL1knn^RHw%$K%2x%Ldnt$izkf$ZFmZcZs7<~ zxQ_IV%#ew*@4)%cU01KWLe=EQHX69hS+B;1sKsWzk{PWuh%?YD`!7Rej)tfmMD%=N z5nFig4V48CB-&g4rYEK76 zQ5Y*~XC6fXO^3Y!Vfr0-baEg%8vrt=d|rH0ZKqdxC%?iFFFG9AyJxzEp9R%qLj%1E zBJFQx=<;g5;r9&c&i=mv)hgZyqbO{DcX2aQw71Hh5ZSTo?gJ zUxs_?i3)Zwkx1S}ZC8@_(|(?IUDTY)d-nd?uc|2_m;j)pwBmY9=wM5nyu62=^|`pT z9;*^7?yfrop%HNO*93L|ff^ogY0ioacZ*Ir3fvFjO~N-8(~_!go|lySL3Zkr@R%{3 zOT`{>U;e62k^dVCRo01{l~q3uLq4G9owpS}8$6*E zM!h28J^lE-LA+)Mt%0XnrPaL)lGD;aa7oN&T!;VOB(js#$~-k)*B+>BQ%qw{w)bmQ z40mF6az6P2ysc92ty*%BAnT0#;;uS6ny}hpS=eAD&sqw^rSVFIu zFP5-J#QZ*3;EN>;tp6{TuoXS?!0A6MA!Pa&ODOVREFt#)hb5E+4Rrnes&#!|wT|=u zcdh$x0PCyrni$dtVx$RN9FkCOE@>f^q|HAh1hQBxi2x6X-ERnC%CkoXD)}na5=_CJ zUji%1i+UmzW0q;rT+XYwODhH3&~G^MM)Bgdjt;x7jkUG8tvQ36Zwr{SM&u zd1(m@^dte0j35f-grplfj6G6q6x1=HhwMnX^^_Xoe8``M?v$IHwfl;IDCkqV+?A_ zJ`zi|v#Qb{h8lxvo@H&sOIJ(*OdNDxY|b^!jR1s6jjFBS&FWhVX)z{PCw#h>;Nu%i zY=p)fZX*YC8FuZC@%tq3tdkFQqFy1Idn(h4MQvmGkp=w?uAL_pSoqCAS~>FYHvxy$ zvGF6=Rb=rLYq#IN_IpFXCXu=;s<6*qDJP)CM%zs+YzMcqixq+7@-}(Ru+ifq%-S9k zH&jkMFlc*4vj3@jjoG}vMqy0W5FX-fnC@~+j~@WASwus z3+v1iCs69}?_iI98y=Dz2+0P3kw-71yrQ(g*(x`?JTaIv?dSfljCl ze}ss7saO+i1NqBPp;sNY6Y2mCSK%6gF09lJ-TMzrh|0S*d1@phd{_T-!gWUg!F7JX zn0vy`U>K5e>%hiE8tu>mkK*z}3^ral>e6Iu(}C5ZCs{L9!Ur?G&{ml0%yT)}V@@`qwN$6ab?qPM2C3arksh5UMslCOk zFu~CezRmzg`s18OFE(Glg=dTuj0JvqtlS||4Jplduu6e)MXjiCLqtQ~Z9qei<6AKQ z*gxan+xf(p{S>|SjhH=A33AzY3%}Wq+X}S3&Hut-I>5kJL+9fU@9r!ABF0u#w5*;r z$B(+Buqdnc`)N&~;2WWpwkZIsct<4IkzI!qPbxfpCd}_2x(Nm)nL$SjA6!G-F`%Pt ziHXpqCXny1fB=?MC-h@jWQUBl_Q@phmm@-Szi^;$j&DFdWx7SXE)}F4Bk?ei=4ceOZoAeC!l|vLTB$H)%L$ zO%(Tjby7X#f~;LhXlUoJoH}PL5Z9&y6{KFoi#!`${PH1W22*_r_y*D#;L3HPq| zSJ~70DtrG!i(k@7&c*uw(Wd|JbXTXVRUlD|5s|dW^Lt=ciZ&6JYJU0$C0Wu7pckuY zL*0%2N9XC$wem0(0=`%DP}jQx{*`qy3r~rwZ~N}(04YDXvh`0;1F}s zIWHm)kr^>6u=#|WV5|=Redr9ZSYR@irWza@YZDUinSy55<*?#Skbx)(1K3qXvqNXM z!loVM!eGTP`h!S^r*xyGyv($AYrSJ>>yMBg`n&zqFSy$hSzH~8D7FGe1L_Kf=GyGu zB>K}CVRzn&Ox$`{f=!bQ_9#S<^7Aw8)HV_U((hh$V= zy1+b2^ckh%0GD9_2z^*H+q+@^;iex2i8z|Kh?LR$bkMUCbBq-%F&->R^mY5(uvBVt z4N&DNF#>Tq#jAB*zI=Twvtbqj>{7qP%q?d}sxNJS%DrsIM#m83P_Z39(J^s?u!PzP zRBfXYp>nD?jyziSuTeicdft2z#z^8ASFOe)-b0d83WUD7NqDY2$GJ zmE&xt{*&XBgOxZOGtJtEhIOakhi`Yo#PaKsPk+SL;Op;NT zml9hsF&PDVUZkn`Xyw;;Cc)!o;fhNf^$mK5bwb*TXwQUB#z=8rO;UQPjT6l2WVh)} zkJfNzWA`C8y?jM8Gat{U@G5G(t*bVjKB!sNLc9dyhsxn|?YNZ@nFo4jbq8B30s*_g zg^|_;$<>()UNwqqB65;)`r!D6$NJ-EJUgDg=Sl`+dd1ftcGUBUeGAjliK-y!tCRh% z|6a&Kbv<`OzJY;ZL4tv?{m=57l%0hwsg%5=oQkG`=>M9nX8osyr9PUk-eTp3+9d;k zcvFE*0X8zCRgzG_SOYs3wf?DeoJ*=TIWelgpMZ=6Ie+GQg}{X8s!fs*g~WNpS z``<$Yzn{-LxIXMue(W`6$WUr*Tp}a3FlG@bO>G*F#4JR7obvdHqnxnEL9PdE$OBs* ztDuq0DN0b2exE(OP&P|&!YdmO*tXc?@6*j-^CT!*|4?~bRe8aUK2dtvmVTz5S$BZ5 z2+chm+1q;MPU0tJjBTj>QWW<~kNT7b9{FG4N605k*KD(%0;p2t)Pkrwe%8lv><$8*!osyE?s64->S?_T4cvS8IGfXI_; z!B`2sHlg&zYSzA~H%E2b>xdofB=m+2qiE2T012zvgzV5YZu@d6)d}w^Bj*ti#zg(h zB6PfqamM}$a^*o|%vXpdUQ@Y^Bi~WK!7WgV0dV;{MwSpVO2&c+Ej40Y1-Xa=oJY69 zf^ixe$V&Y|ny~ECw#mc@&gnz?;$~*C0;%Wls}{F{1Ec+)kL>?7pa-;Iyj2bw`M(TZIdXP?i0|-7Sl~p) z;KpHC7%N0r-|BGIvEn7MQpfw#U?N;BXcil7Rkan$cjGNf2ZF;wRkdjwF41kQDs|6= zQkIs~*E(0#S+D#$z08 zb;!Mc5Hb4_-{3kLuZ!QQY`R4j)7Ihct%?6QeAY4BDj&USbN%jk`5lU%(u{}njQjSP z#yhJgW`B<5qfBOvq_P_Otz`VU_3oVYiSZ}@Wnujj3_ax2i1+4@SBEn@#X+MozI+(q zVGNATYjyj(o>uSF@E$JlikZn1=TH8tLdiE-kLTncFOWaGC}-cAQhweN{qz$L={nm< zDfnj+_>1vLl|E~z+QR~`>vD(PxcUnFt*GbE!F&+U<}7eoyz8RSl(RXli{6MIDa4O5 z#%*7LR=egON#NGKQlj5^;g2q}Yx#Dxy&Rl3W_mIXgX^h|22t0Q-YteFpST|S{eu+8 z+3JFOtH2od6UUDzg-7j^P1=}_*+*^<9@4}Qy(TX~mOBEd&mQTeYroMwT>Ovxp3j3q z+$VV`Kf-`zTC+FJt7!67?Yo7PF|{E@iJX9g-o4S!EcN5Tf`ja}3aE9b#L?8{ z&IwClKp&3~+YJ*SttCgY{nX?-Q$MiS8d5<%J zqC@M22ew#Fsy}!sBVS$95cH?OD`^-zSQT(BtmJ>ku}L7Lm#m>Jqts|rzzv|La&)OL zN!nLVAVpTfTK~u|2GNBVT79drYhTNn>>orz zEB5*z15q>5ctR*W> zfJQARo(Ucyj|Pvge~p z!v3r`N#DeP63bS~CModE)0&0Ls1eL>eN#MGEU+Fvuv&I`H$P4-o|O;zBEH{5!T2AZmZCAKrfx&r)raU$}xE@xSWcIG8SRn>EH*{1tA1)qCFG|ORc0S@dNBB_EBP! zE8|8K;@rvX49d>Oi1+DlDsx5l#9r-hu1y%?lGlw4whTx+gd|)=`(z#S-1Fb=Lr7)T zhB+97$N$m+tQ#)e4Ho^^Ypc15Y-0M`93Y0j#j5};geJls!-WDXbET*27OQ_Wi8UYJ zO%-0?e*BC~PY7)a!LfPPvJW#l>`e)%O--`q)Y_ws*Ea$0;h{ye!k$8(j}Tp3t9V?Y zz>}Lh#|-RlxG5||LKBtYy|IOB0!Q3cIxz+d&ZW|!3{z*uZ-*HnedDj=!PtOct(1Wv z)OB;xp%hEUw~-f6S)8FElKD~12#5lpV+sM%j=i{lLUY# zTCk}OjzBDo#SQHO8C1l61Q$BasXPl`3;}+aTKbRTrV)t{_yd3VToLV71<38V(l;z{ z#o}V#L(5pifJSed`kI*swZG1MENf`iL9U_$d~s%vWuQb?f3~Ph z^CRf=^Z?n?<^We3KmkTF1o7I%55)`*6@Dwf2x^}^Zczhix|Zpg>x_I2G3_C9n=IGF z924<-;$s8R>)i*1iVVd@$h=C9hg>!!v%%v4wuso^P>|vFoSZdQd7h0+4{gu&?Re6h z3uqCKQw-9prGRmQ?fsp}{TFs3QFRB!CMg1%Sx(!=?P;$D{pHj}3s+m%+022z0}rYEN&CmYq>67PP#)WtKpalCdh-*OcvJ|eBY9(onuWdCjzaCq!FA(Ju5 zR+|Zk??1qXQ)|{NIkv3NtM;F62RIX)^_EhAe}h_l)6&>t1B#0@Cwr~!^_Y})@Wcy+ zfJq~<-u zp3sT+$@yp|B+*l0RX$4p7iIszq*>H$i@Isswr$(CZQIUD+p4r}+vc0LQK?Fsb@E$x z#g14j;-1*&PmDQ7AMNRlm+l?Yd3HGwwV#p_@ZkIKR==eO#&ON{&3GQf{lGq&2(8yHLcRQjHcYPa3%{UB|unG zPJM8+C_=_th~UV43EHNpQb98S?b5&gL7qJ-o+<7I*Vg6JOjTZ46<*!Gg4M^0Sl>0< z1}6b20KldmC)FS5c$IC_m1SI&X~VX(e?7>3=fy;^OdmN)R+?jNC;CKU_l}h?53=K3Tx*o3xH|wA0m8IgIR%(`imzrf>(Q{9q@XC{}k8z>> zNd8T_z_SspD3ytbvd?4}x%L^dD_Waa`}z=Z7$7!G5i?qg8-g6D$^;~7e^-pKcZPTd z9+0c?2AtOP1d_5_GQ6l0Ib#ZZ_5jdrlL9n|5xYo>-C4@c=dWWuj)T?;2Z|Z)eiY9J z=!?9&D{-!nFfc8fsIF9zQ=$F=CrPpz+QW%$1^Yq?`N|I2I!e=8{P}Lfl8HF+1%kMZ zpS(5~wiUyUp*E%_p0EBHN~}_q&7{Z|bB6YDRlJu|XM}2$qqo#43?l~vM`>7qzlt)_ z+tNmQ6lTLEVj*kzZn}5=wwfWljR5{d(rJ_hF#$IHNm2v9Ffw*WfrxoYn0-`Bxl~D6 zqao)WpEcz+CY|0~rcJ&agkDOi^S&2P%hVs@Le&dsw(f^*EOJq86DYdLodri)?#Q@_4F9t0z#2L>t42* zh)YM9raes%CpEMUk%1KMyR>%<;EH-nrMdk zJfkg#t~N`-FJfq(W&F{jaQf-N@n>zWQn?&O;yPjL=Uq8|Ob4{7v7dXBz?uc^*1uRp z!MUnep4IITK=;AST-LjzM%)UG?dH!G_G1pSjbVb-SI@atvar)Lhv`kaBC?BaRk_JJ z_WOQYsNc|H;IXON#o{nJ(mBzBgO~vZl=x3a zY_i{SOHIi$uJiKk9_ zvkO0esqfo;{b(0c%-qMjyloxU#Q+`dc@PNLg_YRq>ScYHx3iYZKo^IWn@YLm2iyS^ z+vI8O>hh>*`aMQ7pYUXm`z_6dG1KabmT*Em!Ogpd#)u_teMx!zB8rD;bQ>ljRFk;> zoFL;70b#=SoFO zj`LuEXo8j{!7nX1LLaNJx`k$jppa0}GM$^up?&X;wJbU=Ydk(jDZk1es8TWe{9K}N>sSTvZurQd~lBOHVUliZlgk3Gg1IC7gNRK;r zqGhvcCo{<|_j(m>4(2uj`ee!%eYz4N;c!F{7ez;wiTeXf>k~DjQFeAk4MqPIcYV2k zZpkK`KmUAGzCXyWWD_VSDWmu{Ll)Ze^Cb3wd-&xLT#jzm@1v&_4(~xraCjyg6LL<} zKPJ?#U9U+!y*mLb(qjVm6Tk>Q5CCGI4R9}p0+Oyoh(&QWII)C6Ab(KO<{JBRZ6Aey zZk}}fihCmqi|-h{zm7_uNVJ^JR@;SJRNE1P;W6<=wpvd6p`P(LNcRT&Zx&^-knfMQ zNUOs&_&#D?D@`@5N1MAV4b82^&yUr4@wOM9%BOmLG86d{k3C1^y&x)oWmyA#zgrbk zwBv1IsmU=e(PkACS`S4mRue&4L_O6Pmd*#{?lzooYWJg_qzOg^O5b}Bp00u#_x6rT z=k->OPQJ{)P~f+5X8q#KJQ|Xy+s$q!LOoYXz@&F*t_Y{Cl^Z@9jPp{m!_HrUA? z*!1XK60+6El5E!ivxM5I4a;lFBP@6l^%=+Tp{TsxEeW83t&|fi^1K?o6iSYj=M#Es zU_)i0l~OTw7UXL%{6|?SWe~Ge51Q;IlPu$t|8O4bN>K)QWA3%gSG7j1%<=W9?V&L( zm8t)P{#r&UI+B^UFS^W4mARbyaNLpvMVu!>;`5^J^6&QQzoLG^W+<-8jknPp4Zs;v zJ9_SXSyC#?<&)AG#zIiiF@23CgRj4P<%2wr1f-N8um0sT%wx|sNJ-DTs=vE-Vp%)> z=uup(ggj?W>q;s~;yr&<^Id*m*ZD;y0I$ssdCLX4SS1Dvg$4`-|-;}emm`lkn zvgA;(Y%5yI=yKIRr+O{-9$%mNbXScu=i8)WOGWpe3L%-x=lTU$u39W8caUKq^A zyxC=$eT)~Y1)aJ(&0kg$?g)jPJ+S3Cfc@SPD_@#r3v(!BJ=4&UZ(n$G<0ZO|aU=JS z5`dhlE7iy9O}Q@X{x#qUs@)HV)`>_FDELwg>CNTt#O)tOxisvD!9N6Rg~(Vo?)+Zd zliD{&YsFY!I{pgclgv3U;7~{waZbG}@!?I$KWkVB`iaFq3|MS`^?d~C2mgfzx(mAA zi_!7T>XUyjJj?nB+tRZP!}m)51=EwG_)oaMjP!Vr8~hRXo!)z(Jtjc1A96Jpe99~j zmoj#EMw97A+vt$6L6$%vPvW2SKXC{{=~Jx-D>y-!KoL3Td9*%F(o$5Nu55t~C|^K=xpiIQ})VU1;rpV2C|F zGY`C=UJNNvr?;ave;eLxBItxo0cVwx(N|7yi>8%rJ&cZ$*9GLEA7%q^&fI4ywEF zf=Iq{Q!(bCUm<@fk4r_z$HG4D5{TY9G4{~T`IcgkZvT*!t#)bJLNJXCj#@S%*0Gl4Jh!sxQ$DJ}!nJePf+!e5p& zgXO>JLk9P)P(3X&3DH_O_R6K9#uCCG-mDN*RxWE<#4(=m4owD(~%fb<@XO`Jcyx%avn9tkmBC(nw@TnU=SSF^!$Z=8GHqVv_>iBuT9`mt zwGMZ0hPaaTFe0~Jp|pN^OWbwxuUJGp@+ikq)leqSh~;tHal%*5BRBpd3l*h3Jdb^V zl3+#q&TElXg{gNgW4m_Ur19jg+KIO*9=YxOORo8$M=nto&Ckz^lUMGZ%;c9cwN_LzdgGYEs)x!jHTW0;nE#kAi$|OrRl$Fi=VX^nF5-_g!j6?Z5Iih9J#pb9!R! zlNcQCQd7LH*%ZGH;UsWSDn=PkOS1CFiFkmEJny#X`(yBs5JXilN8fw1{k4JnTa_c> zOulj}mNi_y1oO$CQC%4ZD4^&2>5V_1KY-gDWj(Lvi^)6MaEm;h&*sb6{S)>mz*r~3 zf-`j{T@XmGp!STktts=RN1Pj}fGWtevJ*Ci1n+{TUp86#|3*fmAof5wMdo-^+l=gi z|Lwi-(9(xchV^UIVcl{CQQBr(k;*4SWAnDVhb3%@#}8^-OA8f@n4yu6-+l(z#vzdt zwvxB9bWET40lbWpf_i!3=_|gjUfs>skIzhOJ9N7c{C-2~h)XQR8+G=0wgt~yjiVoBWHZ+t+VR{yPH}XwBw0tat8n3aU?JHYsG{CXh{A@7Z zbiT=NH{s0(ksyY@ChdAz#)>{y_Ll3D_7sM;p$fAhVShjk%4|vDejo6sH#yBA*f;!= zr^2bSq&(X_aFm$lEI`(r5Rjsyvtc@-A=(R1#6Bl;wFKZ&b+}G+YhCns1Zb9_t}bTr z)h#e^_l#``f7#r3b3S<^E^$>JyrSYmL4qhIu>tD?--_bq|KKH%?%*9hBl4NbYZmZj#gPT=!Xw zALY)dC-LXDC?YZb&hQIy=5CzWTL9S<;-cp(@qi@c7k*&(D~E@3a_=_uxPP?b!B2gy z<*QmiJmvkvCY2+Y9GvjGY80C`ac{fMo6|nE9ysT0gs;QN9g`w83JM?a+?BX-0Pp@^ z^hr>!D*HT8Z!n{4s2MwquQTQ?s(X<2tIMhHVhwl+v7S@)Lu9tvXdhEp_N+SM{)8 zWz{ly4vi+f*Y1m_bPve9!K3RMF6}LUd9l{x5moDVrFqSgw94$SCNloM#gv%q`hEj< zl?`UxxWbEKT#HTgwv9bzN_W+PW@n1$J*|L`uC1XF6yW0cg^`1Y1OAo(1lJZ?6>s8G z{s<^?u6n6D!meJV>lXMJORkB#bp9^DRoX6-t0XYTy%QwLNidQiRD}){DR;)4l|-xg zxd}~4YtnM=6x^Rvn9tKsiFyi&b*E-LSBUuNJQZWAGxyd(%x71T<)Y}rb7@UILHT;F zPs#zqZOT}{oMYhB#EpZotF}%a7Y|jfmF<_73y-o?EXh6u4Zq<)jxT}ws8tmVpNl*& z&D`VZE93t`_vi}1g5KorlH#AKHsVvDdJs2=)7MiI0o>!XY`fv`PuUuqN$`*_mXna* zZ_ZJbRSc53P$`5e;|rW72GWK zTol3<-Inv*DbCpFohyU1^BQf< zp=1yp&LSAqHS;R}G^2>f2(T<8v8*s%)XP+A#T5iiDTb^_;S@?NRNBDNtC*`*Pg%!R z6}YOSvsXg5D9e|~U%*9IvjUg0yWxND)>zDAF3YX&&}~-E746s{@YnrW5Ma;GQ-M7* zZ~bi``cL=5&PQ=&!EjmSLggu^KD|_Sls>Y9@v`ZbA*)OmMG{sueI(2;Phr2*y|1l= z!bfa426Wz+(y-v@3^S2XZn=9Kn5lM={6xW~y}om|cc5fng+=Ve#vH)Z%E>-k0K0ZosP`8U4GuF^Y2$r6(8@YX(gDoZDwTG*Q3 zG5G53eVO4cvPPWMF-o|vVX$Aj2pPTfNq03DU*_V+*TvUexTvLh*1CJ_Y1V*ZCGrrC z`2xqjh~N}36se&9THsL5v?-HW5kXy%$SNFS+SSmhC23I?Px5M6_@Yrrs01Td=j|jVOY|!fpC)(o6@N$-Y8{T?W>g{N!G1J1kMMEa!?C9!Fkkc$>Kro z)iEN2JzH@rrCbt;u7JG_k^fP#ofa< z=KstBKnS;y|Ne>VmHx!_SpV;`0CMK8uKyEcc>m;+(f-k6O=FM{hl+?w%cv%A{y_?( zAUlu>4*E|%d4&4B6KjQPMP0gVo|fO@o%o%4D)qMl>(cq&mr`zDTG-E?o|0#)){p1j zJ>KWu_VJCDj*3Xyo_Q5ly?H@s9q$ygi!YpCa+RM<07}H(D_sQTF5=%(A99Wd4DdpCv5A= zw^oZYr6D+`=Tlqh0jKHtrxMb(LMNKzN{;K|;dMDz80y!zLs?}Qc0V8^%_F)?UCq_Q zIh0tS>c2n+7y1v#D60Aa868g`oGyNXmp!Tk0xt5XU zSL$&Gb_(q+oyna=ow9zTL^85-6fJy|04sbH)9CUDPA;)U zOaV6NJIGurl#2FfY~$rQcVtR#xU24pFVsD)InRCuOj{04(RR6oFKi+x0>z?@9nv`MMtEH}q zM%RVqbIfq+voUmV@fzy+J2epU(b|lJo?onJzZ=ljoyhzOl`y>}gJ2bd5|V@6xnRQY z$VEQ}C*)GyOx0b%C&0)j@+$RVI2tPVX&+@@bp|;L@QnRo`bNncpK^mOu}mkH1LTIn zapC;St2m(dXkUQ^X*yy#u^sZL_zHEmcXBqJ^o9z|w&`hTicrnV^|$WXZCkS@Q#t=- zf){Ec8%%E3)&?zU+GcYfiN~|sW-CmtRS-5)E?X-=N?KJ8i!%`?t1`c^2QB>5e2`x@ zaj`W(GL5mruPcqh6TlV0@quPuG~7?OhtBIPc-4)k`6?-YgWX=Cvypg-kCdI&@3EoO2Tw*T;M&W4 zv*bm}fRa0!S_>t`z#OW`akQ$lp(e`5osj zLqbs1i@JZ=&DxY-Y|+^WkXC?{Dw#rm$+c)7q6`V5s{h@gJa{*B;(Y*VG*4d_9(Emx z!T_WarzNdRG_j4G1bb z!OyYg)|Ftz4s^h!XS3xt)Nf0_lfkx!4B^3cCel~l^y74@H+j~w@?cqdz}%ena=!!` z7>!(+tXh%2KbsS%M(v1+n!lJ1B6|xUzzmQacl5HS<(lSSW*C}=`0*HM|K&0GBL8=s zFVO<5`akrn@8_W7`u}>+{ZEadi7JWq4I#pzjZrF9sM4}rBtK+YSy92K2BORaPJqTU&ECHMTe>4W6 z3HN_AMiNpWU6vx59}PwT#epj}9_@cLM%w?-7_6fM7T7wA|Nk|H;nM%q7_ls3<8HDq zlvo7hRcZLW1s7$eQ1<+QIJ&)My+$!O47eYER&_NW%X4GJ@M8bc*T$VS!z{M=Ts7vz zQ?0wHx1hJMP2OjcSVC`$suQ1L$%YVy94P;SRku7~AyDm&mQl*?ATLPo60#iy1pycG z^d>U+nS7vP&%itao7FERLt)vGBSqji4V$`hj|0Lb4uq}Gs^O`e^%9$kJ-xWyGC0W# zGb*-2;Zwaok;B)Uw1$_DGM*wt?4Yy8zhkLnyK(=Pqtv!zM7zNw8pt0*<|mwO;+!(m zmO`lb%A-kL1n$1zNevsNfA54LyghoYw&l2v#p9A+^@MF=>PiLvIv*1Q1EZx{a1;eL6`M_o*#CddJU;b`JU!+8A=Ulj3M2gZU}V41ABD!3 zBEF-wq6f?(Ep9}(M*xnyu6D}~LGD+)udCG701Uo&(DVDf#+S zKa>#tGc+GvZ-5t$GhY1nGX1rKf+H6Lu(HE-UZYNprQUy?c**Oj68|#M3T@T9k~@8u z!!HDT@c+bWwhmfvxkdL$uxIg?4XG016_O+5MM)=BM|YM}#D4Xo%!{B?cK?TZ!NfLkU@H33k>Sc5^p`fCU&mutk67*7J^wL{OkNhrWqp? z?J~XqHZnkbjIi2vdmF7gSrYTZFw#H5rkZ~kMzr#OF^p!(|7I9okNWus-Yi15%1JPX zmqs0%G5c-U@&qq~zuM1Jei(*n0qm(00h}ZFe;CI8e;7uZk`3d66~uot3`jd#rGSk8 zH-|-Ln zNzkZ0^5Y7qy0}IiS__SH0;akkevvi&j>#wn&A^!V4TXN1mutnND^!Ez-zLXq#hOPM ziyh;^6AI5Wqr@x#6g{9n{IzO+P$?`K^LFeVTX z!~f-ZXKioo_Ww(Hwra-ss-9r~%O#tY9xiMHN2Y<13%A}(MsRX%QR}oJ?wv6y+;QV2 zi+DClMpaCX?~>>2nK@9)u|p_xaG)_Q!hp1Ig!wzh;n~0E_-`WbtM4YK#2SZbtbPWCf8hlI!hC(ys|eMv>0IaH=#1dbpauvg=orE}?q!w-z=9F_WGUK@4%>2kJk)kPLlXS3ul*v}6BhQYLP$}xu!?p1T0 z-DlJBS=itQFw5sD5^jl-cr`}$?i5JHzGyX-GbrHf(2z|7kx5qyrLK4mQ0$t1W(Haj zq=Tb-xHe_^AelqKjAYE!C%<4%-jcgQ zDU5l-iIl+`>+4E^zF~|L#kFEJBdSv)m8&nZqMQVRGSdk;;c5#xla!h_yTN9@)bmi4 zahvP7&4No}a1E12NKAIYCouhhJ7LMIBYF5c1z}^%0fYWB&9iF>h zdKerY^O_fa6B1Eh-B;1{gWv)5+^oo)c7>maY@bM9I02!ubZUzYX~fK5%)&MXl?RmT zMTg}$yMq8ML3^A42D`7Y{f#(78Yp$0Ubat|ehkR~tzrER-C@Vj-2TxUG&&koU7Uaf z!FkEF6&m7~1Q>@JCIhgye6~;Qz=d1QPx(Q2O*k|3z_nY%Pt}GmS-$UC9-TLOqQoxM z!FSISBc98lf(+he3Qwgc z+_=*}Wtg&ksY$HmAhQrk`b{M#U5mYHi%xIaWshK}173(CRt#Ua4QI*zg~pj3g~o5s z7k3>}8E=y$QLKOb6)>j-ii~&V=b&&=4SoGTHU?*BJ3jRRDEbl}nM)PZMM~BIoQU_h zEm3>TpUKC*_^Tm=I4&Nj|9Zl>>rQbPu?09?o|(Of`vh8@WG~TDZ>ySbbJFqf{Z^eZ zKTV#C@nNajQ<|Clf(f$c$@{h0w}W-5ZY%7JqOaS zJ*P@X2Rm2!R5_;^Xt>ARS)x~Y#YT`0uH@o+((I|n-mRf;e@OmhfjnHDd1Q81q%%zs z8ZeJ97pV29-a|pl9&Z~mEiuZQyrJqtzJ%7Kq>@#%sUkmd@mQ-Hp|oZ+u>D0CwZ$21 z)^Y*~Y8did_r$n>=6IT$nbw{Lzi0eiV3o#IUw$7iV;jXCSu3N4>-C9aGB1L*iA=t7 zHBy-!uq_SlnN-zY@C+dnStKdau)|)E5lE;B#^itIIk?W~FV<|?;A92kuJ%$Tz0ApR zOFWP(W_maLbWMNwh{gr^ z$Ms!?`-WzEvTn#-4RdcXm%hbGyK?De;k=2;a4~fxk#qIJfphU zJR$vr$P;uI{qxE47x9`4`|i**`!GIBfybPIrno-s`weB=m^IG07yRMFTQ~=-dj;bs zKN6cwp{oq!2t<_Ot=DH6*oaOTzwn0-Y3ImhNvS8IBp)c+TPBSol$DwEbvGH9Tuwo zy7L*!CN=5=5M5rOkfGGC*F(AR8LZ{Xer2C6>@~p~mwxi8Sm=&o@4MX_*JmzZEV+fY z%40$IFuU29lnzwi(2-jdUw^0LldF?gM(;S!z=NmSV^fy-$VzAB*tp$0OX8B}>@B?6 zGq}*1SmpdqN83yqk0)6K*Jc{4d_%rR;^Uh7%>qlV@;46iS!Fw$J_$kQWyR3~`YrXc zSl$7;624te%naqKS$#x$UR}Jdg1$mMMv*o0k4EAL`6=kX|AlRa8LB8IP=SC(e=zL- zCAN`sG&cM1-!N_1A8eB*Xi1Pcd5j`OB;+-5i!?+giiE_@hlUzF7z#EtEG9K7_CM4q;1%(+wI}my5;b);?SNF zDylnskMSko#qTud#_#mY|6$57&p$qLZXZ#CP*Us5t7r0xd^cgk@t?)vGE*5p{wMlfVHp?&usLk2o$J!kB%30iw{vy!A#C(#PT&)if;r`zBs7;g zZZgO{J!uJh>YAf>dQ7fhaB*nfvpsmq^Tr2R@3f%p*Ozh$-aa_^Q+qs7t>*{K$?eq| zll~A!-V&M}Rxv8Rzhda9j4e7;}Hun7|2 zMrqsh_XMG@D71@92iL z_SGQ}Z%&VDi^|b{HCV^=prOm%%BMM`;V>IT*+mv(hw_T44q4An;rdUKKCUVu)YRFK zt8W2QuEK-mADvujVVM&fvg2~Fjj!4htGr|U8GmbE1)yKOHqM5>Rfh_vW`(4cI+a{* zwD3xS8~G|S-woXHuJlYpQ{qU_K}LS8yEoy^I)!ox}AKDH7&H>Sjdda~p@c>{fo) zOpl8+8b~P0KubMIl_-k_h1WCAqbZkYL6zna`Ly&L$Lc=IUOar678>ZgLLN>ujdVrY zVY$R95^^RhBq8s^W~#Pj-x9)!GfD)I(lB7I(DM+i!^eKktS%w+e_?}EowHb%f>ZuF zPON)YbUAdPavI|nVv+d1L2a~D_#|X?8rrtQgvTh%mMJHM9@%guO7rm~!YsN?L7D8) z+iYXrj^1&%ChDjxHIeg+bJ1hE=HkSVCDJc#F~-ARI36$^3h4TqycKCB`}xr!$4NYT z)sQ2{+Ab8>cp-1k>#$>l z#hKues1A%8_D5?K>Lf8d8F^@B6S2Te`J#!JD#N%@5+1I2X3-g=Bs!p@<_A)a%$X$d zrz9hiP`?#d8|HM2PzThWcFC0pDq!u8JadL-Gj22M31r9f23B(#P#n9F8SPdCUPi^z zjG5ky3vMmhaPnV`QGdZ>-D9>|^BNfQ<_mkxy)WZpGdd#*y5Qk8XspK6+1b(8#goo~ zuwGLzXct$o=!V@KfTNQpm2t#hn5>wNALz))5~wacpTSjcrBFo^r(qkt;_0Sc@+F7je=qxuH}H-Ac6hM% z87yEVYlG_J_PqMX`==?&xs%wjZQK&=8Z-QFBzcVGQwEa~M|R;Huq$i^3+v56lJ*E4 z?;pgCRrAB)I@zy5GtPJtFc-~k<@VX<%8|b;r%};po3m@wZm9n9g?%P&M`xVkg$E| zbgvwJJAnWw#b`isJUGqZULKROtTQX$04sli+ zDfsc7yV@$=(e>Iv%twcO!`##T547Fcm*Evyw%n#n$C9$95AU#-ND+@ELbTm!M5<4s zK-B|(Hy~)qRcG2ydLOX^$E7%`C4(@+qH6R2ES!^@NQeB=@&mg{l4Fi7!gdVT2-LxS z6iyHxgwif{DFSCwffOtlL6V{LdesBj0D3B}%GF9P@A+C+)11)NPW;b`p@h4DveJLq z0caMRNa-w$_n~GqoghsE!AYJKMH@FmY08uXtj7xO)gd3wkSK>54(B@P-vs!gjDlk0 z7yIMr-vWXM`;_Pc1A;2}CAX@dEh6J@*kOn;@`eHui!{zOA8o@7%Q5GXPtc`H!j0=|ZgGpoIp5gB_72<810n*`4>Pru(E`>+ zd=&~g&*a0tMO{|T*ppt3-jlYF0k;5xTXCN|vMhQh<^BB^w19j;AzBw17`I4?gGskx zJx1)V{gX?l`9tY9)~2zte;Kw!)D}xH#unRDzd3>LTgSMLG2e=mONi{+m2gubWmZ{J z7qo|wp{)(Cr9mVr_2YEZTYaeV=DbG0gzJ==URXxNy;;H-ot%@hz`m@*%xso?F317b zzv3vY{-{^~iK!NefWu?K_I;pB{*z7ZY@2ZmiQG!LBNQ*m*jZS}+$lA-K`vV=JyqD2 zPaO~>t4O7~;_f^fEfQ<|8I^har291CQ7J!`3VYwS|9Ww$oZ>0roc}NSsY{XJc3a(* zaW0vby>$^ztMt)oEOf46JVDn0?n~?UmHveS$`LSLpfQPmnc(Q;f)xvIpLWAWHW1ZM@bu zw=-mWx4KgKJysol?r1ZfRNz$GIy|Hft48$3-yIaINc9=2P6VYnKux$eg^V$4nCsTS z7c~&1|9Y}Kohdx4BR5kwR{vzuM;yYgS6FCqnveN6BWpP=*6bS1Ys!jpYpRRl>$xyg z{Vn7;qUg3D8wL8qem%}|Q#^-fvhbLuP#BYK2es&y&;p9CE)SlPS6j8zkuG-k0Nz-` zbNK5W1Q;s6WwfPwhIg2PEW-YtSKF2o#j)*o2qv&A%<=dTwyVSeeo6Mv)6i}Qx(yx( zyMNs8>cgHGkcMVWcO09RcaWfz<9mgNXMXow4(1m%0WbYywtxtxI+tiScC?MMEb?ah z;Jo1;D;l-+LP<+Dl+d}+i;aG4xK%Ftw_>uTbJ2`4iF`tW@MQzD&zX5sJ+D*q4Arj+ z$t7#bE0Q9VEHT5P>ZtSSdsQm^kieV{<~{|5eh9b-hoiWqPT=u3T*r(YPxx1Rh;0%} zkE1w(4X>lucu#V#o3f9)FL)9IMGhJexBBv~vV`@Jhj?)U4excn)nT(2FGSU(@h#c- z;p3kKAA)M5Ii5$Hj^)1DdPSgNA%6C9xxaTzOD4L^A{V z6R$SmZ=N%I?6|@cHDTI?@Ca$?7LR*7Mc#d=u`9gQpB9kd3m6oE^9_E zeIbMy^##yJq7CMwBF9C6fyB%l@8Bvce4F5G1|3rA&zXRdTc!&^6%bIZ2@ zHul)dHc!$mFsPj$R_{2B5Uf7dWiM8;D15Zw0y{vyRmPX1V|6l>a5x#BraDJ4)@XUi z10KfDGG~V6Z)nA(Xyp(xK(4$ky&;=GWFZgQRWT- zDlfMW8%n-!czgr(7uZ>}`FW3)t=MFpm)kjwbDxhqiVI7VQ2yCD;fMiX4D7z=`aN`% zR93X4m`Fl z5m&q6_1i+SPNvokAbqpjN;`(;fVLXHyDVUy>S5kdhm#8do{tL1>gB)X+`qq;n(kej zEH7h2RY)sGb**_Xm^Euy+kot^MZUX2=ETqxg2T!e&A9q|&&jurxUy~ z!|I1`2aawBGl2c^=i7}=5siTxXL!#38F+oL4mAA|rG6s$r!&|LLi>c>mX)5v4~IRw zaTJ+Kx{R2>4+K5hKiWq~ewBTH(k%2~H?^2J>R^_nK{e8sfiu2{;yh!Zvnf0MEadpg zB_W$tO&&;uk2rsi02lLdwjIC&kfG1ZB5e_IX}@YXSXvDkBfWBBZ9os6K#1f39+DqE zv|K_w=i`x~J;y@1NQk5FCLUu;EdickX3BXa^_hd3Ud@4*Vj=)9?F>5cg4IYWLEb8S z>|y|YhiQAM8Vu}-N<+9K@gXRUW?ddiJPtD}SRpH3Ht{E+blaVQ!k6W7i(iWAyjLTI z0RYlY^(P*7rH{FV8O{8|{lva3M@qZ9(30q4jt7_XwTPzfVDuHdpO>C>1wK2D7|5iBK5>h-epg)tgIQ-h= zg|PCEKsN=Qm&ARyGG(;y+MW9l3>-Jour|Fej#2q67-hjKMd|CrDs@2Ly>dg2-d3B5 zyl@cfLHgode&#j$M(zlNcn%~^`+{zK#GWncpSQDe17@YSpk;3e(P)gzT34{`h}_%| z$8@IFpCNaHrd=K_u85dEgJCF#N+{Ro3#vPrbc0na-SNZWA8_yqwVW5ulXg7#5&)Db zdc_-e&8o$^>q~$%DXF@ciE?wu*1}Ue2_)k+fuS7I_+;-g`^4;##qkp~-mwnvSdgag zj!pzWZYEfH6LxgRYZ0_g+ky0K{@Dp^LXtV&|O zELp(XR|!~om0Q)Tp%D5`PCi(XtXYH(l$6LRw7?M>`*W;fp0W}551cu|Lw;iw8!qkz zrPO@W94GVFvQ3w%;@K`8^P8Q9s1l@>tKly%P+K7Dd?XGeC)eFYmmoc&s4CqQ6u>sH z-I2K5W|rJ)vxNi7TKSj$$x@=L;ppO239Fdy=Gq>ZG+KO(M4R+as)SV)MtFM z>)H*kcw7FRZvJjfTfFO#?i@yseW{b%ySOWxAqXU5YBX=qXe~I4nhpZQtx?3>*_7|5W&7`xP&S)*lB%sz!l&2Z z^~G1oK%(S6Ol@YC$q;sCO(tzyBE=hjb|AkFhjvTU>Wt zkr-j#qo@9lLsxG5qQL4Ib6<`Rx5zlR4C6!EBf|JO|LQqt{KELaM{0f1-U9=}pX$!k z{u$FZoMHMd5e&80Kv9aOLvGm50LbW6*-Y1dfD3gxAyu+_A-Q(m6yMp}A@+Ftpp1F= z1w2#+Zed?Gi%2Wo)sFdNzEX4Q&p#V@7Vt)RyHRHeIi2F;LR;~91ZX=1&eCBV*l8sN4(`jmFKg~gumrl78WXw~Py$nS@3vfBW zY88I}F$SFCy^B1JI0q_+q=vmQUvyt4z)T)+s(k8$sFFDmW2)XE>fT*S=JorN-G-4s z0DrS5hGUruc(95pcq|WXx?pmZ-hrpL=#mF~k5_r-qb!?~!mj8 z=|kgTqmd|0V_rf>b+(0&6rHbXz;LrJ;eJ*&r_bafz#LFWq^_Gn^inE8<8IaA=Cq_! zIiiU~ICWACF=~~<$9;qb%Wb-yie_J3u?MUyKaBjTGFs2}i370p0%oM?2$c2J?;Qxw z%S^9S^MY}xom)yO%FJhlAvq>dFFte72xt%A+i#cqe;9km#=5_C;ku1&v$1X4w%yoR zv2B};ZQHgR+jg3y>9e~3`jzoH?KLbF$(P#FIw<>vl1GnvO{-_v=t2& z90(CC-7yiY+XxXYUc0M4{RJx83ihZ!i}p?7SL=#)=j2GieUzO*09qsIk?8Q1oJ0OE z*6!e-1$&h^HBq+$2Mjp+gHhD_8V-lYELyUi`FpkQBIWPkf9a!Jl&cWGnA66dF90Ch18$rdj{@0gPnH->_K$nqp-18xX-#`b1HvWm`J41 z+ukRknW@Q~7VgXk)J;_zcpv=QwTiJiDi}{(3}&Mpp*k%-92aw8(>lG_Q)`-jww>X_ zE~IR=n2WAk%DU-whzj}5j)-^HyGgJS6`%7El}WGTP(9hU2-~W1g++LMFv}cs6SzpT~()*X|f zO?8EFQWPsWt%Q4l1qCBIzCif-409@!c)hE;hg209p~s{-0xkYI-Lc1wG~vyali@E9 z)(Q@q;!fZ^`U)=$LNYgY+QKq#EyrX{C5x+b4Z%4YpcxWe*CN>mNWTyZ;ctrVM$6`* zOyiu`Mq~~-75W(F{IPCeeMRN&U4peItdiZ~`_DP6t>DxT4j0g>2#QRn-NP@)_a{fA zA__HADRLCAr5FlhcIei&Tj#2D&uqQHui)DPVrh77;aF`EU-^Q;&UD6%Ckbf}+XCF%Uqm_`1XBf`@oFiFJN#rs9>OgH+6#|890)8_JX@%6Jj=pre#Iv)QQE&yzT>^gVL)bv*AX2z%JgP-6=s0RwNA~zvn{wgJGd2zh1i; zQ4QERp)_|a3Dxte7|yM<#N5GDO66DJjToYhuE}sv z7==Bh41hi-55MbWnY_a9p8V+s@8-CkXLh|!U)R8HG%_PH>><7)ib&sJMVq@8-8spK zF74rn)_IFHjgezclC#QuH~8@?9??V?%gnSQ!*)%-D#7f}PxJ?@e5^&Y9Km?*47+%U zZ8YJgGW@~>d9~&d=DIxq)L8!_%i=1<1DlU_e7nKaC{#e|J=fS(U^?5kMA%G<n)4 zxQ%+!yXy*zE+g(cLCkmZg>H^`ZSf8pS*)>Zn39M@BIu4qU2{o1%+qy7mS^x}7YRs& zs)85OM?LrfgX2u{Uhi+?1BC$xXqUr{oKZvn_bF8Xe1F~)@agC!@aZV+fB1iG|L+^T zV)cu^&qaZ?0$MH1e2e=1!lfW~=(ysANRWciFmMvZ{_y8^(#1OV&a7*Cw70q&xA$g3 zpuJlKOo6Yo6lm{$dfeVZ8Z1Pj&7Qj4Y=3bxaK4%P$KIWG+ZegnMlcB7H8I#0h)X?f ziO#_^ZlFS4a+etiWydrKXFoM!;gMyDjXvrq(VrbDzF9rWmup}?!eQ)=YQKD7o4aGw z&_CD}NIQ6pCW(GSJlh2eBO5@j&{3i^Cb`Oj|Ni_M?w^iif#qs5%V8s6f`(y=KhTR4 zupXF%X(7X7r_T{7?kaSyHQEraQ<~FddGgehM2HlQklAvE2)!Ah!HhZGWv4w1;+&;b z7|*U=G)df*PLK86SFS14K6-`Bc*uZ}FTkoQqvPywECk9L{gglbH(K8d*o@jqs30xFZ*1cRxPXVdq0f#k ze=S5{uz>MvbVft`hA3B)X>rU(i%S@6=!uk*-)SH9I?Z3RZAW&BSRO)D>28nUR^nMk zkH5Z}A8Fq)zvNsEyZdt4&nOAMJvY54II_D*q<#0r^fD{(lFd8xAB zw9{;^N5vs#8Q~O$G(`vd%>H?FJ>A&C%i6cohrv# z;>y8THQh(k*aDw8?y3IWGCq~*Qp~PCk1WBrit_UuA(BqdJmF|zp*M4lNanH-Rg2kb z7%zD{k!kFn{J1$Jm|@zpW7AhXOod{s0r1OOH^v&vn8I9hmy#088i>vkF~&+xw4JU@hUPak%j`Sa7<#lISIe6M zXub>=4meh_@;cte42PaO&{81gFFk)oqiLzWy;2gY`Z7JL10T_ggGT%&Kz0wvr7Hck z6`;-=#|~9+dk&|1jrmxO6thkj{7BdM+`$b{_YmGL9K+^d-J>t! zO#VW`>!RJVy+r$npqwImW?K=X7G0eICCBC~^Rgp^T7yb&R=zD=Hd>?g0Nz%dIxKcxfKVMGeL8XMM)tfr-ZEkzf^=EWN%ysQV|^PKkfm_0c>0>9c(=QIm8sJ zpQ)e$(a1*2bIA!M6oedy1Ur501eYL+h6Lb(YXd>nozEV$GskApFf)TbWPHi-o(}s! z&kr;lu2AJaiIYBhnyRChFkBV1eR;fi@x3{J>G<<`5-0GrD>%PstBb}5m+how(B2lAydMr+xz`8*PAJ<4<309`K?c@X7)Agjdqy2Mb(A`dl0=41Yf;c-}q>c-k%3V116zM6yd^pB|9} zc;3&34v~J$SLJ?<)1=$xK>J&N<|W|cdN7?Gs#gLHtcB$h8BNBqIW>`asaK(XARvV( zW|(!k6FqW*R`Xek1a$d+{!$BrEKY#F zYF9Iw9mg*NpI%*J!V%679^WEH?8QyYi1xRTxu(B~rsNskT-gr&W58iOVGUe2+CKYH z-E_UIibC*W&X%ULbX;>|q_O-W!ZkOL&{Kh9_KO>Z*m2si4a#DcK>hMMOsPC7YpQ56 zG`}&EC&Fp>$TlI0TknvbtC`TL&_0YbwDDa44!CCqWMms>^*+_S!I_wY2z)Vur`|5O zXKywJB1+SNhl@Y5Sk;UmhFnk&aM9?cm`SQqYH3kGX;E)0%h39(bEJ5N*|!+Z4ht|L zaGR!c1bto<&pDZI`!ZsLjx3(NIdO(+nZx=bp7?;?6rqE!O27a@JQ2wE0zT9$Z=%0G2+k%4-`>ax|e8LChD2PO;ad>w4+F>ZZY+|D2vdLgf&qz!_UrHR2ow2Iw9Elnn{e77Vbc+a|uE*AqGsVjG^@>hzXTInYA=W}@ z+7fAS`8&cHrN_4@GA!B$zYczxQECghUNT|b*%FnxV8I0@VB)F{D1Dmwb@W?ocYi|U zZeSnQ!$2zMFQi(3e_kHDoJ?4Ya9IVux(Rr0#{W`ET_vAfEx(JyyisL0L`el(&5tUX zcq8Gkk1YQgt70u=m-oXJY1{pWWuP&Af#^{f)r8Q#!T;W_lzg#T4+1iB1Li-DA;=rK zSi1hZnA)spYF#L+MLp?OliJo2TuHP(;FeOdYJHWg9UhaT^MW5M z<1B9p5g=Z_IOFox?x$LMavjQwsvbc&0oQr5`S6+Je0_Z5=lch>kHVo5ONc7yK~pJi znKo2W-iR(rzsmgVyNFVjrFZp^4z>+&<(cPPPbo3X(mQFSke1*eV!u~zkH_@c2S%Xi zmSoVMLU(*R(t*!Xho`s`&L$+^Z~R|XtB|aKQ?bWUtT)>0NznS1<^jFX{(F9b-O%ys zj{2+4g^F1wQvB=t&{4)mQgvPix-$mGk8FkwyRBV>+H2^+1b_%T-Xh~wN0n3K6k>>V zGKpD|?$odq4=EEn-k#HrtH$u&=6F1l78*4ipNy*X?POci8|90!bb94>vOw8kXct&_79PztBfNs~r|3Lw3dU=P-RU6KoR2JG2G ztKcc(%A}&cy0B+6q`)8(1Te@I(()NR4Iy|d{}>}Pu4ut#rM$W9K2h6yWyREo$tJd) zrF8sw&+C`QNzJT6rN;Q?q(JII^tLS-gHOQLHFf=U_`pBMz3g9;g&ne#q zJ(H5&IW;R{r4_X{Saj!?y8zl!T}F$8v&* z+Nfz)tqEM3%cfwAQ4{JDPXMd+fZDq7s6zU((kOulwNuI>k&>*D!KjKZaW(}Yz3oRb z4r^w6bzeuq#sS-gtJC}Dbb>fNxJ($l#3;)YGg{|Oq_pzh@{<$uRCa|-=iu_K^Mm_d zSxl)mU1!S?Y~{*&4o49>dvS(u*7e|@BV|=k=H+EO)*E_LK@r`E6_@$s&8T(xRZus) ziZ~-?Lg)HiPOF)S4tU&Tn86e~=8}}>iWU){1=0|l$=a^zha=9kz2+tn#Pw03=7@ac z=FBw2jyl5e`_5?L!-oDKiPYn*0=_oTVn-j^Ze7MThrdX&t^4%~zF#x|(};LMKT zEUqM_GVA|ntq|bS=M|pyb2l4Y^k`_u_QZ;4(ncqja!umJUu_or)2jUh791AY_6F%m z*e#yuT^butFf|v{Fx4ZPXZ(b*;TOyjWKP1KFA9oW{X5iI6yK<>v2M6!KwT$ftK@+) zB!Zk;dp`|f^4&I7d-5J3H8Z}6R)Mt>&t%Q&vX8jmv-Rx240T1;@q+;l-{@5=tg@b6 zm4!}~r7i3TLG>mq)!;Ym|K^z*VbmxUkY|}do-zGrCt-P0^Z$ov#p>@WxN2yh^ue@J zX&Ci^#N4X&!0ky;H5~$F*btZ)e-L21FHv`}C2I@d7Ia(ts_`>}eX5BiR^VNN%a;S! z*j+Obrt!t`+5`Ax)oW(jz_0few2u@a32P_^Y`g!&s;$z9Y5bQyij1Z_ap-&GN62~& zX`7x}6QmT}x6-{yQqQ`T7yz3+7-NkHwlXdQm@okUOr)`T>uCu=4DSG`K|;Sg(cKszjp{ zzKy!AL^Z~%Y!6Bw5=>imo5c*iw2gy^`fI5jIK4jsyxDRm;KosQN-(J~>#hFI9quUEr)2)(zi7S9-FG)0ycQ=W|ivW>5nzs(%s3Q;s_% z$(pJdQm6F%R97@%C&s33lLenlZHgMzrqB6#MiUE6$!)-ajAKo;+8P&0;ZKvLA;2ua zdLV(v|U{xP_^3|s6^`?r{Gcc~i#)sZyppDY96lCg|%8AM+A7Rj0Kg2{jNP5E__?xXMt-<=I}0jZ9Dpin1;Y>08z1rw+ef4 zAUton0}wt#!%a)p*cU+6eJP00pdn0?YxymF$xsU>C#CB$sZW+H2Js6yOVaDyeMfVJ zZ5H3EZCQFbHwH^+%Nw3)Bw(Z!fzPaxB}M;-b$R1(%MU9_hCB;)=RdWrk7%-^0JIzJ z>aq73nmf^NgYb)K2w5|~ogJDtt8tZ-W54TI8N`eVMKH4B9h96%r4-&d*JWQRJ?*R3 z%48t=_es*+fNj_)I;in;(`MpGSNBLeyPb-Y>K?H0k4b1bcw>rW@2DuSZcAiaKhwq% z#y14Ye}Zb8xm-f+TgIG;#7fQd0UCDtpfm=Narueio0s8>%0#Od#^STCzL)@-A zyduondK17*i%QOibsgdPs#Gc#1$H>qzVXfgh1{2ti!bEAh9xuhOWlbnAF|c(_6wWx z^I~VyZVG+0t<#Mqs z$Trm`%?QnK!yu|T;`*N_NF_#pbD@-vrUDWIVqC_jKS+j()I(`T8}sm9 z)AyXLM-3&_Me77*G0fXCr3tGi5i_1T;J#)P`&){n)9_GIxK=j>+QKr}JMCb)eB7B- zVp#-|0M3^@wc5UtMb-Y8nyWW^74Q4ediXQ(yK5Jtw@r-EEvGPyIwd80D`n=EORR)< z%*Ek+WDk;ivIkH}Vfar>>cU=!ke^k48mOeuU5jS!;3FGe1}{BN8EzsH_gmRq#{McmEtV8F-31uqiA4Y{I@P@w$`tWnDT=EQ*b;YM$4A80=38bN%et-0RX=KMAn{u|2?< zI_xJlU2Vo$a8un@g@$t;#TaM#j8r;y`1+{mUd}?>7oyfu?ZL+JPikn2!a@-k4gF*y zBNg?@F`m3a1JrB4&ntY_9LEuDAyi3p~f~;l!4$rY*n1zj!jkG1cYIaHXl572FMn;V<-aveH ztmEReL_U3la_vPZ={!O5iOthk6d1$<0F5w?IK`0rS__WGwFV>5zg@$ZRZgs4$zpMf zLf91Tft%R6hiTyV1xq=;Uu`XMDLUns9Wvo=HV3R(FV3xKXP4&m1~0sP((JhfJ8GJo z1E+e>Gkj>6c{0X7LvnFu?5@9_S~qa7)p1KvbaYRigYgfEet#NPOq}tB= z4!tC^64;Xq&Ae~J@rF8|9p@jnQol}->Y zw)l&%I@OA@Z`5DWWCJ2cjPqpDLz~~7+(hyj-Z@JN3-SuG{};p@J6y(8d&CL|cu}OEt2dM(O@E9>6F9UIoYbnjd z`OtT0UZk5qB#-nVNKN_O1b08mF~W#F6X-mseg!gba*@HvT{>w^Q;V0jN^9|e&cDfH zvr;Tz@>mC$JT_?j2sBIffUW2y4>cR4XL>xcZ7s5jRo6YRnrD~WSGO!J(jjBBPaDD7 z|62e>g+o+PGLv8^?>NH>)M2m^*WWm-K|+oa6KNblIC8a+&~s?3{MF)>W*=J&3ryp6 zPhiU*1EOYu&ExCsRdu)mmqS}&8jC~hY7yfpgGIj3e(ox*^w!^J6sFVP1dN5*1{F=@ z714SP+Z6K{;Kos2(!$4=u;~94#{(*Ms71a|KpBdkjGWU;U*x7RTWf7jV`rW?M)r#i zQooTn6v^jqQ#y^7MgtW)HadS5JJ?H}GoJb1(rr07MSyXx<((!b??W!JU)U%IVkEK`$H_=(X_pN;hb$e)q{eiP zp}|@(Z;tvx(AL2AXR08TXBG$z%Rf{wV^bI7_(LIc3O2Kjorg<7=uvcYidlY?pNJ|83Z*F7?6V1sZld{u*|c zXg5%JAB5K2@YElr;@+b&C+Vz2S`i~;wtjb-{mZbUKlHTMfuno%hJ^HY?iG*EJ5&+$ zo2E*|@gd!Xi}7*QIZP4uZ}Pa`azRSX-s-=S$0u<;W9Ci7e+OSkFX`JWZ@!hSqZNW#PSL?tA7yoL)7}RK(dFz z?U>;bXAGhdKiC{D=O&y*KaWZ>Vd_~nAG#tprxQ$`OU5?=EFOQ`u!qpp`=@v;{kM1= zo$IKJ__uhB-my44-_jL?icdjfL2+QPCJs5xEx()!tQuyoGe9D5NBihGn4J-G)0^%$pmJj1Y-j!fSAPd)<{sRNaPJ62 zP6Z1ZIP4Nf?)m>2c8=i+Nn(MR^8sRx=07_eDmeYuRe;yO23>kkYu~k!i9@94m7{=7 zcjfYedqwCUhSFsgw9oCq8+EP*TU&~!ll=b}#Lz@afuk-r0pO_1F11aNx_{vEy89&e zf#=5g;UB!&{>57(5O3M;TdBanaVU6wJIx{u#ep6zc|=vo{=Fe zoUPWzz4!DzY^KsNUD+oa6DzK3L_ULgCz+aa(>}LGREs4_t271K7IisSC+iGS-W=Ul zp0Rs+Br}hv)Uw zsWS`%+N{dW?%HY`(X+9&VD-5%mi4fq^xhgTI859+@;tZkpmfCvMsMB+D&IU7U5hE$ z(C-oNlSALF;L?=O3IQHp8?{r2{87OeOPx(@5sLvk@)@nP*Jkk351bwX02}n*f8}Ab zb{OpkD%I{NlofQi9i{JZRwP<8m|%@p5Njs5a6zNUiMQBf!wa&?8&bZ!E9|T(#M;Vf zK4~yakN0y4Pu71m;UBc{HK}%x6_s|fbf53L} zFR%?e{5~T2AFvrvva)UeiK~{BfcjpkZaupI9$FOt{D&dq%m6m;+n@K!MrFJ0bYJF z0_J-YrSS0P>Ln`hkho!ye7Ms6Z3JVDdu6PSGaJY?pa14slu(k}lJ#BD=x?&=BMkVQ z4Z!}wZBtDK4G5KPFQy~k;vDCsM#w?tfddhn>S*quGxepk<6ycAG~*+Olg3=x6Od{~ z|4?mDYz%s*Ch^(LET_^_o+c3_FfQscYSM@GT+G2ax-_oi=8pV#!6~``@>~!LCu}H3 zpdR7(`HQC1aPh{3ZA=S}II}kjD>Fj}OdB4<9!dL*9zh64xNl;WJ=C}hq9S!TecuAY zmytpvvgiq;gX^e$hP7nOApbYkhB*I?wYC4k8j_AtgkBS`*neRSpR|(Zzp)l+=6J9B zZ>%wMJBGN#8Np~I3_tyYwM8J-&^;~MfLNQ?2qxzj^h^%=+9}ks16S8kq1)^~>=@Hj zhjY%!hnfxyxy$|wYjZ6PQU8rKA;@YV)>{6-TK|t3(f`33A9iNW6x_$Zu*U!8|H0bq zU#xNeAFLrb{laUGyKM+4!H$MdDsU}?G7!H1e`feT8Yz7DYtURi@L#?p5`6i>_@7}- z(aF-z#q{57Yu1Ez*IfpD&MsPPpPqT(>aY($Ou}m7#PGJ%^+(5Gj>8UU+kEdbq;R3- zlR=C#D`6oq0WV$qRxH(+5I+PR50*$tM8|{{!$>~T)Sr*E&_pJc#IBR(BGqbR+cNU& z2Ix||$0F^iUE#RdcDrdm`7{@pW%xKQZ~US@X@{T3{tb(*jxevhpWmB3SYqk59qCgt z#t(0x(CWh*s&wf!-IKS5z|Z*hE;xV*4KDgX3cnP?bqMuovqe1d&+74 zgJ2ftRpmpAt|$D&%8|f8Iqn4T(Qf<{Sm4VP%usnphHt=BhiATA-V3!?d45DD_MCQp zrvZU+P>a4d$=!o7unOO_i+ldt-o=|G*dALmrWbzi25qZV&H?DF2H(w^BSl9L6NJ{_@`pn?|=(mfVmD-VC4=K z6#un6HV^w}#Gm=QUZBBQ5YspR_EUMZ7VdJSHs!~kQr5>T99<+uIpPkB4pr#K{B7?Q zX*lR}ZhHkhnW0M;8;uqDqphQ-z`k5M+e&aFH=Ybh_@OrQQi!%v=USSaX36C>Yg5k6 zeKov;k{5a6Fj)A$?uni2=B&Avml zZmj9{IHwuvo2ZbPKqjj1hyMUfB))vFvLViMRW)azFi5*_*p?sRzDM>d=O=CoQ*yhWU|`FLCAS$Fo0Swg zMHs3jOjDahLQ6tdGg+2gv@G-V{m3}8#&>QSF30Ku6dT{^d!D(JOSbN3LFxoT^}B9! z=2**z%J7Z!TV4um*$r{@_LUUcRPoeQ*~;`iZa{Ca8EY2Y2YX42idb`!!{q?&Ac0?c>%U9-tF zhtk}YMia)HRBij{Dv4C^4WeIA5qE`9?MH928qnOR$J3x|Jk-E8Vp+tUxTq%~g=Q^& z+Ubmc>gVD}58=vc(Z|53CD|w33!2H^8XY9QCS(&F0OTh$t-oS2}@Bvua|9C zKbE~Jd43dR=gVd~;?^pid@30-?WV_-Z5LucZ-}_#2Pfw!wtZVS)8b&e-*#cQpTGcj zJW?HFk>a`ONTRY;QQNDrd2_9d0~fxLyen$Z>9k=p&9Zsn8T(RD;*bXZ z#^v?6<<17G$z!P~a@JGyXIasW8Q|mFoW43ch1^^Z#{zCxvm#^~)i>-4T(D+yv%S)j zAgj7y%$IxF+3{^?qyS{L9F<%6T0FIH6C**kk;|sV=*F{#>^@;uHQ74HYUgAgP8Mm4 zgth4fg$W+q0~dRQt-K{`cm)^$gcH*q!6^LY~F^|i{b@u7|geCsUzf`zYC=B-cWyu z+RzZvCwL6WKUG?`mJYW>Zq4C2*(xBKzRoAelbGi{#NP1;IfCUcoF z@(o)z(}D)fy2L0lQ9%`;;lb1E z3UlV|6B4Ypqp4*@A?W;!tLt`Y*f3S@so9QfU-8p8?f5xCciVWRS{m^LuxIY|(F}R6 zTH&^4MBC4Q3Hd`rg^tTS(`q#54zT^|Y2P)CyXM0~g6j}_F7WYYbmtLl|5AvfhIf!~ ziai!^%GuNK(fsfu3u7UHX3o3I1QBYrf`{* z)0pOCaNP9Qx;m`GOv3rQO2rxX@A3W_PeFEnh6xTr%rxwobau~nZ z%i|ymV&#Zyn0g55SVAwh*>8GtDtIE%;j)|IaCV?*PDNpi{3%m0Via+l{3MrYKB@_P zcc8AfJIUMEI`Lz=!vuW$J4T?giC=K&1VFj!0L;25In3Ir2bL2x$zL zdz3D(V?7JPblN9ltT`3wB}ALV2dy_nz^ecXw7Hy)S7H>AI;(dv<|}*=7c#DT93`kz zyj0i&k$FQO7Qz1e;8ibA)L!k4{X1)leeefAPp@ruXUP4`Rla6L@yN6{9+AGa^%VQJ z*bP_-#8g*71-$v2kGZlv6jXK;_jLH%miEtg8sKHQl9 zbCrw23mlP1)7;;tTk)$}dcw*X9+T>36@O78fzG-VMG# z`UTZL;ET`3VNnQ-zGKP1OrUrRw(Rq_gA=Cy=A>%fBVqQBo8WU79vh)_t?H6$( zK%C^bqxwa_PbbiCewg-JMeZ(|6(i9s_S&`Lud=S@t{b%nZ3{orbfGx+Y^=-UPtgn} zZ!K|~oH~MiCC^*k4Qk$uxS#EcGRw`iXS$?R5<=_oqWmWE8Qo@;lUO+^oobyz5jfO| zb5lCCtUPI4v;vZRfZ^@0tbttbx8JiL;T1{IFpLWN32m$sg;;?n)0A~bv~ad=#?CaO zl_Ky_HMADiK3SvMxjHSK{9gSS)tVp^4c{-p39D#l701=FKCYEO-0>b^*O%u(aYLWm zm99B~O;CnqB-<{D<=G{n9&IptfcmuGDGD{s`;gX}6pT}hAR z>N~UE4H@-0&E=s!>0B*$Nmqg*$D&dSOdyO4Q#m1EiFW4FWOI3}aaQR&Iv3&4=p8gn z2O55Kwuow;_=}tuHRmaS9JZ33wy21OU|@69F$$&IzuOJk=#N8z5;V{9ttrSK9Gc_F z=a zy_KoSzpoOfxnSMVj+#%VW-evfMWAp&YP!_H;j%D8k(lDYL5re12o@OLwv&oz{=|Iu z2UGRleOIe$3Dh<)^j23*^lw^T(Wre6RO@Qhw!W&iwO!V>?_S@ge$38T+Zw-y2-<%4 znSQwvc=+?@*X;X~f^%pXLM*|+?NeusMsyxx6-Va(n`vjUU1^`K6> z4DGkj9~_8&DSIl|pQB$rZZkmg9vM7X9@L5HN^1v&Y z-11#57btoj-N)^prZx83c=ByLzAeoh7oTaZP&FrH>(po1Jd3>jQrXKmaZ-y-?F;9wTrse z(wjn>!#OY_r^LqKX(NJHEOM9%M3b{t-B-OP518_LyVfvckOk#CYk5v%hDmS+nwqXW zMmBHYUDG%qOtf1uGIV6j4+Tw{_>Nn{?LOR z@fMn%^=__SCH{p6gH+K%7p?^rsx3r^= ztY7Y-6ds4NXw5JqX^xoU5nh|O47hAizYt63X9gg(jmRHE>1>*ADiV1PLJRf$skKbJ z{G4#@crb|EBZ6c00@qrhB66{2iu?%2h&ewlqs_bnDEF0}DzY|8)I^bDm@Qnh1Z}Ic znuPS{u(fhVbVgq~8ISBcaD}ut70^&l;Mia0LW!Ma4eD&m+pL6raIeNBcZ5e1g=jVf zzBrPRkp$Kkm6QQSjI~wJA1rgCgzWu^jctS#b3&dqgBJ*-*ywQ(!$cA^o5-%?ehv6m zGhxM!QbHaguS4V=-&j$QfOUxi#$;{bAetq+cW`5@!1Q=fCMBL4k02t;FWlJzg3Q~=>uL?fH%ktUxeQPnn{RWNjI>CT$fg*?6cM2i>nNL^o-GYr zVoXWX%3Il3P*T&=37sD2Ui zAr<;1$t}B^79)12nl5}?`DtMQr$OPqcF{f~`5?F|TpeYkM`Aemo*A|Xb*K^Iuta&}#kUb! z#qBZY(gJ$zD{=#uyaO%;^LSzBWF9s1iYKv!^muR_mut2B@Qn+3K+tu( zrK(#g2NvA$CJmb#<241DDxMF4Ilg|NVtR(`7MooUCDse9nJ0_aoP~4$RaA#bLR-ne z!EI4w+P*2ZKEgW^c@bkV*3+AhK8o%Ljp`;y9Fgo4gy|s)F?j8ukkfz-Ba@X+Mh49t zb?|v@-iX$9rCJw6typ@tm^(_Gn)NALFWu=4K*i3<+Lul+1jZ;*KLl$E*TvjOb@*ae z3%%BSWBZGnQbO?$(sxU}?~^Ipb}E`L5C%4E)uK;Rd4jXs1A&PHcB*&eUD)|c-HBrW zRGExXP&0`nP69fWI~|~dSmzel*IgPng!rmHSWUe>g3Lrjmbjs6tc*9IhgR5=8Wj<-5I&DD~{b-=?=MFzOTpYjd*j+ z2jUJ!pa>jSD1_YfxlxH@KE0NOT*|{Lqdy<~KHMAM6vi*=tgj5kftO+aL6wS>ykXI9 z^46ly_gK>;9>TFlhwipJlW@e;NRH&l`HkdmEeuil2)iXV{WkTHhb`soyj{N~)c8u= zx)Pw(m-?r7v+;4Jw=99sLqzA>J(4{pU@t^0qdAL#%6O?Y!76797eVEdD2#r>#-0WD zMKz0Yow_AWM4<@b1u0gWQTuM zgtWIEESjmg5c{D4RZMX zCJ%L3Ulq=LvKCuPe#0_&oPy6cv+X`Mm|K$qcP?+CzmBSNbVg~<7|Kc)FP_AGNM$JM zkJr*Wf_1&CM1Btu18Cj6OM{RMc)LE@f9Gc^NgCmf4Ux6#xEELAz<^=q4`Ld()|7$4 z@@9Aws6$yy0FimDjVF^x+K#EE0BR<$@+Dn<+AD}CZ*J1=B!~lR>rM1@5mLoYWsT-z zREa|Bbql=_6IzO&g2}?|HbrT>Lnqq!!=Cczwv|6^L=yGR7q^>8O6MW$DX0ooUp!-`PmsDJQxL(!Z35LEj- z9GG>#tfG7`7Q(Cvx_J7NIu(GsE50AIV;7heA>jLtF>bZE(~{UkyeWKB0o|NZgdz8< zw^{~36GOZF&@}TiEB4zm|JWH4WWUjvnOlSxk%s?~M7+w}5wH#OCD76-RC0zH4Y_f{UaV`qi zEJ=9pyUSwqh9((jVpLQqiHB$m@y|jq}*AJp;fXnlnQBk`FVtO*{-Nf)XQb*kO zbr^n0w$av?A|5jULs_!wbtvj{e_5x4Sg&M zMZ*|Ee-mYxiA6at)M(0HiKVR%vUkVtcryv${I^#&${H8LB^n17d07bh5>38%|RfIOQLG3XyFyiJd=23-q0 zYa*3BBZfs$8NCjLemu@j&z8sWAD^KbQT+)Kxwr+%jO+0w)|tS*Y6D^yjM=Y)N5BlZ zjErjTC25_KshQ||*}l}I6p_chqxLFj-@J4RQJ8-kKeS5D;`4w@KuZbvcdq%}iQBcW zl*B78NG7%r!p?X4x8JxFvAfX<&&t^_(8z*~3yv&y_CT{;^#ku_7LJ+PLD}AaR28}x z*bE9$n&+iu(v_MooEa}D>^GTSkg6nwj-c46*G7F%G|$+1w?g z-2|-tR$PY6qJ>gmx0q%kX3W{78nFYonjPbGK2&k9L$cvV$0-~^LguGZqEM(w!alSk zsXngdTHk$RQ1K|E^a!1lRNhjR*VYVHp~E?ua*Q)PIRd8|cVK5&I)g2VuGZ{89bVW{ zG@vVx*@W(Oi2$v0z~!Bp6^p=Q^@_e?P@$LMnDAGLj>^L6cI$jqq zLrQ2{)zW#{nNKC)&XLVM@+_5-jXrwDdg?HeJx|5WU92eoqwwnOQFauph@W$X%xvqf zt*GcL#hKN}N2HJi`;&}vYh5I?&@nj`?$JZ+hNX}GUfe#!+W2cFHr+ITG~KlS*o;d z;{4mAkcc$SWQU9~bBT}qt5rN|n12yDwBllJ)LLZ8H@md0U1`^GY+X6*(4v47ML%Y% zm;QT%^OqTdU<77y;;O38Vq8!UQ;I{am?m6xcZl6=jEAF5b=oy(?T9`8B}BWhM$1Yc z#Ozl5i4251{NRW2{B&;o_C7MR@6r5N)+j2!5(qC#Qi8T!NAzwi$P-I{yD@{hRfp?VYc49)H$ zeS#vu(xU|vqlIKpLN8lj9v9M9jp!PI#-g+s=Q_|F!^ZtkIUNNtLX1V%9}?bU!(-?Q z7=`K;OS>hKN8%re7}#-4bbzAR(B&Lv$KGm@=ABWG8r`7pL$yiUSKUD8oNhrc-DojM zu%*yrBp5N?IM6inLZ7rH+?Q(UH4eFg%_f7lp36@bjH( zofoK%0bSqJvqa~jPJrs_#hrx*l+HjFR1{f!j+AeSZ#d`H3uRsoCPH^#G=c28pl#P8 z=*^M~U#`rt`6?Kox^56&tSN-ih=SJ&ZL40Rw*cz#f>+GSX!>zF;UqAf+a9m~hQ#)` z8u1>&z0y}y=Yj2E&WwL3m>r_<0&ndaWuE#DN|0b;r`rnYGNLM99*eLQKI~(|{u}+a z=R{rx=D^s#aTv=68ndaVvptx&c`pug-RS;?I5l0b=U1inXQEBm@S18}e7-STXmi(U z`NdFh%i488>l&~>b5`wXri5z0P&n!SHf=ximvsDd0;n$WR@ml8n%CabVp7*&3Ue6_ z?0~Ej^+~@evR{T6ndg!{8AYd*2k0V|c&RG*xhe2aN*3e$mq$Hc6~Toan1-IYp;#xc z0Z*0w5EeCz<==Y!Em?g!=&P*=3VElZr|Z_FNT&( z(Xkeuwl(f?v{xc=+d*Z{n`4lI?M!>D@djK){TF#l^3-z=V;`K=mumfJEySN(SzZiFq~@`*ga8 zg1tg~bS||LRu6d7VymPlQM`B$tK~(rL0tLBTY*FlWXOlWauGjDzq!F9ZVAi=L-J7Y z0vc}ZPxG5tUV$qI$n=SLK~nZ?TB9}8Rf0nXGl5XtC+~%yOcZG#%l7zo`NVOQX90S0 zD-!u*ZTmHg5uD+f2k0K5{c|j2{xPr(2EfzvRa^$)3rL2!9i%EqNs=UIo5BT7Qu@4R z?Q4%!e6oM*_|L>D@%aII!WJJq$ceW7Xbj}&BMuGHiJM;1LM3EQ6LnX&X#M7NS|BL( zH+AQ3vMyC67`A9$5(co*hiO_;RmXKryEekl?(Rg6w^)4uhDJJy5iO-e9JrO52Y_m? z!}7rOs)J%OiSamYv0iHk_VVPqyk-thUB8IM75_~0z`H>f^X7#ZLtY56tipu3-QRDG}ifBv_O+F)!5 zR2k=A^V`SP$mW~L_uI=%&!5XaIs~hcHt_sZ>Od^aA~JIM!?Ghn z_TN+nsuHCRbjo>3_g_F?Slg6^vyRbdvONAPD}cTv7JR79+lbERN(fikl_vjUp=Y)b zi6%&At<+LrNScI}HHq}hZcJcKP<+y2v_5>t@CPcd<_v&7;rbdcPM0iqQ$$*~5Sg1F z=WYoML(iN;?&>LS7H1k`NYD&7R4aM)^s$O3ptmyO7-(-Sf0z^SPS-R|GLIsrF%Os} zMH!DA&Zgg;EM*l`#hHtYdce+MqKNRdn~YV{hEyuH^vu>*^=hLRa@e#daZk>xVQQ}L zF#K0MWL;-sOzaPVBt_vE92MxGoTeOqNRyIjN3DUWU`}sl^Fy@HB)I4d;biqrIm4bW z)gE7h*s>0?z)U^Dt38XowCF0q1_g`KRmbF9hrA8kHc18TaUnkMoRmO<<j-I~q3wd0dHg zuFNVq&x1t-aoZ?#>G|L}=9H-?B)Dx0?uxC4SI`WImPIfimBazqFMVU&w;d&ombbE#@-RuM1rs+5i8- z_Kx6LvX%CLXfyZdGVYGSIlL@`k+%;3)Y?Yp0Oymu0lsGsR#S#DJ|dV=p`pXX(kLF$<|N<1)Z# zA9$*)A+r!UXRS?%lWj>`#pf|pbybWU@j8&snt+gdbPs*pG^|P|rgB78l1hLI&DMaQ z5YjmS$DPbU?CdGPT_@wMV+tO_D_kVWR~h?Bo=$-KhvpFJ@FMxf2z9N)Q`3B`*jxh^ z62(2oO7;_}##P73E>I!gNFeXZ@$*L;?vu}X`vu$-okd(^4o3SlvRzbV^>`X%XrHrg z=?>Fx#p_lzT0G7?zZ$8(!x%gu3$8~tiX6kgI2j^58;z~j>sxJ$sD`h3z6e_LY`sN$ z^I$d_XBC2zVRLRFH1tbm2O%C|YoQ@Tgo1gV4}Ykm9j(u%V;n-Y$}hVEp|Ha`zB`|U zCH~T$E*$aM0M9S9M(2{uI@omfq>uEB<~V&{nE6u$0H(cP2ThKa!qpdgX5ZJ;b0MNE zB@$~v`vsvj=(xiy-2#GE;JxDl7f?7*#$`vel^X#8Y3l`E=YHw0pPyU?xa#oV#(nrP zSVied0c$2xJ5mZYKVDMhLLN7sYSxWq{X4Ezsro4cga|X;UZkQLn(W#-LM!C{UYy48G?Jey zWB@xq6$3~t`?&aSp{Gu%?wVHv&b=YWOmBer%v?Mi3mwORN7O3xOF}+b6*jYAgPC@< zDv~OR08o2Ka{WPL0k)VnX5(lLB*DeaMUNiV5(gG$h}BBWCHFOds~#| zwq)dWP@$jR<(;~#X3cYwMeRVgJ$KFs!)+6*0Ff*7Z@B-w0IYUw0=H2A{Hf#oU)xv| zZ5{r5{nMoZp{KNj(#6*BnKlMT5Qjg&31~4Y@Q1-b4Y}U{AHuJeM8PTkGth5LDl;u` zR{&=X3bm+0EvD+zU_DL~gHeCvc1BWav6i!Ly&*7P8TcU>G|9=6VYUq+Eb%j6K6cuT}qXQpavtAe-cc!sirI?c_pnP*r?XD4w+g{w7d&4C5l zika3$-dU&40Vmv>Yd}c$NtBJ&IZtO8+r$jLlN3^Z^cTyDyN#8pXcL7zw#S(x7vcAZ~l; z0H%wpSzFsYRgzH@5v~S_psKynK`OKJxS%TQ&9P0}M!}sjZuwqqqh9Y4Zux;FbN8|! z)>{^)E?LithZ)iiK2P(QaSe2j%o6|i-I zZ#C0FJx-6&3zFl1vv?L%nh3C9Fg|8?j2H_#c+rC@F2U((olARSnrS@)JDmgM7vPOa z;{Uut<|4f{Jd_)Gk7M}zL`X&HXK_RS%ClfXF8!rXY+=kho1v(wC7!>fG0 zVIVYKIt~DrgIv2!9!ue_<=EVY4H#{0{_{8j@90Pi5%??nm2yFgwoY#B;<_!i@(Mw? z8|ja88qOXPoZ`B4Bi5W+>o=-_Zs~<+Tk?yQ-&or=5@LPJ@HZQ)ffbCR@pi?TQ7PbQ zCi-(QHKvDgF|@7Lsr_i2&efbW(B2WrBXweA0!Ye}zOOTdCz?Z*&ObG{NOE-+N_VmG z1J@IYp^q^%R66a{gi%*6CrWnPQ%jFjk_LAt+*{ce8;i;Q2G2~LtqN_)9pyS61)Fsd z*`KF4R=&9}Um6y&dE|M4a>mBfss-f&(3RETAgO{G*;R2f@< zrgWeT#&r>~$wV~iOBN?|W0~JpxjP3CD#kkGi7zZmU7dR~8mk1GO*}k1>m4;IZS+II2985mazP2&f(R({nL)WFUVPCpIh`24Xen<5K!%{OCVbxpP)sjZGqV?f7R;_hN%R zBakCa<{`$om|-N~-4_ee_svq23Or#&OOpz7Z@U2#*uVzz^zn6f8j+5X)j|!B;v*GNB-we%R^=%I z=otLzNfv?(4T=i$B1qO?a}0=SHPV`I&@to>a1#(O2=}r`9W6e={qhkA)k(S&yCHKy zrAk7$3~!CNX-}fEAnJkghymP12hAALqsSHbnVs4UiGGj+PxPy>qNuol;H&K*=hU_3 zdRz~mG-}0RS2JN3Kr4XqQ0?FITb4}}TMO1t8U3Bch)|kMMpB|*udGA?iqz+s0-Hz> z>W77csi)Q~whl?tYPMs$aV-(K^}p?a*DF^Ik0U0>F^n^mG%1m6C9^2M^q!ih@x=UG zP(tfqM8p>wrNVU5!%h(@3x-YAIwBjHO$S$TVxuP8ABmF`&}gg2b7LeX-2tVy0E?mH z#kA*rr*KH!%=YvowS^d;E1^k~eOW$K0YbOXaT%R626(dM4ZXCIq4ZZ#z+EKrwl*=s z&shh9HdJ7Ga)BS*k$?F(&M#S$lP%L0dst)$)NZJE%5VdjU*o7l< zIP(F!`l2F_{H!kORF*Wc|7_=@VjPpYHwRDErF7D~%kB-f*`gpl1tgHgna!U5YV7YG zdGcz@DW`1u%uPTxnKciORg_F=0q>B7F?t=Q7&0swx;xk|UkK0)?`>=Yo1 zuj&DT6M~UHr67NT;CiV|mJ_dl%eS9dzrem{I%n_qK*dACBcz-<#x->FtH54r4T^CH zot60!RG{bY^xMDAR#WJ1h9T=EnMl=%C`jmtoWTYrD~3tsmYJ{M;p8oFc;?(QK4-y= zu&eyx!Ql!erlP~? z8xm*ft>Tjht(Pzqr*Q+2k}cXsaE$0tyfPwB(NXZSSVK3?Q*HnUaa zmiV!8&NN^0&=HIwewI%^pK&@XWm;H`4{7mF+jP%v&iEGH(LGgGS_30QhJH{w^bd;# zcc&R7=2iC=SFb4egpHj4OgWGjrLc$>O_l&Mud?I5$#rS>FbN5(&j7Q>ztMGTtg^p& zu}nxP?|VDbm8Mm*iuN}x7V_OLUhZpQh|c3n9rV%OeomW>&0XFjk!qj z5z+=qOPTN80uAkzq^XD`!(m)q97YUYN-akRiDnEqBqpMm#TBeK6^J_z_1jOQ%WP9Ex5*D*=0;TbT$#wf?jT< z366sS#~P36uyZlYVVG_wI`YSFLh*x*6|xa~s@86hSzLR*OjidoC3VD~g94^MkA`L< z;4i7|y$JmoP>@t?3Nia}OsO9SMag)-{<1D?>6cp4E`HGP0hiz#E*rPN{9vT#YdC1N zKewoU0cR?Pym047zYoz(6sFdhY=#KVyelR_r|^&wCa0{$Mk5j8HqB0l^2@wqB$Grm zCsUPc++kuWq$VCTigbBGQp;E^Y!9(D>CWiU@#PbO)AV(%_twM(mFiM;Wx4hM^S{Ic z%72-e&NW&r><04fAl*&Kb-j;sKY+lon^!S`B6@BV@ftt~Wk&Z4DD=~%bJ)r5By?w$ zWt2wuEB*|CtrEpt9*$0`9~pjSJjI4d?3O!zjaWp{JCAn8R?r`hZ0}BZC zaY#K45#f^Wp%GT16cwe9stcB_9ken_jWFy`K&*1gB<-E^zzP)z`TF z?%0lhUXYNwo=M!d!gY?*d7;hW7DVlmlBoYih|SP(3S8&w+^@b|NV)8oiOJE04%2M=_T(;uMor_D#eqJ7UX3vMNLCO->^yeKPkFBjM#0 zm?MvIvJ1sMH-+jl;i-(Qs!70Jfcq)b!th%ixI>Uc0TKH zx>(NrL1Fc!Jt=~?kF`JpccK%! zHwoO~ojnN%N5%8IlqIib2e;vbwmBD`U}y}rXN!kooXMS8fN=fs94|6GBc82`3P0s| zM_iu<8SR#$<8KMD8&W;3XvKnq&@HH;nu42UJI#Phm(`%<)bm@qe5w+vu`RgbawQ|r zi1yi$#;+L+AF9-s<4Oefs{{ttDNLcBmjE_ku+rsZcI9VCCGxuUjOm6GxA&s$>%p6a zqw$iU@v=JT2%&VO7AMFcnWin-MI}`&P`+Ow682(#p`fflmEaYZjG&Q{U=^0+n=&k? zI4gOJ3}i1rmnK1#L3^Bc8!`J8H{}RxddPH93%SJ)qfFS+CH-Cp^Wx>$$mR*jN^z!e zvmh|`aS`vQ{Th?{K=f4m{<_q>iMy|mm?0NTe+C+Sd&QNoX8OQqh+2Fw-1b1)k!@`K znKt$wFZRYgcJD)W*F*IA>qZ37-C47w`SS4wU|;yxWlu|DyE4M7Sk(sWlcCgqj5qB8 z$&W{K_NFg?9*A>40e`&3dl+#BGG@vUxi_sdD~rG$xc(juDX03 zIQs3S2|cmqB>tqp@*nB5Ak{I_S{aFx)Fge&LAxpf-EMuq<3C z$@=iWej&Jh&@GDi6p(*qwV&lU(tJ_L%S(gy`oQEE=_k68rEXZNyApz&vBgfNnsO2M zgnL8nj;tspq0B6pOk9Ll_@L2|)N2rI!AR28_JYuANAIApEZbI9ZNes~JTFKTLx!yA z?^UT>OVui~gPAD@tgMMvP%+mmOIo^@=Jtay4sSQ6N`Io@Q6{NcewgEAA2z=eb6g!g zXDh~_tm?bxb`L(NkHdsfTu^4}n9^C(C-rBl#2~gQ@~}QDc(qj+`LeRC zM$a7sqS1Oa7UeXTtG#N>l!+oxZ=f@C%1>DhEW)Ui#;US}3G=^1c{!F9De_sTAYc+n zqC&;h{C(=cH8LR7C;Jr26s83=uFoQ^19}rHAi2zE$&_B=#+95m2RfNm!4*yUNBhPR zv(Ayj`38M{4A`DNQ+R!aYrat_*ElzaOG}xpq45W-vj9C$V~;$kV&Sk=39Lq(-Bmuw(wElqw!JgDOvls@=h4G0 zXyf9#h|1keBZzCpNw|${M>md#Um9ln5-{f2=|%qGE?ZF)PH@N}a635g05ud-OT;Ey zw2eClQZLX!2mCW1k1lwCHrHJTxKj%pSV=B4g%n#(7`C}@%_enrvH}-4FHN-pVKX7o zV>9*6^9UI&QrVK@kEE|nqiwE6?4VlpO_~@XM3iDE6B1U1fr%mlq(92NW1Qk zC48syVxJ{K;{|--(j)%Ld_iY+_AaM;GTVF({WV^vu>K*XUPZz&xsjPz5(gv(EphkB zU$B<}E~r!USYO~B15QxNRuEn<X#AvC=o&-NB`D$qg@QXHxoC_`naAYC4Fl8-TEEd{iVyu1 zK}EdS3giYd^G~I>t0F2sYHL(V3(12)uBAh5ikyQ`0RKR3NuKy-+XFWcvIu6)*yU6CPlS}3e0gO{L(KrfB$Q}` z)|hdQB4O@)KxgDe;O@W`ZdOxVUr4uMG)5wG8}y3OxF_o8RT**y@2JL95F@xn+hpa4 z`PO9_X1DrWjTIE{u{o^3rlbsP)mg6s9_8$@D>5c=#7fe-pB3e^Ep^-0TLuM-Bn5=q zQ|%RY`8_XrX=%2ObHPgSXPjC%$bDj-yko_|63!|QKeYpAaT4bwRi855&qz1QvK6O< zhrAr0R6hCw1DbtZL%H^HY1z%#CKsg_UMbm+H7E}+c2h3^7P`Qbu;LzU@~1qRLeEj# zgKY~oUY!q{xK07OBeD;@9-SUPXm=#}4qwPmtic+w%!?sWz?zvkmbVg0D<(ace_MrV zCCmKl%-Jw2TKr)9eZJ+~*RQItR!JR@bs;`9Bg3B%XHc=8bXWU)I+tJn*PPrDoNBuGm*b)TTYCR3 zOZ>5QFs0MC(>F9Trn56Qr8Bp-v!b)IH8nTXx02F#Gj^bpl2x>`GXI}Jx!P|}VhQEj z*0~{rT2M?NlGqm*MExFLY<3;kFhS2zUp!x){8!Z1A&9=ZrTzjim`|hjF~96W#j9aW zQ)i(mCl)H7r%|?KVcGPw6ynumZMj2tipD7o2jT+j?zd2jl(E*%VKfyl1!YEPi*ooo@V69!(;4kBI^5W1;vI*vwh-Q4)2 zdC2KJcveI5DA0$$DbV`_Z6Z*IbeclPdtWZo1uWaJv1*ODG*IEv`a9VQRW zTF-+5%S_vP0oo8H9gJJefaO!%+>vC z8Q76@(oZ_4a8LrWS8Sb2Cz3B6GFXmA8&FqwN^=wBrA|7N)D}&2u5^gE_@?I zO~l>~@&X)*!IKfJvTMLyX*(VEjNj?ffZQ-g%K$kiV;YOXVZ(=1}NZVLGuEDobyA3wZo$y}^+&ARZ2ZGEZG=40n-XY3I)eeEy`2Y{{JIsgTm91J%ZrJMqFhf}rmaTg?$FFBEW3E>IDfvE>bi`BAR*m;M;WL~*Smgs?3q#Ly4P z8yQ31PgD#($-A#&2PKT4C>*9lFRdfthJD=AVmz2{BVQcqz@Yt?ixW6KnlpmgyB5gc6e#A*w+X7>!n~HuHw#GXYwuVxr_V@vISp=Y*R9~r<%Sutx_$b_jzKK@@c-#+Ig`xnEPaN zt;hPRq1!*&k7>c0D?y?_3PrUJG>{Hy0P{^_=WKns7GM`Q1UhE>Eeh+D4=Yi*T6(|>Ah7ucNu@hM<%}a2 zRYxN%T%z)A&Ep&$vjoBRpMF+j)xiXl?1A?hZnmp2w?(^9QitMsHY$z13AkSBAs^p<;YBNBn0I~ zJuM?IU!)GC9U!FY6(@xBy%$BUu1f5Gl95}!E$Dq5kZ*Br=lMa0U^jD?P#;)_taBJx zN9a#-hOT|LZyjW|BMQIjMO(`cmQ-0qaj^9M7`x^iC9>yuA+~ab<{&vE>Ul7jwg9g} zXt_uEz>|=UAE$m9C_}D*F|<_d!CPQr6S3Fv118gJvxJtLoTR?0Rdi49ZN?-x=jS^k zKP+Rhd1xQ^p_3HPYTKGtA9@QsjF$ zAvLT7y{x3}&rB}tOgvcjfCBf7;vGnK03D+WU7V9HshrNg@_53 zA1x6_!iuKUYn7givqDA!CMEdKRb+UiIm`%k8IF0`#=L^6yhWsa)# zKqOKphbc54nC(|^F{7Ub?Ap5y;rfM7Ur`s(u4IHga7%e5sUXzR#gLb3-k3_^QXHEd8d51uXIXva@jtA$Xo7WqD5bo}t`x8!w;kF0hGsh1Pqz?E8Pl(4Gdro=i-ep%A z=p;ambT-6}QXvm6hrnBqGC-|}sG=|&+TfHZSMjZmU@4L4ui!BV0xDCe(`!1xLx5V% z4ai?BxnUT>VCqAuLJ_>Cbw$(BeEeqe8Bld@+4Ca(Z0fk8cQfY0c>LyxB8e?MF-7>5 zF-F1U3lCxV4&p-SruSBB!RGHW5zfHOkU4HFGXR zE~_FYXGZ%{%NBkcro1-t?essavqisc2aPZ1{l%<`*Y=LeJ+Md zXyVFs6SB)o*uIU7li~tmr*NXH4+Ry@cdKv4h^o&`j5RmU_qeVpO%Vs1MTUzf) z3A{+`h`N$$F7fsOg!IH6of1Be&1rDf#5_}7A4(I`m!tYJtz@c~66}Td>t-2<<>VIZ zzeQK>vz<28xDRB6_SL2g95>;ap^XhLtzCQ>#TD!t(l}qp zC^6uy$wb@-@GTwO$a-pLjh1S<+)S0j)OkS>1=gi`mt!YUg2&+mkUNj>nI7|FLdz0D zHFy|}3Kop_pip>^zs)c%!+)q(NB9mq`XtA|$ea!Wvz%z8e+5ZXBuEz4q-1H6n-m!; zB&B=RU(;7`P$mnu;MC&C$#7iW(NjS`V6!FEr(bC24-wNSZfawg%0>%8aBBFHh|zFb zZj2|6`1FPn$f&c_j`XP0TK~X*L~j|xm}D(7f*i}FFv3f6&{%Yv$ejLbeb!)jcc>uP z&=PtnN+=JfJqs%Yrl|qoBoiv0vv|g+XUAhYo20c(WfX8EH9O&OorI!SG^etF(MXeG zlm=UBH5oI$QuAoZk}qJEnt^|eOCfQd$=zag@x96Y)BuBdIkb{7im@Kb&lJ(#<{LsP zWB*&Ux5Anl(z6e5x3ZoVd9*KukMeBM-YsUQ+6-?dGS1?XruLF4I1F-z@vjE+9uq2p z`AJiPYe%tO^Z^IK>FJ_^GgZn-ldMurm!YKmN$AOx$Q%;Qie>^UIQEfzmpwQF&In zXkoXp>x|`j?TlHJg&_^3C%RN~Vsynz7&=6OVx&lf5%uc86Qp8XuClDR=w%#>NI%PQ zEuby3iH=S()eZ>y9?6+eR1Pzs&2e1N#tg$Gt6-KflU_6|dSbH@ zsX?aJVDd&LBBP^S;xLPn?WmsNlqBAQtm?Ejw>wndloN*|=|n;k?-(i9Hsp?f645^ z1OmJYF&wHYJPxK|d2;A6^2|Db;U*E`QucU0U>SFV<}^a`{)v+X31ZD=pWO<{WzVKB zKF#>NIuqxhTR`BmnS#r%hhk;i30NeqR8wj#?1N)!L!kJwUtEw=dMyg<^o3QiT2#sH zIn^P+;bv&$mz=c~ylFNrJ#Q@*L(x?qgEFBN0ZdrKl=h2X4l`n99{i$9d=wFICOph$ zW1a5eS1EW*b7k5#81+_!73-k@yVF7Hp$utd(zS{!Gk{YBzY5)%AGz6oab3*Z_$yI+ zL?|h(7`-c9yjgjSH@nzVDs<0qgxZ!QM^*Et$U$OeOZ;!MjDuBI>#e_*yd#w9 z$T{c`$DxD*frGTQb8^sRJQtfOu(Z)=G*(|R`XH-#^*lQHUJ_pMy5G$EX^6mPe9Kj> z)@s>GUqWi%g1?$5ETrjZ^#Rulo@&OR9{W~Dj)t5|@sbp_7C9zF2G{hGiCqMJf}jQ+ zf+ItT6nV-#FeGz4B_yH#6bVwDVxHBZ4V7aM*V*!{UL9u%o#Xs8 zE*u9M2*DKd%lWS{@Iz_S!Q}?hI009Z+1xlYP(O;@w?}u|YX9G>r(5?6f2sawEP*Y< zQPn+bUsBD)n18>zgZ;e*PY^bljK?qb=Ec!UnuMq?PIpu~&V~v~?d&pgEiN+STs=() zJP%lMp!pc;hKjb6{O%ca%i%sA`1@7beW0QJH*>L?g&}o5*WF*dTzrgWsvIwUOs0W# zGu}GZ#DqZx+?ZJv*^%R%n|jKTC?$7`X7G6$J2;?l1F84q^~oF!$Y?KP!a(X2xM zuy#juGX3YCLvxsUh+DR;`HCVJ(wq%fpY_+4v7Xtxb?>I^CE7np)EU{A@p{^va9iXh zO4UgQ=0wwO9XyLxx0NtSh$8cjb>w{1LDS9gNtODk)`YVCWTY?z>1Dd!$Az;P+$Hz< zoy$xeN-#r}z_B5;FcB0FG(y=h%MzPR$Q9tG1)16HCLx5Z`N+j(sIJrR z!s~t=_itC?1F3 z^cc=#3u@Z?HswLBFIe-i0ay>dCy~Mfb)d3O$Y^qiNXm0ASMDtCG8cE) zxBE4p?Oc0Vk@>I{{w#xkqcWB{l~hTv8xjmt^;pyt94^X=Np+7afQ0R*x9dk=O11XU z*B1(5mL|Hd*`xS0<6D*C_|DMw#&?IDhH+9R?(Wo22k^8564*eyBP8fT$dfqlTqwb- z14!u$#{^L~xyd@7EAga$#5ok+MY{pt?aC{NHaH=T)E=NHG*RECyB_O%U7{o8P^UsD z&W53@(Ju!NbKS^)=_q-yd_TI1>Vs+5o~Vj8*eJ_B*l`u6bPFkA3Vnd;Y)Sa3Ab>Oio@vTzZ3@(hLq~ z4GuHp6W(+rB|fRRm-uf!+_7@=2e2yi)9w8|=aFLiWZb?^HCSaDs$my1#J;&o&5qB$7z5iLclbPwN!eyoOl%2Ah=Mk1E zCucunyS0N*Ql2!LP7+$`xXr-*moai#fsr=mULLxns&`bk*)Txxd=7a*R%gYJz!QD6 z<@gewY#(S6yzn)<;g3mZ$&f{Fob4`K8)ngo)x$K!?alM;F&?d$7QEyD*>mYud6_*( zDSLLFZFhmbOdmX8YGURE4uI*!FtshDSg{;SHMD@aOYNOQLm&-ba);}1^2FEwBYFo` zopp%+!TgDrJKanz8UB(dH%du|ee|Aaq zxTVY9yzy60OMvT(V4%=n%SQ6*)7NwcwErjOR%hLcCg6dgnX?DouWGng-+Ta=Wz?GyDQJm6;HzIuG@R~b~A$&wy;+SAGe>hf34JRIV3hj z&R*{^tWUuY2sNy%!Gy05wklebHj^LhUr}SjBj8qs>|t+fYb19g4^d?zq_b!UElJ3N zxm$9LrXx8mw&1J`psIvbmR+up>GwZ)2dg}O7Zl!)pAiKoIaXP{xOrh%*hjn;J$*|u zs$AObb&9xG{J;NrUnfv36*R^RCbIft*NVs{kB zHL^sJJp@-f`;?xjnsCjkA?kZdNH!N$WEHoO zO*S2sn$H?65Vv44F{v^zE6%^9zJQnWDLO7;_wNvc^{(F31_3G z9}qED^vwxR2P5K73786&k*+9=GzQb=#=d*p%5$F(&*ScoIWy|Q@ecFmEWAr{uFGO@ zonC@b^x1Kp?l#Go-2jl1?G+%YtJ21zZQq}?G|`NZGaO{ocUoVS*CxPURs;)ATg@dkerZKsN)5w=(TITHE7FpZF_aMlCJ z;oE!n8CUa3lHrTt+^Z9!tW~0{6M$|M`%}dj17KxPGdbd|?i)%?Oqt&%C?@io1QeLODt`cWz?>P>gJd zuJ@#TLxvo~I})v_l`JZ+i{+X!q2N-%wwZsLosThK4X2Wc$MX}Ne9Labx9?M~_xsPg zk99YR1x>97jxeG~7AKP46ndZ?Kvj?4st43Y?ANZxBr}G&PH1u!Ti!#M=O)#Ul&~ad za;a-qVgFEfoPN>J#BTWEVPS2+ef0<1B7?kk^G_4`_jenB_6rS1?P8S=)%w&8>{D?{ zMnzHgp7r@khedg-pS;Hq+C@eg)&^r<^w3g~>C5nj27nTQ@FXGuP0z-zekBbYP>5Fss#Q%9TUPnnmFtUD zpfIuMmI}WNw`PkbjjCn~w3OxR4}*`bkL^kBU@$+v@tYB^mo1O!wjK2ww};V;muPjxa2*}S7!>`_?es0tDEto zkS|x@nB)VG*KRnzTSE%mhlAgY{!4c30Nl}SVd7&dob1~z5nuK&6YuA0j0yLJDE!OY zZ6@Bx0hz-1L;NQQ$EOnFx0DSx6^i#&m1#@Tp2Osah<5*<*IbAPNmHhXs9ON3ni zIcU<15RvhKplvIQzMls0+sq>p`d&Jh+W2|s(UDaA_&hq|5r7AI&d@9G+HpSeiQ@d) zk=wk{*r1TNStIG0xNcmNEv@9mscXzc_`JIiG?L|o{`)H zuHU>8cbZMiRV376G}ricJlG;z>}m(Wwq$o2j>+U|X4o}-yyg=QLXjx96x4Ym(ax2y zZTpTC1h5v2kQ%8v=cBd=R2|vfb*VQ|M zqxc~v1;e)FcCe@f@pi?8xeVD;W7@roJ)tcgOodr>F9B&1xtgWE(xSaoh> zda@*O;#8=zX{q1vEn#rEoa4r3Ka8aDH8NZ(1rrT?xrjZG-NxV{65>?R0G0 zwrwXXwr$(CZL6b>&F*OO&Q~+@&$spNt<+s-BHi*K^63GGF zcOOkOxkaKucDFauYb)&S-4_anod^$`av1hKNKsA0Q%Gabg!O-`VAh@)QkjC+wMiY? zqzzIi7qOb@SS6JUC2)uYduM7oP7h=l*ho0u<84)vDph(!q(W{vHR}!;onCO#sJu#0U+?0 z`$q6(Q)>bTkb#=?m|AYGm+0mD@`+!d+xh!*uTj9h1N&;N&`~}*V==FoV$yEOp}jjq zufIYBpB4N1i646glGKWvnAWd|ZvSpefnNu3iENgj8sg+ctowp>@@STWG?BiABB=a| zlPY<3ZkFw6!PxYerr4YFmCyn;)RR;MG-dkUjX*`k4*rZ~_lfz(2tZ~z5(@xIxypZ$ z{RA49V5h3PS(BQu@CiY+?= zArLioatt(In(hJLfM2$}EN0Sh&-shtBVVRPRNN9!ZP!M(Hrnpe6)R>|kkw3Qwg(Ad z3a&PEmc76kU0$Ll0!S-LJ1lXc>vmPw;xQYTG(>NjO6`+u&QuC$COf%soY>ej$-6Bx z37qkGR-E|)S3F(Io^D69ld56fO)Bg;{bdZN?OMd5%aNtGY}1lVB|naED;w5Oq0OQf zoqRU%+DtHE@!8!$v+6BM4>6eR`}n)6%`v+dSWLlsT&NZM`Ki%kDd0BdA14`-Ws~-) zV(k>^d>y#=#~D~xFuA%SlE^z`o%jN_WCh$pIUq%+E~ z=keA@lT}n%<3>{3q8KtEu4FUP0OttiUyU~R_FwFLGDR%^q)z(~{#p+}&$4icJgQWh z%2YVMfq+QQ2etf=QVdgWKmffsjw9xb`Lcmu?w;|xcRP9bxc_sq#9-Z~WiDQjYeW5- zEe*)wj5^%#fxVm9!W;?wKys&Sw2LLPazt-+TBZff4z~4aR6FF&@}#TaOXOzs$1ggB68|N1ofdDzZ|U~yGkrC0 zsWb1fXEOEUXtTm*Q69R6QmM&L>Td98Te>oXGHIsB_6)B$RYe-H!0e6E?57xGnHPQ7 z=p`v{e(QYH$9Ek=lzt8t&6V`t@gC$S|%0RJu#Sskb}+m-p-ZO~nQZ!?6vVvv$5+-z6p-@pmvr z6KBkClWWAp4!<^PDfl-~TAe2qj`;>m^c)bTXvh|N?xs)Ppx=K=!|K~7&emsp(mOV@ zC=l5(JQ4FPIjlg#=DT|=&FgE%FXC>dm>$5UHGHWKARMW?#zt(Lebh1|Hvur zjA~!I(GKEOL%1{b0P9iQlbi)S5#znn-M66XVihOb>F1r`jz)uc0FF3on~dMcTCL!EZI~J>&WIStjjLx)LS-89N-B@u zpWH^=9SkRUrF8Y1*=JmX3dFAABh!Q4XMF znVZ&`csO%jto`}(|3A6I7TDrRd>hgOfd9IOMcKv4)X?^SxUm0_I%wtqKl3_Cb}&*Z zaG?T-VkCCxsU7TECvY4I3EDIRKHv-zZVt7t}VLA6|_|aLm*v3InbB_0c!9_jy&2{w;z zM8-#s*A6{yYxvMjfFSRs2wY#Gp$E;nURb90bWC-fs#G6kiQ$b2PhZh7j6m3-jMFI% zYezT8d-%DBY@ICqihe}>t_8(MxBCVa&3pJzufe^~!$%m`_t%IX)5gh|pO5lLATzFr zmS`Hm;avd1VMvHD=j9l(OH_}2g~xo{kJw5NiRljIFk|Z8D{&-5ShYRmqC;wY=BHb$A*u9+)5_8jnB)=|NV-T-R7t)}otiLZAm($PbRc%5D6ve=ixyqL}CSXwbksfSsd%Woq-tEpGBlGJE!X(L;3V1TSyipHMQjl1<~FsU%q zS;i`6Ewl4Z*R)=VI5QzZn;=+AR#8P&e1n)U0fst55h$T^t;jdRGGt4Vah#dBVRpN_(w%4qiRnA`B7p zP@OGcC=_j5E=$zIrG(}KT|t3;i|yQ|T3lT;EFPC=({E0-XaWDASh0m<5(gO}mw z6Un0PvsWZ_oO-SBhDAwq;^gAMRaQ3i`ayHy=wJof?8sZEdrh&Rza<8Kfv6*AOcf3r z^{BwOT!^6-Q>i|89eFQ|P4tFjbmr7_P~Tu?j!Tn*YGCMVEpV#g*AVXX3M1kJ8*i*N zD{tV9RhJu1*bB*{pr1K7?Li<(9=} z^_JGJHh=gJZ>iW2L%;M6n`Jamzfg0;8QTvc)vQEssLnAX|0@Q#2SbxP28*p`Kz1{I zFckr0(WpESh3^}Z?w)qO(=pKer-rP8-nVx9vwIpw*=DoywFoG%kB;uYdP}Xhbc^|} zI&kXupy+Yz3h8YyqN4C6J1`6aDj(lhuzxe?2z!T^dKIWIM4X8OwFT1V(ycoyIVG|j zO2t)1mF}tc(iYT~>n`qyQB@3vchMBDSpVm1@g<5?og`zbJb5SUV~6Pa)ToYqUTz4{gWXkL#oGCZpaFz4cNP?N>g$qn7rK-n8lmV zKKWM*DSLhWmMh%`BSe8Fu$}MQ{gq)ga$B-VlbW(OCWxHAuU3I zR!+T3MhZ{0sab%CgjhC048$NG_0j9w|uv=CU#VWaDl2 zOv1Vqi`IGTUB}3ccM$BUA*(F-ir#HB8*8zU)%23eWPYK(P$och&ppzIsHIs@=`}U& z!pJ-QNr#voD^w_)t6|XdgN&%HxIV7|c-n;S-b|}fhoK972_{1LgrzCMsB(%wL zYJMAisu|}Lxny>Scw>hwT9%I~GxV&%oW&X^zGHzR6kM8PX-#B@tgH#fgU8 zESAQ>wonaMnmj#nrcR4pODR* z%k+0EWG2>Jh6*Bm?>nN>g%!?9y;xq|9e}u{DLmqm&R4@7uy4do%u^$0Y=VvX2h7ks zZhm?Wlqfo!q;*y>VXBAX=7^6^uCtM%C>*^%k3#>_7vc_cY~|T-ZKPPai;B)ChG=8m7^|)O6MS^J3-OSS9(U-KmELkA z<{VN!mY3svl#|;{D42rfw$uDC@UPWUx(8-uCEt)wn1VA_l8#2xOxfkTK{R8q!4(!ccLhSYw)3 zZ=_9|`-(d@z}IEnPst^rG)pBc-6^cY8T{~$(9jp{$pdQXma)J$TlE~-iA$2dY~oC@ zdce3HrVMI-40?qEs+uC<2y^7;En3pP<@W509B(ksYe9XO@0Vk*{9V;=QC^DRmcSqQ z6M3QFPdbOuyzTf!;}jzSJ24Tn;epOUS|f4Eaymoxv-!2!oG4EZel{a5&2$qlZ=ClE zQJlMT327t-a4$BM7mX?e(@MGwG&^DGn{I=)M>@re5)=+IEjv)PPwicdBM8500`jgV z1{$6C<50T$Lj;yckj1&N)9N*RLNY0pP(c*EO>x9IB(V0z7i4LP?WjIjpHM;z6Ns`q zgB8ert?9JuE>A1+FN1>P&V9cO!#X*jd$>M^ig4YF!4{L3y}qQrj3S2~lrb5xF1 zFCVQ^d1j;jAL={Z^nW1sQpQHDPnl3l5rgC)d#FYxI2ji`hw%~W6fn;3 zA+D|0QEip&>F@!e*`kXrI^j;KVO8*kTUts_ps{n)lE2!YTdamDc_rmvsjBa|we-Aa z(U@{C8CHhgb@XII)*7TKe*9kvxXb9Txs`9f76R_SX1bMKja-}z{~sQtMD5}~^?+aS zAvTTlLJLMC;uSw_QShZp&>=+N5u-|f!Hd<;k!5^4-RIXpA%fN3J^i-=e^D;c+hu33 ze;xiR@;8@2Ed0>n!HMbF9B=;@wwKxWk2wN=P!DKv)R9}n6b?4h@wUduwsmFnvF>XEnn|XY~S~Q%6^!eQfi0zh8 zz+&7b`mNc%qo_ap-uUgi#>_d zZwQ*c+@ZKG#MNwDBBP1nJ_h$q+Kw1B+o9{K{s-&Um~b8|Lp_viADeHk_U@d^UroHo z$Tn!*Aq=z0C@8}xf<{{pFUd$=EIJzFv(@~@x<%P8x?JAJJCAn%gLQ9ItnPg`mD}f5 z{skD3coz9oU7FvF@ z-7693-)#4(k#>t{i%b4x@qgHEm=rMCu@eDA;f*qCy@_q=hR=UdtvlKGmG?YmS#lF6 z5HNprtvH_s6sJlt;x+FF#g4s1oF0!=Hz*@8zNzm8Wg5Y>&QNoBnj0ZKVFrLD-bq(vu~-cJxS@>oPJmvf%1Z`W`R#g_?;i=)!FK<;Nn1J)3_{?I zlVdClUjd94TF$qLu%7p_l3TEaihTpPm0jkjHgpDZi8rG7v#E0zB=PiDzfc+vfp#)=I$c>PO`cnFnMX^7s&y?g9 zJq{{un{NrGm`R$6Pt+RmDm*?+5~dxtS`eZ?V_K*}Bo&0^87|iFFr1692;MD2-z~Jz z@e9urX{kM&7FcPM&9KEO(FdhSzZmD`zOkv8DOz5*Q+#qe`$LZiu?|w>ib$O-KoQP3 zB)(OFVaY!Eb?Ao~)wYoPJ(Cs&`@-h)H00*^E6D|?l)mkpTIv-JXKUbz8>DCVWS~iT z^AmQ4Z;Q?IapT6F%8Nc{Z(wW--jA}%>R9#a*hB2at?F3A=;TnfJZRF17ZT1pC<_0v ziWb5)c?BM5d*XRkNhVCi!uuikNx}F31H=8Hk-}%aTB;L@^5cgH&3}F6rfO$tZ}&g% z?KN7kKFKR7UwPAqSu*^D;E-6J@Zm`JBD=t_UxY~GV4fpC_<={#SlQFjET~W_w$=?> zwL90Hbr?0aEma#~U81lN66zZ6G+kQ$p$9qfF0Xl1rrisz~whu}%srL0JN%eKf901}I&N2dk8` z9jewXQmC>fs9-t`5DV9;@M6%cq?RXqe@mI0R(%FWSgkUn>RpMX3Vei$1^_MoqA{UD ziod1Xmmt#yGrBTrT;al*x{Dg?rj5V;g$|{&;88}nj3n=-7KXat8m#83-CY&6ypnwu zJ7el;oJ>dra&}xn zJ6HL*L++dfCnV?Fpl!0=#4d1er)W>0tHjv;WbKy?`dhRWo?pKtwKj&bV`-(X)LFl; z@aoXs$%7Z6#v!6)DO%662!Ji?O4=6PVGLR+QllqvThS7?8r6@Qcm&HRW_4`o0m8x2 zWHjRuK7mWAS1JSaE=Kv>)NW)bpb@{jPf^jB+~_PnJqlyRs4Am07M`OU&3tLi$gGR@ zRVELPC|d<29z40GlbsSggC@AhYvj#G-VP$7=S7X%wZ!Lp>|5GAb z4KM}z_C2*mDqb*7RZIiYb+q*J7#ghWsxSw6g@QM0&C6yQsyQr6ECdIhisFbe&Ad4i zr>!6Mg{@mQ)lU>1ZEz@*x{On=XsdKvmRxZAgR;|TZ?lq>qZpwsaf zom90|INKBCm^1akbGS3XUPWIRiN92_U6tHR&OL8gh?h!bjm&anrMx=frOfUC{&Su@ zotrhz_B2*xtEO1%A4%t2uS&=PUcojYZsf4gS+H6;3qTIH8ahLX1_yq_L@LKdjX_(c zVvG;?A7Ym^eUVa@+>gNDK#C<6r*S9Q`j$|7vSCywH*?d&y_pX!s!SM<_h&&nOA%|M zJ$2NWo{>{ihmE!E(tEk!-Q-7P)yH^ow-Usfk?-P0w z>ZLnn`794}+>%DEEv(IfgY=q=kIDRP-=5X@>~(WIb^gs|Z$Q$SY!_MPzS#y^#TPcc z)S~;dDzd>x)zE8=&C`yl+a`oJsYc~tSr@hIT=^NKD#pFxx6;NQjvOf#M#SXe{#w>P z2R?;z{Kk<_n{nx&%2B%STRG-5Weq_@C<9(w7&v@R30m(Br9sg_9tPTR#n-hY#{ z7C0<nc4tm)>O1;S1{DuK7%B>xZSkH8NxOu-G9>VgpZ^dacLTf)c2I7wo~b8y{Ey>((ca9y(9*aSr324W?vH;Z#<~7dSyg zaDJ#_bGT6wq4$NT!3ku2qcXIxz;ip?N_xRB@7*DQrdau4H*FbFU8b98bJrjKD;nf? z6;8UcpN$<+Qf-n6AVR4!zAtdQ;CbX&jTt)DndCU35ku&A?KU1w0?$m-TLSTvtbKYi zKN|>o_|OaD(g?-)L9YY6aNq;yC@0WyAbCHxDtznixaCT4?jEn_8*jt=-gaH0rKvj& zOYV>>`fVtSXUL;BM;C}`N;#;6d!RjDIwl)_PT(rzLL_Eai76gLBf0|c!C$t9ebh1B zplYA|_nSY%^m+xAhE;iq+$p`3RW{SOTVy>NCKov1DUle>>n=??RJV`pF*j&&6;@YV zkNA1dLW%M?a5>b@f~EfhK?aF6vG+GPd2)(l)<_vs17@3QwqV3>UUD|cc#tJ47n+EscKsCNBht3qjz7<~Uy1CNTdye$$Uc%RropLes2$RV)Jj#q{1&wCGR*pYwm}*XA*_Fk&d5i1pjQeGlnzN zX-CX)EQx%I`9R67${|80gMlgJa~kS1Ztj@)1bSsgT_%K$2PCsY?&Jx6hNARw)h6jlBeZxt_0}aA)WK#hb4q&zilb}Mu8J(^N^MVFn)Kr%38+cjS`Jl&QwMTReD#KiJ(70t0!A1DM}D@K{SN zEvZfA_e0{JYXFg0-@gHWb3eqVkh?jyLMO)L`cq7TK@&!6I3(Qph?ZKhcD%8M?yDim zf(YN%!?o|0P}DT9^}_8${i@;&aUBM%_MtR46?7x3bCWC9pDNqTXv^-@Ws4Vn_j)wk zyg;5A*zM!6W?=Aj)}Kvi&-2$W*s=Ac#`Dz~;S)yZ&MAtYGCWb+!^RC|cMyxXPkSo* z!>h)YoQGSu%$51&6RYGG78PgTa|uuRbn^ZNJ4#cdBFIFw%;g(Vs^=Rt%Nfc>q57^f zQ&VS4Sn7hLTQoMsV6feUbrmOUCN$YSouhSya)(Xx%_z%pH@UMismzHMG#&>#Cc zIAruO5{8ghZUo5V)F?d12AK1H?*zY5MR?Z?m79GA&Ex(hRhj#KyjPncmbz$lC0+qSK)^d4raxT`y z9~NvToCA5$*Jd_QLXmFACu9`dYjQNN+ugNBG+CRO|FPu?DuZancAbW#*_`anBrn6d zk4VDXo8ER{yWBLw{ewr{wpTR_s89-+8u_j#uSKh4mjI0XJOgdvQ?0UfYgLz6St-?c zyuZv_*qXAL{LjyNM;p80a!)nXvB+*wokL z)X*Lrc=iOyV4IoR9J=1>f5fpQds|LEMPu{bp1$Wo8@k_nJfCv9P<+PrfYKVf7Rg)< ziq(1s@wKtHQ0Lm(`ODj}ubk1Y7#Cw_LVgB**xx2wK45l_m?OV<*w+3{sbDYwh22lQ zGIx*#(Lv-|{p!?CLF#BZzLc@mR6jIpX=p<1+V0Xo?Ofd)IJtXciU>5HfcA)x zzys)&74O831=)?6&Y6rXdrXC41aiiF}WVH?Ii}kI!gbP-B0;B1j z@q=T~&jb(6Bb5ev3>alLp&j1Orl;z-Sq;mz_UJ*ucT{uVgZ&kmr}y@8s}CyTmL-z%NzRlWp6Q9*{?~?m)tNYwHCPN$?729|gOkd00C(Tn7^TaP2 z)(Ka&SjeaSPlcxhD6WehhKAFv2Zx4(pCxqCE4XHzFJS3BeXFN^YxLVVs#lO|uf-x*q&!R1!Cqo6k+6{ zrOI>NS^j;EhNH;wkV_><)aAVyh4pBF%A*@R$$ss|nYx3>_$pNJnTTOHq>=MkiCKBg zgww~ow;$hRLVl73^d2AF+A^K<*s&gV4Sw=b^xlk0zXqZ*Oid!9OC5Gvz6#@Y0yqj2 zY-=vim&HO8|5%Xb-io8(eJaH11uP3>1<`CTqMmJHFLj|+ZWfx&pTzy`ay#!$M}9O- zU8PjP+1D69Wk$g~-|4In?|kh4lWzA9RZ-b+L_&<{IkZr)Xo|q<;pMEfj_xU8zhHQ< zMQm82TDiT_ZSYcC=yUwLJ%p+Dm3eV)?M+P4^kuQRSa@Io7ycAxX)8Lsw}N83A`9~` zk8n1XeUy@XSRUa$58s)@CF=pq6KcKl13ULG7soIJCMm1D7Zdc2v7+bqFa z6$Hee#90UtPb3&3ES4_JCZp?-wmy@GZw~w!2fF{mmxtxD>^NyQ2{(RNyv}}9AGps zj0l4)kp{u@p^ZyW;XjwOh!J;g;NFiNa?5IQSfWQc4NLyQO3;9Xko{46H@t4F2%}iz zQ}hd>%r3Jc$QW@fSjptxz%(k+X(jnxevjPbhoVxkbCcj#)B8Bxj=VFEW2*MY`tLL{IvtMZ~j*LUCMA&A(JDsv`^kicir!5*q-$r zqtE%^V6wovYH|1I4GGK>OMex!SRaKaNm$jFf0gX-s%F3dXB~%>Gl&ms6%=1i@8bv8&HZbHAV#-U) zE@jj!DjC$~<#tR(yud*=b)1MCHnmNT<=c?D2)8w;#wU=TtQ=XwpIjKfzbYC>qMg1H zezCib900Jr=B48*GvrhYDx}z=a`8xB*x0mHEi1KV7M69`YWkQS#W&Tai>Nc#qLZ=MP@$Jvo`4y*7q*Dm$2<& zmP02SW!Moiq7}HSxkr!2$3^B5gL3{pNxNC`dP838AAtcvqF1Lzbnub|0vN_nP1<#M zzLT+bN{i@{+`Wu=zh2~rWiTxF?^j6-@c+t$?v7i6V4)sTCKESj*_HrnP*LCv&f@sE1zhnnZ2VD z0ckY5CI~WwileFQp<4qNEPAkr7mPFyA_=a+E)65=2PdvTjl8rl3#&D3$-gT1OKIH$ zGk8Q~zvW+jFSv@sSxI{7ayv2@;jFSqA$UEfEp0nl<+)wLyYvzDhzGaabvN+!% z+WeEmcEN5Z0!r1%)nc3wVXR$>#yn&z{gId;k%N9G^IHPRWjukYLZ2qY^r#^B4gw<| zne4}9`6y&_05U@#{(j86v9VQ8lk{;@h;>$d#vz8^RUp{C_L5Cp+bsK!W5qDThh+lb zj`BVy+?X=P2?Q<;T_?-$VIvET2}>mTQ?&yu%>IF{bhs;my(O=q%d2tLmj?9 zM6Y&F6cpU&z0v3k_5IQ8Lc18_7)D@%=dp2JRbvH{aYtBj=&${+jyr=>Sw`ajhTx*z z9H;|iEApY6pl)G=wj~@`5Htp4AuSSvam$3n+aTPWHw`eck%r|mwZ)`@?b$D->}guu zOx@^)JL(}sLYx(c7Zp}Y99y{%qJR8t#ErMrjIRsI_|qNKI=)$Xm@daFH8qt;S=MHT zJ1EneqHZEUla#K?&+}v{Uq~Hso`8Qft=LLk70nUIyU%D&>da`*S??>?G2feRfoktc zlW}sb{zh6XH(1Pa(&8ykEP*+z-j0|y75zae*RbHvCP|AmJV+|Xr+vz?LeX$aT6xUG z5itUfAfHn72_?r9z4WGd#TC1GRp~PFBh?`Yu8c;>i+8%M%=laF&l!8xoCEciWx3~#O1T}UUw&*d18MVMAFnJTx{B>t+++=!>< z18m@yR<^MAW=9o3Ms!GO7F9#ay3Wv|H{{HNr-r^b26O)2J@G|tQ6r+p=g1557`-xP zqYW=T!)r+I_=x{SX%~JnRm$z!gkU%LVq9CsGAJOp>%7IvF%aB`8&FKSY+?&zY8ukW zTzhg3Uz`ZjoBbVRE4k2bP4xm;oJ3cbgjG;*iP7~$tXEYpXZ>}DbzYAm;|GP#hHA`p9?+wlWnXE@LpncHP@y7faHZJ?k zum_3A6zX+CX>n4ULPLm=645#(>-SGBb8LV!{bpu7T{tMxwly`PS0#&E+ElGAMyi7) z8W$T@4AwnyUe@+EMOyED85 z9}&EW-xBts4G6De4y4_sK=nq}iQc;UkOYO@Nr3MSx4n)PAxn733?8E78bH;}yU?{I z8a`}?7?vJLiAlOu-9@KHTJvSN@Y)tP7bnV5epCYF*K^=4MeVU1U4VWO4|#2rXIGUQ z3SAjQ%N8ybaK@=mLZ?i)Rwc1n6!{E`Dd)1~b*@o=# zXzJ69eXN+{Z+NLWa804xhUuKke?Daft}jG>(1cuB&y|-B`qIo4rI;L?BG8r4baYFq z;k^^coJ%?cmC1@Snb@rH7gcYixF+9}&h_2gIK9 zyB5VVBzX!pv>cRq5`V{6AcHW>pyNtJ`7t|;a3)6u*ut_3HN`T4L9pjuK+fti8?q?` zNtLp!+Ck1v4Jt3U#sg+FIg`?yrjrdQP=v@^@MX@_Nm%poJ{27pHOEB!jb~9U8Cyh| z)v^V8l0ikDTdpp0nbRQJw`>M^;R-I~yP7P9=9_vdGg`D_4S^Do(N)e?iM3GVRrc1B zQYo@!HCAq9uO&Uh&Z}8LTRB09Mgdc=#Ds4eqe_>qp)h0xRhliZQ>ys{3P@YPFGf*N zWLqAVlQ+#7b`6)TU`sPeD9=K)WfvM7hDA~oaygwDz}v#2206}}m$o1>FjWkq7-=!|H8&qRWOknmc&9i%AW9-5 zF}}3uvX&l;YpSYgC^Y`nrr#E)i?LhNNScY7k0v7P>aeGevadLbOs?Y%jFb>kbBoh8 zQ!LmuG-x*{3tG5A!Y*|6YFsU49TW21p_mTKfeFi4chc4s)0u*v1R#?FW~mpN3P;H( zh$%AzMA?Ko)B~ba6|)j(?n7b5Q}RRABu38Zy)lvSX|24)ds)n><@TXAuBE~&>t|sz zT>9E!)P-CSwX2@2be~s=n;p3hMT_>Vl97D*H8`D@RUBW?r?P{Zm-ndnA3Wa=6h^euZ>7qYji4?7=ISqAUo>Yw z_9eNT#JQupGgQuGMC>h8uXm1p{HTb&*D2`ssQ`NeOU%7dHn&)qI2EK+)I0oE|sNTj3c>ssnNPS9O%B3 zJDzvF(IyYM!3So&{v{N>gU~*GGA#?-2eew;TZflMny#SBjfH;zRxI zGyg+X026M}Fh(DOSvnc>#pCJ?&otmb4~)4vF1vj7RH$CM3j_0Ryu;QAD@nDbwtp?E08&XpXE(8Pm!clWFtLk_Vym(!%XrQGxA=hliv@zUcoKe7zgS;x)b zX^sFtTY?P#R?o=ass8Ol352Z03{WsC+IM?|WM7c&K>`j=50}w#v|*}NpDT>+DC2l4 zU~uh$$YO1W!^`N>>>1}l8Nx*PN@>L-k0+~N$e5AaeHEKFru{-SSEpF@9P&*%)@Y(j z(L$1<44w7qGiYmnnMmxzOTH(W4kVMxL#2yX>-4P`O;ao3Cu`3q?rbUe}vRSa^vEqAc(WQ#hPG$dYpCqQV_{@{E8`^kf7f zfbZ9gQ?^_lGWHKz5v(Z29IdjgAx1D3y66zibolSRlYU%mzO>E8x8l@XkqTX)XE7s? zTYO*X0$`(Rnp#;;*1V|1vhcZ}DvS!`^rg>t3-86$suvZkf8wtn(KraCgAdZZ1mi(9 z_cWLaGT8Dlm{Pn6WLsxsk1-O6^jkQA&#zA5pF|`Yp8sAfLx*$!?PC=Jzz_M$NAg+JA`cXPy7?F8A>6!aumy|`yf zb)b{npIB$?31?-R4bWw6pp0+HF937&J27FgD;mO%M{CA^PJxR515h{XvQ&cjCkc_=-e?kbcX+-j4y7b2Q5Jf$k{$BUxfH%fS!xO9z^D)3_m;l?l?5B5UkE7yEMY~*6 zL_xkEiWE(j2(d7{$lKHwxy4aMAKbIgrr%c@QE8>xuY*9K{f+0_VSXXH3mjkgb zHd>atkI{n}W$wBQWWUDv&s34LpQVrkl|BaAmdCgV7YISJ_kotpRES}=h(WSXfhCrF z1PDvP1}r`Xkl#(pc1}1J#^`z|rH+38LjgoN+wzfn{Clqok*5L+0R?O@^sxa`~a6X$rQ! zk6HWkbicNR>7n-@m8$p1s5rjf)KSD1SM&Ho8Xw%gAiZoLkDn=$O$y>Ab@vQd*JdEu zX;xfaDWzN|YRaRkFjJO}iTWL7J-I|Rz<8Z2*hQs_yYkLW=WxN%Hh$pbA67Ta&24`z zY*5SFpqr<`4P21x>)`HjU=I&e>j&rdgDq)TG`^%(qhS;2Z~UqcRWgB$1|GGD`+ zr~X|`aOuB;Zr%oM-~+91gLY5-cCbN!-uf-RdY&KyP5t#8{B>QydVc-&H@wfeQBXur(Tvn91(5gkuJUT*LQ>sLSf^3TtE-@ zUf%!LP9tcm;*{d|xJ!@y;|KeHHSYe8pCIpG>SXx;4!tev&>ncI06q@S8Et0St$6xm zr9>r0Psuke>|dvatO^p6i;Kx2vWEYdWy`u_I37CNiNd?uZwQI$T|sQ}B?Cd#AS$7* z32WPl+Y1N_3lIL9|B)qUNFslFn|?Xo&EWasd(ofs-wXcgxX}ACb$7$5^lB3{bb`VO z$V*#e7*C0J`W^ z3AD+Padv--_yZl#o7wN%uD^`QIF(p>cI%KAAgU}eyfMui$2hX&JO5d z^3nd$i#p^kdf?6EqrXa!uut@VarTbEokY>PcV?1IY}>YN+qUzM?PMmlZQHhO+qUgD z=e_6FJ?Gm!x2nJF?&_}HRl9brwf1^`55w*(xetCAH|ijq$ekF(2RdJ@k`LzKG_t9t z=TA`ehc-gR#9Ocy{Ul;i+%O$q2;o@i$)PLF% z#ng-qLoES19FOfxgeRVXu_*DdSvD~30s}*-8TqWe>8A@rS#%nv$m23E6-%3f9IG{_ z#>ru@Z38X#c&P1S$k3-fw&~Q@rZZ;-J4`;u$s&UIc`9GRXi1cYs3LV70~ibPE>sim z&_btfQu+*gb8X3TML6U+&7WhF76pdGEn}eUUuh=Ck5f??)v}$IcT>Lc>MM1oak|4l z1?7^WM94^i$N5*Qq~h>T|A+XLsZALFAe_4H+*Xo_`q-!spvLrBlcR_%MVD}ejSAaQ zbi`op&6Q%Nz?Ul;(F<$RN*=(N!HPV6SX(Atn67GxZhiW2`QqGgfMj%XrH09ov}dH? zUpI6z>3~pwUDX?Hqh?2|f*sJuA#dr*_ZO9a%5o311rszM5xAd zX(A^1C8cv?YZNZVHWQ4GmNW21U2LFyNR=Z)QoWovPHR3g%jTnSK)#oaH3iy@sFj(TpB)5aIrWA5Kr=SsCaZ1%KDyvKA7n1Cds(?z2Fc&3hf(n}d$Qqfhm4!cctw_ar*f=R z%#ER2Q+1^v%{9J{KNME?u;Uc^s<8Tn)g*3)gu?U+ix9Gh21yfa8H4Q?Rl3%A zQA3T9P~2BOhY{U1ou1KIB>#}F5Lc9XGMTJCelC5g1cP#ER+>_o3ub1~f}T8jBLzcM zE7@;RF?++_^D6`kS@wHdLayKCL!zIR!L~g^F#zRW%|cUTF_1ky^I`%Jp{kLR6BwbsNCKmUcD3dhx-c;_o{sc7oI;zzueZPud_ zrUmvDjz1t?;<7-b&oD zX->;2F48854ei4#6t*9^39vqgI-+#Q>VuF{eN`0%mm{UbYdV#1(8}6QJa^8VSe-WK zMH}IS9LG5%264f?w&{7ix}My-H+%$3rvT0v3}qoRr_tyr6alYe&89k&x^043(l1z^ zR6c5!iD3%%MD2UU*VNE?c!a$}N8y&`REh2>X;Wv@x_G8|o6J|MjQxqsEbi76Hyv1A zwrx-B*MJ4qk z^o}F`A+xa0{{kOq?QoWDcFrC{_V-gHwkP{#9-fQ6x`#Pz2XpfO#@p(|GbDiC2KXtJBrfZ%0k1# zi|p~@{M8aD-mEy@sUASUywB?4m3@6HQqq4oa@&CsA&Cn*J)N&$_o&>`w`?N~D@Iv= zWs~JR4Z@*4wD;Hx)nh^S%rC};Q8Wa!#8KNC{qGs+!JZ%k-9R(rx(zh8{XUFE_Zv*oB*AC=3kkeoO150&a@5YEz!!KTS04X zFk9%T6qSMqDv+^s#xePwGP$}6Ek?-I*T`J^A4Am_DT89g_dD~o4?wO=IA_?9&qeyfDj z1*Zta=USURQ8|^_(Ndzo7YpHOp($lz?H1rAc4@_e5j2OHZfMa_F_h!lTl*SH$BJ$n zCdCSHi%^A>s+kskMW_aCoDBbTj?>lD1SYQyXzu#()Daxq_;}&ta2rwF%hrVE_x)ew z)w7mVgB(%jjUGWxcU{B+;o^M-6|FT%mNW)5*?ujFwW`%r8R0qBu5ss=5hilf^6D!j z1|!67>*}J@-ZrSmY(y7=#h%64asT6t=1OKDuguqV@UKhl<;7!C#ycwi`WFTD@sF*h z{RvFaZhwLaJwXnnEhK%U60Ft>gbXH9C9MK8YlLk#-e?cgXrDIzi_na0(-m3KRe$i( z9M8dzEzE<1X|a>Z2Q%CLh-ZWo_V^I{Jyzi3zi1r)qb5rx90D8t6QoX~0|7Dq4}w&A zD}(>7;ZX;3)B5RmsHS-xy>qn6fG7Md%uK+ZgcB%}@hdd`pOQN~mQp&b?tz&W{wYM{ z4nH85hUsFKRcD5EpaH}}vBq9od?f+ZJeQ_0p0*7P$0`2n_*4ToR$yw`ORuVGisyOa zJHzw9Yi$>RW-mWS~$0QV-&GL-`-PjF- z2Jcm`3mvbooW~6X9Pj0znO*SAb;4miu9s~8ac4?6PTdB@xDM{c0zLj-Q{tYDS8hT( zUD7V@1!aF{OEt7!Uk>$ic->~-vk8Ycf4{#j^-#K}LJ*mKc8IrR|Gc1&Xulc@H&sD3;H3?%{3gT8UAG?! zGLeD_gCP&clo6$N&y1}~NgfMj46DM8(t2PXN0iy9E%sBbK)#LN;xUh!Vn8edxv=96_wO6=+4Ly`3 zueW3oYd0~bc2W9|&?;S}0#TP>@3LjlI^peB2?k^2k4(+@Kd5{pxpsz+aejl1#1jRv zO>xMlVWNtFHayPl6+;lFQ*mZ=!ay=dGOaEa2Gh?riViBXDp!`n>FgnlS>s^mN_!%h&}ak_qCr(s8rg%+Liq&!->{RO>8|+@q76+z z-I^ESMZ$CKU3^xgbHZ|x{cOx$N(AzmY63~(!JhsF^dMAi1A9B`=UQfkWM$N+wQ&qY zKag&IV*lnrX`3cR=%O=4yWo7)jCT{Jo={*e6jk=GoF0^@@V7yv3QKsZv`Qx}(%70f zKs^^4i$vI94Vnll#YF1A+94-deS7Lfs@D9OvyMLqG`Be^JPKB&mv2D@DDg43LT4iN zp_izx8j6&`s9Ki(D3NIfVtzy9N^+8vUKxd>;x|@CTT);e$f=`yaDrpKrSiXfhbt>d&e(9B`6&7~4QbA5NyqQ!M&FoCw?ucq~5;?k8SP0F%Q zWp)t{#nHIn5i@sMtCC`{vI!r+lw`mM7aE2|!LSPrhr3DiH#>P@Z7Za(+3cqwZ>yDt zKaLh8DyuJSnY|4%{>_fP8?Gogm3zL$lA4=_etaR}BD^=j*&d{_(qd}V5JO6H_JX^F zWC>(*n0@ue;p>4bbu`$V_yWAdM&a$NEX@mc8%660D%06^t!6LXAmQ*0PCI>IchB5n zPU*{?H-+rBVLbZO-X(MTkUaO+r9xP-d}t2a&Q$L=pDf>S;dBquqqp1si<-AacHmhsM7G9{d~iRWkEyJ(R{ z`D!@44jpL!bWwY{*SXh%=#Kid!jdc{)Ik~FQ?!vuJdx}lhYXH}+Dr&7CSq{(gTcC? zbf$_fl5{>)G4Tq7^rUlBISmvX0%#4Rs&}+es?{S$d5t#_{81J^d7Z;ik{754mK|S4 zkPsP3a+EW8S`nZ7u_n}Ct8-XS ztk*DSnGSct%u7o$oewLY$c{Kijm=Ml_c3urQ(_uzM+_CS??)&(R69^6PKUZ1422XG zK>!WY+Kx%PMob!1S$NJk+Qt^F$WD)R1I%>PTuFywj}lVF>!DtxnbKk?sgFPAZB~h8 zagb9ogU85Vh#sDPJ6fg_Kg39Y695K@P1A>w?2lNdRN`02CLC436jQ-@AttSoSex1o z3a5M++S)Y+Z5f^qv#7A#AyU8PCgYR1L&w0T!h~7JL|^=#3J^@+xNBW3y( zzlx-paYmzp#JXwXFXauM63{~fFwDTu?t1Tm;^qxZB*U;69M5KA-V=8uw2bs=NmY?Q zkaVE5YSI7BfQNaE5&`XH=5M9>gR|97drdw};jkM|kBTJ=Y=#YAvaYL(UqJ;7cZ4Q2 zK9f|2A7Io)Q?4Xxl0}x0p@6gjdmhyW=2ioX8S%;na~}p9!7Sa?3w%xdbH1^g9t}HB zEy(DR>w2+Uh8Ehit!H+4yhBp|jb>33%fΜ7Vu=bx3>wzT#(??y4{eA13IFfQ*Fa zn&X9BHAVSMFZuK*Oe_b*0FT@MfU4%qFa(@6IgX#J+Eg@Ry3Za69ybTxlqgv1!Iyi3=tU{Eh=v&rQ7+pu zXplTFZnQ=NR*9Lm>fDw}Q%Bkbj#$>kRByW#PU49$nyp9aw{}Om>$cIlV!-pBchk{5 z_GGm}&<9Kjl{||jIErtXq-UbH^&){#$yyZAK*>LJmFv6KDC|C~Y)f#*Kk_m0$nt_Y~h9$USx z5U`aZekQ7Wjk;IM`@H4*DJQS_2%jXK^f(8i(ZS`1iIwAU3!ugl`8Twnx(H)XY7Tkl z2tIx$<+9JNHwqp_*GDdU@{kCkNnLna*et&?R_$FCQ zc^5%-7e;v9LRd`>`64-3sq{x);TngCt~k6I=As$`nlVq{s0<=}Q4mF5%Dzm=5^bNn z3W0Fja4Y^m(ujkRJ+iSqwpgYmMB~bg#oDBc_n*{_fOh1#9S`lWtQWZd3;oj$BzGv) zE73k>$J9KVU1V8L<5;k@1%ZwHuMO!)wlb3ziN$@?M#$!}dHeRrOda8Ihp+2=7#oYb z9WU)39r(qNxFbSHIU#8KwRR*&a}#uugse?>iurI5+JiUFPl-EGipvxJv8Pjahf$p$ z5xV%a@NoPCsnE6L$+%Hely1k~3H}=Pgon8$oaO)Z;a!CU>33 zDdcSqPjA!#vy5@51+!%OTT+>BxRlK#7^_2!jIyupJW2m;;ibGX}HW43Q<`l*RkG2rIwVh-?AL(>HOI8P=*v=yJ z7VFL5|HGB@{t0hkE}$-Vfq{VZV1R%){)geMpfkYA*yKNZ4=&a8)UZU5zh%VhMp+T5 zQ6p5}mRSP|wJLMs64vxTUO2l>iL zqqmeM)%lVjW$z{J>9F2l^oO?fYtalJMh8yQYq$?}?=|(O!+1*IC_f0q_C6RJjavWN z6X~&Q+IxdZ_w4vmUUAYN|0%avB7oigK(s0RMS`GS{ZoqZ=m(NFhW!Y%Vr6RAw$#zS zyt}SurbffGj;DaHq#idnBf#WL!VPTnCZpD4h!uBncGnYc@ za5V*dlC5$omj!hVZsU3zX}msXlI-w4+K8H<#>)hi%I7;>I*`G$d1t$-rs;CwUxRGR z(`YwK{LSITs#|+j_AdjpjjU9EtKH11-HhEP*p(@}w$Z(a>vql@rL8}m8)}pl8`gd6 zbmmo<+9BR!MC?97JEWMff6B3#?%YFP<>VDyF=}%OCsNEVLYI}m5N=;L`g&q``8Gm} z#+sn7L+Chvrk2cmV{D7oC`P`q^nc`ZS+IG63_p=Ss z)hJfB-W;NL`T`ClfC$KD_4{x^D%r?7) zRbY^j*^O`^4X+m-WO0rv2`HLEN*n;qgW9a8(2jW7j(njL`A81stL6zcNGmX9Hkay8 zx$JuZKGXZ9-zof&+ZZ~48V6OHOpkk{7?=&8KD#1I;y8q=2z9FMfpm%!VQ?dv5O8k% z95nPsyntrVNLZm7uozaOiJ(f=XR!-ub*HHS*(mTfy+b+3D54tYhPOF5Ojhw9 z><=m(5D?G*aNiKJvb8Z0H?jIT#`#};OPw07URaAb-=9|FbtW9L83O|l{RF_%rHPk51kR=F8LWN9NRm}091x-Lc(O(OY&oOC;kTnNDWy*8r zAaUjvZo9fBkz~U16=pwW7rS0>UB9n)Kerc?eR*#K3olHKe?CM4F?cia56?IMR+iYV zDu8{9LhqsqD*k+k$nHYRfqd&i&w|mv)!_T=k>jh#!2>miEdqO==alWauEbWQGnz(napZ%yjl6o8P4K|LQoDa)iyWqgql#FS z^^qyj`Ak8@yoh8W>*3~~g_X*mJ@rzU^Ofal^fWgY)bR-cnn5Q+1uyFN%!(lbrUDu% z!%%>NeO{jt)|1#mLFp+3cukYI%>4%`GTd1ZYPy}ZPE{MxsC}aCSGD?QwjV#aGLDjr zRQ8GKcwabHdRU{q263$47?d?m4JwpI2v6H;vMi}{zj8hXn?9eBqm%$?-)OE6O`$-S zRahKx$1-cHwqOqLB6DR71iFPJ%e(X-NcWs*ZiQ`jkEtfnZf1N#fMJByN;Zfzq@nSJ z=`R>_b^SQ{b~4xqd?ez|^E`!cvryQeIpnz$2rL~BCM9tTx`(R;9J}v-pFInj&=gHl zwP(tTMLJ1@DGZ~+H8RBI(#5z;4-HDc32Uo_8LiD==HvF5;QrXt@2FVw&PP-{Tr)DY zOA<0jgh!5pomPxW5D}upFyBKet~-^Cn%TY`Y&nQuF;Sf=1z@kQyS2DIQTT&=V30s-`q`b;M=Y;(P6r*O@s46tH zf#LSMpLIcNHEfjwG)@#)tLr8v+a-e{k!UA}?Uk(sh_Ja(+eYrU&e$ z73*xJ5V3?!LhPickI*Pe!Cu3$jtMl0NYnDYLo1vo%>Tl}Nenr2)kTpA7VOm%>Sh-b z&m@MeMZ25?k&fGx={cO(e;2F-*ngR6&?^(e%-<2NO0U?i?!u3mdZ=S9U0583TlV*P zC}P;gkK*1M=+tV3*0Gz$iFJq&zxvLH-&0qd3r!Kw-RH)N5UyK0pNlwY8P;39)4C343)@o?FY-Wu7;aDA}fS*#N_PBKL$T-kl zB{Hwiy;jJy>B(-uBaY6oSA1}VHE`LcRKsd6VQ~Jd*d=B6G+yM4#|wLjL`RZ3j~ZwN zIFlO6oMUDoDe43`_5rDANR{X;#T#GN+0R;;3Gn-tYFK-z*SC!UUcG9qO_!RSgK{I- zJ(rJT@kBjoFpsJPZ-PTfma!jXPRuddnMrt63N-_)6am$Oob8aWiiVWu%;D0kH}BF7 zNc}G1sdYzj7(4J|szqH7{{ZB&eo`t$KR?ykL^41VmS_Ty&{`#~n znIKjfg=efx+G)6?u`yP~=H*bzk(0whK+yvOFX9NKBR~tS5PWdk(yCO8w;(JdB_!$; zRc2|Tx+PcypOdVD7cz+fc9M)KXGcG<$U-bQbown=7Q#h?ylB{!)tjgaS{mJy(;&Is z#4wrMI^=&qf6T9KP5&5_V8IGr6BJ^D&ThI#1T))^OJWFv0JNiqoCc_n z1m4>8<}Mmz=72N5A#sfvz^div1u}i-=pAErkIjdNaghIVf{)h^1HNbor5$s8S8JGp zcJSJPKLYj2PO$V3{VA*$xqsD=T@3m-Y}bo{n>L8*V*~ww)CSfeywj^ZTg!iAC_JSI zt)$ln7Sx@g8Oo9x;!aT)^#GPvSAL5sD0p@c?+AutPLpsGThG&lS(QH`oy>80Tkk@p ziLAlRx;Rj$ljup{0;{Py3#wVxCO4%B+O-G)05LCWq=`R>EM{x8M8ZPJVmOajsQ7_4 zS6VR0<6`-gRp}q(QxtNaCYudm|0D5KGeq&C)bp6Ig-ProjKO)X>lw!%TKt$5|yLdwi{^YnrED zb{f(P8G5Y@^51oOFDJLKXc#yj-LNg6LX|C?Eonf-A(g%*mW5ZiIp^q4lucWP7Cns`6Av?CJD+ktm5?DF`&? zHXOno`6dyPLnRHZCJAF#1bX%AmR;#y>RrKO*ERS{`W1zY5tYg^;Q-Rcc`r>t+kVMR z7*R_APAm&oC}*!7&ymSXrXj;Ne$QqAaQfru7sFP#H=ULWiA1hIE8@qbzCRbE!YxFPCrv2^>q)ZU1UFa7JY_H z*}?YaeQUPKTwL{vWKDCX#$9Q|F=qeU-(@{j^Z_Zrg1UpCluxavW z0#=%Jim6FvTAzxW$3f+FxkI$S?c3jd?9gdlkqw{1i5+Bl>g z69e7Xb6fzSZ=D?8ba&DwOhbb?LBS{FkA(jmZ^ycSG2+cC1RAV>pG;Kkg=>GRYrk8v zWyOGTQukW_9VdxS()M{&14CLm9$;CZ;+%(Yl7HnOKP^x`dQz+tp@hvnrwEWuMKzT$ z7PkKH&3#&X{aO$wR#(k8HISN~4(}0O{`#5Uo2_#=&_kAB;q|oy4)_rP zZsB?EUAyEMRZu4;`i)pV`;MOn?ldmi!-rrY@U=t^_z^>{{l`?bjeksQ`q!ho+yazN z30{n#_pcHz)%__#-voUS8^^F!JNILX{D*abFj z1-^qYUnNEwLtF9W5|xY0%?)i|CDe&H&UMU%PQiye)p3r zEX6Lu*oZ_eW-D*Z&fI0ks(hW?VMeLLsbN33`^rcV{hTpxFBWZXVQy=bM{PdHzXCa< zgcNLu?n9q!N_Mw z*LWtjyp)+erAXg8Ou?l)u=a+`IpGw!EE|%ZXa3;k>EagoECImC0A$yb4{nyZ3uu+N z)i`+QdUYNh9Q;_65)5H-$4I8 zMJa9E?7IF11O)vK4w3tp_V_UbNElySiK@bTr zJPmGuBn$M}7^i8aX(c{*68=u~^Dyki+lfXbOP_=E2@A-)n3&?7<{qD@-sbc9y&^Om z6xpaTVYt~B6exp5x@5IrS52=K_+v6*w96~x{lIzzA;czrKXQx#7u+{(d@!zn`fiUo z8Z-Rso7{nqJd^O7qkjS_Rs1o|V7k(!(YsIosu;MJ9&bb#TTawFwtN3Gg>VwTumdS; zyssNP&4PRwT8{GeVi-W^K)Eo{&?8Em5CtRI+iufdb|9Jo2oORrBQ$eU?aaBOG+5o$ zYh0=33m{Ztlo>+BjoFqWb0>#2&h2`PF1f<^JB`DZXGsh)?ugD#7+Bi5d)Q(7L@Q?=36=-%;24}pgu}zG2K`rjf$aVVOriX{Exdz zZK)p>PJzI4N@jSiJkMZa4=IelOnhwWYZEEa$M9=%fwZ4Lx`Dntbnl-ttB&1(}1?FR*$H)(r3^ij){4SGo=ZTOxv2M zuKl%sxnyP;LLgiOeYEBsBhdCVi+qsQ98JQqbf+ovMf-Gdn2p>PiC7d(SC*|ay#Kz4 zqCC2EHGTsD;eY}G@%`T{B6%TOD^Y-zlZk_f+y76xwRIp?0vP|Ng<7Gqwe6fdvd_t7_n48To*hmPT zXnIDxa&AtmuGArp4R)HvutlkAyHR=5dL6Wl^seIQBE>pigE=n4rUl2wwYxG!0?H5PyasI4HppRI zwpu46Cls+Z?A&IWN%{_HtuJMU#kY8;O?|Y%kA^$>*3aO5tOhdaRJUYYm?!$DkdyR?5UtXmUACtY}vWEM3i@uZHCV#jjiuy_qoI7?SJU>Cp7FOOh+x zkDZ^BV}>I4IC-E#7{su}7nGhl&N;flZO^;LHic8{gh~TiikQf^TKINc9b>@OfRa5^ zR&eaE*b8=?cnYROi-ck;ct6bs><6=ATzD@Xi7*}hv6|WxJjbhCoAgHr*A*B8hVbpb z1JL+C&V*T;*)f<998N(e6o$SC$-j*_=@abqCxm@sQO@Qu*$Q}z9z|QCF+Pzm4ix)y zf9(iyLpq@@4}^?R?J^v>CVl-EpTGa8>iW@!q7@(m0g3#SbUFVYSM7fSNVTBclouPn zmW|`$p@Y?k`9YixOvJ?rRRwT^%x28RV+Qa`8ig2%f#Xx{O@9klE~#mWoR_QiiY+!3 zHJbZM0+og}88%jGS~n)XnR{L;t#36w&pzpPesdmA7{{gdKx|FZd0sb8Z++`}c0R3n z-VDNJyFvC)zd9pSB?=0KdEJEAra&Xj$Lt10+|>RpK+Sg=F#k!VxTk^{7)3`Bg@!O0 zr-EY{w_+6WM8|J=waJj6Q%n3wQFc8z1jMpR3iPzdUTZW-(MFL|%PeJ>jN6uskYA03 z#=LIE3j|Yho!xVlk;kO1Rp00Z%b-h_dW(_AsD)%{66DYwG;=d-MlJ+XjYcqnCTJ4e z0~x#JQBMrb-sJIFcn`(}t!Wf}P%Uc<VO6Hfa=Bk8)2?#3L(CZY=YX~wm#Tr5|5$E3&h4wVbpd~CeO-4v7 z*|6$F&TWKf=ilxGTLpXhDhCrawF?K$8d}zZAzMYYmfsFITepHOUMo18rNF#&HK%s?bGn-e5y6C)_m=D?!V%_D z`LV1}sWl`R_o&1BzBXX*3)-JhXZg2JXZ+MNYX}ulfotWK%>#SA!hZ6o9s-0luN(T! z{7Vje#F%*qbJ~jZ)VL9Wz?A}?_4i4Ks1iZN=e`GJT!n2RE!OTmP_K~+=Ey>9h9 z6)n7yA4&Y&1r773Fkryvpv8EZghF9E41QA@fnZuI}8J5`T!S8?`k38BlnO zJ>qc)&h(!cApGyW0rYvu)Zik5qnN#Wb@}^?>jRQ&!ympuCK54iVc=y!LB#0mdOe%9 z2{F^(ZqkR$W+n}5FgE*RZcaiqi(6|q=G3#;I%^14n=Ra!Xz0Ce8%=1X`Er$Tp#9b` zzOv(*b*P6oDUl>890`e_^U*aVZwwF7rZSiGHcc%KcFbbOg*LWs=4TC`+9k%UBXgy=+|;j)QUiyPnhhN- zWU$`<|~ z?S(SsQ8BT-(Cmv4F}7kjNP)oekk*#UNyWhP-YQ=rp===uT(wRTh0=Ci zLW9ifjPezs|GnAD~UGs*=<(#YuWAtsv_1Zgb){ z(qIDtG%;wQq&k=}vi4XIYM|@>T81u^=*?@iLU~NH)~b_8viF>p1V))MEz;piN}FY$ zUTr6LRpRB1No33_^rS?9QmG=x@{X#eM_L%7$WzrS=g^?4noA~7g7Qnnswv?j9huoG zH_I3RRrK{O2d%S88-guT05&AV+|B8W(h4gAb7Kv_0 z1gUvP6^U)v6sdY=xX3F0lC&#C%Uu`xGfw(?oJGR6c#+2AFnVa-NQRsI85_{Y7X=G`56M~pFSA-Op~(5NvkLC4I3dP$oe37KHT2ZJgVMMmDp5ly z>t7X>MQFZM`?onA=Hf}5Ubh+B=U_zEv`UY^%DrG~N{ONCC;fLRUg178KVa9vq>Cb` zk>=hJ!>!o^}-{Ap%sZd1=xBoaWw>bqiu`2oG)cpnJvq$q#9L=EM8#uozDu^>u@+nZ- z7(jftFdviR?46WN*-F1-bS11lL1s&*^o~}B@adV7|Byh9t5TxxcIyci5J=cAN4Rr&5^X|_+L z?D24R;V)8ON{)B3Mg`!<%5GJ_3Ry@sO+wD92=xson^)T^ByjNV2mS4LNEx4E%=gsb zd4(Q9QG1+4I8A*Le+w1+4B1>#2Gd(3^og8%c3B!~W8K#WsbPI%UKPQjRh5>)y0O+q zT^nk9ZC-3q^0eqaWqTFRbQOyYz+1ImiOZ=|XnDC!Dgg6AV0>7nsIym|sOCyOkOHqU zz5JqR^k~n`Q_bwhgslfjvPhR`=!r{#amy{N-m_I!41X4Wqwla2p`6nrXn_uPk61hW z90<85-_;A;hWM~o^=mWWM-Ptn;tFNcxYo?^c-hil)@?pZW`B|Sf_U04&J)Vr^l#{H zm;h||R77{n$BlW zT+lLJ&j<@P^VzA6Hi&wJ3A(^b`)XAmeUh00^i(fMMmR-@7>4Q|GPA+2G!7RSv1(Bo zT7F>K0oJIe_*RCJ0hbM-4o&1;-3>DWwml!O)JNct`#Lk?Z`=W)siG@kUbElzKU2_e za8T|*8&1UQNn!C+r~~V|+eB{%+aJ3hXxIHi>xt6$;Ciu=P!k6moQjk+hy z$c}6@D_kt46yg$>%hh2dt^ut>Jo(UHeH)>PxVO%veBAK zY(T*e64VYu8~{k9Ghl3{gpQj-BpcyY^>AuO%4++|Je<7)aJF*>FN(Z7B(}Yyb>GM~ zNR3kOvIFQOIIiMUTGVKVyyv3cNrLF4&~RgdXNdIrmk$vxcFv+lj9!W^2J@4gmu5W3 z#?5Y=_l^vbw8v0zb2m{Ome^{6aif|tNAsJK>kON42aPrq#$2o;Y!ULOXBRwN*OtaD zm3f~bu&20~s!45|*lb<8-jrie>jATl=8|R!mi?jFW_2kt>yy~bY%JzgOeI5@3DYM` z;j?CYtjNg+w)y#2L)@I!a2BN2{T+yOi6d37s%oHOC`kvl!K1chC(U!STZ{EjVQ3Rf z%ANbZrOw@QD` zmRpCglqb`bSG9z_(k;K^h_-h}Tvy+zq%0!sU!nVt_M>KOp2TC zJro?_q0T$aWSw|VpIr{O9jjZ? zbM{dTYCe4E1vZo~x1}vecCCM@0!LY$^On3My%uE=WJF53&X74^e4J6xx>N)^cKA58 zlNG(yIDB>TH%c4@~mw^*VmvsjoM&#vmx&ePMUwK(U$9V4ck5n?5$wp!FAtJxTTx1 z8Qzv!+T|$zu(x52Uas>9UGOB@Od$#f`D7rXyImwllCBTLWWp_3h>T?XyJ%GSPo6{x z8eba53T@J8i*S)arnH#+7_~u$sEZkc61y-$iTc{SAwVv?a1$Y1;@f=-(v~u*Hc6*} z%docFn~%uK1o8y5<0PQkKB}%+k$*o0@y2|skYn1#QpjbAX}I*5m0dAIf`PTdC1eb_ z_w*juR8)XVyigG(u~U83zw~MO^{tU{|6KaKoHijg68Tbc{hWiGG+gColo~o-ggj}G zJk}(I*i0eict@E74ax-ctN;O~F77uE+~r(A4fDPOpf|DZ&+@qD205CE`r1;1cr&B2 zxh6~KuajUa_Y;>N^uX0A8EZ7}7bif4$`a*1!KYPkt!e7qvg&MIw3e2tvLTQ)A|5|F z;SrUqX{e}RHMK9qO-}RzqZj-UazCN6RPB;k1#&!F00}Mt*j!i12VSO={V$!SOP%W@ ztK<#7>N)_KX%>q4S}Q*r+xyPax2VCg*+6~JM!N+L?ULBNR>o(JfUb5_j>EWzbeY4r zT&9WRfI40KBBprFc!pqk-MC%IWyN@efNkYKG#Gl_xL8Qkit#LlA-zLc!baq6@A#72 z5+4g)pVhD;0DzlG9uMwK6U5|pGo1{ADiNo<6)3tIfVvtXH-`GA&Bl4iNPOs^O`vwd zhd->^ir74^vNC1#EzpX&9b;LeR5i%k3M)7M=%&jXo_yfwro$WEeDJ4(jBU?*HLQDF z<|TQ9+T9@>dF1jmMMo6JqK!aZIJ;>Y^=xL2nUVQ&p3G%W$TA(6N=OKV?Wms>k9p0Sxfh$b-oHxO5j0Q&KdN*2}KT6m0NzA4UdtZiRPz z2uG2G8=t5qLl&rH;HFIF0_TNr<9IqudHgMki`&q2CL~vsjvf*o4VB=+ zR+UDLL_Jelt*-XHSgX-up4|pMf;2{KQ|CTml@6avV)*o%J`l{t-r2QycW&>hdCc<2dwuVqJY{3_hyZp7w@|*a3KPCZ zK)p0zciRzjt(u@z1)k7g%5|O7nDOT^hUkO8GXU zy{WqIkn?7g0u*rZhCG8zsMXVSYmmSFzb7?sgv@$nZq!eAo%Zxv@zfwx>3hE;NMA^- zv}xXOSL9{ePp)21T)oTB^TD`@7)F^O=eY+@g&h&2SCT80=Yk<##P#i&cqS*E5N@Ft zk`4~%mRMHT?jjCdh7XsL+(^3dYQ!E$riuRWf_0`L?|5j3Pt$T{Enfw?Su0S)#|KW7 zhPzSdk)zdj{d?p*`R;r{afcWA{x4L4|EPTqFjDAMf&l?}lmB1xY5zm1rDWptQ?dVF zCC|(hS1)7{tXOd`#}lp`+Vr zKvVAAS?i-~Blr*6k@%Mz+ObfF_Ofhfk54*pXiv+4ZovCv`4^`f)+F%J>>ruh*z6vu z+Sov8Ck^jB?VLuTZ;WPN?H$?JTFQ z3E1F5-oOp-#O?rupmAZcSci5Pz8=`w zUB5-NmCe&%Mc;VVf7k~6Ru>nfRF|u=l>Ivzb()m&L|j<#WUsg;#gv)wy!4VlZSf$K zlM%=BQX^XnU22>(Rq-9H;Y4mca{rH^ShNi~SB;Ia7u719_D&Yei`;PmjwCt6* zA0-0zmNpHFZU--+pJAE0k}FZU8_Pp%5%Vip7-dXOd=OPc@z~U>h%{Y#?wIs}O`jYh zyM{5s~3xQ(!+ zUuoa%VRv`xpo>e_$948CE4;^AFs=l~CiF{K?+ixanA2L-jVInFL>tKY|D)`kqALrd zbnU9xHY>JSVa2v>+jhmaZQHhOvtn~6*-1`%^!WSlK39FlSU2lt?W;Y%`MvX<@1rHt zq*=9Y@zw(KQ4T5CeJCo9_5J$-s=2bFzsEq_&WF#uR!zH|R|@sAwu8oc2b{EN25N7& z%P)HPHc#74lP+s$0ynU@Sz+667z0&-e7FYdFV~h2= z6J0WCrvUEoS+yFFt?BsWH80ck?sT|>LEuJK$ycUbRHlw4|1bvnZ}B)XVfH=T1NtL_ zZ&A9UxM?NFC-;&C-O;}fx9I*0xE=jlQ=+IqXt9$MBXd$PUYUa*rEgmjw{e9}DMec*}yAJ-COt zGh=c~V>fGhiyO3y4eZ~OgYet$6Qz80z(+RC55Rnhsl+{=E3W*bMsU>QX0zca<9y@U zK96)49ELOfg+JAVNN-s?)lriSW47zxQzQ?H@T(2GPm{KG#?M5NS^vST2IvKUD&}<< z0FXwIgOOlcJZAAm40xfqYX*VG?IF%H@EDG5hivwx=(xAcY5=_tZZ=e~YmqE*`5y3^ zujTm{N874L9Z29-PYrUr!=4LjegzRDGDM1_qpkd%r=j0idw)W;`FZac_DmDB5#lU# zREf5cGF%bVqU{6@HqEUDJI58U&Vh_ua^@b9aBsUYfZ$}lp$!Jj^qtA(E2YKmOD(af zvE3j`oOu%0bT?fb#9EiTHQ(3H6U{y%umJwv&sg^3bCnQIIxVD-7mQy;fPu2FrL5_a zh_NKOIw;U(&IsLC=|^Zm^y}9Z?4|%eCtR*8LYDy;-t}U{9u!tMEofuvJb-@&FO=?u znw~}no_AHSHo9(EQ=hmAEJ&(TAhI)W*q#(Gcudm!I{XTYkY6=&S91ta5U5YqBRLSe zc+iS`_b-gFVu+A#f7I%Z30W@y5R?osA_e|M(!)G>2SWDRP{~>w=oUhG=JBF!4(4$r zJaGARIKSn|3NDndoLi5&L$l8;R^k< zFJ(^Q04>w+N==_{F+t{>RpivAYGzi`7+K~7(n3)%<6JxgoiG!=vA>~$M~_N|lRU_# zGmu*}d_#UfH#94MAeVAmg3;R<){4AaGQ=SXAgA+1MBa@7B%$om?$gov$|3W^-qoP+ z3k|fI-+6@tXb-rP0IJY-R(D>J0NJ73ilJ5sK)NQ7kN6XAD7Sowg3P-UqqjCJHxl3> z=`A_HtqJrc_T@Xkq`b3$=9h}x)f>u<1;8e`s|}bn0YHU;fx>}RTt1B7_GuqX%*sIu-=Ud@dI%Q6+ie@=Q zPA#ftNUPYi=T&{KWam|i=UF6YQ>(I4+F2>fl=Y&&>0n8L?V3O&Dx@|!a?Li z4q$=lG9p0Lnf&b||I^2IMGU{NQMqkG$g{O%vG~31)10|s1L~4U?iNfKYUtE6R|2iH0 zp-sb)2CzrUye$eoB25|y%`<6cI=mmXx_9qCu`0c%gvK2*8{oFTvSOat7-(NXc_-WlI|u)(g`zozAewv3ry*=0=mDW)_sAVYXGj z%$$Cjo6+M&x;Z8+RJ4vjpw3&<%Ow0WQd4Wsy)!qgYov&GfyjQGU{-dm(Y2dxk8Qj^7R$th2~%-|XyxB~HO4QZ|6NU?u&9Q4Y1 z%uH?v>*5&i-Yh&zdwV9Ap~!ZH_8v?ZXfB7^Td~fNCGz~dl&2+8=V}+sk>VIa;5xIR zswgf~IwDtRWeY3UWG4L6!f{VLC4F?rk%o(Iy7)FKqHh1u?@LkaL+}3hwKZ(#Z)jad zbF*A3i72wDK8i2-FZ*TS%PW(buVCW2;-g-gmgcl-sA_t9yx*Dmq`>qMwdA>6-oZ;m zv4wn5gH8}j2$rTqzfEn(SBN&pWy^`i3*1I257}ASRDhbmNyfa|yK}E&BHeDrMCrHM z&_Owh%J~)i_@k%ooFKKMb55S(Pu=4MkU(k1?QwTob+VWd1>|n>>==UcXp;u6Y7rxg zeif7Si*2p809??I40OotoxW=c(ky2;?PXHQQ`$*Elj#{IGRGV#xfD_B9Q|X;Vb6x( zd%VX4GH9@WG7KiwlE(4$|KO~{e&78(XLmDvVE=cpo+f4Wd@`TGCrO1!>1D^9iuH|$ z+|`*OlB7ul*WQEen5FcMQY>3hVu>v->rLa_(g6`e_skEnF20YMSkL)%&oo|w>Nnzg9f{gYrd>6X%Nxg)|@r(uYaU6uW;g@Ol? z4tA1_PSiB^mvTb`J7y%}seYsC(`fhS>}o;m$BHri_6=iiLkoLy6LTaQ4a7nR;T7Y56|t_!v&K zvHoL*7?<0=GP>HmG1_L$m+I%mn47r}ZT9$e*vO6Jf~0>w0bhQRRBD^dYUw<3H3r;F#X@$I-WvLnk#w-d z$v#%&7G@v}iR{ZBzfGkwn>H5cU0R%>5J#^iWUgh8&9fpRI5YRH`_b@ObE3T#$M@}B z$FO5T!L75D|D>BcAVdkU5K;wh?$(8lJkVm|XE&@RS%Ei_i|Yi!D}~b3$}o`<8)&7B zxt!N0Ni;>d@&Lr{eEX|=@ou6LyQCSlpj8lPkZE-qv`xy;&*QZ14%L)HGIhX=Wj@(! zRT|^t!!}<}SqPw6eht1daD-O0uC3OSbE`U_IPJFv^~x7<=YzA-&qK0J$(lT!XzUv!E>!E z^rF@mYN`g1(O{;uBz#I=+KQ!Kh*#2Y=?`E%E0I`zVc^H#`DuUA*wToLtV1!n~!!(@q-?d&d^uz&x3u>o!& zRicgOXcc@rz&g$LV4462n(F*MiL~lz3WH{L)okG>2Rgti;8Kt-SIWQ$`}}qJ@4q%U znz|pzO$KBbM(ApGeK=cTnYZMRj?TC%#F7Ju#yUeD1R(ZX9_BBPiM>%peGsyt%@h>RQnGUymyJS)jwqIxqtRWu9;qS=Qms- z`m=3H<(*($J0UEM)Kr9QNb~lgSXJBYowJ6TR-O~ftl(ee3_An|OXt&SfA*$p?h4|v zFLdZ`G>4m8i*jds(KV~fz889?YSi^q===ORQ>b$m>0dwxmr^r-_x4OO!~)dE8f|VS z4_swYZJdROXB5IU*y_(I&1Dsxj`)-BmY`cXV!}#7Ij+r`#g`kp>NA_#rTec7;87d6 zz8Fv0=K8PK9#)f0`5G~sc2X=|=Af9`WkzW@3!3(73|zL78?iIyr`s_YJa5uB_EkpZ zSitqote}?hM~1W@PpvKa#{BfgVF4HOUi`f{Hr#>CiqB54XTISvRq^stW1EJ4cQN+N zanWb1WJQuD6F6r8h<#Y(G27~&om6$w_Q9&{^rnvm2>H73RdLRaq7Oe9P_q+#L1o0o z>B84UDF;_PuI$(R8B{@5vSY&qyhsn`?Sd*S0I}3^bb)5lZ2x!s>srY!;TGKQMOkCC?J8Wmm2=rarY~ zbN&gx?S%tMt^U1d-@{}Nmi?q!GC6hO0orUu(0?9+%gZ+EnH*S}JNOgbPAu-=LdHi@ zJv3%ZNgeght<4olb~Xlgc?^^)?^H#-itVxW71^0^KgxsPuMs|*x?C@%GDn>=K{sGZ zVd->`F)$NGLJMVzUjMZ&U0sM48&;hbp0-qb~BQDh};#KdNRn_Uwd+^#yOjgc}qDt zW8@A_*ar?|qdy2j>0@fgBTVpc7v#D#&ckg6LQxLxx-!$#XKVDqP*!l`KQ}O`f?a~|Eq`eW?=H3v4=K^bo5r`6|aY$;V|Th{D(WvfpYX8xS8q5F7oiP+X2)j1U2r+93H^c0wXY zHo-@_4Y6LrXWg|yMwFPR-Br({oS3)WC2*ovjYDQBSbSs_PGO}I?P{j(r&A|=B8xPN zIbTV+!fu>(fZoFEE5oyLcn{!r)xm*)Fb>RL5!YD~z6yx_33{C)J6WsYmPOsh zX&aX==w%3-hh7qcnf;0_SxpdpT-lB%w5!Nnxh>pOj}L!pFjfHhkM?cnxe4_{_Nvhc zhnQwJRJZAWz_Zj@fcc2&zr_yRc)@Isx(v!&Qys@|hBg5y(sKXILXn&{PAM<<{$Gm> zRH|^!WJo`L#8UnE!Slaz^Zw_pR>|4GQQ6VJ*2((+Ps^Rw`1bRTx_pXlYa0{m6Umzc;-XwhrYHP+HT^kaXIp^d= zgch{StI=7N$SjgI9F@&(^58-&mD%LUjQg7E_G(U0H4w?Zq1i2McRI=W?KJcHfIZ~ykl7CqMS~J^InhjC~zg{A)c0k(0w*S?xD`mMNU^Me$8P0Iv=$Hbg=Hm9q$&s zy75Qc$$;+O7b5JY9CI69x)s0hg4Vw8559b;2NMG9`F$yOeMzTrnQtsCd}*is#@75K zZzvXksR3WlMv3YQpnDpl|sUt&WRZJ30^#^$nh8I6=uH?>uW+Y_JHR+M8~ z4jz9SMg`R&iv97gSEx-|s#8NtXEQ@eOiPH#%wv4o^NXqdb^;B`6Y5}D0PL2B2h08H zIhf^vE1-323;9e&3>77_qKmST4!N1xki36cVS7zvh~=o5I^l&e7!Bj3TCP8dDiehX zw-k7~nIvwT@6TRZMdC+p$J?;t+7}|JXcTv zFPvg-mZqy zQHYGw{Te~%G?Adr(aoJO!yNH@&4_FxARH$Z5#w1>+btz=O{0=yw7R)f+WpGmxxXf< ze@je3i4~x+9j9XW@hN3<13jRG;w}aV6y#gGR@kjMOkw zg7i`N#ba@Xmk;r-LcP@O?)|lR@de>b#RW91iaBNaf9%O(*dg~B6Q#fqYUbo}^7IOI znd5Z*0sxitWISQR4w{e*n@V_wX_+GVt*|}g9YNRXkPoMA1v8sY(-h;fG&VWTKC5%{sNst-fC!F_sKC>YchDBVHt#ZGl#ca3k)4`QDAnsK2JQt^WR1MP&k+7@*0Do zdQ1)+YV(Trb4{gbrI7Mt8Y4W~(c1J9%{+{V-;w4sLx_PT)M}9?p26kRD#_&)#To?_ za4JR>n)R0DtS*-6Bjt{8ZA-ie*ib)$9(4*KEr;=!)vD8sJ*C*D}oU7Dgdz zC7PAzGR?AVX_mY(8Jq!PGC(#y>k4?oL+R%EAqnp}p{%9SUR#4KWq88HB=t;6jItTAM%<EzqN|M_qfWoYEE zXnQ^>yY?ym22DYna)fQlOW3((_@%_zz_i@|5zg+!X|m3tXs-EsiiWvZQZyo+JTPad zcg8cHO5Lm!e?& zP}D%#6?x(tVqidk8-!5$m+U>Hj<`15$f|ZeelJ=cp>g`vd~A^mTGjN=YWclGAe$jI znZ-QN`=~0fS!>iSKgvb2Hv7l#OI*ySTVR1oh$H%l_jC>yPjfm&o9!$3FegsTlZdPH zI6Hg6!X`@A^KE8JyIH0aIR9b{o2)k1eL41l(#j0dv0 z))5L^h|)WP1iUS$TF;F&BhSA0bMZ_DYZ%fiyFwa86GU0<9()Pt6uiV`Z=? zB57YFDQ7duF|D=*wPRR!NTw&!63DHecIkFVWI<^yFKKe{+HW&IA~-F_`ud6~o?wc- zUZL6`71-^mEYjJtSNLNnMmO5&CTFZ4wtlwK)&KEq`6i+Hi)C`0;hA32r2qcg_ZlF@ z@cj|t5yPpm_eNR?CNozgPQkEInv;AP{ZZ z;!z;Y7kO&%OQPweN4npvh}1Gl%-nC8$xNnXU{}^0g-GL43c-K!9;?DCl#XgQS>so) z$#;u567Jei(5iq<$%aZ^7U zHWD_EiF-N!Dm}s{7XTfEEjru!j@2XQ@?r@%qky)}Uu;bPYpr*qk+?-j*ozepbc+Xb zivt@K59DU+({c7B{_zq8%P4fB96E|5CYaYhc=~}q_ozN+US2=9Wxkp`s}Pt|@eiF= zX%#6$zcTs{uHLcf{&%dZP|j4PLfeAYarHvHZ+Ar$5d*TgH~P=qkRzh)bd-BY9uwW{ z5==%0W#zU7S{6`WUON2S)b!^riRA!^K$v-l_51q@5RuZl9F_Qm3%{q^FMdLCP0UU)V+5;ou^2FZt(gpE`6y^__I_=;BcdB@e#2e zzdM^6Jdx0o1(PQv_vM$BqK@D1;Bp3uGud7Y_%^+;77vF$eJK0&>eG1ovJXCzJkp*% zkq!Mow8rDJh3_GDfStDjMYo*4onia}5#vQblRg=D`2uI_#Cd`iwf7}>Vq^Eoyzy!k zKD7X7JyImQ`i@^XX39h}70g*mjdx)k7$?ib+Vab7Who9VjCVf`fjnOc+U039-O*D! zgC7)<5U|`bn)d|Lviw&bkZ)se#G0=r(@KOX`=e6|wCk`3KF|$)4k;+;uXHI7ce0^)@Dnu76T-$(6^#!20rI~uaq#5q7q|%~m^8&vE z;Hi6m>Y1$R8oatE4E^ZTm3z}fE(tp=PV~nictN8)k+~-gD>zZoS}?3u~dy~s9a%jzN87)0n4lS|9GlFu9eg) zloU_`$^2@n>QzsEIXOZ-ZB@Zjc`;>tE6oTXu_Rxh_TL2fQcTYAo?B96 zTC(L5{sN13b$E&!R0=OLUi1x&A1^tSt&834e~#Nlte?ZIo7))%a*BINY0QyQ=sfPT z>-^nDKwD|0?1eWQhkQ&vK>p0uPq{D@i>8;UV>^+T7G4>5s5Bs*uEF+!(9=^GKS5%L z)Ap=BZ*r1Lr;kt%>bB&=r~ofJsCP)#yu7s}dPx{%t(C0n+h4{RU1n>k)TLb}qQZ*b zqiB!Scv`o5u;@)1h%S=vdm?JfNl5H88(P+{c4E5?=202Qd{SAyH%DW}&X6V@HT_PS zoQ+f>xLOifFHa>BHneWAP{_AaT3+e|8?Jy@uE3a7#uu~%FRz2UXzy$;PhKlkweoYD zA9wIWWV@UvTfsj~U@E`(8|S}5J}Pw|F7S2Hxtl4#&g7ZaFSjLOEXciG zYliuzrmzSOup!uOosD8eFs;6WmdK?()fC%d8VmOAl4)DP1UfYNKkZPI02QFv%)Aru zhc9L9%@l33Y2slOZ)rNmdisfsDLG50-{;dN;g69!^+7W(EWXt{h@QDSvYvzaRWAC8 zv88%a5-Bkm(Uuq!>}8j#v9bM76?`k;Qt~PeR|fOFGl03Hh$K#-j3hxpYP2JY{-TYz z5FTEQP21>{|ClxV6l@R>wh#Ukc$T>mp>vz<*>3@`?SDFtG3f|tH-H5HKGr%-ov5`; zPU$?^G7Z-3GoIH~jm8mXGNtHs)(za1Yb_xV`0eyAqgAuw4DNd$rXY7JY(!#Kd5qpf z7MnF|us!bP|Io1dp`A9y&hn*sAli~;i$%XT&GF7dU!`< z(=I)h)HvZO-L9@gV=_+0EjKN57a(P>laI%w!xBw)AGF|HxN-6ri&L@_4XQ{gI%0aP1s*FWWFGNViM~Cht~pL}*1r9V3SeT^jA)iU6kIedNqNU?aG0CS_0- zXF3dwS!gn=m544{H&vQee5k^8Ffjvy;Ga9|SHkxu&M?1!?t zdB*ys9uwiOLQvb?%xqn|Ip50_!7p;JNbY1)Ev01LdyrD0ns^ffM^fRXJ#k8Utz>Ck z+|5#cgK3Fq8b&90hHxZVtEf^nNM&dHCMg*yj(L&4ksTH+W5arMAiSIQvcUg^TT=V* zeoKzOThNH|7MJfxnfir)DfkJ^EI0Q6t24f5!*zbJxV;v_>mFK=-N63*-MtBo5sRrZ z=tZBh*ya=@tm2G)MeGdYrTcvO3!Ua0r7Rpn&Dse-vd{mFGK`u78SYjlKm)pi>@v91 z&B!77r2$?P0d5XuLAm04xyZmqhfmOh(s2It0m70Yhfi^kT3Fv-GqcEFOn0I%erPvt zw~5feyuH@pSMY<%A%>D$CC-(8Lb<(AFn+uf1{Ax%FIwqiIr*32*E7!9e^*I|?ZceC zqFqwnB*FaQh};0ff$bY4w=eIkm+)S&PZHMj+#Jj1SkvQ z&+fZH14=o3Qs1qH_2vxB&UZyTh=P1I?CFI+aD??r0IShH;bvTK|ABp~^!H+%*rED` z0y?RHJbY(+vt+lOU|*hDy>dX%1HzJ9^sJjbOccO&;4bP3A@WxmAXf=M-EFVs5h53C zzi;+`}oaZcEB&qrG+=!a2n%%4nEK+NXlt z%9@o)#cLwv0p<8yVKisbTr!#Ie@T@n^ZY?3hyk^L$gIK>03<-nql?3iGD9N9?((mfLXHV1K4neJ9IBSsVub|vihp724NA8mZ6CI3Y5wT zr#6j5^%F6*?n;TJD27`3RDzXlFs?#F8vfR2OJ|2Ui0gg(fkf8J5N=SEBaKP>3^|7O=h(BAf3OBF3)Gw@k%m{ z%VkR?ul7SsLC}fq#r?KnqsQ#ZmT#bV_4f2T^U(^DZAYLpMpV<&iS>kG@3s#{f*zuJ&xH}i9`^GYw4HQBb%CxcX zSz7E?X=Up4254C$ z;zlo~8>EvE0-_fv{NA}tyqyBo zk>2YO*USFRqbp_0OEEv`i{T#-;mhN#a33XU?doKqz9J{Th&pp_J?90kew;zpM;qoa z{=-l*#3C7*O5Bxr)|V!f$#-Mf7GD?@=a+qSV>!LqlZW-84gN%1Y$ZDzmEae7M8$q| zn8+-WE1Dx)%#~d@k@#YopN4z$a_&X(}7j;T4k|R5l_613e4H%m6@0A3e)QN-kmyJk^b0@h zOn9^^>Xm)O#eP)c+!kagn$t6Vqmg)%S=c3hyMlWI;j_qRD4EkUd9#vu!(Z4Xd+Ubt zL^jhrm7;pwmflS);{Wv_RPdE0z%c*4X{)ohm}Y^&w>P@+Z*UC}-v2V$ zThZjZP5W-z46Oezz@j>|Hr_J!7yomb2{y_vjjR6gP293|#0|fp;%7RWnl=~PmgpDDWypbLcQw(PwL339>ytaBcVE-=lk}aWQob;~ zJ`cy;H$6Kq$9t78{9yXw8CRo<+EjyPk@Xn4iTe+}2x=B>nmMau_SJB!!(O@btEhf|CECmJbUiMQJXWP^(=Zc^ajZx6I=V zbY!lM02*~@^#%MWv-x`!$O|eh>qeq~vws`t&3^`@JVgC03}E%bS-CoAHfoRx{D*c* z`w-|m$=;bs$Fe0up(0NV*}-oRjH_AY9_T-8qbfHf{7-7iquY~`<@$J*QC0Lt+r52Mmy)#&AC*g6{g%L{ZrHXG{i6Ar>Lzqy7hEa zg<3)a+gUt_P^H*mLflavJLcR#3PdWW_*)Kz^9akY=n1LZ)Ie>8*9nJdz89$*Wc-E z9)9-cP*l~149`qOf&5V7rCKWMp+oZeRWx0r2Fa@~QeFy^laslzR6p9USLqz+E_NLF zOwokX4q%^>q;meElkvk59gz};keThj3A4~bNkl)brrm4iOggGwm3AMYZu^Bh)xXAT z6^EUGoS2(KNG{(n!uq$9-zY9w(X#kgr*n5k=^w(sQt@@7ky`kd?^v9vmoT(p(tnB> zuUd;)bc(w{EOb3~d79N?m&>k=_&-L*=U7S=bZVxB3-hnu!2%TrYn_2Oz4NzP_`QRZ zT)wgVjCioMZCsyWspv6w%*65L1s$6)BNd6@At35H-OH}`!0H;*8oz}*v#x?e22Qn6 zy*mzgztuZLpwkrmIujih>u2w+9X?@SF;9VGf#|5;EsWKaa$v41b&<;Atsef@;Fn8R zgb-RVcwo?iGO~yoy-s_w7=)d{v!n{Aj)gh8AfFC8O>d24QjPtAGN_)x4058HQgKH{ zh#D)Kj=^10gX;d;U<*dUpLDo?9ToB{&TLBH&MM^<&TLZP=(2nf5PAlxdXzb2GD=Oc zKKbj@L6PtiO+CP7XNFetZXA(->rjqs=bfWgm}fcddb6nku@#CWhKMNY8ufxf)9@qw*`VArkHQ^{;-V)6E$?T#m?wjX4Fh@~3qHP#D zpUdb7#bFeQUo_9p<9VQrPh_~xFG^LfiqKo>jnfz?t0p8tjvJ;A$*8@(eZdz`B0-M~ zuTGvE6bleoX(TwQ0)tq(xp5q?jjX1JWEYG4-eKzhU4S-B$NM*%8?CSzC&W$lgmNa& zv%6{DTbKwDF^9q=j`2cT)uZ_jShMcK@vV-(2(S*${(d)%xoB?yxBUd|=?IRD!4K)! z4A|(KpQWh7!a1KG(9l6a_y%6TuAB~r-G2octBeh@uogWJlSCtscmdn$uFUBQ(8)Te5RUGK_JU3 znsFi2b~)#38x77lhN}NbfO3cGk)>`N-D?H&PoR5w0ya`@!V=fn`!>_~Gu9kq!hFdml}3 z6mc&Ge*%~i{bG^Cq9g?3wNv^Ro%$)3 zh_h^Tlb^8|&WMG;9~?BIj6(VI_|CEVB?Btv@fv^j4V2kE19OO!b%Ou2-%Ob3LVBq7 zRl<|F|8X^CcpyN%-3gQQ@JOWDb8)*FWg_8R36AL(^13N8AAVHCR*}*4W*dgEJvg7% z?v3Vdozn^`8>m+I-*h0q)?eEhr)`T&=?x`s!$iIuB*Pjg#&VTz4vJS1;X)Sl2;)Un z7&1?wE26a0H3WAEVBtoMu0y8SLOE6i`H;o9mdNywbnRM(EK%=o7T(R~^xKe83vq~yKM0B%2?{b1>gM>Siah_ydMCwrM#Z?}VID$n`BPxS z$RQv8`pO-Vat}$}H}e?gDcf?FO+N5o!(}M*a93D%QVTp{4$(%H9qc^QjaIHa-H9Bl zkAJ%2l{KHr$3sCPFr;;8m=^n;ai@EY5g9ZZ__r}KTl=wGvnw`V{Vy{cO9S(QKzbcVO-aghF>#|MSkDyCQB}{hjJr@J+t?UnP6~=QUr^#Ms5w*ud8L z|7((+{&Q=1SZ&)0RR!&9+c;5X!6q^oO*K#XmGKO6zfm$`O*2Ayv4ovW9mybbi(G4A z+oka=6%cR;IErHENrJrVkB1Z$Gfj$sgF1vmjA8!7|3p4-&tz@+L-cQ$XRF)m`W2Y% zbiEFIewl6kc{x1F924Aks|l6qz!PT1gfwvBE+k6)HyR7BKbEMT^nx*pW0;wkgv?0U zT5gFbI?>QW>NYA^{4*(P@n;v^p){017s+9jyt6EAMwA0KHpG{~_FOgzoEHi7;><9s z)Z~%J)PfU}&jj@ljH&FbdGKHM(JS==AhY&d)U|nrz&T4}Rz6ccu6G$L+su`MP1m2K zlD`t(7Sk)kUcr>VJ-Wx{K8(3ZyeG1bNqDrF8#4Au>1?fHEHFDI!bL`~U2CT`Aa=?ut2ZJZL!@HoU{EtoDV*LM;cjTZ}V^w;9jmP(q{$LC?|f0lY4XHy@8$zBt+ zr$rXyX>m9TP7&7;T$MoakrSWiPTNn;%M1n&9cxSks!~4AF7&9AdM0F&}7I z+Q$ggUn)~260r*=8Ei9up?J;{1;eNCN>h6zi?vL+Gdfp+^M-UtkGHh%!*xnn>1v=m zttgt{`SgVjT8X_wX~3!|5aq6FWh9OxVyTW}@1tgQjl~tGZ&Y?wyit)7_$w#HxBfa0 zLxyt%N28t~CJXe2FL}($|4o+3)Ai=(4p8=wEEA+^G&Tb;=@dC!So(BosTY>$G#AA> zM&fEn&TB8sIKCIDa?aX-YsI0%sxby+MRkGTCo2E7I`aWPQIKZ@-$-XPR)^UEr(y_X zt~0B4V7c@w7`vyZm7=0F&>V>h)YzZGJ4Wvcg@r*lGzrmZZsz*AX17_g&6P;ere*V! zeH*K(d~aLRVo`P5oK7!sUWKi6*u{K90=5yUL~Vu= zyy&6scuF1MQ<3p2uU{R+QAZ&nVvy+mRD{J`T|($LaxC|xX8WgRcrFnhb{Ku8&G_BK zAE?tAMM5^Qf>?rR8AK)M#4c65gT{f3eLi+!0t@}dO^mM9v6$N^3oK!LT68>}KLXjR zs8ANIGXxn^laz%Xn_0+>lO`DQU#kyrkgDSwsC<~`ku=Ly3H76v;U#x?KCd<*0FN^( zN7$By4MDrny`az>c$+JZ{*evLeT}~_I0`&#qE=(P(fI^kbCR9M%p7N4JIgEnS8DmMzIzjQ)x5P(5ft3(WYu8_Q-LWb}o#WP-jmInwyIK2Zam5$KK zilZJ$y?t1f`<*}8kZyTHK8mS&=LssiI!g?{POlPb`);o86YW8Tc=8}@^WqJDdJWm) zI|TxQRu?|E74oyA2hsvx0ECK^DI$?GfTbyC z5NkW8^p;BYkfp#ort#?Ex3*{stK6Q~k@%NKL)<7Gj-?w@IKO^-+pN%56`lJ}WUw=! zi12*Kk2^|pG01sa1AMOfOmk4cUuKN_DRTSx`xZ4I@2@kTro{0{!g@l;Ev%kFfei^_ zwdJTCF-(inT2^30DrGA{L|xWPgK-x%HpRFtS7I2q^a*Rr1iC-}8P#HAX#$Z1X-fi8 zc?MCg_~{6E8Flj5NL#Palip~F9g^}#5_~eV^K`XZfJumzeKN;xw## zupufIK1Gf!>jBj34aE!w7cwOnwRt$FSB1lK8y1t>st|W<=Y>l=GBQnaqb|pR+IGw- z$%ca4>eqkU8vPHYB&8c|fa+WQCHX!HSpGNauac30wSl9kh3$XQ&z4hc{)2ut`h^Oj zwGa%WNk~1s5-}}g04kyhYpC_Bg_W#6KwjR0Ba;m-#Cpbbu~2VVF8@UQ_lDCWUtXZeM~Q(IfEFZR7>fSjL7*$R6X#lFF7~~m z5v`wDm9!s2Q}R3!zjU5^geq?DAdF^k==PKRXs)VQL>$}@(W0VxHYP0bos_}K}MWqXty z9dp6!71~Cz4S(cQjTlwhVXEWM+->M+x)g z)kKE>-ENlXr&5Dr(?@qX3MJ$6CJ6YO-dM&XSb~5c_YNld`k!gNz&>L+B@cJip4Q6> zifl=lK?qFzuXiBLkf zEDX+gf|6slC!0H|7^*n|RNeX5$j?~t7K}Nm7Po@~tniVInizh))el65DDj!{&XnoF z(FB6u8s$pVtjAS}Mk@Uy%&E3WCk*xr;! z$+~I{q(<3$P};mrah$!Q?N~2Q-9t{ec1E}-Ld)~>1_c#HU8Jewl?_G1HOP+D&w00{ zK<<8crK)kn9p)q_mq=Xfk0F17M)M6cjy-Q4Y5Uu(j1u#dQHVX83C5B~hOQU^h> zAgH2X-6xkPb`lHvI>$0ZLaOI;x|0hQB}+P z1hQHft!Y|oYz*Vo@^8A;N*N(kcKQEs_D)fngi*F&>QCFYZQHhO+jgaG+qP}nHY!nR zXVqlSRrfsgorj26E1o0H+TYpYWtpCqKX7k&zoFJW9bXjl*L)@_nd8tQQWH=VP=^+1 z8S~vW%Q5l6B~a8aG!G}xCE_1Hj?_TZi4+NiU&Q#sBgpv|`dcR)lQZz8S>ToWlyWG= z3HDH=t$xuUm@eVcr|^sSE>6Sq^@*ha-${-y(F@sYc z!Z)Hwzi|gb_V|PmH~@1Yq2w_Wr6UsE9)yWLN+S7&F`vWt&J;!WL}fER5%DjQ$bADe zgA)+nTp+{SBK{-2Q|17!S9`%RySQU`j~BVqA(&6iDE=_2#w#}uR>Zfs#EV|jpG3d> zYqJOS33)d&h4w}ZkVXmaCXGpdQTMECwg+ZYi|!_ewnhukMhTv#^~pU#9)%Z`hqj5m z+o``Lv3wX2-{ul8@SF1!{_^fYywTUj8ZWgfw&?Wyc1^2Nh`uk?%ik(6%m*~DzTw`( z1(yv<=^ho?uR1XQ{u8zSACpJJa-h87XYzpkOrHOivR~D~QP#}e%=Ul#*8k54D%JRZ zN02``82Zpdnld%rs&#c3m}DcmPGL(bMl>7)I&`qQUJ{uWc9T;x;hVy5f{XPR++QI% z%gYXY;|hO?pEEZ}PXMHq1k5}xuY2d6uX8&*Kit#bW3)h{ubM+4AP^Q{-2n}3eDDK_ zAIy{i=a{j~NFd09u6@sjZ>0%afF|P{F}WVE(-@Pg?i-@URb%zW9!S`id?exQS#;H^?4$sUVC+I6pWRh1S8XrtGfjybmDsTlSgJ!{UmCn=HGjHsBi z+K?5msk7tBZA;PEmza#@U5c7KdIc~3NFZaiu%_`aFyU7Xf778aiJP|sXblJ$QCx^h z%uHptZ0lT=HJjFyQ$dZ{{-(o;BU|?>+?Uhh3evH7q}|-i=rk#NjFZ>N482^?3P_s) z5qb8aUg$76r6qs#rem5@V2)Q0dru_#KyJaQnU*DFY{lVdt~0Z6#`Z56X1kZyu}U|R zUMquR!-!ZhqC~8X9d|FQD2Ql*b0A_=jHg<)DbjAiV~7pL**9nwt4&0OZ;B$lqO*=E zF`NpX#-Zml57vK4A=vG;J$#ez6RY%!ey_R2th=cQwo%Bk*ZKt^^1aIDUoWlcVb+d^ zLYH-NNgYxW!RzO(KJ3vuIYPO86CxW9p+_uS#dE!815Qi4zJr|-zkQ#ljlKAcbw;9 zH4(@Q;RuQZ^*ZmI>#=sZ@X*@64BbIgq8-J% z{d?>ZA$O;zYWFBe9nFY*rXzED6Q1MT9F8eAZvkv;n#^@tk`!z%oL4Q2qWJMjA3_)9 z0xnfhcH@%OfU%sqpsgth*|P0{Y%)w^4OCj1O_R<{URm6JZpTe6PV)DBg_s9$$qLn; ziMl*G_p0+6Ls?Vi;O7^9*MI}{AOTF+e00vcBNScE>ml>F2bEWhxOC8-;|ZiOn={Zp zF!qqdT|Iumy$5Trs`(!$TM5wd_=T|Q&@W7lj^Fn&Qm?=T<{D113;w>6a`{Cam53Kp zOsdHbSKd&el){k0;hZlnx92Q11H=|VyZiuKpb#}f5sM%{Cuq$SevM$be89^cYJ-Af zG4%B^?*@H7^S0lipJn|b?xw`_^#D#wcz}q?2_i6oI;ld)5!8~ped(OTj=cZ7{!rFY z&yf)-*XP9vWT&gO-&pU%A$qB|l5=*FNdOrCxaD3xr3Q1F_~gGL;sJ^%*;mvD(>*Jk z;+St%1ZH86k~sg+H8m(#pkCx@%E6`ShFfL#`iSQ!1+1r}5pF@9m`G~@(eQ!m0U~Zd zo>4+Feb74O9Ng{k4ojXBL#A$PkC2V&9pdiH@ngVWWAS;IFK9X5aDj)Sj1BxT(2TF? z#FMj<3`7`}9;}l1y)Zp>D!T*podLf*oX$xZul{VLVKDMb52D#T(<;qa<}xd`&j*A5 za~l7TNjS`@O8NcMv#|&c1jP8?!C-2l;{O?drKaepEDQbsuK(`KK!&HcRQ z^Zc*BkB=CER<)7pwF-NJFI8cHyQz4Y1H%I&XUC*eJ~Z8pu+s;x((>eRioW>;p7`>HTh zTv}K%&d5E^Fw0DGoqW5kE5g*y#AYX~ySr-5dq-_VSXAyoLRt({FGeKOvOlE#d_>y! zEpfoLinOASpnrNtyR1@jpzD=x{t1NtI6j(7wo?jJOP4XpyQkxb8is9h>}`J2x}B=q zbkW_8(oGvM*fGnxw#F0nJoQy*;Mr%gVJTSZt>voE7d5^u*2&>AB}GCSIQ+W=v{gCMi1KVQCW)%LGnTIRq9oK)+}Q3xrvWwu*Hve) zzGzI~!hWotQT$&w{|5oHwZFwdFDC7cflo0lOvRUdHzjr)o(x>6oB)Bi!8_H+G z8V;nd#9ei_-Nfjdl`P^5I6ZNskv5!9wo?$TqXkKas3lZ!hb+I-2y?HPD(OAk;0eNn z&&9pb4G^yTE{p4cEe`8au(%^Y1Ij!047s;{nMGX6(=%1QXew|Zz)C2j&S?%ntuKgj zp*JGev^#V|ZW?Bd!ZgJ%dT7*MaP*&Cob$y&(4GOnD7TM7fSm=c!eoTp$B%Xxzrr<}o zoVoB)IILswQOy0Wt{zjUn=^wtR=wM7G4Ijm@^bxtzAx|(|hV!}`t9);RDL-n37 z@5vB)8QNU6A|utu1^Nu_i;yj^s^UN^uK66quMlmcOrk8{K4}F5vsp&4;?pxJ1kLNep4#M&fe7>gNsayeFV$nk*yp#nVcS{KnZwe;uDrvL4l%3JIfxAy!2z%---dk z&W6KJeZ*fda1>2`<5eC0B*uiw??W&j>Mp$Ls`X3_Oj>0HbqscWP1ek;d~hE29+UZr z($MH&(B&=>EEFi)w|?ul7o8fFdIp3%`YuV_B{t0WH2NeJT90kAqq8jZ9}%wF!){O0 zMvBA4X{fX|Z%6`Enu`rlY(b3(9W2<;71qA3w=ldP3*4xb?s4VINc$$JFh4Wf<_GH?;H!NF=OhdR*e>`CZi*S2Fi+ zC=r%l`m!vvEKJ4$%fi7KCJAAV@h|-+(#P#olP=_@L`piu3gi=+$VREjzIjF2De*NH z+P=(`Mj2^+N4&FtAOpC?QTT_MpzTWi?X+e_l74Mut?kIwl%CQgy|K*VMV6o=B=CvO zQQqX=S>;AV3C&-j)a_caWZ59i9_sM& zinB&bywMYJW9FBjXe55(Z8=<|s*7c>62--vlsxGC*Qs)mj)M2~BPUV$k(2zlDuvYS z|6`WetC(1t*%|%ke9cxZub;MOq`!HzHl@$nX{RJwaU(5MPC1Aoko2h{1MartktR5> z@bZObBz8FfEgE%YV-c7ZMark(6cDK(!9urbJV@nW&LvRkOJF-I87prCL53w3IyzT# zGOi{%l+671^PBB<7he_!^F7|n%yj}U+yST}`Jr)BNMGGCYu}*|d0*wRa>v@A&g_l` zxuJ5edvB=x#KSQ--W^JjF*(^by&-t0{M5r?b`JZqb2?^fN1Aze=?*z@I{WlHE-aTu zxiQ}#!wk>;JF_pa39qlxP&(<(=^*b8-LfOX^3L?it5k=<7>Mn8Dun08=rlspR=8jtm->(iEB+V?|AlEi-OLwYklF}qU{Xh+HKBss zM_-Jk@OTHO?{e3*6$c~7NibMd?cbCm9w{x&}jpgIas?x2WPc*>~==RKr@<^ zK>n!+7}z2kYY9Nz6&taajJa2#V_}Um+;a4XU1~1S9P2Aj&vlFghbgFR_?VN%I3saS zQa}mT=G&XMPRBHgMY;nlZEglWXo*gIhR^{|BL>3TZnw>iY`Bq#{nXbkdQGw@4Eysq z4GD`lF?dGH-_r^Yoreb5ngHteBIh)kqb)H(iW{akXG)`j;ANDkYc*ZXs4=&nu9N2= zQ+6{GiJ?#)qq-q2V#Qp6{79^$2}Y+vboTX-G70hxAzEF-X4&WjMRb%CR5G%Q)(Z3d zddNM$4-PLWwZ*_ai|CA#)U_YE5RxA1iV#ZSf#aT?3rou zo)ev4TV9@_IxP)I#q9LBWo&xNPD8A1nZrsU zHN<6Z-n35OnV7?-3oEOGjVH!-%eGCe^v)K+!`iy@J=Ut)B8 zqPMT7odQ>0qkDDtId`Z1=I+Jr#!6VRXV5Y}g|oE8(GSxjQ|2%JLR{ew{j03Q-f5nz zq$pAA9QbXtWsh{m`56kdQuqip8KDFV^zfG{2qgFPQ{jT&E7aE^M7q`SO$Yug zMu2Dd5I??lK|B4arbJm+avDJq)0Kq#pH0-%%LM{^%lXICt`GEr)rC zXmK9pKFwU)MW|N7S-WnATuP6E*vLI~+6DG!(FqmuMLRchcdkn?H+cWZ1KK}69knMW zozla+G^O#d9#R!8!VC~FKtT2-HJE>o0aZ`U8Hj=Qg%(iU;CltJ%X-Mp*Bsn$!gu@y zf$&fDjsdqM*``dY`e2{4;y}PNcJxbQ@ZVimFvnezD48tep`vJ{BlHS}OWhKDVnE%H zgj5=(TBuAyN*FQcCo@IkK(Mj1CbGSlXJ}0Xb74c3vy>;A5|QthawTMc1#v)xb1dVZC zhJc(Kr|zpJZXfV1%KHKhCR&>Y#d_4euN~!%uKWh{XTjl|vmv|py%TSyvo#Do`R}IM zFxwGl{v!So|3z>gQmoS9%pciGA~;kI)3io39PDszEsyIN>}*!|UJZtEDVnSKi&AtE z#3hXyZ%}g7h6cBEc?*m0$5~HzOD<}#3mu&ZLG&iv61;F4(d;8Nk9}m+_TE6%U;J5 z42pIJBB#lU;I@}&I<|!$Mzi;%kymSmvul*bnYPdSnNX(bqXSIssS!k2-4+`&JjDCad3L(VLYXuXrL&w)FX zWRB{zb3`xUlXAms7y(wfE@s%bgJHXO*CDb9=Zina$eb#c?iNxgG9I$E2SKXXy+9jS z8Bk?x{v@T#R|=(4X(`oZmdLC?p4cs+RlQ5za@ADjj7L#ymqp&g1 zKch+@+l`?X5-``pJ%JKrm5&h185|Yf?!|MNLrhdXB#|C<1U|=Ht**%R8$5H}fu4L) zJ<)2o-6A>D^S|VntLqX~DiBievZzf*ib%0RPQ3zqsyJXS(DKZH)k=s^UPgiQ>G%{Y zsS%8!j+JOup@lO>oaFqGDyXfh>h5{$MMSV=#=k4m$`>e=Ir$YwM=0LXZbqyvRGu-l zmo?cdK9)G06zTRuPOZI=DM}Ygysf`{?ru@#$UTB`>pwc-*xIvh_52N3yeh zR2jJH%wqbcJ9`~U`N~Jxz27vPa3{i#3TEO$nozeup`pqc@UGNOBOBMZ-IX-k6wPad z053dSxOh*rjUwM-tc#w^q+pb$3L)_qUz)#uT@(!bDQUOIR-l-3W4#KJhe{X#q963$ z34`;5#eN|PO>e}i9~!Xf5bQuiLSr0-iiiTDN2O~R!5$d}YKsuEMLRbXwH+BXMoo{J zcFOF))`;+NtlbE?8cE*xY2kW4(%PZS8<#sm+aZQOPm< zFgFgUAg?xPAk&B-aT-Xw9<44SFgi-O&(e%F5>*sW|57nrieWgIMk4b)HC;{YNI3DD zOw-13!O~I%hhd^LHEE93AMg}mw8PaL2; zq!|<*_v(Pe7>-1L*nzkcs*1>sTeg&N>u`kfdQ%oAJ7h*)5mdz*A;6aSGNRO)n~r1= z2>>hxgkI>CTuKYy9uTlJp?RPwaa~xQ?-@(?j%LpI820RRT{C}F=$YHFv>cnQS@?ui zzpt`?vvG#!eXSqzZf(8}JqG+&Ow9$ei@x&{RUe501SJ07S$a7*x?0&;d7C*iNZC2s zTA5h6s{A}$rR-hIod0)zeXE9!kE#ZizXo$MUpSt0+)c_FEcSM04mfozTM(oa5&TIY zoSan6=%i_~E%&C+PBR5!OB=N@!j|=-x}}YdWn`^psc;yp_FWl9@53UG!xx7Bd&HkD z7sDi3!y?x+ueE!gzN^o^)|c{s?-wROI!>$Mknr#KbR)bOiiod+zofX0Hc1YKnd2?V6sAC1_m0#d-BjzTFEwvQHpx#rr@bNH|S!QN> z$Vx#|nJX!-qeP^is-`1tt|n2&)H5Xmpk_1#^497z?4^gds@(V#&^Q{u%jJLDQ7*N8 zYU+|8mYK8*HBYQ{;BIu9;_rlOOlI2}acQ0Wwu!e)nkUSx-E=O;0- z<+LSRHQ7^zyPbfCyQDa=NLlWDnD(33lLlXWc2T;xZ2JT*qWW~$%d_E0u_W!{ti_@VWW0s#Fg_rU)3M+hAiM`V6-Okr_uQHW@G@%xm2WKk=o@X!c?4{xM!j4sAUyx}yJZc>xkXQk z9t8V>rT@ZB!vxXhmz+MMK|dmQv$TBtje40Hsn1!G+(C>Ai>3^th_iGmUzjnax%#%j zX#J`@t6Q_4!ZA8rQ>HQ=zNY^!&>Y*s)AxgGOKs_}ycuXQ!%q!AUa5yu)j{o#DNuY{ zi;~*x6>mtT-TFKu%*x9)34UKgoiTw|(YNtAGZJiAi9ZO}$CW2O zz=Trw!$`5)J2m~rm6q(#!|?=V#EdifR31&ZU)#gES))1L`a$0S^!c=HA0PQ#IB(^> zfe{v)1hH4Ht8|@>!}o@5Hk5o>Co`68Y3t4?^1_n|n+tlVFd$4w1GeW$&xm(Yf-au& z1hq0bKl0(?L%^*E_V`Q3DR#6YV zfSXanSV-O$oE|+D{Xv(8)$~DJ^;ehOkbO)swh!aU)3E2vC4%wHB@JGES3{6p=p$VP z5%_aUcO-x7LbbeiOE^E=a&brrt^`M4&<$rOUIYq7+oR}Bf#qcnE+9XJt)YhI1>FbH(+|&hyg?oLzkHPWA z=-#gi7jYl1z3?E~hg8rz!BZ$*8k~2)k8dDrkf!3aFM!qqUZw)a#^&5=#bTi$bgU8f zcIKdtUSmLQ23vDrmn$yN1~{A4%wC_cJ6s}buvhOkYOX5K2Pq%rLixwEm<1+ZEEX3m z9Ch_jV+_{;k_WKO>;KE4QNC0l>F(`{K8TzG^Y&#zfyxx{O;FvML$%ff#<2S?jFD?g ze_4M8-Ip=wr`H!4zx4;f6O@0vT~X>cids=2^}#IR&S`n?UqH(V(V?dba9-g4{VzN@ zJ3YbeujfcFFmRicE56nn8jyElg;uE?j6Z0^FQhCOF~C@xY%!tU;=(B4SPmjF6Gq~{ z*PX!P4bzHMVm3AIc#<^PA+V1QE8`caqrIs@k9 z1Vjc|3myuQ4T7j50uuyTBzRDl6cL+lizqWRwKr$iu|n<2u2y?deXv-cP5TFUBAK97 z8@IOYwX0t##ar8~ZdvPG{^8*1-X!gAu%XNU+sN|gt@qC#&;H}y&GG)eSAQUg=uG*Y zKpZpLQ90(c!*hA--I-l|4Ew=a`Y(|7dz0qSZM+TQ;Yc3R9V(GQJ>)j&jkBMoL%PEG zAb)+|_!ztnjWJBviHR{k*>{<|4uc_3WcMk|cnG|=R9FOj=R$A<8P8BE4=GR8G1R=T z*3fZN-njh<$`KU!FZH26xSq;GD#&?hN>SOko?*Xy)rVY=^U|4;(|ZXI#h@4_Ge&#U z$$E$n)v#PTUb9f@k6&k=;Ly+H?7xGE_KdOYQ}5bhsCP!=J>Kr?LIs#-eVF&gA&~h~ zoLK*k#K(J{m_P-%A21;ECoV?xKP;5Td}1|!=E&V~Kp9BBbqDsgc2#zY59kFHb&1ox zRtDWZtVP_yxz3|us^P(gl2{Wim3?@#0$UJnRIa+wFM)uNJqz`A>%vvyRku7Ow0G}Z zA-qqCfGO@_7B9K7A+JD9TsmCdTOF3w(Ff&FmghyRYwT5tdA6-?<0+ExaW`hLyS6!( zXg+$0N&bPdjGK2mEe&hQkibi zU`62(iLQ}$7{JM7aCB5MkB;63vt2bcYb)a@V0(Gfu>MeXBikO`J zp3x_gfJ>mj!KX_QSlMZrVD0=DKq9gfVc03IBAMY+D&B+IVM?_dff(JcNXP+PMvojX zJBd+KPKM+>A4el1J4ijV2H;Y@eDr68ZfEY(th&GmTC}wua z%$iFX4rg%s2v>&eMBCYottmi@O4TG;U9e>ZDJj0B+8-FrwGQU#VrlICuq&0u9u?0v zYOnfjDi@gz{u#YIwgGV&S~^2?6t}7tG*>k@XX!j6I@|V-BUX}7BBna9wW1@grvB2K z^%3f~*7u>kx4YF!eyhiqHXmk$Wzbxf8=iKANodd1tp>?;b1}+M@gB}u7b%ts?umIc zuZ?*ZxOGI`0U=bi4Nj4)Zq8kAReZ$+gap9*Yf&*K>}tq}*cTMXMav2Y_>pda`7R9B zZJ$C+HO_%tYe8xAki~|&6Xt#X^fF>b^PalhA=Do3;9+9rJ*kmKs0b$jT7@}!5x=;$JZr6~rthcQFD^B-1~OeyQj;l)U^8k0sThq3 z90v?uTE#sGfRz*Z+j4IEFt%lhMi~RrhI^Djr#j??X|=?$+0aB~f^~&kPPg4tu1?Sm zJeaN8dt_P(v3A_a06AkKqO$zld>}NQWYwCpE}QfLHId=P#B2N>DTEWQyb#nITYEgp z)xU)axr7pLSqy1Q{bbVq3%oH3!# zE;R`CmPhqmU++fR6)c+!_DtW!n8ljKrQcYiV3>a(gR(aU4ZQy-PA?yL1)7E)8=xGB=897oQs16k(LKS3&TuvY+JT;*(0c7;iH{tvnw4_O={|$#E9M zK39s#ewf7Au8t(>jWrUBxbz;Sj{uGj5G9 z$HNQU_d!FnUj7!Em&dLQP8A2Xkh)Zl@YlR$o?!K!8XG}sDFD%1nTkg)!l?!-J{|o* z=#JHL9mkM!(tnu9O1~S@BD#%esB-rZJ_&2GO(D((In_N4ZP55nk+_`$ZiK0;J*9{Lrkt=?_}tiYwcc%@j-344O*oLWr2K&Bh}SM?aT?$3nn_#Xcg zjg4vE7yfGr+%dR;WqtO*`y10(sN2Zzj26&Ood#dZr4v#{0GgH=tswCXYFOs)ycdHZ z)V6eFCj$73@u1rfZc4@>=Mnzc8457phpjVP4Je1Pzl>9mK ziw?(8q1ux*9EuYuCdUm(3I5?C`ToLZLMo3C&Y`Yy%+d^9{)J%?1vH0b<5<@bQ`d{* z6tocxGwQBOdpk^aWTs2KHvoQ*MUP={1RXt)n)cl|sT@&F>*5$|#$VUGa^x+8Vc+uR zn0$)6V=Q`}idtQof;W_M+C*s{9T|XdJ#ws4GpHSv{}UXTQR0pqKdUKW!c630!q=)B z%7I0pB6H{&Gs@XIgITW5I%Y!$;|MK(I%38Jvp;T?g4RU~;LKgya^f+{7pkRIBD1~& zwU%nsHGI&ZI-~QgWo21jemcu(Cd~puPq`H~i4{*R@pSUYSF0CE!=+4HlgO*&LO6Wfh@LlD zXKW}7nnGpWJI7D61!;Vnn!i(52B%Kj$gq1hp4+=K(XEd;8SBLJ@pn z<5uZGH%a1FVi^=%k08s(BY&sc=Py{(>83hUN& zVR3v68;dNAjaW}EcB;e7npEu4hO0eEo(pQ%#M?u_%H6|QofxQhenhmj5l+QSFi1qmF>a!2L_o3b8LaO7(}13HjTMz|VfC3eEanNU zp*XV>m=5{sChVV?^Ni*kufY_!Zh4yRiSMv{&Etvi=SZf9g4WZ(^cXZT9AT8!m$d4> z(Q=UD2+Y@QY%&^E$#^*k`1m8EG79Ok3(&VQ|Dmi*(Z1q&C-7$Ec>YktgfcmT1yn znk%&K(HLOmgRMm)gDbIlP5g-J#QL)q5GC7BF;&q`sy<4p7$KSpAi0ls-bFq4Msw=j z;)H;nI)~Q{v6&^=FeM2Ey2Nj>AJ&W$GqN>EV{)?mx_DsGwdkqGNOGcWHsom%TLVF` ziVG7%n>#q${jbEJRjp_r)K#<}=O9P2 zEIBDyQUkwX2#6QrP_v~crRW+tm@t%~iQSDU%O>pZ?&jPaxN$YQ;x7f8UuspWEK^uerxO7>YfgdorNM zI|i)jJ1;DbV|1+cBN#EwC_Ch~$3O5;?UQ#$?tiaS9K&HVDVP=dD2qcW$avqAVwm$% zjLKo549?~?NPX4B0kQ1R2211QYD?@B_;O=m{iNqOl^`120q zV=uJM?(Lw$q3?5IMm>Yi_YO`)aEO2dvW>#9S2M7$SFuCHNsUF1V_T4y ziVMfsNm@h;3@h(o7*dP4~bGauvqSsudaCABJ-)&<70IE-f@n}D3y51UP9)`ux8ye$v~PV zoyo%lM$BO09{@p-t)(dp@8-D>6eXj4XR8s@-$Av+MxYKK^rka@+6;U+5(+XqK3}K_ z3yEy7#+nh;Z)K}k1b;{j6$*mJxZz&jr~zd5iN_*c9ASQUH387`e-K*;>jl&dMJyo& zifQANutAZ;ah6r>N)26>_?#E-tD|+iRoU!maLz`}OJyvS3<5_Iv^%QLDt@#aSHRZ} zSV={jS#2%=poc9+2}eH0T#H()dyG1nMx&USOZ?ranngRe*R_(V3>J>9QEqlSu%u5j5_2rr zHh=&44y|JLbfLP)`BEZ5WOzqSX``Dax!0P!e+d7_DM&I)icO=quxRA?P}gc+gE3Sz zP`cT*_3|;gOZ+n_82VXG0I*-RxqZxN&JjwS66`pxjjjv!W9jiLqQuhb`Ro{d>rBxG z6(mj0&hKu3dPeiqYdRskO~dq5{kx21|~vP79=4B6W&?u0w>Ge)Q4t@r{q# z0~3m+0^nMwtY)z{MUOruhJDZyqFe?lyc@bc?xO8T^xW0nL7B7Sy!^lO3#aS2a+BTb&iW?)h2b+ET|T z*P6eaYqwr?5Y9<|7#d7)IF78ZaR>fGMUcjwlNS#&MlNQ1MpN6tZ&TIKzC3MvhClzS z6qKVE?r5R*#1K%qbNLb;VnF+X?pMFV`BFE$^=uB#n$rEn{hUK>KOiaE{cXMShyJbo zLY-kAF1>QU!JBQ_;koRUGyL2A_;9_}8mb)i3%;LVSk?W=Vd!?euOfJ70CgZy6H+lKN8&Q=jTep z-c37ffAf)b{YUW%KbLs)axC`RSD0Nc7xdqt0B3AniP>ieZD|Hvm*kv`g$};xjw1!C4V2hk@J`BYW(!zXi zCF7Mbh5-PZg<~b(Yw-e(mPLPq`Dq1w?#1y%iM;sDEqX>~2%arYsLSSC|E94=f)jMJ z;h$DrI*k{WC->jE!Wbnq6AAOH?VFJ0C+SJe%!G-L7ArA(+1z|$#I~1nI7J^h*_5c1 zH^|sbxk*u!KA@8bJQ@fBfaY#KZc%$(BzTxNYjFV;AHL_-&}=m)p=NB|N4?nyPUpK=3zKj;H7 zVkw7(bw!$`&k@y$fHsP4?v;!$gz)WA1OJZtq_0NJxig&iO45M ztN>RQ$Pf(2>`q9o3U7*n^whVL7iB4~I4bW9e}bl&~*k)_HC$ z7~7X)IzMzPNu82-YVvmzMoiK3^NsxBMlXn7h;*)n&+j7M{oXjar6A+$%9Zl?KEPrh zZWiI*NTI$^W6?a;;W`p2Guu8|Y8#|i>QHBT{w-9=4mS55Vp4-Et~sKj$?4~}=iR9z z*G?6xm0GHZ7sCy&b45gF^}r`n@iR0_9UYO5O%XJ6p>q5Q0fZC>U>L-}Dw&392`QWC3K6=Cg)sfq+_` zDa8&bt+1ngK*tx}66q@I@2iTQ^Y#@H&CCrmVsrb6g=DyloIj)C1IQGLY0kb2L+%nw zIzcJhiDu-C^Fn6N-Ap%y?X+0DOsA8wN{nXyJSic*@rGy)6eVkV-hF#4-#5UdFTR>j z9qT9t>o8$$J2JHuWAiAA1g%EoYKHPq2HcjsZHNteV|YE=KONz7x8k%P>L+;-;17O# z!Xp?$A}m!~067>T$R8T@g!dK!eTK$Jhyh-rtvCtKj+q)!Ax5ovso0OH7URoD={&`4 zhnwAbZ_9@ig(HG5$|h>c62%vPli}V<-u=T1g%m-$&xgn*Zcf0yQnwwmu$-ro&SE5{ zB%IS1H7ZCrkrj`LiXvrK^nK_9i5pR!S)PdX!tP{JAmKQ(IvM$*ZiLMzo73m{;L^j> zDx0321hvq}agWVIO4H-xbW&KtpX(K**5UH!<)62qlq-KI65*dm}dXq z64Lc5iR;Z02rc7no5-~6n-GA29RBY&?+T8xKOFu~e8m5x^8aUigeHuy>T2e{ zfY-U1Tv_`-QZU>BX5P>Q(}*52-V}*~1w=)B`n5Vd@^sT{(_7BWK*JmK?OL_Qwb~AH z;j6CN#o8-$0^ky~nr_XXzr+11N^n(A=hJa_Yj?}uru3i7S>hxp-q!aGdH;Urx$im8 z{-^K0$Jyp9egMb-tQP8k`8C}jzi%g4_Fby-PDua5u|7-yULb;j+Eu+)$GmX-uTN0Q zQBeP{*bjfcK#2Qxe_qsq{X-n+CIdJG8s>OSPu4(c-ZP1M5d6F2f?p!&`0b7xKRvO& z(?cleahjaghm%0$N83@$Dy45eUm@6WTAtj2m2Yv_d5w291oHsN_YQ)2nyG&k=>JZ} z1!$-Y?vA_B`z#I(Xnh3&xjY4te3$}({;tNS)8;5b_u&eH%6re7{#&_~0&t&>TeTj% zLZIt6-c<$nnIG=2yza|1AlF0OiF`@IIe>M1HEz8{V*6Kxx^=c9vHRlAv8XhkyQW#x+Ba#ZM|juL zO&3R~8Ic*&y2hb@CCYIv$t`+M7ei)zIU0<$3fgFVGuc6m87+P!mxw1Xt)v$1b}wl> z*dkZbFS|^zhbqH4n|q^^vl2Fu#z}-GM}N;Qrp(kj2f}yRrWtOy2=VBYe5vVUOJQM? zESFQdeg-66NxBJBmY>}byn6m)hKMa=eMQFp4lQWvy8J!SCh2Fvjb85%1viULildkTci z$>UeHb%|+k7jcSq_UM^Xz-N>vX|B8F01_w}2G|xc-n(BRvG@+!4DHWnl)H! zR3tAEo?Wyo1CvH7H?4|RvFmBKQW&mR+QA_-d_Pk0mq>KjV)!B$*qH%$$Uj=j@}029 zMO`Ej>f~3kBH5KmZKPxuix`(Wyz4i?h~O_7DQ+|=S6erg8ximn==P2-Kya3EPpv}? zG*P~z#6PlccB}L5spr`b>CouTa4oWxOC#7^|HWek&h2dG7tFE{ zvyg83;lRlt%uofIuu<cg;C@S~b9D8^wI39M2gTd+6JOYX#%x6k-8$F|wm62WVw^;SFU5l*B$xM%7 zcZrOniO6lV^(s=!8b(Z_HGHktn?@PR=*W-_=Xle(4P~eBM8yl3x%ohE3Tkbl6n!$cvz&R7VfT%ly=muw5M}Ac^6X1W zIj8Ft#!^phq6PKl2&2e~E{{YtZ^n>Esl+vS5?_uSvq>&D*vmPu%%cnFhD|%@NJ@5+ zpmnItO6b&>qtZPUu^P=>Q5i;F(6$;4N~(Ts;<$q`M1u zoa*(%ZQGY^@C#h05iKrpy2s4kRK`|>a(o?w8 zE*_RFc?}jjMXBrZR*u+Alb(rX(qe2dipI(&Z_=88Aa$-8XMr&(u;1M?%^lgz>ESeLA&akuCox0!XVY&#duDCXr_P1B+cH?J~E zoKbJ^>d+^>2UCrvGKlNvqjlijunfOwcG1~X*GO@vq;YS?=pEau+~*F*lBs%0F><$u zUA!pPkQ}QsyKUixu`kaB3l#0Pmkg!>nij^;-YE0o&jqQohd55OX~_OwG`Bp}klCcw zbk{h&kD0~VQh&7bRaFv->9udp#@FHROw?;vQFm?rM(nKeUTp!1Y1OGSk5xvfm1Mpo z)pq^I5;4#sRA5_qJb37*Gkt{QhB7^2+1uviQj!}STZ8YL2O$BQ5)6o>XcndZ#9JBw zfQC*_{q-MuU$zGy!Eg1;_0N#w^MOc(`Ewm@%UPt$Aum+_$wP`Cvc@jav%J~8 z`p9{-g^0kP|2%6>vPT54gjh!y9wob_on+ZL6WJ9R>VBP41UpVhH(jz*D(~H0T;mjQ zUw_1#`!5;AZ%iLW%6joCtj7z`0B1+#(M_F?y&KbV_oIk$W%dyxv64Wd{H>SZBjBa= zk;ZC9FV{xCFkTIHF#*cET8+jaBi0o67bf~>@lSZ95wC8-o0iqQu!&)$BD1@wUTS|B z?HslNUzd7ojVp3zMC)Xop%;5GtedA0`D=ye{55REk}kVW?U4_oZnm_Zx0+5`r*;7N z2c&SCt7s-}q-euOXJA2W%raVTBxUuSA=2uYxBI`8i|BxpKineO?uJpAE&SPn+_+6_ zY7JFCMMx*By$9a`e{_%{k&_RPRqm_&iE`BVD2HEJP80>OjX~IrSb#uHR2NrS`gq@; zg;qjRj>kK-#X1HZwB&ApC1-~j@k2TdiX}MVLnfWDdOE<^(@bDAikTr0#b637;M zHr0J#-EW9I!9Eq$ZeMAu?$`;!99%qe+Du%&Z)aDDWM%VejB_=xPG!}etf=;>;>rJqvU7;eB;49|C+XOB(y?uKY}>Z& zbexWD+qP}nPTm-A?EHP!I+Op*zClf@Cbg<+t>>w|uYKLY_X}(ufjJaRCAhFl?lv4| z(quZ=K}9lsY|PkHnxP2mf93Wq+fh~%FA!ci z-K1b^64`x}7G7BJCB+pbcLv8!P?MZVrI5vWjx%FLam<{W*rtScTyX&TKTUA}WV!ni zeWKyPiqWJs0l7}JqA>Vd+5X@Xg{b@!F368-2$5=O;nwv*TXubuYva0!89vaf$bftw z5@sJN(-ekdf^hi*FV|8NZF&ILJK8MFdRTWsG> z{>j(eL+5l{VtoKS5T)n&`y_M;Uo@|sd<*sjT_Q0#1l_}+oroMmfUB3@orlnQ7<;v1A^R$dv#&Ae@iJd~h*-cbeY~~>Mv79!H{fRbKKe zof>~D;~^RVBX@$X$4)1EIbxrwf&n~Xh-vCt*~3eAt^!_~;oO_CvR~gWJ55{3(NOre zHAWeAec%^Znq$$HmuF3{k(qFKXsXI93wDQENKSpK(d>i;Qzxvq=(;JwG<$2m_Ag;E z@Z5<+tq@165>*8y&t#!{IGr75z;afHZ(DRon~9fVP?!@WpJxB8dSGtR-aRC=USXBS zsx#ns>_@^Kn~NcjDKp_n;QQW1{Hn!yXjD07%uoSO2NUlpQBb6}qaB*-Tv#{vg3Is4pNT zkHbB*rIWu^N$nnAY!HxFi}}xSSe)bYqFBZKce&k?f#1nLIiZrx%0g{$15Bed(7_)% zNR5Ghn)?n7zuhm6#GA2ni+fxBjiYQKS2}u-njJy&-FweiYisr#*y1rfi*pypOC(<~ zl4efJhV$CJ+&0;U%Lb6U7rQsp8KDJw-lN?Q%L1T3m(!J0*uZ;6E}aOyn+Vwbo>>a{ z{Kb5-o1clyYQ)o;y?~;4CVIR>#J8N+{f2n`96->2;i0x&;ib3iG5fl_bb7q#>qITra9?S4?~wFT#eu?} zpaRA00tuzJp+1lTDTgoNSwoq3+&c*zx6Yohy>Yt-_`b#>fiKZah&MRWqGg{q zQOO>&76aJE*?nzNaK@1y(P-~wpnrYBRoz$n&gMmIO!HMiIM;|CHw0nY(O;gYL}*2MyF|tXXh0A^BCTJCG+lO zd6qE{+f zU5Ex@)dOVXkyn@EqsgLu=W($neWx%ur!7g<7P9a_?jHt(N8oiyu<0|5F%f*7z@y zcNr?_7M3OwNG#G4SSQI|r66+rHxvp*7K#jNxX`VEM-sx#yrr@8cT;t9@v>eEK{ee-TkS%AwZiKK+maJ(SO@RBb!4wrliE zZeyeHFm@05IQmMrrtQ7sVK7^?>OB9(Np@SM#?*b<;5*TLRPF4Z4gZC-9BOP)Azv#w zxMTMK4(~2tu7QRqZj;m9{8K}|clF}@Z)7&VmEb3?IEt2tz@JkMHO`;?1OHte=6_wQ z-7_Qv9A^6Q2HI0`{^HnGyTvDz8+0!w+bkxIofv)g@qk+$8-1Pxa^}R5%2U4t1eX3J zCuij`ErOfm6~B>&!SnUogs@G-h_N)2Z(}jVHJc>{VKcLwcjGG-ZB8Or*fd+`Go;HG z^?Gy|IE8LK#z*azDPWM#L=K=DRyanUCm*71M=isRHW@;#cbGb7BG#&sy;!_*&<+!E z^<>~MqT0r=z(P!qlo^1bGrQ1gsx4NJwOOKiq+DHanfM#R^N6N6!4plOEM07?n^%16 z0^~Gl&x{GSv!FaJR>v05OWf%(wcL2oFU6xm!6 zM-nWQr3fL}Y#R$CM%3NPISp(q>$dB_f-hq!74kG)ySfxyGZl=jKw)X^WD>2G5Mp8_ z;(KaEEZ>0gLQT5Ogt|wQ1y^rby{x-IAi}!*1$i;5jaq{fiWjT0n9D8D*os1)LGr{< zuyI8qK8D47S#5FYDkUvNM(Pd~L5^>cEIzytb!=v{uv7v z4j%i1(aPjg$||%o#1XUh`I|Djz~C&nS?oQv$}KFq{2?~F`5_+q>;YH6+&-%U`s`t9 zm-nb|ScQdfXN3O#Dz%zFpX>Ri3 zAZ)g!L*)a3L0bPqZ`k!RW7mlt1 zBUJXvmrkD#ux}M0!v~MGu3;f++wHgIwov(6C5#?wnSe*9mQ`BDHV3EyDi*vFkkE4F z4>02Qom;<_WoHj@ZT*(dVkhkFFc-mMla9hMBt>xcnzT60(!2QS*m z%e1|EXCtzBE!xJ-Q^`dP3Wt87*YuJR?n?wrgMcIwd>&dsp+CK--%3jCpXdlp{~LV&qgJ$9o@ zVmzx7cF;P7Rs(+2>X$||jM&h*Nlt8ovT1U>OKRm~Tl0AKpr(k-jg_|0gBeSQR_Us2 zNq1kBqbcnp96DV;?^=f5<_7IC*Qqi>1*f=p`c=22d%YGt8vWMLWTnPQqMcg37MD>5eCt-T5JWbQM@(<#%w#N-0EN}ok0=C|R?7&518wtOY zY(7q1_Qt_Oc9vL>M=g(|B^e1iGYh8<(@12_=D**g=*V5L((|BsaAZyq`xPPxA~Ttd ziT8jlKo^%Pmwik{*AI2Bi|5->Xtq}U!QFQ7cj()R`P8^)lm=u1p@8*A_#mXmK}t;l zD0zV{ofIdoe8P#b2AkGF6TaL;#fcM9pPXo^h^&ZvCfJ)%&zbbWggDqA=B`9Lke<@| zB5<*2GQrRZL1lKFKbj4pjzKvJN-Cy0i#A%Fu)ZFn7n20b5ZlFfG8NYMum^@vF2#r4 zlDw`fcERi+Sc40t_I`*vwB%RtH*SVS_^F`uS`k*fR^au`U*&^w(kC6_Pb|nfd4@G4 zCB7*t-z>;nk22}tHxiC^NH{ejjy@Sj=*T(N$KduY8OmtiTw6gpejt^2&&B>A3IMyE zT-6Kfrh_Q^&OnVcWgvZP90r5_SYD3daGli3dnDRoE)CTpa| zO<^#oJN4EE;1AR`M3$PGFcam)S43!!4E2i}T%b3dfN!k`1=iV%Hu4k@?;dd*c@zv4de0Wpjy~@i!2> z$COI)Bh-a1<%il4V^j;Ltcrpa*e??FB+Gb5%tAV`en|p;U8YIC+=Z89cGwJd&b!ih z$&hnRsL1LkAXz<-=1ZY?8lseqyWoGGETEf;HFW7qnZbJ$!QX)&TL8^>JFch;uED~kVhB&J|i87 z$1*#R*tV~&-^N0Mum7WpNom6>mHgKjbCLb=gYW;i7j|;BGZnVCakaJkuUEaA7urK* zdHE!ZgXN|>Jrx}^$|i_5l=df5!Oy4$YEY1M6Gz+xNv!Ot;Pjs&7ojWY&sxw{syYXO zs+8o!YV#X;D{9)!Y8PtFcCPtf%N5mkxz4+p<8audx8Ht*_t%}zTaCHSv+WGIv!A!8 zhd&rYYk&g?S@r{NI(dge4F7mvJ&eiOaBg~FpdACl<36*F-JT?O&LudkeEhSrEAMn) zUJQ*()uRx>${7}Oo!$=|7oD?^p0=^RI$rtb18gj&Iz_z#{wTx5m`ax@Zz2O!+0(Vl zko>iRLn{WjQ8u9i83wqKW<5Rr$-YL{kMMvKsJZrrD_;Cr4$n_e!B)9ZSl2Fz(cB3@ z4AuyJf<;5Yc`h^Y{c~{SKA^ZLj*yIkV2QGm8DpBMcK`KYQR) zmwV?bglvsm;2A3DIm3f+hr!7%B^UDNG=u4GP~bNae(PzQ>gku%s4X{tm2oGg(As{8 zm(LI^e4Fkb0Xmo4*sL-4*9NSj&sL=To*W^6I4=0HGN@~3KreDMU-h9q{3Tk@4k*e{H7Ed+AOM}5II6}HRaM<+h-pi?AvUwlivq%_xXVJ zqqEQF*CgTjOFX6>dK0&w%^n5wD+K(rfj{H0gT7=A z`nWIlV7>ACtq^_`6ECr4pS^((M6aGJeK)Qj{gK{KEy^iP2UjjJ7}KDpDMHRXA9*2!+Qo zeZ3r9>k5jw@XFIrZHyS51>1;lZ0y2WzV4aTDT+}oVK`KRaWjd9$aHX{rVwi2hiL#| z#QgvVTAZjUTbSXX>H3$UPx}ZqwlLg5^YSa*DcLGByEoSGBG*Ra{tg8nE82wxELlbh zsAkBijm!2xG9Dh;cmq~ZA2Cj}a%svt0#W`Lh5A1q?CulHj_cf1c<>zRBvDk?Aofp_ zyk^D+W5ZB2<@FtL8Wec&%T25`P{WuI>LMN`4sKEV7?6jCp6=wj(=>z7!O)Nemcylm z3V;_8sj@l?$<2f5?7D&)&zwGVHH0JQk zzfD-t4E6CgjSr}HjfSGSDB+JESgeI*4*~{sk}MmI?w$eBpVZ2+)|m+%SuL10@_5z^ z|8gq3sQA;q>b+@B^5P;P4vlBCj+%+BqfJ)rRvaE%q2aeoIbP3F|BKl1o)6 z*5F=wethjZ=fvIME3VTkiv-8>tw4D9pM5Y4K!mT7JVcjLULM1@1G_Cej3-bpRn?Ul zr^@Xe)HYCC{yW9?tp^yn#+NA>@4bG z!g>Fo!Fe9_UUZhdjBfb2K7UkcZE2&Ei8hGoY@~A*&&R8R*?5Us%Dc~RXNJNd##0<|^Se|Qwvv7dVZa$LpoH$fexHd$uIxdVO%jI_@nsPdF z#<7rM|MOY3|51W9mTeTS6pr^i<4HfQv!kD{rbMxRgrUHIT8L}>Q=v2if|)c)n(92| zcpzblEGLjv6KI{jlvU)kE63RHkQW1`@;pfh=~DTO15pUFUxLrK3GVYK1U*dCb8P zhwD!Kl0KgKLmZTv53@7#az>K|L!Fg;SuBfwy7SixN-g{}lr*YB+zx&;iVOxX*sRtq z)<=eVrvyHxOx4ltzG_vy0R^vR|KcsF&?|^*B!P&Jo}x2PaC*csju2uj|k{ z36gWn)^A_l6|BE4BE%qm&}{5%weTVuYl=R>geue1K8d5m*>F#FV02E%)R;AJ;5b3k z%{1#tAeJpQ2$||Ivk_@V+-q1`sFAtIP!wqY0;l=@)?!QQQ(uGJ29`*)trI79WwAE0 z2I2i#JajZOo0ejC@F$BSkZM!=v?iXzgQc8>;v6?jDjOMXc6N`27gzCOEzMmY8YfY~ ztQ#x9%XXrhDD;sdHo}X9q0}!GQnX{Z4o7Gpl#nXSvUTDm(4m>JRkWu*hXwe?b7aL%ZQ_a z4AVra(Z63NJeGfcDs_czs`%%e<`Q8o%tiCAMHFlwJ`!SnkDvq7dC1bxB=+}V{xA2F zVm|-ukrp^{>-8KE9HaSsRuzitB8V@Geam?V9HKh{GexH8Bdr38;iMF>ri|*3(=?eldUB@R61hW9XnN9@R2jRQlK)J6XFaORnM6MDdzxz(oEQ zZ`QY5v~fLn1F=|m-R$SP<$ziE-!mBdu>FM=SJ_g?ic!nXG@nXFsk z1okCgx$~risOe+9ZYvDk>!8E|AhjnYLr zZsDY5RNc+#--rkSHEyE?P{-O!Wewb-t`V*bs(xMcSM*Ok`*cGNEnd>b%xFmxp_f!G%+UM&|(;OEOAU&>rl zo$kLXv!au}urR9U0_Sl-CAe@S^9u?KxU&uMms6Bp7U6I~EEmPpxWt5q|_JtJ2Eo ziAt;CIQM;ma!{s|@uOHYB!@dqKpR)oh>)!EcP!+;<<29ik6hysvkG4`e z)|N3MXM`LrjVJv??r5L!-v#Pec;ldJ5yZ5FoL~LV6VDFE#ZsnMSnAWf{3=z7wgfdg80_Jki$VHh`uJ&d1VLUx0Cmr+}*HMFf! z*KZCwaC&Lef5kXAx-^!?12UkNM@SW5(>8ogD)?RR^?04lFa(=|i!uf^xcKg|Nw%_V zY~M^M$*&?M44*YyUMCC+$X!ZE`^&scM*JdvQ{AsS3m+FtT8+M^`BFWJ_W)s1uyuDO zP!cj1O{E`;QCQD~qhIV+<^WyS=r-f>oGUP$f8AwB{a6GrtlgZ5+}9kbHj4srRuffr zKJV+o<{eso-%|aN30<4#QxX=_L68>i`&)_N*XmQJze!3sTBm>?B;e-eiJl-hB0 zLc{|${`}xz)UowO>dSe5s2h;bTu+6CNMST{{B;VZkZFAF6GE84xYfUd2nV0;Z>wgZI>2P3B8VatPoPRMeHZrs5>Sc<`pMbMoBO{NWrx57MA-(|x z;*8=~ejd*I9?=0hcXyCiRmURCH?8&;lZ`tVHp$RZqi|!m!=0VFt#(RXzcZtQ#;y2c z_*F+~a&!F{|K!CD8xMKKfSFVVG`6=nd*8rR4wUVM zk|DMUD%CT6jM8zFXiO&5G|&Dp-5*?}s>t9`{#^gELkR72TLbxF_pANuBwjs3IqutS zAkqI8Ef&ge@HffG{mXHSF7lwHW_<3|!KL|QH zLkS4ZrLjjmu%_r>j&EL`k#28q!l!1Sr^Y}j14EGf$rK=@_dWH_j7h?F2MN(4k8eS{ z#)^=?p#Ff$YV?6e4NhBEO7ZklZzJdhYv+~qwe6J&=}#lnrKBLv@%%aoSF+_fLXl4A zD7C?SpkrK7#RRCg{~Wl%Df>N<6C4=5cPws5qhWeASBbh+Gra-e0v`!k4iVLzGsT}f zfi|;aWa-57V(FY*QdTyS4en8(C_0C?9`rfRJG3{Y0S9HCG)y#0)XkXGB&P_8a16gSu2% zUJy=D|K0l4Kh& z_jNZcQV+=0q?^D+x8MHtaOFE#ni`IPKlXnkdDmJ_?0F0rQ7`=uG7=i@+_2UmzyL{n{dTvD=K~4dTLzJLL9zSF zD=ECa60d@I5Vtq!HX+S6iH#m#HlduUwyp#+f%p4ZZT{T`_^g9uAKutHaO!=_bp&>y zH2ZD@wn4a@DE21SJAEsgP&dGdQ3pun4siY;ea2#KLyj$oZ=+Lt1JsUvxnmC8UksJI zcwmQkvl{7lnFyRoz@90L?u?)t-zP{+WiE`b84?cXx8Ot7i#2lre?-q4^N%OwdqEkF zGt1o~UR(Fn(tL_DOWyOlrXPcwP5hGcUeZDuf~NO9e{NrtMN_*OzIg!6{;2bjHg+3O zHFU-DLT6&(r;(Mglg!!0NyZ+SX=FGQC3M9+`As80WOnM#m>63H((6R_O_0J0onv_+ zQ5x?BcC74&{e4xt(7U9YU>INy6HZoJNdJgkFsL8ijAszxp;mgX(>S38+4eu4v!p&q z)w4;eTjHjGZOU-~wr&DRNz^MMWgg<1dePoRh@m&XdC;SG)V@jB6nBXfn4wkNIyPw$ zV-ZHe(T~*I4n|J9Ob}2(%<%2fCo)>teK}tXjq2A3My`rYr2-JG^5>KFYuz;tded!b zqE>+I*x|%SNNkgysUubmZsQWNQEd|ZPN-S#TB07ENl7BUoO9+gqR><9|woVp{k0-CmL0KI`T9pW|jBF!lmakK_5A^eskFN)dN%0pb!6&ELdp32P zpufe$N>w`#QLcnL77~lE@sYru-jRR4^XtYh2}dzy?4F=~#@#d+k>hN^k4fB@@m@n1 zRrmbd(cQBw6sulPKf3p}?LR~7Z5>NRS-R8CM!MCO+VJypkFTX6nnsycV&Yl4RO753 zTaD<%#`(_A<2dlX%WA*}I6$Tz&aNY{K~-N;^JYXw`3R}j?B1QY)u(0})k;I&Z#-RE z4Z_P5=Ap+LJ=2fH9sC@jRpH(z3(HXd>jGoIGxdDZG#EIcQ_Vb^qoI*>A<WNb|BQE?(_ULQiuMI(d=oWiXlQ>(_^yMn%l7;CMc7YV+Wot%K56dTU@9 z5aYO6sCoA?7@F)W3xTs8U5o}J@h>?e@nr5UnV+8&eeZZ+sU^c|?uEsT1UA>kq;c0* z*HQ@rrzoS9rSiaXxvh7O-Qk)c%Z$5pYRITp!eLH;@4ROj3n)8n$eFKi3otJ6{wG}N^7$L#B>^4&DjOTe_`^Y6<1Ee;e`zOmz9)4?ioeOK_QuE!ZqeX3a#wYpa(l?2$A2ypSye2~XS%ud z6vVX*u$QtUaS*EY;G_I>a+49ke^ey5BjJB$x(3v zVUtL)W4x2UquO2V`N9d}5e&3B$&PhG|G;`Upl@Y(ZWB)LN2#HpX)7R@{=D>1TX))N z4y0L9-BQ!*i$3PAF|a@KatB+@Z)K)mlyy2a@m>UqmWFf!}(0va2GN_IfQVXVJW`=fLK%er5*HTjq1+4nM zmZHGU@bE-1js4m86|4pgT?k|Ht~6vi>w@)0nBRU(o*Cl_cLE)v9lF@zXqve+is^<5 zo02uEvx7!0)c0JjsI&67Lq;v1jiaCNW)}5EQJZcYK~fJQb-NwHttshU@q?Oc(6OAF z4Z1IsY#ek0Xmr&L3oqEC+H(fTS3BD_9=U29ePhkrogITaIiB1J16SL^_L{Fzx@P!5 zYTe~WbKB(sj&wVxc0=&j%WLO7c=m4k;jL}8{o32TeVbeE2jtJb9;NSsU7MerJBNM* zH?H5{H_m_lBDVGJw-Y*qm12Hvu>2r=h8xb#0lp-eqwrsq_sjkJVIENqoAsn0I^oN_ zMa+4>1)l|bDL4WDNIcQ?8Navp(Yy)yvjVwi{bt~!OBi5(dI+TZ^zhNnjl4O%2im_; z`^XC*?yfTQ_gr|FjU1a`;f}LjQ{)0^CYW;abq?KC47x zr${WZ5NRMdCNTy5023UenS9~tV%DXoIfr{t978>kVF2ArvnOOU^Gyj#W3<%tw2B$O zv$q7K);XqCBpr*Y3&T% zRSWPC5y~D_cxN4xr$d+ezHh&$JvOfwhJ1_v0ePn@Z20RVRsFu1CN1p7)4s+HUHL13 zgnM>*VByNp?@pJ-;3x6mal6^zJd8BG_oD)7%qiZga+K)2gKel(Cd5^Nu_l0_uK==4 z5O`VWd@AO`lqHC{sCsHPMQ&Tb`CMf0pDI@xW`o~0i5(dw^K5k*-7yBB571)uFlnp( zIi23g$L2*aF=t{`;!QKv@_68hIYN1T&&$xqnkit{FW2&VZxd)%>^<2?P^03^KO1uE z{LmnxCr`p#29SC3{hd3voa;Ym>+@!oW_Y;x2cdN1<>8)FnA+!!x=q3mcs99@(R{uf z8>a8HGY+CmIPRwSf!&yg=8WAq`?642?3ccjrx`+5nVG2h&$>S`Ki7VlsMXP{?VWm)?) zJ7wr<9is7mj(}%jA4z-aZ?*&*nRB~+x>MtK#wSz4md{7jlH!lJ6PL}CI>Ps$I|c$S zf}uj*O>L9qCrAS97)&2fz0reQQo5O!1dMM|Q%8bzc68d7khxh5CAWAk@!hbxnWPl# z$k};T*Ib^AZ zR}So{r>Z%i#$eBFCq<8Kez93SppMsWoUziJu{w}tDA%pbN@QCoR0o^;AJMncp7gj5 zxLc~!g@#)&xY>+yFs7Z|xMv&*-P_<5{Sqmam@4VT6%FOgD%SJRrLI?6OU>50G1IMn zX8l$b0lD+MZP{}E-j@6}CE&Msbb;zzi`}x)d%jvQxsEmO40Pg&a$1Ye^kmF+;rIRDctdL4+0Lg1aCHp5FozCM1QUhJwAEWP*jWsI zA;Mhh=MJ=oi55?PF^lZBA-GnLc?S&7>E{N7@Md0>gLtO~kMtB*y0&uZPOQ?pVD5dx@)pay&UO@8swE#6D5jTU z@5f5PN#5;JpY*O29FwY7?yBw5!$e`O1Ek&to#WMeE9{%7RG6D>&v=*xs@E-i@(4D~ z3^(bPiOTByncH9IjQTJ#9a*R!$v#**V6QIdXmiUpPuBSYx{&A(ENMaD~j{dw3n_GliS`ULB}*MV)!c8%bQT#=>ZI{X;G|SwEEPqLh;|7w5w# zWat%D&hI|S>CyF7x^zpIJybw)b6yGp81yG+k_ z#+pMS&uJ=jw2C{&bjj6K)0EYYQZlJyz2s^ST73C#I+PBUZzMbS+aq>$u~{Zt`twsZxsqyEsgK|G%-v#*cA$sv)J(X=^mY+AiX2;^Wn4{x1@NTd$0s=diskest*0gU zF+Ne-FqOQ;aE`|4t(}9F57l)<>*-D0*^>KW@0EjYZQ*@4XFqedKDWD&t!&Lie(S#Z zk?rL(D}mNv4z~=_y}Gu7B#L9?+0I@-3c+F9<8&W((6cB7ch(Rg$m$ALnfv1UF;oWe ztQH~YGoYLKDHf*|b2|&W6g(ls$1kR5`dwcrX}QqZY~HU17))ow?zT_xX;eI z4-*J&$?iL9rre(0=yvxPoxgHB#g&;{ym5~vBK6hn7Zf*O)=g-R5MIBWORE#z+Ac&~ zhp&EZ{$%OcyD`J!@9r9}@WI6uWOYXr#D$Y=9P8GF zqx5F{EtNt4>F)W|NHwd+unKwQpil9yUU>6$c?Y!zpT!?~O%^WGI%ozNkvs(Kb4D!V zGuvmd)zuKC*0^?FVp8lz(;W9No7mUoy1Ly9KXZ`8=PhzEs7D|;Y^95s$r&V@YiSYs z>RGz0ytY~&kSxb3bvEzBxf!i~acKA+8Ls<_dI9E>!w=QnW^^s9y@#{`Lt=p&RS=e+ zF!*J0?%)0Z#fpT9`#@P$I;ug^8;als>b4+vT12t$=>?-#u-ljUz}h9;8-jR-{SusW zT=UFz?e=F>_eu><_D^5<1E|^}p(A&9;OM-x2d{f)>EcCO=uRK}1BP8eE6~pivvpw? zXy}E=bL#7p%(K6GhWZi5KfL{bjXrn68s(;p{TT_EQ+J{M%v4+8NXol8wRuC-ojp{0 z3#jblYtj_nC_v_`X+PDR7S6Tc_N`3U`h9Oz$eOEGezrkMC!Se;Yz?m!uu_C>wQAM< z59*D5DqB&1k6w}Ymgf<9*SV3g_KsSfr zoLv&_l3kfRwm;Cpmt@O}G`h#hkHOt$gg@IH7)$fB^8`2m&qzq4dcz*^PJRZ$yH)ui z{Rqx2{0SDj*A1~6KU$VEYGTu^QhYsfNR($~gU@j*_*&ez&y7{ic(E3*Qw(ap9Qp02 zCc6cncq>sZZh@fAD70Ec(pD3%8uU*Lr_w8u+poiR*YUyllY%h??@GHy$7E)q#x(tB zqdd>Qu@`M({_}*<;tSrYfpDeC+pLG$Ewj#*J<=+(U7jKIfkGfwO*)0uq64#>VR%LJ zI48QfgZ>-$zkTSPU{#~;d&4SBY38kL*0@q;1ps>#vhv?Vxs+8bjcY6&_Kcj=h;vD_ zF!lEmQ8y0Ki9xh7GKjS7YNH62#jrK`5&u$V>UaAX-G8F)^MSJE&-?EMk zK(VF8tBwvz-|$E)l??{pK(15ZK}oSdHE34@hu}Tblu2Pn7XGc`ERp@~PSKBMae9pr<3r*68bh2G|GAJp69Ze*E}m`td{j z|H$aLI$Ma?yW5G`JDD09TZq~G11L@H{+kx337v)ZpD}jTW#w%?wr7xH&}h5?Mn;bx z3wWW{4#aT5(h9s8=^5#vIoJ7}VKhv7SZo;{7MpXjyOqUqRlL@a5$rJGUxlgRQT7Itw3o6;d zGh+A9DvQam`(pbNUb8ebcFUApGst-<=MPr(4Op4C-$<2Z^>*flD%x z#V10ZO=Efjr=^iU_fP;iDKO~&FrV~1#n@xNw>TFLaLW(}{;UicEh=L*a0|9-=)1#% zUjoX=FWGTnT^Yh}gbGNWpYb5w&UArdf$5!b`!hle zoGD51*f?`ZXpC@~Hl-qKECgD3(eGbEOQVd>lt{=IRZm);hr6+hc6uAh){B@o(V=Lg ztY4F6rnCv9G$v`Ht|CQ&hr5Z_i5|*X+do54v{N?_jqoi?qww0vl42+C>XpV{N0dOp z%5}>XP~cz1T<^6WcbC_MgjWLrA`AP_1RPPDr-n$uRKj4gj6B&*(!KnK#58#ce-Kqb z+~E}hl?0cA!$+wM0ewA03OB}6wwX|DsCOc?;Ux==KQ(m2I*&cCuH?L(m%^J%)ws`u zg!d4bf_I)`02W=_k|;Rx)1&7h)7KU`I5v?*3+wHP@_g=scAWC`k4W_D8FK~3(9NEE zS_^W_336red@+d+Dx8#2(kZ}GQQqH{8^|sva9WF#UQV#;-bmU9eBGAn$-#=EWK>g= zB#P;33jlb!@tJz914oMXUl@|bcrE)S1;j)D9u5U6Zgz_cE0Gw>N;ZSrNyJh6qbDBl zB#lSIHc9=2JXxY4hXNtAIGH!eTx~oFeFm^&vCx=|I<1ZbD^@ZDXVP=<7pRKRBo?D2 zX)op$)ib_66RJH_r@21w)9`7lJFnrb^ie^*yt$(G8!_Kwjw?qZ#oATe%c~&|+E}h? zxVXFCWhvXl&7``fYS8rzT`oePan;2{H>-6^@${OiK5Kyf*t=OXl3`2a5m-Z=b!V52 z#xf}Y+D>@+WHd`PtlZgnB}B180?^$^_`cXeT})`U19sGVCVDiYYkGvMsPr*4m;_Xa zW_xZ-3H^JKbacNoq|1h!gno?%J~Uiz7TDU##hOo-B+7K01!%1qbSP&hV8mcgO}H@Q z+#CfA37_BbAHf2LH3RR`1AQ90DKeEuFvJFp=GBF+4iY;*kOH3$3U+wB{{nxLR^#rO z51Yc_7mnp{P_9#qwmK*<%i;Ko%AAycHl42QAm#^=JNOYJJGT*V+C@?jyKLo&* zxNbdo(~eDsdtugZ9&fOcAV#Kx&?QK+xqgP7v8kldfDyxE_=L;xJo1uI6G$ zkFt4Yb{3_tuR0>@APFun)nhIWWp8IG@)=W=+`%CA9nP+9kEqUHJ0Jd#l(AL>_6t`m zsuq2YmSwsgN^?_5?&<*VYxgEZ>>H*nW2C+~yI(+zBdTonseP6PFl?IZWC=Fw2M0D^ zKv0=qU{(sCZ0pLR+h)qRW0)fGwk#CGeCQaq?A+SU+ZYqh4?afX$`%XgBea_0+0)S) z%Q6qh$!O2>r!mN=OcgV{WH*fF)sVZR+mx*FVBUk)^{$Z&=Mg@33J?#)28xKX2x;H` z=59|s>jka+`q zLl-`H-1%WVDK-^Yj|Uo0sp{V|&muASWM?%V$TrO98=GRKn*tKH@zA7Pf}O%;Bv8rI zIGU|B6`+!aL;4G3GCZ8Tis z5P(GGWzi-~m~3#SFgD=dCzXr?Swv&E2S*h)Y=!h{a$LP!>3i@aoA+D9Yv_{4NKrWo zU1bx!Nq9v)x*tmQUDt>7ZQJ!z=_$T9SosG0V z$(pU0_)1WrcVD*~c39b);Twq7=>zCS%kYe&E7Zi$4Ak#XO|8gUP{H&FVFxc>AFWXB z^WugVn@j*vgFhUDHb78o?V{RMI0Goz#g&(DC%Hw~z+Ymv6IAPy(7~<^Kju9)oI(2a zJ@8qsP2^JC57Ciu?*M4g2H>be@)4K&BI-%a!*9B^lx;@UQ6)<^0AqI>2$ucr)_*H8 z&$|#tF@dGMI)6fp$l-+e-G{j0k|x^z2eE|w-|jg)DR!>SWD1O({m$nTKLqv$R$=Tv zM$9na>;S}vng%Vv$z%1o6nV=8elm>OU<5)c16hGdxNBl0T)Kx&YUOzir zctrM!L@Uc&t@E>OGFa{QdNOU<_|)R!$u`Nac&O%UZ!_RYtlOFf1J zo+oGuX=6_k&;9x!>`@|~<&oxMB`+V%*mD@n=59fH?;E6V2Tsm*VSx)w=td*jL&#)( zkG$zU$xk1;8@YDULdD)M=L?QtnDxUADc1wzkDdAw>+@MEz+SDe zm-ayk67%bFo~=V`*aXQdlrE>!5>9m?s}!Nq)M1gyC=KF&8TCm zzC2fXjaVFPeVX3?_*=JuXvWj0O7|o>htA&WHOe7NH>LM!>sh=*4`)P1k3X#(RiLghxDg8s2pc(-uX+#*)igN>DINIU8}fEA&RqS7cLeR;}$)i)H+}8om-Wb zlOTfu+OqR)=xfH=A*7Lf)5vE)Gjt&T#v<*{{WI~h#v6A<7I{^3_~QP`^Ku{8n@}i- z^Cr4_=OE-eY5a5cM9y1dVo2S8SE87947nCEG5Kj*JZT0Cs*Gm3XG6j5x%@^8CQ~d9 z^Jh;Dr?`1;>5qC{x)(~!V}$!iYd`I?kwaGc#bBCs%~oufaeN%G7z`fjG5E18`FzTU z)60Lic1Z{MDb7Qdj=LgsF?6qa#|##F3nu@7?#uLf-LHq3qrmv0{vq6BcRKgLV9VFq zII_Xs;^nz`C~?BGf^Eogza;|0oV)|ZCk;>=6RWUF*;1OT9+EBZ!eOkqAN9~~63mTj zWJ@PhI_qDiv22=KzAIlV^ObLsXe6q*C+HH^>QW-gTaJGwo8!Q>%C1{()oD`4R;PIi zD9KWUCA5T!Z48jm>Lsn(j90sKsJH~sdbFx~6k_m!;Xj)wE%5*8_gD3OzJmJHF6!om zkB5^M_=QI?h9PD}F>0$DC(Mwe9mf{MJae2+AbKv7mT`-&Et}N@PvV_nB9?k0M!l{O zkDGIKtn5@#&g_LNiGQ*O*-`6>#XQ)4GvN{Qyzjx0)9<(Z&QNQp&m(?x-y0h)$`m@E zsP!ebZp4r!Om^-`U5Z_Af(vDKKFpPv7xv^-!SIy8?ey2qUo{0eF`MLGFfd10T!m0q z^=h$<)+IZ%TG%xfH<#tjKSG=4JY=;aYphbX3967TtnjYTY+?M0bwhKkd9M)8Fkh8A z@Or@aTMNtTm)d7|kAe5PuZp|@g4Tk4`i<_F>Sykc8TW+m)hRptT|fYlc)K4o&NG1J z&JlfcjA$0GOhzze zP*E%)N-Nzcs0^ebsT|$plI@5i%UYd#6Ac?2jl?!;QL+>w;v7Xa&fb0m0qR}`$e2$A zjvd-oHu$E7tvJF+rR~74g7CE~|J5g~v#7Oejp3|b{CwQ> z!lZXM$2_Th`JHgThIi_ZL~ZEJ6}=6A>^5_<6KC?Kqqu&Beh3ZUpRUpwtb$%$S$_!4Cs4QYIY<@)s3UQ zJshA@O3$TPHFBi61W;EV8+l{_RjYr;0x|27GZe)l6-V7TBN&7s>3i9%24u8Cq8lN! zEt1Mh7)`Xxw-;%><~7udq$YXHjAAN2e_F{E-b?!Do3-OJ9+0fpBF)^gN9$D+El8WE z5`^Z0`w#Ay)uUNc-OTktbD<=2Z2bhZvzu6gG%}XMwU~RWK2~y&jmcu5EW~S;(JI-R%38*Oo12?UvER ztL?RgyF5bY{NNG!oZO;{b($-=6+Ly*%M`JdqrTMbky)1I3(KL;m+?33i7s73f5Cho zHG^KfoPpv4)C?VSC&%efyXx6D4)p=YYRA@PNgW$J2H?K|-ZpIYJ->H~$DiA_+cusz z99!y1k%HQ7wF)9Za_mkS&t8I3*T`^$h5OY0!dh^0&+CzJ^X z>0DXo3S~&CTyrwvDL*eoD-=5vGiL$s3$pXJsIE(=s6tw%nCgF?r*jRZXg6gDk<<~9 z7Nqk^5U9t!(TzAAr75iUn0$a0F6E2Ou~);>f>;OfZupuh_D5*l&=w0yKPk>l&Rb@d zBQu(kSY`e-Vd7=Uog)q{;g%V%_CSv5Ec6oj>8i$z^Sl@AmHu0jHF?JAgQK6me~_mk zZ{hg?H8Uc&1YV+^AR;FG$SwqI!3{THtMjvXnk_tqVm*AUh| zc`n*#P*8Nvws|Ihtb^Yx_ikxZcF`-XG+b?LjMDWh8^HmsCxvq@=0}MBWqYh5xnH>c z6|Q|iA)f=cdItv@N}j@ zhf+7OdiRmF;I95ge=$a&O2_aH)ANDc^I!*=-X=jJT;wgEXGaImNLx-`?Lcr?K*?o) zwm1A_vd-ND7AdNO-hGl-t=YFo6@nHRqT`sdD_xr$3=zWlHc?7o*hcH-{>&9~nJgLF2ItF0YytpCY?R zyfhDyp$xm$s|6dIzuMOx+C#iN>vLDxF%PajxjP~RA5}Y~1Rp!%&;oA8-}K^#D6bz? zhBj8Q$!QBrOwk1rbU!L~F6=z6+F}ZLgebw1J5Wxo@zQc;!8)26zi@bt>=KoVW$Im= z%nKmwK_@nZ@$s_Ui51G(c&t=8IK>d06v}y@oT1YEt+Cps5RgC;47#WQgfgptN3 zYhX%|wLeIfer}d*-fBrP4ji({DqrPFp@83TM!&I3mo7O`shZ*AS9LS}nIJWBuRM-wX5`QoK~0FuVXHt0;Wh3o zi-}|2Ai5pG{85ax5K3a3)LmQCwpNMwvO@Dlu=UII=j^`yJ=C%9$~eHZi(M1r66zqS zl9bC@m|ljbW}S!)kC&GtEuOtw_NyLsCCXK|Yj>Fb{(U~{{CVsTIl*`CJcqH3t&LvN zEl=Cld7`kicuyIUF$uKdEbNL?Nn5iqrKp0|Ak~Ea8<{-=E+}Me5vG>$$cOI9e8h0+@3ITk8{^$W6q3wNCr6i>%#{?_^jsvSTeU zN_re8zwLz1oWJykN>0QPj%w#G` zVCOF?Nl7)~Jg1-K8Z7_L`OGbX-lw#1i0IemZLYZrAqJvED|6jkL|4e-Ue=1JF%2Kk z1-BQ(BpHEltpQQvEK%60Uzh1Nk%BAa` z(_+16Zro)uC89Nza7|*uS%#+HQ#Q&4}Fkf_T!hpAY!g zqMQHQGamywD_9z-yac+>k}^FRPI<UO4h~Ibwn_XITS5%f z$is2zKJECmzK=L;3;lc(v5#ff=cp3$|iaokjGsA%87c(`Wb)t|XEhG}o^kzN=(#T-=ZY3O;OQbA+@6>RZ8ubM*+!`tv)5(oBM@c@1xqtq``ZR|2_Pmj3(}=3g|oE8(V_@li7q zrH;T#t~6CX+KW{bPg?=?L-UmQ~s7} zSL6cG9sJ6wz8veJQhO9XZ&AcIWf$~?NQ(dlO69^<;gTAhTUX{A&cl&bENW|p2eFlb zg+rQljgl;&V-3YF@~QM=vXAgtfY8FQ<^tF?s62sISEO(t(R3l#9(;Gi{a)^&DMuMa zZhAHFN+#%wikbPPShsinA0vH$;7;pfQ~dWA;aYO)-#P>pvk%Q>A}#b+ESCNjqX{UM zkOGrq?*Vd*j(61D_s*%5PvQgB-)r&vpU<&Pxt}6tDX@z2x7&e*6SjL49WT_IE$n$Q zwT#=bApyxhOE60OKN`rLY*A6RI&7^=Sr_|Km>G0dYlh5|b@wWJ^&R0Ol>w&TWE+Z@ z6Cc5(Gh%;!`DEA3a&G=mbjRWj>eEQs9@3!JQy%^HWxiEC?5pM&Vn)sw8bC&x7>(Fs^#(bqB8MB)E* z%l9>)-ga-wQnJj)GYvQv1?(BHy`_(zEF0P4;xvaZpp$P5#$mHA8 zBdXl9XwSAZi4J8V>kjg88-4IFb>d%uG8jSRsS!L;T6|`O1Zj7l5DI(H>+T&|6P#uG z>ewO`fDQh1Euqvd)7-1jAs1}r1r_J=!YF8f-8%2-<(&1Nv z_s>|iHh7xh7NL-(pKTP^tbGl4oz={AH1sD>Nj!wVDNrQJ?!RQdplc=OkZx|9Lke1N z4zlbTLWLBdGT+Ie{-SLDV%3PTlFDbu>$W(gQqH|e(?ztr{%r}dsZ}kWxe|#3r%zpp zgcq9BUMBoiRVN){i5t6(p8VRhHS=z0*o)|5)ZBCe^{f?{xZqFOVQ8G@dSInQo-${| z^QK--<8tKbV1&e$(`nTrN?KM zx>MoizlI&(q{VT#<5yI{2yyJ$4<--|Q0K>|9(!C<|bWcbV0? zcJ3VK+9lLiy>7Y8*NtWVe8d*%Z@zoAeW#n41A^gm@?RMl?Ys}BXc-=(}2)RO2Eu0={qIU?%m z0|rqux0&ya(@QxP2sPWAc>aDe3ci<&dV+v>% z6qD2wQ^-c>2JT{+rK@=CQO{8folwlM_QqF(5zSTQz``kXyLxF7YZ62>BS(E@SMC9$ z_G>h$_}-CDZp1ZI?&?NDDq9XT-4$+IUxg}ukl!1~HLVbXFNSo4n}|s=86aX);(-N? zJVOaPKHul~Q%kr;&IO*86z`dzO;-JU_roR*v_+ly+c>6|Sd>JuR1T6xt`PV=r7rR- z2{U`Oj(XE{hmCAqNef;M3rLDbfBd#xQt+$I`#Gbdp`m3#(XhT;U~k1iywpm$~=Q!SRzqn6Uo&C|5Xb^Pi@T3&-+-@zuHVg@sdXOa5E4jB!! z{Mypay`)!o#H8{;K<|cv-+&0V|I_#RI3sWS|J@0R_)qh*Gp=w&f51i7HH~Rw0LK$I zAnA`=Gqx$p8F_QXB3JI3iMOzA{n{B(^h8lTct zN~A(Fd0t?VsyC zPqRXd8+XK#6zAn~MrxQnvv~GB-G0SqB+l=SOL?1KI{K03CuC(eYi$QwJ9IBCA_fI> z>}6w(vWej>0E3aVe{AFy+Zq4rKWya6Il>{qhPAB4f`P^ROeqIZ-W0T_hc?9S!ns-^ z9XNsutE5w+j|iC|uECz*E@5XGp|3eu0sV?2v;P0|wq0a4lx^fP9*`}Skl&#%_-y!yKM=h%>yq4V;uu)YhziieFQ#z{5}B* zTO{{J)GrU$_C)6mQoKm!M};LzYHr`e(?%WyMHPjLorSi*;*v%f1$TP0gmlvAry&++;lOT9$ZRUE$_e zy}3e3TBQ*JjM8L&6C8L8?hs{yK}CM4vab!6SIn%t6JMJB&3Uaypw(^~Y~v=*TU_tI#CmfIr*T{s#GWX8Y*<9Xsby|z-|x{&36W_ zoZ{L8MYe{9Gr8OK@^@K&_XlF|A=^u_-ZR4cZlo`>!PwGRhAdA^ksOAE>;)k*bR}xD z%6Rh$M1JkxzTS5s!~YZ|fK~p^3~z`h?TsOpIH8+y3~LmWN!wWFnK4`V57W23$Txx2@n z5uIq$YBRrzW@vZ%zA%|{WB*rz{S3$Wibrvzdh(rttca)aK)S!8qDc(nl!4G~+X~!O<;e^F;oLR{=VW zi_PjVue$s7Yhz5W+Uij)XNsyUQi7MiS-xB;f!@AN4Rq5?j@GuEGCL+FoH96sC6s{~ zMQ>d4{oHF3`evpB@lR`hnUkffCM%_eIGWKGu4ciF##s1$(3a28&OfUf@K2H%6t?72 z;``P#vd%!~y}{)-E}%;iyIU5O3M}(o(0RcER$F^a{OFIh1mXr!pvnw_Ts^cyHjxFN%fIs^cn#?C%fwgpx%h?9b%cMzLu_ zz9Ti4)@AMiBDus20)KzLD%V|eZ#~4p~Ixe$@*V#QjNz5Iwgvlq^`Z3|;BU_bI zMnerr68d|5&$#L(JwJ2a$JVcS_->(T4r@+P?G16d0x@nG(hq%Y@=>OQiR#^b{yp6o z>&WM9Yv1B@%jVEpM+zV;sxfZ1R0d;c1YoK5I6_YhUbOD>iZX_m_MrBtZ=lvM5;E=x zrj@+gly0F&wfO%tMqKM`jyKy@G!f{zB{R0KX&XsYqlrZoUyH1~w$7swTl}gzrh1IM^9CkEkEW8nE5Nd=N(aLY zyZmmBBl@dI@kw-<#I#CvXH;h1Y>SZvnlf`&d@YP{1=j}!>y{`KVo48AEx)v4IJKFQ z^CU`AK{hNsUsl@#;Zqs?$;4~0oB7;^{(h7M=%EV`Ya%~jwP#JxR)4Y-zXL5%=kOK! z9$urLrynHvHdFm0cDHw1y}t~9`WbvT_ywkM579PjIu8@QAxLSMJn9eM7Lu?#x-$CB zxO(mW7J|}bVv%C5cO;2}91n%6jzL2G@+)T-#-DHFbJ9Tw;cjEQe$Pp8sy|fT{x=A= zC#90?AIz68bVOgi2>s7!t;)t0rnZJ6hBn5opNv*}r~iD!qEt^G0#%G5{{-8^xWeDf zNUD|y7BggL(pc!}%@#AFQbdx(w!)FB6pRTAaE$iZtQWySG+`qo=l@m5c2Ky5@HCMT zcoiMn_n{|5)&8xyJ>D16f|u+6xNFFACRbSx)}1FFR@>aq(;hBo9##msj5KTsWYAy>15U6HFf!AQvK=7o3!of`lm zhf5(+Ux2Q_=_cv%eZk+7YAzNLoA;Q*WC`^-nZ zWCl*+tqH;CGl2<^&)LAb$jCUB^Sx_>+mRgSb&iX|GZcasSS=bFE7{0>HY4<{X>WSv zo94>zCd4TtP?~}}`{45^tYYxKnfb){LnUtW)D&KnCnjd`TPLSNI{sas?zu%wIvV)v z5K>28&H3m^$hMzgeJwY)oC4Kz!aI3l2$u!;%+)zfX@VZw5M>P=d%Z7x*@2!-?kQfs zu~sbZP)+b6gw=N@cq)=aa2N%%Qz2Acb0L~F4GARo-2YSm_S)EY1}7E{3fE}1gcX5* z@=tGo^S`2Wc_D$04|$&GqRyH)4#F-x1ZvPR&M8v8p#n{+=9me+%bd}`Rz8(EjGw2c zoI-uAlnq%l6MB#sL2pFRc~fxk9La+vgWpaMbBQPdffMUFqGFt%p!)3?yanz;eeeV> zVi5=t!^s&WDuqsAJXVNMAqe}z^((Y)Xgu1%bv#Y$5Hon0z|+v5I)|CoCi91RK81*- zlF!#$=qKic+0r&t3U+fE-Kj1+lWawi5TJvsHR+KP@8qc0jM)KauofmmOMM<&g3D$) zj`gtO53~H}!ZgHIG5?Vn=HnYX#Z>VnsZQ%ht-Fy9@o=zi!FZuitLRAV+6XKpHk#${ zHBZ;$qc7x!UyO|myI%KLG-xoTC}N+fHrtP@=TzDsTAmcAC(UIl+w1Zj4v3i11XG#9 z@~^3JBy*yglZae7P+JmF*P#o;qw4lBEb^}eMHtSy=mN)0HmX zhVr%mUTo>TFkWmqhw%>cT+E7xPE&U@rAkJ_RiPCm-iErZ#STn4{nBHeRf+8@pb5t2 zRii28Gx*Dj(H!*&AKtQ>rniA%@(0XN4Gn*iv*~91WWW}#Ao;#~Rr`4RRLmsTasHC9 z^*rtUi=N%s*ylz{lj|M_9$|dt#*^`MyiIXe?3h=`!A*3znXjWnpr4>cLfc!*_7A%A zaIxDNN?_!idz{?vZ2^6gX>64av$n?H!UM3jy0gHTVNtD1aiT6%P-Kyv|aohqMf9-ppnp&f;%LW#=4`vuPi=bH#wVEOqtQA#oXNU z2#(osBlE*+&QE!7Rmzn6v4O_y zk_}-dypj{e@A&tp2D@yM&_5)bXNrL)5{QzK6qXqAnessX*`5e~Z=5Fx;Hfm3WI`v< z%P)q$sxX$fd| znRFQOt3p8DvK-S%-6)?hGWXB*0&YziNgGqhE>!w%FbS{*xNAKtIGp{0^Yj(Smk8K4 zYK$O56S*dlbpnR%q5}D(-q21xqvDAuK?&hx#+T)QL1RF&C2)@fSOU04IQAdU)@Qk{ z0bFN@0_iq{lF0lJLGZFp^?D17r#@ zDWb4UiPN~VXvE#r;LxfRZ`6|2Fd|3*{K(S~Q;HzgERy6AB^xKRBnCN>S(1Zl#tH~=%=gjrIK4D6bb0$@|gxGZ46LKFVq&T2a=tU9g%@(fG%-`wDG$vop}+v*Vw?Obd#*G!VEHN z$@)KJC!geYvJ!G&C#t;{rX;~S;vK&Olsq!VK%e@hayA*lJIsH-;{lJ%$09xxyn|`J zd=dEnb45@xwY7KqzrwqV7QKICsN#INywG-g7&i2zMM(<1Uez`5?7>vH51y)*rZW)m zTJ7RJMPBW4Zi~gBLTI4-KkD?4iCM8l1mIV(no*IO)&E^(8=MF0rfF#j8|3ec7ab`V zuM=r$tj?2Zr1qV8C(efS!$)9a0 zhmFs;-khD!yk5}yF%tA%zU7(FmAfS-@+RBAG4B2`bqRbrH$|#RJ5jp17j9<5u#69j<03 zdg#Y^ovmDZ@sf{5LKwLyB@*ZuHkmx=h6MNm)oZ(zsgwNnB8(lp#Ow)?isFysy`U3xhDqlA`NWRsX0pm>#Gxc^o?jXQ4$S9mT$grB4YcAU3W8C!pvjuLtBYzAFLE3S za$dG4%d)f1k~6PpV;r6FW5~373IUT^-W%;lB~ehJ5r%I7`0|cCcXuh-O2BYuoC$8i z>WuzkObn-g+2Y-51teqDXN2Sy|5yevN%(EF?AMoNa>w}L#4k3w@MxF$Hx zs7p$yc_`m=py{VJrhs<+^;p2PJ1z0+_txNb3QGTNwwDvvmv&Pm^101?}>WrVN3#7cHB}AA`aV9Yf>gw z2I<>0`WD2L=M$wb?l3GV#jOKeb)S-24jC?1032B{md>}ozp+y17u~j3jy3N7fO~QU zEs%~Szv)eXyKvsOim(Z+%^kLj$!=*Ino(%Qu+}AzX7^s-e^t_{Cq14K^5?k>3)0d)w%t`E;~pbhY^D9%M;|=bZ<| z?=Py`n4S|s-KTXlyyA4WP_5}F{uo4j~hNSli!zb{xq#n%Kew7Ce+MYLY!fIs9GyhD4*c2iVv z=@O$0PwB8)=|D16*e8kKLX|njT;Tvegn%x$*Q#qsK7@dw=D6DH&!DC;e)S`GhD)#9 zfoCWS-rS*6#}H|V7JXvZqo_|4-<>*h{K_iWnM<$0fnlhII&;XerSA#JT0}!@fsBpH>VfwN9U7OZp+4)oRPzo9J|pB{H3Fbo;V@wO_bcJ2lTBBOs`$ZjTDV) zi(Ot6oRwt%QDXmxUz0Z^>qNtc?R_s0Anppz48JQo9S@SRa}A~A3tiKqcZS*P!lnyS z(~{O&Japvr7T5ZUy5idC1f6KrcRia4N1NF16r;_&eZtv-KuyDYlKO^yMo5!|n#pus zCxO05pQXd`h^%BtG5S!;Kl-oNX&n0lbU~&m;h4?!N4gpXBjpIAvP(ZL{!p6S&x|=|%B8-mr7%9N-wk6`YB2Q4} z#Q0cGx!-%U%cMK@S}B5A-#&I~R8w8ZS746ur*EEEtcV1_o#YlxD{4hk2A z&kSJQ$a5i@Li_Nk&{_euQu9&)X|IX=k}{G~bJAPGN)(i1qWouqLVc5I=P5KWW7w3u z$y;G=cfv6v>6=BmuVSxq_!K}OTTT|gh z-62t-GuEuT=pki;pOivR+VKO>FH)=rRsbK&IB_z1-Tr~j#r`06jWyS6D zO#oMkF&z5_J9!9t$S3X?WjYl_d+?R{(F4%KCQLN+{Fv*W)&8U zzMBbr_Eqr@x_zO4Hf56teBMKS{x(kueFszeE(9j3Kgw)ja2G_() zRcPj5&gSGjn@jpnDVS^#h3P=6VRg z_EA&Tn)*LxC4d4Ex6|_&>+ZtOvZ&8W^C?hw%!RS!zL~D4oT0Z}sLwc+IXJVFaRTwT zl4@_saemf8-I;goxq6ab7)NtzZym&M&9GhBcOkiYBF{2lbIrRIk>_sMuK2s+T!Glz zr$WZsWA~ujH`HhN`APWWrb0$y-!j!6)r}u9#b|lc-lcfFO2WPvG5t zZg;?K13=%B;3Y54kL%N*&`VlDU)BrZsA}GP_d(=Y7uAdQ#5%wW@8|@eZ%^=o0`a`o z?J)VjWe^ygBe~fiefd&E{67YQR6HF_MGT!?B<-9{ojwD({!eAIhP46ST+(|Cxp(2! zBQa$F%{P+2-&YiE06)KSGn2^)Bg)|eq+oCB&^&DrcH?P4k}?~%%WG~cXB{iGQ;F(b zDg|sY|F)p_lHzt(8B-Ig#$J@B4Y`y*wSwQ_2sQ>1c+#&DV@j_0kv@ucWFPGbXfJ(< z{BoOL6$p;~t*%I4YeLKugZ(NC@B|fc`76XHsRQ3~yk2KRb1WbIkp!{60jX(i%@U5z zgVI1YY@zf*$`{f5$PJ{1%Yg(L4M(l~i`1o>s4Y873_|YG_ALC1hQ(uxadYtzC~p5k zmBV91otOw&(Yp3alg8brnFLujQv%(_QbwmzpQj~vRIavHBJU41xa8Zl|C7tkmKxq2TFk=qgf_kT~gq@ zbY)HGKKk5h1Hi!!T+6IC!u6B%uR5^mQ`!ia&~&oRUatHh7X=4U04=?!RBvOF#MC)O)CDf4(1V2qaE4#xjf3U7R+*xUcwcTEn3UcifHOn~|bO|QK- zHfBQN(Ah+j#UXKXG|U;t)SxLHKU+RF%l|J`nR#8I(J(Mg=_xp-rBpFkLMuh4hWsmM zwtK+gd0oSN?ww^D7?pTA;#HXXcYaq0^ilRP%2d7;4&;}ZM4;T=H#s< zvkc~3D9*O*W||Y09F|G2D+$zEr*eZ~z@V*Kejp&|$q=cezF_N{>_#>A9P7t8CQ87J zbY6=WS@vNIIKmjxd30G<(*$Xl!EZGimm&#B@PuBEjy^Ri`PsHU zhR9r*%4&u-n4X#Ez@nePI`=|1=&~m{8wd46je}Ra_cU{a>D4g(at6JvAAaIoE*WY(_|{{>0nC^@qQ>@R9Cw-9W zQn?V73OA?Lh<HH4w)@dY;C6Fa9mOWm;)52-ZoB{6jY}blt;$T$x-fPt2(6R z`$?WDzTF1Mp)o;7RCa%zo2#~?uX<3KIW5@)jLMwcRx+fouq4Wzt6c4V*}kpcwF*Yw zY8aF$qMM^;WDwz+L4DHtWqXVco(-p8u#GM%b{lmh~sZ~TPPywZnz*GUF)4-tXW~RMBZ}sPZ8nzUNgD7)Aap0MfK;&BXdTN)^c;; zdA1a?MjleRad_eLWU=);bq84)Hew`U29|nDg5U$!>oy2kEUt%zfeSADMZItK?j|Zt z=hHi#<|ui-X$PwSO>bZnvtZ}KAptx)kj3sluT-Kxr>g424K+GYR&ElcanU&X^orA( z$`-S%fW>-NSk-y7Ki+PMC0J}q!`NCVW5(xLomr_q*PoGTXSoHcOZlL@8TZd&Q#vo# zcT-9en~P8lZv|;>jmjWfulYI5{hsMPE7h4qgE1%q$-M1s?#4GZ>0%Ibjqjzt%l0_=#AG@a9?b7G}}&d71BF>SABWX47Pr(bWdgwW6z(O4j+Go@7P@o zNP(O<)UqLGWmkeI=jujY0f^nJs+RuAu^vMAo(uteu5coW#GA}>%+uT*k>{q5;bv`i z0C=H{TBM4$$gkXowlE4wB4>oy@dZ|WVOH6u-MNe^zSy0v>woZs=Y%#4XVtf;aFKKC z96@??h?YaOxhBMgP`=#XxRxVvFIenLi-R`5rz0E7VKkrLgBByb1J`}=NF+GEHA-}( z%SC!oGP!FHdlzHk3>`^W@a3XD7%)IA9(2w<_`>pR9G^MTx%12I+&E?XVVu>cS6Bl& zFmGy+5gehGYqL4dggDonkqFp+t-O&+TokHaY_C#U^m-5PDYyR7(f{-2L5{qw$`z%~ ztG5`<<8(|pfOM%if^DfMxv~IesUe9D1>bKH87S>|K7MH?6=!7lx6(!_{>;&dfqBFx zRhoY*B!HqcN_>5zAVsi!Ta@Ay*ow;Om?V&S`@~E(EXI^M6i7?VPQAsRxT#bmnFI>7 zW2i^27k^y&lf@vsE5E8b?*5i<%!J83(S_9ed$Obc$P_Q>3Tu**O;`fc^O_-}CGQ)t zxUm@kx*x~*u5@WK7EgSRaO#F!~r@9|# zZT86NqmK3rqg|fOtB`uWzyZ(I-4X7J)OsyNuo45X+L38qK33y)@=FhN-ERTQ0`>gH zOD(TF7!JGQxeYml545uv)a3^`Ef2(w%Wry-L|aH6(DsAQ9+4{3H}FE#M&y5C)d_1z z7PN+T2(Pu$Bzni0uBosyGQEwQ&biq-oOd-+3$PHdw}uA*ctsaZwPjOAr7~NU7^KE* zu~8Y^e)M_=B>gLr=TpwRBX|!ccqmQUfAm|(ItW^?9FFe}*9cb>OJkKTb z?qZ->?1YFZ@~aSlox^O9<~)!M>u{+D^#+ zparAxlj)Fw{y(*_%0E{SfdSmmM?f?TV z@I{nM?T%eCm(Eh^u! z#yeu&*>yF6?hh5;$R>vyXs{pOk4*A?e~do9nhWpg23>Zvy$j5DJ``k8TXI7ZkFa@= z+Y#^Do3?jE$GuVI^`t1gU?&`&6?O;0JuvwEaNqgrgP%jZNVK=mAk))h;PlFWPHyxN z8S_$vj54f)ALs0-upF0Agq=c#4D!NHuDHy#=?hv7k{@N@BeVa9`t&nqgwRLt{2y~0 zQOgLik8XFc{YZj`s9%`G9uD6)8b8J4mSgrI+!IbWZO^}iYi+)Pz9)W4nU8?)d*80S zLL5ko=60ez5FirgwuAIZ$=g9Zzp{G%ZLUXqS$T1;54ueKCT;@;o6?Rx*jbu8g-uJj zGk@9WRWiQ(#|U=2APKcL?NF9^Esei8fNWOFra?#Z6n#?!8Sra&|Mu_oPOV?@%zkiE z3EXpHJbU!ROej5d9*KxOKml=9V`TW^RqePHNDv9IcN~dAMQcSLR z2`%lQ{pe4){%}=#m!R5M)2u!`-MYXuzqJQkul-YmWHz|@XWus?6Z78lzwPwBXBsPj zpM(sXZ(qJ}{(k}N9Sog5CpShB8$)O3{~G50Gr%P&>&l}Eq4Pr)X^&j1cB_0>H!aF_ z5kpVHMWaq-vx5gmY&kB*m>Mpv*|hxp_`9*Ge+hz-hVTn|+>d2hU*bqaH{R}_$nxG_ zcRpWhyE%CPeFg>&Q%r}Jk-4lYGFAP|`YZE;d5oqtBPgKk+qon!WlEF|BGd!M7;C9L zd=0#{Jp%WQSl#g7WE?N&Y;B}AJUgcn3LeR?i2Nsv98yn8epR*Y#CgaekE|T)hu+WdJ4>~lT`$Uk8T6C z557YVoe{Fs6oNngY~AEaQ5;-aHM#6Y(uoj>X?q{Q_5+qz_h42XTD2`j%~DKd2{0`Y z^>~aAo3%ioxrgLSk&JeVlEXO&>CQ&ec*q)S5-zP+_+h{g3{q^>%U=T!ge)@pRDPi; z{F9vbZA4*C<3FH_f}2e+zV<6r1Ua*Z3+Fo8r2pLF?{kGC2Q{Q@aW? ztQ^opgj}hjTd*min`=ZegnRjw$bN3c?ml^kyjCm)pl7tC!MxI7kXPXw_|gCBuMhc_ zmHfVZUtKHQ{@N@k>%0@u@53gCS%g?(K6YPevvSAo1sXYtbl$+SOEWyT|C*9(_94m@ zKk;$Gh3bV=+Lcei&UOoq%%>hwy*2*5@6U9tw2Pv98G00$2`4;gVIF5{(qR_cDE8)}`lg*>zISVEvr$3~`p6@YI`Uq+C3(0(EEjMk<3zb|ALsu_# zTw0Y~9vU9GQ2Xcdg7V+@Qg2NM{N<2ezMMgS`NH!*IOgK6cEU3X>R8)W3PT)O^Je|7;z+7`qQD&pU zesdCof*|}6F7=nyAKmEQhI5eOe{uGX!I?$Tw&)k5W81c!bZpzUZQFLzNyoNr+qP|X z@N(`sx9Z;e>zz|IYuEm}*QmAD++)r;hKS(u<>jQeQ^&)U=hw%@$_`+YpEWYgUaK!M z@C9X4AzmR~C0?nvoB&@0v}Hg&>AhNeJ57o(l3^Lunqn3s)((|DvIsuXN^1xQWP&Ms zl+$(Mq*?DkRV#XqQy=og3RA|g&?nk}No}3Rj4H&kZ1p7+vM7bA!iRejq3pa!vim08 zG@T0l`n;%Icg&3K-!(^D2VR%(UiUw^fx&uKhcaj?Kgjn1%uJBQL`D z)xtfNGmM6#Rh`=*1N&=yfFzU_Tn+x6NfkPgdf^dw&gniK{6YDrdf#(M%K)rK`u=+r zov$y39vgPM(i3lA7Sihd7O}rh^tC)NkPtVJrD-xTmC!Ir(V+R4l2B`+;ki`fc_Xh- zWk$W)kN8Lb1~r6F>(BNcpl+SCe})vr{2|D@=cKPVGt$12h(n#k9+%v!l#$2-tv=%ZUb-AcBn$)V|?!+c6E3^`ITz>xMIdvMi- za28mW0bmX3eG*u|OAZ(NEb3=qR?WtIJ$iGBcG`fvVlUr-{_(7#gI`7&k3js?^5f~_ zcxvY9Br+55^mqXOIQ>f-WyMS0IduzsDu4B&l8B4d`ilrEFlh;fAJg2rUzg$J*#}C4 ziWMHUl8lWpO4JjU5J%9B5RQd8ekEMed<7U~;Z`Q$QM<;I;!1{awTfijE&kYdSTHAK zp6){1Bqz`Jg_uF^_=S#sf(V<%-Vn33#3BDn=jmO%31h9)kF=D&dLl=xVT$ZM$;hG8iKvpXPTO>6Drc4yZ=VLTcYEd(|_y^1dk+MxS} z{l8BDz)Twy3>W~=3jP0+v;41%PEt}?!NtVUL;io=drHm*&i{4g6{%<`DXF1+$wX~noCPgl>b}oT6Tr9> zFPwZuPd|3`{C(Eq9!Uz1rCkWC?0Q-As=CqI(eyfb>G69*?IlnxFds1oHBta&Sodf1 zCOZyIcVJ{6+Ke;+ZW;2%6ObI1U{o?JCQgsYM|8{QaaS{KU>vlUn?yG}9-N%G_F?>m zNF#d7FVw1UHklY&yjWgmOEOWNB2`?O$URxrxmW2xEl*`ROU5x)V=+5qKzWrOYGwNB zqad_}TA|;z!UF2`@0cZc*j-S_Xq+lt>FA^WY?C?F>0>^zenHfB+MY(k3t#9uQY5~D za$}4vCO<)!F-e3vtMPtFo5{?Kw!JVKc0mFSwq$Vc`O3>}ytsj}Ahm(cBi>rtJf_P~ zgy;~}bu#hHveeCbn z*$7TZ_@cM+BgD>UuSil@viQLju_FYhOmyesQEm-_)0iRFj`_Qp8Vhp?30qDW2#$}h zl7H(tDvZF=T-58O$M1pZEHS?Dai@N1gyib#vMyX6yk4j!!&aZq>>C$N#XZVC7Cn2_ zWdw?n5qH>KkC8&9RFVf*I8AnT(ZQObZgLWipWzusv3*gj!AXO+LecG#)jILdUd z^5xMrq~G?C`=l@`3ugM4b?7M*T#CPEq~{Sf9_}ktL-gUVZU`~;phf@f*}}3^-pBxM zAEbggVsOkm`u%BX68bR?^On+fcwol!LD%OIFbuX=8--Q_5sX^5iWJ6rDVA*vFI`7n z;vr&YiO}IaNQZxiY^fozSx|HD>*k3o2SVQ?)?v=*68d@rHtSbx1TPl2>iI_g8|R4i ze5bEY_>}X!Vi74Bvvo$Qk-t@>8b5we2+2K|+WSkaE;!43UWoza2y~3(l>jEmT%PnP z@}p2p*Hq1&5QHkGB88<9{CzBB>+%&Y%6QH5YU|CJ50)uccI?Ell=jeM5$*g@@!iNz zQtR|9tpm}U_RdNgSe>+rwVRNWEX`~OeYJq?41En>uYny$wK7xnIu0e|yrdG2gsZ+fNQ3JkGjlw|>ocoeoLyNe_Jf_%dVarCz5Xb!rCQK)&kW zAMg3Dy~WLU9S>dbu^hmjZ6{~kNWA!*Z6{ts=zh$GwYhIZ;d|W4L3ejcb0EiMFJWIR z$X}q0ZzMuZlQLNibbbD41B#$}usz#JlKyKg(CM3q>wZOL44J<**>ksZH=;MtBWoi~ zRY$srAg`qDx&tkVdE!wRpAoNb@uN=T;p}Z>dsRAHHL=MWID77HP8sob%`TEhlPi2@ zu3$P-cdroqd!VRj9YL<8I@~{}?#ahaVz|Ln?O#EOOS8Ys-q@*<6!pd; z!cOL?`H{?PfAGiq3{R@1u>Xt??nKejU}GMnNY%mc8Z)#_CrLI0Ss}7X;&>T~HAMkZ zGx7qK+*4E4PL`Y=JvL&*&4VjjGh6mKicb=`Zo;YjL8I1&=z`d+HL+p8s|yWEt)Lje ziDH)JN+UsV(1 z$87eI>Sr62)fO?!KPAw9tungACz^2b{frQ3WUR0b6R(cu2KIF`Zd*!Xsy}s&8pV6r zOg^c@RqoJeeI((-G3jXR_eq-f8gZ`&CLlmk=|Cq9I0ev+ZVuSWKfuF#LAiRs20YBKUhc*%plJLp?e~_oD&- z_i%@AHsONQ?D2?V<54Y67F7EFeZ$23nelpDuKo{^P=+4`Dp2CAGv9Qe{X4Z_AS(Cm zj{da`zXt)Wy5IBnbRz6TVbsK?2k=vXc4*yPoRfV!zL`{7;!;)ptw;Q_WTj#{$0m!^ z`$3n2gs9R6^5poPxlScLlI+kq(V{d*VnoqJe+Cy+IoI#)0s5kUYFM4q^H#Vn+m|CX zG@RAXM(EY5>B@t0mFbB(dWqL;XSVUzrku_}&g{&ruqSe^A6bGIB@_E21efdX7Bqrl zoP;A=nZkO0R4LNTL9`7)^L1}^mDV>CgCqmN3um{_;!I7v_*`J6CoSk1^S0C)_9B%v za$@4Ob?)9VoNS*AWF*)NC)g`3O&3`sxMwoY5jSBcl*|@XlU6wGHC_Jb%%7@yy`OtQ-GYWAW@D}QVP&q;>a#0n3>Qeh!U&ts~TqRTMvk`uY0D1r^{;so6*w` ziaBA4t)IYsN7G(2FsGy7KX9FG#budHMZ+oxAQvvW2p_Jb=nEaVMuj`Qm<%e(V)eR( z_5CFvmn5YM8p#J#GufV*jp<6tw82u!o;H41xd$Ps(;$kI&RWX;&)6}M^CNhMbqvSB zIFWPobL+z~l?^7}7?7Z#&I_kBLhJ{FEBC`9^KAJ*eqBpk6DH#irV|rttek7ILtf)! zJHqxqV6f`c$f`QMRyJ5C^TYmo`sfa4Yfpzm69pl1&Zh|C6uo%5yCht*O6!uwY|XxI{R8I~!Ud#Op5#DVb}3)gD%xaFq8W3c%wE0HU5kJ> zot7k-?91I)W%SMd4xGK?Lu9>puh^|Ur#7JyTg$xU%#Ch_|DX&H~Guf z&>t^~eYJGPo8s3O`NvQ313ukGh{nuw3U!Fw8m&@Cs2nS6)~Wnm)ms!%)?YAMq|_d= zz%{b3H4H;QRKky+&VLA{>roXM$?;2#8lmxx!9BxeZ|(W~;vu3;a}x9SoRVEPlGh_@ z-gO(H`z#gIn~JyytBeeCOXc5?nzH$UE!^p7-m-f3 zUpODLn^&=CB=pRXys8E;AZlSKS&B-5h9+qJ>vPKAq)MvR6p2iFk>4=(tDlsG$x!w{ zO(TQjPuxxBMM8E06(tz2}DyT8fqm!gJ%bG%~KPg$miR^Oe==8M!}FqhaLJ zFMfk z=L-yu&u_7wW2i-lj}7B2e$tKVW`P*xJ z!N}Cuw5^i6n89HPKVt`}`K?y<0~p4$9rj>3gHbHg23V5Axn-Q^%H&DgVrlV_6BR$g z0<6q3DSs1EW~^0pq%_{L2gYvna~EUB*L7i#rnMu^LSY-6f`m-4-cdb26jwZzDa#{)nEN?XOrO@+nIxi~o_)RV`Ig>BuZe zt-xC<@?7$u%2O@#ZPK(++WPUa(B5};I+dq4sGMG<+qbMExM#_k?vU43ysFi4tjaX# zFgCEg8ZmLT{bNKYCuHlDW{IFmdId^=PCs?K1cpk%=#`^nPP09 zo%2sNni!uwDNdx5c-nNx9?op~2-}2{jG1$VIU#pcvN+;nTD4_Y*@YUdbYr&IR;*-Q z#;yk>t(4vHV= zl_`TSyqIT#57+iZe(%N7kB>bpw~{%{9aoNvEjyo(fl15vPM}2(5UusCPM3TD4gzb)^q>=u<1wLKd?Xsi5p;T) z3^K>`*9($?aF{>*i|LGh7njDU(v|4-4hbG6%X_w28d5f*cUPrveyXYE_M}OpB$O}G z;-t1&$l0-m=F)JbkdlX^RuD39(fEiBrdsLDQT89PsClMdS5NS6Tla*RM-y1U!_p9u zXcb5fPyQhc#m4*9iZ?cPO^ z&tORIZBk+IA6Rbr4QCB&8@r?O2OMe-?RH(hx5eTklVI`-u=Nt1@qd3P$ zs_lEn%%+SRr|r5?hlNN$7WfoV^al{olR?0OGDr0PislOY&c=L#mFVjtAr2FczJgoq z*XT~?l~P@Tn(%GtK#PO7wfri*f4;PgsTOo{fJ6V@~m2NZfe)5!c(hQS<>9Mq=jt{ zI`&bgCT;r|FfDG^;+?v0S}1Zv-aht8Cw3B$lTnlZX2p7Bvzsj_;at}=6lm(3ht=sz zT%P}9%jbn(%TUuYbERBfVV{+iW$g3iK*4^q`joLbYI%p$`-ffv3);0Xrx}zRbS%(G@ z^Xs;9E=Dcp$$h64(vN{op}0;VhLxy%J=~JH2t)JfF^0S9aYD`RI&KlmTg5A!-|gaZ z?|9=C?+-89uFk)~40o~bq|;%f&KNJ3b-mD*+O+sI)UT{nvC?4HFRN9#(qxRAg?XPV zx&NE>h(Vu>dj4TMUVld1nf@EBhpe6P|1P4&???_XpoICEO`B+)Qj5@hq%ofB%yZS(mv!4$iDgD_}!zo13=njn#8b$tSrxLN?_fi z$>~81z3!5KCxAj$vo&czR9%K8j7VEY)5b`+B4Ib_o9I%|KFgmB){K<1XhVG7t(|Z4 zzGtg(|5W1{e6WQRJcOC?=w3~sBIj|PjDJU%qZF z3fVcBa^bA&_QLRgiUfG8rSLii<)CQ%%$UQ&{r9+||3%M>xEq<+|3|s*zqljSEhpp^ z)NdWB4;3qn`7#!->$aC1FFwDM z>#w`0ogJ`SNIuv-t68DdKmk+<)G`DZ7_heVRAY4{r9?zZ=5hjLK~zyv%%nlZ1eFA( zJ8cBEd+2P0^Fv=;XsX!pJC8QzYD2-;k-{!?Wt_Am^rL4eRPv)kQdIhi*Stt2#hVAh z?BO^P&Vv0BDqRH!)gd|M59Of>DqST9*`W)<>=A6L?PNzg%GSTTX27f-18h&yxdtaw zl`ggaB!KF)^flC$$GO{&9?voY6`*d&LHox7ZLm|F2)HfPfg^G%|Hz(l@^V^}IGS=9 z#2zt4NVD0HJxRy$8LUQR+b)Rq^VZ7fhcMKV+yi%iec1-Y6Q^~I zJT?-_uk0}A<1Zm&Zb_1R;%+p~FYYqeV**ues_rT*F#Wj#?$GScH)7EpUKR2I{;IZ) z>{fnBPpxZVU^8EZcpC9F<0T`9)rW+LN z{mt_p-|RAZcym1IMmuvVEb^uqPfwWXD1#jnuDdnONgDVru}-0mz}Oq;hgRbM11qe* zgtUeml1DhOP4;JT#B0jvky;;nT7jKPlv2K^L~#b|tuQCaeNCNixE@cwm55WiI4bi# zcIwot0gT&%$7DQliIz^FLm6?{t&Tn#ca^x7XJEYL1lE+_t)7Qg5PBQ#0QZ9@+HNm` zfvb)-TI^08W8Zh*O@mSQ=^(Q>yenjxPF>x*M2%|Y6l8v$zXV~lS z3w7E#BQvGk9Q~;jOZ1BaX*7Iou7PM)e|j<#6tE1;rUV^Nl`J5n&6zN)?EZ9nadO!WZD@IR#Ux0uFjs%TFC`lcu zMWS&M2?)%+!54~l+O#jx3bet8rX%jRR&K53gNH_~W&?p1vd0JpvE_Vd!_4$C%tnU3 zQL4a{&pyT96UR`I%jRwL`{Ucr;-U+@4ja9)hjfVv472u)>jemOYL)^n4Lgay-9^^! zUkXa#Y8IYucJ{O4S*e|mpjcY%H$@%es8^1JO|77$Q325gMSl!lT=!KF#NjTUkzxAf zx-Q4o-;lSg{Tl^7D$UR6jFN@}-}?2p|Lh8MBPcGnuK zwG(3}73e)zcnQ`@b%^I4UYb<@<76Azud-oAXi-31@XquDPg#H*++UD4tXhyavRQ!K zc9yv$(TM?AAPaH?K`uLW1Hu2za3;>#mz`9Kv@=Tk_iQc7{s)P!CCgT&b<^!Mea5Av zG)4Zpwk-mWq+hl<#;z^1GE#!dSa3Fgn0NZJnbAS!> zJTu{JzNa2oy$&h?FBnZj@Ze$RwlOWRl#|xNz==W{nuQi;W5cY=e zoMB%IbqT4iLtuB*9w$x}!6s)`aamos{-eVuc_5XKtu6xZQdGYIARLm;iy^(!f4eUZXg`gp?AEN3Z)73nl+ddYGc0#-xZM`Uz=-UQ0I6k%Z)1chA&t=Q{-wNSwC6u zJ%zcq*O|Qqy+gNb!JPm!2Tngnka2N+Ta%o$&z&4sy#XE{+H!o35z(~J05;4Qu)rN# zkRJq?Z#B*cso)V)kYAjFmI(&L)a+acv3Xb zrpNbDKfwQ;_=x*IY&HEPJdi($5BGl~@%cZ>>LiT+d1L>Hk%*NqqjF@H7t0aj4TZboNfKW zGG5X0lBNc*N~bRXL8ilzPG!*PhY=F62z}zum_5m4|8XQYIi2vreM^8g2@(UcgF1l7 zoz6fDEg8RLa6S(off}&4AHi?UEn>QdkKU`|C!;)?NMnlj$i(cke^i>f&h5xiD*3~^ zPcwE%5*uQm&ki&6s7kh^5?ZyB!NLxOi=M$y}A;Q7Iv^{0_i;D3m@q~wn}lC{%+ z!TqCdHNb#?Q(^%^I;*G~!Ytkk6vTnnDlBM;dWWELW6MycwGxqyHb=kf#X;z>3-U@a zNcPx^7*O;y8QaxB{qNx63bz-^wVc9WU?4S&Oo%cyG@uD{9Ddvs861y{?U)F4j{jbH zNIOPp$_1%H{8WCj@Tja7Q-B>)Vu1+WzzBS0l$`2?>kq&AAI4W3qe9rmQ6`DAw;CV2 zlKE*{=x%Pe0=g7tlf}cp^(*|xQl0ossZh<*=+4a8fAg*)dP(z;$V`_|75CnB?MVUo znB{I`i=y|6N1C5vxdnOdD151P`vzHgPjL!k9P9_0>FIheVNCgIoS#92v3s*>K(>e` zhZVpzy&)dmkh!j+0u9K^;ST-Ul=$6&T?h{H^pX;rUc*!l-~WKIMB0Ly4E08k3{s|X ztdZDy{vT-0{va#K|3Gv31C8YW5E^MaJ1YZo69Z!<7b7DRr~ksJN|9IYXR_G08YK#~ zDgW>OxIY}4VYEW@FG<6B4A{_5iZn;f^<&AH4;pf02y$}2O-@XY-o$=pk`!{^t1u{j|jgtT~JxWxn&;#uhW}KtmJP~|RF;ABEMQoWJf(bH- z3jVuY)O9H|y1A^-xz}E%vm*~+=}KVLa*bcNloB>TrhY+I55b%q3vBXZ`dJ!ZXmde4 zSW9XH(C6yJP!*5=PrFv?vMU1|DiDBo6d1tyKf~=mx6)-7gg3UxGXx*CRZ~MpL&U+L zL^d$EVC2=~_3XTZxHU3&z-*x5L@bg9Q)0H@&DE*>qc23BTOvP2-pe-UuDRIn_deW_ zzHcA1DAnd{3h?XKZ@=%ugOas1O1te&kB9B#mbh={6W-{$xn&*CTTMPNeTbl&Q8y-v zMrJ%Wpmb;kSo&gO77%^1#7RN(W;~vhn{o)bLzV0q##Ll*cKkgTJ%c)wUg;e#%yRBc*_{a1&V;_!T^B4{{7u1~4{LXBU-MoRY%}f7&aOeX z@Bys*nt6R6PvCwgf~9DH6i_~vo@wn~C*Dn}6_07H!st#|OFGXqGM3%Z2Oucs;1_Jp zAtiAwxm>{^TT;y~HIS->cN1b~2)=nO?yDfnEx|NA+Z*ABd_gEm+7PgFcBi|N>$4< zby)Ji447Vg)yV!v9Gh8w6ZO!+>I)C)QPUTDf(`qO+aq`I;`9YLfp`3h zAI4`}wfI1s#y|RCoW?)-fSS%d{GeRCWwG}ZExFG*`rw?-6+8Tu-81?oh2^JlP=j&F z^i|%ccTiJs+GGC(e)401MjKSM=f|2%_+f)`2dDW+CwSxV#fSGb{05QjH~J<<_MPqr zd`o$fYw>Xd-hn&yhQsm>yeswiqz3Ml=m&cXyWf5Ef&cm$`*4H!Cdc#fYZrp&E9(KD z^E+CaksE;g6Hf2sKFWVS8^4 z*z6P}%v=Yu&}oQr+IGX|*Rh%_H(BzmW( zIqlq%vn6pI5c!I;4ksVJ@v?HH5ZuISI*jo6Nm!o%IZ9QuW zmG>`kT+t7tYIgfBM1fI^u84~%ZN2d)ikwsjAI=pcZM!|s>lr#SXN6R`HH4#mHHhlMn66ii1y=LT!m}Sn-Yp1^ESb zt1tS>&G9=g#kX#?-Ql~Bb`QQBy!m@js3V*s-{4bN+ZW^t{?WUUwjWMS?(8Eu?2%%p zZ{&rX(U-fy~HTczMH5lVDuRcriw!=48 zp^nsvozqV}1S{B?U&VQqCBAuKEr2g*fI~sgz>{0sS59${9R+qmgy%j;wcqGd9LuMX zkOziCuKlM{wcqel5AnnaK+pbDDwF21ruB<;rAJ_~1wdlyy3%v-{-XU&p7TBAg2YUI zM%?ygC-a?f@hxyEM`G}l+t$Y-#F*R?0|5Fodonx0&xdYBJPaL=K`>!pv9-57-DGGr zAQD6n!$ME5K+1CK~8i2mBd=#2@; ztV=j+We4Gm1pSFuweY$(;zp`gZ_>o7fZGR{Cg|MOYRhJui%o^=}VG143rZY_+(0 z$SjWkRx2Yq;{!{4fM;`GLluq$r*4>GQ|)&*iyx79&=Gp1ozv;hSe;S9otV;>;c;*7@(AQXOu zmNLd3f`i65u&~GO4cBdQ#A1&WJdoKRfZAqi3^5sgyK5;nkw$=7$L3cbLnrdBz%noEp!wC3M(<(#c0I{Gy z7(Uv3MP?_kfDy+!ko0HGbU+oiZ$fPCZ3WM>5=gxORG$pPZnh$h*#fs_V7J&Gs7|#` zK~@UnwKy5kvM=v#dHQn$u0dA;_UICn2e!8S*BCFr){D{qqI<7~KR4a|vEdyL2yJ;L zW5iev6o~E7BfX=s4 z^N$wZqRj1gZd0$DkexrKA2IZ8cc|>$$HVNKuc_GIyTfjyFT8-k@H={ce83m%Epe}3 z>m4$??=jhe^bq970E~Tuf0lr!S@|Z6gVF)nNYb)o7ociW7$JfYnz)(3>A}{ zfDQ*Iy2MH5Da(e?tftVm4$j;xm+;^a13LamL4V%XHga}!LPvVr>!~T9);47NEQ~8tY<4w!CsKAb2jupUh&)@o!Kt7&4@h=+5joOU4-9trlVilqU_}GG zp{as44;c2iqhp6no}4UsvPtAQ-+FP`{Ts&zlELiBDWz5qDt5Vp=Y*Cjp5kAld-H(|ji+m^O@kgsI>k26_;GE`u*9K+xtO0XyTU%qr=z z3P24+8^oTr*9EWwvI+3dvo$QuVh7m-wx{m30knYd@3ArN%S5{)6K@ZOvor2bbpWox zd(+t&kEdb)FF^bU4EO+dgk3`aFd#f2ca&X5Kq=laGC&TP2MoZe&j)f#15k!XE)RPH z?u`P_19igOLibhxYJfW705Jo>#tb{4{_O^WTKAowjk3Xgn05d6VF_`hGt48K4D zv{DQJFam)5Pb)ByGQgZr5X4^?k--x*8V|@uxc3qU~5E0tyB2Z-yHHOaMm! zRKNg`C?FaX1B?MnAEs9wP)Y`z8kY_LOo;I_BPIb6gb$+l;{sek{auOlzadwD8&Uxd z6Vl-rKmb^g>0kg@kxS^G2fq+v2M|9`ga=>=zzUF808P~dy(9oKASI|A1ojUBcmXb< z00E?bDI= z638px7D+p{;;Je4y833}krT=amCpK7KU}Nzx>82HCqDy`VijOEj^G=Jh*%Z)tFaj275r;J;y2!*2xL|r<-4`GcfCwnPo6SYd zDfvc(8M z$RpVMpbkV+D8OWY{TI>5L9&7E%gJOH2G24%gpkdsqA8`7hYk`BAJsWhA`4N9r!ts% zaiSDT?QMzbcsf~-Y%8s-%)<*QB&Fq8$dPTTRPL6W8~X((Qj3xj!uwsbv&7lrU|K@4 zQ$0jUtZCg*e#QJGl~Z*05xYPtB&Y-?ui1#+I7y>*#Q0#fm8=|MDy# z-#Htjrf6n1)N0a$6|MBwnXBh-_*~cDM&6D#JF~Dk>b9nApSjE(yK`)%l61n2`gRN) z&t+TvUThRi+;b-4B)!3Ds6K!9*Leu!;7lRjDsv^Tm zqFFXg)FDgN=wYnGg&D&hL~W{0KV4Vqs>FGu*wYxSRT8+!S_Q2&+s%n< zd5(1smYlCs!%m{9iFAi@&!*Hj8fyAeo54;Mn@it_8IJYBXiqu+G@ zE%qhLWBg-`ge`c-Y>6iAbb#1W)!#01P-bp4NJx<~U$q7`en%Xp4nEN~hobH!+EGe( zOS5h;6oK><@xD~W#ye~a#amoetPr6n%vHyOq?aw(NQw>lUFHGrw1ZJ2hWOBbXYT|` zv*fc@Kcv9o=MZTu)+%CGhMB5oa@G#kD_6z!X)B&Ukl zu>f+X+{v0yk1tM9d=x~5Z+&S*f)=5#M2LwxoIpO$eQD*_o^fFLX5WQhdkEFyf#yaa^PB(BK%kmycSLZ^!}v6^g5SL6&j6fR3>z@BT2 zNc_c|M{#SpK6rIBwM@hbIe53IwG3;O@QcSnhZR~C5vDjwns+j>(|4HpUwD( z3|Rx64A<%H!54Y9T1wE9a^`cQg-F}ow^`md)Grm6j4)xQ`9yWZwJzF(Py#%jMx%a% zw#EVA%y87@fhC!9$T&#xvRc1m3d0FcV*JaFUc|SQJ);Q&txz^Z(#&w2a##X{CD?)-&O!k&o8RD?81*XPS^oas&teRM6Gp->+(?+6Sk(IJUC$+ zDci|ZH(--gWW-1N+GrbI1RL}BWhG29MH7q^r5Kkg#PX&>ADf13HM&^WdK*U~kw1(R z5`n&wG#aTkPm44fXcqCJL@GQ|&5j+iv z>p{`vI~|OTIB>xYBr5PYgZf=uG0>!m%e*<&Y1Cx@B9Wm|a|Yd$6jZtPfD$c&PI zPpTJ!!9Jg$B|7~H0hY-Y*<|qYoeErk7iltP(D}VczEez=w6`>Wy(U;z|F*~q2OQMd z-og+5>duocO)@V(3Qk#>2LDvvm?lgJ4#Ogb%2p-C)#N7u_7PVG#d23D$E`T$9@Ww` zDcdZorL!POlP^CN0BR8gHYFbjQm0BWSV3sY^=RN5oX>PHPZPEW;#_-Aj4Y$2WA)n= zL!sW-Yu+-dsNh*K8ck+wh!jK6OF);Tzybpd)-?q>E2ARhtpaj8b0VKEwV~~re@&$a zW`w4~J9pFM6HA!?Zp6A$`nKc(%9Owxxy+`n{ufh z9Z*Lwk(~sKXjVQ(oZL~w^7r4F_C*#QLq>AM7_0rb#d%Ivm8IukrzbzFeu1N)P0x;H z=paz;$OK2;S{k{oN}VZn%{)jk`6QH*v$LCTJFNH~V8SV5W6~+sL_(D2`xX6w>b<#4 zX@2Y`R)amBwhnUG2z4zfo*I{mj0(-91m#F)CB3r{nL7K-%P8>!XAMP3%e-dP6X%{Y zk~&oAs)$A1auEZkj!}5dTmN4AjloXS8VYixx^QN0>(?#@qqO!$vDEAq%*(=X_Gh z;o0EL;Z?Fi2eEOPe`DE2SN0qF_pD`~GA!wmrhiJIC3Wfo z)q^n-lqdGZ>}YVVw6*s#ZH6NqeFikt}41YZok( z9Y0TvEOfK6mRu4^q$V1h_SAQppc*6`R;)oej~%5#(oIg;PtyHP%sh-YQ^i%7{)N0$ z^tF2sgqJa|$cB{Mk6sedfT}9|%0iR7o+`UY8`Es)LPhGqjNbe_ZMJBegqxXTZUHrd zU2=G;%gKj*`NF9h%%+xa{O1_;d4|HWJjT2e(LE8Wdx+wAxYl2CWLvp@&9vKgE?GOm z)|oQZX%D93bz=s2g{(E^1^ihVw(_JpH~7RySw=lLN;E$azR z3SRcy8t94yB&}pV21>lNdmHJdvAMVy66)cc?ZQj!E8>pU!?`39nGr51jsDVj)38c*$3Zs>S&H#; z(Hv^Wk;58=0o~{1>nwKE<=K$HW_f`S@Odigfx~2#e~HyBMpEoBFidz9bvg${J>Ad` zQKd;n5A}bcO6EA0t*w$c$c!OK-&1j^!2r6nA4k|pZC6?Fek;!JP3usOX~Ig)M;cbx z&8u8xeRyIH|mc3wJml1O3Vsh9maWG;51!viHov7#4LZkf|Y zkrp?hZ1Jp%c+Zm>ryE;vj)%BlfEb4>fey2+Q-&^dXr$r|E#7fv!jT&)ter8ETQX2$ zFz-s5+a3)z7av=YJR2=lD5yXA_g*wMD4AW6VOSDAf#PBP&2_00#_s5~NcmR)O<$7O z(vxNroXI+X|R{=V^>^QG8p0lbQQzJV@x` zqXef2sZt5Pc^KH|q=pyy~y;@9b$s!$)=y8CkM$)#FWbDbv+5 zMjinynKYq;Bc4!$<_{~s{&V)6_YR8d6ri43gPvuewjFYhbXBdSMoxlY55x;eO-GCI z*m*SaBS{8>uc0{~rKwK##w>xcZ#Uz|h5Yxe;rVBp{C` z*CZP=qQ^`nl4nJHKcGA6&QC7ztRB+b9hZWc- zoNAC*W03_%d*qyBi&Cx5PAd1@w{(F2)>cZ+9gtj+T@`BiEbIU8+njqJ>{mJT(N%FL zh-JV)HA$)akKJNA;>4RdJ()blw`gi^q{-cYG8e=N zD{m%l!&9C{gv<$bDVnN@yNaSUE0LTD+s!FabA~fSPaF}0eH}49MhYRx+q$;aRvJQ) z>@%93Yd`Yf@WN;+N`&-h5&n%=oJXLa`L&4^=%Mi)@?iA&G7pr6e&IY$L0r{S&l_&gfPe)~~itf_o#Z~W+)%KfM z7WOYkP22LSI?kiQ-UaX!4VVk|?XA3wgv{NQ>Fz$pm`)7IP3~j}TUqdAPD4v0x|ya! zu#sb3xR16~#<76!oTnU`dpVxBdP%y=p=a-2!j#j*IV8+>rP^VTE!*som$^IKJfhD zo_=#;U{fUui$JOPofyTx!baPFgKAFK*Sc!e3;!QjEa|vNs*zP@PX@03;3#N%s%?p@ zYi{&?MGO(HAxUQ-i&+mcI*sq&DfcE=Gu#Q zJ+iiaaUn%vC9~YKk)GyFuN*xf=R2h{ef9Ul&%e1v%{1q{sG^+we}id%bA!FTF5T6h z(W~N3(N*^DiJ01mb@CO?W=fGXUs_{SLMD<=52w<0FxU{5;tW6YBL}oa<<__^*Dsv{ zt0T?+yjTNuY$&;BvmzX`H9nn;R5$^^w4w)J{+fYwJ$mFwDs@4K3+>knqoxHk$8ht zZO$f!Jl)bv?1s}xvm6cFKF@_C-$*96&+N3bw_KgbeX3g-_Feu{sIfsAHsaT<5e`H+ITr|9w;Y;RX(_yK1a%2c(A}(+=0SwPMl1c{44Hv#v7E z&C}V!%_h1X2bAsQ6|^U7cfT8A{3vOWWYTv2QqYp3$}J@Dq1iq2?4asPA<>m7vQXAR zcj@5_i$K$IZlrg5-#M}>($FT)55$|J@d)*8q#^4FXafp%I!%V{o(kX;h5v|wL;xA4 zXAA}mNLMAQ;cr%f?9Ht^vKwxo z%S-Q+0GXDAn`ea^sD#xyoX5$;)4UYec3XmS2X3a+{yE=Ew!)DXhxZ0{f}NXAq!IA2 z+qUz>8T_=wl4Z7`qNeRyI$6Kz;PXHp1D2ztN7nIO$1xtMXwscoY75GygJ5sb&T=q7~|D3#;aqDSH~Ey zjKNt@%l`zp4 z_Ln@thm#+BUu#GEuo^qjt7^h1FF47UVsxfu<1)cXqcSNbdQI^9VN9k46MUI6L3Ej0 zs;^n&Gw6)-wQzzD`|*>!9?T#NUU2hG_TKvn9_K3|t5-58O`Pa9W3n3<;|Bb_kg0XN zj|daSxYdsJk=^Cnv5S47F_|hof{o2EuCGTjoHQZBF%x_VGE^Mvr=kyCKaRed$NTtX ztUv7GkrkUE@Ayp6-;hk}GUS<wvPn`^| zlQR|h%FhHR`|06}_78R6c*+dPNgjjy+UgTYK6EB#dcxPSK7xUe zi8ztO&+{WyaqPeJ%+%!Eos^gR+D9sEMwjjcWjPR*-ci$}sTUF@aPOq8C9>G@0z8J4 z+bzDFpfqfW`|{k5c7wKyTb_tD*4wC{pWt}cMXi8-Cp)m48@7MBFYtR zY+M|{`3pCzI14P;azgjUXsTVzGS~I6+YF5UR2T>A_zn#*$KkF~Cp}Bgk{5jo zeZ(CB-Vwn^f{%*eL&1kNajp>OiSvc3@!leMckmt&yf?T}1Ro$EE)W-r;A41vmVm0M zTm;_^z9Ym%!S{r?IQS9u@(>{|QSTDsQh>|&Fd;4%e^S^s?H7I8qOo)|j{D?}M?Fpa zS%@n_+X%IXTBNXwEins$%c~0O)5(f3y%tw0Z0lS_`n((>*&2ln&Fh?%yu&f5(_?o& zm!~N9DuAnL1bdRA#>YZqHd|u{2bV%z15#wD_t5+mBtgL^0wGEn%E%3U22gKcdKKCxJTTp31Y_k#Qj1% zARbg$i4VDKZx2E|L{ths5qwf%#WO(#8Z@QZeiI)S;t}yEF;6E~dq~FkIIXf&h{wd^ z08a?qJ;R1dKZoU>28EW;%KTg5s$4d_4I?%CaMmOkYW*= zBtrdE{fzh`znpG=p^$VqS(@2S7UB)HTBtjyI}7oqcuR=CinnngC(_&&3nztmN3EuL zu_cmhCeiUO9;Vcw{OL7w=gyp7KXXP!>HL|q7tEbHABE4GIe+#{;+aV4if}C2SSlU; zmAbz0QX^>I%^rg|vM$7X>QZ92y5*6TgfBcQmC|fxw_^#hU~6kE>NI4`@Ri}!GD!;P zP__~CfLq?V6gH&OF*i3FUmj_6M_$&jmc?fekMa(~!^R=k&>-ajE$zt7ouJc>ktLOg zfk_Pa$YLqmCsr?yrdht&dgbEe{6>y*?o5cki4RcmhX~Q!KDG->uL>LJXaCC5c0-Uf z2Kws(WZMn7d@ywm_5;a}N$W!?RI*cWajs#BvFs8ti(bKGq%|fT*VyMfaB^eveDA^T z-o`{EB|Fv`V5LKoaP)OB{Fe+XP2jqP%buU8gw_(3#?7bVEh?U-G~!pyaepJkXJFq@ zN^-rzX3JqUe`f963JI&GPnDx<*Qs^8R+QH5Hgk`eGfKnp#;h?$(^Dzj_hk*Se6$y5 zDZD(N3n0$X2qFF+`a~0-3-xI6g%Dq=G>MF|U2~N-VbQ!4-=dJBXbCZS6FwC~l!^}z z`_fRB>UL4NvSnu(k&il-afJn5_)3Vc)z^i3vU-XT-$2=UOZ_X&<2LU}XWE#@zYT2z z;V-umF&K@EkXM)qW^!I>;vbmazQc6_t z97#2GNhA_4ZOwQ)DRQMtrU+qeZ9&j27QttzRw4#9U|Vaud=&oM3w<7pO@JkW=99G)Sjsta)Iki=@JBl^p z`e<`pwjy5Ydi5fC84H8Nr9*`^B=|7RF~cxO!jy9<2|hWlN{3QIhY4*QAra!Ckg$dl zr-W35hFN9A*f>&0g=OhlIOkR>FXNZyV{7DeF9E`o3#$Sh8Jh3RuX?GyACGPy9{QGK zfZIKeR}3EudG05KHNqM!tdZ6z*w@*f3;gq~&##0C8KF>~rXz9*YqT{67>~uyqfHUt zoY*5Hi5zJ~(#fB|a;7>*j}29ZrU>;|^*G9E3pwfDAmKO%^k%#)PZ*Zh5KP;`8fT3c z)&y&!2z5tWgqz7es+buY87CGZ)Q$R88?TM_@x9;2#Xv7pZYLKR-j%zrLOo4AovK<8 zPe#a;iboH02|!H_*$s0fv-fG9!fb^Cl=Ax6IsMhctHt$;AwuU8;-TKaegd%X2JZ~S z_71y<&=;Z4^O<_GPJ5s~C&@1-*=1oKbIYqmeIo%T0gM6I8{ntV&-u;7?xAedGFe!a z7*;JP;Q)XXz)FBK(Wr^COKc&UY1UAZnY}ZS8T^Klh`1J=Uc5lAIDx`V(YQ-|5!xQ( zcZ#)xX6-1fovfXOwTnfzU7<_qjKh%(pJY6Mh|ma<*`aqs?~&xpL|2x&jy$AF-G24P z^#dZ*FVruEHC5ay)GO6B`6O=z7xeRFiPA8i`}3G=DO&Vy>^ zP+zo@8Kpi8G`p@04%vP`26nPWH-BpNjGB3_RT$=}i{r>@YBp(+09 z!cJ6}$>sX7=KHd&8!?hpdZRM)GC+X%xD_ z48qfZ@ZdZUo{9oW+yZi-JTUVvC(fS0$l#L^7#T+&LYqwEO{;_w@eq*9r_UV@8zxsf z<3tv@x?f4nw3j1n?>AX9#qcd+;e0IMxizy%yr#2wJ|7Flmn*;NpgNIuitnGHTaF&t z+W$63p9pOV=8Am8&HP9+S!+Vu0p0Ry=rs|VM`}lC4m!w_tA1}SKKAoGuH?gJ)$qAe z;^xW8X5PEau$fnHkq;No&r3e7u1eXSVETFaav-#o?`2MmdSo#2YVYl#G(9mp<6cID z7KP>ubw72n!kYKqW9t0seM`v<$2wDEqK#(f;qrq?q~pn13v)Ft=0}@WmoAOQ(^v#- zjF71(*DXer){@0SxwDhR-Q`4CRBD84U`G>~M@ZJ1*fZ|9P~@s;_4jL%IF7duTpfOU5Ei>4Q62Sd&X%Io7z- zD-#YPLS3yMsIVoSkDQdfo^B5a;2_ub!m;K=GMZl6(#Zg7!yw8UG(M^4YlXF^wTNbl z>1`=euVH4oPVrE#&&YO{^)u_<{@%%Q*^*E1$PB{T3tYc-ri-Mr7l&PagUjocta>s6 zDa*r)2wfk#A-`G4z2_;cz15e5wGY6)1ZXVnC)798H-)v>+Fw{t_Ull{-Cfy`S_V3=}hsZu3giBgr&}nV`o)-@@6sU%q>eNb4X6_H3}O)J<%3x zlw4r@hABMg=rZjm%ASrBiueUQ1Ual<{S)feN-w!m%s)Fp?oTFCyg;#hA? zg#Ii-mxL}CRx3i+sOyDwfRz+l229jL*CoOHkURa?&&biy2 zU9sn;qPShvj*V&^6#86*boINe^;GBbzOWAF!-O^!3ztJV86;0br5tFh1aVnRd+nWA zW@tvr0ktNd0qvseI=t+ajjVeKen2~1vu?u+Z5lL{-qRu&j5M#2f-bE{sU1*5r#7L! z4)Hy5?7H>OuD{)_`#$Z^@K$=X=@NtN%$c_{@OE%E6=E5K*;`(ua%YdESvxzm#8VYJx)azoGX-_-S6~}Y zwsoFnt4$XqLOrp<*Jsx3PSnp(RK{Zsb5oW1mE5ZMEc@cP2t5o2_Eci;g@1VXua=9@ zBUr#aTs?v$Sb3hdcw=LJb8=nV5*!yx+at=WGndMp{XVB5;|-xf_uQVhwJDknkuxdh4O93ZpC4le;hn+ehT z9Yo<1AW1&9h|H+Y3*iWvb%;>64L&TaLrEd94#R4*w@g1>`W`6qX^@WB0in%8txsmL zq|YaTUt;^66XdlD;p8CR@@LLLP@f3)SCXmR@Bct(yJDgmq74<=Y=p|N(C|C~>u_s- zxD$0f#8j3FjcdBlw#CGp$DzTRd9Gsx(38g7K!3%#21WM!O%IA}Cl14GkM=2fG-Zin zw@5u_B^esNb2ur#wdlEa$maxvm$N43L?oCWNw+2ArMN*2gKNQzSy-_>8QU9u{k3FP{UKeq zn;&V25H~bpr?D-*Jf2t?FLj-t$-8vsoq}Q3&9@`I8phtKDA%`m zlvPfVe~7y`?F&nBexW{rCmNBcsUQjkK#STsp-)!%vSsvP-{p3p&4GaXio~k68$w?r zG#8;S5t@fi^3;c1k$C`Om0kA(02ydBZho(JS>?rMq17O>s9#V!cc!~@NA;iX-vIXC=Wma|v~aV~YO&F!S!W3AOsIy& zw5hW@Lq%R2&!Cd!37}g>`;?n%;xvVE1U~KRC0{r(-^+2Xj>=dnHFzGCsSiEHZD~v2U`IU?^mvl9jzE zOMFSs3&Q%Nb+%@mBdl|+^At9&D}EnH5tn7j0}$vAc|^>|iaw7My>q*B@Y^>Xyg&%? zn03AgiI7DK>InM7r~f*ns`OheIXp>Z49T2+Bh*MuDc)nQEj|+e+KoSG+SA+!WdV3PvxFtOoOivGM@L z37t;5Q@kw}D;-KI6V&}i$HK({tRd8C+bJwCWSDO*%r3KghMvv=xy&}Sy@w+8F+UUg z{2BOWySa&YGdf~n*1@1zu|zm6tTmQY(XY-rkvBuGJ0t!Yg>m~xaiu)Ol323LomxqQ zy4J_i{<=McWQ5R4w0=S}H6gU^aJb&1dx!M$jHl2RVoC_PdQ@*2n}czz%-kK%HX}TD z2|_C;{adTZG9!G>6f#+SUKKQX?Fv;-slf_#Q|S`>h-U-N%Z(_lPs#;&9O#iT^2n$> za2AQDozv9v9M=eE&Y`d>yB#F7-w#a<8zI!M)UQcAW%hb;wiMf34fZ#s`gus@Xua1Q zALb?5N0YJ~Ls<6G*fu*#uEd!(=SXSrbsXTl4io1#?XzJc8lMaF)=z}`n)-UqgdGqR zSd1v$(XCJ56H;tOnQ!a1X#A0FDCK1KCjlA&js!Ri?NAUp72sb0 z#{e7(@D#uW0Otdo2XHRH5dfzFTnum#z;gh90k{m{3V=%i?gF?5U=6_a0M`Os4sbKT zpMi#j&=~-i0Q?(qDnfq;xEkP(04D<632+C%I)K{%t^-&Pa4Wzq05<_V0B|qB;{Xo< zoC0tH!0`ab0jvdh7T{@s#{eD%co5(TfbRkB19%kR27u=QUIlms;9Y>X0NwOT_yXXs0ABu>C(tUeUaLqX_;h_=5=kJNSkO{s%Jbr{I@ca3gXK=>pT2 zhLg^I8kKmH2)-M9&)vP;%vErO?feU`a-+(d^EvC}zpyQMKi-D`Axq$9VclYHLEehT zTZMRqq&FWDvIO6bNMhXva683`SNOJqkL05SFXQDR_yI)y&!F0m)V+KH)-Bhn7owu_;Ebj@>3!oTtkWUG-eNc7n9%X$p5|K(#IF2uc z?9hcGeA&l$g*Qicw4F9_;c<|>c2~csc4l68FXi>JLAXX_XX)?1I#Rbfo(`|_Z(w(6 z=d+U&4078VcN(hPEfpLXv7Ed6_KQwFXFbQh(aAOZT$R8}nlvC5yWedH#^gq@ zfulm{)E-fMvk#w%c7mZ8*`^4Vghpg_(~(R!Zgio7q~`o3_x`=*iY(l>9nPDRq*rThgmW;t!~ z)jA*Snkk`}TWoBnraVI2^o*PX=Xui$H6zSIQMu zl~0yDONB=c=c`TKvYo!-J8jVeJM*m|li5vKKtX(UzAX;-7p)QZ+L#=hT#C^uEGUPl zsu}p&X%2s??h+KdL#Avu6w))L6zp6ANUenG+yJVDOr=Fn?y=8$6W#Oh`dQnOle=PR@e={XX<7tX%1yI>BP0BNopd) za?<3&|6ZFm>87}o-snAaFF)SyCmXNyKkFS~9T!=29@rY~rwpM3v zOAhGF#j*W5ICL5$hgf1aaG~vd_t@X0k$HUAKFgo(LQF7?jWy0yQBEp~>omXF3tCxT zf-`=DHzWOWiPgcIk>y@ORbF8w;*r#b!{BscK{876z!f5niDG}8k~F47tPR$`PFTyO zVtfWOotT?gNd}mdDj{3Y*!{L^BGpm$dpeR9P0eg+O|Q1o*2;5zJ{riI{;*SOlUSrp zN8#zYZB7*8kd46Vc$#;NS(D7*L29G~_B7ZqnocgF$1l&s^PPNZF`@0vd; z^KRyLPV|Ti>9=vNw9d4Z;b?l6yj0+uMhM?4}g!)OO@{MRRs*iiTRu+dr*t=01L2cx#YfuBUo=3gJt!AP4}eeidW z!ealuxedhVm2!zN!*(2Ev%$D6{*-}a8$FdFX%{|byb#xWe#vJwP{JO+2?1SN+4pvQ z*LLu=^++0r5^T!#@7UGu%PBdI%6a)z(vhdm&Glt^Uv5GGAC9s=|3`u?7K_B9DH4#q zZRZ;+B64*=&UFyW9apXlGtFMVs4_axnqsGfA^_A}bc(J;^RHMlM;?|btz z+}SAXwBLGqXm9o8{1$B1QZsA<=L?X*p|ZiI)Qasbe)|aqsynmf^L@&87Og$vGM|_) z%q;FloUe*wDcQM#L6Vvt;F`?7!R4LiEAb02$2p=q>@6w9UH^3X;@QrSB@wYavRdl@ z*tSktyWp+B0%4sBR%zwgwLi z5|b)4c+aeyjzUO6muuy8mJHue6mCL$Q`<}@7PPjy>ufqs$~!EYYnNI! zZ&toiVsjR0-r{C63{e7&e`R8BWJM%4RURhHe8IO*jykHQIYNqzu@w>Dmj<`r4EwnI z9;5T+p6Qv-9(JR|4aidNm#5f`59HAdR zv;#FZ`31zSa(t8{D}f>TNgh`3#FCl?1%H;kJUGK%)6JpJPIDHO@Hql3k`s+;;xlV% zX+}tGdPWefI7B?vJ~)hB@h*K1Wb_n2hi~?)EA~e{rsQ8&_JKWnMUjSi#7DKvw^r=Z z3!3EzM4F;W>?#=YOB_}9s_3ryS8RVNW$(SKI#K=nF6DA;zIrs`azv_E_js9MD%Kc054X@lcss6Wo_577WZwKF@FvBTL`cZIbUH*O`e*^HV zD}O(Bo33vV`8pOXtGJ!1WhKGeS*WZ;y`5=g!*6GS zvh|GLPBoN-ZfAN~iFP|PZl){+vYNq6Cs@r?R>KTd%L>`ZX6t zOaHi+{yC=EETLMG2;CeaJWY6^6$8r3D~6W@OVpB3317!dqN%uzS;f@iZkZN$li2Og zy0b8~a*1r&K6ZYuWdppH4e(kvz-!q6uVn+gmJRS)HbCu8ovq2YXe0Nvae&vx0bUyi zcx@aY+sK_Z7BU0h-Eo*@KIH;FrDa9k*RsO%SYH|FaUSbl7A#v!tk}4g^}MOWcNO_dH@AD!yQ% zwt@9ow2t+y)Hkq_MZ?#zewF41)_+k&iMfsqs1zI6z(pnWZmUIQy|s00>k?6-u4AQ@ z#v0bE(%QhbT~uPNW!sfl>)4=sOU(6b`%0_C+Q)A+D?&TgW zb5qW>Y|I^OEZIYh5F59sX#6@hfue6BicKQ^uV>q^BiU)}O!~cwxa&DK%AskJ4IsFR z`eHRR*}*hg4rL|ma2hE`u?luHxYrJu`gWaqK9gv!0_0;=m`s_jA&r5CYx*`@4X>@wMJ)W1|-ceS_Lhgp=fK<%rR zQ0;$YJ!M>X%F$izr}n21^`Rr)DWnci2g*MGQQeB-0y4C*On*I5>joT7#Mg9+qr0$JJ{3!qtFI6jhK0Q(TufhX2n`IYb~372irBk z%Ga{02|-_iIh3H72E%S{iVbWoy_i?2x`EIJR!xDLO076}SFnecZYFo?pbV3_hA2~!9kw)6Rr28FgFEWVO z2!!@l*haQbWudReeW}KhLLX(;{QUIM-IO&yUs%G64b-q-{m}+J58l8QFRG9tie}rg zb!>mgDQ!3=i*Th;Aq9%B;3X)S;u=U8Qn0Aed7f1iY1bDe>RQ%Bq%s{;cSp!URaB5D zglO`W+>DY$rsSpVYKjubwB}my!!pSaCB{wOM0PCLhtX_K;$eHXp7g*w*lg_j_GGUrE7)tw!R&SAX7+~iG<#EdnY|S#WbXv}v3Di&KhAnl zz5(hkBpwPCi;Y*Os?$gqy~CQ-=`3@UwRd=-lcWj)(dmSSQ^xKlHQcB8zC49z966p*2LYBdeya^-8ahDr&tTIDIV zmeQ%jLl5FpTOzxIY8{ERrONGUy}E#g+bZP>bs^=@sfBCQJs{lVbUlu8+Ia3s zc-pltqFU{8_wvf!+bee;Qf5XMTu{c><4xT`!_p={m70d%L`nwE75vqW=gsSDYY9q#0wEEJf-Ulb$?h7GK4-3 z$PoItK=z67-Yp`kcBFNfgy|AB;t11w9AVlEgJ-oIJhsFnIvjW<>scIBUAi9x0#Nk9 z+0lm}I+qMRlmdtSmNNA4P07$BehV3T`P^+(oY$tOjZUfJ1Z4R6+P7`Bb63ql#)_LD_1FFm4}pZ z%2UetfTm0e^iU>yvb3KkOM{*)ebtktM|-mLY)_V6MY6O>%F?7KOCR!N>5V>F+RKxr zy*ydk%af(O)Mj-l$2ez1F}1}R-;=3EDouF@9eZp(%F<5m5Ds=6)CCi?Vr}5%tao{_ zW)E62qW3#mBaZ)stV&0+Y853~4))eEOn01*=?W(BVoqgVzT2A48;3K#mPX(%pi1%^E|8Q zRD|@b0f)!>J&xwo$~`x*(>ji8vZU9s(<^yJ3D41Pp1a*dB)sY{R()hTqh0nh+jSTE z^elU@^2}g8l9f`2PbJ2l#0mq_U8m zN9xPv$|82HvKPBi*_%|GeMp7bm%Xp-$39gSvu~9B*$+xsDNq_{pf)N~l|7Z&G(;Pf zxUx)1D5oo}%6ZBG%8N8)-%`?YfSy7_a91KE4lT$+4N6)i_+NvHv z!|g({nUIcyRo#I{A!p!G$R2oQ$~rmR3aCXls7ZPoq;_4QrsTkzsa&Y0<-m(6ZF1_= zlty-l+D0MY`hqMML(48>g;&O&UKuM{D96xRMSrW+106NxEXs$Kp6kks{z%%x`An1- zoh{Xf;l;u88824PbLOIRJjLYPwd}k~twg(poe$MQBhBLiH}8c$7389h=`QxYKCt6! zzXEe``_~eSOR^N0YSxq5a1bjZg>`^(2pg;%#zrfLv&qU4M2(|}OO7Tna}06Gv8+)! zp0y|^kQ#U*J6!n#J4!j3our(?&QVTfmnf$Z=bR~fyp?)#G$}0yf%2qWyP;KH=o~LZ zbn{f2>pYd_I!~p!&Q@t;=-}YfQVF_FJ%mE43>`|LkWb;cu2Y5QFcOG|t4BDN?EQ`< zyHKevzmHvdKI=n2mz~G@mTzR2t!0<5XMZ}6nUE~iYgl*t9WTJ+@Q2xUWaj>Pz$L8z z0Jm$eu(clwU5U`mp8Z=!{LPeeSwK0DrppUyBwoZSl#7YRm$Ds|%h*)qa<-fDXSP7O z(q|Vh^k}kBJ(B38$k0*L)&TvlSC3YY@!7?rsAb2h$Js4YAnHNa8#k~ui^|){sA9Dw zxQ<;_8LB7=tz}o24ad(lB_V3iwPhtC68zVdBg!wmu7~)cxEmmQZgd`UiL!E8s9%!Z zWJ@YH@y+cr`<0INB)gi(bq(82xt0d|b!?P!J)21L@{Y<)>;UCfV!_+k4I~6yy_uTp zk?cmvGKvfxubv>;=tj9@Qy@bp(%Ya6{eeV?$_A1kI*EuJatRtB!HI7#Ix8nNl&?Up zoZQBAEOy*d!1l2@94snWolc_$RiwQR$A>_PnAMT)q1 zm=!{gyu0Y0wd~&W-G`0za2~rDsqTA(h1Re$WsqJ~TK*J$O03~>LAIL=6s=|VQvy5b z1Lw1SORNf6Y(3&03|zuSmIPg9dI;QOfo~pmrs)S=rT>utmOi)6k=A=H=WUCuI8XHL zMFeFE2?FH->bwU@k9>%YP#$69l}Fhw%44K)J$L zZ1o&soZ0L#^<0`BOg4hOtDaA9b=H^tq+Xz2NF6_r{Y$+_PMW3cJ9VlvEtYuGVu^aO z-Tf50L@rkrD81E7C7%n$QZG|4CvNVgsOq1nJk=AD_Pal`pd%QsAO^ZpUE?s&Hqd4% z{(I_(M~fa?%O2mzo*-6w606ux!5*g00xLbjw4$dGKqsqZi3|vWU6beo-rNQ-X zv3Wk52xW=5r^M-fJkCUQ=>1B~AB+5U?3$mCaI$r`QO48rdt`P#_G5jBwC^)Z`5Wt_ ze89F+J|tu3V`7m{NfiB^G=$G-cKm|vp?t+wC|{GI^9?&w`IcRz{DWBI8+N7gPhyXM zv0IfN*kj6%WaRwB-X#iu5>VK8fda`QmlLVaB)M=Eby{~a^oPl=7o@ull3j1HW7syb z>$|bltVz9^W&o3|U{UoN%3-km*^$yv!G6s5o?Q6eWeoPcvmaxzPtd8@6HuPuS)q9 zyt!gMdkxCKh%!R)_4C=_auR5NS;Mx1#WZRW#>sZ13t)sG{Kg(7D(M9Z>rcaZJGPC} zpb^r@9Ln^-FlGdXv+jWrtbbq>HEcZFE-;#H9~i^N2F9`}fpKi7z$DqQod}h&q(NZ^ zLuK2OZrxYyBb&Ih*TkK@Chn}>A%#PCBE_8@nz%u|%bAJlsEPR98f!Ff77n?T%)4@D z!$Cf3!8I&c&~D2hFvT}DZS7URwcI>VDCON=1xHCbWB2toMzZ29x!N+X!jS^~DqJB@ zU2abiYVf)>KfmXzQ0^rJb|KnKWkUng*zmwiPiHD6^Z6dh;T9{EV?yq)`gULMl?WO5 z2C#z=b|AsAGd8flE>iDiPb?}Zdb@5>u&iz!duJ}S{oQ#R*n5jMu=f{{i1^z&_Q5*# zVKpK5QDu-KJ|^Y91T;}@noDX4XuS)Y%06+h-3n?+U^iAAn8*4BY6#oisVD2mOsi+p z0t>vJEM^tzed_&`zaJZ-KA=8Gm=9;Y)Q42GW-QI}j}YcT(i9l=^P|p2_q}p&#=s{X z9lh^xY75$WzhkD)E!c4Yc6u&nmzE8nE?mz(JKfJWBWg{3>mIE=}{A(t?Nezv`o zg^Ghal?@ya+1}PVJGes{X>af7xj0g6m(Xp5n1;T7# zU#)NF;*RDVG9BY77nzs#z2Z47FfxS4XkG81P)?r0*A8e z0*A3114nqYA57Ep6Tz2>&coSZ>XQ@-k~-O_J|z`HqNRGPy$42<_w6`zK{9u}`ZT=_ zcydv`Q=cK8^6BGnr08(jNQyz9sl@hQwI4mF_%_^Bms#;q3n%kVj(Qj>>wi1@s-4mq zID@HyGkxPP}9FF7*IkI2{UG|a%+9T9QVgO@4d74=mIp-E051`4@@ zeWS4UNS#lz>;hkDhS#ju)YqL-W;o5lVUDlMZGLE9&c)0OT;gk-@ERxNG94QC7Z0x; zoZ}%zSpgA>*zwy;d2(+*a5)j{PlVf_bK>@f`leITagLZ8i&G|FU&+*h>+H7rrBxqS zTJ2Lky!=XARDIAvRIMSVy^0M9T+K!YuJuIK5T?ln6tG@wpeL%vdd(ZF>auwXzWScS z1_^ayg9L76B5+$LHi$zp3cW2?P!#nY_1zpcNXWH8`n2beb)DECcFDT>p8CGMB0~C& zToFicULEi{wYH@mz8&1O)GcT z#LIl^U`ho=|LaQD&6in5+3WhGx;njgAVSV!xms4DRt#UyMh-7gVeZ?crO5Iei*hQe zUFz0@z9r@1Z)yp3@fB>T z`T=#N%1&iRNUp>ZS$}CuGa1^&W+4iFXs>=#=%e69|9%r<)Q|lth9g8ip&|CE`kBLU zFVY*Z&;I2P(?oF_n@8eS(eNKgCJ3I5Ky9kl0*?if(ViX0}6s! zqNuxR`wcL)aGyfzX2>p^JPA&7f-V78;UT{-bn4RU~C9&acA~EHF&{ zN$ns6f!OCc!LL3`l?%v6$j6kweZo2A%gVz1^-U6gm6P!&AX_4GL}1-7>ahUG*AaJ# zOE(i(cX}tRD&pqJl+N)p;+2H0JOpW+WzkG8C3-YWyRIo>U20$ZzNkvsMOjO&3+7eP_!riu^9#f*> zi>U4w{7WJY4boPu=r%NSK^Ol%3PQUEdAH@Ll!}JS`KMegt30%*-A}Z01+Jpa>5tPh z8tyNSC22URC_D)fWea*#r(#G#LHxgPzUmWIw_&Lz(6;bR5(Z#Ok6_^JV4z2%vOxsB zUMR%=QB2LnWyL(qdinbCDv9+%F9K{|s9aU3E?h1Oe)~^L`~UL1e;Tj-IgEX-ea`8+ z1wTVBG>V2xCq7SF{_%2x78IWrSez8LVft=Kp-LW><}ERM4J^->4yl+yRFEA2Z9tO0 z_(*T}+ZUu_86hD^s3r-Cofskcuno%!;gE;rd>8Cv$zzrG&kww$shBDM#|ni+jV?uv z9*7zvVu&`08beOQi&6c!y=_=ArU(BDIY>R@HkW7x=xLIuk`r9HXFdxAbxe*pD#=yT zCgG|X91|L#7O~>$Ld?d9y@mi;6Vq%FgCr{5k(Dn}bIe2M6Lpz&_;U;W^}`t;u(thPJm6dcak4GMdj zH{&e`dusDjii?$H-Gy++Rl14V7rodJx>1vZ$rZ4<3hCT{3f!dLHZ&ADY&-8v zPtuhg<2~3Z-jg$aAvbv+dba+&1@FsS@{4#^elZ>CM}G(LF?=w;jbFm=<-_;`{8GM) zj=ajR;BWFP`C&emf6vG9pZL}M4?cmP;&T+_Hz+Q@MhWqWN-CeEl;e|?O8i=-7QarZ z&!;Il{CcGopP{tpvy?8>zZbtjN{2m=3eDjj^)Ka1X*b=Z{!J|vjzClY7&`?m)Kltd z(0SKDK);pHO1YVcQ~qEs(zzKd*j@vFtxLMPwL#mB&E^^Dq) z&V2yJN#(;(LeN434TXSL1_zSLRQhO469JF5a-pWkd6RUdqf}uyR8U$;Ip)tBk;Dzg z9T*>%R2U=4nk%KLn&wL`-jC#&m`Fb*B$q;xt2>#@M09mXa`7BPQh0S=Vi6J75HcV` zVhGk8kQE`ytQDB&+pu;_&nh%So{G`4Op(+1;-@;v{j;QG@_{E6>jeKLpd-8BaPg4> z0i{L5Wqbsr8>@hU+#L`TQNmbPI&}#+NG?UpbR!Sz%`;FwVEtf_0ZAhGUyfD0{1#uw zm*4}aq~@NvU0|~+u{jsGeDM;HTgkB+_|T%qtjqul#!Yv1bguHIEin-Yx(0ihi}4pd^0ZOTW}HIiYxhEe2DMEjr5qC zNXgp5x8p9p1E1wP@g@E=zQLct_xK+Cm_LhO^5^k;>i-Mh7oWn#O#x$x*|<}QS%*`B z3TWc5lEuiDz+;ldIMl*-f*|QHFrw+CAgZtzo2gX@3^eF~+vR+m4yEyHIrnD3VXP$Q zZjmB62^k`(Neq6#*#y{}=D9nx?+HWJHC^009(G!%TS1$r1BPDRU7I3;IIb91lU6Y)yNlw=b4yQ{_XAxFga5~ka zh!$(J%S9b<`ckMtbb5d|{dMB>gT(1?5U0OMoc=aW=w+WZX5 zQh?P{kTq2pJ5NzqdqpGbNoTzklMPfXHcWBYDC$2}3B_l}SysTw!q}Udh-aG`=VUfM z&dKa5pObMG{2_&}0;SnyV%y`un2U3$73WYkmEMp^M9@WWs8*OmeG{%}NgV29JXz;H z6_FYSX@VG<31YZG5QE_p1F?SdB}5(pX~jirV7>J(Vd&byxEK58U*g;*N#~XqR|#w9 zKy$G;%@rn+P4Y+yH22Snt5I+`L(&|44}S@ogYU5!-Thw}lQqw$IVFSCoHC@~L?Bfu z2c?zrP+6$}os>$@Q>hHSm8vjUsRkpI>Tsn}1FlhO!u3jRSfyk^u2Kg!D%r4AsSDec z`tZEc0A5lW!W&8se5f>qLrOFFUg-osE1lslr8%5ZVu(r$G?nwvQ(9qZr8QPi+F&)M z9oA9WV?*TvY^HR=R!Zl@(zl+F9Z=Azd<6}=B;-h!gdAy;kR#^@a%3Yck5eBu*5p#L zgpbKT@^bz(q@+hja|74Hjs8rXq6&J{3%VBMXKZm+aY6DLP&x;Ps4{8qPUIOMn5M#Evh)hrq=<+tia!zq5NS~Q&Mx5`GtX?`ml z4X67p&9~QktZ=&Id-#QWvm-Sn3`x1#-FGs`W{nnmnIMZ+CMZ*>R z)??9dMZfh>G+f1R-5(8C^;@?`!`1xOylA+(-mfsp24cGQt zy`$l%-|7?%XZfvG(QqBV)i@f?_FK_txUS!-6b;w&TcxAn`hLrah8y@T77aI)EjSqs zH}YFQMZ=B#))&!m6TkIgG@RqN4o1UG{nox{xS8MD6%9A{TTe#AF~7Ag8gAjYRz}0; z`K`O7;g){u)@Znu-i`bWd<{8o=>xV_(M9}Rc# zTd`>P0>4#18t&+~YWS8!k2ofmqz42pnCzHR(BfA%#nggF)ZGLt75L};I1%u1enQl~ z*k*Nsh+63_EJcFQz67BckzLpidnp63pE3{!DueM-`uZ+P3%9VJRG8XSu#^DNOJg!nEV6Jivu2&}E6Ut=VqFjqRm8tlQG7a}C z({aBt17BBW;+x8B{6Lw5hm^VaxiSyGQf|T@m7DRXG9Q0eZpA;91q_viOi>mwTUpFJ zy^!HuCj&Qq~x)M$~LxG*}?8po?=Ut zUF<>SX|`I~%^p?uu#L)dY^(A-+pg?oyOn+HIb}b4NqLFAs=UJ9QeI{6DhJqM<#qOn z@&@}_d6Rvoyv=@5-eJEg@3Fs>_t|OXL#`?xaYH%GL&_1JqI|+5%BQ@%@;R@ne8Fod zU-4|^YhGXZmN!+t<1yt2-bVS6w^x4VU6rG}hjNTxq#Wn{l@olZ@*BTQImyQ;fAF!& zUwoqSH=m-M<};Kte71`GW|i?^+Pd%+qb{3o;?~{tH3-}eZLSBeAg#tB422);15$CDcHtxtPtiWuFnFWuS zk`Lhk9HD1F0-wm5<9n?l} znc4)dP;+3R+6?Yeo5OB31~02E;DFi^-c(z`5vspXTO|TRTf<7#7Qvm+4A#ncNf9i1 zG%=%bG_Hog*Ksujz6{im8Z@%>V-@(Cq4?J{A0RcyBb8%`_?ls@n9teHpZtuOC8-qUP(MSZG#%eZg;Wdvo`3wmDPo-e_T1tM|08IN$kt2# z;o3swS?((kQWbAcn!YmA*CXVC4ka6#9~Eb8l4Wqg66o;HMrSg8cZ^TwCj9CA&yvlP zZUxMfJnR%Rc4FrN5n|vjF}0ZCyA~aOnG|V8@gmI*o4zT`CzW&;VkS$>wxv+17L>BS zV$x5F&Xi??5cKreCN zRbPAS_Ydv?fI7jut191}RH^#;x)^=BJiIVQ{XKclw!G(^*t^JV<~HmTv$Fg}JXXZa z!@fnZ$|~7oYW)`;Q>%!_G-AZ&}@PaJs2*N5eS z{^GzLz8WSE3@Gu{eOTPgCDHo8|CrVX6(L6S{)e~nYAu*AO#1zXJ{aAPr3|v^)4GjXoOf5x8QKD@FY~xVpL9#T9gK_BOYlB*2(DCz z;u>`r?o}_tAJh?eOdW}*)KN@PN3+W6<*b@|1*@lC$(pERSxa>S>!eO%UDa#ZRP{PG zOP$7UP_Jk6)fsGoI*Z+>&SoprIc$wOmp!J=V~?vhu{`x=_OyBn+oRsfUQ};m2h;`Z z4fS^RzPgBgtS)Assdut()w|eH^=@`ty_fx|-p5hBpVv~C@_Onro}(`3E!7pgwfX?> zsy@U=s1Nfi)YW`~x`r=N*YXF|TvGeik=pkN|4x0B|Dry|PpTXEU+UuuQ#UE5`h?=B zo0U{`i&9qIs${C$lxpgBB}?6*)KzyXP1Ie=dFs>3Fm<;wTHT|JSD#g;sLv_atIsQQ z)fbdo)xF9c>OSQOb-(hy`m%CFeMR|JeN{QCzNWhB0X3{1RLiMvsFl?>)!OQNYGd_7 zwW)fDl$pb72la^BQ~g*Ss(zx5P(M}YsGqAht6!*#)GyUL)vwhh>Nn~t^;`8}^?P-r z`h&Vf{ZZYe{!G1ppsR$}^@OIXziTP#A6mNlmsUwVt<_e~XjvK%ZG^Q}S}CFq z)Vh%()Jx-9e@)RY(KKzOW@uMwrZ!QtwdtCp%^`*MX6nCCD<##e`yqsjp^5J$#8YgG z-{K=!e=RK6wf2Oov{Iyc+3HZZM9vU6TDSb!1#L-O9ix`7j#=8GKpk`FI=)Y`l{(1x z3Mt>>robopGpv`BjyfJwaCSE9H4ODyEkJ7gYne$n@bN(}xQq4@ZlXK0X$N3u8gVdJ!7=*@HxO zpvVA`9H?x}N`}|?I_>;*sFp|u6pDoQ;FuT<2^J~()^@xiz{^)+r6Q$XgeH-2L;}@z zsGWybii7*WjYdlO;59bs#7HTBf_^7nRYdfPoJEmR5o?=3P!YAge>;xbhT~&SE~Lgv z?Zj&I^ESLXmJ&$`P`L2ASEqQv6E-t|3=+__6S6hxBi8`=H-h{?Ph7lsbvP632AWh4J46Oo` z)ha@!RtYL;RiLU?73yl$p|Ms2`fIgds8$M+f0q=PRdFvD+Axw%E={7W$_Ie#b!Y!9?;4QAb?8vik3<5 z)q$$GU#m#xdIi|El2$penZ-&?;W`dWb~OrmbOTE3Q36}e3)r%^&z8NVRqY30ZHJ~r zjr~g2gffD2#msYBE}brTxKyCKG)_!@2V=_RM*Rv5oqJ}DJSQc=_krgsnZK{`V|_X% zafbM}h$m}POe=Uq6E<~u_z6za{hg*8JzdbI)@Bk7&mtP04du1D&|13@I%zk-McRD0 zURwaOwApo>*ny=0{|B$d{XP@n6Z zn+XhulI()iB)j0+LUw^3w+r;RU7#0d1M3A8_nZDpyFf2y_^w5VuPNVWT1j?6dYn(r zHkH@M=j1+uFzkr!6UKI&8gt{{t8`&8yA!V~BG)2jgsie@G0pdVWr^Gk^bzPkX+e-` z9xfIT~E?!TP=t+`9TcD!06>5uUK5ZLGpB*H9o`R9uPU7fYa0}J9YtO(vgbw#> z&%(pn3$R|>3r}eK;3@4zctP6_Z)-0RzP}8=X|Isvd6oF~HMF$@n4-Oo&9pc0eC=4prVS?vfR_s969_6dHYeTK)i&+!lK z3x?Xa%+Y>e^|hZ^Gwo;AK|9JiX~$R}?F1XF{l;_?J-W*F>l!T(UG2jifUTtgXx31XQAwZhmpP*Zhyfn=9ZNj-R*q?1rdLo7FtcAmhb#5DP@ zacRfCi%UE9O`wvd;AdJ*IZ^opYp2x`f#YmCT&LA0Z9v0kv5|i@%)?fJa$1|cqGd^` zSdqOWy^9uW2D_vZYr{ykNUI}PfM&46S~j)Z0@ztME*0T;upn`w#7?WARP=$fM+qWl zx)3=&<0MO+0AtT>QY8&8giGo2m%?yaCoN{wOM|6nfTNdzG`%d8)5}3My*$*=GvPwL zB3z_bf=j6$tyfA)crY<8P+&}4ppZ)C3zTw1=GBTg)$1i95Ej#G6){RP<*Tp@ijjh` zZSizQ42*~{38CVBGw~76ZKg7BK2^br~~Sd z8TfQk1NGWO9Z{l=EXdUBKrOv4)YI!hH@!ae(Hp>Uy&;U%8^Jie3C!1X;BLJsEY+LA zvw93((pz9yZ;9pfR#;K*fmQXMSWiD68|baEiQWcV>TR*D-X6Q?7hrF_Blg!j<1oDo zj?%l~Sn7X`-ZPQHc0R1t>dWLX4U!@hW+v3!nF%#_QbNtWCQx&SV>ub$ra(RHDd9sU z<@Qpofeb56rqhNAyK^wnshH_!l1`oC=~R~=?S}t=PJ8*$mq~;P4-SavMGU#ffK;4I zR6-`Em?@tpqPu0HA{mvr9dB%Iilrn)&MaeY$D71r5?drsqy9Vb=Ayl};Vm&z0{rx{ zG_Eg3=>*v*mREGaD^3Ak`48+z!7I*x!7I*Lcx8Uv94+if+a$TC1YR)$UNI!E7y+*s zl2;7LD_+6jJfBw{#?6vf9>$|0+76H5(FCuo6$+0&ka%Sf@ycN0m7&Bd!-!WdBVHL! zyfOm%>LcM2eH0AUN5d`p<*-P<0+#4k!ghTu?AFJ@2l{yUM86uo)hEC)eImO0BrKy( z#tQlr?4VD@?)r6jw>}*o(67gb^#z!#FT~CI4BV>E#2xx9d{&>0`}7;|fIbi3)o;W@ z`px)-ehYr5--<`6|8M$2$xHLVg}o32~K+};IzKDQAz}jc^d62@gr0 zbBmdxyEQpjJDHen3uXrLS*--+xOXr$(?d+jfLQw_*0}q=VpXma+KvmFYPoQJG~6@) zAP<%h1sQ$)F20{S=sZ_YybVN8Gd&D(cHg zCs_gQ^p$Y6z6z%555f)lLvW+M8W!qnV2!>O*6HhDv;GL|(ASgnc?@3GH^8gRWIe89>(&q|MM@#5wwYT&QoyMfwiBOMeQN={s?i{xq)BcjG1! zpLzOo_>BHMzM${Lm#P0jeSbnKeNDX5TqG*PMftpfhZ0ihP=Z(9Oz_IV_=Gj)1@YHY zafIX$6>h+RS`#w03reM!)*?t>_6qe86VR_q4)KUXVycoO<%!UF$PV{K(&9o%15@#i zA`BuE%ID#o(oIUIOLEQeRj<1edPusqM6S0BItWcf9!rT|vkIN_pOJ)Fcuc8|v?Xg` zIuV7achVzHho@zol;1@^2)6zPc>0@AR(}hs>u*C_{axs;zX$#F_hF#^0gTbVB~|1* zn67^aGxU#$NDjem`eC?3|Cr$RQ+QDS4081^;Boy+*s6aGJE{M3`ge&BIs;n8KO?-J zh&y;)u9XxP?dvkYk;!B6TxYKbpV20ZO?+gPY?5c?N7e{pwn{+Mld_(@E<++&;bphl zdGx7U#y>Uh7S3d}{ff?k@73?+PJb&Fr4dJJVd(X=F-ZSKW^jyFE zfsCDa-+=5QI>r4lGh3`mD-fKrB+pOHSRiSm8Bf}nophtw{+hM|sJ66_!hCz&PvNIT zHRZa!Wkm)uiXX@*G?4KR14Yc3o_qS$qU^Ce&K^Zcd6R#vMD{QP_An)Tm;rm3l08fx zF$<3=G4p&}fsYe=i27&xT94x|q7K+&CDbJLI7aMooY>=6$k0zf75z6jUq1<5^uGu* zPs2o_>6r$?9D~Cxh5~mPDm-Rr@T8%`H--U64HFv}7B)9*oMt#U({OQ);o)sY2yZvS zxWp)hD~!tckWmF6H&XBkBNg+EG~8{Z<6a{JUo#^3j!_mrGRouUMkao1RK%aD|8GW> zM54jR;7zR+v4aVB!B5)x)H3jifaOH4AfC`Nc%0=JkFy*d3s{a?0m}tC2(cWV2w1KQ z?$BBnWH~uTNi5f<1eOaI&vGk@u-wX$Snh#yV7cnaEEmaVx$4O*_kduzRmHH}gT=Aj zL;s599{yjk+{6D2dbtRVTMl9EzSgt3r+=aw)y@}=e63bmgEZ2`%ZUC{|!2hRNu5FyQUJ6; zlI2Pjr_f-`krc4T*Zos*?KaF^0u>WWM0_5poA9FNs<->S)-B;1<3B$_>b8uDk6HPL zOgUL0Hf!;7AG#u$R*onBO;H~NC*l%YAI=3eTt6oyk7}#LQ8jTiobCyJW`h8v&QKXd z%59P12H;jIEvbmL%VO!(OH{-{gichCrdn*bBpffdP%WbE($YiWmaEA6$Mwaq`lG}c z%quWQ@mOAd-b67+kvRY6V$Q#X&c_2gaYHe`Z4|#fnK=FNV!CV+T^8-cCyM#)N%7mn z#Oa%h>9R$1>Ge-{KjExTkKunhf6^+bksc0@JnKtBN(eFe=mdQ`1$pc(Nq@K(M5Zi1vUD01l0_hJq-pSfx1jD3)IEc`S5RLV)V+hc zZ%|(x)RzVIsGt^(;c4;DvP;}rH2&Yy;y>RNQnmzFNI_BlOk9+^g~hJB$v;-2D0c%< z?n+Va2BO@RqTH3Dyj0;aOG*EodP38M|BmX8ByM!Av#5iV64S`M1miLi<-}uSMqmB7E+qezq8Vm7O<91wYEG992Cq8G~jV~GZ;FrdI zj2X+BX{=z?jR#m;<6+jzSk1;7YuSy)I(Cb(p51Fa%APbfuvd)7*#YATcF1^=oiw)a zbYnYjXzb+8j9q-3@id=eJj1scyZLToj{?TCifcTllrf%HDjF{+HI2PWCu5(|(|A$2 z+}N*-GhR|A880jAjaQT>jaQYYjMtREj037+ysl;#Z>Xz{kJZPGPt+%k&(x=l&(%G~ zm+C9VSLz$a*Xjqxck1WH_v%;159$fyZ}m^(l%^X$YNqj%<{3Y08OBj9(>SJ8H-6Q! zjT2fU<9DrvaZ+nz{Hb*`{?>XJr{cRpzzjZD?ILsUF5r{2cFMa%-VgC1T6=1RU_0-m zb)Z%lhVxb01wx5|DSV;UQ7Sf9^85YCIi(Y9mdT7)%g5>BPir^ zypV7*7Nf-fTenj2N_-Xg2|+~EQr%jZT-a9Bff7GO!~mKq*ro=asY8ZoKo!%1+NKSS zO$S<;F0?T{7-)uIm>Gt9%~G(^Oo5NhRQS$JgP+W5aNMj;nshocvoz{v28PWtSlTR$ z70mKj!_34wW<_jlR>GKB1zVfdu%lUBQvE2fU?8m3x=NLtAz82kO}o4!%CGja&w^&mF3c|Wzqg~E!ecsu z1*_x7WI0zBHESRiRkKE-4p{I>vFgOkCKjwqELe|Nus)PE8$fll5!5#uLsPQ}3^tp> zaI+bVGMmE~vjt2tTfubme7MnU4Y!zW;6bx3O6U4XUCj#$_1MC{fX zyPI8buGtlDGrQrdW_LVj_Qa3OUih&&48Jfh#b3+|@wnL=PnvxgnthpWUd%#ff0khm zV43D1R^1%Tvdkf@fjNvdGcT3wJB^s?LF^^j*TmYeU+Y0F3y&w+)*TYIb%%s)-63IH zcSzXQ9pa44VsS=hO#?=*&(eKHhO#V6>q*CqBrChaSy;MPd}2+RAidMTmPwXQD?FJU z>=+bJ0P#w&4$ctsY_G-?KP7hG|1|otd=PzEJ|2BpaJ1YrPnI~~50D?Aof;3&E}`>U z`Q*MP8wIgZfWIDOP@ugIrgBqYK8O{WzO4{Zv$o-DF{1CAqv3&KeEH}HcHrxTX(?Cc z|2{ez9#rJ_N%_BbjD`mn`F%?M?|vLzW=U~$70xA&P6Vu%DsDYi0zE4hj-!ic`=GYi zEZARwm5K)wncNCb@QL)P!7fmmIL$h@)8vs7<^|8e>Tc{dQ3qOk1t1OH90Qhl1vuuFkYvPB-kb$<&Dn6fc>`=Q=fO_%MmTBSOk8^lmNMsKnt2RyuiE@FEsDMzUDnR$h;RvnfKuY^M0IUF2z~qGQ81Tjtk9Ic)$5D?lRZl z^X6K7+04Z^%ysy-xt;)L1Ab#}#P7_<@n>@r{$@Twfb%3%&6k*IzRc3iEv&M+mDMow zShl&Hz-9+KkA7=SzjZU8Vm-{Athc#~4K$x-L(Sc6wD~L>Z$8H+n=h~#=3X|}e38wk z{)^0)<2nZG5zI^qKw6dnq-BXfxjPbpa*KkQ$tG-++=ESlOg2+)?xR8_HeGJcL<8Y6 zIa{gI_qj^0-N*+dzyE+j`DZTFdIvyMkcZ4xlG{TSSG6E!ulirEG;h5i4EY3Pa>4eJ9ws zut?svf4RKQDL9yhGZ^y96lj zlR@wSG&DaXF!>1jn}=bjd4#l)kKtUsfp1wReq=fLh2`Q`mWM~JF#c|p!qZj? zQ>`=>w$fReRh^ZwYOrcnX;#C^U{R|KYiLDSQ>z?1-^yeitqQEWRf%0>Rc3>%s_atg zKiaC1AMyjUwLT*B29H8xtuHaR#cJd)>S7ZUz%VfZ3}X_&FggH+p)5_iNNOq>Y_Qr_ z?(0>F_0cXC`?ui+*ympZYO^kKPd5V^vNl>jYMBxuM5c4Ue}B?!{9f0_duja(Edm=5 zhYfA6zjj~gBVnU-;Y5xJw9pf15pP(M#@;p;y?Da;0M2n6mDWoiZIE`mC(C> zQr}3>w}zjDiOhCQ(qctliuo#T^^#QF8ijS!gjld6c0?Cq!H%dxERaoFNG#YzaD0;# z6eWlSI}i)D6bp7B7HlaN>=IX9_vgPViVzD_w;`BOly;&H^vt$`#I3quS@poN>O-2< z5HhSrP|0cnwXGazXf=hFRx{{t#bBt_0v1~>;XbPsJZPN{>#WxBg4G6Iwc5i`t0Nq@ zI>TwJ3!>Ex4XZm=vwC1Xs~0x6dgDRsGW@|Bj=x$X2>wSgW{qZsbveQRRV>3A$Ii3H zvv$_itcx{)&9Saw^R0<&fi;=kWldo(Sl6;wtf`z?*Ky05#$D@rUdEciD_Apm4Qn>< zYt7}etsD8x)=m6Y>t?>tn$PdHZsqq`3;09ULcY$roo}-4}>cF}whiwSh7|bVneN41O>WNVcK{0?8Io9~Vf}3IvkUIWCZt zj=_?SA4oL$;|=A%O3|c3hH{=1O&U~E8cETlCng%77#clKV;^woYXv+9!(Uh;zDmnhrWmgjXp@Mmc%a?5P;&QqZ zHGu4U1ITU=Ky-rudK(1L+rZc6M98Qqth&P`rr$3?!>t*_q_ira;UwN6*Rc;aTM9laW#<^r+n>_K456IDT2jnVH z%sO;kEJIJMkIO28o%l)7;C#bhA@`|RGnuaC;b##q4L{GrFSEoW6bUQ#N)9Xb3OO6U zI)|`g?<`@(U;8lb$ra*BxEuPHEA$F4_!BUG0k6dC6O4;mBu1>y$BR0E@f_$*Fusjo zd^^GT4uWwa3u_m__|pXA&k&67h9+b{G_#(A7S{7{fwdR9Tl=7|^&;GFy$b8C*WsY` zCcI_64M(hZ;8W{8_{Mr4PFaUgu@0kd9l^%dr#Q^|97kJU;1$+aIKlcFCt2U%wbpld zgY`Y$V*P-Nt)sZiI*zNYU-1R&1iozjhHqHE;|JDBvH>M^H|uW()@jCVV4jUE z&1S5;t+3X%#yZ+M>tP%0Lfc}4ZI_L-JvQ17v778tY@waP?y*zZGCPeuXP0I#+Znu+ z9pUBevb>TV<<;ygUf(Xq8`|Z0j-AO{*%f#@yAtneSK)o^s(gT5onLC#;G^wY{3_}{ z(aw_4{TP(O)i6~XOc3hf3(#M?L>mHhO<1oD)rNt`8|JUZO|1aB`Q$is^NE2i z(VaI8(uS^Q?`W6G^wSyqa_ur7y7^FTxPWe`%`etQ__hS^u8ow?-Gq11Mo~Wtn(>a> zXt_qc1#d6ct-JYfF2|UX7^8n;jB5*78kcKh{K*eGS6Ujptc3G2V)8>%a{P%lP5LOT zIGWg!z`r?bwp`WE;CX|C%=<$e+PZBB8S#|mn0S%KzO=NvS*bB514 zCBbL5{6U~zIr*^P`6{TA4}fLl@$ZA~u50Swn0&ktY`g*}jHfCvG?8 zzX{)T`Ts?n)Gg_K)ce2mJ_1ovzXb0}MDu6H{8;}V;*+XQx*nf>(c-$YJeCqC-zs!Y zGR`^y&N>ojod9PYiL;KcE*BnCD$I4+7Pe11Xt!{Yrg|SYL>(ZlTnlYC0^4p3p4|j8 z?50rGZU$BD=1|ve0b}jfFvV^ItL^r1(7piPvpd3vc4zq1?gC%fUEyoHI~=onz)8C& z>4JUGwlBu8-4Dyz{jruk5F6Qpu%&$oUSbcy5%y3VYY)Tm_GLK59*(o^5qPsb61Uiw z<0tkN__ci{erI2Wzu4pOS9?7EZckv$zJ?k0L>9KMW##N?tdc#QW!cxWX7)_h&Ys1( z+jH1e_6=;3J(pc?&ttRfo7k=P&1|uK3%lRGl|5wN#@5&i*d}`++hVU^JM5KgpM5*q zZ!co6*^Ak`_8shyeHZ(}zK8u_-^-5K_p?9kB^>N!T(ehj*IpU-%dq{yY>^`{CxM$~ zgI3XMJ<0DI%-K2RAg&}bk5p5B=J=vBz7de>Uar0Em75gDQvae^~r>?)Wyfm zs_b`(=5~BnjbPks6XW(zcH#!JN>>n8Ua5@@mTR3yzlmhx`z2OJidnApN7T_Bw9XkPLz zQNQFnghXuJ**Axi&vr~cTe_N4FsXgI_L!gO@Bf**5lGUY7$hksmAM9Y!<+C9RKY4Z z42P3~S`j8-9th{~8Bt>h7s`4CZkNYd{v58o7A!j#9D5z4*^fX)`%&m?Z-Bw}Mi^;7 z4x{Z&Fu^_qlkLMW$KDL{?JaPpy%p}ZcfdOPDR`5t^mpuCga%K;m-aL8qrDrB+RwpX z_VaMYK7`ypj3Ij;X4o%cC3`hwJS3aijeK=FzcT_Tj`F?rKovro$?Hnb>IWy~IX)Z^8)wSDYCC6-UN@ z#bI!9a4vB{B4}fOz`YhelUVe%E6{od6yj z{&Z`Nf1JE6=xsc+Pf|<6X#SQh|81s7EpEXyDr&*}T1?W&6Y>3ZBxNTjjXdije6eJB zRMI9{1Cuw&$|@3^>Te8rE+HPt-r?Lc56Rd4TfSZM`~OD^^^03BnT3zw4wAN_{*7My z4m>I9VAkRpAWE{oB;Naqc<*cCy>B4X{uV0P-$9oB12nXMgqVF4+StE9C;J%mvX8?g z`!~4OJ_$G2f546Q-|(1y8aCT!;3*OY&pHS%I2>MgG6FIZP6oc{l)+b>viP=B9uGO0_?c4? zzjG?#QKt(2M*aVGswXzEX%C&WabkZdtP7X=`%A%b*r$z`i~e`!FZyRjVy?nSEU!GB z;IzL3PHTwo2KGfJey?5aPh#RXa>6oyg987zfrTb4C&c;D5YC_w*#$RAek@l4KT47~ z`A?G!#I>Y<@f?Ham|XaN9KoZ1oRvq{)Aw#dch-SQ$3TCTrG%-9qlNkoqq5FVN-D4A1_ei;ZMC^Sm6Yc^nVZ^`+pZ8%x1RH@?t)LC5(a)m}J z{wY@^knWr+BB7wy9=MQig(11nR0NCBL7sfDiWn}lz`+8im3Y5+BFTl6R&u0_APw9I z{_KpH@$BLM|BodsxzjJ-NiPbOAezRG##1^))U0$f-AK2#;a`OOII_eWt{2#1mjrg~ z5GvK9;Y$*kk%uH_MvfKpaV4+)9mLt(TjiLhHn|4qb@6X&E>h2^rmsZ&9? z&x;W9!-7`i!IGF3arQw~vG>y4JUsm*Oj{eL^D~0ZBlJ6o?1)xODh{_~Y532<{Kci> zxJa$3HYI5&>PdvmP1Px)_N5_XFq?oq8!V?TI8HrCbLvB;(*SBY4WXUW2rh6MLuaQ6 zbaQf`r_&UAJI&!Drv(gl&Vvz7OIYl*fn82Jc;0Cb-#8b*&rU};=5&TrPB)T%-H|&z z(QQM&QxbME9cB%wVfMSW9MGh+_{f+ zcJ60AoF#0Cvy=^Yma(gy5Xd-$);^GYRWuTs<5r$n9oN(1L5rLpsha=!Db(!qI6>F&I)^mh&_W1KgXvCdn{ zwa(kh4Cfu?M&|=%v2#e-;v7+SIv*>~IiDymIiD)8I-e`=IbSKCIA1HDIo~KhI^QbC zobQy=&JQYgepE|2KdI%NU(~A3uWC(KQL|iCZR(s*n>)X$EuG)h4$euni}R=2+xc4^ z=$ulAIcL<%T~NolOr1pir%`{QQQQum+C0!1Tp_;`d&l1t*BR97<2r*nEv}ub(RkpK zIwlBwxjT#4l^L+pa z%ml4i<3anc8dp z+u%aCAKu~i#}#fne86pw>)Z~w$-MwKyPa^W+ZlJbUGQ1AEADf<;{mrPzU%hFLvC;U z!tI0KxfkJ4I{!Dff7~~QF=!K51l3?*U=e%^TeL~qWU>g>#rLSi2Ov!?qwc3;Ta&DW zb4yq>9`6dQgbcC-ruY#}_?fg9@~s5_$iPBrS+0$9t=^sPtSo(o4U%g!i?UKfR=T@`IsUHCp4@BN%Y%m7m>UzhRe4S+jHSf4kWP(+{9UKhB546@brh|DwbCQ$ z&%%NwP+!!yEQMOpNO~UgmOw=%Cmiu2>2%krVk_y?NP6;C(p%*vReE>{OpT=Eu~IwWjRM!atiUx>#Dk>lV5!uHE*bP`AzDG740^r{;hvz1Izt`V zNIY*w^7B5MduHiQmPT)S8%r10%wwfv>7)?GN=vPYl@UI?3_8EGgpQ2eJ6U8vBz+qz z>pyTg(Ko~In<@I1iIfpnr@m!&6XK)`P@d;v?&*wZ9;>hu78f@sZ0~_?!kOP~8><-4 zomv=RwUVZS{S>R`oUbiqMDYIw0k&!-U@KIJxzd&C=CR7+(Y%OUZ%>crNxhg=iAS6l zRVM#c#CfFPaFN}hv2<3RRb&~g9qYupQhOP%%&W0V5aT`g5CU3JFC)et!dHkonBW*E z78kg~2xu=QpuG$--4Rg39SQZ^Q4n)4hc@mQ=o?6Yli{xHAZFXTlzL7JTf^hM(OzaMHa2{&MG{=FY) zZCK4+i0$0lv8%fXd%KHqfO`jyb??OS?p-+9y_>-JUcA-4pTKx2E_RpWJ??7U=B~ky z+_m_bn~UGO>j-2YVV1jrrMsJ0d3P(T=sv+JyHB!gcQb3^Zeh*n+q84Hvrg{QtSg~M zclRkaz}?9%b$785?lWwxJC$AIKFg-K&#`6h3+zF6FWc|F$PT*ud1d!CUd?@p*Kl9v zjonxHdG4$HeD`(U#XZP-xS#Oe?x%c+`vxE8zR5?pZ}GA2+kAriF2BxwpU-hW;5WM; z@kQ<-evf;EFQfhsx}OSxfOkSI_!yeVO~KRQCFmgqLK^la$e*eWfzmht8mq!tl!5J_ zi{B5kVR+E*Bd8txx)<8YW2JE$^wOrvehc#aY{RYE9JLRmz#e=|Zi1f5 z(xIPrgEp70R-W!)9*vUD+CgpYM(rjj4L9Kr+Re&0;PS@7bv<5#_~909KB=dVHpKyO@xd;!*4#>c4@*^akY9YK!7et9}cq z{(WkT(ifZs!&kKCgP@=`$@G~ zRT<}$uYFc4ija!jB!N|rsM}bL{N&P#L_WfwqC3a_tKEu=+c`E3#=v!iGolvLDc8Yv z$e&iZLMRpPw}dm_5zc&13f)go&ixsxxJRLu`wKL1k3l2%I4O0%5zhP$Bi)lQ#xvk5 z&xGsTKVZ81C(LsHf}7pHVS#%Z?)Ctdd!!b49M*dZZ1Ob7qyD=*GqDqFKd7SJt=&Uk ze0u&q#E_WWpjJRZkeJ+n_~b@8*bppas|Bmn&FYpyX3o9ZeZjZAzIgh=wTL>zj!4^BZLt$|q5S-t zg6%Am1`;QV!SYyC1P{LmwUTG-nAn*rJD9Lzb@J146g#-EirN%62d5Y8s{2H79h{+u zn-A~G`YS|P`$REL_|3dhV0kIvc&U))r9*2k13G$TpqCec-dhPvl8{VZpA5x#MygKlUmkqyrb>WQH09CJ{ zB#c%>1Vi9Kb*s8u&UKYa>?oiob`(&91kfGfWj_HlTm`Eo5fq&3x<3&*`KF)yN6ZKj z5mYL~(D_rk#SnwoU^h+-Vl&=kVo3h%30e@}-zdQ`D(0U>zMj_Lhg23Q@i54?zDsOpGo3ZCd*!Z*mZiafz7EECnZG@I02Cd*%|cHaB8FljCdh z>CQvYBx0w#4M|gy5s4oO9V#Sss8AkM6*^Qn5*BUJp~Aa`T6pH=+%xI^cCaN}@vN6% zb zUOzbI^@o$*078d>gbsty^DehW)xo2I3}SSRcjLs{-|T0+y)=1rf2@7&q&PRQqwyNh8p^&>ttswE zwoYx1=-lHQ;lCj?B=VB0o9>)EHv<_wL^h{qT`kopZD~$*xNO`k-J*jObcfb^=sfT| z5be;a_xMKfJToXP2?JSIIOKtSE7+u#IP4HY>hNXb@5sEr{W%;zA@JCCcw0aHh0SNR zFa36Q(+64~DW*CZ94_|_~N&Q2x2Q>!7YvZ@Hz8m0oh5{DSB6XTe5HXh0gVwZokU3W41N=fzalWlhq(2^~ zU)#ZZal-GC;X!cdPvTM!tW*APS^jeY@2I3w#3yb&u?QdiCK&|a$R?ZMJP;U!Y>!by zd`^OfW#saRK~!Sg>e(S`u07Lzt&Vs2oZrTR`me|~$6~MQD?C0Ze?iKoG)Jo6Sd84c z3VVSxj!*euAUZpl*VmDqI#YqJAdHrNFT~47q#s^gNOJCW@edD-=l5HISe=i+nnx1v zQy)kHMW2mVYCkk$K;lY0P{eRLWY2%(5>5*|J_|O-f^O7T{)S*_H|)iegyiGTgZxg- z6pEnPU5G$pP7|EZ8lhCoouorjr5 z00 z-38^vRPnM>nar;?93ntktXAE}QyUPUf#b&!n7b~48TZsB(U#Ko{sKOn&jE8(B;915Wp{=^*`$3yQT zzKTjp!rCjJQO?Xi&jh2Ka18_OYq$Y?7=(jGYntDJo4c-^>#uNi-|j5OL$(AV2d0!fFV>hZ<)YzXav$ z2iNt$F8Vjf>l~w(A@d0%=K7V8ARw>@mC#JUu?KSH9onN2bSKAb z<8dRa25I)K$Wk1aTD_iIL1O zfB%UsvP0}G#d%NT&1m&$6oXoemZy8i!UV~=fJd6wHN)-9;xpsR%<&V9G6cHsZicvp zP?nklP@V`wM0F9{IHB7JT_@SwGardmF&udKT6U> zTbXTq^3`KzANstU z6!0$P_A4n6K<`HHlITW%FBjd?8&epk@fK59wNb0A>w~8p;qj5*Xt*5pUS8NeScyXy zNpMAqYM$SfsCzH!-dvIhrD@JOdyEdAX+=Wz6Jso-NHBTExrCsN@HE);*Y*rOHhQ{D zE*p$k@L^xjj0`h_Gn+P1Tkszb!#4#fPw+&A==+d7p z@0FRN0a7a{B~f3?pFjw&#>t#a7hN;_T?lz6cZL?9(DMfqGGCNh=+e%@j3%o0*;UaD zSqt$U^_Jv22-)Vhv8Ot6>{yndO~A~eWs_!V?kPlz?j^;kAv|^Y5;0y+zwdLTjf0EO zRi*4iGNa5y@)pqwn95a%enx~RK%m=aiv} zUCU7{n$+T8*D3C!z&x~GjH3d+L!hxo@11PJ|Nft`b)>TMdB9&?s_dfuOY^6?dT2aK zQtW-*@UMqG@!b7nl)y1aw+f@58+xAPQ~MNpGyq&gWGt1efD;AJlpP-GIOJrPglWyA zPeapNdiqEgV|+o4)_b)q{WGTQ?kojrq#(Zk!4MWO%$hN%z~dIO+!}V*Jw~&}dqHRq>gw6bW2*H#w!+7J zfff($>aSh!Gdv!44;z%1sCW+%wY%bH+<^Ooc zr>Jm>+f#yrA;ayWj#m$I(EyJ(S0w;;KGdhsv0fW6%3k|{+8|#B{LD5XYsUFiS#CQA z4-xTkw)*aN_U?AYKzHXrH~;4YZMfea{1s`)2$7xmZ-X>G zO)4P$DPxq^BQ_$9p-%Nl(%8yKX|^0ume_u#3We!mh{Y zKasLRRP)$og1`T>!#0rk+lygzg8rDAW#o?DlYF?z2b$6W>xz}9c#APYdl z3ZdKCwWrV~tcpSoD&PGFv}|F~hHwkA+P$=A@Mg&jwhKDjSqECbQEOufz%Ta`_1|j% zHvWkAA+CA=hboL(_|7mdd!U0dY6OA!9P}GW(XgCGd;Kh~Hcz}9bQ=kq*!(#R2+gK$ z!RvJt3w{{)+dnRoB-Cc1=r1Hu{6&hyt-6DWdeuwW)~x0K@Q~`-8Ii7;8F{Ma1i|UoeycZ#uAdskbF6OnbA>6DsYD?lfhVfRNVWuh;F>Kx()VGP3w|_clkY6= zeTzC#H$yB~ikHMf%V5yLu3hT0p3o#)8aTOTcQC!S+P$l<2zS~?2u7<2C&N%~K)#P` zX&($&C$-p39tc<`rm_-=2mUUsGZD(RCSpC??>h*OUlslG*;=AF9n>R{V0u=1<8~l! zt-&Tn(HO@JYXn;&RF4~G-0FLQO^u_NBT@-loxKXHr)6B5?nooV$qqe!u!{gQr^LX^x*#)`{>*V8Ee$6=1Q&DqM zQyYwuTohZA2#)F@_V7C!k|kTvc@o|5qKl*!>Ma-vHU6TVq_)FKR#zasp*@Zn64Yr7 zTb80UbB6*&_11Go$uDwum9{nZr)(zvPRfF{;u%BxyBH zO6Zjk7NdMT#JSQTWDrRPXHh$1ROLrzaU-}d{ZV5$NRaSbZvl zOSJ|iCb^(bEgnu!hSr-w>s);}+qJ^XieS2qAqMfhWsl2ssW$$7cV*zi)mpV8hC!;# zsC?JaT4{`@iB5%pCy582CmA|MN6Kheu&{M6Zf^FvV{Wxp>*R9fAA-Fx9Wvm7<{tqQ zEgJr7a%oIz(rQdA&1%en6S!m|cs(N+GKz_Fv1m3Tc9X|^tMG8uBN)#~6)|XL_ zf=3LuO?wn6Vd$K=)EY*T%r)+(;eQUwV}S*VgyhjHf^+T!=-;*|EZv);*>wkQ9Z04| zTnNPXsJm4e)d>G|D9oNom_6FGZt{8*ZFj6Hixh3wwa&ekoqwFa-M-!c4p_UZ$J)eX zmTbjhwy;b2KeSZR653@P_)GA0e|0^|P>}5cA1g?i3Kw5UlgJjIFiVH_3UIe67u~_% zV%#LyvjZGs*yI--Fi!znftWFmc^P~7cTSn1wK2#*&f(<}Ko`mVBLg>q+RWKgjPf?J zdY=R0IJYl(<0Be}v-a>%Oipd^9#);H9~s?)oA%BV#3Ro~pKwg~b~jSggtKp^r*EVSKA- zkcV<8(boyjrKTfKc%}TCpU62v#m%J#215!*%$SA_rh&bbFZ9iLKnf7YSw*#4lwwKq zH<{Lb>D_c(H2&7JbWZ4iZ80B*x3Gw}`iNPJ`od6yT4?lcB+od(w#YbP{CL!qBJs{o zVw7Ao`>sn}&a=M9iChMr7}wJ_d_Y^nu#%odIR%^6?8?ZWWRBcG)Flt>!+?6Xl}S?U@a~{^5wJ z&)O|lk|dYd@lavE`^57&(QzGU$Qz;Kb4AKSc?cTSXt{$^aUh!Dj-k!$AO0q2MJy=F#~G zYF1X-@M`eMR8>V>{g(*s`QWAVgQ-iV&x*d8U(*0rT&44uo^Ph`irA5S)8I4M`X&C5 zt~39Dt_zZx9k+P>{8a5Z*%9C&^%f+N8rN=H{fuc3R(Rl#C|taZ*y-?X{7zpb24mFp`TnUdNV?h8%~-_V;aHZb@JKY?P)`qB_V-M z>RZVWU4h6&L;lP8@!z&Jf3oh(vvcjtQ*rHR`FwpS8~ygFRG9spL=>WnT^I83Uz$IL z$V{m_wb8Q*WWLm&o0KykvGcF`}%P5sE6#S!|%4|Lm@gtqglf+}zM6b^LL7P)Y1 zT@;eqs?c*Wra!@+|HHweh_#u)*{i4yGkjDWczXzzt^VgK?>Vy%gMf=Tp^_TJzvS1= znZgN?Ab~B$2E7m4Us541BE0kmr#H9X%SblA?e=@=BBx{}VIb+$p8VP~hq7z-AJ3^o z`|4YQ<8yDU5N^4Cj3bJt#b?+z;2T2a)HKWNGt^gBn`QDD5z|q32zAQt+V%+J{NgQU z;aDZ}=={ww6;!*mo5xIur6zn>rotMR; zW^Oit@!!z&3)X5YKVh_o`1Sw9Gj~f|?HLdGlan(3jXn5JYIq|)F{#Q#9P1V_?H+{F zNQ)0dM{0G{aO&*dB&#f^-Nd#ft`+J{ui^-dGzk%Xuy+mm{|WG)u`nFTat^;LSL#L)&VeHbwJm{UHW{3zO8Ke{f5)vg zw`CmBL>pJx$s;#!9R9J@duFk)pIHse*FHHwK&KE$q((W^KBiw*PVcFM!MtAZ;P4URAcTsDcR7QnrV#hL{N~3+K^s+kvm*xpq;r;XNq}EY_*K$}DI11(rqX zwVnbo)O_An7r{QAHwV$xj8cv*`!cIcf4gl?&gn3yJE+mA4lS;!gOyN0tsn_U%jp!l z5AT7w52!9`BB0vC-PJ6jOMlaoygn@Dx5VAmng0Pw4{=}qInZ82vUPxPJ(aZkPPLTM zeP(vzNRtaG1xXoS?SU5WM?rd1LD4w*&LsUko(Yc*7J#)!5+Ea7xRDAsT>5={0-|-P zXCMnXM9Isz-T(-3bnV@*VokC!DKHwi>CHqf*AW)#c({OO09?7!=DO$DSfTMX8~8M~ zd|3%ph87eMeJetJHhrUUj+{$coLlsN+KhkN3~$+ln$vj{4k^b|c6JHn(cwED^%PQ4 zV=_NGMI_Vl*-=o_TRAyJhS7%DQCY@z?`Ox9ZH4Z`StbU6a1nCazLdeLU?h-2dhM!0 zC6L5&C)rUGwg1f?;0Cr?Rm3~`k}seOVxZ>YHK16nUE2@s7bx`E&ruryERvXSv{INe ze@bD?;(qIw?8h4IjF2>-LR9h{N98nFFCPL+#3$6ZZ_?Ll_2v{b5;@1(WbB_B2Lw=V)`_PIsba@^A|`&NSBNI$rL19(ckYdQ|m@vjOvPXq#WR?-rp2y zmK9{W@(W*_lf!J^E<750Rt)A}^lUtbqp3D4?6_4^AS>aEo->mSRkE~@`k0eJZD{Hm zx^0+xP%4;!7`4Rh)ntmnYPr5KZeM%fQWCYtM4hBTxzs|&cwHgA)@WTLJt*D;PopXM zz^CqYdV!#6!l+*~bR`0&jG+T&esU$jJ$O!(~TQQ1Iq*(W~=tT(YP zi$+bxz9H9$unLEwV;bNcuD4+t>ij3E+6ycPhnjCVS6w!zlI?);rNY(0Oka_&i`WbM z22~sh?pYd&q`{a~=S_y=dcPE4D#}}XsZW6hF9O%@y_k;tBTvID(pwXNA&A-oms)5Et`saFs)It#s*!|*dF<9eG*zn z5`4tLj&1&DWP#sf;s@U^v0G_l^$u{0Dm6*0Zs@csVPinYH*xJyV9?oj97MZ5ie-hT z2^WMt?cq^O9oWc~vp)&R)SW6j`6iv;vf2P#%3%^ZTHN`akr6_TfVkGHUikjqTYIvwr z=HsiD(+r<{%v(-|A>&wmM~a`ycCDFMRYuIJw zGQ`q0a59TAGm00GS&OFj-JYCx${GE{jZg?mTD{o!gzmp>AmmDHfRc>_^vdh~tZjbL zYmzhh@GjHuv)-VZO~BteJ|XB8LJ|m_MB`k!Ir%>>D8k=VrG@k8f7g8^v2C}TlXdUU zgpzHV*N%FF_Cz^Vqc)?_oSgTJSd3^SUO9A-{C$Y=HUA>x^@e4wYayB5-hRGHv3ztH zhy06GPd`|RlxpLbOOD5PRwHeZ{{4FOZi<`BsZ?^a%aZRUdXQ8;D7;X)Xx zcEr})_UCz_kzPh;n02989mj>FyoAwEfB&A`GIn)26$jX?pmuUHjc zIi7|2ii5TiQ zE|#Q$L~Z{dom8D8QqDscrp@s~Nf|bQAvbtKi}d0TI+^;b&(tCf_x1Ec5Ydfgy67if zdJ#Mx?N1r3rf7A(q>G(GE|iA0;GdJKb3fN5=Ff>`>LY(n5*}A}eScqoxQcs0jZ)V0 zXE?R0(c=5=wYUt#xJLw(5c(UPW?VISB21Oq^@`pDyi2qHSe;5f$@sw7mVgZFJmI9v zi)+(8q1Q`5@l9Nz{R_7(g709>N|0Pd zsAm?Uihvn%nO;r&X{vR{3r0r1Z@+2jc3trT-MPg*t=2!^>1?Iyc&<0J^?$mt^eXz{ zVTZk{AxLY{^#w?6A(aJ4xW>o1Ff!c7xp{#|xc@mk;C^vB@&YjrX9Z#G@&mV`j)@PZ z>>W8asGQ{wc)3RuO4AAIcatoEM(~mSYa?x`3>uc{LG^_i4k?!5@gu(!XL5?tTGP@r zcNJS&au~*Wa-!EKQ$Qwkb-Teq4&Co&4>e{_!^xx+lv*z1HuEJN`xxs%+8HIlBnlRj zF291bjy_7=p3@P0*6JVU%-eB4&O$yt#3k)lEgJ@2OQgQE zn}W6Q{ev>-f-%Ntlfpi(ojk@qwXgOPA#F~;mca*-A~ z_#2^oh2&U6gH1-xdifSA-rF+9XO&^rwNt=R{1l8j6^e(aE=qa@!+MuWi+UAnjhGR) zxGrhUyT{Q%Oxszt>3UQ%&X@>eku@DGf`ZrWv28&x?aywOWM1&2LFJq8jcDhdw#g6ZAjJb+#NAm_Vl z4ENo(Qsw4Hzm>Rc;>q&GfRXrIBou7+S!E;@Pe zX(~5e3(QPDgxRXD7Wr+nQJ~cUWxDubR!341w+0{;(mCEYkble*k{2P874bm^Z}gMQ z^P$qViVHG4nwy{dVyCwb$64aTLu~9x5b%TJZN&C5KDv*%PW!)KQUPTiApwWk->EiZ zd$}ITTaL5A_1cI+%P*;R+Pt{m4zm3x_~7?8oR}Bi3u;f6;a^rRG3n%ne{1;&25S6g zlMU3MNziEAG17ACT@|`Yy|Bq1S|6z0kpZ}p_OYDNw#L><*jH#qF&sW2m>|EAy|~s( zPTBwo6gV<$C{zYw_KSB{P+RUTIwtHR*8%L1b|;~aPKcb`pC>YCO6?}b|I;*3XjZ21 z*piN7N^bDwuS6Z@!Da1Q!Pt5_{;9J_6{6wyWnZjA>0M{$+O%qTU(%uCQbu z395IZgYzQyomw~DG6P77T4SXQX( z03VE94*Ks7gT~|d#3T2OsJp2>smW9a8-i)tAg#V!9J^8PC66e z0D>Tyj-SBO&jA}vfqp4t9sK8_C}T^6PLG_=MQayJ8a_9K z8R(3NQc#8bZWg(YsFIdGCU8s#m%^~-QfoFW7R?j|n<)vdTF}0aQW9>}0SJ${tdPq> zHjkd}1D)XrU6VC4ofuOsZhga-I>yHrA)pPf9hgVmr}<9g*>~`vcD*6o%Ru+gFQ>zO zUx_5O%M=t`FPFr z=uy-npnrXdtTE%`kKy7saoWmMJ9H_x^6ye^Rr%4(y6x4;b>bb$YyBOA=jbC{$JwvH z%R$d%d2_p7U_*d<;yp28zpJXC%}@31ayzN-ayyyzQXm=bYZFwFRy=M0RK#ujR84RD zR9I`VqbB&N+-4Oqr&muc%$~t2RZkademzFs=GT^D-10qs3+)xm^)JzL5L!(RGj^f| zpSm57rNWKK_GE&8xN8j&5kF9_oO zma2J4wS#swc&C~=#|eV}y=c7&`M$g+8IWmzKeNLpzH__xwh1zKy&MT87rcI-vm7yS zx=D13@#m_XX+apyUEvvMMTml6L5NDNOBlZ~hUQx6AJ+~_bb@tr^gZ?OqV8X|?KA11 z9z^RvtJ`5B}M=E3& zP@V&qmzE9;(fqDyablmN`rXq!IImk3GQ>*Mph6X)$b1k8TM1b)C)HQ4N%JH@hnQo9 zV=p=$bf$^Zl9wzhS|046X|vGac)U1fBwzk#urcCqsmjoprb>r!dB}YO?uv4Gcx3}m z4Z-HNXt@ofy*d<`O>ub8=y5bE0Sva-C|Ex=jy^U81j0Y#KL$z8>`@6*3CEQ)OvwHB zJ%Du_!8_npvY{+QkOsFpN+Vv}gmGFD6gCX5FFO7ck>tTGovXfiS6W z40vNV42laUdlfB6`aM*EE|{v;A9REgzbA6H*JAyEyB)Ax@wy`D&mReY@d$F-Rh1&TdrLgM(9hd;Zvy^E+`546SMHx#Id zafp(X-i%awdhnYP;AdQxBqQ0n&5rirN##-5*H_ZTSw`!h3M=JBjIkFxpAdjG8_>{* zsarslY*p2g+vk4zDZW(b5{JId-BPQ0wPQRGOULdOm%8wE`FnryjCoW9p6wBvl2P|$5I+gxwD)u zl)exyktGl)ZS&o`7;(_};@iXw;XvJ* z@XBvI|M!M?fIMEXoH_EIwG9H{+|X|E8lax%U61Q zzFAhX^Y8RVCW{jUeDNQ9rPZR?gI9((2f}@r=Y}UW(dt4TXTdtD=gQE1O|ogK(5q(4L*W`&)=Tr3_)gbI^p_O> zHLaDzUO=Af4fN17hoQKpQJI96vMBU`u+ub4d1mj8%{xGY+gL*)#l>S(uz9(Xm$Gn{ zMuYPtiTBG~UIi3@U))zVjY|gp>6a`)%VOpf?yceV10fK_J`JT%gZHG#vEZWZni^7> zkxIgKXHw5rSc*KtC7!MavfgJ;#Fe0^Hv!Qb1Nu^e-nq35bzPna_C_OhaW8l+b2}rm3v$=xtzMTk z8gCh%z~x3{Y;z)(j0f1rWyTtj3)1<?y7tCu38s&+%SB(ZtS9!f zJRf1tYu1xNyH(*eH|vTAzKnwRo+1l@y4E$h0xUn%@skhsHMQm$*dMjN?w{g=ghXsTi^goqVKc@Bq{EO{y2X9bC-Z_T*y3d+Nr>^c%t2l4m zRAO;H-oBM)vHpuWZC&3By?L%gHXY!X@O|*R=@9Y##stc5l{QddEGo8)mG-dEEqDYY z_xz#AivZ_AA9qsQ%O49J%1h`z{;wZ7oA_-XwpSqASwzkt>K6TL8AKP(PEcP<^iSqX zQ=e3VKJysc)ydXenNY%8_Zl-0m-9G3E&cP(i{TJf%o=m2JSfh$XkS+hJ$1J*;E~N^ z&$9U$RosL!Q}@r}n@c^5+N$-03J0E3^5l@V$rscxnNU&1mfdF){r;N=KcIT>ZUcYD zu^vb{1;RB!vZ04Lsg{83;&3$xEEsy0ofRbkcN^HE@dOo345Opn7fYYUCN2fsW3M<8#Kpa)%6a zS+9C4Yki>(App2nX2O5E8UDi_G=-)p$do`5h%I60j$&AMKAF6*Bij|=$@f>jtQ`|M zEcbpV0q@HbxKzH)ow8yc?Z>-6-MoD99-amP#mGzdcZ1N`A1iZgVC$aHhFH`3=Vv$p z5WK?I2Gi~dNw9$aTd=7cUWo)&gq#Q7pmJWaE3Rqk4IobRYBtMLof&=Cr3Yl}>^sPl z5?f0rM|yZ2kZH1Z+Gaj7FDjhPhdK!$K=iK|r!D z{2V7Ubc=<}kBU84c7SkNtCU=O*C$TRMHfMy#9N_0peTrFK%A= zRS_2556Cz3!|8nESw4now78u#Xvyr8xXU!CPCj`3KSwm!=NjPsywV0iW;nmE=tE1h zF__ouQ87JKdr_w?)z35_rafJIDW`AaE;OXayb18HDngg}ru!78ovy0P&@vybLt^=~ zfOOLwPgC`Q=x>Zqo3|bD940vpY)=-r zxxKK9P7<7y3DJD9&bcDseUoZ8sbRI1{D`x4hZdHGMWS>V)m-9`ge$rjK}< z1^bi>$%xku4OzAaE!$1M>PY7=hzlx|uNW9`gy4O_1+T^nnkV*+S(tt;@SPG>+S8AS zpPVu@t~aVPgYS1^WuJm7p?(3Fk!I4x^P3VFOZ9)`$65^^?=jh~4tEy1JFzfQNM|cl z#pAzln_SJ;cGzi|UL{7pDfJ`x(ntD@jqS)jB_LuTawxOb8w^aY_O!E)TOJOi8c^;# zGmPc1@cwZm>^fFW_DMzzOclx;>neJfd_2&f=0the-;Xd|Mx?J)(_z@J2S`Ti#)rj%6Nn zA~5yz4dKexGsrvQoFny%!+b{L@canK%J$n4okj%gKmm?LTlP(kZ)pD>HtM5PBbIwz zsBhziMt8ujOk{*TDy$A>m>bLSewSPALMt^4BZ1y-!}UHx1>$ia-6EC`?`K%H)4Ggd z?2_lmeNsNDQc-R9SyAjJ>vGtb`i<)uaJ2UwfhPM>yWsWO#+X!)^h!-H=AsCjw2I`SlWFP!^uf_zesalrbL}hN^&%5QCq1j- zV0AgkJ=pEN_;&6+dqMYt4%I|sUJ&yActI~)KU>!E0$jJ8k53sO-#dd1{x(xbJzxgH zZlN+yYN49COZO_;A*ur_VevW(4eB`|n!MNys+$oU^6@(^E5U|4F@Q|QVGYLUO#Lj|76HgvN*G`1>fCNX@)>kE7`7V8t*Y z*uMZ5?+64kfPT1ddkJJdx(35<*AAngRAZXbrxLBE{6VE%3M z42)>hqa@jX_CHg*MeW_fV*k7g)pHyUM2fE$`$mx1{}Nd7H;+aAF0eNrb&lJDTwR%o zGg5@;T@?&2nwe!}bko*PZ_H>SIE@@3`GJ?(ugjN;WPV%xgAvu#PWjw{ZkC1nE!fEZ zNi9UAsr2UGG10Tq!fb}q9BN)kMa1wY+iwNwZ;5XhLx$?XghgK{Q{@dH%~ z168vFRV?x=JV;U=ME-|c*E35z-?+V~gor@oe-E<0%&3(N;vZqvFS5RPWb=~S6aHA8 zw>#`0Z}<6ju&sN3sNzm|V5S?&+o?{}^;ThtN2&n{$dUfg2U^-g%@BjOzJrd8`@2no z!bZN0@zt?xptuHfENzfS5Uw)9rzp~cHrq;l=n9TmkWvXzBA?$TlGg`EeesyzV0P%O z(KPmG_VKfIq(%AsK%S*3(tDynsB@&zv97CdRAPYW5Yz`L`V*FljsbmWqvYbXBYtPb z+7mSs1&$~&9SE9q&q-t`T4abpWXMKj=t5*DOJqn#Waub7$}23&D?I8oEQ$#g=MfDD z`3H_38EcoUv3KCYh5-ObF?i~exUiwna?A(q=d~?V{Alz0F?#?3#G^jt1xx24(7U$f zR$KLm&T*GLbhyRl9{b35e#{rj*Kt&v^C;}y^NJF4r#s-_{Asi$M_Tqn=_}6Cm4ViQ zeTj&00;clr4a&jdJ2j1@MDi?rMYL}$LZ*XEqi?OmPJ!yis#hE%70hgDbf0R*gc(F> z!;-Aqq(B8+5zfjD6f5S;;|rA8fE@;b{4O?7ZhwmZA`>dlQ&E#Zil48EZ zIIUu=H)x|HE(!5DUzh^tIm>t-IU&AVLhOSQ=NYEy72SQ-EO6P|^=SHj;CL4T@A>$^URRxa#HGMdIk9mu?21FJq#-{i`5}a` z;S>(4lMwAYf;$U6i#>*KI0=iuYL)bTH#;w4FQSkk?qLeHB$*L+{=lZQ0rrkOqif74 z3xOz;o=oGV>GYM4Bn3Z@+Ae8)uejt$@$+8`dctHp+()t7)~zKw+ZGn-22S0EP(v!f z5NTw>-&fF}m1G1q_9ytsxFK3P6Bg!ykS7^|Jqr`X$m_AE^B+zh24GM%25pxY%iuCnR@cMv zqf1Y2H5tb>V$Wfh|4Q#NEYCByk(wUX`B^&=%yS#iUJn`n=uFZWBqIU}lzVwWS%vvA z3xi93X-s+z9XLRTfXc)s!7DeBjV>{4(Ub!f9=WrCP5wcV(|V`#JXx5$f2^%3iK1jk z!OoqkiVu{Y`oa#uOr^7}4ZBEsbGCe`s=S}ghjU$_VHo}47-T&cc(A(L)K{-!!VcTm z)F3=~cJIC1E)Yt~`XhD2ZFFc=7wpV7CrZdOWv}_R@!y>cxOwl`euQnbnuTU;)u)sm zvF)EL=Va?koDdH$rhVVGt=%^t$?+d2gO$3Tul()4Y2PpeF`azSby0k7AI){+;y+1> z)jz4*=PJk|0caO733j917k%W{7t}0bq{?&i=TS>cRWS5YYy{9xO4fhUMZa<*ld1Y! zZg`Q%#_l0u5H&9E>7qdBrL*+QzdcVgS3J8ls<<|)IR8)2-cg@e&*fokK{|mUoIR^MNTpMHAi)!dF;IDnCG=zwH; z(j^(Zrp3P|2x|Vy?{;xUy=Q33uRUZOSe>T4$WJ`9uawv!62|KdB4rh7{lcF}Ro;oA zB*Ie$Sjht$1cN#IBk!1hYx;8)|MbIFJ|@ck(QhjlJj7Dj^QQs4JVd7LbM~Yd^$q!p z;wrlY<_4)pHD=%4v|rloZKu?oo$Jdd8S$<|ea?ee=xrw9JuSq(?Il0_$iHx#6)4tq zJ2U+FE&M{9dDRW&@QJ^O|1Id{N=aS#z^O|PQ2`$%OR>*(@7<1d>(dEk5Mf%v%iD2+ z<{)S2^^@H?xIbt2oiqITV}Ad&+A!ucxiaV@l*<0(v&aE%+yI_aw)8DTcNp_YHK}M3 za@*u|%C3~BTHei9Zqp;W?NCTThDk zdX9cf*Xk+ZIy9@uQ$dCq&eYSD$((mi6pwAI}RI zpb|MC_P$r-D6asMh`8~ISmEnEYew9_A6)Qf?slg z#dHYv&(C33;kgbtoycOlj~_p~d5YT`k8^fsmB{mr$Y2i{UXxHYR5x)~ijy|LDqBYJ zT!vJn#=9uDjIkMeHC#8R_<}}X8oN)ZPjD5`g{Hfd6Waem2*@)_Jl%jHI$`>;{m{Sc{HgM8vUlvFX92%H zs>o<}AAe#QI2>iwU$|a@*pv=^A`tc0$!~?_oC97!WW&^g6vH)x0mL2wOs#X<{PEh$29u5-+WCu?yr_WW<`d)38970+cZBdyFJ@Uk zBFd8^{j!g5E}Nk>zdpmnS|V%2KC!dsnR=1l*=8w#J6=5fOn~9Dx>dOG<$K&t#!FLG z-)_Z_p@cdH$cyo`l$4Cr(fy@B#Sad>1N3KFwWC+s&_Va)Xwqs{tz7F*5tEk;*3~aL^Ux0E!ZLS;w+sR?vvBO_7tY+% z)ZS`8c+vav8O=|W%f)J8&Udrp(oZ<2`)$dt4`Q@^xs~EisMmYHom^jtn3?sLiP)<` zJ%6Hi%%@2^(W_yDHUUDmH`shM_~3SNsc<580}7_A^bJtW(waih#`@S-QWykR{k;g2%)2|a@GCo6plfxPSOaF@5nsL@#>N~X)hkYx*} z{)I$gk1-K zxZDjAh`Vc%Nsi5`Rom=GoEwqy)UgBDQti2DC9qbl4SORkq#JvIGuGC(it*Ej&ao^F z2NTC)T1Csvh48kh=ZS`^VOa`U2scp5)=HTo52`XFrs$rn)T~y!^l*_m<>_AOrQwq?-<&Mp+$i51je6e>F72cJzW|;hAH;6{Vc5q%90$?;5BWzKOK$t&LEFQe zY*^OhGCjc{?*xOq6AbcBFvvT?An!0q-b&a)q!H#v5W0!f!cmuLamDt6`M3cQJJFsb zi9Md$=0M-nH1S2Ywd!ycF9`1 zV68oD{N`*fj&rsm{E3PDj4qW;=)n@P884+;UBkUl{o^6QKLL{b6N$5rLC`-5iuxx* z3I7zR=zknK_@_ZP|8(f%p8-StGhw8E4smrJadiQy)~8^be-Z5QFNOpDC2-Kc6yEYb z4QKt&z$gCIaM`~GuK3pym)F6c{@tWv_h6!bJtq5~MW=rQ@%cI8^Cm3e--6}*Td|6N zJKpKvf%W~nu!(;+w)F2Y90Zc3amziKIvEqC*2YAsWn>u`8^~qgAl9}g+f#^pMX|a) z)$Sn1mcS~uVs@uF_BFB6WmhFDxdT5x{1T3N-8v5|X#%`Yxhu;WQM+MYHo7Sn&y3N|vosqk z6dD+-0k_l0h8z1i)3cSP{D6%&Oz|0l;<{DIR+@!_beXAsw{fT?7dLl^EBR2p0Owg^ z?0Qtj*4Uk>&dU|lh23!_FU{#i{4WvJUnZ)*LR5d1EQr@2-G2~D`Co_1{=;N7yaUhp z--Q$YGw`1OJY4X9Kz8JZWDQ&(+wme6@n0gEeuCBfpOGc-IjP+*u(|(BZ0-LA+xmaS z?*6Z^r~hm0>;DFa`Y+>f{}mkPzlxLn-{W-uHJs=F5f}Tf;|jWet^e0ZtZ?ipsm+z~ zc7xjM4Qj77sJ%o|+m8#anbtBy;B=g8ciT0hb}4+q?%{#sF&*aGy+rb;G8W(2eYt3_ z+x_y`qb|>_6c@C=CnxRqLs=q8x$r`4^P5t=pPX`3ms1u_TYm=Gqb>eZg*#I!+)#wF zyb)qx_}f`A6>Cd@t!!4#b1k}M$N&FxEhXVFyhmiC5>*2J9%`agqFZeK8<0TLuA~4$ zFuzyfUpcIX&LfQf-*m>fujCj$X2<=*hgrlru; zXm^awz?62$;3n#N+x!6sg1?TDcm&1pmG{x!*RHOc{PW@j4yn&kF4`e~{ zz@3m8s0Eb*wV_6!4%7?Og$99o&?HbFMg-T!T%Lu64Ao-r*1T`?Adu82P73qhY6 z3qhA6=cnPUh}^{Y(NEb6k!rD;5)E7tT);eBC`?=^Vyt=014V4BekizbZ_It1wzsh0 zBFjHqfmDzRB$&a8W#`r)&uNImTk=o}JV+E>!21_G_6+zx=@$jPhH*>Jb~^|IM5SX2YN$?Kp*H4=nK6A{b6um01OQbgkgcf z@MvHNsez#|Gw>ih6&MDK0uPb+4~G{5BjKgMC^$;x$-t=SQ6~7%*m`@{*m^r@Y`wi< ztw>#{KnIvC4w>ZGf3+927n4TLEa?#bl<26oF+1wMTXfWi0D@k;qfSVp=1ytUw3kNB zN@>(gksbM>OloP_&pLDrcP7->tF&(^j_bCH_OT-S#%1Z94Juh%3f*trhqLx79b589 zw0wx-?I@jOJ4)xhN*DU7V+J?c#8>yruW;&Kr7L|^yMQ#BQ*at}Gto#@y7L8|eM*l! zVT@Fz=YPg9dWCZ!dA~1u|k@B<-~y zmIV&bfb}x054=JH)~m2H@EYt69D@CU*NKB~z>&aVGG^W;jdlV~1>S*+fp^K6IR#e& zXW-|+S@<>Z9&zz~vh@}D-utMNttP!|`wF94Gqrhj_ zJn#j!q5C@ozK&=~^qP#G-Ub&t8C>jOaIs~Wi#Zrr(q2k3@Pb@#b{MR?jwn2Tol2+D!Sk*BzcR&Q&UbvAJr`t$-RWnB}RR;37YWAEOO^?&;*?2xP&#K;_o383* z=H470Y%JmBiQf~Kzrj|3)w`U{iM6{D*prlMVNq_UPJIY>@>0@Utlb5EAbMXTdjAOS zz;&|OeuhBc7bp?<4JrhFhnj&uVO7uy&jxLEdg~+TCkn#g42y+#9j(a8Ja#!%c$f zJiPp~;qdY~NS|TE;VdP?#5)A(+s6$4p$xgX-mL@Slr4Pgg6Bw#X_C`0)g zLT04)0Ix~@&dYC~!W=F`dGLQD68t7OAlxc9iL>knaP_n4~fAFkP@s2-e4st z608Cpg4Li`usZY&X2G!F9WWwT3nm6@!{fm^Feg|KmIUj=vS0(^RYTYmYzy0i?chN0 zE_gZEh`805xYYzs1)ISK!RGL3umyY_Y(*Sv4Zj52z+b_(s07;?NylC`t?D`qW;l#h zU560}!(jx%aESF?9tgu~1j1l45JoT9DmEv~fiN8T2g0yt8bM?R%5*|Z@Tfy<9u1S* zp>0LFo*aEFlI!flt2vS`Dk5#OJ*U$ZX0C@uT3lwfGJG}E$}xQ$!Ic;3pSx9Mq*Oqv zGD>_>hKtLiqf^O@-Aaw6a33EqS1*N@e83FHtZtA|mRDwp-*`B7P@^0_JH`~OG8~IC zRtTEN6Us<5@wvAO`MfuWBDLg*B%3SDy)w4IVeVy!8(;W=O_6SR?0X(KG(q@oBuL+l zL_LvbBl)Pko*4Gs7^Xx|e~ zhiwV_y?4BM_66mUo2vE-a(2!&yI*Sd;u(%>m0GuxEowm(MSNF2A{S>z4IU;wG|T z+?bV6<}g+g5AmqDQbx2DjzIxkvJz2GT#rt3Q{#D4iP^BtY}_OY>2Q-#HG!DSb4|_S zS-hIYl;XrlAgKgB;Z>L>Vo8NRC=)pp{8*;O3O|?fSh!iYfH^7b20#du$$3(NDO0iw znhH!p_VKo#<^r841X_r#5a=W!&`DCDll5dCQ_Al*5gsU z7>{BmRm(dbCEjX0(&D33j;7QP5>Aj9oBgy2TB1vg=d;1;YJ+=_LA+i8NZ1N#LJ;Df=J za9r?ZniRZ-vw{b4aqtkX2p* z?%)}w2hXx%!S`5|;5pVL_$j+L_&Mtq{DSoke#r&}zhd)(m)Q%!E9_A4Dmxndo}CH) zz|N(G*oCxob~*T?;tu|#=)vnsvEa{2so<|lo8aF{yWsCi``{l+pWvU$kl2t; zn$t2R&1IRD=CeGR=C;gB(=02~JeCb4wDydeRkZUWxC zz`$!7z_&@@bz=fw9tG|&am5iOiz63XN=fYe*h?wMlt*ByL6uzCTNCzs2>ZQIDy`g1xSRy{>^htAN;7n6R7Z;)wFaVZS}>?FoAa!rqCncO>kc3XJ`Z0%Cv0 zgx$m#N0c&-xZA_tm9Tds>^%s3cf#JIz}W9BAoi6e?C~b-@lovYw}-tCVed=W`xEwl zguQ=(vDYdf_EjeA2`20bQS1q~hkY<%A41q4AnZd4`vV2WUb}$USDUaWny@EEu_xXh z_Thwm1YsXV*hdofQ3b|cr-0bkn6M|AuqQ>aC*2wr z-DSe=ieh)&9`<#FeLZ2{K-ixp>>CPUj|zLl_V zD=_xP1;oC|gxzhz?v7%2-yZf|g#CHKzK5{yChU6(jJ-(#v2QkE*G$;8D0c1ku)jdq zUnJ}Y2>X7*exShEn-&oJ787=l3A-nX-E(``UnT6X5%xob{UBjKRAB7Q3W$BH3A@*X z-5bU3y*=zl2>Y9a{TN|CO4yGT82jA?#JcP9`+N2{Ul+3m$1J>*xxNM z_T~k|zTJdfH(}SK*!A1PewMJmN7&C1_V)?>m^Mi-i5-0%LDkKvF|cr51OzC zqu7JDhy5~P|CX?SN7%0r_U{Uey>$VxKX1aGX2PBp#h!M1*sl@x9|`+)!u}Iszg}SM z_Y@HOZWH#93417tJ#>56ekd3h0341(Yk0b2y1;*aCfY|q%uopIA zFC4{Q`1Y_T6ZRCs?jY=`gxyhK?ClDOeV++?5fk z6%hLiChSE`*o#K77ri~~0m2?6>>O)c?%~Hi3UfbZ<;A67wQM2K z`@ezyNh5R%-$o`GxWaon`uwa(1D(zW6WphX|&o+K0~_k zIwBAA`qCS}iig<%N*+PEpGeRhQbIi-E!4~O2rXFi z0KKq*^uh+w3mbBa7uMVOwI1@<3UjZn$P1G{eB@>iH;mrh2ylc(k`@?k`niY}aF`yh zPvqyUv%){8^bh}rBcA`J(mDKijs#I_A!~`ER!7#7M6IH%C5u{7SxXT$P1aIH%`R&W zUSq$@np4!il{J^BU63_Z)K1EpThv~YHBHo>mo<;5t(P^gs4bQ?pQufjHC@!k$(mo( zhR9k#)Vj%9P}J^`wKP#{AZsB}t14^hqE=GYiinynYehvZQPzs_8vZS7#YOFVSt}uG zpU7HCQ9CVbr9|xwSt~7Sdu1&{)Hcdmrl>8GwKAeMTh_{o+5}lECu+lFt-Pr9lC=t= z)=t(cidth?t0ZbQWv#NPWy)F=QA?Axs-l)EYt=*zWv#m4_D`}_L)5;IwVI;#zN}@5 z+EH1%L)7-m+MS}dRn}^W+DcigEo$>*t&XTombJQ~Hd5B=iCRBdt1oIDWvzjz-7RYk zMXi>s-6d+}Wv!8@6_&NeqNd84A<0px#b4l2lG;Z|Q9Md2X*{T*2@niTgu_6lvs!J!>EEVL8HhIZkE(DOJgv>RuK_TZvWHZBY8#dV>5xH0qs?h3t#dqey2 z)zATaBlHrU2)&G_L$BaRp;z&f&}(=(bP&G}9m3y2ujAjLH<&GSm?egeFn8!priYHQ zBB5ifWauqcA#|Ko4ZY23hfc5tp_8n6=pA-X=v~$+bc%Hgoo4+*XV{R?SvD&49vc^W zpG^s!W79+D+5FH4Y;ovAwkq@yTOYc>wuLUT=R+T}1EEXowa_Q*Sm;xBGV~cc7y6uC z2z|l641LMI4SmI~hrVXNhrUr*=(1uDeXBS^R}?Muoe~ONRf>kbSIUHbP%4J5DOsT( zl{%rHlqR9;N{i6X%6*|g!qncTXWGu5Py$sFVH`d4*FpZpftSvEerq_hT z^ek|v-w|`<>}Rxkhio<3WMt+xmeT7`V|7gpB^wR(j|IUPEf}E`rx6CcFVL zExDty!8L>U^t)qPas!QKM#w?5RAl5lwiMDLL-rEW@cmZq-w5#|y6LUKntl%?rMCfR z`n}*wZwqPZ?Vv<@2h+G(${5Cr8of8@R>u7CTN(4|UFc2SPj9L#q^Ea_Y0M9f)tQC! z?aW@(VDFfnImBq_0r|d4nz<%l-P@+ubt3mq`1xtk^#1e{1Hq9#2)yZoA(%b{il;wd zdQ~M26C`T!42{|6Cu2XuaO(3BG5h=hqnR=C8Olb0zheUaZjMJfwG`sSH+edK40Zfi z>iBWg@sEH%eLVHT1jtOEXnLBmM#q;jp5{RV#&ijzmK>+U{cYyJ$=n=>z&f2TtSR(@ zr-CQ_af0&%WTa1r^64|+j`W!(SP>_q@{nS{8g}csP5P5D=Qa=BY8G-PwxBN{q)!oj z7seFy;rScYXHC8JUQDAp{b}l)Wz;##sdH9P=d8SiQ9YsnPro+i(~pcf(>Rl(x8A>r zF`mAG2)hYV(l?Xh*m{eZ#;5|o+!+(*==@*jIn(PrANzG?)9c(zuX7*0&KKx)?!U$B zd^ncgDVeX{d4-tpYD~Q|=2nBq@z~sYomzZ@xOF%tx5gT+P9WI)yLvp<;|Cd-u^X%z zmTcBpWY>Lty<7y^qkPB%E`C6-?8Bw})Q$2{1i*#ZdhQs#vA3Xb`f(Zx-iB)FC!kLH zNfVfQ2AFZT5}%8)AMrFj;u(6xvoRm>k(fI3I#_o&M z^e4pb|5w~~z(-MS{WIB{WJ3b%vXq1*WXlX8jgru%3W8urkREyoO_~&?gY+s*L@5G> z5&=<=gn)pk55?~DDeCvsCt_DrY}gRK|D8LToeg)hM8Cq1Ip^-|?zyM^&$)MIcXI*R z{Yi-Io@Y*WufH;8`77h_vQ_XSOXglMUNl0IWnTV$%cL*hurE;=zoz)4i{wrEhLV%M zrG`nDP#wRQRL6MonySq=Rhwm~mT4)^P0K3wacOxj-Mz?CY99Uq^ znQz$cHm(4up>{4}9=Dys-BDDTZ`55{oKu5zt+WqTI1kNGuL;2Y)WrECYc7_$*Wa+zFT^5enC#M0XO^bGt1wrkXP@ zF=xy(X7m|5W+8vPe>fr7V$rR_HyUDHmptxd%QQZ7nxz%E^6O56=IK)ZER%})TrQ>| z#59tMSuPdxgXYBITUxQkTF#T4B!#qCJ79@11> zBiZ~&oWg$(DA*)@;Ge$mPe0Ty>os8i=4Y)m^k2$DKM3LmOGCd-wwCD?Y%Sv+4&z5i z*D~v+`g|o9GaCAgk?OO-6tmXQr%Pbj`P%T^9g>a2xbij^Wjm&OJZzZ&A5VmjC!svu zNm=g6)XF`DI=iP@l8#hYDd?Ob#2V&+*#oR9y^Q~OW0*F~C z6|<=VYbp0qh*>5T^FT%F>ney@Efw=%Mbc;;#H^Q!*<6uG-48LFq++&IBvM--W~)@p z){3}z2gK}@irHq0S#B&<8ka7~zS~SGrEA4YmPQjEKh zobE>`*1ZqidOxY|1C-)^lrr6iEX$e3=5nTixtw`OYTS=<|5}1!?1aIMlvKetFaV3X0aN zC|a*c8)t#6nyW#p`M)6SO=&gXC5^@}axw298t+P@v0G;4Sc{dv%B_4KihTep|0A_> zk96C1DqxTDTRGb{_XQ~YDOT*CA=SSKv2EKMJfVKKB-9`Bg!%>v^(_+WJ0#R4B-9Tf z5~@(@_bYP0UxpdKNd5kZYysu(j{a@ub)NqNp1*?C&7aZ*)IL-0aKjmON}X}Uzf~F+ zY)9Y;qi~OnVmyqTo^VR=L|WWY$8?8(L%6>JY3PZjC{K*k9S3Bvkgf29AO}TzoKjdk z8ax^Pv?Rk-x%Hr)TChGA*2ht8Pke}EI9Q>znI{QqyQOMB7W{s|HGi{U_9>sc6DAAKjbv+p&?guBT_LDaxu3-Otw@^ zQAH5e9b#^miYX3Ja2f|IIG)~cRv)VA=_@TbM}rrfCvK$R3`EEWA>@M*@*yZV!$K6C zk`VW;SY)XCR-TbCXcVG9nvy)$J&u3tcB~>`7!RLMkox?%OxNKSUG3#uG!(i{hOSef z>s07EO{(jYvPs5UCW#7lZN@VLlgz{H$%%Az#!9 z*X@Vv4#0Jf!gUAXy2s$U$I+UOP@1QRT6&IJT-VBUT{F{lPlvi?{dGv_KZ(3L0rO8H z(@#k={h8nmtfs~6TJmc4EKK|dOgsY>GvAgjXPz?!^)>`0$+A;+7hS~Z-_Y?r#OZyc$Oo1wlFTWdm+9I) zsIDJD*9%fz&xUA>O@dl$qY)m|tPzHYp(HLKp zIpp@B4!I2Fe~~)mrN0j|>`^>d5WYVFv#U~=y)4sKcCSj;)*D9AUYk_gS4=^N3^!+& z;x%?NwJ*hM+-7Q0I<9T0{_&QwmSDkmVp%QG1>CYAcODRqvyQoJ68 z$$k`uxZ|ncU-x30;T;X|jRE+^0(^H+rgxlVb?6|~4Cy-+D~ppb#hud3ez!uc+&c|| z?ve`nuMn3(Sv|SOXF}pEczm|hS&Ce_`AD*Rq)GN(@WooH1&?Xc4(nY4JT3(u zmjREZ%1QIhuoX}kST zML@k3{@y0__lH3*Q?U$7Ev#?xeFJ%CdKkVhfbVy~_q$OG_n;Q;rMljIAudy$mtI-^ zGytd#%ek`bJqY6tQ5ElDOn4;3mF15rusZjaK=3iC*Dgp`!@n9~wO(w$?lQ8_`6NW2 zLVTZs&ZniT;gA1fpZy0i=L{r2Cr!;yOsRv-CG7Qev61AnphE8#A^l}kz*kTKUzIFh z9hT33ej55ljn}$s@!zQVOq$!7@+h5yE8mtz>2s-=EV&r$PrUC*#e5-)L|sdfXe^J! zhw#;TMB*c9B)*g?*F>(|r%>)Qsd8Uc;D)sKD~S18D(34Oh}Cz}SY4FC*QlGN`P8&L z`2Gl`E(5;501Cf_fbTcKyF)Weg=#ME4p&j3u2GDS$mvt4w$B!#JA51Je&KcC)&uX0 zq-dX=s{5j(f&Z=o$?dC5QNAkDh+V2ca{H=5j7uu!`wFC?uNK6}Kf?BdGz%IBWI>B^ zvcQ)B(MeRr=cYuTH$)cvSb?1P)rH_>sm?zIzna+6vL(35(gs^ulB~7#YN9Wl!hIP? zvieA}1{CXS2!k4t>dT}QUlwKhnuWNU_;aYm!&{*i55Aj_f31*zttrXZM!I;oEc?)> zKeKQ6>bs6B-_4k>1J(3(#C*3SN_4V-Ms^ZzN+M#I9e6Wc*DEptka490#KzX>? z5)rwX`X^fEQ=~l%nvyyn5z6K4bpY{91)~>2-19Mkgitf zp0;lhG+iv+f?YEOEi+ux%=iGbC*X^mqXRw;J*K=10s80bd@B&6mB`#xHTOHo>|FWY*agQ_xAnx_*Xr83A8ttsn4()&>Dz zXl)qaheiQ@$du@$Uyt!^gZU4^{OvG*2h86I^B+b?{=0uB+jt zDs+^qAdq$%5q=s-dj^q zH_9(D_K3fBDZ_UTmc0$j-jS`)?2#3N`Fjxdz7z;{X%YmMz0Tz&!FeRX$4G)tkOZHF zNP;MHvdzY1?Mvsl)#CXsMk?f&O@*$O7u_bkhwJ+q;kk(Le1q_Oi|||`&G$Xv^#e8X z{YXuHKT%uXWs92a&5*S+)vRQyIoVJ%#V8IV13+%Q>X|L?1b;xCEAY;rK>JmyqY@>n zie>r~b9(b~oIN_!hYh-g>a%J%MXM21U5%t9)h_F^_LvHMPDG8ND7CVz-PkJ!$5M_Z zSh-RxsnsZ4by193ot$b7s;$-vfu$<4tPlFlUNs(GNRVcIRq66v{^eTL4eLEp>m5?? zWC!GI_j16aY7kwIs;DWHsHTMgk2AQdZ@0L*huqZ-;p#?kbtYV$1y?sI*VUEm)xzHu zv)Q_EO~Esqbl=HJc{Ug8nVZL4tXFPcxGgs?LVq0TV!dm;3U#{mB929QTi(! zm0YY3STsKs!%1avzKZ@M7iI)=}C#s9}1JhhA53H_W$^4mmoTPA)%1N3Kl+MQ)oYd!}0VfSP zX~an;Cs~{{cCr3oO$2KySTp`ib52@t(vp*#IBCU6YcWF`K5olNJ5Jhjax*6#IO)jA zEu3`Xq_cR>t$f@?jJxvjZJcCt(oLMr;p1FRy1Uo_u-o~#2PZu_>BUKJ7aIuHhmZSm z(vOooPWp2)K*$=%$AdT-%*hZ=hKjHZ7;|WI87^IGM@GEKX)~GKZ77oXq27K8Ta<(dh!6 zF4XBFoi5ht5}hv9=`x)z*XatKuGHx&ovzmDy*gc^)3rKXr_=R1-C)Z-!Ui1%ViZda z>0wJ(djeK26IPQ9SoJhuWz#`5*Z`GHht%KzNI9;Uh~lJ@!@=R?u$y?Y2{aAnXo}=$ zvIWDEEf7hLb{t9$H^-8T4dG~l(*p6tNs>O9fRl?2H322YDj#c{)U}yla>nvHkRn*B zqZ&sPe*0NdE$}LAN6w~JEX20Kx@2uxxrWH+Un4qsYBN|zgiE74&FWncxM^looRqq z=j-*z0KHtLBW!kA`1Z4HRPzAe8YXJ$1*eWZ zTW-k4$QK^ng^6XCnob8Z!6wGi#ikB4@%$jc$~HtHUVu!D^Bue`j@0QWr^44RqXiqI zFK)!D(;3FQ;W)vhD^DS9vS4K~=CpIj+%4F2!Da|HQ?OZr%@!>5TGZK2?^}xuzS!VP z3?6h%s=K9}RjKj(I)htQrgO@oYk)0F9f&iL_L|1_vMt&4ZDC|;no`2%9xk&T9zhXm zzGe3qY3v>`ir5ogiSMT&td;tRG`*IxqDEo1a(0?kQGm{Fh~#gV0q@zE3?t z*?A?@t(blgzd55LyI6@%b*QJ=sv@$O(ov{+Os+g@Vq{=xbsBvCK30rd-&#lc%En=%n*t%kh!Nc`XYD2MYJwzyW z%%Q~C+9X1Q0&?&_7@ZiFyV#}_Ui%Jb@xFym6;Msi6Hl-Oc;|gR^=Cgn4AUplT)Kzd zYdCB$)q=xD!2!*vD*RfV+EHBCN~#mKic-`LRA22x&D25EP903Qs6(i$`W*F9FVisf zR~n`MMibT1v{D^Io7J(jTfKw!spIG|bv&I>C(s2|+}vnG5ww7!!ei~Rh}$8muK#^d z8$FM0y%@pyW{5sELWGe$jy=z>qS$LoBY%`_EF1YLi2T&@BM)Ilk^KkQxrrzK3iAcG7 zsgk;eYN%_;t*#?YT~F!i25O|s!@)6J*s zb=XHn3K?#uCk;3EksJO$mNNPsQgs)FtGmge?m^n@r3AH*JnAE`dmpOQe$x)0X-5l- z9eqqY`WSYE*^}&U$mi+O%J}*J{rq?T^STwWEzgrv|04hx!kj|r#~*Qg9A#Vk7wcyY zBl(NA#qXqHsd+_83>yk-F=C6^P`qxjBY!vDtS6AaG-v9iIkP3>bulT96|sllmY4#H ziepMK+s@DIXvqq%UC_(+%Y2SQsp)*mc%E%X+0N6ZZ#t0^zIlvltB+HLdWbUB!_-=R z0x&v4ebpiws20;`^(akJOK6&UjFzg$X&ugOQct0roJKi$n)ayA=>BUAFH9nj-DCH{ zBZH~AS=aZQ{@ZW&>3@PCf_B?g_98RBBgcA0!c;mMC9mZXVbz8{gR}FDwOHY@*E3KZTX!Ql8+l!Q>zJ#Xu z3UyOor+(@iG(`Oup!FurRo|jz>f5wYeTNRH@0w{ijOrT6Hwq4|3rBIO2TeB~G~DR4 zCr4B^;51WrBi0*6=G}M_`ZJHGN|}2c|3P>EkkZui)KL8hMp(ak)z(nRX|D$~2sO4q zRlVOs*uVKN92W7rG?83%HzaX<0cq*FcF~-5Dp)Y7@Rncan?JBJ^CNF!s4U#ry!2a986}Ct>=6vlL&} zONe@ZIO$QGX9cAXDB!lTdWov4-&3;sqs2Fk%^O;&_B11_=fI=<+)D|L{D+Ah=JjhA z8@iKfVX%VZepi14g6)dWEu=#J z3gu)Rr~V3UrrsC~Q0Qs47yo&+c#6O4sh!lq)-=3Exc;Dk;^SD2@cR4#*ba&)y!Pzk za0vW}3*@Jd(>Gz%w~#;ci2k)Iu4w{1vy0`&=lZK`5A+pM|D;IuDmm3_q-jLynnF#r zFlwjSC>y+o7EU9z2%4%z(mc&htF$QEqE(`YFy5tAro&nlD$*QOqB-eBO?+_iR_t|} zQ*9%@3&^XF!f7hCx2Lm9h~hvRV9yA@S&#lpX7pb&Ai_uWbtozJ2KI(VBz_n5MTyn@ zMn5X#m~@h5##rAd+IU^nyP(r%?DDik$Oqh`wVsZb$32K#=``DlQq z(r~RVP1KU)|}qd zTF`l|C4Hyeq=acL6}#5j%!3xxJG_IQ2R@o?6v$0f-xysmil&Xnwe()EQpw&Zyf^aT zJT-CF zgZK8cI9|u`4vr6Te2(Kg9KYbWhWAI}sEPyI3nd9hG7j`%r5TR4I6C9V#nBhX5FDd% zOu{i8$9x>iaID4gfPO846wI$+ex)R}#I~_-Ay8FfdjOpOnTQ9$?ckfi349A`$*rGd* zJ_^(NDlu9=rMi}nr=PT#2E?2Y`-6=`kF$UPm$U^R83nzDcVA6q%Edy+7eT5|1VMS(JG^tmk-6|W|?aj@ikgdOlD8a=J9Al3jp|;1^!6RYkkFm#&vB!_GL&w-*|D{6x>r>Q* zqQVYg{g6%7)NWWT=3u>*OYv%Vc%+5ymzIpqQF*x3ShfE}Y)+a~biu;zB~ zCn2>p4GC}4E-rioEeg+X7Z>)k(j~%0Q4u@Ck|Q3*b8 zFK}UUb7gWbQFUcBy z!ssw6APa0Ivg8QdxJlYROV6?rS#qVPv@SE#G4`0b*Xu#=+GFZ5Gcz;awfEb7a)wN6RuwAS<4XC|ER(mR)j6`GUa56EZLM;fzGlOP`$zs!6 z$!ZRTBhh$RSsPi6u|y!G+U@A^Mk?zdt0j_1h6BL?rJc~hKsq=OPAQ#9R#PGw@`iSy zFOW5KMU_5sj<%Y@`V$6Dgf@4kQzgCUnnS zNNOk+)fq0pSEBKBLbd0SH6t2Pt@$WVAQn^21*#bzQO$+u59oZ0(CX8%#VT_u8t+%K z1QLzMy`?Wz`^eO{ybKFZ#8cSgU_f~frcI>-x^Fr9h5|{<7uGZS$ZCbBqv`sBS0K~5 zJ+9R2(N$fARU~wZ)dYgMshfpj{obFIU~;o*RuXuV363aaJ? zghOGky%AxQL^U_*GxRpM8LNokKy^p2I_gBGFQ(_%f)xh`umzlyuF6&f`bN^>bAfHh z*%#1xwj)nEGvs}92jYVPy|JCD`ntNiFl#*Hm47!T!c7bZ29@o>X3_&FMZdl|d(}y% zag&tn!<V9$D7AG%`ELGKO)RaaoJzNL_=SvaD5!Wfo}_77;mudOjs-&Q}q5(}s6w_^aq zLeYLNTU7n_hQh&UAf{@nLPzi$jijCyJW5tuG!%}fqmihWCZ>j|6T)>IO4N^V5G6x1 z-0zKvBfC0LBa|Sk!#hj-zWcn`5Qgbg=R?OZ>{up|4u^0a^$VZGu<-G4Fqjw|tY69$ zjv?k(D-d~eFF6iRuV8T+%v zLaE^5U>er08%EKBPETHDp(+PG$N% zt8P~L$?AUkG^n>9&!3vY*Mr|%qwi$J`*6LF5gZ~@bg;=NG zcq+9U?Lct*1v^n0+I+b})h(2=uI|W|^ObUKQl~y$E|m25vXyex%TOy7%3iEGSuD7ESfS*W z^^OZ;_-OLk#iI9Fn$lsrsy{}jybM*;PPYoBl3oqgRr6Nf=9_-se5$JJb!gJAAfuyl zwkvT9?(`a_m#XH<=}F2Jt);T7zngb3h3z_CRf@LjIwiek=d4{{WHqY;IY&2|)Nbl^ ztuJ#76?S!nbLATDj9y1}9IHYNr?ibuxz4mT`9^K;i*9XFk8Z*lOnL>X>eQv0t`PgG zF3xE}FU}puUh?B!XNHwdvl;qL%Ku6ted8w>uE3 zo^dX(s~SFTSI5s~#?RNT6YN+v=dh8{hcr|9hkNX5Sl}s=X(t>YUN4+I#&L6RS5qr!`fT ziTcXDJ!8FIRe0a!?4X5K+E5*fFM@x`f#yIR*E>F_CEO6UZx=|g&yDYo@oq8X7_?^t zvIlM<^+EcV9e51faJ>_QdcyZ$`}TmA{dv})ru{jtcV6Hr{-*%w9w`9)JuJKr?$cz5 zB(QH7m^Nr=0h&6fV|h;jYT|v`fNT*17~g+}_Yr@>4DG=MBpo=0_l*GC2KOvLA;30* zPtt}tBy0kbIv`8{dR53mBo?+bIiQv>txKiNTZNMF$JY2jZ8pSD8*kY6C{ss7S9QuL`7z9$Ue8tT2A273I8hb)^F94}Ln+_PAxVo`+ zEj;b1&b&7d82Nx6Z|2km9=oVv{SwrfU5ZNUm*X#2QAZNh4%i{3- z=p10mfOd}#Y%=4bVS~+PbRFPz0q4;?3Bnoob;@M`JDn1B3Sxl3nKW@qU;xLNQgw=C zfJ~D`X_rq85j=!Un?jhUrXRp+M9`zH8c=V9(POk7;A}+JqrVvV*$A%3OfZ1d2)|8} zHK4Eva7bk|sU!s9&=%PKmt)H@tVo}$^1X{;oBDlA_6lQn7c6;n8g20 zAF4UpnmZ^MI~ZG;m5W1nAw<4&EJMMe z!oi1s%ZDo;ZxSE~CZiL_G!;p}7m{dbv%Kkvo~s36Vo`o&2>Q$_+y&S9-Fyu=T6kEN zUCv%lUw53I2>BiE!s43lfc%9824SVEx8M-zgJmWE1c@6|P2|6=d1I!Q(W|{;hJBjz zrnuDs=zrQ?>+5MgbH|_loCtrp)5)jP_SkE9i(js}>pUO7`{s4Ixwa&vbx)AgTU~8~ zp)l!~rT_bB;I2GQd(nVDqanI5wd%Me70p5;MM@^o^{V9wR&LchnU&25T?sAe6dVnG zAr#(tbOB1U{TB@nsi33GFc>mj{JatVF6yGi`zYb7vgi;Tl2j+NAdEE2A5XPXS92)Z zKO#)hr!Ugy=SbEy(^GQ)fPriZB2$ZLv&kWps<25@74(1j3?3&aB1VchlD6G#q7rkj zs}FyHkxWQw`!-Kcl3{t}1}c~m7wL$~IWrQMSuvS*R{riX6Nt|YJoB7yoDz^#_}!&H z3Gc-qXsXrXBD$-CJ-ImT4djYm0yOuh+*QctT24v5P5jl7!&*z%)p5B>>6CLcYT+<- zzLhVD(|K7`k*`u!66(n%vn2GDNES5!u5gE+Q!3C2%KgZ$NWAJU$H_D`ug?s`w5w5Y z_${xSQndY(%}Rwdr`S$8O9=%Qhxn>mUoILt_h)esdxNtoY;_mvZlwuNAlTEls;6z} z&8-D!B%UMnL!_q|?x_|!!gPYM+3Aw2PJ4U7uLd&0Xhi2L0tw9}^X)P&$=ocxem9Oq z#8J6}bD%X^MR;8|*eo5LxvNai$+05zgW4S-F_7?&+TD44T(WFWQ-(mFKMp=NM=Hcj z`gPn0p5Jl}LB<8LlX%X82qgbDyYvs%B7Irvjg0))2ru+~o7gx`CDzqf;7pDtwjF?B5Z)Ap23&Si<1 zC1*tK+4<_uPK*FiWh@A8yTzP@zZpr-Yv-}nT^qtmj;xz#W=Fb>L+JP68*qx!#r%g= z@V`z&@uLk4Qhr*_zgwVZVg*WhCFJ4UZ!BkvSuqu$A7)8e^#(zLi`onV7z}J~!@%2r zf}}65*2k{FmY*qPpz(=ipdOjDLGb0e!(>Q3FVX zi+3;O{CXOVq7!k<2J(94os}joBZY}BMnaWJI%=IzHOZ;W+8Ug;6~xwg?@^O4yd!c^ zrwlX`+NylQWp1~LjoFP+FD0%~Q+LstscI(-w6%IsBe)#AZX(Vl+<0m_+6~pdOq&Sv zEDI0CiF8J*+2$$3aERfBru8XxQsJ1{KQ6Zuy+)=cD+`xdW_&wr5E-eFbeira4iD)F zx3=N2cv&5NBSTBDL4V(C{|W#9ag6r z%o*QDSsqBTG)czeEuj&aG8FLYjJQ_H&e07Z;~uqi(rV^M&{d7{tBOb+?9(0NW^j6Q zRG>yD8|pCV@zfqxCyuUmCgvwd#84lcNfNS6;h`ii?@bf|!t2t1G!|Dj7rrMaO?ps! z0gj1~3JdKVG|SH1RWpbwToZe91LoH*P+ncW^6Q>S9v)vpdV81G?kM~(UZrv-t0op6 z`Fomef-p|JKXOKr*dpaPu%KVlQqwe$pxk9w`?3&dI_0?w9=_UEVyFar*_i3m zG2|qV=ABOYQzH2`_FmOTAQY)BQeqAQMlpH7fun5Ucp7OQi7b4ft=(gfUTuOE7o|c?V&A%sB-g;7; z(T2HQFDWQ$lParfYpd{9$yWe)V|jI?=B=^khGKtdMBv1vtBD4)tZbkFWKS6Ig*~tJ zRt6UZz0}w;4ll=K-6B;)c_|xmhyNU7F*Q|i*>krAzp>U57fl^s2$#vu?^qc3cMuJeUOSrwL?p@xtpI2~g~7SSQ+I~V@V6{#n7F5s5G+>4qJ={6_y zn#_%xumx0$+81$yeCy89_Y}&9I!VzSv4?qB<9UI)7wsB;qXK6TSO%1ps(cRHX9gm& zbluuH)k&hld^00;6VGS4ig7|7?10;V%R%k@_UBu(lvOkfTmi`-M<^cL6$?Z-5(z7) z`F(=alb9NUIq^f}>)>VAXdb!=Cv|!t@bA~ZoFN^RIC@osuz|$48%pIEHii>EO=kH- zF#N32K(VISV(RdTH zSkBBo6DZaBLR5DN&}Y(E9hMa0z$p+tT&>2f+x_%EOQc1cJnxrRjJ1ohTt@r6iJ_LY zS`=N~Z?h$)fV(+~oLF<)eQUy))N^e6Xr9U70;7j$rmI6zF(=95YA?-9tdjM1BF5#D zr|KKo5Hcyum8!~Q!)&>#add|2CHch_#Br>EiaSRA-kK>7x0hO`y&vX$ac6!Yu-l0B z^RZ=1ujv@K%{e1(!Jyl*9Osyt6H`{1Q0{Pu$Bd$`goh+;0S zc)Ni37Uu$%-n;lF2#B;f7GahiTOT_bF2D-c0mq&sL3U*OPEcoBDrjNpO@>j47PbEJ zmcZdo&8iherEvbteRd;SYy_#Q92rW47V>J^&>J>_%3BL=E8ak5b@5&8nsK-7;pZ|o?#R%>emOuP>S6WMNmX~C*Swy*6 z>Urc6dMPzpWTTq{O*YZh-_7sqccEQU+TL84v-aN(?kQe5G|pune$03>8cRv7_mS@P zhx;xJ{NWrq<#~(x#+0 znp&^0L&_RvGE3T!utVs|%$enCU|%P)qw9)LF!k9$d;-gwAU0`QXQ%lPpts1Y=e-1Q zrTNXAG&pWRJOSv<`x?Sf$0qZk=_6LFAvPP>Ns}1E;_UDc&8Cc@bm3nv?RZjZf+y@a zr_F5h(`&+bFMYKJRtId{N%i_yhrHd<^yZktYcBt>e*?llE?wH=^}$gZ=?~Ca!FP=` zhqmmH(fW=rL|258dDjbKYs8$%n*rgsS6 zLbP+@ET(nmMLeS~gq2sGl5^Bhd>Q6IZ`<)d#6N=|vaja8qbb-GpOP+*W+C zFW8~_aIi{j$bC#l_zPbF8v)X3N)qppcAks`9qcCSB`tQCDaY;>j06fT(3xx!X=N8| zVri6}49mv?Vu0C}v9nFzADYY&(eDyEpzx(Tg%zBY@fLq^EZo*dIQJW9qMfg7%-1h4ZAt6Vc#v^5jhO%?&hwGWz0Rsvj;~>C_|FGhRZ~ zC|9^5%)hdFeT7)ZN3`vO_GlPKSG!C|u-zpYHKj6?N$OWr7~$;Mv2dUt5}c(^uUqdN zvbCOB5Ik!QH}~ni=5OQow&GoGvt+kd5f|}|nA)6Axz_kR1od=R5D9KR9YWpGA4 ziN!$A_#@QxNc5M0C(+3fXAfq4d06`djx)@$0HWnxs&W|0(Ucc*^`D-7yImBG+k^U5 zetCQBK#~LMM+1j!56}1?5wzv_M5QpZC%Es`Vy7(Oih-)mk{6sy5fr);E;A3it6pve zfCBPFO0Pl@t&NX&w1fmZ6+Id5T2P_4@nes}QmaF3pd0R~onhfnM`tm)8-{3&5M5pw zEcT8EgdDym+n`p&$-v*(&Di}uN8#YE^Aj_R+U^>+0eS!<8QF7(JYN?re*fQYp{ zq0PP|j_vP+35dcgH}2<=m&7(T^Nl_N8^kIbe#0t%?nvfSn{#J$?((TO$mqP7o#kuC z{Ki-A0?Q~91f>O2bmoQ}njc(9qdUTfzfd%ym!sSy6YH;f;_R zO;`Sa*}Tl|Sk2C~#g__()2P<1VmV035WLaAt~cy`p!ysxSHK=_ zAiNGXv~PopLSRg%7fsrR7jn<11pQ$~|1#w>E$XIKToj7{bimHXT6s z5*wTl>~s12YI^u98DGtlJlt5H`u=pHgwy+RwcD7->lP0s+1hCJGhMZ z8F+6*J{eMgZsO5qkt3Qe#hE@mGcpMO&V5~n<}Y&7s{A*8fEJD%?S!1c6$B5|9;M9_ z>o4&L`8C)7)$6xu!d%Yyq}p@!gl^xK_V{lyHoFk$#ku*#_c+9H zi6Z&7SFG%{wNMICDeFq@wb`ieu+lfs;+utRQufHyvb)wem=x=oa9<#GaW$_-;WM1D zKb|M+r|HKV>!$7=NVr>@=Q3IhX?K$RC z3ymWUvk-+@o|=K+=hjT?!9xBpAa33B_UG|N*tUajb-+J635Pqt-xFE5--iQlKcWXr_!1gq%-oOS1Ma^L0Pbyr;e6zPar-Jjw+mcDLGYg>v%nc+4<1^PR4-HN^P!LmRBSyY81RP*!RhpQ*ZVw>31uDMA8rF|f4Aq~t2~d^W=b_2j<(oK$`@YzefLsRUet z|G1C=K2UgpBMHUx@cbJ%sf$SU@uQxGERiI%<#2;?a(%1iLViox9^cwOS=AJ|na)O^dU`oHqY<{iw62vvkQkk9)4&$3ZatFM|}4XD1W<0V+oEAh4;mj zKQ1z68uQ^JiRQkzt&hX$KlX*NJ4o8+5`uoZ74=M4im(JB3iMkJM)+d;PDJnH<_-OO zMZ}BmG+HzZNwP7r*0j&9i_kAk;DRrF-tgS8yvm5&3tMk;-JjFDrk6YxOT7`z+lz*l zkh+sD5WYXN;S|a7>Fw#u%e#U<_*Zh>sj!hj#gNn=Dz?Z2S9bVSH>!cmSqiPC3@BR% zs+XNEI>Hsrup$L>^n0_6;l+&MQ}e{G$1xL6#3978j{xxVv$8vcQ*z$&g{z`Y0hRc? zlO4hWS8_o=HyMxj^Do=C=CvGgj2O@*-orrbB;7eeVstUtqjnrd^QVrq`eU>A<7%fO z3MSvQuX{h>tSqT-(A;&~|OmKo2XPDLtYTdI?e5g);E`w(15VW#mk zO=sJ&T}LIS@v{DXO9-ziyWPp!(2S>ZU(y~NEJ&l9({2p#i98M6HX!w`2#UH1%>(HZ zYW{|0$xH$XdP=R&bOHUknd^>+SUah2-(C>@8|>pr`|^7`S1o`r5Vu>(y%z8G{}o8=piC*MR91o_U2Q5=`<8FN>CVB$rfx?Q z-b^F5&fob)b4W$@WKXD)8%bxMT@z;`rxCm)Q(u&3`b2vuO*YDnY(sh6{q6EQA}dlY zslSJR%}(cy*+EyXd4ALGD9{gf=vg0e4~5ltZwTl|GV~0&8W8}+4OA)!Cm8^I;W+&Y zh%I;{3cp4Hf`1VZ9|<3_ezL=U0#I%ZZ#x4B<8S$PNA|nxNMD?W#sG-Ic{ZUhdgS4n zxVP_wVZgDS7YH?$k zAEf8DbmB*>;`Yz2j{IvM{1bd`;+R!K4-kGYeb~k{gFSFVXv9+EQl10_Uq8Il%qRFsX52?Yw{i;+U{ASTP*=SS}OQ_5=f&feeFYFBX)TYs=QeT&i2ig@RG2p zHfD|TMHtbz+SN@93CH+ot7~V|#2hZ`hB395SVh!6b7vz$+%6leMa;O80dNG6`5mmd zo48{6(X0@!_(jI;2`JE`Jn1$u5niLi-d)s}*TlfxeiMZoVy(k#;_AOUkaF8QVuoI> z?kK(m9!gJpH@Pf|kHuSKJ%<~;XHUGr`)gg6g}aQFvFC?A$ zjk7X8GbOvCV&cJQzSc!peDE_rsqKkfn_cgXw*9HBiQ@<{)m+{(Cq{I2Bz@AGU`b2Z z6sBvD#$=6P5#RjM)RgsYlqc!KMlFaXRixS(yO||b zjNde+?OOl5#;3LIdSkcK(&r-g>c5)(8Xtjk1Z537mz^wujg`)~cki8#g`Xtj*?ot2 zr3NfK7rRLneO#*cI;Xj4J>SIKC7VwhN>?}!=IpS=18OA#aw9$kNIp7B?@)Aadd3WIUSVxFg|(JG2Qn{6u6yatgbSHo;|67qNvJap zt#rAa_Y%;yZZvODt^!Y!!5_LyOi9JwMjvIAZpbf+;dYzLI=S5*CgyW=GF)UJY{Ld? z0v zND3Y$r>QBE(D^p6IVgS~>RtOtSHXp^Ou)QpXz;7x{L^6fX7(1qVRt-^vebNTMd{@@ z2kQ^Z+nd3R-B|XWF!r5j_8wTvP6EqLSj#A?7#*%%o^eI)Ye$z1Zf$I@CQQ9^ zZG)089(MLu&IFjG`)7=YmIUd{HzKV5c9}u>8)PnRK>^|6;#(GE5L&94Koq3EJW?g~ zQ;V4>QWP1*5`kLj^Gu@r#|(P4GlngPz|;q}+&#d_Fnl4PCN>DoXWAc^%0q&5$^>bU6RwSPAD-Dj5~TJ_WGY|!%?NSa z8V(CN=e)8T)fjfr6i!Ey*h7}xAk7+%SH<sOXVt-GdGy7w3dClVr@OHv*WmG^!U; z`@O$uj82q?#F%^#JhTn9*9GJYMT9!~O@6Jp1&#pq_u4bZAopvdzwmBruT1E|MCrYH zH-+`--XH@%N!oy7W7f*xI=*e2i+`WZ1(ypS<6M13Aso)EB2#%oZCn&}A*h1FpDv1^ z5&N!3s0#6oN_h`TRdDjUL`~^Ms@12Xm7m+ZOIxj*H!7a`_LjzBGu`vfcKuo$x9shx zelGy9W%1@}j?M>6RAFNwO@lSXpTibboO^%n}E1;Q1JS2ujb z1o_u@J)x5*5*yj>6Oza}kC8chS>GV`NBimOgr%h78*GbLWu5uyKkMS@`EHH{*Wj^T z{PY{rEp`MAzLOeL;^}9IlB1m*KU^dC$UT=7^06?aFpL|AsD*}n0!=V)yax{{<3Ys+ z+|Ibvmb+xgn^X?i4O^7lyI<|uCPlj_hpH_D!+ifvhxdT{MOlXr%(*TX3>xVA?;3LeeWDJZ?O`pO>tsA&lBZ4R(H`gFUTi+gH|)v>>hW5%ZD zzQrE0R1$y2Lv6sgD%RmC3jZoL%?zmdcU1Qr-#Ia}LTHSzKTCh@O3LCR*H3rZL)`IBA=STR(ZgIv}wpit9Wok*sM|O^6ffcI_~zrcwFKpN~FM zx&H2*U+l4tz1Qa}JmB1>_2Bvbb7eP705bb6<#vsxhjqR#@7F^(N6;L=gl-TabPDZS z(i0i{hKvoSAZLg=;+wn(P4wg>0JT=Ehp~P;1c=k!c4=R3=)Ye6aWOHr26jnhMWi^3 zl2&k4DV|=+t?hyl$%$scv9Y!4P{$7)9Uh0i?p4)0 z2PD|(`Uj)cHTO?goSyqpH9hy_J4>Gl5yF(u2ccw=@r z3Pr{Xu-ssR$&yj$}$*-AH zFASZ|i;S1fKInpBgmEydLd^B+R}1}{=a&VxIT5M!MOhc3Wu`4l4FdSB<_7DHhviSS zvmZr)*Cd>FYRmnBzjg*@3#-sCh)XOEsVFI>M{G|Ty_J+b*P+hp7sjTU-)!Ho^I15eLYp)#h-y|1+=DR|xby{y@08S6-& zhp272gn9xBw;`5NJdwf=DycKiynK#p)wzEGiGCXEDP4L~jc{jsFla@-lnVsstcr~{ioRvF;}i1WNF$rpA3Ftom@VVmY$D@`?1n}*BY(83Idg*!GUn;T!l7>lGt=*+ zVLq?UcI8?0i(@ zUa94ra~QBQKMw<@lX%P=uu;;vqlZpv4n^*nZhv>^3fK{2#-~3=j!ln^ z9G(0)gujQrEq;meob;6wu+C z?zU*btkbn0CeZM?O-*6kZS{_Rt|mS7vuy|UPvVg1U<^yG;vJgWW`dS8spGUPqvmSV z4*|BZphX?#-bN-gLtTPl?p3PWoECLo$CY1w+CIbatNyo1Kcxwqh%tqAAy4)|kzwkq-6#in)FK z?c5v0w#%(W1(KW)H|&+aQ%IfYT{3swv)dSK(<*({80XmIZA`iG-Pmk1D1Bo}!|Saw z)7`_~nDuCye%N7b^6fI0+7rRMB}&Zkjh~$Dr8aB5fzCMc`QiTTN9F?(N)q(j8#m}J zVq1Hzy1aVRe|pSn%oA2b29`|*eXhFv?hBc|m-=HA;K4h*Cx?80I&tt>;1U`TL-?13 zS?Yy)YTm;oRFF-m6CqNB=WS*!9YWwrhU*=-5PtQva z%wa%yXRw4t;-$t-ybqrF6Kq7Ujc-Se14G`(ur8Mh`!*G2l&gnMNJ$@(WTx&^sfW#w zN*`i=m%XprLD#OAOCVra3vWQT22+=M3f(q&AJR4j2!8w1xo_-c47BbL0yVz>01A4k z0AanXfuY?D6vxM;ecfc3)O83YlT~8&$D_)n+nDD}bpWK_YZ2iS1foTMVhiA1bgeSG zX*qNqP-c?2BPNgI=4G`pbm$!cHAxWxu&|rxTkzXJ;rwmx{h<_!8@2AehZ*_ ztFDIUCcui-jr)k*OoTk;aN>WF zZ%^{&%4W<%Ii0{6BtM=yh`dVy`o4I#tM;M^N%ayLFz>+|GVCE4((VBqlW!s%f0Kl^ zO~em+jK>doOehV${V6*LxswKoyl8{aUc5lOFA^Yym!BZ=7X?uIyHNO45v%cwsl4mU z4|$vyIeC&7+i9K`pXq-=TGJ*NCK~AiCu~1P?YL&rvlIfR71>emf8)h@ZN@};t;9sX z^vA@%EXqgmI#7+dG)dU4+u?0(%SXqi*O+u=;5CTNa5b1tKQ)wP5Sc_z$2Fu_P`H3! zP7vSJo7VlevP}c*QrGjc?k^puU!G7bDn61rSeUGsq5xUu&NDM3=Pd)r4B<$?GFKAc z3HdL!rv8eNUNDjC2=0a_J~-n3OsqF>{@IZAi|CI`zacSc!0_j^(1~#|^wA=aqc{zP zsz=nxP-wX2gmrcbwt9pwU==@Y)c)~~6gO@BsGdrzi6_sSlT=&oH3#FP_YdD?ft=(` z>bcM$-`N!0huk_Oy{cLo{EfKvHxR{E57p%>Q_hh_uu zNk?^B$EJ^CFx9eIMkVyaR^qNrl-^44tAc?er}jjkzo`U@Ys%+J@N9w(4}T-C1$0=-vcOq3C zqCLZx!M*af6EeH2n=>nbXE5r#D4qE?A-nPmnHP6J?l4pO99)Zlv-lHkV|$?3iyn~9 z`S=OPdq#_7Q=(>DONH`)!KUMf_OZd9T9NQeM3dt51FO0;W1;xWZ~j_RaWAs<+T`|R zr#hn}x+ML1IZM9Ml3RW2_~D*=b5q`1FwDG48X~)mL~_&gwX8a%vxWiNzE)Ny@E%}u z-W$CcT?VM^h}S;x$F?LkvbL}n)U9c*0dsvKEYILPXoqwXNs8kSjq4(zzV8>lYFEoG zUD31|`>BMsm675CdErD-P2Y{e#84gf2oEfSbW)iFZ|4DL8rTGHS6dy-Ly=OQuZt~v zhP%&-Z=5|js?K^5@A>=lS#nW-1D5Ix$GQ1qFG0#r(*i{=d`-K_N4P5N5eiUM#i(a6 zWCpO!UTQbV|6w4r_Ont?f*HJ`ku*(cn46y@wCbL}o2im`#DnM3mM-f<@PLdlluvMb@U6}6sMtS$n@F<&AkHGtZ%de+nz<(3BO(HOK#?@ zxMjXKUy&9dcb0v5%`p6)tD8}=n26weA4DkN_Y~iP5_o}WXMOI2&-eBtm`)P$) zrf-07lSfAzZ?%N_b4{7K-jCGo=|^WswASBKHcsMk)5o+LU)ZUF4OzYim|o57ZGhcF z5k!0W7eZIhAKI0g*Yl1#mnTTwQ?RZ~0#rcCEH zEp9|>us=1fWcOIK7#V6YZa8WnR5huJ=+q2Yntxxnft1nEJ+o@j+_HlQH1d2P>>1If zdy?5?VX=gGs5TQ3UzxAUEikl3+wbSBtn!aDo#-#GcE#76ZnV!(%zi@T4s|Vp!-e$p z!IS(U_IdbWz(*{XaSx~_Qm2}d6mTIp=?IS>qlK9fqW^)E_- z;tcT5$_;=&B(G+DteRXGsted`#N>STOOr1l>C;eI3)$#lNym0>JeLnZy;V?eF zoHpNGFtRAN<_~KhXZnb?*tvG(bM>%K(yz57;I03)h}rwu`()j1`cE1yD~iD<_%Jdw zhYV#JEH3z)u$0QRc)w3cLHf(t2#Bz7!Onc;^$-Mv62a>H`DRxH1ja5~k%GBv4P~n= zcaP!J9ms~c>#%jlXDPyZ*l);**+gU_{B~{M2~FF?32iOEwO(=kI{KG~Wsl(;tnet_ zMRZ8fC3t97(wm{Wr&r-vu-vA0{v-KStwd;s_*Z*MVY5cEn-9w&q7U35SJdLFqWJc# zqUiRnLwI{gPBhQbbu7=+bwqnnP6SV(8UMx*;Av37edNQ*wjk)mT(uxjCJcQ*ydKpT zj*HXIW~1^iHRd-=3QRIgDoin&)ZYp;amoS(P@<8ciExw{Hj?GM1=yiF=(42AvNW;E z^rAKJCI7N;0`VC(m3tTq+(S1p&cD+nDPtGJg_4Exp`FW@0}AX!c|$wV&n3(G3y4F# z(a$Al(v%%V{oo!cm*eMYLtioQzgvAP5DC45L!#mco&T@hXljn&d1=uixL7KV@OdrK zm(Wo(e5LZhc_`5`^h{|h^n!^{MY+JO`t(GwHdAMBi0z;eJa{K+>@fIxjOdDC?$~)Z zQFgc)3XW)Hv;q^+CX9V1h7*eZ9j$4FKz*%bihe<@X}Q3Etx}+%7K3zPk5)2OzYqg9 zRlh$2Hf29J!z6k>cJv6m6KeDw6ev%|=YXx?dAVo3U5O9PTUz1_64ytU^H>Bc&F8>a z7BL@PjNZGICD*7Xrc9f}75+~K4sO;oeazJfjA>p6ydzSXGWx;C7s>xxwl4uAjQQuy zEXM`|ll(s}+q;;X|1U%4&-9$L5Nz;4SG37?kr`w7GevGKDSg#tyd# zj-6~%9Wm>r8Th1xQDcjTYEz1P_@a%TABtIQOW`G!WL444Le|HKWgAVpo3*(Ur77^< zW)O=Ok_!FV-skA8SXZTK%}fKW82o-N2h|Rgrd5XFxih+*?8I5F7at7#IJ()*x#N>% zOAXxd&HkS7KI0Ad8y4U0kAY}g&C}TlXsZys0_V6sfMX^*>Iw(1gqY11BKd6WS(6N? z-{YxQS}#^0bY6jj_1Z1MUbSnf>K%@>iCV9zW-N~mL}7FO@LXJnSNe0Axq&&3s}#2S z(42`gm#-?bvEl?a#$z{)G3{BTv6gzt)nIG$dtUx{yCTonP%CYMQ$=jq>-8fU z>v7jSot&7AJt@r{v8xnMQN>bC_CHLcCe_-vWU`o}`s)#;<_eG!2NW>v`K5K4kjZOK zU96NwbHp2lwYOz958p1xH(||Y<>}3HV{a~)I+%$Lj;9+&Id>H{A)I;d|7J&8QWR9h z`uqW>a%;QIyiEBa)kg93G(t;|$?4tKTqI>GuQV3n@jZ3!o2pu$5Ujb>0&Pkgy7QP= zqXxrPXX|l&%s(0rJ7$lvRHiE&{p_&H3--xq(*iG%EvEJ=wx7vY`!<#+Cq{#ZC@~T7 znA@TPA_1O$JN0qPqN{tUb#CHNNsx!4r+1GeIdj@r6Xmu@%W1w4MR({W-lO$wMsgR< zf*wXNX|zJ~G%`7*2Q|J$7~698d8Si%9gzykgk4-aRdvGaPw`ym`)!)%BDM+BRy-n0 z9fK4RDWw?v!`Ae}1#C_EW{Y!SGK&ms>noMdq80lWf9}HR$<&QUb6-SKk^ zHG9Q(s$V36GIp$d0h;0TcgsK*msglm$|C&ZEBqu=;53(EHv6L7rgidKI&qC>YO~iZ z%EZ~P%$+fn{4}O84X?$vZfg>vxe{hC&F-K=%}ji!p?u{`QR_xijDN0C)WT&QF2{k+ z8u*z0&&9Ar$Cc8#H<4L>!_cMURNH~jJUTcn=gt#Yc5UYJLP6jM=sEaYu>mITVY5Wq zHrktUba z4cyhc2`!K>BaPnHs{Zcg_ms@(k$zQ#lXJ7x>h##h-2RrC2$L=w6OG~-G>Zt!3_G@zwPI(`Ucq7lKmSAtTpq^{IM_Ms2(D34L89|Bg{<&lZyyvRnHOnZl~UMANj2FR)xjt3E)i3G9@Cig5VkhO}+#8lRy5#bOQ zllJ|AH8|@cHN(de-%M0n0EmH-oQtGew|9cSw!#H7N&sP1jlJFv6BD9Z?W)Y&xfNJL z5R)E*wwVkE>S8Y&cG?dJ%?E4jeIc94k}&`aO~>z3gh8IReG6MEJ5BZ z>NU?;d2EZT+wvo65vsJBfCASKOP>H`#97)kxW+CJTU3f$4b?K~WV1GglPGZ|_gOm4xz0o>+Q30r{%w_^8fE zm{UJ!QcO0pZbO1(AGDCS_l9^cW}4@seS-QN9hMyib^R4}3wNq_g8NzoZr76Ci3JLP z6ziHG2v6CA=>6cqwEi?KrrJTymOv5kYM&p|9}E

T-@b6|}c@X}UgG!LX z`UjSQc@@B>{y7a0@BS&>y8!SW%O4yRhR_H1ZW3k);a@Sx4B4l5pc!ab0j%n;(*SYq zuhG5p0WYxpAwjVSeemxVVS5n0KMvZ0Z&X3{`)%~^$UuH9e;m*b!XA=;{NN4rUJ|BeT=#QG!z-6Q<7#!7_s zL47t38iEI?AJhf))dIQu_tZeT`;PSQL_iO$Ph!v;!WY=PSXcnmXTzW{_?Oy2UQj^o zf84BUAkBS4`ga1LBi1Jo=o#S){9P*S3+l6JFaZ2Z{h;q3Gxk3=RW*>|egOSD2{0f_ z{!kcK`rx9U5uq)33)2DCgJ!xv2f-t#3+ozuPNf}ZyUa=0vv!bBK?GD?cJGQO!=&E~$=w|=wn&*h<+B1OPavpgX z-n<*IInh<1-wnGtbNb!Kli3aVCgkbP--;v;aCJ(^mo^vTB1FO$&m1^=3Mc@8$j(H2 zaXWu82Qi(<707|u&QyB|J7eksnNH~o=s|R6vc347QFTGJC$O9OK11a$81jX(xl=iVQxcQ%g2_C=<`}XsA8As~$_uyX zfE1k3FCTJMPCG5&6@r<$XP;;F$Mz177n{4gC^&l0_DORe$|-cRqHQLEE7uIOlH{@;Gm9CckK0x8VT(YBTuEJorL3jzS0IIa$0K4w;|s@4 zVjWk>*ZarNT~wNlQ&6%5{*W39#{6Ex-t$KvDf6T_Xkux7`E-RhWC^XTtWakNpNKKW z#;{SJxv`0?G>|_#wxLyFi}E&LG+*|ck^gNQTUxC$kBvL{qTi#IZxlZ4UNs=p)8D^{ zI8bI&QOoLP{Rwt#MKhW0*LYqr6`S$on16!)ra8G7^z2fq$6rocEJ>N_I%oDL* zyl}%v`fYs2xO-k&VNF}4+O!;{-o!a9W4XeH7H`je)umHHtw5H?lcbv6shYo6{$o_I zHNt^n_3t_vwF2Xkb#-KhzeuI8X60{WlZOaV7$~%To&10du(XhL89I|v?C1W_KP6nA>y0_-okF(cVMQr|hN)1lwMhiKM7ck%fk+vbaw9@ZGN@70HH0F3k^Xzr0^R2WFC|Z?R1d& zUB3zaB@|@YC47!uWTsOr8gl9!eT_E^PyK0oPI*!^+bweFnC$|U;eclkY!b7SgJjk39_p~ zLb5GB{VhTA_OSsxWBO=%_L6i*+z=e-|Pm88??B3ZkMgq_+$**Jazb zXd~-?3ienK_gm~k-ilz)OvcQj=~>J>PiJL6Wd3lm_w4rfOP z#h2>tX|3L-d)D5t@T4nnYZCdv=0_>qIw3NDX$SEk{5yZ)A_BwJU>UA3Cyb3nHt7UH z-fL?UwOTT_38U%ES`W7Bg+~tA?!Ls{1>8k4aDmstHgU{C{TXXsp(tTaIYiX6Mrh4R zwbQ8Gs|EARkQHWssV*YXPa>JKE@5S+SjPkD%-|%sJyR9MeeZlzd=^oAzkN~eMAl6z z(upzH($)-LM|`^ffwkDOI?GARtgbeZga;%Y9SC_F56%nm6dWL>lMieA4oa8K`SiL z;caHnh|wbFjdJ*)C1rXK6+S+IoKQLol3=nE=PNVlBN6rj!H=Js-S>b|L&DdE9C5S1 zogUIl!o^#i?Tj;huH6c7<<@amqV;8xINKOvC~A>;Sp1=)yZ-qj84CJ^c{jUR9@E5V zwrV<|$1W@-75*jYrLlGbY-%}Sn8wmc{R8wrA^jhYSS-+8>*~L7p8O|Y@%(Q?TF%@_ z%*o0Azv&R!YTC-m>S*5x&|^|jAdtV~El65$pm3_Sv{k4>%I1KakQWO)phA)0q4~J- z*S*dFoh_=~>E#RjO$SM#(=Lzs0p%>dr-dwE(7w<0KYeGdFH(P`vm9~Es;jzJzO=n> zZXe%n{DD>@?6wOUg{{}gXd0Lb=aBUI1<;Um8B-PsV|Cd-Ipcas zZy(O}jg2jtgESGVyr-wuml5K@MG0@)>Or^iNLTo#5o7#8GIc|2^l=#E+Rl$r2Z<&D zoU>TBFu72WSi=lWmIQn@G>@mbxqoB;1spDL+`o4$?@0vm7wIN#GtA2AhczVtZJ4YU zXI0*0rjC15l9099*rN7AKqGWZWus+HWTvz{<##Q=(}V%LJF&g^u&eZTk)bP2dsa@-$i0DQUez61=MMHv##kMHImK}F ztqC=@yqs*w;F*n56Y|Oou9=4tho{=W*IJ`^35(_UHdbFVPqLJzZ4kvOYo38vIy4|? zfbVxj5Y%})?S!2NHvN46M(R{1w6VW*%$0y82Q20>wEz&|yW?VXCBM#~E1}&kIV2bB z{vh2ZMgp0JcpX!}Y(Eb82IXqp-{i=HlKDaq?Gp??(L5eIpX)))SEV|Fz0{Yz8jbA% zJ~cI7AYiVvKX%~rc-n#%H_mZ%0YOsH9~~lJPnjy{2%LrE|!NaO!o@?p+L5O zB)fT>ROML5^xXh!in?ios_+#+v4T8K`fqT$S{K6;`sosN*3^SM->EKYaiBya@rquh3AsRL5NF;Y4*T_U!G znt1kv>x2+V#F9J8B2|oCoUGxQUsJX+JE5#gDc7N_v2gqGXg^Gf6t4aT!6ZvFz+W^z?<< zQ#XThE_H8|441&XeL|mqM8F;yOXkA$=YmJN${T~8pxtrF3tO^)H`GDuY@ zr)Y|`)g^V}Q1EDK{*bB<3dYuBQ5hK>|S+CS8N!FV97t zDp1J@yY)+HBodssc|%W)V6;+|U&ZpB4AQy{*1udEFQm2ZeQBQ2k6Ul6Y;yYFF&A`` z&U{*cy`URjOM?W!#7=Bsv-JIWPcDSFBPz-HGXV-B7k#uyPrKOmeW6%1SiP*7s!C-09FPs9uViI^d z!5!LEa(}DQY8&v*JGQTn2y&_}YC38vF?@4gKA6*`N+h5@Q@-#0Zr-|P-u!diJ>H&X zco7Y-RlRTBvYZEFXQUsxSWk*Qg)(NO9y~33cKf4W3sCj%`vRTib#D8tbG+xmd+&%4 zPjoz_dKhFLB4BGfZcWep$Z+)mJ6^YJpV_GT?t8fNUnP+IJ`>^g-ik3dk`KBRJtu}v z{tQzBCI^If3QholiQA8cJ$>F+b<6QPd*Qtu(^pXJ@7Z9&_FLmKKR|Hq=u4^7Ih)~= zK-OzI?9W+0%%=|dk4(mo+aBh?I)m% z8|8*xYCoYz^e!n%cGg3jI6K8!_Mw9k#xDW8Az`n!tuTPd+-OTQE|d%EuTmzQaEeFC z0$oHzEQGWlOR!J^gfmBB!-=c+Y%g1l*vG{dGNXLS8Jxd-^}Ra=An-v|9_F=krKl+_ zqQ-p9UcJ)?>)(nBwq>;fI=C_a_zh1Ekv%(MOBveM8;hZ1hc4&w7NX1APy8h(axold zUqQSN=K-RWIuocqeOnmk;iusNSoYf~Jsob$x$0(R06d#(4kIsO9PP%K8_Ec5_lAV1 zVOBj16H+rhekA!I*>7R>?Pj-g9k~?Q*GV~Ly{Nj&@g&@Z7)E4V2Gq}T3Z6n=-%C`d zy3iXGI}hXKbw$!74GGHfv`KjpQ=Sh|FW_= z>-n5h_fg>_o;9O^0e)0=A5oY32v_PPgTltIrW*Sky|o?R!rbP3=-^MlLfP z(`*rFDP9~gq}rVV=c-wtGI(lpQ!XnoC}9BFkOO;D^|tYR@u%BiQXUzmK#*2??v(S% z$79;eMSYgk0(|F*b(dNj*e%rJ5XKnu3mEuet~gV_0uF2*JfiU)2l|}#Tns8Pd2SIH z>;^l#EE4zFo4o7sTfOp)TT^vBf|SoZrn}*Qc67Rc`6u@00 zxBj)&Sh`swedZ^h_d#FFFbRjhjvtUf#12<>!AWVJ}sov@rW5&+u zi|u7VyIDZZ?q>U5F7qd{b3tST);u+{m9c0%vzF!dON_JpddmX0`pg7BBd@x*HnGA{ z`ICqi{uKDnCr0joJ{fo7^xH4lkNjR-Bi`@6Lf+F7m<}!5-P;nG?s*=i9*|{=+%Hr- zo0dv0%y43e(5b-<$SZ9LbFa&6`8TC&?NtK#)I zEl)zA-{x|r6!JZ>V={F)iX{Y2)%>GTzsP2)tI(a03=$DJ0q1%&PPJ15(nC#xrlYhM zNsnkfJA3?;E=6~5E)!*L zd>EM1Z$gn^djB43o~8ZgraJxfaEGjN09-YYTZoKEJk@0u5L@LdV%ft=KE zrUPW=xC!sDzFC&l-k(qZpu$uW9SuK-cJug=%Ogw-xWYOgEU{pgw!~fqFTi}HgFP16 zj3K$cqAQxrganf{R5EzW&dS=%zzUPNK?@R=7?r<=fBhEI--d?(r@84R8Z-R+wy#Gp z0PkP~*A#|tim!W`7{1{TE?|o@^ku)%=H!Ga?BSJEPccGk>~C*kH(w0cf+0GosvB#0 zLt)Bz5rtZ-;hzsTy$33`ZOgTv6JCnW>+w8s7yls`4Q;n(~a!G&Mrlt6~1j)}SB$og~0cbyK*Ot*aXh9$)}y zqB9&sdY$(TOoFNPiPJhSSweFk^*g5G9C@n)dfX?Zyuf-jDa`olY5|| zRKb1d@BvEj6?gN2ob}4o^$I=luFO7ctr3)K2#s5S)4MD9O(yd~6H=*9L$#d#NwuBt z1*w7JcfX4PZjPh8S@36a*oA}2qLC$;8SfAkVSDU`N=9oC=1Hl(Qql}ojV~0O-R7jH zJ7P_b(=I~zNVC@zrPo-TfjOfm2c@W3sySs}fi{)~z&DuzcIJ)6qZ(I`BGGUhw=v9? zK$a*Pk+-sGfXv$e+|B#K-z5Nw{2@vlBZKnBl5@aOK=vu0BEiW3?T7IFi_?Z8@Lw;B zO3}12+GuW!bV+AT4B(&)?PRGIF-azsE_3^9bHc@W&&cLlnfc*kvEz(=n{KlT&3=sG z6dlv(rVc+;qGM1aOQ6v`)no0o${0)Le!|;5JBw&53++rz>!tH3+I(BAG8q&ow(+Mu zkRd_?cqzj-e5@53_oB#919zyQ;qyp>RJPFcNP=<3WSZp82lyW~|6zxcI%L+0$C8IC z)5BS0jglHNUb%*!H=b1a7mbKR>XE)TSkL{ZHDX~Kum79jA=wKv$+Khs`tQjo^+7SV z+gWC{d#A$%wlpDE%6!$!?8?1THmF&xsytT>F?fp-c)UV6i_++}WUabL_5%c~L13$q z!luaFSY7Ggz{=QJNOB^4)0V%DdXlpDgp%JsFSGpfcW3E(X*8^|22TzT^Hq+*$3+7| zO%?fv(sk2mDa!b1S76fHfNHr6b9Q5J^s#UulN~|F55_|jvss7!vMyo9mTwPR!6L}b zYSI)EQBH?t31VECc3%aiUJuA}LlJHTI(Z>loqlP^5cYAuBWa!TH_d;oN`*S$@`W+_ zM!7yQUKb1bq!Gx>E&bv(IXg$&9{~5$e4;Za>{29nQq&tf>;5w<k)wq4^8&BM5_Jk?7 zC(P;{MQt9ObzWxje&TYhmh_h?zq*i_yw$PQ8I(6oaTrrdP3~?l1=mxiQC0JNGYwMH zM#ygtWCUgBTpDN%dU}w<9HtCwv?*hZDKl7{bTduzjN0NqyCxa)JT;$W+3(Pa9k$TQ zcg)G9pigcU?yDCKlpayaC0{Ci=7jytff~3_6*uD`8vyPTjppWvuK{zy;qw8n>-xYd zGs1HBY(NG=ZsqD2U}gP*zZ1PsBChb=6)#1f@eILHHr-_;RXz!knBVORu9>%JU$uIh z20x(r4%9o4|ATUyowH|*vaXwd`mm$YfcdJ=Rh=q%06t{-;vyTMSwTRDGV*k10d{eL z|Ie+X!ad&Rh`CtQ7o2~*ZNRq79bcj+>Rgv!{rmnmYlit7grftOuHFrO-BANnq5X@#S57o@b6;=ViLb@R_ zTjXTR$npf5Sye4~-P(UOz&ec8LQQ_Q7QR^X5^LVlf{-o#Rw>*3uPbgZ!Ok3l`Q?SY zWkHH2@}sgUb^6i@71v9XlWD`z^>O)kE2rz`!%aXv7d38XVJMc3>GYsMdg*b)88E$2 z?mm#K^ZeSdeTnuPZReu~qr(gg#C3Wx0mZ?EC+cq#BhM!SDXoQ|ymJ<&-03Sg%0gtjNOqVf;6xg!bW!|;Lw zdcQzKraXx{nnFZ{#8pRT?~B6?%Xv%VXG%P}5gOeHLE8`1|Eo(#drBag&;tQ+SOEdC z{;ze3|JEY-pgmNUmQFTX&6uGRgMdJx;XqKBp)+wvX{9wtGlKA8!?;Nk8RI5{(!)U_ z-dkI)JBPelO=o`TJKIqbLCh*--L1fVZoea79qnUOPl z54_LHcHME24Er*l9&3HMM}oh;c`^M1Y8@_A^&W8slaycX{AL~wToI#t40kgjdH^>H zw;vl_*Z812PkjbH14A#bFTQ?0eI$g(`x38z0|>qI;QTFjYk3u8a&Hfm2|RNMd)eKp4BoVt+@gew=1|@96{OzY$plcrKy&>TJw)n;E}&!=B}Ak(2JyTV(OXR&dB01` zEq&$~0#Z)$AQF~9EiS>bh!84)N;6^%s7lCZw!_ z^h>muN<70TiAp@v2yO;zBGy>ev5G2&MJ$8Dv40X$y^|>PJZY)+yIZO6oi0V^mq! zmT3j;!Q(g^jizT1&nz7!^0j^yNk-N&gDQm?M&DuOtb}tA%{+tHB*L`B z!eqF&A5^XcDzhNH)t(99bRBU!0TZi1rN!q_YjGfv1ZlKsbuvQj7yUr8hb%R>hio3+ z2w%UBWDQr3@ylJDbaS1x*3q+#E@-zpT72-XygFZ$I7{%BY~W6^%+-)HCv2zy)~fW^ z?iiG(vhkcKGnyPdT(escTirJaQWQIMwEA*2R5m)!QhZcXOhqJV0xFWzv6xmtzKH~H z)as=lFn28SDjgB6Qmt zdodeIg5--+e+^>u!Y+0?A8Pf=vSO3zVwVXoify_>(Z)%0yBAdGFHK-AD3>O35)8fO z9bOvT8puaR2pGdiBnJ_jsODzm!dE;xR$CR_s$lYGsT}-D*j#kG?D}lF+^cx%}`<@nrQn{M*c=xq{v#!27qm^+-b5IMy&M zdn9uXA`%-74nF(}qLfvU$iSE(l3}M?*D8q3R6^K*&*} zib31-r_Eyyyh%|2OIiCC(+-7W%jKq0dDc=2JqgNbp9<9R7%&KtQCm0bXQogXg@Hb( zuXwC5H}2#59XLaAUJSXALb(NXj*tyai!eXQq6yom=Gt@=TViOv@wbplZ!B z<1|8c1q>oHVkE5-MF$qJtfQu4LNiG!$1;Of$yY)XT!7Y95^JHJ(sA+XuWH}T9{h;9 zJ@QA8l?N)^BU~T2Q`mym2T*Kq zw11EZKZ6?9Ns`JoOB|Tder4+A65ZrC-7O?hkn_mTCxlhO%g`vW6HyS@MTy~(*xWg3 zL;H^$;N+`R#$IG88&_A8s;3|`8jT(8aV9E6x{m7W^xIyGVR`-enmj)kZxj|7J#5#k zdP7#$p|HocG8O-~MuH?5<`ubx>RQIuw9+cL+HP2CQl<<49O-Qn4kJTmj(3FL_}S2+ z#XRX`XklqAaiPr(osYvVyNXIP@PO%JICV%O8#WsxrHjI4gVUolyGX=HsZHw*rJd7^ zqk?>5S!<=ZTHZmgY{v2Msq3v*kFev1q9n6GiTDid^T}80eg<&{h!=`;jBRMHSq!;rUt8bJ|^XEN` zE`$vuk+CzES$`seQSpJ$HehnwkG|W87F^*s6g#CR0i6p%{fyjcbfq!<-dMi9&hcnt zYtkwSD96;^Y-(}nI;w+ed}W->(eDU(%TL}5+RQbI75kx^aDdQGPW%d(fpH1Mv6D^j%srNdT!L_QVH|>|b zkP}xOW556HD^guu-&iV^kvVwhz{(Xn5bGs#{yrMNPD$2$m2Wa6-=|?pt7KZvNSj|A63ZgS zgfvs)G<$pDf_x^ZSDz|1`&YrlEURTM*OO4&;mQ9p4l`+ojflE4!M7m*Cu7&!`bqUL z9oCab8xdPIETt8+_tZdWIYt&7Stt0Zw$W8nE}giwm8}p0QB9}Q&FSSO;Ni)95pLU^ zUApk^^byAPQ{ny_;y|3N8HQB3 zYF@IU_H0(9e9&=LY~EcuCTKpU(VaM}sYDi*Ba~Y!ahWTI&m_xaJ?K#LLk#5sQAtN_ zoOAebAq6rsKZVYW^DX_zl9=^IQ_ND{zGO|4!VNWs}>4gazioHD6J|3|q zjMebdDASMMO}-|EBLrc@NGOV7D&F?_vO|&@ZVVCpI|PUWfZH%wq&tHudXiH}&)5t= z#Ioi_jgb>q^gTOQD3Qomn-a_PNi@Yl!QQfd^Hg z$WP>D294Ks3)3YX5r5`zE^?*kfxj(?^%>ntqUQm-%|6twhZh9AE?@N!@i6oj($*K* z-nK^b4eL&?yaGQ~4I%-=f- z_KzU+0Eq1yMI<6B-4kJ`s(S;-_LUga4!~;L;LNf2o$i15Fg*b1`{QVLTgks`+oW}_ z&{vRz0L}Z{O6%D5F&4^s<%QkJj4~mhZq2n?N}unbSe&tuqo7q{w)GyIaXJJM5R`h* zqY35Bmk-0!uj;=sR$bOPxa#G}jwpU|<%ouftY(z4t zyn85fQZ1yjc(KjkwA5PNxM6NJrc;I>7S%9BXq_jJMpMA_{?ab?!fF4>Y{Ln_r7%`A zGGNm&q^Xb@N?egTNm$rWlllZwMFw=GmMB<`#6$}a?y17szr&2`*UoJsNe>MrI{H-> zs`Q?nP84tHyGEW=Ybx%+6#D3!)ga;SNNQi}+tSt6d`ZZdJhgJ$_gYppNuhw-nb~&H zaRC6?9l4OU;AO5xT6#`aTVr?|68NGwOeIk{S%s2PGGj34QiuL3BDMiGRxX1u?e2Yc z$4*RzpqWwTBJ$OuhGn%~C59N3%{)1O9InTY?^L9brjHWC`q4^d(o-;M z6vR@gZJ>s2Qv^=Cdz^$!NK)hj)lyuakihx}UBWZ^beCjC_y@31i1S7mXLk#xLnt<0khtGi1)_Xk!Q229y41k;-YWhJB7AoH06 zs;QS**bNLXpTk5oS^JHhqjc3MwlspL+k@u~t_sQBvmSczGmJ=jv=lZ@;SVChO_MawHfVG5DkF$$jT^f6m$`{z)vZV}y}7a;2+}bn)cki6S@@<@D4W z=gUgw|2GFT9%?F!OV`^tsin}zjS^Gr+NN^_u)gD#+7UT1X@2yDWRCwR^#v7dJxkQe z)D@Se%ZS4;zvn%R!Dm8ydUqkfy^9ZSh zg?gSi7h>I49oQ4kmo!U5fP_d4r-KA9^>?TWG?)cS!p#!a%@g1@XB&o3s%BKGe!1^) z`RlF@^Jb{CYajWpOr2L8nAuGkE~wC<>6e=~;Gk^{Nx=h!6>X|c%5IhO z$x73(+)+(={wqW6CQU@1Q<90fR)v$P@@Qx>YBjaY$DfUO;)oh^8R62@o>~BZbfV3d z^nli$BWA`aHsi>QO*@A9bAyg-@#YON)ivgEBay%rrZ|}7w6JlD(Y!E_z{81^x1(;`o5D8k`0J6`Vo{@#onv&`VPJgvgyInwBGzUV&CtcwVEal$pQ6 zxr!g?m-jTojMrblT0G(Fjle?TY&lg|RpAlQ4mNF)KHhHwg6n!)@(f77;5i;`f!_t8 zZS27b`M(!V4ON4>Y$Dn%li?yJtQwY~CuJrslacjrad3x|?Ss_5oNF)Bi^ zeW!o)pHkfPUw&FTUHvmj3PB@{wKV0X!f<-tq&1wYu5EVkCGj&I5A{(r{!lki2UIGk zg{^T(5y=1Oucq#qEOZCjt&gaNP2KvO5FkS$b~A5?e719*pN~ zmS~v_RnB7_)zwT1=v|cz!1c*vPeD~6nFfb28TE>%aZNUqEK$gm^9!_wCnjw zl{QPL_cs-I&-{~$_cQBTFF^h5i|wr^_FFHMBkP6Z?1s)`!_yySYh~ovl3w zc=uui9m$JO%Gn>fx8Q{05d%0_3#k7dpJ+V!DlbU@?z{j$&lWN}1A=YpDo#WWk5|44R0KPert{MqLsJDaf|(R-_l#0+;x>JSo{Mr zw6Fo8j0@o|L@S~N2<C8!&#l<@ofDE&A3+bQbL&C@E?Nj%QcRnhT_tm=yGL{)sbW|f^fiQ4%$2n zzD|0RRzXExL!t2Gx zX@VI$@A^-VY#gLeY2XX03tCCE7my~SWGG?@;>?u6`|YC^UK2v@ai;72##N$yh*3Q_ zNC7PM5I&>)CNwHQmr->S-WkA$gz_?$_)=fzBHte6@d}It)F3qxWANQeQ#i zzZDWT6USPT2Sugq(BB6@r0l@gZJlUjC8bQ_s{Q*{f^wl>`u)yP|M-v@YVs7vk6!TZkC5j=aMcdL=zTIDz?3Z{9=5Y;c|CJWF7dI@uJmKVj3rs7GZ;ysQRjYkM!`_6? z=*2-et*O!WWQZlgGJ>MBJPQn_4dGH<}*g}W<%>J)iTeo9ecDGlA>_C4lOa0hm+CPF9ysq4bywz z^rXQEIZtAgxg_Uyl#n=!??~E#6-CnLL5GCYxrjf(5g_K2)J;@4q-gl`Dza@-cYRaR zz?+q51HRJstJP?Ok@l|n9( zkUi>^VqPePHcM1?Nya5NtD}ZblIvd^oZxzDRixtm@FMb}Jt!1`1Cz|TE6U>2a-m9P zI7;T6k-)H6Ao1HzzWTzY$`rCG(dk^PSsNOM|_$S$H^a)NTJ<9I{j{U*9A6@>Ff;@a7FR zvaHxf-#L0v&(~%~1qX|5MtH{6G^!!DU@lgQ%Z>9>aMC6!X5p7ee<^${*`*0~%oeFw zDm{`noStp+g_F+K1;iewz;DtnYaF|zPFfaKl{7;G`{oPsP0iD6lNU=a%V!eJ)KYIT zrprajpri^3Sjj`GrUQ34p?I#Q?b4LZE z+N7_;fz~5Fr8}TUCHg{C1uhipSU50ECOYLSUtTnt#AWHW(mVL zn6ahswLp}XK$PAYz<{{U7Fnq_h|$d@jpH+BS7-7YU{2OWX`78!Wx;!nTTv=5_eq-4 zO{z}^n2vkKq%!>BNnYkkieBQaWXO$B@@Lvf=LWTV)G=JtAyxxmh%EtyYbX^^!HLE*Mx2y;#+#6Z z_6V^o9^F5)pJLU&BL>W?febtCFqK1c!r5Uhw_sxMpuE!5j$(1ldgk6{k2#5|HZ)cX z*ka7hULt5?HC2Ig(|~2hlMUbKuj8mEuB`bYtV2IdW@;VgTM1qi7LV9c+$&GF(Db;S zb%)v)@srDO;6ZE0U>J9DI1#lu=0p>GYr*y^4bT!j9hAXzAE1PN-oL6_1@{hz_r8)16uLU^rqqVP)UoUPrA9!&PG6K=_|WHShx^l^uA1neWY77$C#bBAw=>6gzNRP2?;$raE4pG|}ZYcr)q-xq5j$($+(6?yVnsbrDtUrB>> zxX(3G2uk2Y2ge}L)~wy}WGqsNb9Ov*4NKgZg|gU#4gP+Lyd+|wqnzCXMJ;{3d`h0Z zs3zpG6U}k{(cjyDe2(O#;{PrA*v z)?Hspi>WA8-8(?hH?IQRAxR5rq+ATG7@bG&n^JgAn*JeQi69Q$Ji%9xnVd7nmnS-Q zvIXkyiwe*zdzrqx@Ep-=& zC0Zcei$G`2FS2>a2)~Cj<4gco(E7kH(*qE7EF&c+iO`Yn}C^vIe*Sk*< zHiTa271|G#Ctx?;LU+c_s)$zxLLl4QT^ivn%Dewkas)0_ zGWLL?bXKp)Wg2L-%Nbzq4wz6Mrk+f$J_sj^=F^Hr#1Is5M(!-Qwh{Ij7>R-d-ja8f zC4>lFS()}Uu&M|kg`PahR7M6pTtQ@&CeQ=Sl#MIZQh1B6|JHEGun8(a$(`R(NTGYE1_DwMZ)bC>I(qMgy1 z+sQc#By3RvnfEC#fA?Shwo=<5QjbO{H)52u4c$AZE>lXZxj*C_xr!RJRP#&_xNIo= zfo5kG_f%X1Hh+N%S$fd<0|yyrNJfNx!b62C$p$+r(@Q4VPx!p{{zzt_l$%3;78GDJ z!;^0x@OF7PuvCs#xL7wq!TMN&x>1R1C>wCUbf{v{9AYws*-YkEO;(Q(qyj9it~BQ& zQd~`y2$Qsj9+1JyO93x1dP(eMGCZ1G(@$rBX03W2jAQx!R0B`H53fRV5g!?`)Wlm=yot70 zDdks7mg$YZwt*`dVjol(ZaboQ%`e{#j#1VsBDn>^)C{Xu>E6fvB)?j<4f~?}=d-LM zd$AptKc8|h_Pp?R=iF{mJ-3heNs3l2w}0-OfLDAga$(W@&gqG3Q}Sx+9QHNny#TfB zxq-!|(9=}~rm-@$y*tpzfKw$u@^9Hq)0Ycq?PQ7rfdAO*3cGT8g z8{Nw3U}bmA34E+UdBJmi`lQ_zCOuhp@OS7_%btemN9!*eI*@rU5mG*XW!<0EKaDosZl%*|S#8asR=tU2 zt|>1(XoW=6j`XY7MZR2?x#xlvu;Z&U?PD+{D4PussjRv-QmrHbx*z`W^5HPpMeo2K zi-?A0@5qZ~y)HGSAp=03tl%&PQOgW`6Zu-2e6W7h=N+EoLj&mso-dN^NZ{ljd1G|QM^IG589_NrkEMQcu zNWU@=D6F^~B8~%swFt7xj|Q2mfd;hl56~0>dfFt1En%fZFl1gNB?ib^h?;jT=h4-K zo3mr03m@&;qE3_o%J2??CU6Q<7_`B}Gsg^0} zQxkHx4wjae4aByNw*SIvF>7Ve!j67yMbU3`WnwXRPE@lr$Zd4b{~iKuCWJ>!e1B6x$US*Uuj2ZXlN8n<$Yq2ns)QPkBVywWtQ z#PlJSR4Si1T?~tG+a$aAf^DVrp7fY}TUV%_^+`KzF^^7te7;(tD0!%Wb*m70KehBR zsw6$4&YWU+U~FUb!Lya!LBL;%Es_%dS(nkY5De zqO@V$%>a>6I`tLvKu%WJtF^SJ15I;9jTY6XO6b0tK%0GvAgeWH6Q z(qYC{|Db+`TD?>eB^t=t+0jZzMbU~Wnxr?7lqzEJUh_HjQV8lim~!++e(J(CNG(hL zxJrdngmc7{_MSZ$%LrA9*JsED-O7#O(L{3p=9B8SQSB<&laF1jy-ji}YzbPKxT+q^ z2X33}i4=w4#Ty?73Cd(@=_tc=V(IRf7;jJGw)RZKQN>ah4?kcKpJ>K#}ie$A9n8}WczXR|~!A`jY`yhii8^9dDOO9t+ z?;~KHDT!F5dG;3ev#1s7M{*-T21TJ_FHSM`gC>Z2$y>*p<8@^@*U&X7XRtIoK>bv% zp1hB$u0*+}ZkpYnW@|WBd2psjqM{WaxidpQuy* zz~(LN24S$Q?XO(tZ}74&sOb*mTJJR3sl)MSfJ>~yw+{S&ezJZau zp>TQx9;i)?*z*#MT7ry(@J)j{lZZ@ycbF40J~>6-XZ9^RT^tG9c&gub?3V5-h(*0Q zABj}3yoB{-&cvuHeC!YaGu^D{zjNEHP%g?mm*MnX*|@8Q;8FJFOeH3{o)~<%+9YqV6IUHvrKqe(0eNF-Xwj?jn<}7fH!;{p8C5xHqK*>W_1)$XDdq)NYGUD&Dr{z zy^YblXmJ+mb8GnwQ$>oCZGaZ&zw0wGZ--Fx}>t z`@oN*6P6aRs{oHbUlT!F1KE&}F#X+YIlcx+gqm6%|X##1^Bnu59m$K-)xEOnQqjs*H#e|A9Qv|9-2` zF(D(<5x>9T&|1ziHRb`LV_xiUx}r<=n>>a7ldni#;_g}!OpsIW{K}DnraG9+j_Ki# zEmAFoHdy`KA4-rE;ep6M)qUlE@ZUmz54OFxuAAWb3e-f=2BCmm!ii9;kO#5{ z_oIl?Uds+z!D2C)?1%1)>>&l*MQsAN0^68-p;AE>ag0?)r+thE-2Ju!x;83uai?pX ze+v)|fez93G!~{$OP;D+#298u9{EGN1d}DC^u zTeV*G7kNj{H_FxwZ%Fe4?XK6SOEh@2ljpwGFzeUIsmIUC*4Pkj%I3oJ;B6YtmXrrs zee0^|KI->yjY8`9x`Aq^p~uRK^Ol?3`Dk>_q_$OE7hW{idgW|Rc-6W3IwLuG+Y2V{ z=T^DGA#^+4)rS66lqK0bHq`1hu{Q0tk5e~wC>avVue4W34dk)8<2G8*3s2;MT;uWEV7fGB=B zeQ4*of?gy+HDnH2V4_LnRu2kk4D*zRK(AlNEb}Qlu(k6;9)xyLlLIe_ZYm@xpt$YG zZ^|{U1~m_8;S!?+W3X&K)bA8;2?H!Qo*VKc#T({mrK#D$wTbjEEgh`p(YZd?^f|QI_p0g^(Hy|d^ZpZT1v?6#ss?j#z?)Qd(m-dibO@9kO z7JU~OFC$Ihbd;nVGtHpqJ_QvWO+UfD{e`KPAMsEU8q-ui{g4yeYzs~?CaV~^TZ|I7 zriCpIJ284w|Dl{!(4{Tm&xw%%ZP`LRKigCNye}73u3EnYOgI_L&RsNW7Jfaopkqsh z%%6}Gws~^@X~!{6Mn!y%MwU@H?cP&`TxH1nZ~c%HsGT3aK^JgWjez6YE+M5*2zZB0 z&`CqP>Q=bNU%gHL+G)R^6^CANyhH6;j$NV4_fq3l(sM_hW9?wTQmm)__p2B+3COw6 z(1>U*T6f7UGJ~I$r4P;!5jaXSbM67Ium8<+QO6XO=JvM%=(fXln9J(z z$Z7)+SmXMbaoFh64F|#W)m!**1=Qu7=C1wSm0>w`X8y>Xoi|r)O6Vp?Yh5Q|*1>kM z+N|U1-uNy$)+)^=P{JWCI<^yHWc;71>_`ue#xM=G6@t-olARQKy)Zv_bHkojZU zQd$!tJo#DPdY^)P*dpKR=s@?fcmm)i?(Y3*;ojGsRO5bk-{^3@U;Z242n#Re+{Kt# z&5ZS0VLw{zBs#z^bhsoT-e~Y1e~T^y_jJQ+v$EFcQ%xzymd=aDvgNC=FHOd{2;;gRdS7t3TN;nzA_Wz=uwg0Q23E@PV13pjb@aPm*0+nC4blb2 zc6JWN!@Y$G(974em{;#Mtz~?)hLN&86+2dYbui@OW~e9R@?f<`v;dpoR{r?tj5S^m zU>f;XYfdL-k6b&SGLA2t(?y^uuCKf9DPfjyIFDtV#gsThiEj-hWT6Q&!!Ek^un# z#DN0<2>g#z>(a*drpCgqhQ_vz<~G*<%&-?Gt}D#TAq6i7XOR`+8TwI-@bj+$?jF=O zQh*y~?wugD^3g(HW--ZRIL0gO%INZC^KJK*Pq^*^y~ssdUJh`}h=I*IA8m7vc7NGa zXZFWrZ+8LpU~owZV6cd>h+D^Q5VZ>1XB_+<)s80+4?DmSXaD`Yf8*O5I!Uel2itah zNqS%S5FY3$uy>}(d?ng~4a}l^Jb&jCEkI42)vdBBC`pi;hLP zgfz6r#@x%4U8W0`mSG&fN$s16S2+ds;!I+?;c2i+!Q#OBMw7Dijx0Kw)Q5W0cEf~A z1C=~_Ww|!yIYaJm&{sHv>6%(N2jPks&E#hNioP5;lXG+qoTIHgN_weEbtmXegBm0Y zPTjNI75E6wC5T`gP!^eBNX?a@ox+=U#+9AuvSwGq=_Ry{@AbK?a@nPcD&^yL4sNa0 znh@h42}N6Y2BH-4s@RG6iMWXbs@MauJECzSN}_UTiSPRw1~uM#&t(Rf5KI_#;Ci~i zYqBmIx?lZt{`(+YX94nJyI2|g*!_%{Yz$cWZm26i>=;q$J#ac0_|#qiIAJq?DlLTE z7I<$y^vr{(LnxXD5JZ$(5>kXAu9R%Pdht{Q+l}d(iZ8vTAAZjBNcxbIyL!F13dq|n zEZj6|vQ{W?f4G_wf<9xGbnxJ3&}^i^yU@kP>54dNUhfKa8!d>uS00>20%C?9I1^fWoT_^6nK$eig+}mL#DhX2+QWOZ zq8wU^k#e3_G#83!!#7rTOb2{qBcCDIG1Ok~i&kPz>Q(bm8A)f2x_O=q6GvV{wR#e= z^X~t^APx5`73e|$003e9H;IP-Mzv6OG`IR^^>wP5dtj;}dyB>stK-<%Uo3@L4vAlk zs8|`F`|p!fB7$)~1{b!lnS&de8_5z#mN7}}@r|pPksCvJNSaHi38*Io4>`k_=Ta5Q z{bWR@?t5r=VENdOEtr;)2d(E|yBt#Lx9_t&4w}E7&y)eyZ&hKP_Y%Q4ZP$X_U@zE| zgBV}PU~wIG=}~c9cK2~wZXsYPZpF906QQ!7axi3Xrc7O*)M2v*6sZm!thpw)r|)kf zp>%I@Ac{9$2z9#AGsm?LUH&U!LVp=ahnJjrHl*z^e9S}h)S zGK>LkuXa%IH`FD_tJUJJiZVGWYE%--jGm2lVS`sOOt6sn`D4K;oylWEpmu-Qf)t;0 z*{(Wu?1Cxan-&8~=nrhb0A1y{Q3+34)WHIej3;CSPeB`TStf{KG&OlM3GVh#9gRuB z-P6P+3##9JjZEF8N`dv%X<(6kS3Zp4$&5$b)M+p`9`E#lRp=MX%7Hw7AL3%8THvvH zgy|qtUQ|gi20@r36=oQs2A3r;b$O)cbaUkvR1BQ0FASdfUPb~5?BKqJg|3uzvAV-H zMUc%DqDXi;q)k=_y^W7UT)$W0uS4i{S8N7K7*Wa4t)NM&8Ae-<3St;;_+oG_ul)fE z%a15(UBS1>Wcs6#$L7E~<`N4sLRA5;Ff$0mu62Nh`9Xp0cD(15mwHfq-om(`%Or4Yl>+TbIThTr(_jYQ4L-}V?D{!U1kbDDTMdMG_5oaH)*AQ$|6XN6CQzNMQ%=kA^8kC{pU0vB11N}!I9g$&kWAH)~H{2`!N^&(U@CS+*j*>BxIoOZ)(n>O*%Y0ALVN`~eKE1+Bu%}} zl@m!}u&q&)(L#zIp@1u!q?)@zcQl)ynOpL*EwX#(G9!#n+(tXr!f^fv#~vriO!+9> zFMCz(==UR|*@XpBpTESQBTh1%T$N{K-S67u z&MudEw7S4Hf*gg5ObZUo_$U7mMpN(8@nfX~aS3*aP2-I-s#S(w{z&Q|I~pEm%rGru zwPQ+RqHv&Qyy(5QjW+d+R|(GXWh6h>^}oD z(+ag4&ht78e9j51#%pA$d+05?hi6Z2XmbVn+%}GF9I1NE`_FA3Rje9Ou5?VF87PX= zDbq7mchvb75sWZ#y!-IAU5}%Tm1Lc0bBdOGnY1UPK^C4=Zmb;!0ncdSW!h`%BQ}6Q zN1|bKFalP5@n39XvAjcv_#+LGLg+;r-2`6lz*n#PMYuw}_gXQdRzf}x{$Rf{`2yM2 zuo9Ky!qxM~lZLJGr>}~MrV_!%J@uRQ(TRi-O0Y!k1rXWzlOc1A+k;Y1#SXfK%8;5b z^P6f;P`arNJP0!FL%D^LsR=lx_`p9REx!r7(xPZbQv4k)D{GoK1ds~5w@`3TWm8eA z4JTZ+Bvf|h0NU6BRoT;qD4u6ZD`Yh`7XrsK0TZSpft>PRQV*R9*Y*%P;kG!7N_X$n zCSXS?ZN6i9X@{h`0S_NOd3Vi1*2&HIVcglq{akRmpimhO((4b3P5?Y-s4T=jBWe4* zg7YR6cLpEGK=4`$X%*(RlocIdlO%6w3z!(zP8{Fu9H7LOTnyP z+o=2CWPFU9f2LLn2N`K`AhkSHP8bxpDmqZ>gwn2&cAl&0dzQyW7+G%l=lIryvc9AeE{; z7Yy{>0%;O7at?zgm!pvV6=~C#1og!9m`FGKtNg#ZEjG;iY=BzxG;b^!O4%449&#CS zIBBrh^@lv-r&fy6X{wAKH}}4-D7w1K`^KuM3XP4UepXNRk#OnIM3#-!Z^}z@4U8Ff;GbsJIf5TXX3`<@G(r)c>P_gue zdGwb?Pt?zv`F+1w36P>+j@198`|$Z=dd{d;I3)EnO?#KzuvbK=S9tW+@Tz@P$h&gB z4^ES*XU2^AFgsDq*29gzo>b62f{{<<>1}Q)CUu^SN9?-CZCV~&Z@yFUJW^+b7a4&I!l-q8M!ic(D z@V>w!yDYG%hq_;IOy`~H8;(?8gi5e!tthCevgOHcp+ ziT{pya@MypH_~^sv3H;qx3+b1ly!3a`4lm?GFJJ2zWJ{<^PQ+|jOs&L77ZYfpz zYOD1*%i(B$%Jcj7mC_f(f03q1f}ZH?s!DH!semxcnUk~6XUN4W ze1c_zrCD<$#exf(q>me&&MMNlNm~-3pm!U5d*iri<1qIe>X?-!I4NP$-+5QQnwl#KnrXw(mD&3dUU43=Ypp|(hI2{NgiMM&f6NlP^p80^ z*TtJ7-!+LKvXSGErQ-Fu-`GJX25p58+k=Priy6p~4u(5I}b! zCBpQiP&QuQ$WR9&ry#Q=7v$xVH;z|`@Qslz(%KBc+TlP&u}v}`!MZz$wOkQ~KH9Io z>X;X&iCvY%WxIDW-q+VqLmZd9?>^aomonX=7z8dD06@bJSx@YLDP?m9^B;--Q^=KS zuMU{1*xy{5b7L7!UIkUu5{iZ5nA!50{^ExS3Y_#UFJgqly??~jBrVjj0Z-e005X0I+|bg^Z>Q1k6xRC@wqyUyFIT;XyVe?I}A1O&4AZM2cUOxFFw3O z2QfW-XQW8ax$c9mgS4lBo+ukJlo_;6MevmA?EsV%1_VLICL7Wp-eub1(n7}Z1R)l0 z${`4YiZ_rS6y_@0si(LSlJ7l6;-Z0bnD~h8CX2;t-1L-idARr+M?hzcf<@`%U+z51 zit_pESLM#_cwJ;RLAj&(}h0CD5T7IUimxr-xP6q8mO9t#LF6Sd|wMyk$UUIkzYg`D=*||;Ef6nhX ztKuk)qmblB)U@mU*>g=U;xm|y0zs@RvF`)sPyw} zIF+SJPOVvUlq1`nG*d50{J?+8J(OkTN)LFbqfB5x&O+89q#5YGZ zwI;q@B%N%mD^?DvvwS1?A=Vqi+!03X*ItbYtZsb1wTgE_`ck^^NDZTU!wAB=d;`@% zU2{ByPDA(7?EMen@8rG?cksY1)+4v0IH? z-IJNAy-sl{5?}cq4zsJ##z%%IL12VM$SIeZ(%H{slgtb*RwqaVWeO zZB4kb_)$GBwbt!2&L+=IV{%N1vSk+qxX4Yc*M2ChS3>gECcnWMb}iovfHwhTBqMg~yS#eQZ@7gS2e z(Ws`pt0B%{);5G{^WG>i&(F1K06X)91u_p4_A(GG=GUq%jjcPB=#>=x>1hT<)^Ax5 zziKz4vr2qhBU>OV`>96Sk$RJ!Li5hfRcrod;s6ctCY;l#6`)1zm*AF(`Wz9#R4adI z71>Ir%CWM?1g59EruorwIcy)UGGN9m6@S(Y!r%5%x8njx@Lim39xC`n9x8Ip{%==Z zF7T&UUZ}G!fZ^@zO`xtJGoi2!TQSa9N5W7C*faY&p5WjnA))yG>b%5DAoVeD)mf<_ zcDVS7TF)*6uIv6SbM=ucXkf7dSEpCHgmsY_J^@Y08}ZSPPs9eQvpO-ve`P5l4r9c$L<|L9fe^-D z_V=B+aVHY&8L3m`hIh*=L0wR9EUF*Hz>G$_T9!NliiVE zl8i$6ox#$Tf~JuPN+lW=ya3Pt3DnIVKoSx63C9|W?Pr^}0fce|kJ%^M5()PEJD}^v z=mR@{pj!|aa?69$+kIQI62q7WWhzYTXa{RDfPXOHH4>4KifF2*5aH0*+u@3m>9)q^9|QlygCdA_I?pEYiJX?2lz3znyH>9q&T2kw zx3)&6Gf4Nkn6IogG;FR93^u^nVDH-`8>Z+QyEJ502~BpbU4RZHVUESSqxr>Zhl8`l zmj3b|V@;NL4^o5V zS}H*{W7JVF%elTi!bSV6cGbuZ@x-bIY5%%l%7YFOXb;4^KHJ^H;Dwl+IWxscm%gSc zsapZ%w!lD%WrhnI80>(vg<853T0u#)`;X+yJNaQmW4n}gXsThjQgw+F37fuN@4p2X zb%av5C&D_VXvA4{(g~NWCT(h^4$+;w9=w-CDfK%`(-2FO*qqJG`$ykVYu67=x+*ek!UZc zCNnGb(_E<7iUx%BLl>UK5n&EmR1z_ zk_w0j2fQ;!PQc(9M(W>6B2Ek@6l1_LYb2wy((p~X)w)|NEu)L5GkY?6A#!QGYGEOE z`2q2%R_5tcNRW8d(MT@SMYUse^`t;{&D~IUDc*Dj1 zAjo@i${E|hwES8;(U35fBZ(Xfwc+F^0?{AOuE<86D+&FQg&y8TSzX>X6NO^}swFPa zz?vNjH3cge%(L{+jm@H^Y-_g!PP>D~o}O&bAK$YcV7cFJ_Nm-{&M{Y6JdOMtyP>N!O@AYo?5ys*nNe6A&eR2F(dgWl1vt^2yEnO(*JQ1 zpNS=+X&Ww@x3BDdrR|cZ?Hvc7E99y6g(`74G|=|)m%;j#zHAL#A+`k{EI78%3}Bz2 zt|@LKCw%S`+v(XL#@`;4zrujWwrIz880K1o;7{q0$(R^ zmkDeoQgFh1FBwPWCDVi7d_R+`hTBx84JH-MXALWi(N29Lo6)XvGeEVWr)#d%Tpe6> zJweZLKi-#tE-=ET^$F9uO3OV!Gu};Wg4)nKJ}H}~^fbTcLket|qxXQ?;u~I5pXMGx zTcl7-%?>$MxkS!E?KXPoHwwxp+2rqds`%Sb2XklMJO;L-E?u)<)5uJ)`?3B1 zt!b`BdvPnkEq$Cn54U7^3Fui0t;GhoEpZ|knd zF$FwJ$QHJM34p#v@r_TP`}T^9u&F84+A(tbQKQ#kxR_h!F#~iEp=H+jSh5T{u4+EG z=KLX{+T{#7Jv`1-6OjMwK>@K>CmIGB+*UNN+r-0MieJ?td(2<9RT3+yRFQtlz}H}V z4uUh#jlVIbA!o7jk}jlGr*!)-f|;v9WmUTU=b+TDN~h*pIIQZL*8{aRbg&6>nNCv5 zB`d}1EYwKy1c}9~WkJGP1Y8`~)c6Oa<=pxn2zW6gW?$e*TKKC3hKCeW0Iiv(MRmOe%gHg>DZpRLfOVevD4L(;T$~t=D$k|I0w2 zf9yax#R?~~|H!Zf2>?Lof7mhns}%p$vKOXkIv@%EObAMt(5X|W34=hTucOejHfj|S zql+yEWr2WGgC;Pk<6M>5DWfX$%x^%DgmlfJK;pXD0gJWC z%eq>P(q{$gOfP%*dwP1$=_QkW#q+*Xy=?=u5?cO?3||ZlLZ2N0pC~t&E*!~nXe>vc zE6{KTC>1;xS_F>7qe$K1Z&Z_iO@J-nNT8hGD8LrPo)GxBErYM(uMW!IZb3$-V(Gr2 zwN&%MJP8f)$kN${SLz1Wn%H*M~`pi1UqE!TE?#-!!JWKIGdTLG%v3!z5XKlrc78A>bWr+f& zNj?EZJH}+V34e12&B0AAbdYsTSiMdIVOd<%UJ_&?80ThnOL}Qb=t2g2e)O2n7 zGK8gwv^uHvqhL=65{VL2)qE^?hPlGkXcHnVYtqz-u34lAg`!VFqe{_v>_utsZ?Y6( zCZt@lHVN*cMQNZ7WejHOBpqN?@?wf~`*vw+58tQ+qEve=aVA)UwZNmQ7+KRiJ)9Pc zJZtFmWJ|#&aRkQz%CI*2)FlV&6)7age&vK3^^Fm}A#qbJ#lwxnQO>~V$zk}uuG!H- z6EDNO5*saP$NL$l916s%quN%g7Pq?ayeTZE6^NT0Lz5G{LvzJcRkLDNdpy^9eL34iJ=W42e<@6bJ-YPFV6Q z4;lpeT78FfT|G&3T?5GBjzWwCv~$$b-+WU6i^*eN!&KyG zmysQfU5Lxquo+E%j&E%B$}#($UA)IpeV?5d2F)d@^0(zKs{Os?Dtja>vJ0DWUgQL( z2OnWr@nM)Ja*RL=Sxbt6%S&KxtL3TsK)FQc7+jbvZn%o6hb)9biR-0Bw?oM{Nlkpa zxGhqHHf zJANyI?+ETkzVkELmQIA3#8Xn|iL$lE3Y%A3*eiyh${Jj|h)wc7tJV^LIXAtV+vv$C zLDM*`etI>Lb2F7luKMxzLn7xzR}P$R*sk$9%c4ifx!uHPc4kCJNDvA zfLZfal7EK`(#QNn+LZaEM#@rw&c~+Uutm{ki?RNY+Re_jX*QeCz95{#*$|8r^K$j| z<{AL_>e2ZS7ee`98WZ>7)#<}9$QEW?GQj?5_{+O%e3o$fZc{>Bn%K21f_-K#@_f?$ zD{6sfK%q)gVu5ZsX;cQ;OMBYOjj6dJW=1GN1`KkPft#{qGgSHtFh45Gs-rs#P(3r7 z%C64KOsDSUxWwCPC>iif1^+BVh z#ji%`P!5>|L!5=o6=l9ZsCnr^(jIc0GwOVo2zeYrldf{@4q3%f~x*l?{yHS8=v2c%=dEV#z+)~9tJddGpj~;nnJN#ZyqPY3VJAbOUBV$^- zfTNt>Maka(qf5H^R!*V)NtVU_B+LG%U6PWUt+Ak)v7x20{r@Ljrl~m3hcrAbU=yei zTg!JCZEp?*3~Yl0Wp7uZNXxjuvlEfKX`Y{ZN<`k5;Qi|p_zmB4EE3imfH!fUJ)>nm zkS@G$!l(Ikb=zi|%}4oe>puA?HaB z(;3|*%@6bAmrzaQ%aI<+0OR^#HJII()owJq^tMU8Vk-?8WlPIO64iBg!Nr*MV2e~3 z?3BbtM(a}2^xS3U;RdTl`=z6HX&sOL%FlTW8_t$#z<>vV*GA{O(%>bF60qpPW(<%u z`H$4~rX@H{Hq{=zgJTymc6@yq< z%62#oER{~d?SWW#y+ydTem(kz3;P8@I|3A!3dTSj(BCow!s;u%JF2_S#m!UL(yXqZ zVVin2kIj;_8=dxg&J`FwCG$YZ5FTpD=H>`h0zZ&b1Al{bgJ6SXgJ=TDxO}xdwIVhB zn$K=nssJmryeODb<;^9#+=?LclYUhsUDg)RPdeXyH?4X+a6z3U*_>6hNGNt3 zW3+P45e@#OGmr}L)I0Tgo^s&Lm!Fu~A*yyLus&u>37(#!PFmpLXVh$DjW!^s0}khr z08{9u;Kdhi?ecDLOy*-PGS@#+W&<>>a8UNyyJg4z8TV&ItoW@E*^>3*yQKYEq52y^ zt~0iw!6_+p6Z>9|(CXVakk_!EW0XcCLRw@>&)5Cum<0bBYK44N7#`Znwa$*&{|MwAkYafR7EYY+fay2v8<9gQ{UO6 zEM*=_h`tKer|3Zdk+-tN4r$y;o#}PhUditoAa`P@woLu4TRWT4@f6rikAQs_t_{nP zdnc3!_#G5!xI5;)<2E--K7Vh4r2lC33a&!x93l}ooN(!@$HVPtRDx;Q=Kb(6P^aq&XgeD7$0;5!|1Mi!-@I&v6_%VQvUMMgQAKYL}g zXHWin$rr(1s$(Q;5oi}7+)8};FTBPMjA*@8L{jxgN99s<+#&08vr<~=qEgvHz;j^( zYsh%1C%md{((LlewB5nm1Pf`~tn9;Iw{&xrvTb_h_sEZHtk*=JW7hs%0ZjNh>t8B5p^BG(HL`6`S1}u*+EOC+Rqlo~**Ui0;?sl;(0F)Gk z60(KBXFN|`QWC5{%O)15dXvGnr(<>33(w(e7OYf@TPRCG-Z$y#dPBLbbc^gN`qfI4 z0mpTu&$U0qsN48%Tul6=U%+(?N7b({~XNf|8MH`ziyMFp^d$< zh?BLUqOGx^l(~bWpv@m!efxg|68?Du88LSYVdzrUtpe5Y>x!aD2&7Ew3y7VKegKyNQ#cNN#Tqh9+Qj55#{_OjFAdV>8q ztCe%R%jXp`ClZYPAtzA~rI3mtb&f7m5Q7oJ#Q3NRRItAzm_g7)y@fzaUo5Ez1NyYA z`49cWVxT3r`Q;z=FOqG0G6uPOsVk-y+JhewOFuR|UF4GVbn}u%mAZ|Fi8QeO;<}}M zc1YHAPIY_Lp(-{GFv=fq`Fqdx9ck+;H4(*?;1x@b;Qdwq8_SQLyC|=#&_8Z$bS$vx zDo(=;x)96Op?q?-d%#VQ0VX1pW!8;}VvCSLhFtr7)qmr&G)(2V=1N@0L|Xs&>7U4k zmE5TnMI}trE<3;Suwq@UiA*|N8ajPmG}Z|yt+v%@<1`tC*vOvVP>rC55IJQ~eLC-) zE>#%^C+)|H?JnQjw?!Ws6;=2-g7j@sTf-#TY+G0W?=dHYJU>KSE5Xk(3 zJ|tv^bU)=K46p$8IyldSA=_a%C`fo}ki_}o(<85wWXz7`5O{+-vs(7J|6MjGGtm)} zZe6_Q$bei61J9v^-EV&~4n#JjB}!$|(*^*?@A#*BgY?X&;@qOSk| zg#Y*T^3R&M@B;HhT73F$Wqh8RkYULLMnwSUE|`Hp3y+-7%(xdsBDll#akhmzJiBS`41Om1DtI5T)7FV2S4eoS|*Tv()NqnTC4)lNA(t97hg zS~PZiJh$+zVs3NC(zbMB7F*`ft}H&c*Vg7=PHJlluPS$XbZyJZDJPwKB+}LvTuyL$ zL~6?_D=(gPs;tVAS!Q&6lxoW=EI&VSZ>q`?Ty}7JRE=Qet6nJYL31oUS(C!?N zqjf4xDNV^ZbAN=?-qh>MJrhCXC|6BCQ$AaKaEQva&O5Lqf8}^vmEv8_u&s9oQg+UV z7(J7AT-GaLoqt48>{RZI@@_&qyCgYz0kAv6ez=lx-w<$dNjp;+YedFf{NvahbycNs z&9ZQrs1H`igmEULxG7xiBJcD(gN%61yttXzl7VoYPf4<6QwA%y2bB1Nr%iv0ujrME zfLk;mD)lJu?wW{$!?-wnx@+gYVp!&Js!z+!rC%mH*Prlk1FPs2j+2!aQY`(L{ct1e z?iyURGSDq`VphAOF6b4FLwDrlgynz3lV| zgE*zS&9u;b^2YMnVOVlK_sEU7RC+x>4c7LSo9k^`iO|=!&BHkEoG~}|5Cpy z$IH4%yFe@bNcm`qsIX|`_^63krMewynB$FF>|OoE$IE+mCOdGN4fBX9_z4?!XE63u zp8F*-=*_ywyWrKAPc>>A`eU+yfbS;X-CO+ZD01%24gq88%0CAEj=dgMzkaX>ZJ;TjS{4T+TPEu8A!# zGXkA!RBsGpE?9M8Hr|-;@@dD`&bTy7YW~;Rs44=j3c4rrp_=*4{;4~lvPT}o>j%-8 z1Hxt{{mUrHLvGp9ts=BHMP6pM$xUOTD+h6w{te^lE)j(nN@0%4x%ehmMttp^gVc!x zvSVS0gS0me@AeLADqSoBn8IEV$-$f;O?RkwEOW{XGWiJc&uA1k8s=Qg!_y&J#+O4! z?$Mp|({S2qJbY_9Hj6QVh=}M&?mbP_+n{^oPklt&i7j30wl>i&Ymrc-Z>l8!JSH$F4dil9DgJKsz%jG*7B5FH_-*@x5eNz;m--z+as19_-0UkX@0 zY25-1of5p&v(x7f_n@C0;=DGq)9VlS(Y^4=*XSQ!0s~+q9-%&FcQG%+5BFhHyoHI= zj$iD*u?db*zd(LHOAUxjOrk)9X5#m3Q?hT?U-A*&z zdFN(-;~lh8*u`)ub~C4C?nYZpryY#f}^wg!*|EO!{pg1Xup8|tc1oC^@_25=e z-Y-v5msdTP6^!3FIpu`&FD>x>QU0z=ue&4O`a1M89;Y+)@l-s#SAERJD8cZHqJdID z1}iJ?Jgi&uzd>N`D*Ng(W?uZXEF1s$EOpX#QqQE_$v%Ps9WU#UVXqj5URrz`J1+vE z=nz{1tZ3=zYFaw4#?-u|?B6Po($;9|sNhvmTJ%9Nc~smtaZ|6cw!pKs@`4G#kqRQ{@Kd(*Ho`S6pBgtlxlQ_ge|x<8&(D`uHHeqM|+kxRmE z6gfz(?x=9LSf#L`u8qPm(L*Nu9j)$loaarG94Ybl?A}B2JcascX^KWu>ynrUq~@vB zxRJ-b3DIp60r$@K!mm_QN9D2{FA(^BT1R=GT4#@%7sZWx8TCQkeoqJ|!29x?ETF)6 zvaOeos#pUZI>yg(Uv+800AwWklvxKte-w0)6PuX#A!m%xbEul?c?m8ij$P0&MVkm> zQK&EH3DsnVH<_e(x&($wIC^v_F})D_ZcQo>dYBBBM*7e{q>-!3*w9Im2hK~`;>d^GfQSSNIVAAO@Q(9#@evQcu<;z001jlUXR zQIUyzc?V^sbE1Zkt~Dnlz1ZVxz}8xufs!sFV517d61?KL;U{vPdhNB*p0XiXb1 zdiBy=b@Vd|f*L%}VsI7*YaEKun#6f|xhYsUpq~Y8)cY)bgUZ2u%3_s;@qoy(TWyl) z#K6MpsfwbP!Nj;8pVbLLhMZ!jgo-uIld^K|QWDp=jq0r$KFd_|hU#uo;l&i? z@g;*AR(XlMxX{wk($Ke>sz)1BbkP2_RVl?`1G#T%sG+j2jPx+^`w)s47>7c?wxTz~ zzQ2kDOSk@2<`?@x48v0G&5+oghf89pkKVZEk1KC*6yh{l3qrL{An!OHs$>oo6y-!> zPXx|M#>b{RwG~rJHpU}l8Y;eOb}QMn?7~73DyTI`PNd~&X_>gWNQuv5a+9JZx7R=M4aa?>>;hIEOQscJ{WqOYqsZ* zuw86J6>)oYNP1LRCYH)${J0r+PO13ZuK+9+FYX32MuV^RG&u!RxlGhlAF#MQ7KJGy zvcxKZDHgcDSja0N)4h7FsL&G15q`l^6}iecWE~w28CF_mH{JJ-GDO?P z%ksZd5^Z&{Q#e1HL)GnFiKO6wol)k)?ssi{Qj#)|7V$Ai4O2uV{yHy)qGT$Gi^$B7 zIzYt~BmpWk&6ct_)aZ;r&}YwjxyZQA4ost&lEg6XSd*p6!RAxQ3?{ctU+XN5HgnoH zstj9pjq)yvMc&PcX{u^!z+crusA2M^CVI8NM|4vBk(`Q_?XTk!HF#7^3izP=o!~%Y zLZIGX5nB}A4E_tyI#zk>ilTt5sQtCIw4EB7llp1|;JiOwTWexXYSMv|ywx={l-B*v z+K~lqX^|vfl^=&=xbBD#H}?2^!~Zq5rp=yov?`YIH%&55vwOjyqTRx6Z~g^@R16Sm z>gysxyaaThBNF0dp3svEUx)75QU)_))W_OBJv2$|Em)A6V3jcxW8qw;_?A?>PLRuj^M_LP~of+F^qV2ThJj~U2~t&edg z*lHS?FL{nE*MfHQ)xNsv^>s)?>}w9;g$OGMeQO>p-X}L};IoCS z*k&un$w;9+ODj%^SOq=Rx6D@;`!v0kJ?16gTC{GlS8ipGRaeXk)LMkc1@W#pP;GHK zrinvY7S)M_Sgf<%6MWTZ`DK$@rM!=E+S@LOPDi=ilH&^o{1&mXu~DMRvPVI>jgg&J zv5tjG$IXV=$MdnZN?jF@`RR8#wBku`Ko zt%>RIAvnvoI_7SKYCFQg+?u2thy7!>n^ESi4a}xZA~l6#U6WWS@>)^t3<_C zW5F|_oVK8aNxC@uPdGkO_EKN3D62k}^mVVc*U09{Y-AbAtbK^4x3(4m7^tTx=*(CO z|BtAS*wi?=xz_eU@}^@{yRuNsmI=wX7qWwOa+)?a%Os1TG$K}l2y7+(B5&Oauh(Bv zSt^$l%gEh- z*bEjN%+}Iam&Xmi7sKntp_LQrhVXk=lpuMDDDcQ&cj3W|n-=P}t*JeowuzS!bE3yb z!8!!!GM`nMbysYapczi_O$`uJytP(^ah2hd zep+>!c}qG_GetY=;11YD#(9H?Ph(_xwSDS9&6XQdX9enO%e@#{A|;1J*}UJ6 zD#g&YRlPFKTjTXtc>U$8rq|-|u)G&H`$1^*qr2H>>~lB!f_>>`U$L(l9D2acwzezF z&GsqX7z{ty4IWhfCC@K3R=ugFv-P-I;1v;Q# z+>ZHeDzYfHPS_j$se~3=BcYvyRAeNB-mw%3$Ogus^BF^KXNNtNNsf)Pq^2>unh{$c zLQ`GT!dtNk!-uSq*AZNCT}|`q!Pc!##);1 zqU;~U)`0D_k}%?96UIbaTVfGm#5h!GwvUK<&#kx=%`IHQWH9J}C0m?GLHv%VC=FG1 z9c`@0az;d-V3u^Ovn<=8i#WH47YVvZbc-Z`upD#1)}oz|@-~I9q$*R~B2{$IMVecr zi;iw_i10Azf+Ha7SnsN=;F3Unbw$=;a_S6$rQ`ZfpBXB=*mBmbqN2Cx<5tEgie<XU47HHWKOp74OtsJ8)M14^@equ;G^+9h%)(T(Uk}SVG z?S~Sq&7z$EWsbmGbU38bBkD$$mrkGEvf4=b7pRBmVwhVD7l+2A-kYK{w>nfUaI2%# z(S+ziw-_PNo^)*P&=l4Csw<>-p`s!0jKdSXsbxfVm>-vg(JVrDlo(AvhcRIIG)8oA zi?O1}EsDjsRxDM^EzW2TLT$+ITJsE~8z{2P3i(#DMFuW>pGB4QbBpopYqyvnCc2fw zlrE@_=AesI>GLDsXxL_z+fG)NWQZ(ZPfm+dWnq>sCb>n4n5>H_ZZVa}pC(FUF$WP2 zu3I@lOxMK>w>VtPbc=DQVis9FiKrlP z`WS)ZKdoO@NoCfmKz-JVGI|-1O*1I0@Ks`WsDLH6$ntrJT!+vIDv4W*$l8^;#bQxT zCa=ORyrR-As>Bkv@QI~vO{R;EEg*f>(fGjXaSp` zjhnRa0c0@-C5Wmj+bvd79ZDWPb?ti~_*|iEKzq`}tVw0TB{J$xZn0NQ7pS3Ei-21k zkI)wyNU4(9Nsz4{YLrYqTR2tFMi`yL0)#5a_a-Fi-E!NVY zEj;+x+$~NQ9=Gy}veYfk5NEpCH|$#?`Yc_nbBnV{)CrLbdTw!!IM=PrQ;u+p^T>a(gZyx9LIUA-D1{{)h|3McChv>I=DtmCZ~JgK@K0 z1S~#DS1F>6C1uE;rYV4s(EDXodn@TJ<_n?|L!3>?s->+jk%|37%~#ZuCCu_7aO)9< z{_vKpWt1%IP}h(-{$PDgO@KC9*wT}>hiJ766t-yHEiM+9;P@L6jfU=#<2%%9#PAlE z60;8}F7s11X(kaNi$gvKlI56?B|hW>x=a-gH|dAYVR`{b@2Ds2FN$$QKCM99n{bSk)4=SbndZ6N(yzMqT)#@qp+bMr{-P znzVdGlfkeiQIA;j56EA)xJE2xFw|zOedxa#xwZ`<;1Ib|T(yB^$Z@3~@_Unt`s7+s zN(%8hlzkI>#s;Oo6!6Qy3Kh5vge|Gb6@hF!wCI~PGHVc(M_qBfSm+ivh#Qd$HdV{K zwN=iDv6^lgWphePrlWEh6dhKr6_NqDxCwnCwY!SZP6IhuH;Y>s^opz~)ZrFoZxy%E zhfBg^Z}N|cUOCp6q2WNmO6S;X74MfNH%b>u?{o_W23B%*urDgh+r=Gjai^GxU6EDF zpcA8E=}}70o1AQ`Zs}xLO=hFos_g4l)6{ggxQo>6;85{$N8w2gWNQqmv2Atd5X4}3 zyZ9H@X#`KZiiYM}bg|VfwvoruKGDgoL+veOO%jN^Ni%FGCJm&ijCQd&Oa+IULngWqAj8Uza%c`>5rmb)1uxb}%`Qgv$;IcUxXXSO#PXwQ!qJhG<@GE< zeSi#+GN52CShmb-PlymU!{mh(3<{OSCVgRRHa7RI<0L`pLWWLMxW%KQ9F1F>B^iu# zy5NTYN5Ww&M!Y>K{^1r+(F9I*3j!Q;NI-=O z719hc<{wQCPAE%|G!s#ET)nTBW<@2ud^ABNcYa`e>y_k`RV*zlCmu4$ZOdqc*$ko^ z?V9b}2Ysti7y5!SnhW=WFwzbSiex@wSbD##*>c-Nx6r zMtFU0_6hqGnK>InMCpE7*^y@JLuF;vj*qos35tzG<`3=J(ZvgH@gjMJFVW1y%k=XK z{k%#)uhGxz^z#P&yh)?5W0ns`=Q1ly#u3n@4)rV?S6*wPuF0c+OT6tC@6f#ayKeEG zc;78P5Fg^$h-DuIvE(B(X$M?4c8ib2CvNd6Y5O8|mRo!#KG($;v8^^)dlTH^OYxO1 zzIKam1ctufp}&f)-$IkEcTfzE9_jbuN(MQWhL1Ge(cF3RLVqCBQdN&K2F%vPOvwvMLY>Rc8q?k zs690!bwduO4!J!^-pga8vF#&m7hxodyeKu76osYDUQ!!aLG#;UYQ(*)b}iA)4gp zK&>j(Y^p#HdhJtDRGv1`VlFvHIV={eMg8=LUKOg-#5;QqA-4SoV<}@iF>guGDM-&V z!=8;$N18L%W(+zzF*G&3Hnv6Y(2^%I&=<5n%W27^)*ISB6fsC+u^QHzXpB6eWztwE zZCWLbWmiJja%e1zPAxxv#vnhI{T|M0YAnnCExtq++b#n;YfPJB?3LTtcrRCW6P6WQ z#~0?=`^$U8p!2O^V4qfMWhk|@Kn#xf%j{41mQp#i6N1gmNh}}XYWpy#M$?uwCiA1H zot8S!ULhXQEPqI$wUOVb3xu|7s%11FddFbYC5iEtM{DmmZ}qrrWWm2Ci)_9;a$Bhj zg)l?zH)SyK&m4P*XeFsRGc7GCzZ=aUH9||uL}Y1Vvm{H=&Y}kGt1 zr^MozWTwTy6nvCHC>9HbM%|!xJ8@)FosIFHW@WLjP4DK+Dk+_~aQfWwGtq908$WL0 zggK?fv*i?g=jM$z6&xYIa{V`WPR+N{%Y&T6BFs2e9-&Xm?t5f3*C;L+o+~{&N+#=6 zXgE?b+7Rg6hy_&ofC@5|W^29ED`|vG4i~xnz6ygrQIu2^wKcaD{|xaxAqEY5py0=9 zplLMOthPSG!s%EnS(p}|Gzl*dS`@dogICdk__XYjs2~@WzFJz%Pl@(Y3(_<)xnvPb zcJw_!9BySe(xkg<`1~OU%?O1sWNk|RWb~4n9kGpCeoF`YBbPab$0n<2z3P1cTjXSP z(L$-+vGhp(HGQwi73y(N>NSga`%42}`Ee8mC;s(2;bSbjKz=GEr}Z8;gBX2lJX*SS z?p=ypemJ4RyOPFH)NxQ^`9jtwB8-r8Z{5zcZe4N_(GDrIi;5^TafiYmOP8i)N8dwF zMyn|dVQ~Ea8gGfSnWXXU`K_5=xz<}wUs4Dez#*|H|K=9nI`tEIa3eXw?pmMrGUNv| z4W(Y zPRp+k9oTG|zTk9!sAj=cAzBmw-`{>$VG{%-m*%o7GnEU`vi6@QIx%Qo75E7hI6jO`icUvWmeOt z7=rS2oW1X=>5~UynRSi+x%OyrPb?eUC(G{&kdH-ocJsYj;Z^fL5|ZEx$|Ks7tuGCP zpZ>O!>~G~uCpesw)=IkdHYIe@o63aPLp+%naR5KjZcL`i)^t>te(9K({6zr>fwc3) znbYUYSXeZFVM%H6^l3B3&MqmMI)35o`7_4PA{!OuBoY-%ZQ(A25N|IZ+1=-1<+{{_1Tz!6THO-VQ zpv16Ee9oUzNXJ)EPRlYlT|sjFx@c*cs8PpFL*=wVM!{)eM~l&W>e%$3EFBO`$x|8&xICBBg-} zZ%Ku{6hK}Bsl2eqytdkFR@nBCDCoA{v|{llM^_h}mGk5+o?3-qT9_B=q>17xwn_vY(Ftw-Z+o@tIczuI~UO9V|Ppy|`x z?QJ#k%rp7ZR!!QmUd!W4CygBKh;Og9N9Tkx3fos%?w}m4gTsYu%@##a8|{yVK{xg$ zdm8{W=)^u?9|B_^;giBX#^+Dir}F+Y_PM-%!M>E&uh`dk_0V=<%>SKC`yM9Y{SWL% z{QU`2>Og>s*X}vFyMfKm-3iK0P;=N0;5q6J5Ok;I?UUo{aM(Azv zxsRRFrxCI(KKHd#`Zhw2#phf*CASgsEI#MkDfx}i&*F1`JEeak46yh-(B|{NMi|tb z&o?8VZ$UoahJ451714svn#E@rZSz^n!ND}RiBeOh{tL(3vthMrPOX}v zD@H`E3LWXRYrPJ~oWN@?GHuAb2Iw^{na7PzDJbMQ+kt!dHW=0j!*|1>^F2IMcUg#7n|+9l~>45Kin!un8YZ48VV^i4&S6?Bg&lFsCx6Bq~V}l`%T}Y*%(S z62|d51-UR{7mVBmqc%cTGIx(k8C^<@$Zz69eumfyhdoFP8`B75@lw=8qeG7Wzwm*ycm3V1&KiZpIq z3$8R_Y=U0NJfT2KnNq0BF`$Qy0X^&!7{*PnUKi3LK@40&a6B1VLc&_ z^@0JcH;iO`poHZ}>SrP~dGQ{k8WMKy0sgMRk zl}x1z_OvtfSGwYzh+u1$(#`Jq8qBTOXqYN(TsqxJD>rQ~xN>rholuHN)18;oBVW#F z>S{V^3N{EjvLT3sp)iCELj(*D6G*tUJxcc$0_oA3Kn^E?%=`-lG71GU8U=C~3S=w_ zqzDBv4h1p+1u_u@G6@Azf&!V^E`jtsD1lJVds#hKI%55(4w#Dlokg9V-K^8QV9o{@ zl_unMXc|uwH-M4G8$hIq0u7(FO*BZ-G?ZHdB&P{VFs+0c4X`GS(}+*AE1;Z0=B!K; z)(h^u;IhOtt3+q(eFq$W9xPy8@O8gro>+i3CuJ_tvJ>XDAq4Es5okmD7HY{nsX$Me zUuej@Mp#}WO&1<5WJg{=EAfV7M^-}@=}-uEIEro-B(d3$&gMdIwg3v)(J+b~0~6UI zC}m|Zhn2$-tOBZ7B^<|=z{zYWoXx7?3KoD(tOjmjLAZm}!ERO$d)ab$fvtvD*~#z* zJ09L*C&2scL^KsA!S`$pQ`jjI2CKo1#=Wc38wIGLvF~jIVz&c`_wg;I>F_ahmT3mO z1wEzp(nzc!xG)vr(g%AeV4n>a6aHhvB?-Q_;gSwtMukgu6u2xPxE%T40+*v=g3Hn2 za5?6n;Bq>`{w;9nd(d#n zi2|2}1eZnsEpRD|2`-Dn;ZlB3aJd)ZvK!%YKf>h!gv&z+m&Xt;dl4>AAYA@|aCr*h z@=t`zGYFSw5iZXoTwXx9yo7Lh72)z4!sShb%UcMSw-GMyAY9%@xO|Fm`2gYaA;RS& zgv-YWmroc4(0>P9at|6Vc~Rg}L2&W@x4@+`Cb(3E!)3`q!R0H2%hw2(ZxJrvAzXey zxcq`}`3>RnJHllj!eu|qLB=0}Os-Nm9EY*h$r!ktt(b5H3T_jIEVp2!afgxuyU}sq zi$3oQ$of~6On3ub)VCB5ypOKl$4X}uTn4&KUH%rh)QI<@mM5buPeEC(JxEy&JZM=CiXzJZ$+G6ZR+h)L zUzTSfU)G^4&p}z9i?Tcqd43_v@)DHgr6|kGQI_jbmK#u(8&Q_mpe#3|EN?5fj4`q2j%942g z2+Hztl;s|jR!Rz}CsDksCLHgL4M-8fQ_!|FgV)xnUY4uN!a81z;P zp+Fr4qtwwbQ5^%N>R80lIK-Rxg3|>ZNdldKpUma=2gJ0FS7f;Bj>$>{YLXr`4daaal4aC8E z$JX57$o6R`a(mW^-1as_Zt#U;iM~L|kC2w=W5*J`=UAdQ982_4RLPIDCEo*UrFsL% zypvJbqcY?GMIjH)kR#oSuuo}#9?3j;bjqpGm;Y?`pN8!3Q%ci7FdI6_Gyzkfhh!`*M+F?l29#0C zXr#};p#3mpozjeTW9+4h`>_UEAf=_j)15_<(<2v2PN5{pS7Q^gFlBVg8LhLh+0vl; zFmzNOg-rEv=%wyKHtdCd>Ju{KtrT(nLV z9boZPeXf;m*m;pq0C+Em!W;7a>l4DQpvm+o{`aEfmSS z3C&1W=##gtv@$(PE3aWIuOCD!la$H!K)Db35QlHJqwLOXgtH!kbqMfW4@co^wHwZ#zY{JsAQ0meXI2}u{a@QD(VCX9f7$`l-N z2CQ}7UgNx7gKwuQ(`1;l-I36CrIa?A$U7+^c{g2|Atkw8IUL_|aKp{YO!hlsp$lBD z%u;5fyz<~IWe$e)dW1~pDM#4IInExz9rN?H!$uP2l{vY1dsV)NM}R1Sn&K*$95pHz z&Zxtp4w)U5`N{%GW=G{nWrifPqeEs#i_B(A^GDg>I@Km*I|}n^(iYZEph=c8j?2wx zglih$3aOD(ZHSeiyXh4Q>E!JenGNlJnM9F;E=n%*@Ss=T%5w5dQ#-r;j z8L9CEFnAIq@?_*x3UuKepeIj*T%Ha?xCe&wPB4;phDkgFX7jFaB+r6Hyc;axJ)xTS zf*Rf%PT+muES?AF@_cCE{orms(D96hJ9=)Uqvxufv>GSvTMw>BDePjZ%)5 zN`6ZVeW=ivom){ZSGQCydX z2X16ghyuC?Zep+jPNusC1`ol_g~B5cD7)a6jW8xz{i#3;iR)Gr61}@PVT!TwJW#jxpZ7?O(jAWix zV1&AOuhp?Tsgt{HOuA$TU11$`r5Un=RMAejPu4T~Pt@bl_Q0`e`UWTomA~I9{{Y^l zJV-we5&N}9c$g}Egi;>0Yj&K`$F{-ajj+d~S!MS&!V|evHrN4oD=sBLNycYYBK0*?|G-Cr;A0@3 zkHzs=1O>bpM(}a4fRBetJ^_Nf1jk+}tl^X46g~yc;8WpjJ`JwoGvQ`F2X5o@U>iRQ zcJL!$C!Y`Z@dfY@KN9}MPlf0CY49Rn3m@<^;X8gd{K(INKllYKj$gz){9@LHU&2Q4 z%h^P}p3UYPSsA~URr2fDY5WFu0l$%5#cyHP@jKW}{7&{Jzngu*cd%c0BiqM!DlUG% zlE5ETlKBHlDt}Npgg>NY@Q0Ob{)jS+KSm>uY0wFc%W&JDUf}rCXF#g$PoHg@83Rs1 zEipy<)4AYR$`taU)8I^Hu~Lp>^FF7AQ`cr!2#_0-k4;ieCv}KVD?zN)7uG4c&3J7P9K7#X78#L8oM0vn@gO%5uaA>AG@z zgBW_QdHHAn|9KZas7*c~f63X>g_#^B=aS?wc}K)5@{YQqN^;!xJ8C-2Q?#}RD=YBd z%IGAhQAqgcZz5rpvf8%A!=*C;u#}Z{fMG(D!Rj|=He>f?VG z#^vOB;^f$UVHi)xQqNq90YN%XH!i&z5BcZEoewh__CJDdVSKugu3-+5e-~@& z<7h}k(!K|U{{TAw5#0PINajC75B>}E=l{Xc`YTN3zrl3=JIvyLz&ySm7NFx$DHtpf zs$(VlL$-1}A|nnX+Of(Bh-*{!D8#R&4!@S#{8C_v#FC1*FIP@fPLf zZ|s6MJshLNw-l`YfFu6L$}MrXXMX(mgTHV|ZMI&(9m-3v_UI!IzScU#ZAPIM*ui zwd_ZsJ0hpYEgS!)5x(67<;1P{Xx#eF;TGn9yBU*=W?V^-TuBJyNFXG~o1ia%=mjp(8TyLgsa5}_*oprR56AniE*r>n8>okBsN%-u<>Ft zn<}QTIbteXD5kMv#dNkp9L}y5bJ=ZT9@`_1U{8tpO1wBm@rZ>=Z&9iA6^oTzQLYRS z70M9dRfSa?M_Lpc-T*c|v)Ig6A7 z8_$@s4k<9$RQOdn8&!e})nk@&4!+e{M>a?~mrM@JVttkKl=ErINIB9mTt|k~3EtW2 z7w@F{MWs7axd7jCsXAzO#_HKASer%hg-DY{$wf$s)#}AgtCu*fUTU>!DI2@q%fdsd zFm-;pvfg&!U$7^*MysPT(5d=iGYqA0>Wk8R1UJP~f^TVn&ZPu8j*hR0oTyr>u!j9l zs$1mt&Q2IxhUTl}W>T>VOmRFUi4(vhPDGrY23^D&=qgTzp5he97N+yy_{iWCPOr~V7IabT#5|140?#mp`W-C28t_iux)_hVk36`N|=Iq4i{HD6LcO( zlT6?k&!yVn9%)0B27v`tjx3)jnLtt1EQblREGA$*QG%*u0LC@Q9jb(y_ZGs>` zic)J^QHqcri_n#VH%kpE&DJBhu$r1&i9$#~@!WhK*=aZS!%aOLi$A&qxJ-I6PRb8raIFxX^2{48Znd)uwMLY{96{g`}P=D$#U>)dM@ z8z9Le?4hGcsgr5c@JCiE?{>j|$XCJJU-v*VO#-S2nBO*1hKBu{_!dVta-u}Mh(qTk z)E}=wf_NQu$Xn1sybbB%J?JLh$D#Bg%$#b+q5&*2F1CAxZFflqu3%f)xF zQv43*i2bl$L*g|7S81A~XQsk5<$4@q0**je?FPIf&R(P3h%y0rcat1MbVo8#ARmrH zyHJj@7!H+60|i5lyJx+-*?D&}a$KRm9b-%2SA3?~(S5mlK+oF?zvn5*T$ zd@T>Ew0=;l^@o$R0dT4|2rkkF!)9#=+@uYK$Fu@?LK_BOX~W@LEc>%o=#0{-aHDb? z%GU)|aJI4q?@Sm1k4nW!N^YMsHuovF+cMv$+#$ziGJL7riQ`a3FXUt8F0#|8i^m?zhEGzk(*U3N@ubC;8Ki7W%RFfSW{j z_9$~9H_`Cw&7>^ZE=S$dIk_ZUG)|#BH}7Oh!~X4C9llGWq>TnmI}CY022!-Kkfs&E z5Um)7Y2#qLHXdeb6JU-u36^RluuPlmEXR$9Udj%edj(3PEkA8=berKxJ zuk5sSoL|``19c4pse2?J-LP1>R|e`SaFnte?=(omKz4)~TJfW@KIqn=_>9xx}u=Cel_y(~|ii1>C95ZB=K@ zv0j|Y;+^WZ*eyg~CbgoW9CFM&1tOItv=U3&eNFNsE77q8`h4plFz#wu8; zErH`TADpi(gUd8OT&D$KyB37qS{>}wR>E7_@$k8JGOEcF;2Z5k_)$BFC1`6{rgjP& zqOE0PwRLQqb~>A&ox!GPXR-?IEEd$xW-GOG*&6M9cA0ho+o)a0_Gp)|=e0}O+uA1f zp0<&Fpk2v6)vjV+X;-sfwQCfYcCC`AU9WW1Zc;L|2Bk>5St-?SQI6DZR~Bh^E9Kf9 zidVZ+S*qQo1hlQn@!B@!Y;A`V@FhYg`$S}_b0TsUs-NC+&HhKowtQli;P}MfK)#%_ zo|Bgz;8T!S>khnP8tnmR;g2dj?WtA7E>=^v#tFILs=c6opElBC|`N z^JkY(yja5W%2Dw+q2D>m{mKKV3I{3+vCg2}NKH+mNCz0lcX?Ad>RD$6d6-_82Y_rxa~&+9u` z&VwGe$7p1^8zDWFv1=-{D%E&8{qIlJsEAG;dH-WxC#wpQ!4A<=CYATw{)Wd z1?XN(LYT9oU_EN47&hgzeI^*lyj! z?$T*Rqv{}^zL$`{EkulACnMmKbh^UpG?NLUb*739M;)y zKe9_0Y^^>05znr3?tCy+c@(ven@z`n<1u9q#IZwRjdaFL)(b9C_A2Lt&a&Y`AE zCmS>^PX<3?GbHS0gXgERAv@X72N^kXVgpp<*|9zA*~khCIWG`tBF_$&CVF^Ynh1NZ zHnL$f$+MSKO_u^3r^DNrH&`n$blHw%99n7{`mSW?3)?jG`Fm(j6C1G+R;%PKhG?nI z$xGuQQgZWmvXL~!zXwWD(Xdf_A>AXwGBmQ$jqI=-QbPTxZB%htq~gZ3xn*xHM}jY) z;nW8%h4na`OqRq_o#Avj^a9XF0M|!Dl3om*^l{Kjp9p>Q$a*ckeGb&x*E6UIy3bi{W~`9PZRB-~oLpJgHa1%lb0- zOs|En^&tGL*D+IH!IJcoSgO8?rRl3#C;fQVRX+iT-HEJFuV!QPQ{`~k3`uaTJzO-5 zH$rnKV=%%vLr#E=f|<(mXnHwZ6EeSWeaQTxHnhzz>tw&xS=Z1OL6&B})!;Ljq5NBU z0n0uOb+TqopmR2ij#XY%UP8zxv4P6Va*?qQ>qLL)Gi<5MdiQFJ?Fz5iP@Zf$tF{wDPf~!Dh(A3jYcH%?#^6XDw~GwrimsOxUK!9taP4@&aQ;c=?C0U z`WB!fcJL^-5bI|k9?yhC{cK3n&w;M`xzJNT560`~!%Y1mI6}V^7U`FPSHB#V>g!>- zeg&MSUkT^wSHrdXHE@%@32xJ`g~#>l;5mIWyoE8@`}&>mk$ww&qTdQ%>bJpn`t7h! zzk|i=cS)>WfevO*=x3|u@epsTX0MGu0ZT(G32++>wlVp*gUK&L2!rpOO{kyP4kH~O zOmNVYpp+@EW4{w&zubMLV1)Z0xp=LjKL{>;FSzxGAzpt3lJ!R+ zU4I<9>wBQT{)DroE7|6eit0V|6`FYH?tF!2fa4ktz`noT;%h7K*ge>6_khO4tT5_l zxBvS&`ZFN(m%z}U#U4C|)ISd?`U}uSe-U!^m&3lF6QVpN?E5)Ql;=5==ON|qwxImI zD7>G8|K^5KKkp!T|1R?WL!|sm{+N9 zdmZEor2Yu}H{V(aVGC?qHI42w?Kz}%E!54C2T(UhNp&L^Mz-YUp`vMIM`Q9a$fxYm zWZr{Z{DlYF{k#+cs2_%g@59u8LQegRoca|G(SJv)wIBMU_=gw_4mA`QZ>TWS(BLS1 zzu1U_rA8tIjC80qk|AiMz)B+(PBc2eIwK9%8y%hb!6DA5KhzoZeK0KlP>#(wm}f7s zth6_(a5&M~Asw_gs?gm16%O~WP<+Zqa(?~-e=nWpD5A-s!VW5!%^k{ZzE;L(r~| zcDTO;*`C+ZtcUi9hOD|Td~_^5(CC;5Q!$VT+0g>j0$@x6w=o&gjVaK{mX#XSuNxYwYJLB3~M7yK^$ogH%hP|oMcoQjGVZi}3;E8Xv;|<5L)Fd;!JAmpHh-f`!JnsLQ^G z6O5l=t?@IOnP1?1<5$>d{DwMgAMD2W4`cqlE)LJQOnAW+5AV4W;A>YR>~p2C1O#82 zs{`xfif8#Q4;$d>!U|m7*f>{rcDSntTjc7+0c?((^=J3G2D3e`LiVI0i zUE`Hx*90ZqHBss8nxyn`l_&+SY05~~Y~?W5bY-k-hBCo*xH82xQ<>+QrBu4+$lau8 zp@D4MomywVb;0@eTN6%z!<27PzZ$R@7Ag(Wft~@!NfRkxAuLwDlZ!yZ(QI0Q+1D^! zzSUqK^p$UQmVotrkGh#YmpIoB4PSuy$`3R*%&xNAb)#|h*mEV%+Lj=my=Ys41oo62 z5W3lkaGIpb#n!?JvS#`s(n>kY5y#5mdP%9l4u>5UEo>3oWYcme{M({{6~If%j})9L z>CW76XJ>9W*_llqiR$hrnQwxVZ;F#|Y?FObs+_%JtgS(`rM-Q&ZQHhO+qP|;vu)e9 zZQHhO+uy#q-RZuaba(no)=Da=q<*Z78t<%C^Bqs7Mk?iz*%6v@32>sp_}|r*(XG63 ze2h#@f;0FKuO7}{Ln~`I9ptJgyw6aoH_^ZILQd)=lqRXM)*`7POzM2MQyd$?<0r|P zZIL=ybfqw6rKvPV1y>laCR~2l{LrFA(t+x4 z@6l&Z1C&w$y{{8NFr)&jqm2@KQSJ{&iwEtZB2;IEF=_v1kaxj;+u?8TZ^cpRQuWWJ zI?2AL7%PrFZ#ocOI9@H8@BS;Nr7ZHfXaPPX*``kk~c;9;e}`_jmFCT*wzv+m3ykvc)gDw%D2A)BtLnV4J!BF0jgskS}J4V;k7LRQ9GABM09533({YQ zglR~+nd{W8R=Be=b<<)R8`r@kSl6{hp9_cODoWP%tc_F~8VfCt65=OYkp^LldqZi? z`3ppuL|JXPG)md0lSyWbPDg9fuAEiV8f1)0YZz6y-NyD6+zAc5-XAF!aI-oey;tjS zUD99y(y#+EfJ4P#3DP*p>0p)gGU`^teW1ly%Ncj}0cij&=@PY5(g8T8jF-$xySz*& zxU5{g4W^u(#;(rAKby~Mj<8*sXFU?LJ89n$Ekwf~mrIyv+?Ut`n_={@I0N2m5COW$ zu|*K&DGKpDqY!B+N9p7?kQ;^Njtdi-sYQ++`KCOPbuk9p00CDF)<3E;z9^xV> z<#P9gT^$3VQ2#6fuVcdbjyn^|7g86)ch5&+w z@PdZyf`;INhU9|z00a!s1P;H0+rMClih7?*vHIoJn5@L#0Y_&@D0B%U4S@3rQLhSR zmz9cE2?>ds^SKJk+&h*V7r9v+6-+p2K^qme73L&bo)mG%$x#kHszlP8;f^DGQ+s zjOnZ)v_Z1qL56|-?4e8}LirqkoM{R83zd}?c=X^~JXp2FC@0fqVq zp3_o?ZCO`?g>Hv7Q z?Fa4Fd2ToZSKiNsPWWwBy5N{esa>ZB#n+|I$h6}l>g~;5#mT730uPc41KlC8)RTIk z2Wi&%%}euAb9!uuv}Uts4(BGc2PMI5+>29pPc8kbs{$PWnC6le%T#GoIf-1WHeGrwUvDiiVe^@_Bh zw2A^E)QKF&KccP{PN_6j{uud)1yXMzB8X1h!$PsmNMLNe z?vfE3;034Ylx(7KlC1HWa1w>}6PHe!1|^>v)+qt$59HW5L89;h^8Ii$2s+UU&bAic z0Y%90cgoG(S8Y1bV7o-G$Q+_8beSC6N(4|2ZhycZXzkNB5WKj0jjjW?kGp2!BrvSE zLzZs=(am_lLo#A=-@TB_Cr5I(H-X90y$rJ6h_y|4fmhOg{Qpdz>{59DWWEu0Mn830 zb%(oq!|}4{h0IK{?Ls}syv}<=bu;Y*=S;b!T6RywPrZ=9w(v7TyDp!Fa9QQA2t?bU z;MFeWK|Il`$;gwpb0H>gq+UcZzRCE8?1Qt*$Rwsp)%v(<+IZ;5#7o{XIlIR%A$NXE zr-gm5{eJ5$NYjhh$z>FjlF$xEviFx@r=^(<7A!ARG}{txe4{>d{nCqzH&!q0Hs*F5 z9o_yFM^6*+b9~R6)(4CsujtP8lhi-kvSn>_@;cilGabq$Eyf4Isp;a;rUb3c@)9vwZ5^i6e9+v3AQwZEfUo;C8){ zL-smri*Lt5epVF8x;9Qn#>U2`__AAU_Uj+yjaVq#Uw@&=oVn$1f5;oYS-$DOd3LtR zsECju2Z>r{T8!ujDZhgQ&Rz-!3wu{>&P==&1ln*ekm~RYXtqIZoCzo z?FcLoy+IhioBiN^IS0^NtIQ`hgynfWuDYv=f7j6$F5HFXI7>qCqFmOkXSbbS%l{Nx z#VHc<&g_*%B98X>kqeEa{wndL#%$K@6dD!lpW!XaBHG^FCV>!*5TuZAm~(m>HLYVf zX|?IPIJ!jFb{Fn5!ro@(wkpi^AQhUH>!jDLlc}Df9z~Prm5y!li6W^_sY(g@AAwG~yVS0$feg9~#48lrB+LvLBkcGw% zs9czsK91Kr(0M-!MUg~7JSlVBmR340si=_TBAa+jVE}1m)HbbU*%;=-SJp`f_@|`e z0tUP7ObC}$8CRqaJTVqcKoIdA1c@LYkXS$-6V#CYACMoV17y*H>IUr4 zoPeQ1t*XH%AgEx1n2b0Pz6ig$0zC>MU`fDN!2*V9Z5>XQ;@Mi)%U0n{j8BKx%l7Y( z$7|Yc`kKSd>io;N91(XiXc?H1EV&GJ*4B$O;`CjN4ARQ5x~s-HZcWw;5!)&1-?DqjEs+&(>M^@PF9JNiK~@zq4$j*EN*uw`7D z7M*wx;1F_&VLGt~4GozRpNNh#EL}u zWXhhct{+AxqRQq!k4{LJPOzuXMElAo^IbfBmKqVFngOU!q0VW(KHTpz`|=Wp?3}(G zci`$(3nZy}7h0GR3hMQc8Pe_v273JX?MbIAErOqr%4>Xz+ zpT*t|3znNa(QPTM*0k+I)b)|7#c>JF@tau++Y(w{;gi;Y_O=AtyAJ0xxOp`1!0uzS zGfq4w?eE^P{RG)aIgS>|>=`^xrPoTOK0L05S0pps94FLUk&@(&EY53(saDZj#U;>^Qb*N0*#nM%`6$TZ{<(;?Uyav{9E0Y zI7J1mQ^4584dxuIWjEl9IH9`psOrX?3TH!!s~YEg z0?7>o_TSYBWb`leKj$1QqDK=bO11n^DC`FNuw6Ll-`lF7$5KYk$ZnS@DTT`BIvI@* z;ai7C<2~kzEgE0fqGk!0X0C(z+iB&)6CMWId9W@Wn?(K} zBmZ!`zJ(j0TJVf1R7tv>_UsN*8Mq%)X;vXtQhh6x($bv6Vs8&Jp4HoKr6i8IkrE7p z3aurqh>!X~Q2oNL$@yW(lbF>My9%G;RVCij3Y+3hNz0aMpCu$D)4jdppEJCr<9+S! z5hOfE4u#?uP4CU)*JW@e<^kbaJ}UcG(OFOALwc5U!~fnAL?D0Z3+0buO#5)t1fu>R z5nLygeUpGa^w0+jNOW()8Jj+T^=yd#x#8ju33S5EUYEF88nm-0A)@K~+a$2$ z42{5ptnhW^dyOU(aDr7ne)%9`A3&M%!5K=Zv~~2k>ONXgW|Dgu7fGItBhX`0!(#uaNnGQ8RE%Gr&? z^%;Y#`>OHvU}_f?BJ-X(b3k$8)JFc$UedF>kcM5zPPQ!xk8GOO9mc}iCnaB7Khc>K~(bR;*2YLsxZ1>B%k}hAUmQ^4dqaS8$!v0`*N|NP(YTei)Ij% z9V)65`P42f-mEOn>PPDHr}=zrwYo&Vq#LQj1w+%Bqy15*wX|eEJUh6;le7BF>o%&I zc$w+clqZ4ol#+63NxA9-WL(RyR94s8PZU}K<=mE{Y6PD}M+Gv@I#7|xx-{wvyduq2 z?MK8S%fuUemITBt2VNWfkAiWLsJJnITe=NOJ1KQN?G^1-q_&DlF<4S@WVtd9x=Ii< zaFUIIrT?6uiVXz2AY3WOqe#!y@^ff=kDcG(B^%Oy_JtVLD#f5}v5iw7L~E7cZy+C< z>#Rq=RjS@^q<%4-{AOa}1Lu)OJQTJW<2%=MAUP_ce|3ijh zL(VkXP0d4?t7vf!{ZM~%PVE`lC4y;Aya`dqdC2a0NL(Hop%R^7bhSvBFUV3mEZq4} z=|YTQnyx=s7jniUBSG-UZMrXZ&(cJlS`9Soh{}i1?^8uje@l>2xKjBwucP_B%D_6_ zFVjq=$mN^g6nvFZ32rok4ZE0QmwqZ4d^GuZHK|&<(;!Q#6Oa*mtYby2jCV0W7teK8 z@g(Pr&s#SUh(&MyMLB8)>&F&_5rK`6L{RkKT-FFfuW?xSQ&@IlXYCyRXA*>5u6zE0 z5}f_JAaMxo56d#3Attjca5+lJ1m;&(k&IAVu zmTFlD^;xg|yu44Aq#YrG|y|-;dM6Ig&IFQ&o1B>WoZ9iJo4%s$P<+UWRJT z%XyWm4a<#hJ3Xm>@@T0?elf4VmO=<2^)NFafm5CE?iiyVB^5|;CNaRgdq>D=4G<+l)Gm-t{!9>R529rya%%bsD#wT*@FI_G zO*$ZFvyR~4+2r|-@)ybu#j{YI4!=LUY`Sy&Kl>uOT)AAhJr&c= zTR?B}ktWxDT(ZIf_GYG^ZHe!vzg}0^-9NAIFaXJW3aE~?g7uP)5r?c}IPo3CFXGGM zlEfFpFA}WtxA-t{7>}*#M)nQG9MuL|A4fJ|mly%nG#idQwjM0D+2#`9zL5f!TeZ-3 z0RkV^dBga1CU_Rm?Nq@K&8<;= zQg<&U58@B429{zoI$3=IJv26L@77Da4(IDe0onC)$#Ro~OH}qK@8ZGYq7tOyq!PvA z#S)OjBJrT{BJm_(Tt7#1Ci0;Z7KT=qQ}!^G$-1fkXuXZ@t^%xPVR+be`wCc3K|xPe zp)L8cHTvTAQmNm2yTK;_EK$Y&DTw!?L+?vNNXL$_!vhK#^n!7YWl&DP&7#Vr_N8mY z&EmE8gQUX@g#J759X4qaYe=TIY7<#Tj||a3GEl`>zk`dVou2xL4Gpd`P>=|W`ooHm z42vep6mEl5mw;vz;v+6<)WYC;67!4G5k==h8NS|=C9|K@zx0PIGry))?LuxF>Lqhx zVq`-)p0}314Wq{qLcb%xA;HGqTOt8xa@PYEBp~h_{t&|ZqDHufz++ShhQ+a`Mz8Qw zVyuskONz7}{pIN-z&9Ft^Wi8Y2Yj`roSMNN5BQ?)zn=~G)+-EiU7mdLJqb1D<_yJ} z?k6PVv(e-pCGa|ndf??E@j`Bk?j~`I+#`=E5z9e%Fe}ZaARiXRdIxhQ4Kut%WXiqe z3a{nd2Hb1Z=78CbO-s1S);q;#C0uHc7GrG9ATLuxjZfwL4?KsDS`v?w(ik?-FP`K4 zcYAXGZ?>lWFOQk-KQJ2>)&|bMyHUo($;rUXM99wC+QjIWyYb)AB~?mWvcKew@8T`j zje5zda8QNv%I2pK7|}aHqfi+MOwCsEUzl9+MjZ}g*4kxrDcqiudlhLg`T+2&{4i4z ze`_oh^BT{Ix7oC}nWL}o_q#8MJ_;4+CO=0E_VI%;jk^27P(El&W$TsU*pN=JOwWgG z^ngK#Sa;YT-WYdwWMASx-Pj_(e;}{;4nz``y{cnwAxQN9$u~$u+*=*|t}{9jB1TvA zq6yxdke8qtsX?D_dZ!Iop!ea-w2`7|CN0Pgb#s@CD_LYEQ0w4MM#bY&+UFTC^vc1z z{TC1CPqUB}H~>ST+=DSf>f0xOnl6DI4r4*5JngX^UX-^egF~4h&T|@iH~XkV-Eqsa zDYhg8$b@TI>gE0~1RCn^%;jDjj8`P*0HzAnt5Vg!a+Fm|&}Pj(k0nhLfPj^G%|V!< z1Yu(6p@qneE+0r^cTWV@;3p`ab7?5E>D)^|HKdPW)2i;3Z#CER%!PS7n1PF*zN7S3 zIu^ABj)Fd?om+LK7!1Y*R}`v701r)YRq72|tq5*h+PH4%Q=)lez!|u{XkX-R!lxnM zGdugtD>-}C5_&0mCrBOtHHuH^m4{Z*i$np831%4#{Kl!p3><$U3&4G>H`M{CEb*pM^6y}5I9&maG+%;}N!ZprJdX@Q~NnPv_Q7V7dt z#h_!fe9bdd^SX`d<_Si*MLFkv{hN;fb4+-~wAF=y`-q z7B1SIb*d@s&nS?u0Bm{*jg+&@iLY#!`4EV;BP58SBDMj(SWO9WW9 zk(is4^XV&Nh6v~0$&}}CP}jfdka7~$DdlTDHgn916>*+Kq+uOBAG77R`C2`r>~qLk z^xlgcJb!{{daztyl?a;XegK;Zq+Am9&qoV#jUbdb!{?2$g(v>w7J-bdECj65pr-&?!&t{r z}!v}zMdm~LGmt0*%?{+ zc=ZgYkE<$@4yK871ipfoR;863@ekQpQ!N%%=8bu6M^>5k)qW zL$!-0SGDJL`Fq~Zps_l#) z;KHdGxUJv!lnH3@FdAW8cU`E}vKcYw(ncp~S!%}FZqh+eCU1*2Vrj2V!5~jF15e?Z zc?x}?c?qORhaPWVb+sT+@={o%&VVj*OA$a`JCKzdr!{j*QIw}>yRje-3#X}M>sn4e zQh7@jqAH(htecXbP<4I7Wh=E`pzpwaN}16CdvPDKu`e0s(NYBK4Bw@Zy3X1!yoPG7 z?XZF#VB5Pzl?O|D!dL;}R+xK8BWZc7ao*$KRS7e-3!B2?N1+P% zyEw8mOV9``>;tdl4gcbY``lLxCrZdGUdjg%^zM9lA&KYEfO~DUS}FD}p87}st@(cnr-7&C8Z!j>E(1e1XURU}H^uOvoZgmtO2@(Jx6AJ)<^Z%zs z|EI>gG(3J0LpVRO^<$T|V1xX#;(;iHwzP(oXnt@ogRA%w{D`@LF|zRnOs#7W9a!*< zDeIP6ccq3RcjaP>`+iIO!c|RG&Cj*Z^;OTcZdW!Nf313Y*b>$PN-4s=jCML-j=Og5 zzI=9WUPwLe_~dwja{+%~3lKV-wUI;tU4v@TSI%2$BMG(j{Ac#T&bH8xAFmNN17{?h z8YZtG@*rA9BeYDmY%QK8oid^~2FQnKHC)6Xt_O8EdGUs=H)*DL7@h3EUkubey<9f& zL_u9a@BVS)B_2ZwTqc|d5q=wJJc~o%4%y;-NY%o}NIVST z8!4xgiF^^9@sLa{5Ks>>8LWR6A#z91Nj+pT>&@M%;OH&fbw;Bx^9R&1#?{78IWQ&F zD`GynQYA4OB}dm-k>7#oaDmZ8gKa8gWJy}3&NUBhnl8f?UAcEnQqRGkHyS}O@gyGZ zvv|~`G;@rpcX60MKWjY&xK?iCqDQX44NnRCG%(Sdk4!aQ5p(-Y$#UC16=xY2vDF!4 z4586AA-Q^{1^1gTdBXUamc)cdTQ(r6j?D7MBwaFk?PpThIC)OJdNNr|tY$zLe%Q3T zy(6Rv9C_Ao&DIi}w-58wy-)wXT{f63JRR zW-S2?L%suqwid>l!;f;=shw<}Fp^m9 zuq+4Y72w=JlI9#A-os31?$vK0)oAHWv=Z}S2w4eX#!n_s;Z;&JsX4}K8h1ERKvh9n zA`)QwAQr|%?toIz!;LDX$-pF|lBI;OPi?$DB|u4i?h=f6rXAnJaehgiQOPu9N-atg zZ+E}MdN@rzRpQ>w;{P+d2ux4VCAt}AGDLtg^%v+Fu3;8Q& z?(0WitslY;?qY45=wJ?cwcSB**ptMnXnc?f>>lKPT;FuGl!3xF+uw;M?k)KvED-gthKz)c32YPQng%$OSS4u`YbU{}Oj2`meYwl|2g z+hlixeW%y$Hvac!#sqk0Koj_)uJ1Qa59V?212cL$b;+{$J&hmw6M%2=j!5Gc@+~EU z7y1*OkKtCS({h*PQ+i|u<`bxA(O#MtN)>wTj+a9QYJ~*kReU{+n5MhFStHK3Va5TB zsLPfx(%Lm10l4NsZT>U;jJ|&8OkzoN{!5iOuX<7&F@|e{f~|f_n)Fp#1Z)x~E-)eb zQ5`u^8W9z^r_T;XJFEiw6SimNE(Ck`YtnKz#(x~kJuIo1WG~I&iGt{R5D9n8LHnLu ziJ#a8PJK>!i4gd!7RFj+zBfl?FXRpZSZx;x?AyNw^A*d}<#Vf-@3z?QujB(=XW8B& zQ5aK$0&kVa6>sLsr!hMrCX2aw`lSO7@WfB3p#bxTdjxF+t}p44q=x6KPMjV}=edhI znv)ZL9&W=hZPMCCS;R^s_K}$gnH%V1H$!-*mR*`Q#mHQR*NRYjj#2812R%)QS>cy^ zcMKUD@=2Sf-Ehce1&tl>J1a)jw)DN|Od02AygEC(u=_N&SRaT=@=0IQaI}rR97@)O z`pUAa!Na)=jt-60R!K0&8-ST>Dc8wvE=liVloE6~Y{oq!Fq`UOd&&=Is_SsP7cJVS z+*B^fNSjhONc8j@$eyyT^kTXpMBdPZb}SCueY;zjQn$h$^cQnw`Q@f91pzFO!t@yBH##tQvQRz$8hJ1HxdozpM7S(TB*!v>MkrUBqti$1_dm7=*SQ^Qa2`5(s8pIj9@L4g-QWM02s-zHCzjWcb)O2K~x z1+&pk?2MZ{nKl8RUBx&!*_}`1iWg=oe9^rG$Sy03xcqnIk3iO=qltXZfL8QKCzKut zZZTwXo)vGZN?lHM$rU1EB_#vG3{`;RjGX7N`0U;BwBhO{iOL-r-Xtd7@r-g|kL|7P zu~ZsdJT57BQK@V_o*;S8>k5G^mjW1EB$(MEbKsHbAKRWpwKev5YABRj4G_S$0?k_> z_C&{qNLiiu6KikXI1fp#!>}VxVr)$|kNB=;tQ0RZ18>kM3t96auzW(NjRHFUZCjrxxTus62IbzCtc+8Q&-3OnJkf_K_0o)vP0OJ;TS zu>x9g7VH_Ow)+yg5`?=!U|Nw&9%$zyW9aemL^7FO?Ash`l7iU^!gKY6Td-0=0~bX$ zJHiuVDvWBE+0>`s|zmSPxOdb zVwAoY?q=yB7+FbiAH9Yk@~~cV+dYw;QroWu4&f=m@9&D7#OQAV5riezCb9ZH)7Jyk zdrdwsafiY%L3HRqxQa>)Yvti_ha@mdk94*114@f?S0OY7Nc{cjocJdRwA^SFawSROvZ1uQm$SO8o)`cYYzRoA;}JuCgpPoaMhS7 zSL~f>Y=>RkF;*T>eSmd`Y*8|r2JcOV#T4aYD^x4U&o);Z{$( zB~o#13fjJ6W(Dzg-{K#YDP=UOgB*>=R@Lfyc#Ma&&69&UiEK()z-Br8*zT^RQ|m%f zT^LW)x8C*J&X5B?$!t1gyS=3ESZmG)-Y6x8z)vNJ&y|3l*X#mrd<>A>fJ}KHz+7g$ z!d^-w?0jXOY^^eQcecPSJ4%|*;}1b`QHTRIt^f(rqr~csmNWH(zdo@E@i`HaimUjX zdanHhEfEwlq#|o|FBuP3iU&kPYr1GB9bUjl+eKKVvBss5{N)+ML`lj`jTT`B?m*e$ zf<*+bvXn!x`E1clspTr>Ja z#<;~^mN;6-H`MB)#Y>hnR9F0vJ)eNkE zA@2WK<|e6HDIuw$`^tiAAgT%s{#Gg|bbF?YIJMX)H z8*~{q2U4nDA@wRq3o2ihPP@T`@ENVwO)Z3T$>JP0z30g>*ZieXr@ZI2DK*FPO;*R8 z71~9|9GEhpU1lHyM=~>!&85X!F}-UE$|~euY(X#>k8;z_rg`dDqK_TLTab`R z`pIZf7(r%QQ?tT#nA36sU8($|SfHYaASySXfYF_ zHc`CDYKmJ;TTdZfSo*b)uGvVjU4|bu7D0?#LG#3 zE8LV;T+4B;@VjSwGj`qnYe|Nm`o5s3EjUhR@?Nf!3buIY^g_&FqugaZR@NFs!=%{f z9J?W!Z){NCk2xLC92|JM{j6$Po6WP4X!mH-4FXa_dW!IkSTU?X%Wv3jfA^!$k<3|# zXcmnQ>ZObNlK*P}t|I-Z%Q3o!^Gxs?E4-}+PseWA|HsPz(UQM}JgTCsJo z^N5lFedH#zDM@pO+Q}rIjtzE2|J3CkVH$iBc|lm|=k+)!EWqC5o*QqqR2IE}oTUXZ zM+0v31Gplbqg5IFY8Cv%9}z;Y2IrB}BT;&Uah+DWXwCr^By(1Q()^W^h$`kJhWxD*GY>OI*zY;>**EdbXzW()%mo zx_=F0P@VI|`+V_^OC*;|%OK4ZhH*|madXab?~L|RX6}c#5JEXA2eu9ivW~k))7sC4 zpgIB76gFHxkPyIbSnIW)_H^;1_KYSqD%+)ys~j4-JBDG`L#2t@52Bw&ISb z^0W5yUoDnao^kRM1OUM8*J648pDk8RPRYc?R><7yKfN~VfA(4t5M-OGP|4OnWg1OC zMFw{PqCWpk%OWX9a4IR+EfD07s-Jwl_oW(_Vxp!==l8O#vzgOJw?$qvBvxUdV#Wj~ zj~TD*m;1~Ylb`pOl^p=>K{o{K5gQmcLU4WUfh}-9Uc;}KRl8E@@!YtNG zjDcUw7%?=e>SSkc(-%yngxE17W6HL48^+l*<}&5TVq#+3;P^Q_L1H?;S`;s2qY7vJ z*(E#Ed!8<|TuE>-S%@gO?uZ6Hw~48fsKFMACAnl=N}5?G$*d1QYf=c4>2_hmcGABq zaOZA5#&qb^6A;aKxkwnIZe9ASSNA@?1=jFXAT-Mc8MBlr6RlheH2wKWv_*+*0V@9q zG9^WvKxzx?(av#!@Qv?u=_^T{3$EtHxcC-j&2*`Q3K01%jmHbXBG%=>NM3i%I@JzwGx{p7Laos%#tM~6%_s^eqarY zH?PH1w3p`rQ(i^Z(NvW{*ax^@9NUA1#O^VHYoa!3cz)gu} zN#fB+7UR0Kw|K{vB>QKU?%PWt*roQyGVK}~)pUM!=1-p6C2*H&pEOH{o_-u_I&=cVji7dbBXm)I+SnJ4E!?V(a$hS|?7<2klO^gTiC9~Jgz;=0m$Z<46fjeOf0Cx)`RWqUd@|3|yAR-S?HwJ9jwaaSp6x1w4=ca1& zpqPpo^$PRtOA8@Edym77a;|3#kc|85-e1$>h(M); zsSc`C5Uy+BT`}R=^on)t7r?+@S%z)SNQLzbKj8;U3ePC1%^`0O?|C@-9Fp-75K#K( z6qsZW_~P&JaEJJ`^rLXYJ&fRL?6(yRKIHEwAu}f(7xd(6IDG!8rVNjP_zX?eY{C|I z)vFYCy}@b(qsKbf9B!LDx0u;v*1U^_or<5}@D1ndk9;p4&PU4}cj-4{I&P3)`}&9W zM%msKNbTtj+Nr{~BY{(Qb6=%R$G|2S891Yja8McJur|`VZ?t*KV5_`G)u-pc5}dK^ zeZluXZ1q2GX~}}97o2`=H6027fbRcdtIp=47LHEN|Fvkc@u%j9UG@ z*g>R?Q|*KgZ`PKyY6V9I>4X_$#sE)aXG(lO@44+W)4S57rF5l9v8z6cQS4!Y5CV#m zZBRYFTr<0|yZOrUalL=x>-hoH$6i9;Z$%68PeV6lDADH)hQA&u)sVmCuYOs-gU@Lq z3K4lzSFGKPM)AfC2v<34BE`F4;O-Chm+>1-RA9)o@N;G~(;SLaUAiaXQDO&bOOLw21v^s*BVm$QH!Elre zrlJ4*0W^IoS4wB2-dI~4|7cGtO4x*NJ!7Ks>pq^{QIi|kdB$_I^x^2}j?W_1j;-K# zo+4LeKA9zyC6-9i`a*~TRZO8Lby}yl4mfM09Zy(~=!N+!3uG|Cj5EKSw6TzFo+dOx zZsr(#C2ni2QzQO9s+>lFaU>Y!>_@WG<>tX&<6+w-b~MuJ7opyF0p@q(U6^QtZ!t`r^3-We>UFLB z$tEEt!l%hKdE&CcN~7iKEam-N?30Ryejk^|-`l$Eoe76+N2lyi<1lFpzP`hFY5@$= z26>pEOSH_fGCG6cPumAZ7Hfl3_T--FW}rNkArX^X9=o3Z-(#>8Wg; zr`OjUj^j-+>oeTzZ<;dd-}o*$M6-E(c*_%u>|M2w7f(GJ;GAtwfui zq9eCNC1Y-_w1uHYBEd<>YlO~{BTOp2tB}t{`_snb&t^)t5`3}hB=yZT13Up|I%gWM zW?nC^xNc*Tx`EsHLPQda?jZ!Bg`M2VV{| zOYJZS$0gCP=}|p!9xM(4Wqrm>@Em;7TcYElMcP}k#j#OyLP!5-fF+_ZaYE|I zQ3g)Apz57jp0dKWsz_47!uONEF*B>U&O%!zqZs#06XPeNFWf)j+asi?tK@K7`>GZx zLR>RO{xmFuPr%-z_guB3_k6{CNY9cxeZ+1-R3>K#_C@P!;u6`~qa>z&<}8~Oa4)b9 zR_1Elja<0vi6N|)c}1bF-_Rv*RT4S7N^V^r|I@YVf1JEm91$G4-~a$_$pHYU|BwD< z=Pdu*Z6fSqWB=bz++FHW-dbuNKhaO^$u$Y;GS(9^+Do!JTFLvZ5HfYvW>W#nvE+HM zG7pwi6a9*mQsFd3a5Y{6Vi1icTVPTvK%n8|G4qIqY**$62Ti6p7(sW~F?Ic`=+LYG zV9a0to@9Gfrj)Hej?ndJXE|MS?7aMZ?;Loq$nm_aA_+5-v$~saOZlUABIk*9G0T=< zGI&Pf#XKC?z1}hSavRJh=)SZ2^yl`loZh(Kkiono<9#*?C9F6T@Jthj2c)Y@W zy07Q{wa%ZJ!=U>@^2rvPVV*-fjre>ugYyY{@m|ZVxxG>Im|00by?Bfa`3_R+?u)U* ze5CpQVx{_0!}STl`x%ZggL$tO*kL-|fqCbZ`hnB^lJouS75X_UifYuKg?oDu1N&n8 z`CiS=4YI^)D^Z4@!UGo`V(!+SV!tF7g+wh8F3~mq# zCd3ZC#~cWy7`Q>j-T%*g&K$gkAf3A(`7jHYPFi|u&~Z#dV8RVM8Yc<{`M&J2*PRfK zBkG{6OqyCRRRVln{oB=D%+=HqKRIG3AO@(od^qVh;5$os9GjU{3Yg&T?Z8Ojl^gt6?iG^^JyP*Hgj7m`N(Yq<(cYW|>Vb>dDNIAt$~5 z2PA4qz^d>VHB1U}8)HJE>ssPWR{)KMrh%FkOVKJWgv2!ZC`q(XT~*!;(8MXxH{j0t0KSN+)Bk~N4+ zB$-;EErne2{E_ZYg1A%fA=uFaVl|2a05e8M(t4-GiBy0TmCdD2;+E!EkP26BBd^~; z(}kuB9&@*`h*upW#&@{nYk8`2d$R(=xY;z-hW^HeW_S4_KBJ8A`LZ+MkcFvrM(!7# z3WWwG)um}{5{`RV;#Eyq6#s!CVC@B97j za~@~yRkcwUXfr6LyFn^NM*WHQWtU0NQmsyg zn#d{Axzsf0SjNv*KfDm>b-6#%>q0dQXPCa12V(xZROV~OlA`ke7(kA>^Vg6k1|2&? z%}Sis1Wg@@`IN)~Lg)hM{CoY@jHy%5(*$sF!c&=LTGt1wNKwvMji)QaNB&W$h8~gf zh`SLv139@11Cf4#eicz@2Fl4Rak9pVmo(hPg@uY`nuRABCW$qs=gTx^^wV*2?|Um% zqFPDMQPA?6-F9L()V~1xBXiF$f!f2(UKN9fLhq{PZaj#AN=#CGTxNC zp+AE=g_H5b+Zv{$|Ee`v0^}A1Fg7Tgd8)j+B7uhLn2q$}cq^O2$$Ji-N+K|lP-FgET05)!xe z3>MV%fvp6Ih+=wI9zRW7hU9_hLdXwGDdFvK^j+vRS@@Rv+F-u@LL6*y7c&dOdh(7M z#<{{$8KN43whj0c&TK7o8*{@_`^~$)?UL#b^&lx3)#GJ4dn9t`x8Bqb2=Nb zZF0V*=STUsqkiGN>&d$2bNk`OfLa$MY<%6C`F?TdXDtw-nW(Jl^Ro)kP&Ne^e5J{d z;??`O<^?wKnlx|nehiiyb&8a6uWV~FT6wZEZ0+1zB~JB``%}{1%J2n8CSf*2UiRxWU-`6a%F`UN^*?6*3w;W<41l1DmdcVy@DVfGN)!B*M zN5$SvVIAkRK+hr)RiUe8`3@9iTm~^%4nx;0$HC6_!%z$tsRY(?l#N1LlXw@$8s0Xw zoXv=YeR>RrE`}Togr% z3GO%dUaFI_Iy;hrhbqYD8Kn#Erm-ehsP>W{ENA}3Jc!E$q!0wZVWU{)YL znVcb+DEVL0pzXQR>~KKbu{X5PZ&?qEF5&Hq&!Z0vTcij>eSr$IcqrZ z`;4xTsA6Xv@rXl~y7A^%t?{OkZ(5)!@Da*wLKS{}jTH53O3>EQq z+fTmbUoS*I2YQg)_lAR?iv?B_AjQ@fr^P=r0yJB==qwWE`CDpRvb(UKA zO_`X9ESNU18iaH;4q?Qcn`vL;I(bDBO<_5L++-9^61`ShRA+QeQCg~_T z;kN`iWR!a6gV30TSf7P>>}RKwrUW)V?))3*z-6$b(+DYn&L6NT<|Z{!6~S0~1y0FT zJiblx)WP8s_Duob1!z6WC4#IBo|U|sFK{SjK6<9bcc=*JQR{czmt`jso>?jX7C7c` z5B7lr)BXQ3_RcZB1zneL z+q`Xe-?nYrwr%^@ecQHe+qP}n)@>WNr=M?TzM1!*H2Q|j!)4%dr{kwlQq1KDVUbwvl`|ro7Kpw0_>cA2zG1wj->ir zr-Jw{JbN#spt=Z}H^G@VQm!;Xk@&grBeZ8&&Op>Selx{_N$!fEGxljrfC{@eOpOo9 zBika%qghne+%x93e|+W92~V~C4EM3?43U!vtD0@BvWdQLSYmW7G2&uW%7vkAjCRDU zg(UZK2~qPoesOh0w$)<%9NFPoOi(Kj1YbKU+%=F#D=aXlPWfYaLY}i_aZSa~tXk8M z_u*VN@Y1B@%Duz%iGTVd_(txYm{KXezO2xjuKV>TrE*j_QX^$tId|lIc24Wv1{#?c zTRC>%NC?ym(o0`TBvx!yIpo^*H8$2F_`0)i0)7w44w2dmc`Y@!fY!9tqqJw5yWL+P zdB{74VOxvmBYGI>alGJ;g>~UHF2wxoxGfWtb=OyLXo$D(q9oZRzQZ+NRhvn~x4OUv ziuA0k$v~oG9oQp~X(y{gM|0siF`ldFuL+j3)<{;0Bdm!=ORw(6KLpo8ojlJAKpgW+ zf`x?k^&@xku`!QQ%H)zMnTe^zew}LLVG}N1n~bSDvU>FHaPlx79J4&(xrizEyC}uK zUmf?gdC{*_f5k4H;Js09dcnDRsw=VwMRtDIZu52i$Y6u};3^t9F7IZ9^e9`1&JzgRDHpjnEm(1KZRQb}%hl2~FS#hEEu zBq^Si70lfrNW&1672Rs~z2eKf@s+&;=Y1gs?~R^m`2@rqmu3k4v*9EQ|3mvspp0s_ zimF#h$y4yZ2hDjaMN`9V7}<3gS+_h2)DFe3d4swOU`vgd6Eo4QF@dRxDy3uBOAax+$g2_A=Ak-%ZaylKM|Df9@i% zSD)yCKh!Cjdc}kvaD&fmps(se@K)OK)Y^GfTd2$3r_Xhrv7i5t+H?*B8m$B?V-qPbL5Z-lX}%L>LV_V^oC#LfVWP&oc`utBl{79fTX zoWw5o1d?b296lcID2HuIKBOu`M0_4P1;Jed>IH3*)OHn0TSaK>s)`SU-TK~(rz$XD zOL%4P&$2y?%g;eo)^!I$=#^3fRC$VEE*juc0~0FL==du>tqVT1Gt4V)9}qspUH+*{ z7xeYry4JSz_1g=}HLl)3f0K$YquwdFEiOG(!)v6RL$j=77Ok2SH#=o}@SZb&zfJ00 zHR-=*M*^zzG=+H!5(C;+S|aE+$X6~9+4kmM5v^7bE+SKIAnB5LT_`gIKMGivYeX;c zl~%I*^yt!!o2x)O&Mpp@@^^xzr^3veTh_VFbs}kP#Oa^9?*!3h7q(}Eoa+_Yf#_q4nC&A=m35cmM zIGfwRU)dN@X*|E}?ohfa`BjfVyP`2#D&xg8^g>9i&Y?oF(nd?EhyfC|Y?_4n+}Fx0KeFa&Y)}n4ebC-77h7ZyyipK!q9-dq841^C8SkQNxaGF?X0`zru`IVkFz;SPijHO~ytTP)2QV zotb?3LSPwnP{Vi_TUx<3oMN9!JJj4{q$kt0<<#;V$EPq{N_CjP-mPQIySj@p?l%&Y znPA_paHTU~Lh60y8|>QBj6?~!jL~Jxnv8AcAHFe=SFeF${fI*L=<IbURF45s;M72Dd?7q=RqudfDGS<9Hu9#ga)P{LL_`(%CcM9ZBcr1u5UQp zl8GDKYwlr6ji{|Vfpvhi(j~LAMg6)VONlsM$Z%Xv+G$ed_#>4L_EzcIu1w?@8?@0z+TXp&J!ks-Dcre>cj+^Y8B04f&}CM5nO^M${$ zHb7Iy+onpHPLj(82-lfcJ_gAEAgvxJda5Ze zm0#d)!U0UYU24p@DlU#|Re{xHBot0v0N;@3=T(qQdtA0&IwIw_2aFP$izLJe z{ar_sFVbJ>CXG{kh~cZgJ|+pYPKj@RE51VHb5k56tuHjcFi*Mp1j7@rJR*cMMzBmA z!I?D%Xw=$#!yd*C2X^2f06G0(+*Xt7tC5==;mS3X8IvyW^#9c!pFQV1N`BZ5=s!g< zbpM?_%GsNkN}7n9+L<~T{t&+YS1F8&wmga;ChsDwZL2mNco$(om{F$+`cIyg6aocJ zX%9Hy&XLl%)8w3OllG4M)m&x>=XdzsIHYgV-BgQEeue>Cdisr9&qH?hc5ZKv4=6)4 zI6S3^F#}6boGf8Dbn{WdkM<*`3_PPuxv7@N^$0^D<5YkqaJQb5wbP*PhF-lD2=6eS z^(+%k&)+ncBk*6BZtYfY@bWNISuyAtCB~|gCmlEI%T7Jh$}X1lNP;ia_vuwqv-i64 zeH0yf>rGZ_<-eyhblLuHXBe+&7}s=cI$3^jq&DkpE9j~8^#$uRe1enCk6>jpDAyg1eq6* z7%n1~dH$0^JoiTpZ$_Q8DF6*-Akp>wyAZrI7DGx-8o_qk7C`#8q~?r}Wyo7z;RV|%MxM9I;l~hz{zO<89Z@^J8G}pZ zx>Oy!P$ob&st6Q><9H)nqMUtALv|fe43e zt}W?~WAR{D(W;v(Qj3E&xV=G&f_twNT~#cdV4e!w@s~*q4V)!4jYX>DL=A5trdUFX z%2mCh#g@~~v`NbizO(A5HSNU$`@YIeR^PSR9bozC7+G2S_ncghp<-u^sbyDEwhP`M zO}zb9^Czo+3Ki~4i$!(~_Hh9n9`0hTYBLj~dogy?$SKBC05_p#Q1zQA);Ky5Jgr4C zIRCRr)*&_QN3dS-#WeP718tRr2G(!?f45_HgO_>>N zQ7g;|mu`To>a-Lr$Qv_ZCti0^W3`2d!a->k5w02au1P zN&P(m?l19SUa!JEZM5C}KWO}WJzj&LXvFtK7!HbB$NK&=X6}t?Z1?Ve+s5}nxY-=Y z{_K?C1`4cx;O;I4F_^U{`amv5!dpSMPUS)jr)dti>X_WBy48o+?yBKcoio9^-ae=GT)iih6jXbJ z?SgOPWa*zxi$C8@8X@*dJs*&EMInu<-%^3;9jXgM)e`@f4!V@f`8x_6WAEgi-hgy! z;)(V<+aqGm_13`DbQShuw{8r8G=CZ1$L1Z~Wr+UeQU`(8Sj9T8GZ z&M~FI;vHHj_5#9aXWtQVtrr*GkNHv#3ImzMA!sD_ouIMfeN9Fu)BhK3=BoETV=mMbCeroSJUcq%A;1J>e3qtIv^a@J%sMy##p_ zF2=B>pZW{`2OlN^UhJ3aD)}ui zpN=_+BKyD!8YU-78{aj*jBexppIw_pwtr&i{s1yt$)uuP+0BgR$WmF@CBndf zoALFR?tK2=wQ2u}J{WzL{-OO8Q8fLiQ2e*i$NxV2e?pM|6@94aD&weP`phKMN$LoR zz-VC=p<46#0r{x*Bnjn1qb@1XJUK&KYmk|&PLBxs2CVKhVeEJYGIJRR8k=W5uMiaT znK$Ylu(?1DuK&>IZy9Syfca8tRdc>Rv6z=XlBC}^9*OMy<6#tRmB8*Ni*B+i|L3s)zQ}sKUsd{s!dOT z>9JXLG)>%@(W*(?Yl;(dtu@nehewT2lsl}LWNKWqyaDrA%ADQjhO^7^NENATHCx~8 zZjrRzKE+#7w*@M^LWE|0bNij;&F_hK_7sRN{)_H5�|9MM5zn*!w0RV`OqAzM6Za zg)~l5KPq2_f=2%Nm^jm|Qowfd*+cJ6o@5Y$Z>2FJ_MDZ>glJ5M?$sSx%VpmQBfY4Iu0qLfTYC_(6c|AQT-e zTnKC1JVWJ2NZ~%6Tlsnr%eCbgAvJ>=G7A;oa@=5B2&IfQ04KNnRc8?6RcTOt?s%MA z1T!VLaYZt!jr`;GHuO9fqs~U(6nR)b8>GS3tCdZQMRPK(H%{PG^Mr69bS;Kmab>wC4Io zln{N&EEqZ1_eABsikv^ZXp?&u>J%|%ueX(UTJC@+mUUMvESGQ^ZVecm9BsdEylSuu zn%%jgELh(Xb}<*n7vhy(KGoIV=bCU)hXzbZa}EfEyipdD_QT93IG}Cd3ieLXdQNuk zi--Nq`JMBHw=GNZuD2(oZ7XD5aJrgj)q0w32g;A@cUttW85=HDkT%6Dcn@5lOWr;z zv6C`B+6~hEa+e@8oBe(WS{9#+e9i^!nCTZG>6A&cOjTWUbU_`M2Kiw>R29{gWx{= zgi`B2&L#W*i(~npG6xdLAC>_?!qn#fFLp>)nUzOTMEa+T|*@D(g1(l=}1dc39 zAu3qMswzW>87u9PLOR*9ZWr$KoI6*DL9ZjDQbjg06cp$igmE|<82%6VqAO*k<&kjx zD)YtcWZLUt`l{#q>jK_SQRgKOOW2OZa19oXm`RM+!B7Hs#$#_d4daxNE(N(>C(co^ zjNC+KBF)HREh^Lc6JJ=`NO*H$DXSXi0-jcw>?I+GiIv;jG31|>wTXvMMHwRtbMc65 zW@#HM8X7QhNBO2=NQ}a8c2G`pDTErPj`KMyUBfw9Z~CUr@k&FBRa^;gyqwiYgxUE) zSFH(Gw_}HC?WHD!J+O5Y4wfAwh2!GUN>;vUV^)-LcvG@cb5x4Xsg;;~`McSO0yCI` zE{X57oObKkoHlacUI(1e$edPWa=$+OnNJCr)xm8zn0^4RzHCN9Ph(1DfgsXK+QG9= zq*p%742C*8Ac2F_O~bzjlnjDgG(RRt0;*TR#OFRwyJ<&uR*B1;^euo6#;%gaS_43R zCLkgRt7R%f9cC2L^-mRPvEh=16o&3FB0zI$ysg3jA6Ce@`bdkD3&todUXV~6QI^iN zwgrTWmt@Sk#w>Xwd%4x4U#dJyq}qU$Pazh>Lz>yi>yA!Vs9D@{11qC~4+bV}T(K{l z%eG>%&66|miA!g;UXlGTntyEmO)r!%`xkM>d2at6k~i_X7&+v|d2uY7xWv1}1A?5V z9{_+sGnF&Q*cN(bfG-ykiiy}8aydgsvcO#mD*9xc1;bC=D}WD=0H!B2Q#c>KiHLhO zo{K0?nTFz{73cuEK;Bjq>mZ#wPvY&b|I;?+94DmcCXEy;(Pxk1&22s#|1lqB6biGy z-!Tgerc;?iHm=j+hJLeSvW5A0!P8<@nlU2SHW774(BBo2R_JKo3;;G>uS`LR7z=8@ z?+t8P8nr=eC&|B`=z|uA<_s|j2zY!{--k}QgVHXGGWYw9?rsQ&exwNogJqHA08PI4 ze{*#Hb4lxvl)T0=00AihfPkp~U+w6B-qOR~&|a#`ci&U(FFo$N3Bf^-eCXkRm<>QM zVo+lUzf%1C2%)4T@Q(x&FcLA)FX`H-+gkSQl2tC1&;*5&S8_+9S35R0)NGem=&Y)% zTh#X1ov+(ZX(buTzIVSD5nrLHDON%^t+mW-cmOG6;XTh_MvkzV$AiLKuMgx((w}XnWdC|Q`yBL}ysh_V* zJw}I>Y&|2{wIBIFy9U?22P?IpuVnv1pg&hecGq9goxL*Yf3p9(%!&LCm%2S&qm|yd z$Lpi5zDJX}U9bKGBL}%}C-htI)QKON82n9l#QZWtvE8rk*Y82%pM#nt@vlkYe;e;| zKdfR0xH@kTzW=US0^!?reB}jtWCU=cC+DB`@Q(I&n>qK*C@4pI4 zq1Fp9+@D&8k!gnYw1Yfc2y(FNMD?{F4?q5S`I6{E+}irn#@t8x8^zH=Y#vx>!`(#~ ze>&t-{D`G%qplaMp_8^yG!HXq!?g}WY2#W)9@;Q2pcmRWcF|7QI0n#)Z5l<e7SN7t8U@gjZ5Socl5HAA(3Wf%Wzd#v9*~&v4N_Bm8kM9SqcwIL#i*-z4(kL7 zhziS^3`R-l8W%wmY0S?dS!O7yFK^*miZNWCvMtEZT(PQNR)n+;6}k?ds<6h0-qMEE z*p)@JvWOy0Eat{&bJ$*8X)Ipst46CXH^=Ws&o2)aK1I6@;`|`#A+w}xs&^WTRbwOd zhv9Lm^3^9XA|2xX8jQ%kUymgkUriayG!J=lrCdSU7YB;wz{C3d5f}xSYfrGo5ngUz z0c#ZinBKgrxmn()2jhAURwrdO7VZsPa;E&K8pnvPRv)(_E4*2@3!_S87^UFKnB~<` z?2ORG#O3*r(?>s-;-9(Z3pvdwdd4=iwbMljTf}3#H48{*Y`1Z&VUHHuHJ}3v=7Wg|WI7 z_pxN~uyA4Ko5O@hwE37JaR=?-akLq|O2D~~HkM}_DM2;DboTpNyj&$-RmB0^{-8Jah==@Y1eOtg;t<6u2jLZ`sm^U$;5MHhH^m+3J|>?sK;N{EKjcyEO9=JbwH6 z(yiJk)NI8e6xw$BrtEz|Jk2-KA*JF>p&*_7N-c zF-yZM#77urTs0zj7aGKxsmO;5F}0_3m~ zh?4ki(2Rb{`6xG^$NgC+#^0e=9(ox9qwa{9Tz*;vkRV*41vEyDdBs}H=hH7z$LCkV zl=4dEuQ-(l&5LYEvMpt1FVMUea#*I~hjdksQV+wZYge1+pzs+_s(LIb_`Fe!>1k9T%Fw2o0~o+@0ZkB<*W%wn$I?x z7`H_%8X+g`*y6*OI2+t7o$lF7D`NT>RCN&lCQhXUF4M{`<=r+D{*sr^DHr}*hC&6( z2^8p<_(Bo6yj7@jTZ1F!^~k|0ovy~ya7R<5Ejb#&4%T85h1 zZL4;NFiTOAu_~jtMf5S2IMXqc4MuB{6JW}A^|sHSfG2sZiUOJnZ6-Q69Y!v3un;V~ zoFA*gu4)KnV%b>4GY%4o@1Ei%23z|_?UQmeR4Wg>cbHqRIPzbR8!x9jUKoNd!g>$bV~U({j1caL!_0NSvk`_!=+wZ ziF2jT_;+&rlT*3+cdv?8h**cKlR?!nmDJK|GDmMZheWQ zv5s3ygUXwZf7qKE`yhUq2}OJZN50WXE7OB}8lOutn}D;Cl*;sxTiV4&KI7fwiCgkZ zjtIMJ9$_>Nos-viMxK%)Vab`Ke`%{N)d^^0QIeIl$>s%kp_gW(ZYdk87+_}Qi(XCn z#Fr(;qFaZnCZr{QgIa-QysSt1=iRLd$dWu}HG`G*DS8}+1{;SjmXJkG0K2?G{n)zk zr;{T-God}9U;!#j2r1>1Z8&>Nn{h{x^^n#97vgNzuni8P$7^32p=3fgV3?=nE7YfSu^svx5=$)qL%DgGnv>NVFnbiZbjQnGC^xen z&mp-$X`4`lvH<2c|Nbr^n}j^q5AZdXcK&OE)@SY`ZlTc+bGbdI)SI_>yq;v!GDNKn zb?7O|aeUO|!c}W)Xyd=((JRCyIWa0E=X4+AfW7|scS>#-HUl$JEunuRy@!Eqd%lIY zrb7@ArJ6}M0561lpp}^8O)IWDnESul({0q8hSs_fnbXu zD7WtmO5EF)2WnvORssqCc#^K?ssuJ>gi()?p8`EBSkF-k2^?Fmo`M41SW_X3D#b;R z)AlsLsGNfqsuy{(tr^$Q!TtV;P^*p}iaug7@BpsI_wu=;r!P3gYHj;6GM-DEkvXr1VoI&`BijNr4 zT(ClY5m2DV0wyX`!^kPkC6}7@?m8hkyHxDTeooCNetl%u5Pmo+KlXLpRR(N%H#RGbIGj#FtQzv|tIpT#r6-Bq1wktQ?h< z*VAs`{iEeTh~lJ_s7F%mXJSJ{EQ;MLbIWO^aD)Z-iLrQMtKJq6>6lNSajIHM>Gy47xa(Y(TZgpE7Irm|))jPD7p6wYL+f;S_*MjFlZ3(( zcm@f7JggMJFsI-vDUWo|5Y->p9fSjstRX#DJ0$Z1njlBYkG-TQaO!NKD#9Kf- z<$ePZCP!x~nI;#CH1+R5|DqU6?bg4g0tPZ0dRuyz)C1PJ88WuCQGj3G3fsUv)f)^#?i=_(V2>m(9&H)=tnXlzJ zHH_S(<3`-=WE3@Mt3*-B{%q$zaetPFQBdZWL%GaFOW04X8KvvsF@;j6Z3^TkDlq&qH)C2LPd!08Q26esJnQ6 zyZAi2Qayz))bx(rwSGHP-f}o!Git*VeM|-WTRe+`?-ZjX=4mgU{vKp(p7~9x`#@pgzagryS;M=wpnWD#OGai--weP+d9f18+FR?*Cnk9S0j z5j@<{deiyt=Ljx{G~!FhJfPOnmC*0{jYoP^SU5JFWL};#B!XbMTakQ5{*cH6b;RWr zWbSsVy1WGIDUx@lEHu`&P`2pd7mwdG{We!}oxUNgpk%nSUWF*fPRnH9?XeVtkAAzROiKokxv8hW45R6oP+ ztZPxt{P#8MU7=={CJC}!jJftbZ6Lz#UA6Zy+3Z~+7U9Nd40`2mXIghyAEa1TuBV#v z3maCA(8cRBu@%;n#GL1%6YO@e)Q!RHCf z*03}jzVl*)$8+htet!%yj^sfdKU<)HN_cn%5jG6e5tH zALRKza>^IP`LDs-u%2DXmsqo{n~ApFY&ZJXG|k=C`QC2Jm#NabcW(xYY@U^$OqeJ ztf68i$yWh8vvIE?MJ9;2LC0DkO2VA5{VwK_bbz;&8ad|F%`-_tpmE4vQr^16vPV=QX zwCyXs@+BHT^-d>kVDa?+DtY{&fez3EQA?c@gJxD*eGPx{f%d88m?7fMGh!r{4p$?| zXjd8NsBLMoThz1LXOFi25QY%R8ur;-yVZ7}J72>GK*w@aUzbY92Iz5D!Ma2(KD#a3j@|axinFY z&a_)s=>21@ff*djkL#l9_Q;kCpN%|SFyXHJLJ9XqixuwMKLvn7yZMzf*&TS_xXh_) z_hM$=>a9tveW zH`RuSRSJQw8s2OuOaBjR{||ShwqErAjBg^ae<;vbnD^jPdvVk!jz2mivC@p&Hk@a) zzwT|%BsSpj?5xH78x^W{H$-50YKSyuqGyob@w}ai<+= zzwE{9GNVo#@y6PGCsKS(A8H za7-_m_-()Rl0Onk9tO*O?KO&~>67Bg9+kwdxwXH;0Cm#F&bigkj({2aOOnU&I)Mud zicPndL=G|!5;urA{$3xI0h=~00G|V^oAQ=PzZT#F2}rM1XYU2=@+w+&=@{VgRYVSV z{x_R31Fhq}O~A(DjDrFPw%ieWoTAmHirXSJxI%{++jIq8Jy*EZh(Sg- zo{9Q=6*B&zh1he8uLl>R2Vi*|CSrhq-6z3?V0NUK8d2ha z7a!|R7rdV?85@(hAmReu%WaZI2QQj;N?of=Iq%jidCUfP-q&|J^;A@P!}6->ig4e^ zkB@uTEpfL@Z-6j2%glMR&q!~Ilkcq#p8J=xRva8f0?V1>Rvblh1lABYZ$f1J-K#O? zy9e7mV@hS3z%j7@Y0f}OoZD|n*@Lm91jhZUr{D}%^0wn!{$yH;8>s7$^@8W{S)m-m zsq3Kgj_1HLM>%>$$061O8UyKS@}{mRKDFrx^(kiK4mmnKB&i&M;Ik}@&ptaMR& z4wbIOK~AbZdmdMLT6A(DQJ=~mrqujoG)%ZlDIqW9K*eEYLc#$62Csva42o1rJ9aln zsZE3aCnEv;ddHGG=m<_u)~)G3>-UCqsJKQVV6^==0P8rXPMp$>urhpotd3WLozSA~stZcS zwVbSbX-Q_``ALUMJ>|~nIfyX= z8YZdTPq$Kf;>i($xf9z!WB^LkU?!!^PDSDxa)p3FJ^9&m#$10t`5Cyp0_@CGHKpM| zPLdAh#BEm4xVDPTRmgfey0<+{4S-df{+Lw0xcNH%(x)r}GvF&5RwE3p)g@Ist7w6gPpWbw?MetwY z+ZRUP+>;yQe|)TgzD%!>lij#X?)@MIGDh?cfR6#b;SVc2%H5aHzjgec9KgE+>92ouqI~;`U-@z( zeTTGf&x(N@lHwWrCswzL zfsO=VngLKc&GhxG#ddZ>$f9G7(YcQe>1B-xD(V6V$mzv8M*N=u~<+-MmKlZa0PwPuuP zPldH+47Fym|G#HBT64Jbd;5t#HkT4n!xuZVl-xw#6_;oTzmr~JG+&pS4 z6dojP%7s&<#rWhOBPe*K^d4pMv=x6i_SPL>ViKC8GS4`de%-qQFG`G=IJP4o@i6C` zx41f;4f)>zDoV4WXet`6k?he4l9b=AcXswM#qp`2B6bSgrKX$^vuHA01L$fKW;dRg z>5AieC)909D@EpRL_+bQB=wlsB6=u2a(jh$PVqGfQ>td0JuhsF|NDi!Xna{P7hmdX{AYn&e0%t0*c- z*~Mqyx!0LcQc{Ui{J>lyy*duBC5pFBC8sa-4*+Xsf5~z)Ph8PbvfkOwb;Ip!cQ8xu zWLjIzj`=9`Z`|qz{3K`9_~2H;!qVVYTsu5j2=CuH_AAO1QE9!kMp!w0onOzG!(Jka z1MXe}M91s!hZOL1DIT}4zXEf}*`3I+7{lgGAdLwrMg{~J5k?|W3?ncR$-~1@DarcF zjbK#5%j?8sLen^)XONlqek%g2M`$`kr@^;|b2{XvQKQXEXs`!w*)YiwsdI+I8B!(B zm~;)z3hr%|37ihX$T{Dv!gT6WU-?wmeIyUGv1U6gkZ8Aq%Kz?cG9c9PXFe-Wd#g zB5izk@vML^xQZy+-XKd3Xk1avH!YPDx!_%+TYz8u$U>jQ=6KE4q znC7WO%Q#w97yN3Fwy;)&=AyG=jWh1#bwH+y9ne>BdD31iWb+9iB zLD)WOxqOC}G`|OxGy0SP#RC>hl9YZ%BhDzKQWTIJV3Z-4G!nH0#i{@pMc78sBsgRE zrLg8Xl8e1R1^6>)@DBN5@ro?{Pmea88sZ`K_-A^X?UkympR-XZ-JSckNwU)G5GQ(_B@ztF9-4NVWJ zRl3sPYbwA85aDtjuRv$pA3&fedi`h{%(N{2r{b?MKb_KZy?*})VH+gIWMR^;Haz*_ zlyahh{XVhi@JbIC#e?$>+(Mu^9b$5 z3wYAfuL>^UzAOSgI*mBLkJZZQpc6(hN=i8N)Zx=`Ljy@@J> z)X;A7HUcwD%qubM?-M6>5VR4Psw{Zdu;o0!>2`r4)b2kCkdJmjJr4AsBE>0A*Cpe= zDW=dm#~|Di|KSsF{|$i^!O@|6PWb0j!3*GoOWC*`)WM{lWwX(MAwF6Ui8b(rp-Utu z8BH+Lk1NG(BG$6jZeD4zs@sj2T-F>n1$6LGV}V%T!d)~rL2`FFvDE5JPja%ZSFpCL zT02#Sc%RfMJCDudy8VJ24OEYVRBt~o)fD(jf;cq4&+f;qB3Z*zJ|IzXcO2p?pPH(q zZ8aHc5h{sVD;oADPe|dv z+7(CE?JR2zv(4|?Vw>(zUfE+y3sb}EV+Z#R>Qcw{<0C(Hql1;pITQe6(g&PiW|_WTA09ilJC z_l>IGX@Nhs35cPtykLVpF7$Uz7e}&9AP;MF~Pow1>R7IL4UtR7QjNtAolBA1aeg0+a7ZEAQkYasQD>a~kDJ@1tn1ywAzl)`ae_ye8HU6`vqk~=OjX58H7 z7#Qmxh|`gshC2R{+xxh>MdNO~g7+HdrdT0S7B&fFWFfpCij7 zYe--G2qVK7MbQ3;0;VZyzfK`)3eKQGup{mg4zXdRBE%{dd0+O3g$3(KUu<-J-6$p` zKP%b5;s^;~Ck?Kbt)x#ss{D+R%y_P>FK5IC4#)YYjo}dEh)$N()L`()_?pesXk%0` z7-pnnWj$&uPU-$O%k}68jMrS*z-Sb<*FxBy4{-!NmNk^f$^p4pHWaqCNDgx1sXhJ~ zsJPNQA2!Yyub5zX#lMq_3;Xt(e_E_XpBlmw`0$jz`C!wt=C((AD5!BP!UR!w&^W<2 z@fSh1a7&zr4pb_J#Ko|{Qr+?+%#>eHM*M39Cq5wF%Wdk0T_Ds>65hX3Yk@+wW$#I1 zw5RVu(S(nTbnwr?f@puBp93SvmN6F>Si1q&p;RD9YSRo$>9cha)w0G2Fh^IDQ)gE3 zgP;@+ap^4SU))Y)Z7ry8?@P4${qe+-LedK*q>>hrPCAklqRreY^_Sy%4+O>mcrzHMSdp4@@?tl4A^V9$E4UgBecN~eCK0)Bgnl& z$h{-+K&>8Tafmx;FdeXGV%>e()<~HP6CB4nfC#~1i%VxQZJQJz7Rj!1i zpVe$8xz7{0|1sZUeW@SM;Kz2QEc7_>7QxP=cNZbSRWJTNd%h^gNxdkaI4wr^=rqvF zR3eKPt%Mock0XqoN;P7jd7G%4OG-C$bel?V*GeeA77Ts<+03>-ig+^y$bv>FAV zdyP29+4CTNMPcM^ts$aBgNgpr@8dZn<(LMbSw*ep%>eLLuzDA6h9>tVc(hOS1NXsn z@2XePc0JWimq0Y{-tvHFiBvJ;5Lp2=Rj=>;mOl!%+}CPX2)d;m+O?IPPg1}L%|Rt8 zh2_TH=ydAPrpVeVavo?h?SgKd%Q}0A4PtM z62QKs;GaLqIU>rvLl*F%j#8jE$Mi;13=43bGBgK8Nbn(qVxYJ5bZJ^~<;nRaE_I*; z{&oUV8f5~P4bfpT!S{8n(a#a!TWm<>9i=r-)fTPgMa!Q6p|Pficr6Np`-XN%;53ow zN;%7YhK_^%>jv&snf-KGF}(6p>fekfS+ouBtB>Q3$4pY*pRhu58dKw`e(Yx9Xgs=W)LCHaUhxFeYbH%7KZet3#h||Q-9eAM%dXG% zse<#4q;5!RC4EK4(k}4Pbuet|Eyfat>OJyoNmOn26vOK@U!3u?tOsv$IUqIU4mlbJ%Aj6Wji3l1WxN2agB|Jt~Ii{^4ZOW3zK+q-#St z+Qxgfhtu*TIu9ZAQt6LZ(_l{*$C1JUCBCc^kqe1ci={tNZfR#Ff$sfxyi=B+1p|^_ zbnz(#^6GS|Z@KD9&mlYrrl!v_GlQqPD2b=K30X5F^oi+3WbPj3=SO_mluW$z6?kB|3TS1#@PCGX}@LLwszUJv8!sA zZQHhO+qPZ1Y}>YNocee7`}Wi4Bv0ogGg--6-|nn2?`w=XuirIKOp~KCcOT*zkC*&j zyAg8;Sg07!O2@|X{CNB3pb&HS(doZY%eIQFi)r8QTu*MAXlEz+Y9hn_rybbSSWTHq zDMP>7$15Zt}N%l(+kH8orWl;a`U4f_EH^FmXU z(ld8mu29+lAhSHt9ItRNKQ&oiB>pJK!!(c6Y5W$#JG?caS>v-K8v69H%a{ z!i8EjMm$Ixr0>B23z*!cU%&w@I;soID$w-&0yp=y@gZ|*>?shn7j$0*w$ zF=^zQ)`nBVEIUkd1l74!*~c}3wV@|^gV=zE&s6Zzs1`Dv@%R$F8fud+{X*V=$;Z6^ z613{ClXec8iXEtDjiP;TiAX<<$Z^z0w9T#az)FYUH3>Zk;Lw0!sZA8kM2CtzIWAn} z)~0H$O{$!s65;L;!9Lz8qiL>9XU%Mb;F7@>o^>R>2YGLNo9Wi!+TSVEX|qMj%W};+ zW68pO)FoQ&_7{=SM<0eBp8+h5>v5TFHStn5SfETBIMUBG zAQCN^0l+5lvDfpK(OYK;+$A*P2K)}=7MTI5MQ%JI;YMp?N{L51vSm+NGwKgYE2K?w z38cy%{0*90Ya6IdkH%pOug%`?&M5O(>aZOZq2_qnydsO`InqW)NE$6csg#+yCbQGE zMn^04cGmy;;^(&y7FiR_l}pzUWC@6xgD_wAKbqFb2ya|~`vZ)!TS}OAdkY^e%RsJm&+}Ab5n}7=LsmSrY zMpRK()AQB4ILB)be;mwI@{TNI{`70NK_ZV|be?k#B~ef1b5d`}Cm2aAX^z)D8BsLTG=NgaDX|m%)%e-II>a3ZBFuQ}h9z-O$2!365 zwn`ZJbz+YN#H-Mw#C8gLBe@F0JJ_rBpk9ljO;7K6>4c^52C-T7IOoFJtDJTw|AH<> z@Pb~+>#3q|58)`HK9T2DM*l^m?1C*!#3b4ymT2Pt>Oen1{%hR}3g~D)R%%g8);_Xe;#e>A6$mBcU9UCG z*X}5Se$^AwIJKq+>z;*u+!ME&dE3`wLPw7EgY1IA2V6GIrq|(U{if~JgL(3qg7yP3 zVv=tc;vN{s=qqTiKSb5aJ96@9+pggiQrr3q_?q=Q3_c@gzv2iz@cVmkdV&t}{x4E+ zP+xQ4NIlGKKjUCrGAWn_9>c<2cv-S+rlnB?sdBWc#bJ0uCEgx4mF$LD&_IB4+KN>c zfp)D7ijBtp;QJEDExtwk%UC+KohRa9Pa9MN&y2#pL){g2^?Nl5H69JaI~Ii^Iq!3u zxb*A2YS2lJg)n|z=?c3f@td*8pxA1Y&|r*9MG4?X(AJsTgV$9qlMgXu@4PV zSm>qDbEAcu9fXghNG$D2vV%=BgTV?_l7e9P?H+?0+E@P#9y9x;eNQzv{+wGp+XBCX z>Div$^chX$bc>9mwc@sBv+56Zs{6uIYr$7s+A$9xxH0TJIEuej7Kz(mG;-R6MQqo_ zHUJHgiCc!DH|-dE4@s*yM2^|sOhR#+PT^ZR(h7uvA;}sW2zC{4h+A;0U%UAq%l-B% z!xi_W`}|~rZ*JSnGh4-ePj@nY$#PrIP82rI!FMe_JY(Clu!;E1xJOQ*Mt*;MLyH35 zvqlLvaBaT{(czR=GK4L`T%huYF@J0ONYIB;Q^iten?gw(F92r%H6E~`ypTiQ3V4z4 z_yCaokz|I4IXApfg(lFY3Di7CIWeR0FQ_D?A5e!HTr#8IPzT$uWYVo~0y0*m(u-08 zW}pJHvp}`1HP>rsLQ}7DU$J88aZrZw)??|*sUmxsFc)ZC${SXXDX5HMALYcZsB*At zaN^z4)$*-c3UbYHLgY2D8d9kmzlA-A`j}`(`deR3bn`d3=UW}<`yAqR&0rl0*Qef`&Bm;`_c~mE zfr|9q;df2`VG~XMZ{=RyEI6=rdH-Hj%cT7+;&eX!u{}46xEcsxn7H;-h*N!{1272l z&3zCE;%Qg}(P%gXK4&11#&9rP0z73veJ1NgKz;753Si*^I9C*c#NUjH7;iQ2MI?|k zVKAuiK7#kMre9Y7^sL*(uP%6$mLGLr9o>F+08l+S@Gd-HE_^_P7x<}n+s~=zU!#Ou zAc;~}*f-9g$N z`6?}u+$txlcJW1JFk+g;IdlB?1@jbmueUPU?@(`xH;?Hq2PUSplbad%AB)4a+wyit zCfD`n^W-=H>mEE|$#@Fzi)83QP4J9=LOmDhc>kg+ zNX*z3gIYJ)_7Vl7GKyK_lb)1yLHiGPwV`#um3dUhz*I$9k9Nba(p!7v~lxa(~ zmEWOu6QSy1C%R+dCi-cKX1^WHZpenCT7xs*kjv^yo2Kpj2J2ugH6xm{;7P>W@5<{Psqnz*tu@rQ!W}hmXWmb(I3FrwkV`_a|zv=(2K}powiY`v>KP0Vg1zBt~6jw zTW-ih0^Jnq(t`1!SXva)qUzt6#ulZpGS&?3+zk+f#u-QE-bKql!YxvN>=T*}?EOry z_cml&a&V?B9^YSfW$y`6iuTmDwfd#$QRi8I|!d2g9LI%mptFK5*iIut?e9z#N{&AwYp zBf6)W*kw34+|5t+z)nU4gr}oFtb|xV*WpCXgF|N-u%v0Q zrse$`E#vC+?3bHDE5^9>^?V%~VmUj2V2c;Zq zHbqRFTZ5#qQj*9+tNKQF2F7l!MIR#STq$Evyq1OF6Yphx*eWqaZnt~>5UQtJVTQ)_ zBMqy~1+HtH);@~0Mrif#`k?5}!;8%w!c;n8b_RXakE55>q(#i_X^dr#fsE@=*k0#) z#HdWylj*mJ4HmraYS>Ojv^wb$W9#hw+RDb+r5zWg9GFaDCem`8;`wtyA|U>(WELdc z=r3+!y0JtaNMSI&Rj?@n)^ZF%aG0ZonD~n`+Ee=+Bqi>H@?>#h%O!oH$lqv{$KPo1 zopWy+ZboqY{p77Rea77m?6!YHnVkvDa`x6Cd3ii(qcj82ZngQ93Lk;Evwx{Dv2 z?6u(Vm+74<`+>e1R0AQ-UZ(RS7DP4hHd|5uaIIHg3?xuKP{1m|Eu*DaG@Ngiv~|F? zIH2Z$Ng!JL7?OUbJrWg`o}Wr7sx>>kX^neZ8U?1;mr%ifMDr+fHUB)~eT6B{xMjJ= z`ihO3(2PWVjg@1Ht6%vmXY9t7PHR3C*W{N%<@X0W=QdfvI{VwrAZ&oJHiWHAbIJYw zfORftPL~{kQvxxKb_zT!_bMCFI7l(P%F2R=uc2I8w#N zGSFg^3lj-NLmtqP_o13iz?av#O91^2(|g{r;sG5= zASGc4!jMwndTl|f)_O~Cq3;B@QM)pC*2h7!At=rS5n1fffZQYVOe;~xwcvDd=TN~n zXaaA!$~7O7`SNJPRu#ueg;>Nv+s90(=cPitS(!uc#ok5zP2g4R?V@MO&=u zD*~%b597(~gC#V*3n(P^T!&T~9ap}YiRn7y63p|k3))GUMDDiDzrEy5u&eT zS_tidM&Rn}wEC|Vq(Lm2GgO2v`a;$6QXquhR)vg zHz}oU?v04uPTnXXVA33cL1z?l(#K;wrC@@`{JMf!jkb@i`l1*qHvZy(?thdVRe;%N z-gHaX%gU27Nn3ZI{J=vkpN4H3f@Qe~UfcbT#X!D*#kK(VA1j(8@9uoHEt||tSrn6H44QqS87>cQD0LQf?%-a>VzqObE0CW2sA#m=F8IJ#i@i1= z{F<1119*Ia7MpqhKzh%(Qwpc$zw&Uqc3<$=I)|9UNljlf;$Vcn$FuvGM&I+oDZ0!2 z_!s2-KSol&_-;!4AJ&}q5AEUKBIo}RTq3RpMz)S-HrD^?B+W`(m;cG@4qgtYbwhuX=9p#CCCpjlEiieE{C&@5#O$`=< zdDEO^P`np+(Cly4R(8gyErD@wwiBlH!g&DtL#dJhFipC&mzHA@L zZ829}yI(pDJHja^C(}NyS!0P*V!-hUK}Xi^x5Y#$Z!VxRSEyoPE;2g_w_9UQDlStW z;~l$FleTbgO=k?y%zG2{s-SdLQ-R|iu6||6^ z(4bv9)W?MhXBcSK17p;6cvW3rl#h9h zW=}uPG&r#{+!v;38MK1g3isjNbedFvRjA-|Z~}v;(^zwUob%hXH($%{onF!@F^n@e zY=z^9*NbO}mx%urZx)Xk&k;{F3N%VIiWHDGo+SoGWhXkn1j~J{^r-2O$TQr`Jy{+> zlp2WVxqEZyhdl%B#jf42gF6GyIAyn53hZixDi&-|e**kIMiWqq6b)by>uZP6mxhx{ z5M$Q^9Nh0K?HJFdMEyF0D4jZ(4kU{Kp{WNnDXN>;Z+E;`&sm@*mD;pHW`RT2S2;>w z0hILuDvmC7&r6c8cdn{o7%O0XVjlO zgggD$iIn}Fyf|-vmgIiwr}R{bXKca6^h?NHP(u!gb?+3EvvO^Ne-`S6v$#&?o9B6$42LLEV_+PLp{}kFEdoydt|C8*Hqynpgqk`#eoVKZ!EFqX2 zYb+J{I#qvR{DaAo&ovV=g+W!N1}RiL2S84(y{K*ghLk8Li&|T{E}wlvv}nwKtSAFa z&o^6ic)$EZsTgqW-EXq!1UsGj^)ogLq0u4pcWYUbugXnk#DiIDePnpVuV;F@Sv*x6m$tI>K z%AbMJx)^#%buFFRa2^y)!UU?=iruXvgpMZaGt!obZ;T(Cvnq3yhlRs(&1=YZ3B~Xg zk$Bx^CAsrGpq$AX74@Ra1;Fk?kip6riDE`Q2@>>uDO9reDCx(&E4BMv^UK;or?{x5 zOW(7h3a((Q(?Z68feCtxPje_^d*e*-&1wQ@f8sBlJ@Xh0Z#8t5UUj_5$7IfKimUs!hj( zwlw=n%l3Y?Y$`b9n{osz1!d=j%?<=&>5Pf$<_UokB}YwdCK^&}No2v!T5pnWFh|q| z=;oY@Bq9~)W{xqkW3=PqPOB(24&wEGIfnsiw0l;&(;(EE@S>eOc0$F6&N_DMEV~aJ z6IEHwI=b$)`hq?4DCk1CG`G4~r9W(nqvc}^BWA7(lxUp*bWO(T%lsnKSdoR@Jl-*z zKjva7(BpAp}DOM8K3o0vg7W**_-|t5$LQIX+E=vTSe}(h_K5FyXjnj3bB2vc=e9|w=*NR{zK5?(w6j!$LkyT))6)s2Ec{F1=;Ta}a;8=8f_2)% zW!`}(>S}yUgNVs9xmCC$O0aiAPCx;b#JZ`ECc=AVlhY!S zTj*&#Eh#f)q3jV0#e@N(7C#BJqi`ItrNr~JbJXRbWE0GrPro~%-aGC z5rNQvj)3T>Ip*aO@NbcO-Qo+6J4DRwC;j%;!5C%psjxlbn65pbAE+W}B1dVMsV!Wd zuwsS~J7QI~5D8+ckaT{S16x}ml-SEg7=2DiT{M4wV0{pMGvbJKpK0NsH_jnl?Iv`< z@GMMg7n=Gn>l##9geit6wuZ;b?1w7#Ta@AG4#>vYFOW5_6Z8|g$`&$f#0%~Rx5|v+ zH?_)hP2o%3c_7wqpZTa-JnXvO;f^%OfZL_{Ytf-dMtR^*{Ko<@xu_DmIjxpOF{+V= zxln?4jBmy0g}ZAMPEX-aOVGG2MSo;1W6Hg$rh?$Fm}L7!smK#oF#M9ghotpHpRWFj;NW-LyvgH}43f5%w5Q51j)LKWE zk<=mbyaaYk;(E*LA-z*q1(goZhRlDlN-?-ny|>LZqzLm=0IDaI;KW*lQ>E71M?EcO zcF_JpzIy__IFWRh5P!m1Tg`go>XVQ#LF@oifL__G98#2*cVfp-&{9tqs*9DKXqdZC zo2>>*68=rhHo2?%V&lpzk%M@vbzQwtaMK(^hFoWB3+SP>S1X{sShU<>L#lG~0@DC8 zdya9yrz1q2dz|)J^3!L<0tVo~&07a@N6z&fj9`966UNwS1_r4x_*tZqOl6M0-cM}p zoZ=pw(W~$T9glPmInNmJk!&PNo^cWzk90>bZ+Jn_f1$pyU*6@AN6#PJL+pE{uOT=- zi#jjfuU{|qJkIH%*nB=5y<$+jV&ul$ecr*HLx1cK16~?!Gp~iVK>k(i*z8+huC!L# z&aK@LG)2S8uyy!xRt_wb2ORvk7wjn?%0q-6^HSZ9!JG^A1u!KB7v5$BT8>eB7h@Sc zp0~?TGh%mEfi01gNqT;nFL6-zxMgo(&N=ukXpt=3lYEJ!94OG{d1%Q8&V(W5A=BQ5 zuVWus`tpitnpd2-g?`GBwI_gD&C#klCzE?(0Y}bV#r0V|KIG?>H~FYmn)kqyQ^f}c?0N4G3>y49*sOV3&r%kS$zEA-H}nq z=k5It2S8;j>qk;4SE;a(+bUrfp&y{1u*W)xHpn`N8?KMzyl0!z8*8vqyj6Qhc&YDrIVK~zfTSXswp~e!>tGjB61%BCp+|N5F zi{)z6YN>=+L5#x}5rO!;;D}L~Ow&<)O4H?TZc5Woon}%MykveNeF+IhIhlU7XOEYu z5``w+j8TX-R`bBt9t`SqIny+0>=mJmz{i9mR^xCO1O#T_(~O(7wxppBbseev*c{+h zHUkCLhF~DJA}($hRx@2|^&P7kCcs-+N$kj6>7LRZM)){WP2`x;t8c32vS8p%)V#ZP zjo^;M&JKBKqz4|L6SST4mA-ps++TXhx0HrS8+@P%! zoJXJ!=V`*BFI0yVf_a=E!*0nCqk?UOl0fRbC*9wyHyyDO>0ytLpF2h8fdDDcN_UBuj?r8 zxp$Ex#ay0v2oVl*!KZL54SIJ@cWy<{yW8fV_gyW;Qv*tK4$74nBev8%itPQu^G z(^PQtnNLigKi#&^#DCS>6XvoKuL>Q>i@1JmFc-XI_GhD7t9^q0H>B*JV#6Z=0l)nz zvIjr_0G9th#b#vS^nbC>MSsR+fCLaKhmt@&?ViDJ_D&W>=s;vJSbJ@oammkP^-SvF z`&!&?fIlb3V5G7&i~oS%}Ygk14{d&T{NtK5!+ zVpVWP&4h|}V1X{)8AC6NHS}2@squHxRLkK@uRoZG>~s)~U~E%IaNmz&&3mOa$V6^; z$T-wo3|fH)mp$bZ-G-m<0KVt@oziJcO0Uq6QZyk>vbM1t#EN`QGHx{AvSKX?GmKqe zA*aK}>Az;tpdotl%|fz)!+qdlUI#>9w4eS39vx}599M*+VzF6ak#pL2yo#unlY^5lg9qi~N zObzxme?$2Vur_5N>4Kq{$SqLDizW-bqJKaJAK&dv%%t5-VNZXc2RFo5*;$s{V(?uY zt~VTeZm%|2d_G>|+W>6zyFeA+@b@Am8&!_HsQLwh%W;*GyV>XBHe2B#Qc$dEi^6n1)zV3N7OM4+DygKt!sf|C&{a5*b6 zJu@#O+>=qbiHSxL)){oTt2;EFA%CQG{z@R$nN0vF+~n&F5}e(wUir0QIK?);J{+xQo~@_R(WyM|nB5|spaN+h6xyRJHeN8u(Eyomixq~R$dVaO!Xcs))giIst@!+`X z04r5OJ=TbY!?|vZ!I{^)@miN|*{2BsP7XL^Tf#>QlM>D4%2)|g9HUoPlWS#E@SJjE zJG>K?V}@!v$SMMRM{fngreSf29z4IX)Ldy2QGuS~SiBGtH$yUTz}VJ&Plv}{5}F8C zWVyUa>0|lQ@-d;MfpSDyO+dEqfVv*aK=Mut8*Gtv*sR7jAtY1c^;tHk!zt{GKjda1 zv{Ac9+1-;u>0C=ykeeSGA71k({#JNU;bPRac+cX3iP}c>)}4-AsY;u;himWFI2`nB z^)}sv>tX8Qeqtv2Pw>ollxcSn(DijxKG5}V=6*{)kzs^e8)Pl@NxN33k=odk?xs=N z(loVO%Cn8}4D~4OGO93VA$*d6QMgdO%;rQ;eC?U&GARA9D}cTR6L5pzM}eyWOJsx^ z-u{)ew00}CjmKA8eUw#ut+=R3KUy6Rg1s*0P4wHV@;?|}hmhUNZyXKY0d z`^%gG{6d?ov%!LvI!EY~Qw9DHBHcYG1&<_S>;)#jVwqm;SzV0zh3JV>Y`8?r{F^*iVQ@3kO)3&)~(r4-4`4lF;8# z%mgJ^p{(?h1z?HW3`@S8EZvk^(O+FiZc$;Mn5X83yC?}h-FHG#v7=V}kF5C%TjbP2 zCrFdYaXms4`-on6RdeXFODS;$H}{DF>o2~R*LpY&FR|hxb^cs0p`D0r#o`Q-1gFF5 zLQoDQVh}mSytVC!>XQjdB@dwKG{JK|McVninBqBIf81L}%!r`e!AQ9Y8~F<(P}(n` zr4X@-EL((0gb+vn2-iyqGor+KMua+2?4hSaBG&N_7z*HwFR0oMj0~iXBqk!s83CCpC-fF6TLXK0Y5%Tg)P6 zxSjHFYt4j?b4qvm^gdWrDAA(h;*HICK7(H<9kv=l;dW-Sh4g8Cicl^fiEoyZmJl#W zI5p;})oN4w6nrc#E*nWkQ<-A~k!*;UmUbkaoUTP%h$ZIqSWxZBoMKIlA0 zb35yoEpviSUqSs28t>zk@F-ScEsE82>O0*W%0y+04-_ZX{7+ZstT__C>L(|_zGd18 zAaT^Zwq;^luFz&|sRNt4OP{wCsh3-_z+M%U#^G5e&tF%{DcyNz>KjbiEPXU{D~CI= zrOtCRtYejq&zBM-PYgfkf=I&8SeKJqz@)#W;yYI82B9_ulALi}BFfaMx8GcZhKvnB zBv?oL!=S;Fofj$Ss#H;R!a0tayTYBelM80%$zEz!zA(u(T)3&Qp_-B(v#7ZksN3+&TqDVW+Az9M*wIa* z)ef#`ZIww{GIw0*6xg+KV7k5Ewu+B@mQ4oBJHFO;lza$nTR+z_uUDhYebaRhfx6yo z2;xm%k}h|L+5HL^fPg|wopMuHmBKH)c^8Q5V{WmjZE?t3`6Q-^tNkt7Zy@t){i0(4 zPkw!sa|4Oa1XZw4W}$%*MKEbjU)>zEX748yiXAff_M-Ko&|ntD65J93UBhG+SoMtI z=@xf!7T)dSq#66C=^hd?ru5Ao*@C zUWwxZbYT@Xvm4hUJSpDDI-ZOs$hj=lL|McZ8O00X&?XobfoZYV+@;h=)jR#Qx((~d zlh77^xzP@MFkJZp=|m06@fg&SpUEgdMl@phPK}=RD0YT*@E66bL(($MK<3Vd;1lot zFTd8r#d_dV6$0n8kJxi`AY&jvw)`My@=Io6gcZ@@Q)uFP!Iv@?_+3_Vs_g*6}|gxfQHz=Ki4YqDyj7j^aN3z}*AnetjkTZ&l(%-HZ zS&U^BSxMTx)iBSdzVzbn#4z%FggHNa299lg-1seBIm{&D0T^@ zGr%KiRX`BVVO)pwKiRSC-0Tp0&*n^!I2> zgS<~%GPTy6TF47xdM6vD_cR&|7vVx894yiBOd${>4V^v-z<>drAS3aFZhKt@BcdnA zqH)}`M%8LdOTThj$omZccaTfw)X+)oO+ED0g<9V&A9_KXOs00MndYAkdm+^R-?U;G zIZ7Ryz4tyTF2SJjB)OTj%cqaBFI?8W&Q-k!19b)u3o-s8hFmp)odIt)^0G$OQRRjU zrGk_N#(vT};$W@KXcfX-Xp11|tH@8VrRpZRo0DC-%jw%im-AS3 zscyDMh}aO@L|X1AxMr^Ax-9#=RAK7pzJ3?QyxN-YU62Va&IMYQMv=zgkOpTb$B@XSANy{^r&qz4re`H#md9vR71G#Vz@)WPdwk6bo$eHaS{YE9ni=F296wSmJI-0D%QDLC&wmVyvtTHrt z=q(~!oF|rPb|KU9|50M~e7tDBJwU19KC#!7oc+B`zwXBrb9q+~s3XeqWiH*7T) zxMhsNZY`x7J@Y9Tm!gEUN#jTSZcWKkve}4wxKX?ejrBw-R1;c&i;X4B(2$Jl&m99*NEbdo}73b?w1}`@iGt?#0IkfI@P(L425$6 zytN|#?2h4C2YLG5ylyxs=*3>#>BIhs2s1F8;GzA|hp=A^Q{)s!YYA9Xc;Sy5N<5CYn&Z$m?jRwL$R~<| zXvEp)j+HDpUPRyU4l@(iFB40KRgCQZ8I|0p+6A1#2Yi1Ej`1-btBbGFnQPR2BLWr1vjVX}NB zy&=b#>XJ89i;v(!Ua6hw5<7m)a;TnKy!PRFLG3Y0M8e?b^%JNZ2y358!S+-;9G9lm ziqPlr!%)_59ei$it?x})XXxyIp2TR@Z$%#5q}Ynu;|Uh8RyQ-{3}~!8509%Dd2E+A z2{mi8QloQ-w_5(T9PiR#AxtSW|1GQ##+QJGA&oi`c0>9HZi5J zplu~RTizjE`~Q*9Y*x71%V^LtOT^$bEV?*uEo_TnGPk=O{m_+s^9WM(Y88#^TS>)j!JD$e#1igL zmuJ9zbc|bOLBroGA%;}AbY_>XjeVis&1ZQQ_28Ex-^PGpSDQ_P^d})JHpKRMWXEMP zidKhLva{-xR5dL&SS30pZaWb1*1A@dt_p2k_VWa4)&Tm-v zE%CO1viFBxkR~^Ufeq~BghBU6*?Z#_BMg~l`#IJC@A7Tw@i%~m-6$l~yI7B`MPcwn zMp-se1>!l9k{d8^8M}sK-CUZTw^MK|L1y|Layvl5r^9aq!q_h^BeQ3ujI!@kmvr6@ z1O1LZe1aQ$Q~oayE*N?E9RhE_FnJMdfj_h`WO^7#_f)*I$ouR<@-q#SnFq6u zg7EyZ9dpTXPb9E8OI))At4ZQzkR-^%45g4#B;H`Mn-WICDFYC;#ZI_N1Q2@{KuAXA zYYEk4su8X^xNW%9in74NLxEi1>!q6U+tzUG;v;rI<2E9ZgB*ghkROTU=yuHk7DCa_ z$gT3GkZj=Szy+dQ^SmXSbDDzOx4>CW`LoE+lK4yY=oY5e1-+77p?iO96Qw$_mnkuT zP6K}J;tL>Nt`oqI0JApSfBr)Gj=dN@VQ!1HEWe!d?=NS)(qA928Tyc2?cM;deyouF zE!gFs-+Fq)UJSOl<4f(mdfNwIaJfzWT<(zugdWxU|H21a43r@Zw+9 z^Z;#K42KwfgvQ1|x%I?=f4-#t=A-4pN3u_e4GH2V66BME14b5nQi|oK?38`GQ3Xv7 zLma>PMVN#3Pb%-Dx<{65Oiq{*v;ct7q&btp3AeZtW=N${17DLK?|NM0_eN@Tp^J-$qpoe>Lj z0~IzVy{?vn5vwJ+CN#Bdap}pG1QGg~jB!?#xv3n)0`|Mq{oroo#AvqbbZfv?K9I_B zQfiYTDCmC5+-z&nxNo9T&KA6-$?B++UxoH=q^`|r zD_4A*Pk)AWdW*ZU##(0_p)=@usG}PvI0#4OlKPsm;y}}>S|yf;3hko#+#rq*S=+A! zMr4K{5jFNE7POSC0;5aX&!Y^Hr7IlB!eBMx7t?aRGT+={LR8TMr65|3Q zr8pzeu7FJUol9AnemePpcJpGyJ7!=?tL>47ZXFNkB7a_D*tS{YiM)>S*I=z_ zS)4qY3)6~o^mzcIbRd8D<66NEg*IVc%gu!s79#@c`_@N2(yW^~)(Pe;bju0QEa*rF0{^*kw{mMu zY)YMf+PzfP%*(oN94-az^H@DqKHcN z3gFN>y?-pC@fqF0dgYFeh*FK4c9(CxtVpblVo}JO+7ZTxa9OT3uvO?%Y}YKkC!Z9# zbg~XUk8*(oU&?TrgqwT@lN0WG$>0VzXxpjUqs~~iKy#M>&F)HNzLcUrnQQZG2K(=5 zTOAlnA=l{4tr#y^QUFHH)JxB zAtZ2?Eq~}L!APZ802PzpY7+SGN`TaHb_2d zgSwn`!6iGC#YLh*JFFxlS2WSOMy|U()yvjDRhwg2E~F-%lNXg8BdpeWBUygKPAdYR zwZxt3E`KHbRZ8j~nA4@ow>TOYn?aowq!sqPIt1m-p4XbcQ}G9oU_79hBP z>*B@f)Ut!V&xDJqHNJ;F8OShzv?^$#v~=LKbQl-esK~uhwxwDT*vq90Dor^-FAqxz zON^58o!^jX4u-&|aCZ)x5q~JjN0l0^k?Yyqdm$1j#sj=RHdtG-y1AWiUq}J3j4FU0+9f}GRZ#plMt0` z5Y2Yop#q){!}bUKU4P6TS%sURI`(~J!0v@*lDMl0+3iy|e8oQA-UWCK6S=P{DoyiY zFS|DZxd7VfxIf{mf-7=P=m^%CV9Mh+ZmRob*lu3wX`!Pul^x zaFpH?U`qIjyMA{KxsRVpO_vAdGeGna7W8exykpe-^0#*IZy)QIO7ceq%+R*#nU4wb z7dY1@A7=Qn?%p=Wu#YYF*Ere}C!YLIzAr-=x2ufXZRg#t$RI)m)Q6J!I|Rc3D$dGOtc+E@Dff7JrS1b6%RG zzKMXeqxV;tk(kn7t_u-s?(09}Kaeb1riw8aG<2L9)pn=X2~Lk7T^;?~VnzyCGUi!U z=_C#`h19|%-PeM^oOsif*kn{HPA<(W70`A0t9n_T2z_L#g38?Ki_nH_6^C9m#b=Mf(4OHwg)s<_%!ZcwvoIsoek|v-jyNZ&b)H_(y41DZoYWZ6 zj;1Xs=zWAN$|G^^=QAv5S2K%)0|+x03#PGYQ&G{@Hl=mHcCz}$gls-I!PC5X%A#y5 zsCML3^vke%(#(<77nRsGtVCnh1Yer-RkWl|s{2FNq9fOO@4|HdVmV6g5|Rb`9t_fW z>PitwOmpw_xcB%y`U0N=0(B(R(*9tgE1|9@{lma11PlX@!wk^c`kJdm(_ojGu;nL$ znU%{U?{^ZgX)Ux@80aoi(VHhNdi006zgn9o$9>{n8|dndMCcfu<|T9t_pYLvJENW& z$|T1%TNDlFp|ksA)hdfp#SpTFV%Tb}-Np4+1V`UmA_dK(S{ z0KoO{<*Bogy@Qdkk*(2BYL~Tvo2s;YULEFMi9A6*u%xFCgq z02TD?_@+0HGrO*NCUm~P-mra{UReDDFqVI=uL3NyY=jbop6gwR5cx~|W+4tcHQpGA zG&g4XG1R+I`5RAIsQyW~HQpG?ukS8Mrz2{O@R)o-dRAhpa>g&61%?ggH(i;?xC-T$ zrPf1qF9~p$^AjX$PE3|4&=9wJ*^W4dAJ0U6MmmHXuqVPjsFq_7d z;;3#UtxSx|V6#-19FGuSv)}{*=Hg0?tC({>Fj9+sZJa1sig#9DgBHDtYYc(NjRcWd z#LVmQJELEM!p@8A)`01E7|m-t%|jGqt3`*HPEYIHMYGZm*4MmKbrUr3yV6Qkyq(O5 z9{tlr3|K)h^ZL|z!l;{#U@FVatQ_S!%i?egxx78EER_nR_4sOBJAPD8I-?GlN{_2@ z$u57C)SAK7AaBLImVC$=?oVP;a(<`St2vCOVlGwHqQbSN8m;M;9uKc{jr zd&Z^s(98+CFX2sXaWqFt2wHsBecaJ;Pj{>)`{f=<)oLoCkz(Jrs!vIZJ6tbu7R9-T zwMHZ{=r}QUXee2CNV;``uqFqeR}3jKm_X$i$CQddz7kb&__=1`*F0GAa9mo}VJS#IG* zcvzv*%(Zx*%Cl-8`-anTNq`V~C0hBA?db3CCI3{HP37V)D6e{29?*2gxvXEP3C*g{ zPe_d|?m^%a_I3M7)trCiK-QaU>KTn;O(c1py2P(bJ+7tK*!VGI$g(xvN2n?(GgZVu zpAGbZpy6$F^-d9s5ncXU@)?v!9)v{R5ug>Z$vxstc&mjpu)K&+;WYh#RLn0-Ben zbBu*ov09&T!RUr8et+PN4$2hfbuPBOuaYK6pscu-aSS& z=r82075l_qBB&;=d4h&ZID@RS%{+a^%1xOD5184`hU^Lj*UcB#t6^My!q)e@vKNaY z0{)%v#xRcpcmLQK`0+Ne<)sOSpn@$T*hyb|^qk?trg$chdI6cv5U!#{f|Zgwf24)g z2UKvT=fgdjVT3-2tFOzB^&>cN2kz8O zM>Hri1ufOO#A!6Y?V}*)5yqBGbDonUZx%T&BT#D<-kLg18mKRF>MX9DpLsX79raqHU#_1200EIW2lx@pREyxXk8UpMCjV-I09HFT{ltHj_KHa$*9%s4jTT?e zgNJ!r?3&nWclGrc-t9^$Gw_~W0(37uP&ED?=0Z~1c@;|SApD6hUsP<^jWb}rv>nGR z5Gk%ZgglrUGjn2?dCLgy8Odm1rF_NoDUnazkx=&g;on3#5=9Gy#9!A8)30j={eRbN z|KpbZztQ~uU6*q(jSfjJi^|WBc{bk%yA0F8*pS8|BlbhQ%&zb+%@5&!(fk?#yZGb< z#LJU!9_-IcpBIhwb6=_RpWl3yT}k-S7sc7cY!cQ9TO}Q0_IXCg#M6u-_JKwK*nrvV z*dE_<&R~m0mLXQFGvxzDkcUM6tt{)-EDJZ*1uIK&NS;W3i?JG5+YJ6kf!x9GwWjPW z;P1+PvsuluvMfhpKN)XE^%klOE5di4xH?j=(}Txylp680_@8Fs_C2Gg=# zIHyaTOk>Q@q1W zK%JB^lU4}s2*n8K2+IiU2+attxL<)-olu=rU2l%viQKZIx#UVh5-|5cllvHzvb89*HT4{eSYSTFaNHb)K;%J%wsI=fjY z(6cv0Dlu0OfrZnM*bX|F1T$Vwj4;4tUeeFxr~D@QSK?o0GoCA$EIPOPwNX!kApQjE9Vw?9 zDM|K@Oribw2jP(-&#?Rh#+%^t=!z_Gvz{p!7xl9GUx^Nz(OhJfDYT^ur}4#qiM#&` zZbp+j4D7yc5JO-4H_rd=f3y8kzge5R8#~Yo+FD!N+Q`@%8H<>I;T(s5?BNPkHog*& zQ9n045$n`v(XkP4K+qUmsSPfeuMYi0&o! z8hb-H_B|!Ry|qGh*4-}y)>KjRR8YB8T|->lyeLWU`l!J`xX}XKR1AMLOX5UCUcTJI zJAqocvc&AbLcBD2K0HsAwz9%&49>VzWjBqAT}hH1c`k67u~3=CGJQwjso+Chfs}|E zHj4yo)Q+rB*|kk&b9=C^O`b^FMPjPF5N%<)e^uVSLOQB0onyo_oqOCe37DNiE`~uL z)0|o&c#^0&-o4U*v868#EI6mqEVq>lms8?y9qHiY_#S(G|K1iwe={V#bVefGBk^dz zKvjz1k|8oX^%cCu_l#MCMLht+H7ldYh9R;sO7xeUXg*s7TW%=9NW}q8#AZO!M%AUN z>gWk8vFrhI#PrTf6f_-{NKBMPvMMK*%AYl?fOFM_*`@W^8}#JB;SpDmcOIItmI(de zEE97RSiueSdn5oOmOSCEK!x5!h%;99URET%o$err=Ln)@zK>DKa%>z%-giBYu6TJ= zwTx-ZL^gGk7N_t-)0SnCaq7H*{$iq`ADm=SWJaM~ML;-kO+M#WP3)A>?C5Lv5^F6z z0T9vA9pYlJKtF2z;byFzax$v_2o&~48Huw>+s76kk2k5u=6W%levTF27p9ubS|WTo z!E*+We4+?Id=?cYQzXf?@OTYZmKd=$xo%1bEt>l#)<2-W9S0ud?MUq#BV-fUF=PDTKmej~0$M%44LSGBA#JHq*1<5LI>T4$nuuH%d!f16y zx%*z}erD6)PEfPO(~aoIF3pQ`%(4KU2PzI*PRc?KnVoLhP%#3G&siNpDVi)Yz>jW?`V=NL}x8;8!_ ziUbQML!KIl63N>ZA>;lQGJ+idfiV*%Nz3OAVHSvwU@js)NJ)mGb{-k-2pGXf7EoV+3Oa+REia!c!oQ}EerNroP=Hvw>dO@tw4MS{PMf# zBGrgAO<%#-Nbe8)nWWpFmpc$`&ffiAZ$jw)`SzncqOjUEt;G1)mG`Nsxb<95T)Xwz zuGk_;=dYA`P{n9x9w1SjK&zI2&U=RdUOYXffV7h^~dY{~1D1{FU}X5A*Y{0`E{*)KWnj{&L5z!8i`eHCnxB5GNN$ z>zu`yCZBe>refB4`o@#Er@^Iv7U+bkcequ4^83TK`d0z3kJl&Aw=p|g-9*~Dv{;ekS-jq#;S5!9YweL1vyQ)3Au^nzKpj{n^IYsvR`{;(T(P)m<30p)s5`w;fv{Q zM{4UOXl6J!MK)_1(sr^~UpQi;Nw;~txq)B?tJ1<6=XEw0{!1$G7e^pEil56i(_9t& zl?se6GCFaLcM5OyGdNF`8T{4w%B`&^AoPc+wHbEyO$T)MjL7N3aACety?xx_pF|Jh zyvz`{4{lF>HUZYX9Hy)^mAz8t6j?oA z-nFJE0<4C*pqpC_KW=(a?{>;VtI@Rly)nD*fxS=t`Q>T`C-UKUl9aWus%A1_&NFIY z^U3VNQhgz%dLdQ9P5og9{U9;#LBfwoY-v(la;Z}pLVl{pG3>X#ftC&GX?5Sc_13io zsE?&K&JtPzKb{y%KdX}-5Bm7J((zsikYK8G7fBX5oqPNq#m%%U$de)1V@1^z+xwFTmht~tMB^Ls>QS{jY87>95 z#a#UYQ%t{W^?95j)$#10Z4!K?o*)yV*b^cd&F4b2=vBDMjEG5yC@{tI1W=`u6PdMM zySe+*;2hacGRWc|zPq9K9*XaAk9^+)nY_Kxx<#DGFr6u8y&egh$vB*0lE-il=#ZU; z$p))Fb|aOI;}-sdI-=-_uT1rl=ijlP5OY(o+fpPmoR$hB5&I7Ir5AvH0GXm)p&XGB z2T0T@#{c@WqBQd!sXtYhQ}G7=pa0!o_#hWsFXHnBA2h$B{d@nw z4oNE7ia5ro-kY8^;%YpSwY3cjm34tJauU|k3aD`^EQF}B;j{~+8^MLo=@Ugwo6bd# z(R+8mLpaqfH-lJyrN8Vu7IeQ{?le3%rR$_A0KE(yuUU^jk2sIGk1{vj?!O2~AT7QE zkmNV=kngFAcC?`MhyL8HbSW_hOk2@5p0m-iafy-=J^ubqkW`2JbrdN$(8&U3Lk^JC zPf~Q{4y~ozj$oF+*3X3!lN2ycw?hOVRt!A?{YxaKIb!oK6fB_~@69jGYcwuUs z1xa_4xzOZstNHO1hZzMKdZ7r?wegHCOcf;oa+TSeq=6@vPtdn#VGBsu_utNKMrv)^ z4FyEB9Svy5(GjW}YYwEx#l}*BB z7KG+?RPj+eDzxUv6lc1{WkBKdpg*9KJPjQ)x!21Jq#fAq6c7js$yuv$nWKVxwKyL1 zYN#9$m_GV;e6*K1S(rn7vb<#s zWts7b3}{a`)VIb{T@se-DF%hnqCLhb#-(sd;kxWU8E{KwbQhw_X@NF3O>%fMPN%%V(!FiCSNG_PRHB&aM zV&o=BP$%n9Y*A~tiBA?R&aSg9_iL6rCSdTgGC!{Pt?tAc?en)+c@%(SJ!*=q)OZ~| zO`IPUFTiA5X+j24x?TS;{$lxO=Z>3HnZxhmPHh%6$gK34GBC{&f2@Q&4amDiSPr*h z3V$=?n9}x9MzY<*>t2E5gl{Nn*MGc>JmcPN!7Xil^Lx_aX&mO^jyt-py69_$@i8*O zum<7HBc9!R_rMs12aW^c(G_w+hhB;h$`HVy%SE_iIv5l3iuu@;Neoc7Vwb2yFItOBet8sEao{& z(D(-)qt7zI!$qzmPHBw~f_80_U4|i=Vz0!+W!^ex@J6lg{3R5`!*qKToo+9hEJyp# zy9?a6R>$j$6Kf})n@}K;+_$eI(%X4Fv$TXSYJm24{s^-^IOMv1qh#?Jsw>;NRm+&?zVGtLi#pyqJfKMdYR*u-ax86{U`ksKp=8;Zm_vtP~% zKAF6n`v7+&H?o%f9-(eQ#ydiLT)XpO*?Gnbf|>YbEl?PbWf5G`?NsSBUydPiA7ywDar|Fr(1^W)hHrq9o=dx#5I^2z^Y+Jr_328}Y~0@$sQEW>4oaOq=Ru z0z&z}q%SwP9_=T8b=K?t_Id->Lyl2kkZWKFwnqsdyVN)y2t=1b_Cn{pItd*iVbYK# zjazu>3NsJ>FqTJfQ7yCxoN9g4pa4JHH=Zd6hkZ!Df@$2p5fNm;EE@AG`_VDEp}jM2 zyb#yeb~OLb2F9~=YSlHjL+HQW^~ z8eYv(hVhPLSdQAM<1%O$vZouNpJ}R8Uv_WmmkuiFRJG>tZGvYeC0n;JQU_bMJLM zS3$WLnd(GUa$s_L_MrP;AB9CLL#2z{rCS;6k*Mp?FOevN!A8#;4-AH-nTzvoPG zw1a2ebl+&a&GqA})af_m4GD`|feVt3Hqj?v`vxe5*lQC?Z8xtd4Ca&Xx-t2&48pD z?mZKez<Z%!rF+_A0o~tze%f5 ziwJfcGs|tSL$l2-c7Oz2*Xu{{@qNn&#k36AB$GS?rV~P9*>4H>2wHdf*CI}D`YrqWOct{Sdsq1f#Adni858b)@nHKO9XCwY-nc&sRR)?%yGNg)2f3 z2OThfiUlI|<|B$zJGjz>D06Z{8PJ9rAyNik<4m{?4HU^!Fbe6UlVNBEtI*vt36?yy z8C?GqWX9U%B9)h8l^i3mUzmj{(`kTR$YX9aT5*@>JrMY*DjQ#JAY*3oY%Px3s;6dW zxM{?XFQDG3X8{nl3tpg`P;8ZAYEF?#$k;S+42Ehpm(o@vouaNSM+-AHJ5uwgE;Uk+ zt8bc~0Hy?0Xe(Va2S++9liNr>a5zu%0^d?OiTz62a7wsR35U?yZvg4jr#)ab)J192 zT4MKlh?9|Z8Kk8sJgU!>?6wNVpDF1#D}O{eBY{FKy61uUUOH{_)ZO218pSYZDv7PYJ!+|Np( zXo{kq_Qx8)7lkziZrcomWA$6~qF>>iFV%siKn&Y-KJ78W$PkMGi^QFDdZFy^KX4yY zaNWN3&%Hol^Qfoa6KJ`ZzIi24z)g|5_yJt1wOY&NCucWtckw8^zS=JxD;i0Y|UE88zE+HKA*RZbq)J&xF!#eMx^Hh3>Pe-AzX zdOUOa{W+jGg**d%37E5R-w@|@NC?uA&1ZY5?eHe5&K^B8m2vNrww;~Y*w1U;zlU*q zpB1-lKUS4-ACgwTqX)jRWbc#|-5>WjQIfeDTeY1%26?H(vHYaU-x@g<#Cp$&{LtR! zahUaKp^VXSne&+)s{14U<2~!+d-bQoY%%cD)}NRBoR5TWo@8^M;>DkZp9z9KGR0FT z#BN?hmf0i6%qvd<+tItA2?ci;?%(ErQ%p0!8g^p~pu}eaTyW;*_LldocwF%^ch1>%ZiYt3X;8}-Rf27DfuF5`qV@hinQBUY@gFEqlcGcDVZXe&mflmwq+0;{~ zz`cKbX?`Ix2G$rzTs*vjch7%E%G8WKZ9QF}RW8>ox3JrsS=)~S+%Xul>s)S})1Jk= zfEY)8bWuELP2|dKotF3URkDHxbcuO&uPq&&8A8VZSK+7Bb#0v9iw5eHbu;oD&M^?fqS_9jtJPOYQdfcOs*de~B%4Tl-d#>k`cW!Cg+ zNU3*Dym#K`Zf?rSVFF3p_=1(c^ZPV_^MM6frdr)E16$?u9)moQ=^8bBEqOR*3fPy@#~YB{2}Q zzPLC;7k8cjX=|^b#lC@P_gkyOdL0-VY#-b@3h7O;KDqW25NoiBINnL!$W`iECpf^@dG&bT z8%!Z5J()5ekhGW!DqXC!y=fDAzIb-!xGLh7Opv$s;6nxFdF`grrXEC}osA6J(=~B_ zJp{>xxvP9RmA~`*mXPI#9@gxcE6>b9jL&t|ptZlbCV5SlFv` z=jCJb!)aAjWB|x1{hIQ3!J!)Z61h01L3|hJWP(bRBHXq;3|a1n8Ez1dxJWL2 zln7%kj3-2VTH3ljES+QTCdDEDFGwPXq#Yg&AH+3U?L#L8tz;EJ{w^5`P#s=Me!@z7 zojyj2!k>cZiZ#qbQzPysB``sHQ8_k8lpEr8v@wAO#y64vGtRYkR=EwYa1POY6i&iE z9Gp7o9Xnk_CCqHf|aCMc1=3+0aTJn2MhDJNq+P2Ona( zEQecFg=1@XLhgTdG7&TL{k4Fph>zQ;7}eE~Hfx-9duNZ;@bFUhf@*UBeGj7^WayEG zo0Nv=hS;Os>c^sZ^tdo?<87DF3orwD-b z66ORMy(Kg*2V^`$fd#P$JczQFM6G+*r#pNlnqrtGBxsTM9`EcC_W|*jSfp$|7S4D+ zVtD|qJqRdD@v;r(fnYIJ!u7QFhC&+fYbE73aSXE`G6J(^D^8qmn%t$d^8_b+DY-7x zP?ZbZK#L_AIb-7Xr540UR|`bxA~9I;EpUsX1;@qA~fXpa8Zr^)63qBcA5dcRxLEm*d+GGKFKf65R zICUFX5Zu0lJ!l#b3u|N2&$om|{L&)IOSEVR`@xb$o|kw7OrB!tM4lalp>oWK^^+{$ z(#R=_j~3OBy#K!A%cCVgX>VT`TXp)&RMlK-;)rR2JNJV1g=!;jhTmNy z-Dmo@L;9zTPM$q2nby#~j854N({#=iaC*w^Gvu2Q@1hw79OD6AbE;W~KMqTm zFMXtVyjnJk4d~IdQo`WGdY(P?mtzuRS;k=bzKO#_sS0V-nrq`A#*Led4)JWo!jpkc zaV}EH(y{&O*}i1zs0?!FV2VR~0=tbJ$Wo=+a{N8YhsaKZ0?Dx zeC$=Mm;K_xrxMwCyb$LHy{k7$G zWB9hZ^|lc^_-6-$1+oqXV(}5N3u|F2Kb!T=vvgJx(&-sUlQz#%ENhmFXiH-x3(>IG z3J)TeUUId8UPNvejW5twTfyM=B=oN0RoI>%{Z^f99>8C`XtneD?}PRlE{|#1u=-em z!>Dz7o;E52Wi`BJV7g&!iZNq zz%ldwjl)az!2`rgoJ}Zt$zyT8r7q7QA3G8xi^!iwJXYK#b~i!H-J~eNC&ervzC%?P z6*xaRGIRa>;y&uZ<(^j@<(t_^;1oU_FitAsAi@0UeEqnqiroTRHFG5=JF9S2ZrZ#^ zdDt*N`pJ2yIjsqL8zzDV6szq}J)yogR#p_0BR&3|fW>AQ#0Ak{nwp?XSTjLAew~3( zqArVomQyUvnaFTkw$)8;CnWulPPf{LLkD?0+?7HHh8%j*9E*mSs-7_OwVbc}4IEiz z;I%RITv9hs8E&8_e8mM7)u!xZz`lm+9VllArFG!CG-+R=PGAanogho6n}2y}FB@b1 ztaohPqT6{Zo9PLpF|Vo-ZW@CW%Hte~)Hwu22o#8>mG8y67mFK4E8#{WA(fn^VifBX z;z3EHdHZl!;|X{fmK4{pISx^VGrQ}QA8tKG-#kCFbZ>k5xf2~=GQYF|^mXi;ykvc8 zG;}FGwuXdqLnijIr-m@~F>>3t)h@&~GoiB;p*7zFI*VM}X$fr~)@d=q1{}X6D~?~{ zUZ`L}D=a}T=FAj=xHY%!ADyjt3@Z=G_&J^hq!|#WzbHr3$nu#}vx?P|9*AN( z3tR(D5b(UJU0L*YD7uk;I6_Dx3htyi`W|YST};npVzYwPji z080x~?Dwpc=gc2K2FSGd(64{~;CQP{-2g%hmr&giLd&wOH7sUE(ps8>Lu^8!VqHu( z?`j3Vkv?cMi?x{-q{Q)H?~y~7>j-Z$oG2?-monb&#NvRpVq&rjM2!XtGL+lF{01Mf z{*@4NFa>T+E@MUEw+T(_L{N#;h=HIOS8{EvL9MsTIhk9WJ^a?-Zx|w6aa&%C|(fYi;wMwBRdX9X{*2?HFJ?LU9IB! z_?6YIcj=uGUWZwnP5nL!lbn-uk-W6GKo4vkVlh3Q8)ar!|Az-e;m8l(?;!>_2T4u4 zPs?;wRQS#^g~U8ggCTSY0ORzN`6L@l;kKcRT2e_G#w^n{wcG;>C>r>beG4eq4N-)V zA`R;5aSEAKqf;a1Agkn|osl1WN!_RRxH(jv;R3`ZeJQ7Mbbg}`Mq^d*Nx!}&B~7!O zgTb(A%PxfK;J8jzIr)7?*1p*;<@EoE8QWvAZ$q{_+%LMq4C0`VUfSc~J0db;jWGeg zfW=DV&)5;aHR|8J)3%3|O|R2jFKC*C_+6qbKbBUqA>*K znH&eWsmT zOw<*lbE@wS$2*Q^n_zb64gEdKM>6|Q$erj%;zz&kv?+QNPqS1mJ&j@-jVY5R^s)h^ zetBW^W_^WTiZNQTI-5iAz|cxEgKUk~2xqb-=Tcn)xhDQjY4ZDdE%(SAO-K(G6EfHI zL2D^yrwB$JADlG1sD4Gz{TorxvYJQK?C}|<`UzLVvev7w%$%r--Ew5KDUPvV@Y=Dh z9UDlzmaI>NJlTHO?3O+|OnevL<8GkI3e>2A784B!g-ZPWe##bJ&Aoe_J=mQCfe4`L zpjgVd5YGFcX1upH;ce`VwN;!;48<{O--)u z`1K9Q3x^nHt!Kx@DU2k-8>V$8)`M?^xn$p5^$#qC8#v`y;;zX#c1JVyCIsqLK=>p7 zz-!K~7vBvJq!#!P;Xtjb{Tt$)b=+sLNq{_DSD}>6K~aEG0L72$6%GBd#sLQdQTr-d z`~=CY;#DstcBy-=lCFJ*!l8IQ*J8$bbx^5-aE+meDmfY4)bjWi%pdU*WnR>vjg8RPbF6T_8U+3pk(W3 z$|kE@+4VX?_LgTWEvv~~buD4%8cP)V+=y8E)z*(d+O;hDT;nyzaN0G)(mFp~bYzN0A~auqe5{ph zOAQ(xVGTi~!Rosko8kR7vX54nh7c26B75Xk9_OMFfB4@6wT3%61TGZQ*{Lkw=SE!<$bKH^UEv{WK{oqwM}wAmJu-X1`jPLJs#38Y20@nbo13M)F)Q z=qY|wqdqjAemVsTJW3u|Aiz+21m~e`sJ_Mw_u=k!;GfuF2`>M4fBiN&tsJB+eT%jj zvf&Chg2v;OTQd9PfCfA5+AeN^%2CI_K$@xwCs$m(o+d89fhCn&8d=gYbTT?j_G^%! zWl(*;-Vo_V+~nj56B8aA?UH(+$q=JJxlM^3Jf0W%Qme=DG4z>DVco%OFQ@4?E`JO z&2qa*)9qk$yDHAMgtl#-vlDf0c%wh9(L_xG z#tnCHygZDAOB(RX4F0j4Yve`P4v@Aze0dwMekV(iA<5A+gt&DY_>4ndqx8?%T*cak zs+k=HV_nVht|pS#RpfIn#O7*i?$_1+F0ohf2eg3SQk+HBKOaa?#!jKEtz;V*qX(qF zX!(dXcDz&F5e>5Bn7&)u^wx_6A|xrMH13D-17C-NcDtp8;bXG zJdPpHuJgt-^t>A)QNzy8S?oDCK2!*0D?di5`y8dgO03vR`OL9BJWhB8D^J{OmhE6o zXe-`VBwIB8)V#Oqlh>xNSzYv)Q`H4n)gPvHA+>lvJSu<%C+bvnOiLhzX|2qj9zYhb zStdzTViPj9zbma#oxldErVvWuVgf?ObjA+o8)Iw4w_pR(4^kIoJcjPh2BW2Y`eSn+ zl5^AN9wI>m3&AwY@Sl?BsP^gmIP+Si`!?A;BNT-_{S6zW@n0h%+GsVR(JGh=rs5PP zZ56-T*$hj3+k!)6&fc6vEiQuCKoK6A<{gD(@C{TfbY1 zmR@!%ksg$a@Erf7h*`>15&0f#BVy)AMzFp0g4VQd%)aiGL(E<2HD|F^HM*@iS8LNL zEARr5cUe~0vCck(De0Y#D0>xAeZ#oP9YnmtCy`(nh~XGf9=vC$%Mh~XVG2b)3??KVB^8@iW!k-P~0cgXYWDcIT2gkSu;=BXr? z>IyYN*oJXM+=l|ZG4edZWym=OM#Q%;|9TfG5+cF>`BmnXaQ_#7!+*Vt{Ik^maUUsE zwRAxd#`G={PnZC-vgpB3C?GOQL+`)~@S9O;h$)iD?UUKeke#0BPvmbKceYD6FZoVwc4(*fmchcaW-oTRjEXT^}GhIr8d$TI>JwuQ$(jALmh!F zgYnvT&zg91HI?&G?sY@>`W+;Iy*P)cjh$8LHyP`Mi?zsHk`v?2Q|dU2owWF9C4c$h z`(3VUdudE*{21v$$3i~f%#UW>Yc-*?OxC78i|0LS=~8Dig`@YVwZPh|+BxakyYM8H z%vsJc&-f6}D_EWHz<)%Wb=&*6`GE2}WKc^ z&;P(n@V?M2-C$|1H#@1XhrRvjr#)6L+pQaBDH&w-%rAc~65d7?%$rh!Qo2x%kaw}f zFFBDDQ2Ozy2%!OqzPJva*QokymkVsmFb&!&b4YG0={b#!x`^J+XoXH^dod8zz`M}V zKx2E`B5;M3m{~Z$=|K^f8Mrg4?CeLDE>^il*U9Wn*_Db{8;1D3F3PcaVa%#CQ~7vp zlnVI}`4zTswtHL0_t3t0%IH_A(8`n%d`eZ!G-+#ltg_86YcY^lj4%mH+#eUGY)PiLyiC6~^(Er`4o20ONuI`ixo6QedZw6r~pFj-*rRQghe z&BVqCgS_F~GH30ubPFk78;6RM+X$$OitCZgCU-NMBB1_ zH5i%?9`zTmI<(V}_ekDf+0;&nov0nrX}Ez{5BAl)t33(d%(lQ){{ngc`scaecpz?y%D%S5mj=$&AA2 zSPAm1CY<^Pa9rd`u< znzFBK?PE6~zWa?eOb-zIhUZt*mrE9688(-(xlv%Aqpgjw-)Tz8%9#Th-p=QI0Xt^p*HK_WZZ8c`OVcY<74#J2^&wif0Te)XO_{i35RM0ZA^ zsOI_U-qBBz6s_Y4=(hNE)cCA!`N0&>s+^OD9awt_wxCapVl~tL83ZkQZ6Jv; zvEk&rg|Yh#Y03t`PUK$Zw4(SGQ9qHm>KadtnL&S0D%M=gAO2U~=3nT+W!~!j`sFn% z^u-}#{GUUQzZ|lE^x`DNF&h*{)Sv9tx$?8Y$>^i?V9;(XSkUJR&WhnmsH|vyae6^G z74v3HhMf3HH>y^Az+nj6`G`viMKxjtCg_i=rjwjqT}?ip&o?mL@ae*+c3XV`L`<&7 zy~!b>`1mMP#O8^RAm$=e(VO^P->0bs+<2o!RD{?N^@(mVeXs=dRfR_{k?IyS_=Ycr z2uSry^NHLNVPy;4h-CyslDX`&t-&&G-*CrQ6j_N_xN5PU1R_34?;5e6EJ_4*Lx}zu zS}(|X#SQxW3jC9r%hr=%`lcuxsM2^V?up(N{%&L7Y~Zmd#%QIRGbAF6GQE%zg(9hz zQAK3Ll2ZRec|s$fb?2FTpa^`nrp486G370L?Kt-|S48c&l%K@v%a!NKv>njM(mOc# ze4zul2`lM5TD%+xI~hwEz3<>Itvmpz3qTq7&OJZwC*C7tFO#tx#-cRQH7W~OlhMl& z4>Uw1Bdd^!0+!zqewN7m9Br-2XlHf1m=C)V z9TCaBCzW}7f0a{BX&jcH52U;=Cm1AJ$Oxrco0(prBTJp&5ye*&4-#dsWo+SQB?R z@%vw2_89o)rlhZsk)AIzrT+)o&fiMdF;K6NLHb3lyCPR-Jh_U!D_Y7RK_#!9qH7D6MCpahj3>(586-euJS!-DzfvFSv z<0EfcugituWd41`#x!pWpDFBBISffSde4>E?6rn32Xu7sIym ztd5Ap#sE4zOyC6SAe`Km-NHH{#rb+U@G#(XBR9q%_iBlCQztD3IZ&!_p72vkyI>t^ z5mS4l&rkl*mb3F1g?A}LPjyzm4RXNf$2L_9H~;q*&y=+rCrV9uic{sPt;@+)!=4~b(_W8sZ&@t3$dr=`WB%t2;aB6GH6 zv4d0m=pnc1S5P@XF+Zr`XK(3c6Z73%#;>AwS`e8*e5!HT)k||rCTmI06-sKaK?gj} z2Xa>1#Ss|X4}O9rZ3>mSf^ zBlft8``MvCFWRPzY@<|3<(%7vV#w6jm}x0nKT?uuHQY8CnNT}RoY`1>?#Xk;B*olp zLxJm8`K*5Ae}qy$jL7-B`QBUhUAgm2DR!Qr^-pU9-@6cNX^yEqG0~h1kE(-Sq%q88 zqE~O6C);|$#cKhWUlYXaTywgdY2{t!MLI|5)z}b%6iICi$rp4CLw%_CQNHIg;`Ll; zNtYs%b~5-KMsC;LpK!c%BZyMHBl^SJYM1lmRncdWm!f4A7(sWjO7R4 z^#o*5>5k|#;0qj%2{M~)j33_SyQ)u((WEH9du*LM^mo56h2bVG$NR;4qn2;h&CA+{cg!@_O$U_q~5SzrS5jD z{U)Q9+M+0*$6@+;w>gqPv}xC6dGA&fsSwbU#|m+k0B4sQ)@1LI7xy&ob%SP=)+j&G zMwIOzuQ7EU@M1y;=YPx)OIufS`xl#ToRg}1CZ+^i1SO~65}yK=Ut@)?{YjdDet#1g z3u>(cWviOFGnq5@82ydKl8k_U`?BbvOwuuNb|mJvr)sn~@2o>b<$U*{j-H}U&1QnUla|(SV(hFo9W1yhD6o>g z1cHF*qOF_(r@X)>YHik1IP0X?;80RB)L40QV?n|}bHP9}Eil4u%ESV+9t@%V=`d+~ z%m5Ndopc3e7`k*7{`bY5X0k}r4_yj`UBqmHonvDQ@Rg$7FfPQsFd|@K&^-ez4Q^}= z5f1!?6r?aw%Vhn2kg<}eX2**nrYCtG0>Qv3!R0`XJka`Ib^ydte1G%&2(QE!Oj1Dd zdJfEn?9%LnyC$Q*>gj#(fe2sP37Evr(D)n-M|Agc+iyegX-i5H$`CD7aE*&08lx%JiU3wN6!L5F)r6)#t#V=kX&s5Kdmn}$VTrZ#F&NKTeH1ubqI1wq zaAcc35B=8tpuzFYH=)~jO&PyT{2pg!3mbg6 zi$lGX22`@J_N0shf!y{IiBMI|8iOsDi~|Fk%R`62QAoSJYce-}CcbeIL4|$5KSYv8 zJlUn7&kgfk%TW|jqPO+d)=c%4oIeX4F9{Ig&xs4GlAu51WQk2z5KgdDO$>ou&6^S7 zM+Am1m%H|&bIydVH7}^^nA29}* ze{j`2mpTGX8G={a-C-X-RmG{GW|ITWahuED-U*jDCpO92Te`2L7=@7jET?q*r6U`s zfl^Ji3Uc>}%p<%SaFrg%&h}RabBNV(T=6bEga~*`n2{}D*=3EeE9JZ9}{@)n7Rycjc zZC{`<|No7l`}e8;7YFQ4r4bn%enc;01pITvhgd`v$?pRsMxYWhB3(lJMnDGk@SLng z#_U*Ak*kKLt>mA5K!gXe2*aYnr8e|PVmVlp3?V0{s?}|ams>vGAHX$120-dGZE6m+ zQo|P|d;@5`BM}xQI9Z`Wos^wotlSIXp7ei&w}a#>G)^<-H}!{7AYhYh$#L!>c6{R7 zw91galvZgZQcB9-AAeLTmc|T}P7OP;R>1rwYhZfiIqsKUAnOoEk(6#!#yw@y!~xf5 zRwmO5m}1lFb$l)~U0T z?K}$S`bEoW028+YxoJ-Kd#^@34OF3~xz}iod*C})8#{$yI>Z1aQQuX{Gt)Ls43dsx z$LnxO&qh>Abrl~5jmgfR-`${kVd7($#^#Npf?2FJv&4}%@*?h$_9n75ZwP5dMx$*R zCN9jW7hIl}jCWeb73G<%t2H)s*3U@IEuW4wBD?@M5F(`vkk6~Q7* z;v-<1C?iNn`?KdL0RY#@sM=t~Xr^Rzp`20w#Oo{C z;5xi&<(hFge=sUro3spGy+!EyUdkasU4zvm1Hc-wS=3ml=PwS2o8qz3qKJc*)~tmq z0~I)GceXM;xk$DmUo)S+$$I;gYyT~jAU`P$a6AhHS;E1C(VZ(;)#sM)awVK#TKsUE zfhcN?rfNE)$Q`${>k^Jz9kGXvNuItoS@E=8g!f`b_Y%BwVbY;cfkdkj7$%Bf{R$5> z7OLf|U3`IGZ2N=@q7Mu&nY$tl@8-E@KM9PA2*h?{w=d^Qt~5S9gIOQU)J&7zY`~=3 zgEofiMPI0vNuxEW+IgWIhcmD-$e{AU3f64->HL|9n6GBxCA@RH?#fBsH4eLU8(g|` z9~54jhR`M`n{}~mgYNS0vnjUfef<4bS%rXqxp)5`W>ZHe2V;F}a~o57A!8GLXDcUh z8#`wwF?|~&tAAEjVUo7uSMpHs7ASqJNXB&?SQEUV;PH_{Cm6Q^axM_@A6t^`=vt?a%kGW9;8h)DUvPiSdQlkqIOBvI30|LFBZuUy;cKTz$o%6p`_nSm>YyImTe&W|rv!c#SU|gPS55 zo>W>>j~1iTmqupO7+ICKlxcUO%SC=Wezyu9i8Mz;Z7i^W(4q|UCkiLkuEmQe7uH8j zWc|>hJ(F3bNcttcu)NgT_+3AkPzmcmLd!6AF1V*Mv1*l#{Mg|R+2Zm_bCd=>423g@ z)>X3h=XtM>9z_Xob${n=SwHDG6R4$N@QT?v^2-XVsD%VD1j|MP)C|f5ZW+C@uSW&DTYPIA<#TljY=68tL?OevYOS@0qS79uJ8tKN2Q`!hBlr~ zK!kb~Q)H5sTo=8K7}i)5)gmmO@&6<39pgL=mt?{2vaNsFwr$(CZL`Z=wr$(&vTfV8 zy0CTb%+5J8_s-65cfY?M-;5_SG9oh61K^WONIRwuK|nJ}K3nXugY8oZHmX^{#JQFxQnOT4A2cmdoP{ zg!Tosy#hvW!Dp9qh!oQSGOyPeMo8F{}9jOwn0owSsDOtBk_W`P+NA9Cs+Nq}n$j7QrV z`h&9>>@YGJM7PLr!hl>5QqIgfnE&nda5nuf(xCDcU!cIL*B@x3oMHH;5S()Uv>r;Z zh_vDyIK>;Sd7)N$p%hApdz)3J>m<>hyV0|3v1?*}$rgldd`V~oJc3CuH{`VOYF*UO0uW8crWH=G_E>(iOK z{N40u2-UscMl8`FOU~NE?IF(C6xQY==B>T-_u^}cHcY||Z8{<9tEUhl!R3u2lh;i_ z>jyLu%wgs8M@zJBCiw743{7-AAywu3t}#Z1vH=`rqoj zG`kxU8z+zJy%Gccfow6w_mV^MeH3f!DNo(1swh%sCWB)Vm81VOB>*!c{~cFE)t0W&8&$g&Irc)kW<35ql0;O zc7U4qAvi+i-Iqp}`ufubh*4$)7E?V9Aq~3~Hm;WJ5C&n261<_PDU;boJ z`@6b+9RW$ep?z0)L?hWrB7gEoukk|N1N-B@NcsP9KgDe6r@_*Fe_dA=4O}7VI!z zc_#lT0EWCtl~5d_u~C40rcr|G(%2EAU(&SxNb$14>eNc_9B*f}iJ8Kr>S+@EOXy2- z-6uZLe)LSUvO9D9^J(k;^7_K*=tJ+%2b?}s2Z45PV}OBo;mlcfWQ@rM{4O%L{7e`C z^e6Gajr>q0kVq>M^RLG^IMfWF=alI%kaBZCa#B+@4e2tDvan1QjDL6-Hlq zBnv=KlKuEH^=hRssI2i4g{3PWi%YvARb-M{O)~wQLz^x`RYA*p2zGn^o}@cp%QKFP z2_=)BW7Rv*1JD7(?(cG#C=I!k7kkRY=ewkvTKN9iJC=vS`fO1cr& zh6_5*uq&(E2{b6=S-pvaSzciWEm_Wj#%W}htmLB$d$)*@u>Iu$5YmF{Bc4lDp;9{R zbe^>!1Y!>)j&*Kv70Jh?&DBkxCl2DMbVhiq^GR)I`^&@*>U1NGAjvgCMF}a~lafFz zy-EKQP9R;HVDP8X2AObTIJ~{eAVfapQk{VBWtcbyC$YYYw_QQC1aLxK*&da#aC*() zO;k9wBNCWwa^}M%5@_iQ+BTw_$RP58vh}VPfUk50G=du@oV)7WytlbEG5{*t2GYmI zQdMCfFTJQlSQsUjin1>RzoC=+F2v(J$ ziV-4rlf|PT@}ry;1Gc>GhAV(U>cLMYhPL=U1D#k#SU}W%GGo;8 zLORS#^E<+`yZ(Gx4|VqQJE@3V*R&$QT^Ga{z&lwOkaeewU~Y@?^&5z*Q5nPQP#w^B z#|VS~!_B#e z(zxQSU;MOA?vcA7$ZqiH)OT_p#oaYa9-yOEQ8Q0>#u%dR`stupj7Jf<&+2nbBAEJ1 zw1Q?rKi)T6a8yl(1aP%5+|er0Ah?412E6s-^XEV3j(=cBsj>F+(|2=N`%O9j|G_KkJC2w#T8-evNOLk zYP+wU{#?E9&%=K~=`)cUo$%*`%RW@Vz>eJJQa_hr2y5o%fo;SHL3LN`(T5oQ-u)eZ zz5ACz7gzo!Hvq3xJ>=7qo4A8u{5LfKRVoe@NU2u=l_PS@aPa%%zP{>VcVO(+`UB9v z21#{1h?mfU_>54n>i%LNl$7kWr6e5Oln{K0`!Oi*>b-Ih>4yiAlaW0#kwzzCe z!99*R^P?JY1(C_#R7NWl1;(E3W@{nbm{0s$50O2O-XA;s!EECe*mci| zGwGO2xD3vw7UzKFQ55cJXP~ZiSEgb0d4vcGBMYiy41r#w_&U&7>n}x9m~`J>xOUgt zop$|&J1sUuRzH6!FQMq6-!@Nx=s4&P&6Z8aN`3I%{^pG0?lcHr7X~Xn;oncK8m@oq zQRbh}hs31q#8F6?i?oj%M{cM&)aVNK=zY0P+5q#kKo`@H=f07wROGhb7nz2V{z<8U zR>NG#_3pmHhx#tsZ+Dv*uw(JNhw8XDFZL3-Xkl*Zxo|)Z<^w z&q!!Y-X6W6{l8BA^}n@`!8`+7Zzzv4dTsl$Q^UUA5H3u*oB zTYG7z8;XZB#NM}gWuBke@Q-6)A`)3;{~N8d)9UZi*f`$YHt0?&2T6u=Q@bO}zX|d| z$|F}gxl&CBJ4FLhe5|5<848$?*&I1u%?L8qvUFW?YnpB=M-yD8elO?A3Dv`EO{!S! zE*;xzX3PQ0{NbXjJ~Wqie5-G`n-H9Plef)1)wwj0X27(TCBDecM!Bjv)^Kmgv{P02 z)mPN4TViZ;tqFsZN5HeO-C$>*>$mgCyn8h$sTLK9VsKt8^Ny^)8Kr8x;ZV(~*RrWC zn_8$HHkv*?yz@YdiMwoRm~EZvCi&aSYEts5>|&31DRe&ZSVZfY8**@EyawH(=vnpL zaxHn%aC}wdolt{6?28YqCN($YqqPHNgq;kROG^e8ip{tUw z0@svXcvQ`k;<*L@u?WURH0Bm=_C&kH#71w!7sT!-;Cxpin)u~mdXE`bhjYbUC-7nl zvAwuLPQ)A3dv8kh$2)@{XIij>b(R5CQd^i9k*0ujmLu4lkRy0axIVI-f*f6N&Ze_k z5Qo15u|Su)h{wFRt2GG@U-mgaVtc<7F=Qn=iytD)Fb)A}(Ng_^XGKK;pD{r56WFFm z*$`>WWC6LFfHZV%xBHKaH;hUTf#tHK?n9;=&le%=)}=1bhz4DHS4_Td$a zc0z>&^(#h|TCMX7R5Gldu+jYTn0a}3;;i=2T}104}zk)5&R31{BVoRBra_K z6H4>@uJ_6xtQp)Zk<1>6&kbPL*fBP)TiezgG8I=%8A!=HFOqNFYtjebZ-}D0$dDPq z(7N!Dy5#8GK->W|&)=^YD-wWD8&kvI(fR!Q>@v>&E{P9EL|9fBlZ$spNeq62W zvD8pMqbt?xJ0+}3Bpl*HNEpJX$5rBs3)Kl4U3G0uxS}>2zjvaI$x9m7;hQShDT0tQ zca$}E%A#zi8asqeF!ltOxZY&1&Kft&H4pbmBTw$o_F+>zTw}~@lQEXwLxK}9_8P*X2L&CiZ1{f); zmx!Gl;atWa$rxMVTqaM=5YZwxPvlz!s#}0T9fb{?{&5jCwCnJ_O@&Ueg>JQz>xi9f zh%GsF%al&T(e)Ay4kb5;OThYQNn?I8%}H5)oE)G)gz3;Tx;hF|m3L8XG@Fze4v>5l z3BJyj@(?d;J{oB*)~GznG@-oITu(AQOGCG4v9{V@*?(*}+wMzbXQ4Z45EmDLuK7p}7U$>KbYQ*pls&_MV-K=z9VcIX@{T6`s7HI*UG9jm}uxUcSzd zY{4ZC>bSt(@8<5W?<$%l*aRTRLCOF!ny?qJg_aH{Yw<$d9}+RwqF;ZW4IqH~?K zlCPZ8($CK(8olaj8>w@X{G-t>d-VEtB0@p)hA1?+hd`4^bj!$Zt2wsyW$zmoej)fk zhB+-Cl8XV=wIvBWW~9T*snUg~&UC&q4-MfWAEY)ns)u13=%{e0zEIaCp#ogXJ8I^{ ztFG92Y;_s=%)(}OwHa89YyB)ii`MVdbCn++w2P9}|L8c@zu^k)XsIOaSBUrrsO1X@ zV0xi?|G-nWhiw+QAppf2>Vm2sq#}Iz9T2l;1ogCk1i@Rq!|P2LAV=AXH;l^tt4li& zx+iZ(=|t%UAD3`rttjaZRF3C3(^CDcGofg$-}1|vqg>!ivYfU=b_qN4vb4I9RO!*W z$sENc;U2xW1l1cXhs3%2rW3iE8}EE7-r%SZn9S|(we3ie=&>H%o<@k-!(=(ri4lsb<4P}q6p@g&`+s!k-Z zih%{mUqI?8naSKXD+K`3ZCaByhRGXY^HkupMT}#nrScufi5p{NgC@=iMo(EBrp|zJ zJeE$!37m5Lk3!`hrJ(BZnOz9jd&<@AawiwNl;RjD+})R4?Dcis$|bQ@!Wyrv#pUNA z_wzxW9h->ODG0>or3$>mwLjso+NFPr(IDyac*z5%9OO8eHpR_9bHG|%>dAOu>Y29e zHmd)2<&Y#B%d96s$GSMof>Ahp^c_DHn~?a^WU$>g7<_ntH9V1%O~|CV^3r{pSX*SC z(@yk0J@C5_>dxx?4H12)Owp}9mSS1393(W$*|xYnv)$BeQn97Hh|cGDC1PvsWgXrZ z+G&TuTNdclRHW6P&NiuW8MCP74`yK$S^Z4$Zh+(FWnL|6Egvj2%H zO>}766IDjoeitV*Jy@?|xaw6_H=#snI!<>D?78IXXx&*C9;K5|+$2^WWYw6VL`Xy7 z-iAomZAQw-dgfzud~)~Y}5Hy5VXnBzutxiB8e}s`A$p;_Upy-n1Ps$3u67_ z)dd#z4vO5Vur83IW^WZ@zVH1H>ntmzWTqfFVz!_RAxD^fz7LcXse)JEa&5w}iaO8? z%-M+qbwNpOK({~3gG^_-MR|<}O|=;M&h$V`EZ=(YrnE_HEnBTrOD*ZHUnFB#f__4r zP@3Itx$rtley2dFY<~46vmuBLk_+;5j)XkF!O|NfHX?;#N{XzJ#=PT^b!jgQCVZBl zlAI$LmAj+J{1`j%j(0TFg93=HM z;v=2Z+nL-;ueA|B>PFE1`-Q@I?Ze$M&dEB&tQ$u1H6!&jzU;OTmk-|h$?}RUUysX! zkhUCOpUeaD!@QgW_y^4J$?-eQ3p93#-mbtK317j~j>20M{4pyS`2fXcsBK7P&X=8W z1p#-Er|}FCMR0%e-_$Ys?7%ENdz0VOwO|rzlhm{}F!uE_d({-fljJoq3_a&$H~Zl$ zCA9`fG{_j2Jb1A~(Y5&XWgOm^4g%ai+Ff{OAaPq=*MM?;;*RV);0^0JIuUNjfKHSD zB4&uzn*nv0sYOkXxh6AY4it^%EJV++>%2t2|pi^^()`=yisW?*Up7NYc7 zgq+b7-e^$Xa4=7w)kXYzMBk8`ow~VBrp4#=W@WyFLq0=v-?+d|^)lPd9N<5&p6C4R z@pwW*Pd&Ji2)S{~dyUBh8uB>b8BqH51Rr0-%mKlIu)dnl1bq@pD}lh?SJ4K*C9c2B zEHGL@!C$2_4-MnHFeNsvH9eNj(FIYalfSc1v_70XusC1&h8k%`_P}=jOKULqIh)z| z-4;wx{r4K~f3nH{8)f%z{9Km?q_^^7+n3+!m9gV$A0`?Oz2S)9kDn~$Pz2)i#Hl|R zT6bg|A%PRp9MJHaDO(#|s#Mq8t#o>HekS~?TPsgmx>{DaBhko@tFCNzopDycH#iSVL$4Z112zHoxLXk`} zE63V_Wx_G45I5)8bGRB5x8a#_5CKsa2GNXZB5MH;;}BIaqiV|Vsd;BKsX5auRYIY7 z#^@`fNOAnnw4L6pyJ zB$E0CqD9?IqH%7;f)B@a&)L$PC5!FKNpqp8`A0OHB`kLZikuqDR(>&b&jUgsRt(3wdNU7)x<=1B0nMaG zR-rW&GRVO7^pC`%s}^CSwFA? zG8d(mD)F=0Czn(!81)_Ol&A$!HnJO>C6JdpSzu+bFs;&;mZ}h`rAjcL+?1O&ZL}`* zy$(q$cu=6!*48hZ-6!y%)&v*@V`X#{{4!8W8`>|#O$u% zCl0QcSh!{05=?l>{HL<8fioxBb&$Cj=|$-W==U_JIiexSQvHj9hm&?7X@+$%m{WoM z;7qypQz@8#08YK#92B$9bs#*LaCU4FmclD)?F4=WIG^X**>46FLx+AlKLkjZeej4{fCbbJ7`@rmjgis^C%rHGm@vQ5OwD_!^X*Rr!UO zI+k+5a6dx_Pm@%jZv-ELdX_XXvLvH1l8LkPLiAuiOK;VuIQfbNIKX3`v4=M5FE9*( zSp*{dTZ@`q0ZBt&RoOxUWIAWen!&gNf{P(|OL-QK{TZu#+ZYrwX~*XDVqBTK-jy3R zbBBxs#zx~>xJ^C;@Ma!WrgIo5Fhe%4@~Y4uYE|IUC>AHTX1XwEd5MB~gvHl_^reRU zMEkke52<}DHqN^!82DIqd2R?XFxx~|SK_J4%c!~i4Q%`s#rWo=>NT_&Ve^==B7gaH zv@0$)+u=)*;e5?9#NcqO`7?OUINFq-;2->-d~uM6ddO2p{Xcj=)a@%N$L;RiK&n!a zyS?DV)P{fG)y=g6WU06(#z};DgnAuq+%~P6D4cIIIh;V4Bu!!c*d?!0E4Z$3<99+P zW505)Qks0CS)H2FmZ#siO+GFPOnk&0rSf?+W@+(C`wRHaJY?;li78WCC{SV=SJUgh z$%h*jtyM+`DqE&o%x=3qo=&?xPHnqA4d;wjWVUp7n3xQA!3?q{56KLk6{``JVsC+N z^Sz3fbOJrbL(MkXAIN&cGy!2kO7RK)--Mzq{h?@E7{e9Q!FcfobxZ0>@Mf{Mh%L)4 z1TKpM*A;z`HV(1coyu!FK}&{X=!V6fjWd>G3WeJ^PryE$jw#5ZRp8T8?dDJoFW2j8 ziw!dB-n!|lk?}+e1IletkSWnlQd`AJVJ-kKzrwZG=i5>CUUy zVRWZxw0=d#TYOxkN)G9RsnK(L2c~@F>f`x+4h6gWVOEvb1$_9_yIk(Vf=~@QA1oS0 z$7LM#Jrmssq-0C?kyxGLi)1la4{4dZRtmY1goOtNcD?z_0}7>V)om}#X^7&EiX>TC zNr|=T?0V=%!i;Jap5O!)kE&|N7s$>*j5Hjs1Kk*3}A~p5i)(^@c<6lGnuUL0g}eKb4p>N6!HImx3fkcHywBbxxJJqQZqe7lKa)1) z>|Rl&a;nRmju*dc`{w=rmcP<&RQV6h^0Y3;vMnm4Qc`Y=*gFMgFU?B}bW;H}5uUVlz-z6RfyGBxQO zJ=EB(t>U6&%sOS%#*J?wlA})az6Yo=qQoK`hMIETI@WGj7PPm!o7nD4PE_OEggDIF zAD0n~Vaj#{>hcV=D`SCm`&hFd8H4{xPQ>L+;r5+RxBc9ijB5#Nye;c*SK=(+TbL5b ziTx3}E@qm{+_a!pV{Tn)r@lU5=+<~J{4Kw$Z1xP&xYBNwGF`lbaXX0sUQE4evei{j zyqz8Ks9MFIdmhS+PvXqbRP-YMnzCup+Q$jJ;)H~1?;`vrw|A(xfW8nSa|e-2@Wf8H z9wZiPS@C{VeMecms=GULomz;=F~z8q_{fOYQVPY7{&sEU>@+Vj>hCUSwZ)Oh^lgT> zX;a`4nLS^(VMH*5qpqFDYZxNFw_FQU;)SnpB{Yhbt$ya?PCIq#C2yaNUot00h*`ao zQhqC|r^0N|$cBO#fFO=xAk6Ymcj3o7HOyf znuX5zRV3REtrb_Qv$?oi@2v=C4G>hzsRcKWkO+#0jVBi>B;`fi@EGTOoS`_BhY*GH z=9@2BczTi(Cr}o9HAvnV@Y0b@Jj-ht5S(h$lUH=~1sec#pJG{toYnBM`|@92ePxNl zBTT(Ac>GRFJ+cP&TTE4scMzzX=DT}YumJ_8oER~Fl&ZhmV2FA985%IndLmZD=%tt` zVUdpWaIU5xafB+uZ1h4#R}8Hy^nulp3007{12E?xrX?y<3pa`_7Q@Lgf+gCvn+N-~rlhSLVtMt9Xp7klfeeeKJcGOY3eu~#yF!ecn(>l)=cakIhAjk2+=(B3?z82Ue`)4|jDZZ08 z!nT?6P#!>p zJ;thP1>Ps!Nq+A~wv%u8Rsa2lj3Zl-2-!$}gnh(5lpqH5lpSngeWy-Pt=ASZ0gkN%3y(lmtyPJFSXtyY8AcY@|zHHPeLjK7u{OP3i6X+j!u@~}+hZOnFbhx9jl zXEsOmyZrOKu=-u-fS-Q!Kk9^>3qfWpf@jM419rFGdpDMMV)2R1nZv{A<5t>w2$k&uOs4rfh25&%KJL z%;oG{$#|Tylc`8xR$=?pI4Olt%MIMX21lqOew{vAPnC{QjOE><96spZ+PXtcZ^Sf| zJ3^<_p7+l6o#9e<01u-cnqG^^sOTze2}GXs-zE1oc-8F<=69BFg!iohtEUZ~a184_ zqaCWwR+NxXUx0YGiTYM!ZL4@}C?>l7a9K511vKCQ(3Kc>N;11-IsB%apP?suv{FM` z6oTFCNqq+kKWHysxhjXfR(?I#A^M4}fACa&iizPxZD@#GzK?|2N~X_Ez`ii}3i255 zYtFR>d*08u=?+i7m=MzrV-=Cg1}g#6~H!>O}4h65NYqZ*CIOH)GP)=g+ZlIS6APqJ#V9+aG-3&u z=moK2l zn-`ol;d^)Y(Lfv&7kw7psqHQrz|&T^d$4t?aG=tmA9j`kFdnSw?k||@)p|8R&rLVQ5!vH=RWwBjO%KttDf*am3y z;dVslcEtnpO?-|I@X_g?aqOQ#@JOLK4M-aE4d^LMVLG!lk~O~5cfBL&*~tVv6ydn< z8t^^^JTGMLPC)C*p}0xnAKd$yUY=QN)p~M72Y+_-kYia6wr8a+G(70)|BjOW!inz= zy8EONnCtmN7Q}pmM@g*k87#27Rz^u6scQi^cQTEvM)07B^cY~rpG4l*1Lz73#1ZVt z|HLbviiOUs2SSM&jK;m#h@pD62x%)HjMdfdg0ClQ+MK0IV@V5bz~~)2+P)DPZg1nZ zxa>h^&z}C2x)8dkIb5P(l<0OKEWWMpN6?*8H)5F`01nvfeXYA_V&=MQ8Jn(IrW_@`Spcy z2dtNPXIAqPXR>J)ZwsNHL7%D8*OL0$`+zxT?ybeGF0ZRqr|as6 z&LRz!1azEgCUTDPJ=>uL?ieqa(}U*ACFRN42TnDdf?`bAVdbtL!?l){w=Wb%Y?whz zZa~f5;*t4)jxm5}Gi=0hMii8rl;$Au_)2q?)e+Go%fQPBxx+FndM&n)UVjDTus(b- znrMOi0AIV^Wx}$smtR}o5pot+w}F=7S`F5K?<35eh9Z?&94&!Git5-S9K+Jr@DTt^ zgz-)@(0$CtaX_-nf$z8G+tx197);$$?5?QeZDeUyC&gLG<3tDr#Heb0OeU!6Jxqox1!S0% zCu#*F$ZgpZki+`{iX|Aej#3ro4=h_1mCvAa8UOw&Ow{^ziG9{)e0i0gy6IYb!lmp( zmTwW8k1h@S#$u2ac+P92sYY{7u8=-y-vEjAISR$MXH`g0MnI?311AQc`l+SDRH)mP zON;@-rq^*HC-ovZC}qY2*g#XY*Us1d;Zlc=q9Qfa?EN`lmio;vFH>*lI-6!BXS-aA zju`TKye+9;(4$gePgNfs87gmzE%_d;{S4jDTE_Pf@e_IwIx^zc*hnhTVR4| z%6^F2Gw-EXc2DCnjUyU?{N6_%gpj0*8P_#4! zo*d2_;tQau)1I*tF0WB>yZY9e$OvZ%a8S z{!((OJ})rESUBio*b(MnJo@y*j$j#)Z#Ft7H*51v?Hz)GYQ}aRGj<5^t@afJoq$(q zuho(4^S>O69coxA`2DRXz5m`_v;2>H>wloq|4HYpVxxvEhT=;aW{rei7KdJ-YL%BB zlpMUIiN6pYmnbh?8rE3$QNJqPw=!uxbu|?D!SMmQRv;tCF+4y0p}^DC5P{(D#9oN! zdNh@_dc1YqJ@I~jt!MX>dgqu}u{EKGxLbOM@QU)Zf*7r?&_KiD4st?Yq<>k0*j}!m z4fBdD&xj@L-aR)F6!ccUKRtv{sFTE6mz67%py_qO+ZMB#VqJ96E!g@PAgUUK!uzv| ziFjFNwb7$#oCd6mn8Xdb`P5^>s5NEPd866DRmLh5OJ|l*@#q?&zTG~2mN2g3;oWu` zzN&42yp5@c@QbgLw6sYl!p&z_Tuvy=pmFkJM*C7E#cCy&DAX$CQ2GhQ>zaeh>7`~& zSZKs_(!f2DBfAvYAlQieq)Bt*aFS*ic}#ALS|R|(;*gD1`GANr-YrxJza>+3e4iJX zs9AwFBN1Cg2?kw|4N|DcKl)5kFMhWMv%BC#IV;=h*3jPfPQCdEPZ33?U?mf8Y&Kl%O`&a)SZ zVH;zNNG3G${tE!5aKBPWQJ&O`I}B#=BlO|SPRGU>=z&QF=A3%Lqd=Wf6O7s*Ov)B$ zjWXa@glPa4KiIPT1T=StZ#?9fWp5nsc9CZW(%ikwDfImDV0L48^*p4+CTE^NJ#e|? zKA^>0C06FjpqkIg zj>E1?Q20rFMCUjJ(@1&2KD+{fc;t@vz$?*Da;AYh^mZKeIRxY0(3Iq--Jo(dZpv|}Q-0#+JBwXLfI#zws*FF%uO%rz7(lMWM9Ulo#hGC;)11!lOd zr=8`VIp%qI5_10V`hm^_n*d}^s(WP~MotE+G#_@GgM1;~j9-EVBU15z8LtfG%|nP` z#^vdWy+w&5$4bRo@Eiz6*-44W4_bpD!M1-H0G{vzxlnWsZw8nuGV^#X)V+&ib-yER zZ3c;D@YB9;DNBq440RPfFIb4C5oAmqDOAQS-!Ro0L?v0~G?XUSIjW0bsfuJRm&!+- zkkf6=95Wc8iW5eUAgQ!!r9?wY@`U;(Qp9ehIc0R~ZV2GK_L0jA(LXRaW5NgUu?SbZ zlg}G>Lk`-XB$c3>4ay~m>=HIj)Ei47NL3lgA;{7HKfS6bfBY7+8B$0@x+(@ZWK5$t za^}zv3geGFhX-jJM?DTto;KHgz?z^#yPT)ykcNKMny)<5CW zWgCF2l5Zjs(g|Ph01uh6-Yex%1cSf+qjN^dvVagYG=m|Th;}{7N%O3(xz^U;W&XOL%S)ZHcfoSK~c!M0$Yh*jvrZUj5m!YGb8;}bc^^hKxgC)&O<&$Ai9B)kA zb-Av8ZNAf|@%Dy&D3V!Fqp1DszLmc|q%p-j$=+9}cY}C%eY^h=@rpR(>Lr>#^e-|u zt#_cC#~ZlipPrUZSPQDVpP&4bVk0#QWX^Fm3*D%uA~15+92+3+$o@YsZ-{^ZY!IyB zte&x;Y2zaN@ifFA4-dM%Dum=jA*$MBhIvSb`83i6u5<3EQ?jOBxw=24S(7>?462qOH>@Fn^pqJPA-b@%xlFH9 zh6 z+`~AiHc~4$kocUyqOQ-VcKSv~;7u5<-zNU&bc@7og}ChA{f$RQ9LT{1`b!Od1jx2@ zAEv~k9LAOY=KcJIc(wZUDINXdc#0M>*?pq`Fs9!7?CW?r~+HN2IPVv(&Rv+ zIIjXAFhrnqrRp2Ut}5!?mFgMwiVw~C{5M@9A%$uY7+_Ex?>H@mW;P3i0XCmkJ8)HF z;2y)!B5D3z2I6!QYNF$b85tXc-osEvny0Fjt!>8}G+czajFI8rgQU)g%%{+&FjGRTvu1UK&zT3a8)m@- zku9oKHxZhOQDi{KbQnQi&eeKfOyO||CoX-VH3{-)TE+RPJ*5fLjWuEjL*dmj8%&p( zaBL&VX>tT^zy8%l7ukKdm!X>u*^ccvxyf ziD4q(ad&HdkwI-CzQEUKM@DCylx~GJ)bHr%FTQoXH<%i*aSY9E=iVS!==EFk(`#Dm_o_&!~5xg+nD)ZWSB52ca*$RCt)!dtSpoo zbvObw9nhp)!gr~55Ek`xc}5YOiZGHeR-sfE#63dr>P7ZlC|uP~R)Tt+7UYSsS-G^+(w9SeOI!0tCRbN`P8 z@SjE_;(yoj{s%b!N6_fyQdMFLY#c#%?r*PIl?A{g8e=_w53|2A@z7WY#^1P+@=g$X7ZMY&5qpC8ufa*6q>jsAeAjl!7%8$NE z3!@;XB5JDLWEym~>DAbD*)3po$bdB=zHIhvzb-UkG}ew2^wLR?e5DYPR}o1>>}gusU_A(ClYCPaH4Pap=f*10gX6C6;fh zh{~^r$eDR)L*j$DxB+Tge_t{Uc!`Ljr`~S;3!pi(dJNly$k1-o@EkGHa5@9(ch2mP z%&+hjP~$jy*2G$V#SWN$9O@8XOx)47o8s7$pNZZEx7fGyeuY%6bglaW7!5%nw>sge z){10?sDFgEQg|AkTf$In@?&&nD6-QXjA&+h+S7~MY^#sBnk{yRLYSlNBg?L7NptPZR}Kk_gH z@%gpZ^EB3q^GNu6cM(fSC6K9XB>zZv8kxx?;gy zK#z2z%G@;lONeSVESQxm-xo`(ka2>?YJvtn^NEolSzR!M27&A}H z+e6`B_I1g*Kg|5vlJ8I#WT%Zi42iD7ZC`s~2wDNx?ZOZN>3{khtFm+QbDT;mMmZ~S z5~rFDBzj$aKtsbLFSMsjHzxIix3_BJ$TD>eloa|JAy-@!jKac$jp3O`iJX@jI@9+r z3p&P?7c$5cDsJ-jCU`^W>DmMeUX=o`Gr+g>w~m726a`UhQyK3uba1Y^6zbcI}Zk*6KjcSD)%E>{obh)d&Xak%JisHGmLhI+ltlVv;vNI@C}sg?jAOa z2=)EesXaFOkn54%=Z$URZakW;`%M_0PNNC0U_Ul%DAfl5uUR3mwbGH{Lv7V_>NJzv z8+s=FJ9q!@S$w6j&ca>^sW3T=@I)4J`vl<=UYfhE2kbx?Ti zpnAE(>#FU%>*MwO9R3FvxiJ22T)^tpk11iH2`R#d>_Mh&@j-S0pX^6$*BYLoatj==FL=%>9V^BHl(`Odj8Zhicc;n z+Xh+KGeC)Wz_a31VEpt+x2kCy;pVOX-P~*f(?QUVarIT9Q1Uv}Q@cDZ+JU{ED`edm zXRGO=cbN~$J*6zGo~c;FIzX6y@L~fLxhZ?9lGRB@cbJZpV}iByT9e?c!I<;NUajX0 zmb-kF)dXHnI$fyHz!irTyDf;=w3SGEj)t#|GpXY`@?tI%_5z8hXcnuj-9g{+jLJ{! z$b$eCvfExFsrjZY5*-N2X;G;zB!%*7r z#UTVHsAlV z{P`cEQPIxW@ShP`tRiKLrHuSlgU9sbl+Yx*a7RjzFJnb!Bj|=)>c1=@vBpfp&QioZ zIy0Ozmczhgs8qkhR(DCqFR0ywC?iaD8> z<9L;Pz1jQ8;`fv1yGrySCNbaaWCszli zFxx^_wklOswiWEcPhc{-Y3_6v?rEc3|LisngF%C7U!{pQ4&eN4s-B#Lg1-1cmBw4N zX6w2l{#@J;f4~`vtBFQ38L6;F$3_b-)SU%49H*&ti+kE+{zXsXSYYgS56QpnE+wUO zsxh*KAL)xQzg?28oXBP=TuUl*iEjHbk3Q-*7_dyE4Qw$|!;;ZSp?xr(+>0BaAtH+@(sAf5^QU-PD0z2N4FS2y0 zv}X63Hc7+t1qVf%EaLDl;$9CmW#{JoBc<0o!;@Qb7DA+PFM?qCCBBR2994of-Zy+k zTI)U>P6aKoZfO@vV57?BO2))4Y=s%JeuIR9hlZTkTw;Ao-g@IzVA9^?S!G#E46S1Q zbC$MRkM3hds#EMJSvs}@t_uU{vIdjfE(Q(VfgVqu)x z35Hg2tSG!xf_}-`D!xCi1NKA>GN1?{{mGV3k?;LLIBJg^yCZ*wS9raz=N6E&gq~BB zaz~4|im-a#BnGI!CE5tm5Ei~-RezBGjU4hgnmja~SP@J$^2#ohMEqLSqODhWa z5j~pX)IU?z{0iwCaUA=zB~J&=n|vTX_-6@GJ1w37;;MLN*p zoLasO)LS(8mmH$*GYyazsg1UuMw|{mB)rz*%Q5Y~k=ch*6Cv70 z_V82^G>;HKS|)#T3cgjK__Ofg4kwA=jyWjQ71e;s#C$Xq){Tws}$k z3tUL}cacQ~lyNmkNolA@1@Lc_3S-j31YX#j5=uUt7j_i0+!@H?7J6iwTe^-SxzODb zd(jXqF^L@AzJiNc13}W5uMmBheer(<<^NGaU*R6iA;bRqbxQF61IqtfwpMXx=37KD%% zK-MNapb}YwGNVIJ`7InVh!`U&ELosNBcDyT1S2XuQRbHD zb|g{{aqMiu>*?SuY(eM{9fKB^47p5V8zEkuU~gAS4Lw;knF?R(ZozAqAQsOC{p=XN?Ufl!sj_izi`R7jlj~SPh|XsAIk-# z@@`G+%CeqRX>xPB24d@*B7`Tg(AFem%DPG8dY7kC=Oo{v z_Y+MB5A@M3qdD9<@}p)xGOGy1UUS=V$KJ}bvrn%oYOchpjv3txhk5rYHsd#n`L2#g za+>y8i2a<<#3~2LJ})v|2Fp9bn(DI_Id``9WRisOoFJY$u4*y(F@4w< z4vX@XLs}%N%>tjk%Re-u?^F7jtr?z^j-EI%0dwLb6L*^DHd5&>lR4Trw@4IR_S~)U zT7W7V1XB#xE9t$MDhN+TRU5asc-oS<(@*SxsF|bXMWy^cYM8gKZIf;Osy_RRS;Ig& zH=VbTvt(^R?jBvZ?qM6uRL-CsyHlEu25x6ZPTl$Op<5GBbMctC_1Y|~TKOCfZJ>>? zxn(TFOy^r+T}(D5_r17v4>s4>^^PBs3+)2S-3+gl9=5HA$3ZbvkL7kE8udtJ* zammzi8D=al^?_fp+F_GHccJu}?G!z2`MY9@6IkoP3q1sUXbm;q=m6#R0aps;CX?eTku zQ7;s$QNw6#s28R-H$I_1oeD=62zw+^pt@A6J*CHGO^k$5r(%gyGOwx2EAH!70|PS>~TW+uH;R8%T_YfCaNk+{$<{?L;oL zX52`id6VYvh|Xk0mBV(E`BFq+n_?Z{ta@SFrKZkhE`MGO!tG(szb-7wi_N{^A!M$l;tmsseVMt|eOU6#f` zC=u1m31oVkBFmB`myBiph%j?zPlBx*kC)dw30Fgd$JVw>3~lS z2Ggz$8+=$t;HozeN95s{or_{lkBd^W}!b-h4L$E z?Tz+6*@9(Pv(l~!A3iOB$tpA%QI@a>9(L+c!ekKus%$`}*BD)*Kv`j@>8uS6Fa+SM z@#9OfGfrx|w``9gX@mOTs!fRI=ZQ(iO3bnh=Wx8|y=+exu6bo-o5JRXpY3?lXZ16I zo@){3-V1Tt<@Mio;#UA_ygYvo+YI%@2Mx<=3WX-iN+!E-bPg32Q&q8=O2NR&!%^`o zOokTW5mkeZXWZl+EJ~=@nGq(bq94*T<#JRRm1lB_AwUsVZH{dr`jhU%=SXLCgTW~} zxu%fRqG&h535n;Rfd1y;0l};NI=(WP%;8QJVA!&Jba6$z*(UodpixR>lm3Pv^_ICuA zbrrfmZN%aY6$ct6fk7E}3}XeJ&2P%T#f}OoaU{1ia|1Q@KcCJ2pg6A5HR*&bwt^!H zb509nC9Ff4tjB>l>2D4*gt_9UirGQkTkz-I5Iakwg4qgo<>ItyS- zJng?81oG*cfOPN9?vnlSL8UD2N-OhDqNGwf)9#Vv34)8Ouyd@lyAR~e)OLNtKjuA{ zBvu+bcpypVmUh!7MQZ9?Yp_Lhgk_E?ET}YxcEszSmr%k@6GqWKtC%=rn6f;mb>Pip z(Bg^7IF@jkK8=i{Nh4JR!6LgUA^eeA^CTFs!V`iJmG_tXGmSeM%^v9eg2TCbebla| zI|J(dQfD}eh&?2BBk#aDy^3#2`RZvTN#GvV71{2f(^2G!*j%wVKzsRnPvm3xMU$E_ zhGeratQqv4MeWFZ2LewG%L;eU;Y^P^62&sGP;CoyqEuHxLt*g{ND+sce>T=E!}{|y z@;BSmFNP$b`Y=hlVaTtRJD02CH`*VCLq$*2`XlJQAN$t(qE~JTUp(g*?6q5wE4O%0 zoxK`5L%yDYl?SmXA>SymTb6-u=jHV6gYWwofEI)?PU^nV8~eL;xj^m+-QX{lPp=vB zW1d3fU91xOXwOM89l&(cUqM(sx^sD8Rw!L zXnBNx!S0H)j`^Q}gl~kD_Qis3N6SX#?!3AP$n=3ezZJQe0!X?*M#*=Rw63Nw$m;IQ zcg7!oAC!n= zNBb?4hFMHbhuPw;AZiy#lzA}j@^cq0lzTU5nSpC`uJun(A|O0WGJ-W=M0-?t%t+(m zfod$}naa5xw#vpcIme#TfTGuQg_tTzNuVt%iVR!bi`091pq|QkqTg>^h8k)0crEuD zNhSxsoKJHVbcYRfnC;xepJfD?WkjEvQ6`Z?uG8GltbDP_PG&}{t0EWAaDWnQxFwWO zq{8EQY3xbvdlT$#d4;_5Y?sW-O*==BV)3k1g-y0LY%_O^>&l-}-#*`uPim7*O8Tuo5a&?pZ0s^Y#>exwUPZ3!UGHXBt z)gbushLG&fZ7%6JpTwD5kn_ditC@s)$0>M#KIK6dNxr*ix^1;fo zaI(%9E(5*P>cXhm?8?H^>QB+gsrC-B$f^uhYYW4&95hnru4{OR`niFX^aqNLO}4nB zddhC&`G~=4p(1}8$HnY!{fd;&^cvkox-&wNb4)^~%~dbH3yJCgIb&6q=D9%i%5i9m z{}1Y|pjZ4dGy_~sUusrHQvj+@d>ZKQVfs7cr+^qu8SQzZ7oa0yk;BTV_?Ut_)_Jpf z#{P-Udk`mQg|{mpjio3Ut z?#&Ko?01boY5RBQ2q`e&>imp2u`Xdx*vGTwy$n5>TXfilYm#<7#m((x|#xP zGBG;rn{F+Rh@o)stz0Aa2kxUGrfJi#?hKYR1pZy2@I7$%hX8^v_%()`$f8M85T% z{@GauD{0Fi{fK-oq}Z6NmZ2aB3+E+Wf>nup6o8Az2NmT(fd6SVUu}_EyE5ry!Q6+t zMSH2B!_Pp^^}dsPVO!+t1WBfChI2K!%Jw|kaJ+t6pOw@7Wpqm(#Y;>q9l9$dh)xnf ztbkAyk6JDr6^bd-8-Bs4H>B2z)ovGcu<;hqb0ssk2yx zqf-@6ra~{-)iOz=owYvWr@i#Ufe+AERx(=HutI3VArQ>*F1uS;=JDSAR3&2aTx7Za@jPML2APcA{DFT8?TBk z+SxO{V~L6p!g;5vdDvhQ-X;qTy3wY%7E-Og!76NZqie5|P6lYqb4BM;ZV=u#ur4}D zX@1`E>x2u%P6dDbL@a6R-h^Tw_DZsiY@ZjE_`ryI&p(QUnh84hy(|a?BhZb!K~-WT zDvEmy&QcM zGBSLSee?(Zkjwy1p#vzyE~x%H_&Uw%qco?-FphqS&%c0>|A=jy*7TK{ zpSZ^VaVY$cu`Q(U=%oA;#+nfdK}n zUsmZL{;`fST9^i{U(>K`=6$bOOTEnHcLq&&Dd}}7@m0EP&jdyUH(FyH^!Bd3&-U!4 zd;h!L`w6h~%f~(+i2x0ho;?Fi#>k(IHG?iNYP#Q7{#EhU>P*2{Cmq2uT7By6E60 zXOZPFqQjh`C2LGc%l+v>JsJBT7iytxr!UHnh#_wVI5DL!WX|1L<8frkXlfFOrX+n6 zP?iD?V?3?z4G_d#(A=2N==UB3hRIZA$>LbjQ8apZP`tF~z9@CcdWAw`hnU>ivb=vp z%M%0U6NbFZkEC|unjR@betO}Va2DAk0a3xosFGsp`-XloOK~ZP2Gn5DLz=2w(cV^6 z@o2ZV#Tw_ZNJhTJLdtkwVaV**%2DSUWL)gZz|~VgVHGY7NU|m*u(}bzpmbS^FKIzB z^`^K&Z2$uCqOs18tzjV8UWlZMYf89v<)4}C)=7D~y3VM>J)CeehO}TW%EUyCRKJAh z?C#NJ#u!J}^Na8yN7o}v21XNMsFQ^0HISNT2`mt>+W;Uml&)AQNZNfMe`{lM>}V0Z4q(ZYnx>C%KdV>jx-!bDIeV~Z41jc8Nm*Z?^D;Mioq*`k`^!W@X`5?ud4 z1R{>=^c2)&7x~O0Svle9p%Q92iL4x45WL+A{>?vGwPiUmWhP3IlrQM2IBg|6NLs2Z zkR8}up5sZB%p0$j%BnuI+iS8;Um=ys5Z7aYC&DbhpU?-7BM#n3`kv zmj0q1EZp`>QRX=kgH5M9mA`MIgC1$Fpf2bH`GfCn@yOl6u8mHvQQYctH}xg5*zv*x z@NNqG{0YAme)IOp{pd8A>MZP`gp$ozB2=D`>-JeHR-`ZAbw(F&)y7s&`TW|IDBq=0 z-(IqyEGAK>@tjQJ|KnJgPgSB0q`_G|g5H~N-Cj|JHaS)Ul9AGNMZn{t!IIcwrHX$7 z_d4R;AK$a3pS#V*>tQm+w}pm_eGeB4JQ%p|zoj=e>y(k|(kVh0wcKC;P``b2R(c1x z1VRd#INh+(&F!yTkbHjCkl&fw#Oz?At6`&fHv^b3$zN`x)><_kue+Z6si-`!88x~g@`d=>DylMH;COba^@$5;F~m3(>2>5){d9fTKj7zW&ysBtVoZxf^Wk-J z@K**zt6|sfFoDMJ9(DDIT~%nTWcH`7?#co=u81jx?{%mLr8`XWBkh`!AE(?w!QRsf z96Xu9*gh8fgE`|L!zWK-!gW+{f{W6a4^#Grvt*hxAb*`DBm?$+uI->UsXvR2+^#dK z58E8O>l5|(^Vc^%Xhb`gHFrxQXSRd6*lkZN*+{PKh;n&J^AlPY3iCl|GtAKinv9B|AxyDkheVMX3#-8#2ZZ^y80U7MYm^KNpf$0M3*hu4j&3H?_Tf zU}f{ERY_oV*&u7q)B54}+rNO+|H%33Ry9#{KT9Wih`)YO|EGH92bQ+`p^P~EOWto* zJ@-U5MfvU?Cv8l}K_m`x(kRa_xg>=Tp#-E=qaiR)EXfqdZv(J0b=|fCH8Cw=YK&{L zSgJ7V-lo*4e++1?Fxw=FPO<2H&ENjm+b83DV{_v&W%W0rwt42sN=6hUs~Yjj)GICl=VYw zYzZiU1fylkk)+C1+NnGd?voi)kc6BmKN)sMfl^btMb0@YjU~R3_mb{sqHNjU_j)2j z_EHeR%Q;-i-6K`HF-Q6AYR1dmJpyFQJq$VGE!}Bn=dCp$>&e@RgzO=n^BHrfx)JbS zKIn_$w_2H&@~?rIj6YRip!zg4SuTmEL9&@?GL_!mdrt0GW48qX9EbQzhhAcObX2^n z4l+pT(IO8?vySc(nPV*V&E5?R2GWx8`XT-Y3JoqZcwB+})Bdjma85N+Jd2}gaU$E% zbl15@h*lg2;tA(G1w8X$rYtUy<5iPd!XFa|kjon>}-vr2PssR5&V$Q?#+#|%(!32X*P7D6A zM||^a+Oc#XG6niVzAXnxhP0Y8{6fwY!k7krAqm;2Vw1vDH5x;@c6_Z`rAOAG6W7|(mA=+f%GGy58yrXL&fUZlp$!tOk>?(Iw<0+FFAZwVCWW*gdH5#OQ z$3ZA{+8PZY{+BSu*!dx26C;xvvNO(iSV94VDMIk~RdDGwrqR!$s1EZ`l^rd!5+$vD zi;jHoz^_vzU=v7Dwn6xBdBUSg9RYypU!sUr=c+URZkycfIqX*4Byn zL9S{<6dEq(9HNq`3wJs(dqS@paru7RNesf>2J~Hwh*a_W64^4zL@hnWN&-F=gy8Lt zV)_K+GImTpYQ$tQS)-0G@Gaapeh3cI-BS5qjF18G-jjgzF5h5%sPv=2^bTILbVc!z zlUtVURPaB;?C~7D9}zajKGW2H>maoCA|Nc(Fv$=%rgu?6_&-xyz7Tyi5lm@FfMMzk zv%u8a-^u%7bMBU=pd#~9>8;g5UIiYg*rrjPeJwY`I{+bt7`qY*+ zZ+h}U5Z~U)B8I0MyD2+&Sl&v&E}LD?J{K#C$cIVdNi+nCuIX*bk_m;%B2~w7k8V{+ z7D}q9CTRMwOVZ9`_e@7M5%9Nzz|5 zDLxsasPu58jZ^omw&+tgA?Fwp^KMm>F%xpG(5V&0SSKZHGB;%R#aBtCvsly zy6Q+bw_sfm$Xr1O?%`Yzezxx@!;Fe;l)qrx6@D60qv6QJEOIem(HG@i5sr6P0wv)( zm`tF&IMBycDZ8>OfbVp&Oun;l(>~t5Ode4K+bSA%v({bE^j)0#!x<3_>~2TBhm)H0 z`FK^kE@ZR)xom;f;Hbw#QW%S0@x33zjR#Sbwj=G(IIm#KDKQTyE4rRw_(%ZnZ5e-& zjwx9qJF8G4x2zDMIx2A#Wu9u37%h#e!HB(79;`_uRmjYyz=oQ=Dr+h`1-2Ag_(8vd zU$7j7o%y_Z@0C68a?xk=R&k^Y1&n69*b})0eFpMq9f%S}_2;3p+&UKYS&yz+!x>;J z(0kEUj}LZ^a&5x_0g@G9Z%hw^rdrC(I^UA{X{Cm7Z9Y)QxM?Brj&ZMig^`nS^Y^5< zQeN@eLrdjKl7+6QsR@QQ29FC$=A#;hZKa-6d)10U=iIfzyYa$k6vFILr+qb{!I#H z)yEi;u0jy9ls~C|mLOe<5IwZylI&w~x4V9iaDh1s^8bFv!tt<`TwoD(#&d4E^3f*B zPV5-b1Os)B)<1`QUKjt}yp7_V`X^auWM(QwTL=YOyHil*3fXhm2fBdeuHUgObS-o) z9Kp#D*>y+XVV~$Anc=RV-GsrP><0ZPP80!e2n4&7(hD@zF2G^{gK~~htw6bUpLnZ} z;sd7i7LGf{moU^+L72NOjFuY? z=UN1kvSSz)dx&75L&(XIB?kQ|41FsW4qo+XpD*Zx{?-iDca%N`UXgZ(WZnqbUhoz0 z^bXwK96~OUb)WbeW`19Oo?KuBHcrLd%*g{VF|B|p%&=%z{)mo8W?#*y+PmnMU~g{< zQmS%ZHm(Wok9d@*%RC=ay~oq{|2;@(;gVl9{MpB8{GoLHPi)Qq8YBqW+L)M|Iy>k) zncLc^=v$c^>HpBW{@HmYt5_>4{;)Mux)5p0{KEu>@blpDDGUGr;_nbq2ipyxRs6-${ew@<#!-Y;Y<3P7zA$9u9Gk2#JxJYCp)KOXNb ze-XK%3X+)wz6^mqO3=-Q*daIyOmzl?RFi@05d?`&f!38dQLwZZW^@v43@ZGZj7Ba6 zYQqymZ>&&<*z-SA&Ag%DRh(P~RD;_kUkQKi@r!Db66UyUQ-cR ziy;U{xsuCOF2yRn<*xnl7z8pzHtgQp zREm-+=Yuu6!?NixT1x#X=Ze}VF&-q@OZ3Q{G#AQDy}g)+Gm^-@X1bk=xW_1CBPQ=i8o#M>?j-R; zl;K6E-j#((39hbx<2B;49m-#3tG)1$(id(&I%co%T3}jVzpfCe6GSs#G`kw3G<9>! z7fjCZ5tj#psd@uVMwPkC0!jxK7Mep3sPtdr;s%;BZe2CmYER0ja2zmtVuAkJ{s-UO zXofIs=IJP{vjh)dP7nf2pfN)p{0#I95e zT2uGfIRN`mxAcSaM)E#IpCu_pq3t?5SKrzRVR4TNL%grU@8m%)0m7~0Vm{p|pH})? ziSU7Hjj@AL#q}aG{LK)U}+#y#Ku#jm?LthVI3^_Zaa)&>ec` z;S43}IZyoRFXO{~#LIsLr-|*k4-f49#{SrqLDr_VKosx_a*1N3GV~G{Wf=4UVB0La za+n6sU-yF92I>m~ULILKaN@f5 zx=J?+hOtq^cJ>LgFMrK2!_C__RCFztaT8bRMshU2`HptWJj^F=Y=ZuQKDVJCZ3GzweVD!Yz-!arCUAeL}@iM9tp!NG-SVaG5 z7yK8cw2tU0mPMOwrhhnBkuZD_TkOA7jw35*qCe?YtyN zLY?20O@|6MHPv}~R7(N!;Bch8(8lw$t5>X@ng-vAa2bA}cu})oNl%jFbdH|D@w^t(ec##S>_!08lNxfo_Il>72itX+t_@(%@ zI~+p%t{8}iLOw%{GOBD1B&sVL%|Ba!Mq;%Y8`f(9`ek|@E|CIM=_Pe+&{ymO`f9y> zjNAR|d(lOv>Dx31YhHnwOBbD=Lh|;Gl@E+Tn;BQcsThpLlQkypfvOgS>vNUr-COit zH$eIw9qVfcvesg4^OUuj9i0<(X_y<--J|epb+F`x^$j)=mnPo2E1h23__AGA%5C+d z*4kIzCgPx@ze0ljH;Glrd&X^=RQ93m?XpP13m4lOX?PVg1Ii!7Ajck9!<9OEa~4aL zGAL!1>P0p=5pxg@xwmUQ<&VddScNz1Gh9TFoiS_**vQSr4yn>m1|6IlJ4iM%ob&XQ z2QhWb+JYot6w?-N5YHi5nY#o91mp+N4KQtfe7e@fDcz>&?fc8jR)iG>wWt0FL+dz% zsM*FW&VLbOC&r#BzlbWMtR&WLrO zYkSUeYoT0e{u!Xgc|rK;Lx4{#@)VGGWR-q}t4Xs4pTK)7wn- z;Kz+)2jURD;N&YpGi2d^L?Z{Z3HmN6($3K+3_;6d_Qx^Q>~U7Jb+9+ak%eenc2dq$ z8dGwB=L+h-IK*H*5f{TsH^V9Nj24aOvlAb+#A4VQ!ZLk`9w1W={>8nhhP`mZbJ-ZR zK|6=NHGBAI+jbhoF+Qmt^{f~T^*4Zf#Z?gZv;@7!bp?n_yV(PWUp5>Xyt{hJ@%_IK zZP1}Ydaj>tS@=f-nD#$)%l{Vs|NFF7mH97!_mKpa1a7@i^txyOvVOG$V=jK)K6J93 zu#sqM#ETvFhroL7mQZmb5lny(_ym5ySA6q?lIqUW1R2yzj%bb`$2HoHvyx7$hKqtv zl;YWJ4BhCW+w%Q=rstXNYwxYtAEgbhw^85TcPKw0wa7v=Daw?6vjGL@_j*E;5;rZY z{>-svd^jm?NP}u}T-$Q;;rxD7I+WYCqCE#>)jhMpiik@4i#`S9ze6uAE4^C*qw$LN zg&^8)s?@z`!|(DN$P1d3yuWYrBi11W-Mh>5W)DO@6L_`ir#Ehf3iMY?y5ACXbkgS| zM^T1Mri`UpsYrzScc)3jOqoVEH?wSL;BwX5%#1Gg4S~a=GE&TA2>cq>I zB^DSU0(xsYdMnXyLNn4{H_T zFonnjcUt(XL}KX;YKkIkfv*d_?G*%Mhl|c8*pmN_&Wbz5x&+4GPx`h*+zq<_%~ZGD9P`76`1$XpwfgRk#4P0*5-!C z!eeWGP?(V&(i4!46+AL2%4-*~J*FLLRB$G^eTLfrBehg`Idt+!c~G8PKMjSl}_c(UGNfo112{&`*4nnlMC@T9YlSmlSqM!{s?XQG9#10pt;zJ zc5#p(SO!yyb^)vPKtt5Ln=D-_$TOF>KtV4UHbs1rEhgaxR5A%jHcVP6rIb`iSc3s0 zIhBX|THC;bESSHw&TVM`_!|Q}ppg?Z5^~-M30_^T!#-26B!y)btH`eFV)Anwd%-UL zTY67s3p0$|js%SBO;SkgO?`wMbJmEQl2^qpAS3409uv%Lq3&ShSe=A*b0;=?(J#Tz zV@%Q;s-L8zGN=1^!BVM3feJ;kuEwp1d}}yuRAUrfkVY>4_T&kL1=#&j`>aU*8ZeAJ z#bn-7vMc3Py!-b>W7syW5rEqKrP}U6>LccpVsldQFH6=ScBDZiE|pp3u}psp=AM`J zp)r|ABuWhqj9E8;GrE!)jzD&3bq52RdQwW_akDXLJSdW8ug?&QEB#3iIABe?T2}?? zFNAJ2X6h~OQ+)sFN>&T-o-93Hl?BqxNPA6~cBMwE71^qV@6jDhmXDT|*;({pq6M`@ z@KK;FmVCJCtj4MrX}d#|Anf#(MQ`%*P_vBQ*tu5l?{G^3;#cHcDt@wo6H&+lN^Ey% z{Av3-6XP%53m2A^6vI!yWJt$PH-xFSp!%4g%%{Ma4Bwyo zlr$5$?5f*ywg_mm{6aXMW8(ENn6FQZ#YN*L?U$2e>^UOGmhMoJ zF6N=^Sb#-Y5IoKk0$w=WD)fE5GwMVdH=K$^5Y<~`AQY+FoWjiK6t?xu4ucv@n6E2B zyS3ONHg+1`qQKa+pK;T4vv;d!?Vx$ochirIKWskW$i0=MeS%-(p0ruVJI80@9a953 z5arf31bIbDXo|!&&tCFazmjQfToHUcWF2+z#KB236`y&3`5`%;TkSL|c04GKA8W_X z_HgpLF5R_fCqLNYrP}!D=y}e2bB_?m@+6G%n*Y`XiuFzy?E%HrO(Fdnt^97(iOsAy z2b5!W@@Hu(G6s4f=#Eig1JU^Y#o`6GJEulDE!*DO!$ckbh;o!CaqigEXOKw4{uOQC zo4V`5B}8?(b^>i$>}$&NT>yPeOtc^jymX2{uu7!$NA`5zyGy%o{nW zIkm_1Ff77-X>9*Nuy(YL*BSfNyR5JCq(6d1zt20R$`iA7QxiGOQ2Uy|Olyd0ifm%x zKuEh!^kiZz+ckpG8>xkdjG*1G_C(-9PsT|Kw&q%6_%{oyXcKOZ!qMI88Z)MC^@GhR znqCMuFY|Psdv2*KhR0F4{Dt2$_E-D!NiNW(=Wx_gYpupUk(GWQ4CteFN#DElU1rfM zYUV1KbR?X_qcx&OsPZQQ#7EkQuVCQy9@>jE=^y>A)7nOZ!KAB!*%w*`u5%FLiR@g( z9KNd+C!f=#xkM`^G$X`tK}QCsaC>2cu)cOOG&jbBVA@C#Q)cnR6FeF2N7y4G2E?Pl*Y)`1=K@;Iu#=lmC9YEPk#;PyD>TtQk4*mtP2WNK*5*{!F&<_!g7o4`K zW-dvmXFkegYkJhZy7@!n^#QU&6Tw2YG14~>NY6yISydZoV5CA-!HB|*MK?L;`O5I* zh2zv=iZPm7H5z*$x>e{QIL?7K($ZI>%9`eiM1Dq}vUm}sKR9EA?;IRV3Cp=np+i<& zIr_qlEs8*zi|Taf+p)SYqXoNFffuu-r~>g1tKh|h2EMde94yL&K34Ka`1<9FJPK)Ox zc8?xx5Z!MRGFV1~D^bkjD(EA|Vox8v4qFoEtVd+|V^#d=o2*IB-WJ9IrfMHe@ZHK; z9TemKiiep?iKmT#ddi^oAgItJPO)fu-YZ~<35z}x-ztXzIo__@w=vJ4VDpMTBBsqL zienA534IK#MQmUTnD2QORPwQ4hPm1j#rnM{ML|p}4?a*bqc2lGAYSMjOxBPuI_FSclqpfYi`+ajqzDS)HzY}s6A?4GnSYxz373Up z-mpI_HI0k}xs-o`!u)72&*y&S{>$>P z|JU11KrkOfp7i%Ovmern7fFvF8r7}oi(~gkaBwg5PUK5Ma4%$@LaH~1pSqjGbC=mJ z6-wEF?)jnd=BFgc-!B=Go(MGOhgQ@tS-kJ=t&c$7uMC~9KLdQ~FUP-7ch3Q+hk8#k zRB5~hW5oc~3gAq4b!ohq{0=M3Ilg3ayG(X90jdU|e7rQjzqRSMA@q7|)Pk=yUSfa! z%+$f^K>6x!gixl_A7A8>zlpS8bUog;0i1S50hD%pDQ|mU`1pJZzE!FHXS{gu35}J= zaj%eULFjmUlPWLp-iIrYdzFBHlK{N&Q$8BJ8>i@+C>BlKN-6UnNjeuVj&XM{zTE0l#se83br;NbStjOi4xQFXKYh;5yK8?Q5iw9O?Kjtsefi9N8q?=0X3%iQl@n8bkQG~ zER@wWC!-r|WAP{<;W-$rWwe0e*~+4E?E*?9YgfO2V`?i*$ZiBd>69luCS>s~oPv_j zYbH5372*hm{6e-Jf(%7b@iS1C@O;&oNuafg9utxULo#Te0%%E22q^|*X2vYCqr64v z-E*<0Es>KXj^1~ZnL^ZcIr$0NpE9jTv~)$H$ejB+K|-GHklCLlje+eNgXp{tYoY+| zbiyp6!vgZim2+INGuzF@S5=L|&aj^wuR@AcT^m)FqJ;1*EAp7UsR?~DrQH84FRF$@ z#isqMtSnEnbbunip$UOY>i3~@2EvLx>%oYk{!t-GbkU^$4gaD3V&Bq*^RlRSVUOEZ3SE~tCj-JYQ(3?&kG z`q^-|p_eMru5!|!wCqd{R+((8rq22Ir=;(l?6VL!b2BMEph_ktlAm2aoTl4gLzUP+ zKQYP!Jay>)s9BU`s2(8ME@C(|r!eJOO4%5cJQmC4IHfxnkG9+KPO9w(`aNx>m9Y_sgV!MhE zk0tx4d3@%Zu)b*W#ar}Yu)~Ux2-p@vrqXY=EKrOLe_d2okiP5A@r)~a%bxhKt!=wZ zoW>6CSzoEMC8=<2l6M-lT91WayS^_Fw2pEdlP(r_I?7pT^^wt-w-k$L(5PEWWmeT2 zS659^(B9;eoLjHrvoS#{a>Dl)RGL8akBRnYoEOJL!{M9n2YS4+bZw_-8YPmnuFe10 zf~%8RG|{?^q)a!Ar_eMkrNFjF3ycrTWoyO7wok8OG+(=PJFUHt^N!fq0?yd6Z@DmI z&#n-y8+GRKZf`xnFp`UiAHuHcOQnw3d>NtHNQDxGQ1KoLcG9>`$YyHa)8cFx6rrRxA#aZrv%#G&FM zq-zVJuMZERYu zBHwgnde*NXC9C-`3mT0?A_UUQHC+khX-Gl4-l?g#FJkzW{@R7mX(MiZ)(wqXO=)Zz z5zqvMWxk4H*NSP@jjGp8|*77_8LYbglt!tT#eIPZ@x_8fk;8^98~@%~SXAZDPvg zk@&i6{HD%w?oNA&DQ~9I`hqsRr6uXm16)R}LXc^c6h?*5Bz77LGIo04O&?V?M}l65 zqX)TsoO0F&DU8J2q8%x=u>(q@c6Yo0F_GhjF_v_c$(Oh#MZp^-+>tamI%gK7i_8-o z6sy#?TVsj+l*J+=Il`wuC=sfYd@uz4Grn@~-tn#E!NJiiBg9Dci%HyD-a`5;lt$Zz zbBLeukXT`yNdL1+J^@%Pl5q~(m#SuWg1eIY$knoL7nw~dTW=ZyaGux(yMJ(rWZu(rcl`xXhUHa|={cFik(havEi1zIGG2(-T zNjj0gYY9&JcMMMNn?H&>B8;11%E8i}DQifL%BZTV>cDqFUqNUM6R zINZr$*?;8ZNFgU{asSead5R|VW`(4)*U~M~>FyIzrR)H|{W>`|asj{wVK)h!uuOA3 zf1OPM>?vUwbBCRvH$zRn?EuxW8v4XuO&1B%3;RY1BLt0wB=-bGy(VytDXemZ&f%$M zxsjX{dC8#5*P6S6@u2HRu=Gl4pzT|8%GKYOK@(Iys#ZPem79m|$?> za?Nh?IA6AU*|RJ^^(R=JLCV=^NU2#1H!FcBHk9XReHq&c z-ryd?Mp-5nZnM$B#9^K-wvG#GX^X~$1ml4~RY?(Zxm^a8>^?Nm zmptMDwdtoDT}p;}qz53o7>|$F4|n4&z-_MTowQ|-pey}c9Hk4y7G&x~ze}ck_z%$w zM#2nT>#jVvDBeZkuq_!T9fA+m^-!nTB!pMs%6!wcy^ZajmgoplQvB+x<7+nG?CR8; z?#`R5qc_LlH{n*$GkV51V)dKxnr;a*YGz9E{cC3?3=aBwwG3U~T~y!srCQs`Q0X|Z z?Se5iga^-l+U!jYp-g5)rr4kPp>1-F(YiN3)56;2RJ2=WR^aaD9uYFq>p7HWUZD*6 zCgW$`Ga>XrTLw^Ug)g;k+#=hF}Cf})qI<36>wDXEoPqt0HQlSQ1a zFHucz_Ep!)F>JB=gXlITh5=hC%aJSW{adRaVD5})y;86rIrSDoCavKlQ(@O(C3TDE zkp&bSkBfH%SJ_fccq6q75u^8qV#hg0TC^#>@v(YwT7{e-EvzCfC?2gQA@m)?PA4M&hfY@n+Um1m9U7TRiCu)hCT< z^b6aPjVyfeCd6vG){L$)&0Pk^l2gg3YneQA3y17I4i(jcj{i)(G5P0_#O4jJkeeh4 z%_WHF>2vr%^=b#@8EXLd3VC~t&Jex;x|)wTwKUAx(heHC&9<_t+O`K(<6@g*iiO_o zv#PYWE59W5Zmt*T*<5d)yyG9irYN zTNCqUpR@11XP>IO&#mgNuIfKsUDfZ?&%4%o*1FwW&Kf)B5tGGoyRK@bQHG=6nPko& z6i0VzF@fh<^O4osY)e%ztv(wzp085uYYKhgp>5hSd44Bg-+QIK6X@1`G5|RxP$n=C z(<&W^aP+@mI=R>xrH|LqlS&=KaVms%iKa>zHpC~1WY|?IB#ODDU8K-lSbQ)ha%|6k zsXWb6JXeeHV*cUA@we3tspoP|xGp)cjm506$+HXcwVM)=KNu08!158oN9%PpyEkL} z_?j{*rRY)RQK=f6pd-}C)ZXqMgWt<_e^gwI(Q1nG^2{WX_XFD_36cT$9L=H<9lQ6E zT@t;9r6+;yz1yR7W009(L?fT0yIZ7*&PfgeFTsYQ)E940K!9r60)EpxK7?BP?JCi8 z?=W*btWC1%X#Aup@o}&Gn_hXZgXwC~$vVYx_0bKsGG7+yCX|fT9l;yXN2uI<-Bleq zp079kmf40e-qc_zBn5r%Gk9Y*hwe(w2y>?P5Wb%^s3Z3K3V%fyg$A_$e|Lb54@*1m z|6(3m{wJNDjD@XXn*k0VWDs`X+{=kt3Gy0 zPS_+mVhqthA|-2ax(PaU1saR!IgrDyw4`G?M2MG*Pf@+d;&3Dm6Pv zCt*N#>|q@wKc+Zl?Zk38E+oI*HXNj_C1Y5U+0lC(fI_TF3r_QBobqxM2p zD_M84J{rT6)@}3_4?85jRp1Oc|MlnNr1uO$C~Q<>9R;5iAnIgtB-wgusoHCz5NEPi z6rka;tWJ)OulaBK{}CUH+2{+Q72k69~Xm=O@o+-HbWl$ z23#}{_^j2v1*oV>mzYm+CYAM)i8=oLX&yS2th+_k8g;=$Xxy47>J%!)62+}f_qEp;zFa8X zCYbnqiayBsy7^R%3nl^iZj1QQBja$z!joAhG28w~O0w%^;P9ISY0%&sRGT=#iaxJ0 zr5DRXxZHKrd?$1W#Eb|E6~Z3`uGHmdFmgo6Ig$y?8!}Y=L3Q_#iV{{-{ed;|PC!-q z6J)2wE*E?X=O16sWM5Y)IRr~{-$diuq~z=UGJ&xN7?mWB$@WWHONcz<;Ka# zU%MgAHoM|N??bb1A^?HJAC2l0C6gJM_%2c(r6;B9IEc2FH6|^p4XMrEiEAFf2{d=d zC&=sukH4-P(id}`cIU!!>;a1K8xOf;w4 zOqDvIzVeD%&t-T7L*w`%kzf6k?hFCIJx)J)9WU4xtVVvQ8Qj&{giNycxFTsii?+vm zBRUPjPfv#wJ(u0gy_+AO;IFb@yAeP09G8%N{R2qbViPQrki7_Hr9;n>KYkyQrOdfG zNN5%s=UCs_2M3y$GU2IEU5F%HD4Q3)l7jI~cFWgr$UO>Gu-FqS#cwjLUc-S=OcTbv zaY}|2RE&_mHn#B|8Lig8I-U0WQz1m239D4oH5w5vK%u`fo2y={s%p*w-zXgM>9IH{ zo)V*6?r$*b4svvpm*>NqqnMjnQk(>%Y6ZLx`7=J2S)J&esi@In7SLQ@h zRwt`3ap4d(agDELOxA z8z1MgJ6^8{@K|H?BQ2DEPk7c2pRu3cKm`~qL39Qp1r>DpFqrh24fDSlxBkS$2sK zTasP-?FiXur%CY@HKm|ed3=tQei!xa=n2%e)(tMZ-=;mxq#fYbqg1=Ozo}fG9Kh0S zgjJLZ#HxTwJ=kkE8>O8}=ES79B>R$87n&epFoiK4`lSxBq`X4ReII9nNK9*<3gpRE za?6GfNh!tpkj#P_?m+m=8G1m|g^;n6>w%t0<}=AMuU zEPdhd19}$CpAdQ>>Cy&n?3I-Y${!m*MoBP?{~ndrfS55R9=$lx?>kTAJy&e~>=1k0 zj?f_*(%d9PoElOU;}nmgPo1LinVL3*UYj2Yoi&VBq6yIDCCW=$?@~r@DC`X+1gc@q z!>kIuhL3CK&q~mDeKU%~>PcIF_lK==JhGxOla7}beQMcpz^aJ!qs12!h}DDfZEr$p z&5}7*x{6gH&Kvcv%82gDBS~=lF$$qLlwlfqO0)V#uDWno%^gPIq-HqiJ=oA4);gHh zI{Ab-@`-TkgUZz|x?rnPANO=1U*w`?fxMIDBNa#)o2CB3mKV5!`Myu!Cj4$VysLbg zI3AR2S^YP0bW(B>LM@rvu4JHVtdyg#Ol%V}a_JHm>ks=o;lJt-GS(#kZ6IG3(_aJ# z-M<5*zq$b{6JuEyYiA2t1N(n!aFoX6))bKW_ON7T1yKlcgO#9dAVxsL>HgH_mZll5 zLiC`}IHOj^N#n2Gg28-*L4<*$5O#~J;`3No6sQcuU8QoRahV;i$_rc(f}HOeAwsgl zw~zho6T_%mmdT?7LkC+6GNc3}!_$V$y*%Qk0 z$cgH-8(>~?-H2cyq+4#ad5?#4=(rJC!jqOvG<$Y4q9yrG%VHE7+H=fc^_^d5?^LH$ zzDoC=1Q+?)TMXt<4Qh$e&JxUlWTw?;)8dD;D9%N`=UOF)rp%TNP&niifL<&*oK>7f zlgGLJR`1~aOE-+hfV=l1b&Dst7I)!-O6Tv^rm~I`g&v;sfmvci2P@aQUS8%9F9vj3 z$&@~>_jiMAt6TSIBuUxS>c71}n8eShVN528++Tb=6irdEpX6e<9W?+hV{E4k3-QW*#0d>Z{Id$-_qajmzd zMj=j@^H|b;c#9z^sfIOu!K#y1C88eltc?q!aaHsNYrhdDbp$&AmnZRasc14U77hRZoWZpy&@M`8`$f$ zf7_w=Cga%Ju}p<-qk7|n{#eqy6%M8H)*G&?$6IXJ1o+deHt^-xuxDWL{=LPq2(v5U z#22Djpk91qh9Dv|&AUo4sQ(l+9P_1@a!Bm;RmTKd9 zPP`E|+ERBcC7LJc4L|jO7E0R1AQG2-rFb6C`Siv*e)t(({#CFRWEE z?(VdP60lE*WY&zB8%>Wle=mXz6KRc8gzS*~3YWJKs0h^(ug5Jp04#i$C8z3iS5A+O zc?Gw03{2O}R!?>&4+SQAyF4L(~&gUDhOU0WN?I@kcWR$wulc&akrB;?oO121# zCJN7Itsxz0tYmkzYVs-Ww^-T07k1mFC@3j`8DV{wZ>nqbH|Jbb)-k;I>wi$_M#nUf zP+n_jmUXXjF|3eMUkt(AGWZE#C>dnBMSQ8J9W5Ur2~Dj9&F}gQKVJ&>j8v1`wt{awCgw_dF9F2nq!I1`mjIpNB4UUyF9LAC7i&(AZA+ z`V95Ce*SuQN4k1-z+Qxlc#GPvCOrj)h`&9t@o&rg-Q(aq9un1{v+u#FW5HtIQ&#E; z?d*XjhxC?7b!^bXx!JtRd4hlYlDf1=yG+cVVZ9o1Z) zon1#sU>7j96#hxmw;sVk8@;GH8)6<}cuf0Ccl@g^m)>P_+DY z$=A4*)1rC>q5g2`8_7rixK&yh!}}AgpDCE8ae;FldGS!3se-&DVF6sN^G5&)XkQ&@ zpu68dUWV}D9V@^zcLf{nFK@{?Q7uKhXc4$D@8e@9JGc~J^BweaQ&1QLc_O|G% zbNpaa!(ko&kx>6C{F5RKe3G}NNKGz#cW%fw+DLv~$b;fQk&YuP?#R)&$)vpYappQ~ z)O)HbF4w9ZH}9spro`hN*(r4`-K`)w4D&Seww8cP!se1j)O2*v_r+!`#CFfkd>@3@ zlek9%ZvGIJvnTOHHWj;TL#*{RLv6-A58prRJd)#BDZv(Ax}auiysG|=9cBaMB%?Ft zVWqiNIaxKEgT}i*K^4AJ`@#(nk$En-&OyATxx{u&$_~q=R&G^IH=yBE0{y2k)Ryi? zsSfX6Z_mn^R3Ut04X7B=<4xQdo0AnGqu6-cnxVIi0k0neo8U(Y^6eKl$7JS$xAJ$p z@-%Q+Sf~fZm+xWMJp*n9$p$1NL@X+}y@3u5>yG*qFUn4L{L%;-C=)SMpPdPQxEm zDymniJnvb^gN5g^URg{^w#2gZa;W)KulXbvZp}JYjMAI^Obr`- zM1k_{nH!eHo;#Ck$#qFBgz#4+9S@9EP1GRlLBj>sMo-OeqvFyMOR5REIPw>!#YPpM zZ(5HvRTi>m!Bwxz$qUk6Q9f!U>bVQ{4N&w<+d~@UP$_*YI-aaZeT;;C&rXgglZH7M z%lr^?i^p@^UR!uW6qu%aN=mRDZSVrl?N&fBY&C&8X7X*vmWN(}7{m$wEtIEXj~@`e z$Bnj<4ct`MYyRrir3eOVPy+Cdx{V$87G9T1lhP$0aNm_ghua86twzIM8EI8C7`17Z zq|)5-Srs(Y*2Ci#v{`4I&a*QTM>TV*F-?f<7C_KkjV#gs)?pR})&E|M{uMbt7qVk= zkO0q^K*FiyO(;fPyc(`N=crlpo8<6pptyEsa=-pAD`qtiQ46~c6^J)@88m31!!c;; z3Pq(ckcS;pRn-54_h&9T9xKy87ONM9Xvt083eGx5*p9}Vaikp$KtY&0n9^OaRd$F$aMo39+-g~h&Dy8GSfz~huh`UAi zQ8SAD3=~hJbAh!dgX!HaJP#4eMNqRTW-_L9uI5R`c)gR-c*Yw^I%;-i;`rh3)o64O zwj9T3n)UD%vQ>H+JjUWfrscpJVem;jT0z`f9WG@%0KJ==5x+3YCD-^!zng7UqhHYABX-D_e)vO_>vyt^_@u33SJf?3v&_i<0gFg6914-#X=weflRK z8AO6H9d-h#03<|%eJIMYC@j-dr!|rp(PbiVl^Ds}I1W^msNbzIsMde9R*SBOR~T~G zqD}ouKi+$SwMDVpF})#b49{;C{Yt8!=08`Ub|_+oa-#~jZ<-DDN&q&i|H;!{iGWJ9 zaSxqYuCN~Jix~77y=c}y=>r?Z-Nz@rLiPw5;R(g5(Xyh%_ymxFrN5hp4d~{&KWf^i)teYiYj*JSF#>3>{F|Q24*n1g> z4N2H@;Y3}%+Nn1P5{oP(jUKcuL$7h5b`l#9d#y52Dn_@9xr-)NQmSH8-2TL@0XoJ2 z9c`tAm{9A&kB1N)Lu}e~>ScK2UU=hW{clSdK&C51cgKfXbT)PMBT!$VMKu?_b||R7 zA-x+medeOWbAcvA@;3C;R<*@<@M1u3aS4%Dd=XDl;*QCAd}?@iGU6MAPV|7K1Fk1A z_t1V#;{H|nspp691eL=MvF2#9`y4Tr!+HR>U&LL}%tq05klEWT+madnW-We5Y`pU2 zor2b#!_+4xrmxiyUNxC}`0g+~HH6VS=)A$|H_Uo{MVx|mn>|hL$prW)5S%j6o1{Wp z#MG|?wCyB9`wQ`I>>dd2+O)e`Bui<{jS+jn?o(m|s_o`ghtiV7g200l*LM#Qf8`4T z%Zi4Y?cB=oRpt=tgay+jPU}$_BmV@EU_<2-4%H6ICYr@yCy`X$DO)K$o1vpLzyIRZ z@ejAt!~m0`zArr3h}x4r+r?)@s%tdwz&(0%oEktIMWn32I;18i`{pi|=pNg;mj zBI6oW$Kk;DTcpue2R8(hO4F<@ODZ>6q!%=?cC5BMNLAt&+59RlN|h5JWqF>S$~-|j z?kaxG*jOMY8hHpwyJ@SM^x8bi_?t0ydGz^yM)4&-GC+QH$OTa>2e)T}W}qK-qzW>- z(fPIoP5dM7TNdUBhl!hJk{!ggjYlVPuiLe*W+#ON!{{X$m%HX&7sD8m&=Zj!GPWY_ z!C`cOd0(*Sn?_dE{2qnl{etJ4ifxih*HvC`y# zpTPi0Bc_Vf;EYQ9oHte`Hhv9au7oiQ8dR}O85vn5#J}>n^ace8P8vP98v?Q<|7sPGjHS*VIOl2ny2;-mne zu9-(1=4$CFrLzhjrPd}so&38RLU^cX=1*7k8!JD${7g{o_j>ad`XUCi$=H4 zKQRA+{RLj+iqrFse-at z*X$rWeK7B(euGJGvCM_n20vRO0!f8fMm-i`i5i7OxY1Y$2h&83F=N}fFeZH|8jG0? zC%kUww|R(z03zT<^p2@6y_{g-R$3)?F8A9e^qxa|Np*+R5a0%*_>r+tnZ0@>I1^2B?^mDiG=~a^ zs&Zl>jYDeoppQuCE`nesx#qHeyXrhB-%mH8$sZ7GRFFa$+r)r6b!k)aCYtn-LdhLO~?X?*z}yXExWf4*Sgt*X`01d-6$3IDbn5aT5bqqMbOlC zfG-bam5&ny6J#|UB*=J0@C+;l z2C}XZnJU$Sjn^eVvzjw~w&WD!#@Ft9`vq9)J@>X|J$RNJ)Mq<6ww>UL#^^GdT^96d zuMp0E<(6+gse7{|@D(3D%yrJ=f3*3chgfg&UsU+~u#30pHe4 z*U*doxxwm__Va0MhvU0zdV$l7$)^|y>-k=N^r>dgfl_}{#>0x|^+^`!I$06xr>-LNT zdiA;OPN~4pasx&`)Ig8P>AauupBK9RfAzAC{yQ&B#M!Gw_ce^^@)hgA@b54{c|%JR zBWGn3M~kl!a?k%w4M(btspG0*edFJVGok>c50P~;#c=WHaYSnqfzp=Se9GgjP)X>|jdV96t z5LjTX{Y}T@;NZ!vDe&j&JRJu@$u1h$c;dqyC~Qn;Yl1Qf(61v?vFGrML@C>UY^PQ$ z0|dQ#$FW$R8%+9Amy+qExKUk|nlPM)61Gfohjf{oKxbBsYE8LKD@`giHrXnc&u>d( zosZztG!GH_cK*<{j3x_`CTJ{F9W%{EsX7-N&Dcv?pY(GbaR;?oNF;-TskS}2EODN2 zzTB_0r%$xav){3m*f3Nj@q>e^UZ7;rl?F((TFLSH&{-Jk&Q01^tt1Uy*upCdR-7^; zR#4>w*8HbkQ5xE%6pp{?Dz}ogZ{)(=0Q}L( z{ggpB58@2xf;r&=ZTFtXK>5~WC_8mz$Y7S9?F2PiQ6xXe!QYG|%XDl`Gcw)c2CnQ`(ML^Muon4x@j&z=S zKu%E`{k7g!20_FIg)sEh9tYx7ZkGbAnk#nzcz~PSfM{0lh;-T^iUaLF(n+a6d2mm{ zbhnc062R0oS5IbXk}f{9X@SEbX)6QkdT%o5&gVueZ0GDXTjxwQArv@wqZ;#$t*eu6 z$VfQwzSO8yOtg-tzTqmJov9&0SzbVBl3VsK2 z_P0QUSj?-ien1fLXLqaY%Rh0aU)tn&IN)dXL$-&{xI7giiNPu$Vr)laOy7Px<+HU2 z$2HGijhcJ_|AK?( z4tNFUe%OvS@4ab>;zsx`3<~yGN~RFNGx!vp9~c7eG5)bwv&aDuqqWbc33wnmapxQ|G45Ej`yghoyxNu^Rp{P!XWedi;kiq2ym;^?-PSA} zrjL?1nZ0<{mX-Z_3jT(OMrovEmJ^2VxGlqE&7i(fDtXF6T|bX9>hce^$?r}y!kRsJ z3?a&k{vuP-fQQ6QENTFrLL0L8@7(O0^LIlE8+M5pqr6bF|LUCej~$q2(c~T57ykwg z`RyCYzpK0o1{VJm;`pfO+M%i=^DS8o5pMKv7kuIAGZ>*4*2_Ya5Qa)=A`;UH#OER) zDkZpPHFn0X8~3eBud`b-AO4;Dxrsj{eh|4+Dfx-}i7&9UMH%g)Nq4e1eZ{P&MsIX| z`rN>Fi!3OgoHEHM{SE6M2HQy@J=0+4JN_o zn3l8JK;T||T(qsCEarjDAM0F#!DrBM)6(S#ekwv!iguZXL5D3{n~aXyaBK&>UmP}S zBVnS|s%~6oRO2?oidBdi7wj$i*h!DGG108f?d0{!IB@!6QB}~8Cco3^CF?I_c{RNn z&mmY&nvS7E`eydy&lYWf#TzKQp~+Rf{OogGmFqZJwb`h?k9Qrr1hkrLIXMfGpE7;k zWq3a(Wv_a1OtgQ7!HyY8hAX)+Ee6|=-<28UbO3rqYFHH%BW8K#oVh2O$@4W@C$gFa zoHH>>yxk?_rI1z}1+2eR^z$3+)bm%)vVn$5W2*1A3T}25 zB?2^m=tGPCL|-TVk(5d0jG_uI2{fT|aj}x7EEOkDsdsS}At3ehi(nZL8ID8rac8YU&zt0wMhCGZOgxtGXbcQi1R&fP(R7Wa2F^ z#>(7$6w)%q6FC+8KIy%(h;J?{<==xNg@Y9{iG&I>#t%~CZPqDJu;20?S)v$6jDWwK z!uv9KV$|6pS87@Oabop#^C)-tcidZ#X_sD^IqEW204%RbWeFS9@7Zt+vEJ8sCB4J+ zvj;^I=fuJlF_E(QW{2+QyMY2P8RB`zEulr&Tl0G!m-r%_RM>V5cJVj2<8eB6v2q_# zIvI<+j`dX7k3WUGZw?=MplbDShAk0H8eTM87&^*7@5HK^UNNYQ%pMTL{K6BKu-?f{ znN1bn$bLCe6cUBjM3a_>9xhj5B_FBvN2^h{7@f1|?Dl?BKQNMNM5muYnA{;xHvLnl zLB518cYuMtNz_*5fL+!c&=%1gjZ>;IwAy{n@lv<-i^JprHR4Oh2}!+qtlG|TGWB$T z>2V_Pugv!&NOn>adKm^l4|`ur#Jt^RC;c7W|mzH*M34&#p} z_01s=Z{$#eERSwSC1w8{U{I#g!i}KYYnCY;ouwPPaVMJkY)xd zuJB-U3aW3=uThDx=k|zsoJln<>w@>?2)KikZey-OMmoTdkGs&ID7OCh5P8@W7>CEW znAe0HzKbFCn+T-j0YF;zy%Ag%YTsX9l|3uN`+x{clXg<9s;6rg!pfH#)U9hHC_N18 zs?dc9q^*P-I)@WgLcPCxYt7_)F4#-&?qDp;og{m0y~iByPXwJ;eMqG1KE-WCPMo~X zrrS?5{+D~g9UFmWo}PUhD5bq-3viah3aB^JIhnJlH$EuTt>Zx^WiYQnmIZ7ui8RYn z2rTqO$n45DF=;CKMp~4XQPJ^q{qdXwLA`~@2N%<*tsU8T2$2+|?ych~$XJW0T7I6q zc|gE?Xf6!Tc(G-Mf^zG!Stx;*OcN>VVVUIVO$>h)tB{2@hY%PQO9{|gw+_(qMK1TL z7=_d;ve_(4hoGF|tcTF=tx`lN#a;TA>f;wJqR{ld(%5sQEJmgzRhV(<@y8h4 z)*ZIbi`EekBFkDtFGF!sIUkVHPE=H!YAWc_=BSs^c>};?T2yyKCMC7CN?peBmb~4i zd)C~=dytrDdN3_rrd6&<+RL`FqLd@5+sF_-hp}i4YNiAbcle>&Hg2vY4r%WS|AkTm@(WTDj zl~xRZCRvbckN$^8$5;PF6rKp_qI{zH81$A%>2F@}$Rd!h zQ&3P8B>nXp@)(dWCl7wkBore{LiizfMAsAy6vvtk?BjnE?h&}D0!3WG*2yMA5noH7ao(ZA^`9N(1EOHLZ6RRp*>664Av)x$*qy z&scN}BhQ>Ggk9x z0d&YM{5CbNIdf*hbVIz@X9!Pwj4o~Ks5Cx?b;$~kle&fZGGE}tQ*TM@dN733)<)oz zt?JdIYX#=Xt1oZE^r#fLB&!cPLRl?$n%9ZhAw8rr`!E@5f@2Zp#_ew=Buf^Q533!I z+@70jE^gn6FXHB)6=+fOkYrJkz{sZoUduZ=tudN`?7ur)tg}dza#4TTY|16tF*m9f zF}Tp@DOy!ZFO*dM{$Q}4F9yS_JSqJyr_sb#e(VzYdG!i=|2DZc#;~en-G4NV2=m%Z zFS8NOY1-nH!0kDqQjTc0`42*>Y%a?#~EZhX(bZtkO~wl zr6alUyvC3cJ5+Ib?N#;8&2W#h%j5I!JYl!WG> zGwk?r@1RJQTlH+D@@Tf-;C0HY%9h|#J2|&2r`m9dxBpSewNB@npna6vDr!yHbM9vc zj;X*A5}CnH$9M8$(E;8NHSLcUTvyrE5d8;C5Ep?kuABh}u{$#N!!|ELZ+!nfnW1N5 zamVOUryLAu)Q0yhQ{D^Z3?11L{oPj}yTT+0-j0UQI@C;vFI3|hXBY?+yX#+gYX+r& z=z6z;PVMD!x6QssTWpK8p^n`iCukjE`S|YY4;cSsi@!Y0?2?mZH z?&>4udc!lD!{zOuoA7srF0+!sDDA-v3gHsbXY;IrP%i^+DuYZ@4h(oKB=@Q$^INm5 zA+N7#0;2Togg*{tLEm)gpAhpv<-fbd1Y*yJa5YWj>2I%|t&hMiXQE%{(Bw>G+R%8< zB;5zw_kk2{8}y!;1O;=grS$KmzLTZli_NBHll@Sa=n5*z^eNc`@D*>n%f0Qyk^H=l z8xQ?E4js;9!?HVb9>O4y-UvvYTHQe$;qF4@93@I}yxQ$Sor7WClb%7Uid|!t# z-l=Eo38iyrm&<*sf#yDC$OXP|WjvpF2Rq_qj|)H0X}G~uz3qr@bF;h6>}{lLv(Tv(1nJrR z!ts5+qjvo%r_H07xA!eVKhWkcQ8iX?B7eI-5>PYdFdy1c@$TtTv_+dcuuLQAFpXC) zdRvUaRxEcXiFYfBoUlg<+zni)kcLsoyFg&EuB}pY3`3`HMt4mU?O2##fnRxZ?y6D(_+IGIy6AA%EZsW+e!R z8Dh7@c7OH}d|ul%cpNgk(4_qGG6CL9m|@`19d)oqP*-g?HhRI{Tp zxhc|mqug{N({sY^vTy7>3A=yQi?1U4F5NFvjAAMC*cDrV(hhB95w~m*l{&9r7B3); zdRWxEufhZiPj=a2G8Be3^e{W&%WiK~txq}@NK#Udq%>Dy=~{6&Oeu}+sbsMW{gv+6 zGWq1#6nEU(ah4E?uDr>y=~2d|nQwsNgfqxkX#je}25%(tt2z7-XV+c1Z-`RFUR@a0 z0^w@679g*{Tf%r@Z9vzY&@I+OoB~`z0qiZy(dV{R)+oE7R%Tv{#~$w|P7^x1p&ZGe zlJ!-ipw|)AQ9H%m7U8JX6z;H;!;H5B5KTq=2`q<-8VnjC8kbSBgpud~eK%Mcm*GF@6_(e-Tv6 z*zgN>%@V3q=;T2I{%(8g9-ys9aUyj<=|EL=Oxjo^F5y@MX<$horIbc87ucN$VIO>t2qlH;#h( zGum_qgg2pijdOyLg>d}n)e~$vf0${EiK2kxN(E+a;z2?~16yv^^*!HNo(Ro^yzVsX$On)Ih&oo)`JaDid^^2=gUlw`%Y^d7 z9fyxr^E~#y+Fb@w`p+m7Tg_Hl zB^b5u=pqS8RlwhQ7z!ehzkU)!W`{5*3|!dO(Tcr8b-o(E!T8+#jNr%GQplSzXT|vp zznMgXm#T>Qwz9#)Vm8U=WP0fOel({3O$jS94=2f=(i1_22$)|5iSyzDkHzY!HH3{H z$UVud0ulb;sLdcuz4y!&j_K~Ir~$!aw(=ZcQ~};-G=XM4wBr<&g61*RB`~6!h3cC| zjdME%HI@_U@i!os-l)ycl1zZ&O&eucA^WCwJ@ufIZVfjxEn)N@f zu)yKrcR3hI37V zYCI}cB6+^j^ne(H8^00P@t~TWx6UK@c&d zse(km*-=x@hFo$^yQ3^4 zGcW1+i7#@LSF48kL7Xi+*Di)CS#HjqVyc*YWz*p%vbHPB8M3hroTA}I)KUs)#F9WX zN#|laiq_+@TwFeL1>AmfqSGtP|BHS6WBozs{nYOL%4XTc`yVWW|4&`{m-Y9bMW#X% z#$9E}ys6Euw~N}`XQw8s7V^_8@moh* zrqkuhRLkB}*X{h=w?!#GWDEk;>@L{d_@}dPo4>neZZwb&+*RlZ)U)SKoybG$+=YhX ziGmnQF^+_=`rM)AciLaPrG|=Jc=1KBqF-(za2SFZk^Ihv%;ulUSD$(ApI?#r!^1(| z!1#+0`3d_muSN0oZ}-N2()OvpJ(UvbD<^;CbAM8>y_&IkO-B2{?CRg#|M)=otZe-3 zi?%Di5peOIi?*Y>I|_Va*PFh{{P6+zS%Jauaa^#Ybo86S4+fVBv;^Y*m`um!O}$9j~9^dJMVsnnu94LgYD;uLbT$U{Xt^eVvBw&wk))gh5qwp1UaA=whACXIykH(!Z%P zDn9$Ck^9D&Ry?=L-e(Bo19d!7tWk`4(t&2L`49h5WV?cE982_sErn@W$9o?Y58wG9 zB*cjq$ajkHIK;}R|13s&t;(;#!joUn#gz38f}O%5?fkRFWio=Fs-ZCTtrxJ-OC@E8NwSYZM8Qb zJPrCI4A76w-)UT@rMN6Jx9)&W+*ruv^okDaQZaSSnZxymBFlTNHJg2zjfpuZY(;Vk zx`svg*gPgC>u6KnsRkj1dS2+{BZ`;zl}g2)yj6J<%2UPx{=~#|=S@MgE$;hkmj`6P4d&?(?z^mGRWTK>Gg-MH! zHQ9lR2xc{<6}bVS+E6Yo3@OT4nLaf`M$;=tmdO~@8plgVqbPq=piZ)obAnEn*EZ$G zz#d&p>h*rDJ9lZuFVJd+c*C4qDIx(EbyVT7yfuq5K?V9#=u&fV3cXN7TQ%}p2gmIl zcR?O-UkZnHKGUs1Yf?1R6?9{w1fJBnc8R-oXi0Xtp`s5IhXdXUU0dFh!psPtvc#GM z0q#FUN>*7Dx4@>WADlDP%#|n-(*l9%pOtYl?Fslm_0^JHko@Wo%j-mkv5xZ}8r--X z?Z7j)ZaSN0DwMIK)L+@@vZB_BbP(ZYh({PMMgFYJfbm3+)*yryu~GyPK{&Okgk6MX zEasU0Oh~0><>Y6;{4i+iVfke(-KBA{kT8Z+1yQ(h4ZSqHCiT*3?KrY>5^k+fC2Dzl z>9m<`HqMVsO$v7NB5<*$bXoYczHvd*4kDM@*%;N@G*}D94I-Alb|V}m287e{nHaWO ztz>fHG@xj*ns8-u17Q8fnRClbOuBK$ubF<`*IF3JINfnf_}}y#xVSRqlPSV-A6nEZ ztD%c#WnsULJVu(yX{v;WSkJI|5Fm9(;a7g(BJ?km>$1aQ%(l|$rCKR+bpcawoKF~y z!ihM>=pdtZihqrOd$Z0OqK6lT*mNs0j#b9ofr~c+mA<%r1McvV2v41Z7Lj?G z?1HVK38AQa$|AGf%8_JH|2sVry3@$r4bY&O;Mr%hs^Q3**^Q>B|_9j=u}*J!tVtALBZ<&8~&fP<#{i z>=vneP))jPZ2)Dob|pn|Eu}F9&bew~SEwiJ0?w280OzV<<>Lla>1pY*=)AAP9-G(H zW5w|>71o7%ha9Oh!$AW3ikcLBcvC&QQZ>S`>8Fva*i#ql)?&e@#*sL)6lVzi3F2GZ ziA-~H%`3-@T-C%6iPJ7CcEcdR9Ej&qGcA}p1x$2amtTk(<1z8M3Am4>#vh1jmFkPL zbn9HCONwJ}3IcQhJT;TLd)3o)pSk(CRC%vt*(~a-RhZz@qKK@><-dy@2?_9a@escb z&O3ZLDt@XRUy{z}>p`8J7AaaZ-v$~ij9avF(rvPYJnj!nnO!EvKV>EDhhLn5TG`v1 zl4+=$*TKf%*9Fd)u|VW}N%UM|ul|$@14H1(li52|bR5!CD&{GIR;tE;(9CF*){7;Jmb)_8F54>keF(f5 zpU;ouw~-%NBkQ;NYX!MmT6PNuhkDcl?7s4;_WT?UxdA5kY+)LIqcR$3!NoFHTCsFd&vNK}hacu>kbyokKndG&GK%dN@Rn>kme}~0ur5Z!3GIrr#UXEXCA19avuLWsvLk<4OT7oCN9p!c~ zA@#;92>cki`=-Dgk$X$M#2l0kb$^67aeAAu(sqPh_2-EB)-O;!0(FnVhMG_f-{jvj z%>drr!KJ$3TRBB@TtSe%6ha@DN0I$`>mNGh?GM|H6&Es>jH@$fPNZLYe7mwuKYdNMAb^R$MNV#zZvhjob_}@LcU; z9)9o}K4Af=5B+$5NaTrb4Waz#DsF#DP-UzeZbJ|BD&7f;QCg(yX=aHukQ*!w#8d{_3X*nud98s74O=XIyuJRdaWeOKC+f%C; z6$e3g6~0-R8mPR_ zVR4h3G%pRVZ);k){KI0Uw2rah3E6`>e-kc(0D+v7(>Kd8pvhxDIE5WFo zQgqMNi&cJIKnPwsaV$+0E?d=Uw##ve;PDi0Ru6g8zVKNeJ@dZY;wtY(pAP=gq40*8c6fpv`_qt~(@mp!QN)v}>lxp*sijp`9L`+-UL=*aXzQgn5f~N&2C>?x)85 z$aH`@+dxY(-&Ux-PSMt+_OZqTSlA*-Tmvs03%^iE$}&yLcy{MFgKOB0whvo9jNRLE z3go%7+v!}8^V9ZSNLPvLc2m+4Qz+c*|H#>)JoG#*j=Boc+p7!UhVsMs$4 z#-O5b`GdslpeE<{RDx(>%DIz#G{TtN_E@sxR;9;`RJlVyYA2=}{c^c_>CpL(9^sHEjB?Muw3o+49$) z5O50WuV0k^aVYaMmHq+4{Er%*sRr$>w21MYO)N8)zOIHpDBqJqZjqyk3|)Xy2r63O zH`Y(kWt%P)QXr8OET7p=APsRh#*?O8F-6;&T(w^D*UhpW_t(v$9gtTaox0tm1@K&Zz8mLtoo~Q%zK<1J-33;tY0F5eYq`$?FupJC{qHq zDORWsSHrj0Z@@v7YD5`d^8J0>C1mNOR;UzX)DADJS8KL|mO-iSk7csScdR>=+oj8+ z&1dC5Hul*9^Ty9mGQ1Z#LnhR?E{{N0Gi_2 z(~TY6RU~n54>h=uvWrAOwAZ_YORUfdV2|Q0L>Y5PKu?1{C;nnj?wls{25WJn`V~DCPwAw!;UnV({~>? z0#D;?z_SkhDJi}sgyN}kiRfdm`(V`ajGr}Z`wIzsZw!*OqCdB z5YEqBLT9#_E8kkk$d)AAP#+<&t_)XVPbNW1X=)VJard3@E( z@pk0|+&-XiNxT(>;Ta@`@$J%}fZv!|I+mDEp4t~;#Q*I4zD}Ug8%iqd%kg0teN=`@9Mh+#3ydU1w#JO`=a#ZcK z#2FzD2Vxs>L^bdFpy2In1F3xWu?!m`>V%iQAmFY3bOC(t!~+HJGZzLIY#e^pETuS0 zi16uq5*IOiQN9x>8CeizilK(zmu94KDYYRYQ}Paq-EB=np&#Jr^);!pDU3aHSjh^w zmcP(-)$Ha-UIxgdC>x-nsw6;^_v8?&2$RIA48E!tK(z}y;xRy~C{XIWmdb++ee+%V zpLz*TvXmX#gccGB$T277fuDB5Io^mP%Y`-0AZcANK$Yl$`PfsQ>M zCWhhd1hXpShDjs5^HHJd1%KFCwsJxx>vEM+D)NeAwfbsvh(##Q%Bl-G&(R2l2Of2%>&#O z!XFd*ZmOp?*ojB6u7c)k$q^T>kX-6+8eKCMo>*#O1-Y&MI@GZ6#hc9-Hm1Id1fxrv zZ=&6Y*kw@KJs|~Cks1tUy)}Bp2Lmplm<8=1iwHILl=_h2)zJlkN1uCVo#KuBTA}Sv zR8U-^*rN%UxoZUp>h*=%Bz@_uSN8!*F6ck8kkb3YZJ@zB;Qbe)L+YQ)Wnt)73l81Iw_rgZv!9^^3pW>}kzd!u?uPhV3Yhj0BR@8+ru2cQCeT z6KwujGA}OPanh)@ZW!oHLo}mv`vXh>8c;)X-gsmMV?*2*8Mel@*jw+c4fN z=T7u=x0XVWJ0_gZ?AF`6;6?mVEJYr6j8edkPScUxMzkfLbUPm&Ily^E4^uwSeJ;X>pus#Z(&9F(P6#asEamPW4CW}6DIKBwdeQhCNv&gAt*cma4 z3Gj#mldFwn|68eDJHpUV7YNp*qkn?fN~}!oRL*ehib+1d)HP#Mv>? zl(QZsg*N9>OM|+iDDZ)wibPTRUSe~MQAHMm_P7j0cAomJC^4&o74#%mb%lO`8CLCw zWeqWp>>XaN_y&8^lQI89DH)e|zn;KEHM*E^fN7}V{w}Gg9e#a)IZPI?`jR<)<}pP5 zQk ziBy4_60jlWjj(w(8P1cd8Uhmq!Rmo8k};Y`OrT z#bfN{9kR!~)6QSS)Q;dXs3@@RQ8tV|&_(*^096Zr%R%HD0o#Z$nFid;R zG_x&>opj*=d$m-BB1k_wX{>R@@oqkPFjz&3wo`}G&eW+r*HlXb`F6wcrU@>1vvg<8 zheq8#>Z0`^{0H=s(f&^wOTT1CN+U(1h$dx3*$$ai8Zj(4pNvYb3OEg-J@O3!56PiD zH-Gd*krcH%Jc5?=8!J1{_No$pw z2S=rO-}p`6_PY3WNFNI9smQuT1ir^joxT&FA8$(UBdOdA8P4VD$d#QiD zU0!#C-GJ~y?s28u?H}S}(pT;{Bj9Gl(wULC$LrvhP(UVM5F82zG(Ee217wyFPbNt@+ zvpreL@#FG5N$@i(Ntp#{B5@JIszg>y^H7vIq$rg`owxi-uspl#qu*MYa1#A=rk-f8;i=~;!ln-UT&JO zln$$&v_k5sQ*l+p)6Gp9cf9B79>pbmD6T@)j&QAEQ6r+x6KTiMm5an9!kPpBFM^0Y zm%l+`ewt-EaJ#VDkh?P4u)8!Jl{>)%9sgbts&?!6$G=-$fV@;$hNyxH(HdCNY2t$d z?vOyQgKm^M(E8qAFnZOoilBHQc7?nL2Jlp4 zbgP2xh1%Sa@aiqB{EXAiGN>&e_=aUbyCHUyUsNEVTSPk3XzaEp8ocl$Y8R*lET`3@ zXyup*#471Bl_v9(?x$`Kl%xVAL~|%O8*mHDk%sM1)JTt~%w^c^^b$#6q{heAFPo_* z)lh4|T96oP_%-U|RCb2gSQ6CoJ^8mse*kgnu-{h1WKO2HERb=&8f8V1w7g`Ey129n z6&&q78J)hJoGQju!IsgTZ;y(ueKWKg@X3v75&Q+J66l$6O5^!W2FIk~syO9;9Ysuy znjO`cV@r6_aXVfo6ef*vp-c<6CUk1USPM8z^UW+v7GiQ?-(UX{{^3S&6u0LYX}0C% zpxcS05&B4qfX31GX|jjUi>V(_?I9R=f!Sej<{YX@cSij+AXVGlqjuefqvh|lL(4rs z#@I1TQlC6g(!RWZ_1Na+gSfp#H~6JK&O$U-z`hkh;- zmrcx+hM!47B2F>KG6zE;#VDREL3)ZkN~nXcB91rIZ)SoXE&|{) z-+yc0v-`Hco72yO2>*^#)jxl(BdoPC>WT=9NaFqE1j5Z&s;|O*jTFB9#WQ7WTi*j2 z#5%sm1^b$o36!RBgGzo;kl}{tTALbao-fLkj$~#PWq&Dz7NyAd7dK0wo@TJ9d{4x=Mnz}xS7kF)VRh`{@w(F>XA4PA%dpZPaFcf@ZIMF?y(+=x)l3wJ96ggLstr2%urtvD@cE= zA-#(Bp4kr9_d4xXk~*7DM{Jt&Q+t6xNsaZf!pucNpZ9@~7&YX_W!a9fBaX@cPU4sd ze%zRHPSWn^^+Iq)|)P2UAu?MfDMXj}IblhJVb&0a^ z%w*}9sMx|!sYaMk(^zay*<7Ev!-+7+(r*o#|Da|Meul_;hD<%LxP`U!naC_ppuKrU zYG2#07#8@lI{an9Hb%wYr^apzJQ}WdUoH+L(Tn!5Jy-^IgCQH^#1tfV8wXne(HZTe z>wZ}^a(XN-t4K(1;kLiA;AbF?vV{%;#c zrpn|`a2NTzlbGHhMG{jJ0g^?q*PA)h@NW}H4M6<&yfTUZ%O_)MP^pAzhIGLrwwulm z3$HOeYrp8Y#d5Jf*ps`>VHdA(i!B~C0D*`0Xwv8T{p-r-`t$ZUy89P;Fs;$ActDZy z3q3`W)Mku>2DKWo^tiZ@XpI@k#OTTJIG}3G9rV7B9%(=jAskT0D7&YLHk$0Oel-N8 ziz12v$$_yPn{5A&Kz=gSB*=WKVVttLrKz+$O`|fcu}za<+$3LBRf1@8hQnw9#w@BA zFT6U|V2Jt@hKD1+=oo7;&lb(SRxrbUYQfzkay@8;E%mqAd#;;>3=P;OhCHCK3`tze z(t7@JZnw__VVd{fw#as;X(N#eJW9cl;o9%w)rrcVO;rj!rZUxtu5E4N?}O`Uwvbu$ ztS4HgC-%aOFkPOp20P+7_ptejb2f|+Q|5`JH+N;LH7q<<5dAa1d#0eBKoMaW9`N=h zd#&7cda=}1>wLq$4=BgkRx`e2{ypEA?=_m-Hmm zqdh2H6V_?hca0o!qfF9^Z90DFRgUDEqyXnEQ(wv@#r$EyJ`I|yv z!AndbLRaKT;EcW^8{1H5QX1QW*(ce0Z71n>xW{D8L$Fr&8Ly(gbCVQMHQv%KD8CjHl+3%u3oj!5nN86kz(Ir(K9yjX9U?I-@u0O9oDubJj1}2 zJDvwbr7SpNaG>^+Ct@`5Ja<4L1;?($TU@CP_x`LS;|5WQ=?MSnU;RLC1G_Bb{_R{g zZ_38tdEbr4+TcHlak(ucOs$b!V#?$AT?5NMv)0CK=ilvq_?V8!4)2Ka7Z{9rkYW2=^E67C+xE|NddCoJm% zTZ68U0E-dyL3E%4ks@-2JkcHeD3D#RY)A1uYz4AWp)kf9$bnJ^qX+ms@G<8MONh8* zG8r6KF0hf#lkLI!?fmCjE}-{JoWjSv^~=!i{{sQqm|w%7@)JNE_&L1)$7V7WVbPx= zd1n*%|L6Y^scNBwtcJ`33(UkE8Bvi3s4s!(XHM5cyL#fgy;l-tl(T!G%JL> z2XF_Wnz6VZs1g+p3%+Sn+L9G=AxdAyNz~lo^5mv>0wjSejUwR4iXk<4O2`^^CpHq2 za8-);()8mZ%5E80D#II%9#ZN#<&;t8C#XvTlK9U#>z6UoFmy|!OD8!<%ED1t>(nOsnuj;m5D=JHc#$)Ps2X`tp%Pf9D#f&GMe8RyxO8INfavvxtA z?Mtjz$6}31aTecPa)-|Yd+)orM ze=`A1)nbJHy}v<+=??}?KLt_t8$h*OTS~eow07TO%&?W;);+5!l`3t$W^%WCM!~UpZ3V%iziQpoQz? zkhJ+fqo1sG@^*tj-YGL!ZttrdCoIa8s;E)d*}RqB6&nGy9B41>9i9l}?=#!HkKbiU zyCfyoWC>24{O!iOw)&ni<{YE15>kCgw2hs@2q#@TILaGA874$pEA1L-RoJM!!GQxK zK5(KTcC~7#9iSe)pybUFJRH3ywl}`m_WoX{G=N0#b@k-F2cL-3W&})YI&cAR9OGi) zJ=>$2+CJM%yW9h}AKR~^pf|06Mhq%cj$95ZS@?olUp%o1u&>C^pR?WH=2EJp?H_Fa z0M)s=@T1QQyk{CW{7efBUwPMU&hVP$9d({*$7sAvbE% zz#m+a6ziv?8CT69BGXo9sI{kXX;=9PLZ<1MYSLfah3ln%3UE=~gVH(D%pDZXkdi|5 z=!-7|UHx;aZ4lGZY7c{fPU()Y`;Kw@j+N!`VL8N?aWX-%3%2GuvpnVAJF90`E?O7X zH*~L8W$qKV%5hKM|9t&~@9Vspr5D;e_H;@3WqxnMngw^ga-|H<83Qj6UWrJ64;5*vk&)Qc(fBJNt~HIZ`&?E>~$;AX@y zk!Z5aiMPQ*?LvUbsscFFRa$Y$EpUSiYgYoW36h6UNBbqQ2EI#-h#Jdpm!#g(zuYtm$Sj#Tb%gVL(iHPmx2Jjl~raR!!JH|&G zgzWZ5@EK_M+aOs7Z}Ew(UZGQ?^`N7EmLB{s2l{_~()CZwD%qbRR0jNCzli_iC;iFX zJN@@frBEGOTWi_p3l7j0*8--6Z)6>DD27e&J4|l?bZ-E@#(_S@?-EeFPCM5aldo1q zyxy{zW_#6SZPTW<&T-yGvwr5{#1v=iCv;4#v+S(Xd==|@Ime}w~}?L-{l z7cQgi%XfADG5!7ei6@r*_2k6%>jF^Rfj$(`G2zJ&aDI2nL3eb+0NcGE2+h{2^d+d(zTU`hSTlRb-pm_y6jWgek7vl4O-cLG;Xk- z4{f*Hvaoes4Y6&!6luR}!TvcO+}_9Q2oaZRG8~Guvv?7M{XQ7veSMm~`=Yb`DDU`A zNAq&%kNoqz$9I#B`86I6{Bn)?0sp1e|1ClDwH^NNr3tIL9hAmL1EP$$jCr@Dmti*{ z`3!Rxkk-+1SRRaag{mvF0MX(JI2Hc#^CMdElB??#K>j$TB9SP_AOJ!=xIL7Q3H%d$m-!fy?Za>W{(_epjHPLBvN;Ga9tGs%@MSPCO?)P8lTi zz~o!1?Ohu!`DUL={1q0s6|TUPE@wRFNYlHcT@zuw6fucKGSiUP33Npqi|+=#fZFNH z5IdfJN?yo5WikZXv^59ZK%XX&jk01PM5#s?>>)7#oun`~pI87^B(kyMPZOJq6uDwv z#8x>$Ph+9w8wyS0F)@M_J1fX)5!V+_Tkr-jWfV1s_}VrZ=S~eJgZTPwG}l1Lfs!xI zp}bYc*4o_#lg3LGPYK4qBW~}ZoMf=G6!)+fOb9G<0@TSG0lx`$oR;FTL{5C4AFmNw zB;@QyYnLi2nNvvq_lt))l?`4qiD5N+3iXc2Qs>I2L0>bzKPWMNcGb;lS3?BTL*F1B z#;z*C66>UQR=J5TmKokmaX^|jxfU!E`Ekt(?`>B&Lw__KEw$P8M_F;sXdC z`wa-Dw5)r|tnJT@9M1?cxx`PiIcZH{p4YCByxGGis0InIz>o@sM*)l)k}gG$4#rF8{X8ZTtarYBiq#$E3KM0xD!hh3HFuX*tWQi-fsrw2&Fwk)*pthM!$ z87tZQGo=BJP|5d&A{I?GGdACii(gDiyhJndwJX_dkjoL2c|p!=lJDx5Isv#=;0E-Z z3pCJGI5zOh1)Ev1!rV3o(5REL4=^q?*PY-OG?a`jqj8-!_l})JWxZq8yWwe)oWz|` z;5Rw6OCg##c0(z&X9Jot74v~|3rKTRO?p>IuxU5vhMndMgJ81*^i@?1qm@`LwGzx$ zhV`_X1E_za*ucmrSaL!u+g3=M&JBZG{!y6DZ-cgZ(NGzI(n`vq4|JG|LM!^EMmfnB@H1(mQeLi8)Vvlk zMG?0t4X@v%@ZV2zMv(fqnWGT25>VG1v#y12tQCd3&gO=Pw1h9Z1oA)DH)xWenHwh4 zx`y+lt(1rREwBtjEhqFmxto&Y!I>EsSVOi?u%;j zDgR9$La}V+FX8ja5^|-EVN=pzY#;@m>*EhJ{cR$+6G3~0@(kI~T}U+J6${F3HuHEpIUD^y5{s?eaio%BNh>tabKt&N7k} zw#>uX$$qt^{={#tVUO@!+QDWN|Jw8@ zHs6q%)V}&tzg?1c&g}da6(#$Qg=gGX-mAoX(fyRM(AtNk@&zOU>UJX z#QiuSmc?4M6!s+h<+*WfiFC@(ez*1-zUcgB= zaNeQv6>2Ga$8^Ln0 zDcJSfk-KyWxw-DahU63%9sL;7B#f#-A9lH_K`y^-TtX`Scn1@Dn*Y4a)^VzDQ<(=A z9u`f>=_V>LEyG|K#FcG3{-Ze?11>mPIrT0DD@6v&9y=Bb|-?t&kuFk-&av{IG?1xrDBegUN6v4V9@g;N4&m@Rjo zO0+W$M{&ZxYJ3^a+$=`QjysBRA2`eR`I?x2@tm^q<3J`CX82P|N@$PXztBeI_CL`d zyM)C%6WcQnnvxZ_L_bw}TG}NgW(2L=rt>T;At&&v{aSx@fI&~O4D+W%2OA`jB zqByLviAGo_b6y9qS)&^#b4+6FSLP0_9c|M@az;5CLF}%J0?2lX&slfLPCl}g6`xUG z6AQ+Q-DU>;8?a8^#t(LiskTdr?eb{(1er3!8{QSp^dXp;JjGjjO6A|8!$n5GE`6@+i3`l~&tET&YD2m0`^lxRn=vtnev@%dd~5v4W{ z!qX+eF;IiX+9@H$9TZgXU^tFMQ{`%c0+5#q-x2AIPlDubJ)b zW&L%^mwY%+k0a|QWar~n3)NPZp^8k-Pt^ThnS}=&0WaQYyhRC|J&&fI_Hkv(O!{26 zOaZbeRL8D;VoR5eAfxZE2MJ>clGiG^R%AI@1!d7l@_6K>9~gIY?tFkQhHx+00PuA? zXHJ~V#V^b5dkT=X&qXgz9UCWhdeV<7j14cGXO-QDLrcrgGiI??wsB5iTztrupHTfu*0|t#ZZl+r`dRMpcrRB+*1Kfr?+a zy>xL=mswBWfZv)Tey9veA9n9a1Q?NX}U`XHRiBQbkN6{;D!BV$A*5;VS`S=Yb*-y?>J2%J;@pYa0T4*~Z-0$S7@4eaer zj3wkm+>K1^oh|Ha|NC;9sH`Q2B!K)?)NHdXBH0?6=b!g578X(>f{cI^PZ){-LBI>M zE`hOeW!t26yZEPoPM`P|{3SoaY|ZhM1oLS(BbDj+eago4&-d#qbPu(v3-PV|F!LPe zTkLGiV$3|E$`JXT1M*&qf|QaNNOZz+g(i=xp=>`A>^)4a$mxiFC_%=QHIuOFW}W^C zorG9Nwc2xF_u#WlU4dlP;HAx#RHGdRQSaXh24u-9J%-ldN?-%eI>WItv-C&)wWFwM z9=U{Vn+l)W05tJs3cy831x?>c-IaGWIh=J&=RUe?mBe1PK_Pk+Zkqk^5l)_(x7s{d)D`+f|KK*MX3L zO%Wjo>M^;4MUW9f8{Ofcd!d_yJ&& z@<9I_^|JxpsQ;@kdo6ielK(822} zhV#*@{Ok$thO>X(F5%&mpZ{~-zx{gi{9C#&2aivA`wxUZ_1A1PUTN?8Z|yL*4|lc9 zms-wSxw2QY#SdW?zQ=tt-;Z#-@3Z0V*V}8}j}Bzm7ZJb7Kjc~8rH2aI-%^NQCK#+c zJdHP0EFX4_H&!gqSS-&)@ZU&1hJ~TqU^O7$=fm%o!yGIU8k!)MB~#%g29T+t<78LA zRKvw4-BtFLBDd)Zc_Bdxr2Q9S6W)yzVEW{ByrS8nLHM}MT`KFAW=Anc$sj^QMCnt>J zzGJVl4Rs1$E!$?Kmz#&F#>K6SZf(+$Y$j?vr^a&!Zq8ZqleH0X9=fv&BOs4Mktebu zt0;1=kc6+nYwUc1>0W!l2D_PFzn+%NQyxG>{c^E`g*^2P5txUD+Fq4&`tk;Yak)vDA z?SR(2GS3oB$+%MJl8sg_wxJ^Ep(8|d#>_@gA@XSHbYBu(YKL$jl=e!C{QjJhRO2+J z`!ZTdqm*~+*PUWfQmfXu#^y~#yAD;GVhN!wMRIp?JCg4Jslf*Scyr<#dpvTW! z$X}ZmU0|+j+YN9T-4MWe+vOSg`2RwI8)vRI$YUY$vcBAJLJ{cI$|gU>K9)pE|iI|8Uc2L$o--Ylj{KMTtwE(K+fj%SK` ziZKd}ewfasnb?jR7TQqpZ{W4^k-4&L~TxT3>t_RSbGes7j zIycz)RHvRzj-E+jDf72WHjmLVD@x)}FN$hrNk5hrjImEqJk}PE#ob0#OmXC4I1)u~ z4dZ|2a_9x3P$ZLbvZBlSIV(+ZECuIaW;VRqN!r#piFr`RGRz$@_HUS=Gus~r`&}T4 zym4!!H9#~x>^-&UK!R4p8JkFhPRl#MGHF@hhv4M&4IlUqpP3h>KN=(aIHu)OwPr|F zJh(%m4ZRTX`du|Y30aI0v>b#G)|H*|Ep7W?c`D_iu&gPmz6~Q zbZ0dQl70EJ0pmJG}Ui}pE9@L zPk~e#rn;VKcwhkwd}asmD;Ptv!rsdk8T-jQb3vWirmi^?PAq))XE>rdDpOZ6PtSc* zY818-ys;z;aZ3!0T+_%KgWY&4e@3w*uv-V=(`}C1rRx?A@O%rR5vd4q&}bpiz?4+j zG!!9d(4nI%^P<5On=^3O`V_8Q@W9qSz+Z*A#X|EXWmuhYI-w-8eTOLx4npP)(?r@* z;dl*(`X&9EXHg{z&cd87OYA4*m4T1VAL4c`@?Gim2 z=26&&$Kr@9T(LP9jtxoEi(rU`!&th)$T|7M>lC6KSN4>J6cH+_sY>Rvj+Sy^U|Cua ziV?9L#TsKqxfPURx3Af)LeY56MUi31Nm(6J(v*1v=>*NJJKZG5ejogI;;gHq+Q_a; z`l1}@{as@xH}0e{l-2#NFOR*P0CZKbKJbTq`NZ&-=`WI%J_aT9ciY6lhiB1uulTQ| z66cc(yZ!HBsG;}-s(QXyC+Q|@E_P1=w!GMWDn2CLJUQ~nMs)hu1J|vvEh6PDr zIwwbJ%Dzn`nbSW-_X`<*@5r=>N7dBe)w->u(Skj>_RxD9h2-3m)#2Nb1v^}Uv;M=i zeYvjaYFeKR{zq#{B?#Dk(yo-CX7sK&X#ZS*Av&WJ&b&7%BB*+ozMQs|^7o_2_je51 z0Z27rYFd$2v%>jR^LZj8zQ5fb@Ow9v6oQErH8&!gT9RsQ54AnjYd^oyYR4T7-=PPd z(84NMS6oF8u zTzYI)8;r8Zi#^%C>w6sZeu;H8%Mdrn^sOPYs<|=A5%?0yI$gK+CIOsMTBW6jkz0P6AWQUV$z{Yq<5Dc(aj>N%#F=gZk!46*?y|fD|;tHOR9D`iEs?1k##H9 zzlu{z4AatFwHBhfmgk?NcVO~o*?t=LRS5kSqCtnLcHh@E*G1Fn1!ZHbwQ7Ety2c5| zc*>_t-KQ#Ux@53SVSmw_!M#NAqzr4ie3H!K5_NVN0liOn?i&|+`1KJ36i0+0Tz*V{ zhzgTiY^-!`q&t0i+gZ}vaxFfdHH(M-{_`DYyIwNr7MjhBWuzKgTmKay)Z=k3J(tw3 zI%-?-Y)$1CNEJl;YQ_Ne;W@3(V(G+jPqh1Dv&W?yI)<-*;|qo6EtBr~@xmtE9fHeD z>zBas8~Eipxc#r}zRrU7q}Uzar;V3}NMEZLQmYgW?t0Zesi`B7EP(WMisK;`d)dGNbkuo>@8U2cM=X%r=nE2*t{wr$(C%?#TUwWe!zRnK(Kn&0s5 zJ@>qGzPpdi zM62fWb9>SECgw9EI1cpL90L0IV3fmtAIfc>=0|A~0Gbr$viMIJF%B*h6lEKT7l&Jn z^67icu}^b-DwnXmyZZ=aB`nfnzgR7f+^PM*29rHmqw5osHx)+`V-}9-599oW)Vbs; z84nuQw0vCI~|F`=~a1* z&1(>&Q`1C)(CiPf25HuPsZI(&0j3z8U&{DdpdoDYwb9bh9#y{55xM| zV!ol%dOD%$WR!>2J*w%{JvtX(Z#J9sMd%~$>BsV#Ql2o-$bA|PkE)Xbi&@r)k@$#p zs-(_X>r%qoSVu-@sHqx|<}$RVXn*~{xz1$7!yJbBgam19;>?ls!x7bLMKtILOFCYp zanMlMIGp&<80vQLS2R zd%kOtCC1Rv8J-Ra6_~==R(w2OoH!8 z;l39R^&M;u))l;_?VQYs!T*S62o+QjoIrk&Xl=E38yE;~C%IrD1g2K^th0qOY&UPW zb9t$z$!RwFTvl2AX>MNJXt1Pa7EMCE*qqsC3a~Nsab?j=LPot9e%X;v0%E^%U@I<~ zqm{-9OqIWOjNKz++JwTy<5u(>(B(#;Q{d%BKuYF|lIscBNyLH{WD_5_swKyl`#jOT z9Elk-m-9h%f?~;|%kfyPD)T(2pXB+Yo6&ZhVxkLTI>qCP=sJIzB&rujmHMHARRmAP z#yqf{*V$-w?2g(tSz(?#$1P6;HY`O>mqKT{{F4{jqMtUHrsGvBqNqGv)hZsi`A3vf~^jb z-NmlF;8b_z)=oL)qMSzZ-*E@pGgJdtl0q7jNdCyJDl@MQWr5{_)O%o!J;b?qwZ zxvrY?-^HH5Z*6MR$}ats(<)U~Q<-!Sa=zYEltU$!l9gf9b9YS3Jz>k}&S29joi5hj z$|rX<-*Epq68Q&nLNL$WQ48{uv+`qd!SsJJC;uFY2%FfO*czMI8hOY$8k;z(8CYBW zGvq2u)#k^A8O7HYLIWvTkWyZ-i57Gnz+YY7%Abhkmjt%%dSXk&GHF{7bjDU|$FEzt z7Y+WGCcO>J64W$@k3!}zn2{X=UN>YUpuR#fX15vMleU@c`%N#oyA%GN52#%>S77sz zpnx#|o;weI0Q!8WVxSwsbMg~sUelXgyoRu)146tp!fJh4x>WdFKY zfTaE9h~jH|tHk9o3L!Za3r3bT4V#@XTS(rd0)2ON6N!^N7&F`69edc z7nQqIjgV`U?nNBqu0F6Z(Q1UK}X02nxIt5UQpbW3+D>-rGk5# zBXCctm}XHisHBZiy$kvSs`~rbbmk+TbKveNBS(!N!)%K|w^CbOd#gqxO|al39YFCUsZUCbIS2kiagKh~mY&cP>_tPi#6HBjv)X0Ya#I;mz~A%MJ{1IG1o@6b!z=Y$yZ4F?j71sUHVwwqc?v$z9d zvd6u5MSIk!I0g5i#>#j&6*ob(^_WLA_sMb zYbjgf?+KogK7vd+c7r5k+$(4V@7wf0DV@12;`L+X-er_d!4L^@YWGn|u>DfpBs+Au zJCqj!U9<;e6)ugr?|6rdTJ+t-P`A<@mu~$=kiq)^pc5$^6M?@|Z2;##h);ZQ+l-57 zX7~sLv}pP&Y2NeiY!;qB?i&Z-rC(^E*oT$GatC?^uZ5OMJcYtj8ozjuQ3zeS-9)ud zj`!9^ux}*LM>l+daVvwM$|KMMBMvgECLOwKRZX>L1+8uWkL`cw@w{-=d8V z!RsW!bODQOc@|axTGIY(KhoSG8OIE3Y*!~&_M%VNFCagEy(-VjRVNGVi;i76OQ49as5i4ZPlzg1=Xl9BdC42(=q~2c}1ha2KgC( zQ=)+T(W8IUJQfgT5VyWw0rbI%sOcu2y}?X4qgc;(@9U8b*zRzTx(-fiJ#x`Qm;Q}Y z46Nr94l!)dK8`k8izX-~=Mq}ANXDFgn2IJemS&E0+1+&Y$uRa*LU7>{)V;)H!uSb= zP+)awu|PhR`%Y*li2Hiy1@HQ8K4-aZ(rp<=hr`dck|L~)YG)KFO1Z0*yU#04)mpKS z7XeccTDCKvg;hj5^qp0=K>Wsn$|XrP zPC;4v6T={AGyfc%yT-Y1^q1{ibE_PSB^UIQw53U9gGt9$^~1w}1AW(@NVa0rW=`@F zRjxRiA7QU>*SAuQn*j#vGNj1ilxuTt^!7m0v){74VOp;h?~8cua@+JTKu~;zC3aB> zN*F;%yJy7oX%bY^ue;Wo-uTCQegEk2eXe1%Jim=+0{zW`({L$%u^Fy8*~fO&>s?^Y zQ3Lms4V7()703GPKZ|)XkE#bSp9mv_d`PM0?wTVODa1Zu-A4~YEBZC!PzC|Zr6s(f zfYJxJ@no91?&y&4B4|p#Fkr3^0@;>XgLrF3v4ff+={^GC$rGSs|Nf46kw?ToM=2g9 z%k3`eMe>m=_6uo@0Sw``Be3fUsn0fLZUEte+=DO0^k5^rHlh$dZwtzd_3hKjr^~~E+xvwBxu+Isn#j^d(Eu54 z`SF#54?n?k5v-M8%PINAMYjir>V=%~@O;jA_h4A*Zn8t%BQwY#g$L!K>mS0|?oYT` z3`pD{d4>Dzc)`&DpqpH3JAO@$#ga?cL)kMfJtgv zu_`D60(e9aVZQ<>&s(J0DA6ab+PCsQsAWh%WIh1DC=NC^ihfZFrcZB=Or|kAnQrj= z`F(-iKuj>EV*DYBHXG+rGu`YJhIu?XKzEQfluWZQF@@=5VVFQ@ey!Fw*Vx|k+N84Q zwyS#2ZoEHmKAF$Xb6dN;Lv!=6)l#pX)M|dox2;rBbKi99WL=N-%G_u^aMI&)jmxWx zRKGLn^PJFWkEx#MocfFxdljADmd%v&J2!T55_=BxINz|Koeo z&?%2ZcrHRNWGuKopJVKvUGDFdWH_%;?z_B(R1RnHCu#nw_-6ZdD&WV6^dOk#xo{mk z)WAMj&N4N+*A!glo8qo>D~C*eeW*=C|w~6V_F#ugpeNWsdH5?ncQD4 zuYKoA1)gdu7lVuGGmFvv&6D{36(@^X_cvlzuPsUsPDc8q?(l;OLXkOP?qliAUm4*b z)oJF!$U^sH()6>b^V%8aqJR87kZbDn4H2ctw)MmF?va3$U0tX#LMj3dtF>yOWOe5` z=?H14yF-ut)9BOxW?6^F>DJ8hlMlOgPBT)OvI537nK!wxHvZvC#}>cjR(K2g#h zl0(jfq86ynvwRgRdhN58UCfK%T~eOdAHUG@8*RnF*EXE@2|*s4-Av&6m}IiRODAA( z2of=QesSDGKjOn7a{j*65!)f$@P}9x<`&oOa|(~4@u0l@n1&DGQ&AXoDkN3HFaak3g{50)1rk~jC}ss_ z#()4)1J)%jU(3kg9u$JWCJPlahn!0GJ54E=j4GIn0Rtty7Z@gvotO_E_=a-5;-wXg zU$7Bw=_8rNT5GKe2pe7>^OIW#1dSWM*%xw^0S{b10Z?5S62H3IEl~6E3q`ds0QEucS_cdpYIpYZxafvTH{`(z(o3P;=n%dyh(zC-0f!|le`&%40`2g_rMH) zv}z5+ZaPNsYiKb?T#9jx3r~ZBeaewT7h50E209$2eTl83`OT_={dJ`xw;)EGdWhFx zj^VSq@>x;C2VRl5C2zS%Xk%P$z6$=Tg1qJ4h@&Y{-10N1Gk$vS_JcxZqu33$Ky7>= zcnlayo+qrujmal==nGBvd3OC^e*+K2{31;6(N*`B&~%kd zP^RXa1E9)_T$ygd=nGq?W)(UV*sGCmvo&*Dlp+pO4NK=(h7O@Utf*PXMgt3S(`g@0 zldXLt`Ocf%9u?N0o=?_Yxk2ZgieQaJ?D zz;#yVrkbpAmYO`=z4*p4*|k3gei)ONpo)KtXeL5#hVrEPi%L8${&>hFsU(@tYAW7O zC`YY3wq1ulYcf*yLdglss4KX#u&WegTZ1+KvM{I)R%Y6T-O|M`s(Cdit*A)!BfL}! zun*>&YD%!sz4w_6>l<7Xyu`y35z=Tgr`DMv)NJXOQ%K_;C$_X0NeKhIfLX3lZvnI` zvr?l=^7oK11d~cJ!8YvG2k75Xj-b;K2aN3k6hdLA-~t#pEJ5}KvmO3|crAeRquGlg z|A0)5bgwtYlvdyoc|sE@HX!3l3t1c8wq4_<^+G>}Af7fi*e*zTzacnw7ylx5&+PRT z#5f_o*ZJUdU&qXgrV$RCA&_GelA|9>fAACdrg;Tb4?cmma0j)jZH@hc{`X&bR@6IT zho5(44HW=@=KuY+D7hH^=Xvi^|8zr9!}Pr}ZRl9b`UUAP*n|KTY>fQl9c~FIly^kT z8YKQZab1zTBW0@B)HE&Ys379AkgD`kzM`l}bwl&fJ=!xGr4H{AXNL0Rh5SYC1;2Er zi#hF6n%P~`PyeawdD>_Dr0e>=+pVAf>$9W>U^PUqs19_A&U(0wWV&Q-51y`cWQZhN zlK5=Kn>MiKKoM-GcR_+Dp})Zb!<{bpWN8uFQh}e&?1rej3~xtrf5XDIGheqArSkEh z97|Vq;JS3F!tyEt%a=XsdanpYSCzS@bfkjPn?D%8@MeHSS888O;zON))k^hHd8`ri zD$9P;aGECLl{1Z6YbcX;CZ#Q&TRXY>jI-L)rsP-21&6N4ZglvaM&OhEMK{?gaV4;|5C-kwD zl6H2f;k2K?h)#0`Xiz{I$ zj_%@ynUOr9o_9FBb#@kdasAThm6900z7hazdIv@a73XY4iBJTsJ{*=TDMpc|kV?c`>RRcO(V;$G zF1f?dq%@C4%i2Ab-rJXQ)8PjsTOGGKOWvf0JR|sVKY?ko3su(Knk|#_pBqNht)tmX z0(6qyiq6I?P7+6!T|>}npSrN-D?&yh<5#8L6- z6Y%K^gQ*WnnXA)EfrMy{;# zD_6gi^|h+(M7Lj57?xWcSi*BKUpTiJZ|Js>)q8}OVKN6J{SVG>p*t;hB~VY_!Ax(Z zo-#!#Mah(u?VVB)SGRKq!vL&BtC+$&-8f=p4`DM3e2C>snmX~bd8>*#7zNwziOaS? z6&)kcG-OiIi!wH@xjM=J3TC;kYN;!1!RDyZyncn7{+PPaSp9YROYh;^+a`lu%)3(m zr(IboxkX--x5_66mBUPx>bpHV7%%(#IIjT7lnWyzf;G~Z<+ zj20F9WvvU;x<8D#STfrWVo=$~$+n_7^p$>v#WXnd;Jr5D38ML)b36^|+dAyljlm0; zh}#{v?r-QRhXe5K#c5?4WV%Fp*~RHw7g&4FOh>G|mV_yUj`OUP+r=55R7%dtxFtT8 zUN7+PW>@>D7JOGjwO<4!Be@Oy1Wz3ZTzTYTERlOX2)z)l<#qb^65UnWl?TVweHr2- z3_{Ef$e4gwS?i^USA;4EsLLfhDEor!-~rTt0lZebvaJB>s-BUtxwJjH_oxtxcC=m2A5^@zsyf0 z_D0NO4iLMSBHBA)&)tO=>E~1<$$?~W^LfJu8WVF`UMdiZKZG|4?P0xCd~FSbch5h& zw+22ZS0(*d>f%IedUtBi^T!>TO9~27L4Eh%;^_*nvxwqr6Z}WnzseXO1s9Y zEZ6x%wX{&H;AE_rso^>&m^#rr3=Bc98*uJ;B7urX(k6+@l2|So6~_&T@s9GG4QZ|9 za3iRK(-@|h{STR?FCdqlB1>Z!{t~{uOUe^{t=qS zY$M*t8iRj_Jx+D(lpZ8PaFVK1tdYsyv3cHX#ChG6iO9dN=XRrGHepD*xgq$F#a@M| zOTE9jY#|y8u1d!yr=uLBfp(RkKC~bT&}PE@ieadB$M_y*yVYAs)%dPzf0+roO>zDh zPhW8}_xbuOKW5bD7Ikf*ge~V+ZKQWSW}#r2=JBU#Y^Tdc{<~ZJYdYj(tC(B4)`t@2 zy64}Y?!&=)O8K8Yb>(Lg#roer-SR?q)}j{HKOu-B?#_+|Mn8NaiIRo!e*~OmDO=gi z>!bK?D^R4skr7-SjM!9kPCWVuU8dVCvGKXl+E&<6 zMK{@KX78!?xYMohj!Z>Bk%sJUtXHh$8y2jUGL5cnRlRkT57#~!1X$(?**=0MJ!<>L zdzbBpbtAbfuwi{pXYXCz@cxlC?y0}63oWxZb9(ie+4kej?yWNo)oxJ{hC*4(z@R;EZmja84YG4GxnMCZGrch(m}a7@a?l-6SbUNV}a55*b>TL~mMBy#Ug@P>iwy4H6y% zy)?X7_i?C@LVwrYF!O1eS$;l$_j}%!HiaY!{JfpuJbAz3+{>@&m{9m4<}!BX5Gu*WCYVUIh!KwMgAZ1^Rr3{gNG>A_`8)vzcqe!e5OsgJn z#3z#sCQRb$DVI0JP}HFy$6{)bmt{IU6yzL=Re~*}Ok996QO(P&exG_G)H|klY~w0d zNj|o463xpj)1F!hInmY^*FpRudE{FhC(@}^XV=WHH)lAc0X=>d~v$W%$~M07#(NSfL887%u#`{*Qfn z%kbOahPyBgjCs@ea#|<}`|n&;^*QLtWZ{V1#K9YLnr|zs6G-B8w z!|D7@<@he-u6kPI5Uq@WtmY$=EHQ>lhS26MKigNHjjZX>qgjpQGBdhe%#@*xGs?!N zBSt3lt1$KsSxw`|7%{FF6hB-G6GPFajBwPlJt2lN>o)^IAX4mnktX}@-O~YC9NY-{ z@dsN^)`8Jcv?X;XH7RIvG1~N3%D{BhW|NwR#~!NnOnqe?V-ar95m?FK&K6PP!^6r< zH=ZEbq?q_fU|PljlKz9?4$afq+r)(IOc56bwngg?+rp{~h+r)z=d8aN1*Xc2`GuM0 z8YpZVNOTrCsFv5Kz%XuF0-p$jtg9efZ2VZZDg{#~YL}mvstoh(MCNI6AVthHpo_hz zFmYW*%#@*AMh0R5)Exf%8s0CMW$9L7B$KWUa%~nA+*=x!l{~fNSV{(~vs4p{&OEIU zf6Xh_P0<@GDFY5x;D<;COoNR-?p(%QJ)|4LcjM(yK;WmoP|N}aWYrN|);N6aB`d3ua8;e1 zm`IsZ4*R`z7_!qEI%x+XP4PKVliAyDW>Kb$3F(nY!AtNNVKqhRU@8mUY{n)HlZnm@ zLt(Im2ojA53P~-pCw7JZ40Ca39VwC;FCb0M1gAOe7dMqs&&T&ottLLq#)^xEbyhDo zZ+EG+ncUaK&-89QvIcojI5lRqpKJ)_Wvb@!~ctySLkSW#HV#97W>1o zVAFVX%6E;B+k!cV;A8GSn&;`o0owD!9g_at0oeB8fJ^&&1plU5&o)l$b2S_rlpH7E zOHed6T7-}~TOAoj&s@{jd4j#MO@_hrcLpZ^PY>XY%>B`-kzw1m6=P*zrFRwE0n@fY z|Eyo5*M6kRb&u#3hrcuQ6Jvry5!haV9jGN!kxq(+T(s`Zgoc--0Y!gmI6psbcFY;) zu115_gk);9n@_i}&c$8JwvKuVVao*ECr;pj4A&?;ppzB%D^2ez=^Z1n1Y9tV!!Fcc zzc(DwzchpLl`!^{0hoq1%gQLXsKz>LFXjCR5Opq)WlC^f!4A+eEpb(knWKW z=Qma%y|ztB9e3wnoJwhyxP~sAP|%@tEN0-1&VSY_=kGLunN25C@d!HT=)&{oY_Aw- znRj=E975BL^!-bbW*XNqeK!Xg2;e8^U}5RsP1wciL$|47^g{`xfuV&r>d{o6k+*Rq zok57^tW(6(1`^uPcIW|2Vm)|botfWyb=Sx#c#TzvDfpd-5-=~^IVYj~<^!J>$2>pX z;dbsP3SOf1o+gYc=fKMxVL5kGMvfqRcqrj-z)~HOm<}m>a265X(*Hgj)L{KYf0;p8 zmVQ$D78mR=jPJ9D*w0);2`!&iB5l*}r1LR;pKt=3f)k+3>;gG|LhQ!o$Ck~lz@Jc!I&P`6GK01H>a?GvX;)1yiY784~)DAGG}uM~Ys`}okb znB)h-N}KG=4`M65){4{T(APv8cr)G3z#OP56o?ZMMb-8`SfkvSFL~PwIII-#C3n=>&8@&($fg&kjX-W-zP>~^j=q} z6bPva?1Iv^RkGcpdRa@pzEy?q)K_tgqlu7L^|IE~JYlyrzG$zk%=P9?Ml$;nGki)B zPV78vhO2_19A=*r?L7OD>HIm&dXL%akDS|j+&5yXk!b=3Jc%ll4%F;6(fS9*^U$(i z?uj9A(q+JbT8~lb!~zj|f=BC9pIqEH#`(GS&PqioDTcw~#os`&8=Q~FgVQez6Pg?PS$>3af zzq^%7uSY$dvC}AD%QrC_@=(i~$WHGw4l#7WZ3OA_zT)OHVa@u!lNEa7nRXdlz!WUT z8`r4QaJ!sdOpeVt@+y-@HyWX8a`a)!M5Sc+#vCH*gml{!vd@weCewMeb34g%>U3K; zAe#%f@sOPTr6f1+3MCXbc&^_%vA82Da}tU`?D2aj5I@)82J^{I3yW6CmqWg-JBPfyZI()nC3``79f#J+{2aU5QeAcDuli+w$kMd_nq#^hANYhCm(S;# zY)zi*`%uN^b}+V1Yn`s%s|C_0KO-f_j^*{aGu|h((3Mb%zP6^SxCa3B^za=3Zl8NX zn}{D}*y@%(=c{}l?8u&^HfSyYo{xe5ZbcB!3)Xk_z@x;=#YYDq`}#q~=9lT;+V?Cr zo@gh#zN(-efm66AkG~h=&w@{ZCPLf3haUE^d-?kohu6PEh6E1jscZg<__KkLqj}nm zs6UCI#`LcQ8d!~Yz3=Y)ghXR+SWp;x;SK8(ljVYr; zo2tD(bv=|=QhqnAf9@+iF_1n`!)vhHWFn-&?X%l-B&4Cmc(Xp9S6PMSehFhnE24H8 zRvj~5R?u~_><&)ilF!E6#7P;Y%MPW2PyUP@F46(a7b>Xjk7QjkryYK97LnANF8R%5 zdO+QHcVRxaVuZYP&BnFi|V(p{L;W1JrGdW0QJhkB@TXvmOb4;wKwvy-v$; zl~~7J_-fF|ksWUI)4iXqTgjYhWa8+B4VD(3`G|-oQip{)T_H!nu`%PFfp1L59mGos zW?rK$hWfp0q`+<9=I#Ri^qk*V*b{Ea-GAN}vE+bsa%951H)b{n-w5@w1+Rw5wJQWd zcf_>Qe`a6i{FOFju?17mPfVug&&Z8f^%uOB6*c8j2IKm&3ZNN`82zJRX<>rFAt{e! z2g2PG&=WI>*XW*8p(AmVNo)%+j>p)g+bxIKtHtd#B+r=Y!Hz8Z7wsa)X5dU`oorh0O$2a@ssU6H8 ze0?guLG35#8wDmB*6nziMjFF8pK6x#7EPn5;9Bkn%PZh6-}SBLT8J!j><}(c|FTd zEOHUqnItztU%V@BC<=I~AhXLej!H|8cB=hlM^K~sFMsH643v0JpR_vci*&okKfUi z{v-T(=MnyOEL=#WwzNT!!zjJ=+}oiA9h>#!-w0;Fpn3ejD$|VM@wX_Q#aTNE$LZGM zS-)|aD$12ieL2bq=YwWQZS6%8yQWa-JgEN*z!5ECY_aTSj3&aCF~S5=(wLG=0=am6 z4Q7;Dfx4nrU9~`Ac^8Kh{bY$2#8@_{k}TJyzDVSL<2Hl_l1P#Ha8x9JQU$2q zvisw{!ts+)-+4BC{{Os*%b%)`diI{q%W$J2(cisaxy3$m>% zD2pcW391~K1<|6aJaYM!L20dhmtJnh?uY{Z}wjo{=Y@YIhrg%gN(00yTN=4Ms)27{ym0-fv0IFBJ}nYSY%% zh+;3;=@cuAYCVz6gDOYf>e!kMrqy1kFA&kUlE3K(-l_+7iuhn#`RQxdHv7p7K-+sO z-p8HDH(J~~3Q!VdHYJP7Jp;8#9*PBnDD$tpD^DY`CmVk(>KerTs$5Op`A*7xKaS*# zrr>lr;S6OC05dz1S-~&bTsp$^W+MDw9;1AQ9s#Xq6~C3<=A!UEvTIt>Va}Lv&8K&6RNv! zM~P0As^W-jQ+sUMU|Ddf(Z)oqAtc}D(8%LaQDKp;M2(IkkonJeqx(^<0z=%F&| zY#E2cg1@x-F<;QvMV$3=expt;sUx&3I8j(S?jty~ynonb7#i=S=btEfJ@9lr2sK*V z*DUKoBLzF?t2{OdoMnP-%8AF*`j(&do!IG|s#n-TP;CX+`v()-QtNfC&xW}SsdQj$ zJgwN15tx^Tq%Vhtwn_<$H-{3R_v3A)36B-goh;~^T?$fD6|v$CCA}Op`~b9tn|}F9 z5Wk^1ZPs8?=;i@-ML>B(nao{GI@sxle}GLK$zV-u(I9S%s&5)xlJ^ovOJi&V;fCyTrU`a|gP z0qpg}-@Jzse7Jqcq3yvZpzR~j&5|g_N<0Fi?fS{(X~q_6{`T9=(NN}hEUR`YOQ!0` zaj{d*aeAkoU!`xp_WLGy{~n}x4a58Hp~OxmW2YJV-v6#=`$*~7BYPhPlqUYXd@mCq ztNeTkK#P5&x)qrl$swyskiGnR*_zaTWh8TGD8rMhuS^pj@z|o%_U1@W+cnC_u#m7V zz6Byk+gy!~E2^S5v6>k8Q(mI4ZzGZIz5w|rc@>Xxw@6!R2S|6mW-w(Y^Edx3)cpyw zlt$%@$vB;16{{Ls{Z4N;?%PgI_Yk%F+&A!l&SU?9wj7Rw)8+oG3rK#z)&GGc`^So3 z#MtaVRs>C|KbbN=zV=(j3Bk4AXti~QffOVtVakQH0_Lyt`UHspl#R=DW271o9cfcw zeP;#V!ajFpoy(XdVUwkHS+{-Y-}sN48L@lBD0qo!n_gFIJJ~TiyeBk%AD!q%DncWKBp8jbVX?OsATd^?T~d znRsy68TXTYljST--2l-poWaB(uwW!TK@Yb)5dDcC@SZbBh17j;Vc73{+Z*_cSCoaABvgCey0{_IfWgWU8 zr5&F*gO+Wl+0}jCz8+e&&;B$YL-89BF|Pr_h&zWmxHT7>$gNr872EWuX|QH!%pN5= zSEjhLb&wUsyi)e?Z6dKk79C#9%9qjZdii$h;3-ukD;y6hC>~($L@_%y>~I>&G@3kn zrC#lZBJK`SX;ktaMmSRP+r$PYqJh0WC|2j2uZ>_ujbY{KQ-fJ4N5K?Y8B;!`I}jo7 zW;8&ZayIZ}AG$A!z)EHO8-TofND@0Qv)~A%;S&meca#ooY+b#O1(sQ93kDYaUV_y< z%nU1VDyX>UY-W(*%>D>9auYAa2J%Dmm^`{DdL4?-# z%tt}VZldZ|_%S}HJT*(X% zFI$6XL4L$n1l&g89F_Q;x=~_>g1+EVN9$2%vcS}Nut)dBXt-P-Wid}9SjF3OvS<>> z!lW(Vijc8<^ukLB44ZP%1cqm|XfG7k*}J=rcj^t@$AeL;06F@0pdiyDL>1t?w;*@H z)`AznQ$djP{CNQ`i2UgTpoAu55S=$gl%ih}Y$T$hepeSe+%A_@Ug$QYB=jYz@>)P_ z`&pSio^Yx4gxnJ8_QkojQALlV9y_*Lu|oh@O&21bz|@G!!V z59WnDWrSp9!zn3Ff+D^I9p$6ahiyH2A3*=$;)qGMWV(RFgALItsiTpSb2Z`My;9rM zbB}*$mm@6oXN>~?z5%iBbM5&vPQHH#ub`l%aga6E?OT@*{TvO;)FR<|ZGFu8wQ}?K zjGj%z1-L- zlI`8_8JcHU99(+TfHN}Q-G}gYh}zw%ZXY~ScTl@nDWZlR;#*Rof;ZW7-M_N=|0r>B zG{GZeKd`Ljj|D%)|FClUDRHV6P8L7g(Es3Nm4A3y0Tf;#t@Y}O3L*-kWHm}q4M5PO z7I_ql--SGbsi{TM>%sLMYqkQu5&Zqa$o|4ej}83e?q-@$XtBj-(^=e3Gi}eC9q;cm z&-egfw_IUd=1`Wm>gs_|=ZuFVwLthGK_}t0*%2dQ_yvGGO1Zmjt35YHlQ9nbWEhsm z`}fRN;Y;viYez{OLKJlc*B6?q>DS*tcMLJ0@^LrZ4sfKl@1|?(oH_M`q>4 zsjsm#B&Fmo=Pej43T3vYxHAn3KfF7E0wjQLFR?iXOm2j|R?5mXtTJZLkwV5&?!2@% zfAU!BwMPw$9{#ppVHoN;du~Mvw7Go#2poI0@-axi7>8_D2q??yS=`hL19<1Qp;#;N z*rf+ZFVPE7MU-=Ym$tp-t|AXjpZt8^2<-_4G`vRIcN%dCO%L*+6WBc2&)_?7VK-%# zgX#!M>5A4sGE`o3;;IR(l6^c)<;mTm2#rZbn4}|vziS$_RzGP z@x3rpQmXDVj_$1-0tPEkO8LM6EY*g1H5lu5G_iRH$IFUECpD9I%W!gY9IuX)6KKRl zFCq!lX^5Qu`H+fVj!6LduTUd>(p+2%%mVGH#tXZ|Gq^v=v>?|XXF5e z|NNc23fWoLP3&&?DVDfI>77Ad$`|m>(?%u;Kt|ykub7@sf6lVRNsQSKnn)B_8$=|n z*;OckDl#;yW~`InMZqFW`5p{40$~l|>N>4TjfhV1%wG7EZ)`y&oTM&@lo0~>N$RpA z75+P{mrwKt`CKl~I>eS0gfHjUCnoiam2ch?emBApU z7Zlu96K^br6Q+!3p;deprCU{V##2-4Qns8v=*NN~m6bO7l)u<~eaPrdF!9*NQ$wXz z-)hkL%1I%?1RXXxl>}rtQghFIvBD`JS@oo(bSa6`$-2(e(!;$=c@4P@-{{8%Q#%)IW(n-gNzN&AUWW|2_yJ2L&GVvCHf@mWN zXa5}PS=)4S&nGxer4n^Krj}0;xijH$wpHNFljaa@H$yO&hXw-@0eV}D7e~W|-1-gl zEt?bntL`+Kwq5AY2QMXu+<#3MxB9@>E zm7Ii%uP9Xy@$to)i<^Kw``xTl`^<`7&ix~2*T>^^O%LD;2DXFqZhJ@_hv2yvaWup* ztMnk^fIBfdc?Lu9Oa?=gGP3P{GS)nn>0-zlM>^w5kjJKKDYvD1*a6)Ys{x{vr8wnv zeSI?PCVE&@YV%nP6bX{W$P{T*K%7RxSU{T8x?_VrrmJH$fzE*2riW~Z7dj}>fvhSN zj%-GA=GJJ9_;Yogp+gbtr7?`voV^SyJ+0^m>jHACJHjXKt9pBOW-O?MSy9$nx>Liv z4pXJrJnpe8DAH*eHJ3kHjlT8zfmLx5ts=x`C38cd{J0jRSweDe8!#uT=iGzzJeXl{ zX237CDu_`WrWuIFqLru-QO5!HD&ukb_r?(sQGe#g%n1gyDV0!5<+&o688tjPNi7}S zp$38^;&ZO(nLk|3??^3BHS~_=@N42zg`yH(2G0VIHW6}sW1|6fT+JnWfCW=rM#)XJ zabw~$PTHbrpVeM%7lmBcTp85Uk1hj6TEdro3}n3m>N>_{sFWsc~{&ggs!D+rGj} zR!HD+(v$y1+B?RI5_N5Y_t?6}wr$(CZQH)bwr$(CZQHip=k0IQotd6wCaE9wXQwKa zT`Oxn`+>@`x=LXvI(kwZ-$OOS(HpvnRVoXRjcjL|B0XzGWjTg>sSD#}g1cw#RQXI! zuIQ2#=}SkV-T^l(FxIprD=8&50bSf`&lQ{-EvyIO`m?6$Ox0T!w34(vQ@y&=T}V67 zIW1rwM%V$wv`8DyKI`(z&)lhbQcG&Z9054*V#L8{#UeQ^0i$0jq0^Um!!;}$FN_$? zb7XeJ^XPAgaVJ@E)`RR(E+Q>+M7KcEl#-~g*i)(o-dS~heyD7Zk4rm1o_^R%z0+zx zP*^_i=eR}uY~b%yf{NGTVgBIc2Q3_GrC$3$}ZNjyU4Oc7PMFp_T~W5uo8nGT zAo)DxTi3$TUS5E>LZ0H|pL{s;r{1v5laa<$txJMZe;BMAct$yjVSv`j7};T@`gGGrU2$ zm?l8MsODU%nt$Hxx{ElCKE4rS21wygq3-)qjq~uFf+@$>;`F0w;}>?*%+?_MHVJwL z9INRK`wROC*GXo_$4SqBwDA}${-^z=!F70C=e}Ivv^F5KVM5?7_N0Ljn=tx+6)n=1 z=@1KTSVG}>R6!jr+E53*i3HvLC2Cxk6Ibo!1=9k59+-?d{RNB@o3DqmFR(gXHtMuO z*(YvOq`@e1Yp1dv_$BOmp<#|fx{+3OBAtq*6Q`}OYsEeL*C`yzu$do48Z6gJKWS|| za#)%OwWnAc$n$JAOtCznjH#55d~Uj|IkgjC#>Dzdt~ zEmfaXtpmPpO5GP%kKwSQ~*F@u7YLZ)ojd8CH2ymf0ew0Wb-@+z_$KoWNLinik96~TkOe*|4HOa+mxLE@BYw<%R zmpWgm^g>cCUC+6 zZ8Kz=6%V=;Ubd#P0R&=?)q#T0Pmy9U7y!#=dtsz-?uexQ{C0WAOsK;RYv8(1RLg9( zj{M4h6Bov>>{o#5*}s~@bdfc3m5>zv*5yQ=eqeF<5XtS3nCoZ!cyGK0^7J53>bLKa zEep|Y&HecTC!#CBRdC1;x02iOf#B8Z50%(nq2h*m(?;gQ%rm$fyE}lfOx@U~;-`8` zg;0G}!jYLn!q*5uAr}IxJzBJ1h_rU$^+*+owX>_(Pt$M#wOp#?6t|f zj7k4zK3fPU+Fr?swbY$V>M(`L7SXXeJs?cVtZXzASjt#(3d{HUHkE6Ir_`^4$j?h8 z)q>MD3ILj6i;gyztcV4AX|(Lzz*#%8y`pJ+sO^4!FWTU<218n1Sk?)ygu7H68|O2U zv269GmA`r$;T1t1axYoz;H?3N>bk+9@5v#DCt;HO_kf70r;;Jkrq7s=|H~*VN-YaU zL}i96;$jg}6{ELS^Ay-dZhfY+9JA~~1lH&~Y(pkT;d?U4<{Rw)?Aib0-Ejv3=8yW@ zuebcp$N!72?N{jI@F3^#@PU&f$Uq|S{SbZn_s?+O`pu zjuHh|hp*C4s=7o{avtP3rijrw=|Ym{sRUBWumDqKfX3;Wja#2{wl`lv2xh#DZ>hLS ztU-rK(1rDgNm()|6%~YTDm66YqUPOW^N?pqxf_9gD#>Ht z+QM_8H@XuVT0r{`gh~-?J~>DiVG50{3Lm2nd$m~f6H6}m9xk*ZPpNN?9te?DHg1!U{TqvCyh27KLEuwqrKv=DIvt`!cStlNj^d6rjvZjtmUXf z1g&phsT;3Se{$1q^Xiy1*qzq_s*DyQ;Kx>@9FMb@LtuzgnLu>#5z>z^G9eoB3@ z%4Wk#)I`KW_H)Ccd0YfGkjgtj2*68%TTdP{Q}Qlo3~tb*WFkzw50Z8B@FnP*S%NC?0aYQ!yk)r?0D~Lfuho|g zJoToxP;PKS;ivb}6}oVM`H$EcI>|fvcN=-8K+e-%>vyEF{7H}aX#~<@eIv31n!2ij zEeH+{AE9L{XaJcR<@?* zhWb{tzj3n0zvHvGjg9etHxd3H5UMQ=cQ?(Yq#tbc=Z2$`%M3ht0fGJ1dhGfU^-%*- zdV4EUSo&!ZBz615)%aMWdcJ=od2(j|nyrH8K{Pg}HB3S`^&|vK5~k^_(K#eb7S3k# z2BgRvktEI<6-Pg(7ZRvMU8sV*cD!_dZWp{?vOTZadm6ekI$!%5W`J5Xa9s5vzZZhg z-KxN~UujXF_j)4k6gzEpO*tNddA1kNaxNZ!?aipYKj(WlXcBj^z2A~}wwZ4&QAG~8 zi9e<3zrlKzMsA=#ZRvetUSPa-`-w8CQ9@4#@?y9IGikI)-hz=@2V?sQq~ZurQNkmn zidNwjeI=pM^AZT3=*NT37}5pC&>WKMI0$m=B}S`|8McE;Bj;8i66GPpmpzS;8 zbZO`h6_UQZ1xYX+(&{@mgDleWH%3n(v(hX^lW5XN7GT=OgOuGyV$t7u6mg4a7DuVD zlg>)DL~*c{PfHaYf3P_#8m zEveg^M}x52o7GlaPS*{zH}XQLlP$2R^B|1NB$LUAHwKG1}G87o2^nC|0p@c`Lz<~#}>w@9Et7YV{9Gjo(mb9 zS>chXFknG85EZk#h~Z&Yx9dpH3Lj{}B|M~Zd37ip7j)_T_2XFtcVb-LfD7hh%ix3z z3p45DegyZ3KpkiDrtB69ae%PKl|5%q;3_!enHE136yt*T;xJ)FBHq;CrSzKD0z!FtkCiTedQOkt4-_ zfbQp$T;x(?;q!!7hR2I>40U(;MK`0^j`;Wr*)8Q7sFRYr`$u+)sCUfIm+8IP5j;;6 zeXszp6QYn9#oA?XHn4vX2)(|=slZM2h$sD3GCq)ql8&3@8E!=m?=R%CeU-K($kUs| zwUQ*>SE4S1bKh{d?83N+$6OlKm#C!zv#l$-c|oa?vi=VBh5Pa6;Y9_ebktL0)U)MP z3}CWI*c_a&Gr2Q7Cr03C%VkiK zLlIHD%;Z2a3Ve_Kix6;lZI6yoRJVQ_$Aty&7 zZsHK1M}vYnUkWQhLveXj1Ce#_ANfV$bgZScI_;DH5^5ehy*(OhMk|P@)EXX4F_T0> zSOA;RKA8<-{a*xY+s^2hrXiAI85**!%zjrHA)V-wMI8($fbbAr~a}PO0!@Tg373nV1D`y z0@WAYR1E%A@nzCK@{ik2@{gi{K}EK{n|UY;AgQy#sYxx=TWwPbxjwkEt{CyvM9d&D z;UIJ4v5-^pMD-MM)-eD?xPAQP!c9#gm0^5p!w}JMl^M>_%p7mp*aRQ+C2_l{5UW@z zb|Pnr=RK(b<8W@V(ebJI^r_`g9-Dm#t}-7H*B*`&1HFqP6Roq9>3D$&fEL-239pD8Wj0_{|v+1L!p)t0 z2lnp9X%do*=knpL=aosZ>hq=O9npYbfwtW|RO7e@FdqgG z)w=3v?XAg6yTC;$M(U|D@$KiDh!Ly$uKmomfv{@H+2BqdFMKW#B^z{km(QL&b&9DQ z+CZ^KZvJe|CTE+y>LZ1+jC3N^{~M`yId@*=pZB46m}`QivIy;Y34bA3KWo*STTLr7CZg-O&QkIF*qKnQev9s zDW81IM#@QEQP)`YT$Hu9ny0+|JgjGJi0H`wJHLbxRS~uaeX+BLGg21tFu(wBuuqj4 z$Y_u3>MkHZLeNY($!MFwet0WwcaTHcFvVqOo{0 z=Q3@0;`#s{KHBP=CvE=CeHU=OIKIcS@d|$Zi}BbVgnOX^c@FJK=x0_xl946Y+#sn2 z0U+CBUxu!-p@>R>PPAw{}GziSH7R|B_ z>q0K%$&8Pi<-r{Y!=AKkh1h6N`xQ+u>Ux-aQBXA8e*nW9i_|SR`ub?pJWA=s>pnZ| zml5my1#C*X`Y0ME;&7Fq^?_uXb;vO@d=Bu<9UAhgR7laLGWvF~=(T7bNg`&5K{35s z9Il8adBCwXt%yc?Dv}XB426&(mnC~1cm1iDS_&?s(7JC&sgc5F2Wp)C-9j{(P#!f{ zIbxmJUnKLWSIBJ-neZ^{_#Edrw+Wo!7S-?UwIhI}Sqrc-5TddfS{16#$-FLJaZ^9L zDa7rC<`;4YZm%7!ocM-tt~-__dWS}z7WnnbgwsODl`Zha#xtN?Cz5jn3-)+@?KsFh z)4z16VQ~DAc_zA$r@Ie{i4*G6*yk_fl%Njd<0uS(Hsd6P37r?4Fck?zEUR9++#8z*~H&;T= zs$5dFoG}>1*cvc{3&21_in>ylax0MS&0?ddO5l=C@`lnSg_29)kjL}tE1%GjY@DwR z%-EgEP?eV;m5VmeN33A0X;$VklNr9B4QRTA>9cmq>pFKY7Uv!SQCtdC#Lq+{WDM)x zuQGX*G{NL@iM!M+=%w!@4rTz3w3Y)fN6p+>t51S0+w0~Q;7Q&JtNWE0aeE7XQBBHS6x)8He`hjK3v+h+ zmyloJC+^YSraUIx>^pD`V4WVeIUAJ zL5^+04_{Wd;l-o!wPKdV`>es)V#Ectx5mM0dC$iFYIXomL)|}Tw_Khdsg)mq;XU27 zT;I^MS5}2D&a>};#2UF8+y+-S<|0|=bRK4MZws_|T6JY}1bR1i!JM15_s>8U1>4oR z5bS0iPHJzPP{WjOkAu}J>$sZDg_|W99FvCThcfp1b@QNK2<%h?<+wO8-;oUM79)qZ zD8EK=E~gH>r*HF)BzgCw&?n;Y#%_dGJ$wKj&s>QnHF$>k84PhHE3f1HxGT9hq)Q84>A&EJOGa4LQwoMQXENR!e4Wly-8?l+S zjTu{<0fv}3rWs7&w=E?VNCv|{HwcqE^ISaz?mn`-pZ}-_)hG3R>!g4$hWw)T2WiGc z<)E?R2%IG8C&I#T^l1_h+*CyNh0%jj#9zEg9#oC6wqqgOG&kK;N%pzZh+-wF7Pcq4 zsYXdeJ=X-DA2O=)`LZANs99-vN*HLeP3b-2)TN4_aAzIw7KbTa?)=G*e~Moe^7F^G=9UA zmY#S|j7efTxm?u!@y0Rg;a%|g=&1U{P>4czd5CDodg&OarPMYhm)bQ~AT=(cmsKp0 z0gII@lggVcnk-l?Sv%C4R5Y6`o_;uOlP3?SIy+H6TfW}!>>W=wJ*U62uBva&WO020 zLyIXHm9k3^9E(I)@4b>x%?|g1KpGP&lynN?Fn;V4k(Lywk`2O;B#YFF9il{l+}-hU zr<#Jiplf3DWsW@bZRFbI3-&teV`Gk#=j+5RDwsJUV-gIaZn10bHIi1$nUYa0ya02^ z1{0Kc)C=ys!ksvkj~0}kA-ucBrvxnK%pfrVTa-mC7XB4$8}|%jx)mS+M6ue-6+pt4 zVX3@1MVUAX9gvL_p31p5jB7Hbk7zP)lMGr5a0wPr5IqnaOp=ASkwc*%Tm=T8l`tL|{(xGQ9mD&n$!rzU6B3X=F5#^Fu@5q|zVFp+zse^^5bI zF}*Upk+JE0bN9`38?l9)$j$CsK)masL;IY%n@0Sz$v!js{fGiNc324Gu{!CkguLqH zjK2GJvvonlLS7B%_iWUSfxYPEgxMDAw}C|!wi5goK!|4PqrgEl${V8T2D<9$^hya$ zh`d;q>iz`=KuRAMhJY5Y@m^;|Fp&SiZQB_!-AVjfzRu!S7Q2c(ue$J7E4ol3d3#CO9E~q0yg2tGd z4>ON)Y0*GP>%ZQ=!Fv~p4YM5JpdRe#9A`>B6nL1&nex7ko@Cpe`Yz=iR@xhkjiMyXgQ4p($FE{X zI$cE)S_X{zE8;=JRjLf3mcRE;kK7)xNcFcmXsA zle`yKYSEe+=o-Cm)K-eq;R zG>dmL9sQYJpF&AYS3%cUNfnV|V(+?UkmiQN8_)|f*97>j%g3tOVs?KQtVx9h1S^)g zBSAf9;0Lp8*>ML|z+Mm+SN%+*=|9(fyQ!)v9lD40QeX_)oSHi*9|5T!8d8F68r<>F zY8mvjEw2Y1e_5gQ1q6bhji-|lbq$#Bmt^T z-L74{WE&X@s9vc(-oYJMbnG9oKrne~uE89T??@pXcCz)xdQ{GVb?wABY0;dFQ;vSn zSl)D6iX$GElZc)IqQR2(t&CdnRd8q*oPhMcY_(JD!$={Vk>brfh2si8CcY8+1Wt(r zVc$~8>~o;HzdEHnRa^%IkNGw2Q@;XQcFV@S%2sw8CR@>2+J}gSO=*Yr$t_?_(B-pf zt9_`IAOTFyzX|x!7=X)T+A0{7GI3QtgtPO3*BM?Eyr2TotbXD(Lvw8PM7skrr0K*= zmHP!GrovjiflT@i(*D_1XZs;UGS=vZ%Wb@eq#_Lx2{3~sZAF@gByRD~XA_ASPK-sM z+OlQ8tWAitX~Ibgun1bgG78DdD`;;_FecL*~`efzsO4er5tpG5fv(VdOzKg4OminmIke z$Oj;%JPQlDh@JR{NY?eyjrbkg-$OiH8llM-=k~Dfai-OKg1m*PN9HV>f(_^Pfa}FG zn)RZEEvJ}i*;@cBxh*Zi^G_fjg`*NDpLE~Sdq$3*(43_=b{`zXt_?}bEZ<35x~Quo z8JQE*H&Tk8!iR~^@M+#7Qs?iqMW5-O*k;}EOx3v9Y1`1$V3&?C0n@&_yau{gSXTX@ z=6r9mb_BEE>P)_ZCq1_r9M1DPhYNjLOl6-Vlq6 zaz==w#(xaTc}^t}lX!>bXHOzvRF7+QXiGoSs$=_WtMgAKviJ4V*Fro&wD0aO%HwM& zDSGlKzMP8w*cX^7AW7V|+I1bF@b#XR&+8rohNf?<6ts_`=TZXYRWpd8S5gRSsdtaq zYbXb|Gju=IR#1efq4Jf4C@7;JTew}s$twqU#Ora-mW_iKk9Xs?DH#`10vTaNDHVyS ziiq>I+WQ|Z67buXmPalp(U0rj8&;NMO*>7Vw8X6>5hmKVGw7t{QX(daw=<}dC;Jtj zN`&?5y2)*LYLiDD&?qK3xb7iM)*dSrvwLrFR7eC>_%X7ylru~zLqBagKVSQ7SeG_f z%Nn>VC7r2M(oDy(!*Qd3Fh8`ZKi|^eI63wdWaJmIcl)t`oFHd_-ALbo{al(-QW6#W zSqHxzS}+Yf=Yh*CHDI4I`i4I}x$uz$mF4gzzgOqFNj#e#;Hyw1tY1g4plKQwtB}&U z`ya;$#tTf2j1NDJxk{V$GL@~aCpUFaK#oKyS+yx#x6R0)nRf=*#BO5wwwY>}(B~b= zZ_QDDf$JrSK5_Wk85SpiGG%0>8WBc&p0*3&xarqxwCOuIXlJ9CNezTsHeguQ9{Hh_ z+dPACL=LOmS%IqFL#S`7YG??Vp|oViUSZQ#oOz0*p%w8mN#{_Sw8=4FK`1{R5Vb2S zjfU4PbMdnddCjsMLR(p1o!oPWp0ee={9#Dg=0csC^54;CU}INE3qdntoa`3qjNTPw zHiWEn*az&WXw`y9X4Z zvlW9seWk5nc*7;#aIP%1+(*V&o$}a3QDY72hpE?0$hZ8*w@h zJB5z-RS$PO90v>urzXw((vcz$wPo zW+bZvuvfF8byTjmw8sX9}==1Qd*#eA8%8c=LRJ^dXQYj0; zwC4rBPcF8>1xj#(TZLicjtT$?M2XIwPt}@6EnFmtU+pSFJ5+#a+ zy8)@(j=6>lDK{uANT%xS%c=dYmShA1b&O#4s;^u5RhZHSqM0EFA8+Ug%1)5I(xW%d zGlvs?_}qQ~P-6Mll!iQO=z=-SIt23RxPh3d?3Dadh zT`P%-2Oou3B*|57S?Sq5C9xH+G)q-=#1dGTYdCWH!Im)Le0%YzHq%fd?<7t`C86nR$Qk@cLr-+7C62I!oa$pBS4mnfkRBLi&MZBs$DmPM>EFG|XFK*|$E*WblpjMv?4)Acu4CN1a%%n{8Sy_l&@bI$ z)yXkEd5Iz6N5%Iva!>rPOdr3VG>cD$yx}OiOf*xRf4J%~1~hatj7tcm#wIo@5=~q> zSA}mTfyjv{?m|&ulWj*ljK{*W-efl+H8}r3aDi7tq;ebf0}6Mi3RKTji@{9hEFUe! zW=s8j0yJ=5o!+UxrVZY9O#M3f@MwV#DKtkTtndvd=kuWA{8E;wGd5pcVI{^dO+c3bGoJO2_h>|gRU&2vghBiU{YgId^YnFS0 zZ@L^vO@;rA_Q`vZsG|3lyRqFfn+Cw3pNSrPp`Y&XNa|D`=94m47>L218t_Lc&=u*9 z0EG=BwW9BVm5m9vOLV^D^lC_niMy^?pdiuPpmxS{)&u^T$G60n??FHIxx3AR#B_u0 z2uP0M>QTSHY1KHbBOiHokBxph9TaIq)uY|4Dk?x4(LH`z2en1L42^ngtw3tdR=}?% zcX8v-JM8g{Yn@P6N;AG@oPrq$(tF9-Eyg?9-mS&LllyL#ZTmJ51jphRS{H-zJ5k=e zp;;WC2CT45scGPO7&kNjg!r+60|Uy7U;gE#bN%v+dn1-~$z`5m$BK@+m{NKuw(+sR zJrC#`JT-3Km9E(r+V(fFH_uS@DPy_5#89J<9~IGl*AcmKO2CDp;@Xg$Xrl5N*?VMJ zNlj`);hyFFgF1&$?wR+e+;Q+!0q+6%ssRs$+4138g1Gtt-Hib6 z)55NPjB>f5NdGl>=3f;$V$7SK5cT5s4@>^BBab>ZW(u=Z&~y)#^vjqO783+urJ=ux z9u|LAE;OETcXa}s2Mg#V(gz1c4z6)FICoN(@07*TJH$^2ZYVR|QUC)0v=Rygql0)}m_9M;P0ZjMD z^1wJ}odr)VXgzr!SF8tKGXATSGJvDH3c8fk%?v)p9|&Y&VvbExM3}_|?rL#sMKP0_fA*Q~RI)t}mkbWZB5B3np_< zHG_DA>-M7FwxpvOLc{F}qMslEiU|4>bud0in{=m5PQJZed;t72rYf=rYD0=om5O67 z+LH!az}TGMZ}vk6az$UR9PY;Q=z_e_q`1`RoaxH0eXqA1+v>J`E2reX?)c0Cr*3J^ zGt#qO9A7%lXdi%IqhTb<@#uy=yhRsml(V8Alqe@eBhf++JvVEi?kBPRI2>yaZ`I&R z#K2`I+@~IB>=<}yy?V&CR8=-Sd|EEL<&;4yw~NBxk}H@+yig#WYz^UFB5x@*~pjTQ;2y`7Y6M9WC zx9X&0+3Ys7<{`{|JR_E#Rx}_xC#aW}sngOtYPe5uCK&-y(%zI)&LhJ4>76)EXn)nYN83X``03HyKJXH?ZN{}J2Z%isPP3{t; zqyBjt6SBPm91#y?u)gE+oW*&?dF%MOyS>`x=p*N28#)c?p5*3#Xwz}# zZRdykW^3;2XY1iF0(OlKy<3C$q%Ol5cK6tBhV=a{b=xD$j>WAt3HtNEl|$RWE~oVP zLC3ur&Q8|I%WaoK+sy9A!z+N#>~4q@MnBxi9pPswlb2NC2#imEp?>GkA?aJ0mYAmW z_OR^9J>%z**7Q+L=wqeW%$RYbOoE$w%Qxg@d~^jxOpMOxmI9&~c4;Oie~-PEN`MtAjh0FJsE~ z5vw%LknPw(%i+Cso0n+1*UIiq8dJs&3h8BBysMB$->&PCe$HYS#XL=%PQt{6gZsel zO@`70FWnorxiV$dc7jTi!J6lMxr)ZuY+=!7x0Vl)WBbq{-0KMF&tPHFr(%Gw$$Z+U zBl+j-u5Rk5_@xMo8Z5{iqnFu@cZPhZ|bM4du!|u@jOqQ&VLq|z1x1r z^!-fj0RToclbAiiKQjPgl`D{D7C|00t`z@R zViTM9>1o6zyn2!Rp#RS;@3dS{Fvg6K)bght8+;Y?n_(SaBsndTIV{V&V1&Ro-9qBI zSYzAf&bh4?BAG1=OG*gPg{HjXbt6R#sKB=y-QQp}@7e>V;!82h%Hf@Hrdp3G->jZa zY8>dKf8T=5YdR!F8QHOI^imXRy-baWxyK`l&hBUx`1Kq%n(>H|VcTM*Ww3Nl z`F`CI0HIU4G9Kl74t>`3sO2$Ya2LbNMrJ2YXDe_*P-r1V{z^>%C_@I|@m)l|3HTg8 zF)E9%-HpHs4-yvQIc301f&WUuX_R$CrT_6Cr0bQMr|~-3;Egjii|&|p8Tf5McY&Vi zCq7&MeRvXRqNMK_NYS=V5;J+z>YPBJbf;F_+e(X2PSyBAiLDlsyUQ84G10GMO| zbkRyR@djXfk^p0)?O9mQL5_N7vDbj0gqK5s2!PVYJ$*>j* z7(7|6p@ssUgDvwb3gXTNzu9clsrqn8PH=L`Tc>`!Jm%FL40jfCY~VuXGA2$!4*ZV! zc7Rt*M~yH4s)p*oL##sxsF2ksmu(1~H^IU8ptzDw<9K2t8D40}vGy$MfZDK#S5@Mz znWlGP(=LxoXc6I=tDP28pe?i!{_Ao8u|o3ntSy(#l4EYz=`f+dDY`}@!3Z02;_Hke zU_frW0lk@!sQ-|jw8CbG+`QY}6l;WpY0O%qZ?w8&ls@e!*=p?M-=SOIZC1nMAg#p^ zH&+ZO2wtI=zM61BgXb%PwJq_MZG~o^F6^dG!;I3a3(8RNjMz2)&GJmaN_Fy|(v22Q zyIWKJ2pK5ld()N|47Vg0uo3Hq5MWg~f^C>V+j4hJ&CTpR-)-Xx+goVB|cH*??Zj zPdOQBfi-pQmd~;Jx6$+nChRnx^wNe!R%zpl;UKKgpTV^{#!$Y|jPZeUsP-X$BKnL) zY6zWdkRhZ%_MFuTffpj4(Sfy{%_%jWq}8!+QxDwMokg*FxqhOI1KV2hgoB%rk-4b9 zQM7rE2d`+x#|5l4cvrnALqrUp*spATjsg|kZOhk zCuFG*;uT2+Q4ymR;k1q>Mleomx=k}#ypl0@hqge4tSf65nQ~W?H3?GY>(eF_@0B&O zX$Z@<;%(wS4sGiIOGR(Sg4M*o4fyjY*3hHx#vIKra0SNoFa|DNKH6!BX2Loc;H);4 z5iB72&mkxgKKT+y3cb8vAkxKcGTPb15_)QkDQ=I_brxzS)CCifPG zWr&MHlF|yK7#8(K#ieD*9OBA<9Mih#FfG!HiYpTKx_Ed7G%SKShU6HjnI)&1m5R{l zM5j#Bzumjt!ve~onO77Z7R!>;XMmuadKI9QoANBElq=5d_P8Xe|C+~%m5deXFwy2izE|bJFNXHxo!qfW3M)@4*w5{d#9SQK zi_u&x#b3)`KWJ+}Z|jvsSrD_)a8HyzQVnq*%vh*jq=^f%rwfD)fYFIR|krC33ZNwLK z7h+I3C$y_GZ5fnlE?FnHYj4HjCq7C>k{@$thS-jh<)RgMU_$iU> zHM}Sr`PxIK)sl03@xdab$Yp#xLGB8@d?hS9&xKz4%rw}zbSrti0B0(eFupB1kAoqy zEz1e*97M8K-gaYXmI+oIl&-dC>q^xfq?V>F|^|R)OJ&m*h#wylqNx$DP9~Da2{?&sJ z>O&e!_XmKZ%q)AY2}i!2*9_R2V0#J!y5(oxdT(%RcA!@{+(k&9IB~k+g-R97>7*~; zx-8A3sVvj)vuSP0{H$V*Ui1*=OpM{ z1|$wepH7;OCHY%^Vao-bWI30v+PPcLM-nyFRkgL1+gdGT;A$3ZMxK|ImSzGN3c4+} z((f6$_G^XWC=!|q$d%(8tn^n@^2NjUrLf8%zLlD{!3R8|t% zO%pjmeA%@j?F1j-@GYg5Ytcw$vJO2&r_(C$`M_Ek5aW*QT7+Gxl$*u9kf;=64A?V) zFr!)`=&AJtk`}AmCb%SDq`?lCr`V{CYsqE7NhBZl3Lf^Ln!aacrbm99O3wEv?!;r(*N%eBNOwUmC)wp@qC=e=lou{rp+m( zQVSoT5o?DdqB6gyIlNk+rDWUeJleT2h<+sN`o*tpOi&-OK@e%*o;|QhWepcBd7Ob~ zQE4Hp25My8CV{xA@PpsYmR)?_T{0JRGuZ<|`k8Y@I6u}0ekUeJwg`6q-qWBTnFarm zu519Rr9R!6NudRhj06; ztpy=IBh`^ED$&5N%ak1?yC1|Rt?!J!st39d?EEf3H3n?ng&}G16~cxkshT$k;cw|W zyqQ&mWU1w+e$O-_$#eb$4>R1ktfDr@m~81#x>3Zqm}CU+=3<97n6vX0kg~0B9_Do3 zDhb@y5o-xr;}R~+aD0zL_AZ6}=^?Du5ud|};|sgDNAnEsM5{LJ^jHVx zb?GwLy2|}lFV76d!i3cVTqT$}Z;&cj#>OQaPd#&qv(|xaxRA!g9Y$9>R~50x&6)eg zIhRlm;Wr3htLuU`SRW5nO62snjoF9G5o&ucM+aqEyz~2vI?>PBXDW0T3bZ)r6)(gijGhPkl*Cb-GmZ)U>weEQ=3$RDR1DHi5$H3(l9EGU{!<X*H5GA7&qEkCvmoXcRbYV@{ zPG>pwZZo$46LHhIlO&yJ9$N|B=H-;Ec~diQ&`0dNr!lRDM=esrWTA)A|3=w41ql{q zYdUS)wr$(CZQHhO+qRuqm9}lW(v_1v_fAary%RIj5$8Od*AqL|UijDd7rn7WpebO| zmSC#DR)T$_%|lg{H&+JIm5sJ!rp@uTq}-G*w(@IH!rRim<9_8e%{zm91^$JIhc8Dy zmwz(%aAdan4$=bt5}(so%$Bvz(W>{LJe~wUBVKAs;}e4q&1;|AJwsmW1p52UO#<(G zAu?F^L8JPDd6AOyen92uk3?*CsFxPw6B>X-*Rp)yA%FZ-HevR3TTq-Y z`ynbeAk}0+b6|^8MZ%Bw5VLJ1W!dzJJ{BNx#ILl}f6>2S9E9($G0)bc3m7$_Ko6@cC(PATO2KW8Pux|x zP}H58U+Zt{rI1+P5X;346)gPRf(TM`-9vYnkW{r3_s1L#IP^(j#(JW5iS+Gjr?%xy zP>ZL(W_Lt?W*dUYZ8)zxWi~rb2HCUFyyN6&AhbB?1(8}ADFw)_q!9BakeZDKK2m73 z)C(77cq8{vdqET+vu$$*P>YHvh4U_WvBhj6Cb+VNu`1vwbtTes2TWaYbmg1pP+l2q z^TrgrQtFG3F0j1mF0_`2=FVfZw#UN@SYI(aRPD0=(xV7Ey>|U;%GE}S&wfJ z1;uf8hlV^`gR>G#>t4Qd*6sXE%YQ5AGJc;RveBf#8*5i^0*vRW9ExZBK_j8smt&&t(6U%?=Muf-T(dL_5wTVibqAYSucf^A z^j*1C)i5-XogSSAIG-@^3Y8bCs!H++FsdhlP4`6U=+{jfmCd>sC=(u!g++9Mgu7Ab=^aEwDgd*b_`|%;n`1&R$b)A3VTU zj9U;KGK2~@+BY3+?&JQS+5(iqD4wp^C{_-cvy74Oz!fLJ&0{Q_CPy5Jz(ataHe|xw z7{4dQ2BpRuB0jW)$>0kAodjMm1AMu@Ghn=MwdoTH{&*;abHdHuy$*;631IQ~fl-X9 z^#@JqHH82tE(0lCc!>x8dkP@06VH6oOWq!|!}#P{`HPXb2Pn@gRqC8I3#wg?puI(SY)iN-=K%`cG1FUpp(RoJBZ!hN z4Bw96>tzN5!ptL=MWyjw6P9^>JuRSf?v7ybeM8R>aJBlHH~GL@P&pB|byK1}$87xE ze4mZyZ*~@~mxbRTxs4vlfP21kg64GeqjiAC=Bf~XX8eLrNS;q7nqe(tZDG%iMHdqI zzy)USGrK4Hv(+fIz+44v<$|!JS@q_-D~ZUho$UPT8p_#B)9A_hR~d=xhpq|+9+Hgm zGuz^8>n0wLNh>hTj;{+?+a6>)8QfpcuyhhP1+CNJIakBe3e!mqXd$5{d6NoOghAuZ zNoRn?f@h*SaS91H;K}pK+0piN{cU=aoDddlm6CKRvE$}9pU6_<2mYtzU z=nXhi%r8>g)A;r@d9c-bMOK|k+XPk&5I3x|J>aBT;Oe9<26}qjyldui==SnXuP(|3 zyJ_;X5!)ZFa0Uz)<2Fj77cl-FTra8ZQ=VsG$4MWDc}~Fku8;D?+l+}-CKShxxcfFy ztdLstqOp-!6r;_^&w;wJB@n5|M=9`HhG+%iChghKGZg?9^fOL4m>F^{Ulo!<+?`7( ze&z^UbEX^7eAh?v$ZQm4H~33wTrv(?fgG#wU>EEa^56$ff*GcjF^y#_*ElfK?}cXJ z+3#6}ht)nzf{6S>n1B{-MorP~_IFsk}`B z(WQEsL~`lG{>=@BOjuy^W9YrVx1DHkD})Eeh9`%*HUPh{-Lf=Nc5kTz-wdY6$bNTV z=h8mH_{7^GwyQ$+)Nk!coTqm{R&-%A6&~$YLN}*1A?%AoIP>|i~ zj)Yt>D@UZV`4bZHm++0Te3{IG6dxJ_uB;~y(xfp8>@X26;1l`1M;6@o%wUl2NNEd7 z3~h2Yh`p8tZ)p03C7raaGba7vBBL1G^1YK@Afpt+eyWdH`$W#U%qwaNBp908oLTs8 zKgs!}Y^!ML!3z_CzJXV>$XOc5Hr2}}l1q2C1Zp55gmxq-879mP2a*h9QaOQ^vL(=( z0hRplQCTpjoSZC7SxH%Q!t6owGnp$?{;;0~ZVW^{P(bsU1Xmt5;6+`fW-@alc5bel z6!`YO%#|E$dpD9x`_`@GT<^_38**~R$VH|wu>dV-K?0-?mF$=m7eqTH<#4$}fV%Z? zbZ6$OOR)AFqZ8QsuaDSDLN|Tx?Qf^|Xh^mF!FCho(%qw0A3By&YgkS#eo=i#6JoPB z0a$6icY+2T>W{Q>u;zxCg15rs_h;F=PX}&ZBKjc&eAzi;Ee{?+`6odYm=_okvNVtu zzqiq$7o%QG4NG88%s&yN4%G{G}-2(v?UoPo247VIP=IPO;>nM*7#b7@BhzD>D#3l?ix1-4H9 zT!OO`Uz3pUQCs=4bIwk?c?GB2!oI8$ln_c<(Up178Q}(6&<1#gnG9W7v&CK*UI>h^ zm5^)>Ih!nkst;UxyReMdUBg~KJ>Og}qDYX|NniCKx9)IP?4Dl0#&rmde0U9gfM9x% z8u|;+73}%nVb7oztIHG7UCF=k#wXwiQ_;H;en%VKNkc9`F71Hy9_%cp58Y7c3Xn&E=s@qftJ~o->MG*>JmG)8s81i zj72~B`Xhbp4(zgTp3B<4YD2!KZT06C+)jLTL%!#E590^S#}A&38$Ka3yh~nb$^%|% zThiHIns`vbmU#NW>O(V!58YiNRL+D^i@?4kw&S01cqjhplVza*@0`|PGk^C$xjmil zh%a4#{|jTE{P54V<^LFsP9PMp=R*PjC}IC!ZOi}H{)MWGrOkiLpr$oFeNc`u{mh!z zr{%~1fdE$*h%kPe0YjRH$OZtBb-73q?L%o@nJ{aT*|{BFg$=KG%}en9G)A?wwJmME zER9ED8Lez>8<%&=ZLfVdhVlN~E}Z60pNx=o|H2R5o$+~`_U6C+nE4ot{`S1k1gO5V zN6nr;9IK_mIa5>&It=x|x1%1p?hvMP@4EM6zj-JCAU|?ykCN>WS9__t+vD5+2)N5l zSq9(k(8u#S+V2PlAwPNuVDObtGocy?Zu(gVi{B&c^ie7Cm081cJ4|uNO9B7=-VsIm z8Un?SmpDm2+M>tKM+%>xHfg@~TC?RHa_3Bywy(&Jr>scH=c_03jC$bsRH+e3U*Q2% zeta*;YWdKu~_APOM=YnZsmWG=RR+e0j{dL<_Fmn_tS&#^ZONBig|3}~~E z*_6fEjSXv!u9tNRH))}VM*Ph@t#V#CTkfx5Goir+g{>%+b6v030>x!hu_~jn{kjUg z>Zmbdvd$_u@@Wg$QZicJw1^AI;>?uXt{`ilL(S=#I*Ty+5`5O0st{T2RI^ITe3^7p zg+R=BM@%VuSVt;{miji_M)ylu;*GRMVdycVuvXeCawITZsN<>AYgyOc*N+aqijyo1!w$VNN!(y>`=V3uBPe9GBBsaB*(uDPc^^%ZJW7`J8&3 zGMo=cGP$_6q1XD{oZ6kJ@g%XD+8c?rGo)=3UIt8?MaYB!4#fI%i77UY!{4@PJ1c7V z?H3b$rManPdv7>X)(4kaSL<$Yw=8b3sGbfgPua01Zs+S9N0BX28IC0DYo!HtVpB$P z*$ZOAj=i#jq?(whQw&N{d5$4$wwW>cDrt!l3PNLLUHiD)D=AB+t`ut-p(SM0JBA_2 z#7QM`GJk_U71NY4nqBNrK5M=>e4SgHBtzk;D4@oRw{sgu>RwCJdNNqyG(m6vbeNtA zb<>HbscCFI7k3h?3cA8T9Bz{KoL2E3 zbs&ONcW%QqTW~?Byl_BHhTF0cGxBQMG{X0J`>rgQ#s8&Ktz#SKu5IWCckKP6H?C0{ zvG%}jV@P(kX~xPMMc?+28&1D7WzSo^L-kP|R?JaFnCRW4TH|x})gOj_%?iu4SaHCl zTfKAbp|I;O*~fk@3$u?L-5qmBQ86=boY*^R}wcqQ~J*&p}N9F`Y( zl)q;Nt#1AvYO-(|(6aFqP5G-cOn)a0+dou~YLguZvV4dA`D2KF=iOGAb0IQ}e zKU58M5=ZiZHn|Wl<>`#{s}$PumHF?%nEvQ$;qUl`UYffxN+ez!eTDc95Nv8SgUT?X@aIZA&EZ4?sB_+ z`*rz2&B@abZSkJlWTjM_%mW)h!uJ{>uQ&_E+hWA=h0ea4;=;uBm>)qVJhh%YF2iRi zq0bsTWbfEdaziQO0Zme6?o~zeee5E{i{4lDH3@05QdY^RjMv%S*BaxrWSd!(4YD<` zabi;^O8R#_kvb-0NMWg4RwVG)OhT>lb;j=?2RYn0=EX@Db=iQaM2f~?VB%!M0(H&_ z6G?H&Z!xQ{Wsmf%Ga;V2hVIBS+oHx{#owlf89A8n7pZQY)0qrflFit_&SpnOx(r*A z+bM>POYDq@CCj)OMU0(Oc7lu?e5{~@doeOa$KDMX*$tQvhDg<8LD5lq)}0rxI;NneN*zy zH;=-9Uovs*7q38}7;~N&fl|G(N2pqiSjjn9H{vh1ZT}=;E&hZXn!gaP2x5`Zjd-&h zKuZ^#GKoFu#em%m*}t-+-+9+2Ij*oDczYLzaN$k$Igh+^m_&z+NRP_V%@xn&xsj1x zQlR`dN^vJY!h*T-X#@h~Sy5D@IYmqm8xf4@rJ#!iW7)oldWPA|5m&*_0*2u}mo1N@ z;*+*Yq|JczhX#JQd9&E~uh*Fa-avAs3H%X2+I^2_lbq1q z-52Y5kJiQOdLtMzaA8VB@kd`Ke{?G^?n30>R72<`D1kP>gwX2=tgymqDU91;HMvdM zwgFF&3FmEnEGGMe6E~$Yz^zpNWlqB!UsmdWQ}X{(B{JreA;tHDeD9g#nK7` z)idR4P;kftZb~>Lvf!06E0e5k!hThnTt`2%>t^+N*g#iGUAb(Y_#dA`ur@{tPon~3 z;_ZCVu2t~Hl2A}j$AR+~E@g^G^^yh^1g_}NozGBUkLutEI_Qp1RK4=m)S!Cwsv2bq z{E+-#3WyG+yENpNQ32ivsav&BC@@B*(W}R_cU{UFqgxua1=p75{3{?lTYnTZ^;)yk zbv3FsP|D}}id1-mTY)uB4Y)eJ<~bs)5cg?RKO7|m4wZ|!&evMvjiI3C$w{fXqVAp4T^!i<13|ZJCpTJB{JO+Yzv}C?F`wFGz3P_i?EM5 zMu7NZTzTSF4z_K?SD7o=8L;il=(dO2$2%5-xgDwTMjUxUHt#8J1}txQg1a5i$UXeJ zvro&tJ^Ie}9S``hYm}FiM*4BOZ(*N z4k=JN9B6Grd!-bjjOvqoE*sRhTWI5X>KERLWnw`+Liw&*wJ(f2;(F-3?i+Gf?cSaE zcShSmkGk^w^yD}Hz`8>u{=utTV6rQSVfl(J>5sh4b(`&2muu60&>y3G6k)*qRAI5?R)JO3=!(hHiH{4Cr4=huH` zernpzn<6NG*_>!J6}4Ae0+E0qOP11GiYzx4IAkN0%Vizyra_g?g01M|Fhr4>kT1}~ z`seu`hNQy-GgmSRnRWEu2jY!5y4mD(?kEO52blld;N^bx+0Sx%*YN-Sd_xCFzgHM$ z#Wi=(9Sx_lv&fi36jTcjl0;>9wa8d-gvO6tcZunX6&>Qmi1Dzi_QP4~a%PP3+a=V_?YYY>UHAtxWF-V~i=d^6_X zOO3kjG%Gax-A=P@D_6@bV%CFX&Y{|%+t<+h906XvLL5#L86>`Qsb&!Ex`Dy}4$->wd2jKhhOaeW*j)LOp-~L z@MvfW=ShlXFA20wmo=+OwWueP@8~96l+Q}qOHpkBy|3l9LIx@m7DqIYa2@o986#s% zFOZS_+srNmX!RJdGuz!3mAiCpQE?R^fnLqe4kNh%q2tZ~5E^FP_c*nU0pujr4fYRr z6sE>teSm63WzM$K-KW)b57~$6$qvIKfE74@-x-*U524L3EOK~qcD)XD>5u%t9gil0 zC$pvSx=tlnoA%9` z7(pxs69>8Z09cres;~FP9S|cU0pUdxb6UY+v5>GIUl--stxj6!b2c(syiqWF?zFR4Nw~U z0sb}A48uzUQ2m4&a0~zdiGMHD*w`o=+M53FaPu!5!`fBZpQwWYerJx%PDePGq_ndE ztCFVz)FH($Y24VPs!Rf`AlsG*u2YtB(%#ZmUMTTni5Q>Gpj zMGpT>Q#!1c7ihe$_)H)-IZ`IC1TncPLiI=e8s`vO(Y*}G`n3G0PRi{@3QTT@FUPNl9!q`yQfJXH>*T+Are>F zGD&xrYB%qGh;V-c!7XZp%qlU?I9`7VREucBr z8_o(x_()aMcUmGh%uBPlZ|5>e@Hr&W3-uyKj90Q8zOttxF(NH$-EGa z^#YTty5clu3A7>_>zQXvNsS((lC;7!)+>>Y@**WsE7VKASkBmO;-Xv33il#QtXBwB zwAZSHt-#L0IahJXTI^z6fq%a0mh$35?4n$Of3a#6^*JoD3v_wD3ZLRambewvRl9me zX-SiKIk>A>H9CsRyo4_h%W`=$ip##lt?&kM4ir-%ow){tv0o?mtY;{@NimWt(bSSlxpcsaQfon%c&W zvMGSbXl)22v?%7>8)oYyyQXb-Kx*jWe8>KoN7v_yn`tpDn|eJgh5xGcd3)Lq*b(H4if_pgZ4Fh1>Qi&j6$W8v0YO}AAQl{@3Cqqn z-YswR#9VKnI85pDTP)076Ru+vupN~KQpB8Q+-eKu)sg4a3w5|Av$TrEN^R8P$enf5 zCB{eA%WU{r?U#f*bklZ}DwC}+UyWyzUc1itM5|6_TzUI2vS{E$k#-soFy~|IYiPS#Z3FSU^qwe0A0}}r=nAG7yM_K#?HWAtRu}5gzOgSt-IS>(X-@jn z!Ld9ov7exoa>~fz*6+s+2*I~pn;~_0L5yrvtbY?7P1&AfYR1f9}w3VgL;BibJ6V;#my$HHiiWSc>}r2)wOF}_~MG9xcL0&f@KB` zb~yvF#nt-^ILq35$@TWmJ>sOz<^rP>Ej-ec&$&dt#f`nrNSo4MBn#ydYEXZ&-@+w+?Dyl9JQi5#Cd4 z5cJIPYXy)PWe#9BNVO6@knP_ju)^-y_V4UKZeQoIc1m?KdWuIdA2|DpoIiIZ9^`+F zbREzWpFMUlpHFc`{N?`!Iut>g#O#1Hlj9H0 zGr;!+&%*Et1LQ9%HjnrXYC^%TdDbDC(;jL2Yqg|3VN>b{5H50)LUdtn>VbaYm9&^( za{xmgu!=j9PX02X^{h`g&^l9kZgE;Q{2H(`1-ecs1+(fO&msb9l7sRLM-1upt+%K5 z{$H3ymOxHF#99D=GuZ#Hb)tW&ri3i*gzf)(SQ%CSPwNJxHIf>@qLS1!G^mvxf=aDw zR8ScxG;BSnD5KWvB%P#DQ+7Kfs@->BuCF@(0eZRdnI(p$%oV?T;ct;|$?{X11%#0d zg$zn&?^{mySF^J2G$<~UG38X^$*sxX zikv?v4oNLc%DK!1Sx#*&{U0`p?Px)=!Ed5-(&?4VdOWWv&RzlsPkghVW$>0MnOw~v zL&HF1T2l3abTWWmiLMXkUUnnE_QYa3UE(HXG_6B_sVAN{=#LBO$|5R6{@!I@xVHOE5~7+qK0}TYz*k$VQEk+y)S^ ztSSf)%0voX>=Y4Xtu@pQm+iq(;Y|(_w{6yf9F|rg0p-p5Nn~qjq2o?O4Uk$x*q@e3x02Ldl-EAPn%bo)%Z6aJZF7^ze0Qz)-+{e_n~82jkMvrk+_dE`eIA9ps}?4* zn*}P=%ji^I_pL$pDn8J%qfiS}V0VF0FeO2%sG3J0V(@#p0tWya%Y3LDk5CbPdF+2h zB`mWS;^f!UG*fV_pP3ZhkrHIkP0RWb8OH0YHHLeAMLWrBspN=i4v2Hq5KXDZq61w~ zPwy(DqI)u?v?iFjf2Zv#I_U3h?6Y?6T5(+Q76CNGK~z znrLOCfmF^RuvLNqB~PPj3GoOWnly?KH!3vPU?H1pJCr0PD=C(vT!a$Amu{1zpbkPk zg_IM!Lgr{|67Nd#JY}`WHM84dZ%F_le_&F0^V`8JH)`omEbY(q8A#4fWsaTHncde( zb5q-&smzGWe(lEh3gc~^eiLKGHUWI`9rDwI=yPc8lHQ1S%!_$*V4*b9b{NgucIZZk zhPUrKF8Wn<;lFT<%#vIR;n((BnDbIZAh@9rcAeiEX4HLCZUn|tcX;4`#ceahD>waK zA_#*Btwa{-#JvzM;(5v&1MfB79kVchGoII<6Z32B-rET~7=rf5U-`6fH<{sNTaNNs zHVeTMds;;HMXoAGywE^iTFX}0#46OY4dRl)tSQ(jWu8&izk+|oE4<|VoE(g^yOq~g z@Q?2epw}8p%uNE3#K|a5nnJz<5~SfO6GBbUqI_ov;=rt8mjdp#dII0*B#B&;j!E7h z5Y{2k4$RzV;`(J^j;RaJD51mdxDB@|tdr1DFie>@zCJ4L zoLoxRA5ilvX|Fl_L)a_cJD}bQzQ$U<+ zOR4i20gq0wAYfdNXbgKfcUyp4zp*RR4i*1SBqJ%~!^r}G!(qyxAMP^9a55#(bJt=m+_}$^Pqbtwc_LI-`Y;z{w)@RC{Kr4Vmk_v4Zbp?f zEucLK${$k9JS*JLl;Ru%reE@l2VWA*NHL3a8=;6swwIQ&VVTjKyxt;bCOBJqE^xNkz;uad&m`GTz`w zsM$vJ?Orx1AL8^!6=Q^`;zhL}O8v|8j8GQ~NTPk>(lzY(lVBrc;bDjXL2s&oox_k~ z46^IcYWz{6OkrNt9&TPjK3R^&cRckvgUtM47Qz>Gexc$>A9V)OLxH9zhz8BMg>U|x&K`IG z0KR|ggbEuvyZqPlS*w1otgMdmt=krICorOmI0{M#XuNz7M2$j&C5T;WM14){8B$J2 z-_ujz()!V7ulyb8=g7rH%&_uvc~juOi@Uya(-VSFz(x!o?KsbJzqqe>{{W!;{XTF9 zm|uTIqx-CdI&k`Q91WVnzxQ~|hb!sa7lM2Z^Yt6#TqQcfQI7AVI{JG=0}9S8H%V^Kx#5F7matz5aZ<>b>OxQe?dl`u7An&3tYQvSEe*lfr}j$+GR2d0DF za;?jF9lgVXx@5Z}wIx<1>fOA>{e%OQoqqhOj4w0^j@pW%1}YP>n4Fi%VlDk?uwx%A z>A<(k_BYogjboTB*xebE&Y6MJ!xed$1l5(EQVmK}WT}IXsGu?hR?)ht7l{!GZYXaw znN-T3Ov>cnr)cbu>D4@>>+iR#$E{ojRxXd2Y`(^4qTZnHscW|P1C){_ZW?I z9j?A!qj?PkM0P4ZhSav=F?7mtJ6WCJmh~BAD$Bxc0}MMMdRh9+(FU^H%hW&5r@g@)`{C~*9i*z}4vplal!|1O{g|?uAt8<~u zSbamQSbc--Sb5^@FyBccr2S!|L(_-uQRE13*9m$X&`j!8mT9<757HIzUE90avQBN9yVdB zb|^EjPkI&e{E+rekuKPk@%CBpp(|5NAKCaDLb%zGPg;x@;FymmV7bzV%V7*rS6JQH zkB>{-@3^|lcw82ccwN&}>&9hyN#z?rE0xz8CW-?#i56gA)1;M_Ai2Gc#z?i}6O^~s zmcqRzg7GO1@Vo1!?|<)I1t#eYH-=UN@_kElsJ~gO6nH|3{!N2K*>yjDfx;ifx!gbI z?nUB8Jo=r#9_ahI3Pd-`C&^O?cc6KLQkCj7dW1J*=p?erdXsFg4eVcZ1XqP$*wSBS z)B3VX>G2nWM=a7GX-9LJwJwX7r329#Qcb#9^y?i)LhefqXHb#Kidc83GljoxrY4wt z3jx6N?*Pbe^Lbx7cm7iNqr$Xfg~^WQKny7_Cp7A_xjgDBGA_^1WYb=4X&DGP$YgbgiI+p^R;_I3g2h~gad_{5rLT${m4T28peCz&s|bn8fUYUB4V;Uz>YhtXzQlh?RH zxga0f^3p6-SecJ#lh^nn-}U#d2|JtL@EX}3BtSHeFs0oa_5!Jo^o)IB?*Mv+ZV{!< zS}YRUM0Ba)m2TB)iuhq=CiylT;`k)9w4E(2M?K#uurAKZapE=cqt04IPKmO%vP|#` z%fwQ?q5rj;R9+nfMfl@gOY@W5{CkOwu)U3|t=)gUIC2$brGXg{d@HThSF2htbQK5V zjn0;7^`e3rDiqFnb%)uSvT4?JpY1)7cwT`&6%S0Pp`sLI2s|IncBV)k|NMP`DhQ7S zaMpO7zi*tGyP~fBtsIv8btbn3od=bB_d~{)8=6=PiPjlG{^gwNK#TA6@K~**rHJJ% zE9SvqbO&~###AFFH91LzE$vM_M#-2vF+zk3S)KDG>2)lE3iH*!P)?>;a`3>|-b9fy z?_;n)wP>y{BB9wbaRJr8L6$!`+`zsFj1g2=X+txcdTIA)UTdsg01E&3B>iolNtmOH zBU@?T+8zrwJxwc?omJQ@xdPf4_W~c^HESRbf_--ma|Y%{VSx@E(pF}uK>}~p5`sqR zXn{$MA`4uzd=3Avzu!Bjme=*C@+N@!|Dpx|Q|0|rc00Qm+Wq&NTkY~69lh^uk_JLe z>4=2)qM%Zwq`A?mS|HKQ6csBE4XqHq8fho&w&rzNc5Is6&m;5xg`e6Rw|qt-qm7xb zZ!g9_<&ZNQK~GR%Vru8Kv;UF%G|$=T`|s(oJpi8oVMHT>(Q6H?q0h{8DnlJn?Os;U zJpmITMF*Y`b)NzQV$l~iVNWQ%@Ptv)Xn9BGNh{{gS5$U3Pww$NEoeLF=VTWp!ZhDP zy6f~(Z4g`UtHj9`B48^=c{VC%wsw-J?%*no8G1~Pof(YfiS;E3o1x}{WVTl)%_Puz z%{2;3Z^>et8WU6&X+Je$gW1ZARUC`!Y6C6S%D$C^ulHc2$Yq5g=1s-`{QXxzx%9%k z?q+7;uL(V2xun^F903B`(^3KNm4+jg(hiDT?e`)|-rI!NoEr?DdU~)e(%>E@Hw5Un zlA{TyOb-joO{(s$<2d%T33$LXG!AzP@`TMN9l11}z`wrRV@b|p>jlDjx<;A>w+ur- zh4~-thjltg!`w=4avNa3dV$g}zgRGdsm>>>}!x)?3Gvs9U{~ zLAp*^g>4wVy_;5x@nJEov#EImrBpeg9k%Juc_2;v6T4f}^2@@vh%h~`u+h`X{k%Bm zPs@2jxmv)%L3vZSdHjxez&o-+qHZ`{qINJJLr0h((<2ZK-|t?)t1`AQbOQwtCg`ye z7z+@7pI1wENx%Wk*H!iOj<0eLvHHc6m~5|n*>A)$ba}Vjxo0)^6S-C%(G%2^5bFx! zZ%kyw9t~Wb0mdNkQ~e*Zj9H3F>6BDyB!PyX&^}~@KW=1%C2iR>qVxP z^nv%l%)T-KiK9Qb3;m8x)T?88Jj84MTmJRKBniI^;vh@ie%W|Qc#G&A zA{_$qABD30LKwc1=Ulv^1NGC+GhJi^^4x1|2&WgGL6FtC8DDPSL@KTog!?~kpuc$b zX#>6uxE#}6b~aDaJ#vTLrqR#ktg}wfdAuq9K<$T;Z2P73tyZ7uh_9MhNy|?(i9-MZ=e@Lyrm>?OWI)&FWH?xDlRe1m`34?AwAjU4{ut0p#N)7Tf;Z@n);F8BL7TG zMgOf*M8wp|)a>WAGybQk>3>wrwd$4zD(a|z^~|P^1cw9y2$lwsB$*_~flMVSSRf6O z&5{KYtXkuzq&+w?G0h+q*rf_Jtk@gX=A@vly5^)%k%Ur|S~aY<-D|bbw_duw75X1f zOwE#Hq%{1dx=-_d`ir}oH=X8r-?x>E0Ci*jTOG^6scXkWws>m|-1wnuhl4^}dI7y2 z@_;u3e1$hI`@M8q$79=gD0_JDJRPf{A13F%59_!4gD>6;$S=_m@DqdJTdxH*xDNXT z(D)ApAl=)q4KeV)%i=p6CCK1^9qs$jj2Xhm?G@YkOvT1e_QP+aoooQt(%dcSVxYdrm)&5dbSEQ8G-=Kit#8C6ZGqz$6Z1Cl~GsVvh?t<#8VR5 z4Gg;&E(k|M`KZmt!tVM)meOOXf z*drX+%ZF4FH$+BU(@x^ecHZc^QHH>|QzlW4b}?rZHbq;;+qh1!w{d?IPP7ds5BhXH zpMe*To#vItj>VN2Fkh(<|x&l>!DTO>VYl(H_;Cm$FJ3wNwlV`Z$hC1H>KEA-H~mSgJnQ6KPW>TiqV*@}9~jd$EG- zs^$LW;N&^^d%SG*lX2=k$x6wmq@&Wbvqy_9i@T3nN6hDWr39_W_0-i#_AM6bp$HK) zPQs`ePnqGRM;rxZ5!jma)X0k=*$GN*;VkmhRym55$b36e!kfq}q#IQR`%i(TQPd=E zTOJ3x^cc-DX{>?*x=V7TU2I`^&^5?>aVGlo5=&4eTgb3i4PVLVAy@B?liwL5Z;2#S zLJLTl$uu^#o)TFZyuCuhm656TOH$?_1LosdY7L8pL4&m3m?*Up}}c~B^P3hS#HNt&m_F7^NtzX6m(^8=R5=6%3?Cp z+>Rw295Ie;lgM$lvDL4|A7g2*XVx&3QB6+iH&rCZs3|R#MyPz#%84T(Tbi=#gf0T< zd}T*mD^(x7*)9jQQS`eUX#JFXPgMOOXs8RiL6K@2Dh&pyh&8ic~Rp;+L80gi3e9vqdr8zSGGQ3#PmHWQ0B2pW12%# zTCI4!Yvp@gANkRjd*BD3hY%=v@$@MCBK6Ve*F1eF^i`QneX&Lw~E>(|>9UK93luKB&1> zR85Zci6dJX9Vxb@X39I)qe)%V>e|Ho#`^8tv**(*#`nlm2^#F<)STLGm$byPdqYhQ zU|>ly;-zQA&SFIgpg)Eiat+4ogP$$^C#|FhUq$-cgmVt zRy^*cw1B!Bw2|B+(-Zj;WeP8pzk;-7fXZ(rQ(J{nPN*%TDSGn4=|xO3I3QK*H?_47 zX^d)T(>dxN?nAVrOfjGBHi_h&>ToOB@A~C-GFUeVvw{rVoBb1M!D^2`4}W07mhUCd z(oSzedW<`J+_PIfqWgFW=NZEjr%k!e?1*#D5~!t;+zD+bj#-93<3;Ep^Pa_iJH}{i zf~urHy2ZN27MX5oZ#fFWThHK)f4WcSH7C#L`2M|Q4}1vj{KW4^0^ruQ)aO`nObhq& zkf@+Vo`1wv07iZ;C~#ClW=zWnuEYElFk8z9*zT_9GNH7^7iRG3<9ux4yXF&$gzgz!cT zxYh(vLlBS(;(&WHH|5E~3|znh`;Q03b}K!cJwa_T$ID3xPD@&kdqs23E+`j;Vf-70 z2mHq;H7HikHR|IRpp4gsKUiLrpUlkb$mDf_68f@S@z8q9A1~x8N)lLexRv7ID7J() zXPUiu`Px#JflL7ATK+V&8|{%EoHiaUUd^R%H8DIt83#Goj{0|cD5 z;vqT5vZH-~sB=DhQva_EF914k{HwO&wqnas71EUuFHX$D0sE>{ij%GXTkrVPPJQ#xnaNvs9*Yw;9s69olUrrOJQMf_FHQcb#KNTChz} zzEH#eI4@mvq8PQqj5~F7MJ7;XQO_Qza7AB<7cfPhpxOFEbf=&tP4&nMSCrVQp0wbr z;|{14+I&GcRG4WUt7nNse$muP{_?{0RuG4btVSg(lCJ=TN~OCRFB+YxLh^1*9{+HZ z-Tl@x5!C8YtgK@3?Kc6CmLFIRUr2eNrZPb(9PSVWnRQCG)YvGs)T1sBTpngv?^GL5XuUhRI?|0~GJ*3;*QjoVQI1J5x#6Ui;FF4G&VicGDQwd_n4Zyb z`&#MsjN|D{bX6+!bWevZ=<5zl~*E8|eq_GHz5cfS>GY{34vgGdRH~zn-i`Id5asSZr0{+y%|6U71 z#L~^uX)HR5DVv?GH*qwx0x;IJ(d z!nJPcsMkf%qxutZIvpexOQ;g79Z=(?;XWy%@)8s91xcwN097TK7dIb$_2pk${?zyT zf58r*^2|AI4{&bH-8F{EyTo8a5ZMw0K*F-ao@KBwgvuXZsb%;vVVRYft_)C`lpeR` zY$vvHp1Z0Ko5K`B?ngY3497R-T%s$dnIE%z{ze{68Sa0ynUtjU+G_t@*Dn(7!c4wN z8`Wc4{z^Q1TAJjLt5Ro{WvjupiKA~0A=eE$<3L7^^A9<1!bkOra5l|+j`YrD8vRk1 z9ZVX#Xry5g>;`5dG1Mmjb)kfQ81y(7URXe{WDYwwRH}kcBvR9Z=S?am9h+>Ni7q;+ zAkY$IBW~FKAqO)?b36$bB2p@uRP(N#HrutGCdB!xZy4RA>6zPP_Lez@RcGkFOJaI+ z&}JyA(?*lZjLKek%t_9nga1aG6scSjn$f))`ML9ElHgnmoW?MgA8MY;8R$%}XLMwXCb(V6i z_87EwrtQH$Sqa`s60@t|*xENHA8WCzBE@p@?`qU|u3-W+d4?J`#U)yW<;+Z|^R|i_ zE?2(`$Lj1I`zKaK0{0~+P1>E&$lt*O3B8!Pz~9cnlF`%rc1xDIYJ*v@A}OTfbT&j+ znYCQ^X)TKpB?w|jW~wtEMmEpXCBSwD0rn@>yFKJRu`x=z=0yET6{MuiZDaNs!86k5_Qy{d|4B~z5w_J7!Y*}JQMy0 zXYUwYX}G2BR&3k0E4J;5ZKJ}9Do$2x+qP}nwv&oc6(`@Sy}NsK?>^^@Z=av*{jtVa zKi2g=^O^U&ro?j*q9)aQ_VAY|ld+Umk---e@=)0vqB2sspgr2ik<%hIq$WMRYRGG5pukU#z<>mpX{rTzen&us$ZbN?6S(*JhjChq8JWc#o8om1-n z5uo$AG+)C^Z+-$(E?*Q8(b6%gLr?rAc4$Q=RzixB*88Rs}y_F5SLp)Ryg&Ay_!d_az3z zsOhdC_aPt{nWD?tD6te6VRCvrN@sQ-qtjb*1uL#;9&VvRZSMPRz$QZ&FC?2RnM3&C zPqh22G4nTQ#sMsn5F~w0){0FI`OY%fR}84g7HCF)OsLN_B0Trrn>AUtMaS8-V(MgX zBgSBXlfv0c!#-FrNLJf@-%9xac3d*k&X|^}GFF3#7VUVt{6q2$bWh1Y zLUKG>Xtf&oNKTQH0zF6B*u0LqE4{ZUy2)rcK&?qi2-rR%WA;om1hKmXDL^Aprcq~a z<;l8Qhf2F;f)JU<2yJf)e}G_g;U{%yr?-(!Zj|+aUH=*w4}m3kxR^p9`2yFhRV5SO z#KE!6kkH*I-wke;;fGK;=R1%9+eBW}vg_FaD(KRAsvM4<2wZ@{XcoO;R=*MmnQ^X# z@Y|ZDH7(h#^&=nD6%ve5If(N8wke8=k6QlKG9H~B<{SdwKyfyUE^(P=Rx?_}X6P||+|)yT`$B-Goi8$tFq$)H(~QL1poed!o{G1-SRX~6 z*ew$A7a%$9WRa@5>f3ak8`9KwSJ@Xcj~QARzF*oK2b=F=pGdo)U7wi{pW-A>&xO~* z^{}RrmP_$4ap|p{A+t56!8f?CmJr`UO+afF7}r7v@!VO3=9z<9phMgS+<8NnZxAp0 z4+%)7m3nV6vJQ3tD!#;T&|t5jwv_o6i#daKf-gMFMWT9 z1hB97`AKn${)XpfD}h4-f5USwzy1-PLzdqEra|>96wRm;@seaZVM$Wm!0?~pIYdGG zg?M42VirFf-67pvv!Gh2&`(q=Bek>Sa<*aS?PA)n33)Y)!GIun${ZnIkfaoY_*bqb znxOWO1;sq7HOZV09?0x`!BERHoI)p%7HZCxJd-pojnGSlT<;%_C#PI?gZh zMv|Y(|IGF}`~E+VFR0ej*_&8jzToJ5`6Bag-F5+4qWWFQ(nHnB$llrZKllDw{nz5Z zBc1TE7V7+C%gz458QKeU@Zg-i_f6~Rt)X*vYmE7e0<1;44<8j;VTFVEx|8Iu> z`q;vN{{{R@CXF=(eFBJMJv`DLIu7Bm6Aqn&HHTWs?TpJuTAv^*1)l~a4qe;V2w(FF z&ETc4n<4C9myPuhvqPN8GbfTfeK=!e@7o%MhJ#x}dFiM7C|cQ|Qudy?Tk$n+V&wai zP}kdHL`B%)G^AIF2WVEQ3(z#~75P6WQyij#s_AWwNw&VC=Sl+hPR-7G$nq2c%BM-Q zOanM?@T1J>Il?%@cMVzq$2VQN0dSw{guFyZ6S(RW2QWdbn;8gH`$UHX_@m7;5uD;* zS=OzM>9QGZ4|S>SO&dD>$s*ZSv3KXTPxVeVJf?w(HctqRn#3Dgoa4I9 zl!GPRc7QCqt~SG7(^q;U?UlXXZz=<7*4x8LvSyty-;<3X>`#t$S})ZZw zP7+)t7?HWP=J47cB0f5YMA2`@n6^r_48~CW6aqxF9Vj}0WV{H&JUUE>q6*LtB*CAP zebt?sQOBu1vXBCtKeuTB?D(9B(6)zfE(hLqJriY@bjwfX`5328P}eG>n0rTV9AR=| zZ~WkSZ*qe|$bWtVWIdtxP#KG+YT^kn>?uBbq;~L=9W-2Y&Fnj)eni?m^J(bE-iX5( zh>SKvc?&ENl>R1_tGRo^m9f>&=u&35<-wEL{H0wpam?uGft&dwqhhe6-4>)|M1F|2 z%PM;9+qs%|v)>3O3Q5Sziq|h(opL*|CdDOdorWrMY=?H4QvYp>8CQ;0JO5lo@m2iy zAKyq(TW*$@UeA( zmUuH`R+t`BA#2j5AQK3A#l#Dz0< zSxT?6%Fwf^4f#I4XZ*xUr1VVdI=8FuNEmy)Oy!&8Ng4ErB$u;U@6VLkvJPV+ZKfx7 z$&88~ZXFU+cbA;xcj1n#VP&_V15)%#rx)%Z9JGbSXL!a4*I#Fj@!qmoJo-vgWh!P^ za6e2_%%h!0%i=*wH!nklLd*nSQzgmd-RmG;|)p<-jp$Ar_8{(UTAEPwqq|YOX45 zdNxVAtbw9aP?R>aGkwzZyfC$r{Nsey-;XYbQal-bq|Lrkv5s@f-5e)(o?9FGEW z%`ifUA{KKT3D-nSF9iZBmWnN>AkiY@D?f|T5l@L0G=j4G7P}o<%LO6xt1$tQWf77m zi?d=2hGk!x*wkxa3R+gh=v$y?y@1ZusXF?3XtZ$xEagcl!moCF@PxX9gvN)>-EqGe z%oZ9d6Dv4WX;@_f7l=%K4Oz~hTH?6F&Fo{_A3VnV0%>golEl(fm494DUQ;EshaJFZtr^d-^XGjXB>I{vZA1&Mytc0mL)E!M%UD0up+{ zGc%t8C#i?O1w5iHzHsL-WM# zBfR?~URm^XuCXM*cf^-CbGzw~P6SxPt{pq^+zpuC^BXbWWnH}q6|8?qe12`o=iSFe z6TRoC;=XVq%%MT==uJWWQ0epC$!E5=^|eZxf6O`sDAh$t_RP_+M5b|=&D)PDaR8yv|HQwW~^FtGcXc`S8jKaCpGK)hOh8)DvLs7(fNmGIx zL@q`q!*dsoc4yu#IvV{a_4jtJmo*C~R-qg@F?t0(+s02?n?lnAqKL7X@19|LB zoK_Pm!txDOJe(S{0C~QxfK*1TCehZW5m|O0UOY35h?rhZEngJ100pyzv5q*51=`sw z-KT9Juh(ENA@NHA1+*n-Oi`_BrntrYm8ls9OLR&)Ar@<NA0?o%Un6(V0+S9{Cgdt7<89V;^ zHSuXUe6$hTd$w6xWhhNPzL7EK^kR4^vI+)^yj;p4y!fQ4=lgn0!jV8uRWQ<3IG2Vh ztuf~JoQ3$Opv)VkNToT1I(gE9Dnw#P2!dN2f$q9N8Y1Tq+SB#muhVFgS9Kj0a#h58 zjx@)R40Zzu*2MYR=)QI31?MeR5R=spt4y)Prtx3YqI{+mhU*l*E|H_qbPJV_srk=7 z$#3lopI&PG&P?0Lla19^%F}a?Xm^nehzoVuX9}IMh^m!wG%^Meq?!kLS0muCvxQoGP2Fz6l}F0&)_t(Ngr`k2D< zSVDILcj1f;(K(`XZHc!Xf@A^*ns?bR0A<;2%jrO8Of`zbH|j@2ym_zjok)v1cX#hM z$+4?E?975l-u4rH$H!;SWL&()QsR$0V>59v9q)pfhUD+jHd`uezYk6#eM*uPxX{(l zSaZAs>~d?8q2C$EiC7{S@OA_u%!uY7sB)^BPjiOb9kT-a@lcy|JBUhoNWT-4OpnlL(%xqF^hwn)!RXnx$=;rrk0qD;PKxQ;IK_SwYF){VDHdho%GVVZ zpBlwnaRNFq-2nQe1<9eo;-cj4EfE+@7_~K92UJ|Q5DiOgrEM{MxjgeWf4zg z322#~>%(LBIgks|Rnl(L`-qtfut*wMZpY)R*i0lDt5|#BfgIdFX)Lh3S65Q&)5g6K z`Y6|{t>8u(YN0iuRB~ichpgNnkAr%&c-KYqjk0pkLS=jz9Kwmb^^u?o){_?OeE=(N z#jrYogk?|9Gwl6FaB7g4s#@f%4;irtfMfx7YaW;8ZSr+1a%HrhS@Jqo;z${LX87Qc zU866fK`z}WLPaj!=wF~;OX&>-mxzKn8a}X>h(f3ueV}%8|L}n;&O1Z@_JGX>uszW% zEu#&=dmdv(7xqO+TtO(SW~Ovw&Hqud!No5&NIL~t6g5bNno+W<6adW*KKar(KeOwR zKVF0-#gUh!Pkv>keMS+_N}yj{Z~WjXo$z>)t_od){q<8M;X+Nc9_lPjdZxcWPModV zfylH`#N5b{PAb?#b&_^c_IvO}KSwOqWpJjt(mDkpDC#?7{%u7r7CmLOtRXRd(H}Er z(CL*%>V>ilZS-_)^nRP0_G0n8V#RcIWAFJU)bj+CJVoF~iA;f>!n>MX==TrR11h%k zv|uuwycNPKN#1rF%TCz@8^7&|L^_oQy!jr^UHL%uFmYw2CL~PP#)=`8}k&M~^ zVTqeZx0j;6)`*uQB^{f&QRD|i;T&Xq4Y8p5)^?&7Sf`XtTk*&U=n%W%^}vx^Wklx9(i6a|rd)>gIz$zwBI zmykpmITl6IbgJv(6)o?g$6>P*M24h|-9&K@r%ziNCL1Pr-rCpmkjGf9uM z0tbi-{|ZZUy(<3`3hNfMEw^30(c%{GU4AZ^%aiP@)DKbUGZ5qgfVzh-stZjj+gf8R+%t1}x=8N+fX2EAosBaq&;hln?Pl4ieC%l@H& zS#3c%8wtBagIW9R0{inBYT*l@cx6Sa^$nLIqHLZaH~ln&y7%yX@j{vDUW|Vq1U@C1&q3LkL916h_w#P=`A?`sKXE>@TEpBQ;DAw-IRG-S|dqAGtJ()z# zoR}F$o($JL9Dwuf2rpT!w0;Sd9KBA8nMi*y_C&A;lo5XjG)}fv?c2D66DHFlw}~H zE<|7T4Y&}T^RQq0!1)M^cfXYO``5xN2C1bYDe%E8?;7@7K!e|ywnB$kKV{0cdql}o zfq@PeCrCSHb)PvLwhj&2!WktOaZg)MN<;(wGRa~^jo1%P@L28B4tZmC%rmH6&*^CN z(0w94jIe#ibe-!_f}sNj_zJKwK`OP-u#V`F!>CR{tYkmE>uK&xQMKm81Q5K8uDMtr zStfs`f=}ghO0FA*73IoS@G>8MU5Kq5dM44-UxcK?R9mR(?B^2S=k zJSqw?T(VD9PM_oSzTv8OCL@5%Y^xBh1&k^CH6rP%j@ixYOs`asIHbs=t4O`G7Inh^ z2&Y~dGh7)XUkSTx3VBc+tF+fv{WBYP?zj+qEb2S|NiX68N}>&0;6N%j>}*Jx+(()T zKj8>lswsS>0%M;k!^nmZz(w$5x2+bcVpo|DiEzZ4%a8qrZMnZe6RM}5+guJUNIKoi zES&|RLzlZVW+ZNc-;Rrj2*3Y3687{6%M{8&yyR%SH@mEWf7{`4fNjT?L-)~az=+A` zDl{Y%_A-GjaI)BLx300eh3wp?WUGCMhm1f(a`&p@V^ ziEKAzfK6A}c?I&fWy$k~luZVhdOj>dYk1`Bm-9KR=`Bd-t7?{$tMSmbj=rGpV0~@@ zkU0oY#@UcN?wM{j_NOc4g^ zpzR$~Z2k<&$Z>!}wVw*w(AM;02wJ18J@9neL}*mk6N>IY;n)wRw6Bog+@!R(2Di_U z+vGs8r?PJ4z9DxaAGWKfuos2je8Fvf-Y_hnu+QM>zwD>p z;sT}afl18qE@VJHn;h7Z)!R92J$fF&$U}Psxtho`AJ(#eR>%z%LqrP)NGSR6_d^j^v4$IZmgC=DVOE{1 zbE8%TlVjgUflr4TdbZ==5nxt{$p0v!zLucYGhO+;Q=61=i(GS8IW;0QYt?^tW9n~V z2W0f8C1q@|(rpabwKTSD;o5gpbzMTKcD?&%`&$*Y?n<+Q6-YA7pmv;C>3M++??o`f z83f$lnrRQ{=SAIN)nXb1PCfD14jJ?h1uZfD1k=7{2v?2nPk7=Gr@QWTeq!tlz3$t1 z66g#eN>!e4Hm5umXGObjObfGOXJzS_l$&HnNlPs#rih8BkY$r4EH)X&G3D$?3tPia z30t$x3d>tnZ+Yy2_wtp-Y%?L+)Y5Z-zFJ2NwyPcJe*W8j8kbxpZN z&ucF`-MIV$u@9*UA!Jg1$!o1U#K=nWxQi*yBcj7lf2t_lHHFmYkq96>1Z5Pm7xow; z4cL{ufacu;fi)^1*b^P?7Zf6T3&r1KYE1K%L0Q2({OXE(!K$~zPaE^ zp=MXl)ri!SyL?K0H#tas?-hQMh9;z!k#(5lVBSF+?ks*{rZ&Hs>kh|=zavJ+CV%Or z+2^9h^fqc74gNZZWFGoaE0p;~c2LM6R2YRf$2uSv6&kI{gJpxuhbcGdb_Ew2g~l{< z5dRjxQ+Y#gfx5N>qz7ruJ?&Hyzd^ja6jhYuwda5K5YAkrOFw~4Ni<*t;@|5bNIIDr zxtKYrS{m8^fA=T9MBLz0%F#fIjvL$~<3O1Wa11)Rmlq-To$ zlz2UjU?PyTSZ5|<{ct}M{Va_>*&bAyZ8dafX6<;)&T#Iz<^{G5;!l`-o-umqIaE7q z_UHwaYIAqP!z*w}TZ>5|8w1X9RX1lH&{_sB2C%y@F-+Mr`{S9NjFYhtTY?89VUuz4 zR~Sa_zJL8GsO8X=Zp1MulZA&+0X$)40PJ`q>iYN>+`3%9&UaX}k814u1bIRWwlj&7Q&rd~HYY_?mhD3l$>I&3|YYk4D3dhFSO=EvNw zOUDJO#rCtq?rol4MR#MYfg&=Ba1z^7Sj7-?^!VxO z6%I%3JXsyr#fY?OykmV@scvejJ+^g6%lAF()gkZkEc^piWXcr51lcT}g_9iPEd5y&12;+Snj3Ih(Gt)h(>o;pSl7jH z7>)e(>1d*>#-YB-eoru;1wF~A%6!LkNlSF3rY^*@7B&EIxikL&JxT>q966(_ zYkM#@%CPn<;Xb}rpecfkY}lKh)RW`WEeQi0_8`WfW71+_Q8G(^G}LR;^Uq zn2=;LS4t=Y75kapVRU@Ta~Z#zEJ~6V&RSr59IBO6lY<1gP-KTxrNVVW(|M1GpUqaT zl?~@23hgZY-|Pdo)~7eHQz^eY35;p8yny8h!`2Cc{7=jv&)f}s0Vb_pfZ2zCFPQ(| zG4sFdiOT<_5SnV9UCd`NE}I91F@sQEFJ_2G{(mBjZ-Mv2P)c+Gt6gebnzg~pEYXqs zl{`H9SP&auY8r50_;K_4^@{r>`=X-f(;d+e7mrq8*#R?$I#tC%e#jMpWOEi->|38a z0-+Es6)K4lz?r&eLh z^%hbNBuw@AKM50m2_%dyhCGlk6@)cQ^eZg&QAKh#MJ-ck5CElh21me0^jzA2qa8K1WAwa5;kUgZr8zI!i1mU=TcJF`S0D;84n^i(uw`7D7i-k;-%lb zZnQ#RKyw2g2AG6skn)2^1LB208i*GU>VM)T`PpZp(aLSaT1w&P6!I>kXwJDt>xe_y z0VZI~<*Nv*0M<>^ip57;3^k)E^o|&e_*$zf zJ82u}SEv3VJlp-SqUBIHQ~SoL+~CG`>X*JK^}xPmB*l#D&#OdjaVa%1q0ZC$c`05) zl3Qm2t;(;HkM2#bl_VvGdLw4KbGHyqJXfSOyMzPdV@sK_KMf>7JO~W-=?GN*!5F*| zLE&*H;6iExzJZ0?N~t_yym5)`NBWY@=CO~M^`~8w`jnx-Zo4>Rz>vND(Hy@{Z*bX$ zsW?*B^`=d_&c)#x3*ljpnW6QtD=@< z%We|yu2_45Xuc%aHKJ=JQPM!f2$1KoirvN)650ASo`&2C2oUEhr-xXg8~k#YpC__{ zpXbVO53?dwCn8bM{q@U~0EIMiF^|8WyFTAkyr?M1N0|DF{OK?@nOn>e0jyQQI{w=P zv=I{WCpHi<<~rr(G@583szZ)VW#s|2@T;nCG!H32#OVC|PsHFnv}zW7_ay+EnPuYF z_JpL->jfR6gdnim*ClU`TrA{_&5%L7Q?alwBBd|*+6d)JM$#W!dJqZihwX-lEQ8}O(=7aBfYV_?o0g~DZ)B-aBHEhny6Bb77<6jWNKq)LW20Y2o z0TJ`>g*m026IQ~q0$}8Zt$s-MA>q1gCPi3rXT*K_Bx*On9%HXl z1u?ww!t%LVRa4_}_+Duj+o9`TrF9=L%#p6hj5&?xTWUl4?s;~8Ly5LK zw8>uskf&&yg-&r%OgnHe*>-Dgx%YRC9HT{$hjogsR}@nnwlhXPpXc{8JHoF&j~g#p zn^T~dO)&o?N5aVu8fNw&)u5>dl0+|O&w651rdv%JMT}q&VD}ZCKs-%2!<%F-EWE!U zbo`=5Z6SXvPiwNs$2^ov5oaCWH_Hn?Fi!`YXJRu4ZIhNEVIF{EO6TzGr&cNl*{n#t>V)n*hn+Dr>4_bBLnRIwO6Cc3C5ZM{Y`y4Ce&Wmyz zud-Kh%jb=E)0!_-yGV1;>OW?Ps*mF=jw!|4yC_%Y0&}p76@~PfXaIO2LW2OC_n27D zmR5^NkLXM1jzwzzastqO@thP$Z4g$B7mZ@}%0;1CZXagCwyh#CBurkHRa>cIO# z1(VgOC_#+UY+d9CPkk$;*$MV=wEbhaou|&LIV+#yvDrBMJQLnTH`RsH1f^$tR`Vm; zytfZY9X;xnI?8SxZQy3DrY8#rZ#i{%HX7pXjhPQ+uT2!7xBrde* zg#1J=Re8Q@ZB-T6rN-8-KCrcfI&eO_NI3GSVt0UDI3WpoX9bo371@1a_1o`?MH%7FUSX09n@RXe{*=iLtL(%0fcn5_%UEQenBS1&R;e1e?@3^PvbpJgH3LT}SyVia%D zd+eyYqGMGmy%J^IwH%KSS+UCQe(wEmSfTXUMQ%{*8K)Vdm0B*lxb4g$l`DAFKjwz) zQZ*@jeFJ-lG)CJ4-=s}(-!6rPB<6kd4+vDb-?Fy@yN6TTBUm?1rDSosvUs^VqNark zj1-y7pjyS1u_^v+pzpF2G83*AuEd@vv)!xARNmLj`keXB5H2=Fy6`jvR_nPinLbKe zdt`}l7H8b8FF4eNNWlkOQg)pS>pzZ>y5J;pamFIXp1uf6(F_Xyi&WL;z)AE4hzW3D z!6W%^g{I_y9_RnNYKMlc$+C!AjW$+E9l6NvMN|q60;~@p;W!*(=On+Z^4hdL%nAp9Voliey2=LH47pFX8=enl41B?uBjhozZMZ^^(CMjL%lFd(*wr?nqy@VG zNDNN)&3-$US7u?s2-tdDNC!!5Zk7qSkeHx>5#U*5pObzB)}G~!S>qK=sj(k6>O+kPAHDY;1x9Vr%9adAJak=l*|e;K_4Y}u@R;-= zHNZIL$=^6;agu2t#dZCePQN|Va6tQ#;b?YH@^IAON@q<#Ou^6?Zs5qoER4i@CG5pq z5d|%rNogPWU+?lW%D0IEw^_5+Bp0F#zhsUi8c7qL;72|clTr;|ip^WNn!S%rdkG-v z*8>P}mOAI`eLKYju(A^#GT3X#9yXed&yV#YG5nf{9(b>`G`mck?D=-M`Xf=?J6 zaTtGaud^Ct@Fw2)i^jNi24Db+2n!pLum?6#kZmR!J|(DAJ2pA*y3fHk%YGn|W|4#L zC;4fNw1dEf<(yI*COIwNLjWCy`2ns3DYwzlipq1CNCXO`#9Nhm|{2x>q@LL{@=Nqs;B<=E>4GhivOv1pKT!vpNtip z+q#(VfH2H#R}g-f-`L9{Kdd+RT-`=>(ttku{B`7`!L{|L`G#l?JDiG3+Q>4EBJSm| z`EQ|fmG@C?@7VLtLMM-1K>DKOCA*j6CC3GEe*uc}isl{hZ=oYmkStUG@Q}nW=0idk z@XtaA^&f>!6Q3i`x4v_-U2rrhwolSKibph@M?;h5$jQM2IM5KUB( zg_@to6HunxOL}Au!&5)#5why^En!>4p3WmJ$J4n4-7zgG4w%WbiUYWyS%`xUwuT)3 zdH~U?_@D!CU4}_-{vkq;2ReoN262~>@LP~LN@)&!ny;7wwJSy12{-Itoy~0e8-;)I zat=I&i2VP<%m1o!{--Fl3`4nGl{l}GSY|_sRyt4wO9T&h2#2V7j9trOO^*{~BH{q3 zh=19a9hpnxm>i81)PLFiN0oz1&4$Lz$HqF=b#m3V^SI?a&A;aR@&1nVMP*g#whnOr zB-U3MU_%SU$}RxF{0;V)t+chwh!A*>vVi?gPd#eM6}Ztq8orkA@Fd6ev)V9>gvX-e zhh6qw3CjZ9IN(TU2$u6_hi&Vd*G`=-*~xuxxkn!^PT3EIXpQEGWRsf^pWX_!4QGUg zx<{7pkGZ%{4+O5`}Wbs*K}a*#UFSXDF2r9o7k7aLAgcn4Rz~AQ@hj za{p$%f3Ux9B_GXRMHj{5euk3_@&gEK)b~IJj-qw2r;|PG=8hF;6csX(C)4?!WFubU z&+mv2G?!?pv?@#vvPf|pMo>d?t(k%ehH963pW} zkDJzvf$E%UBP>}fXUk1^=?KR$t?v`+@@X$6>}YSG+x#9vBlvEZMzR&`XRgH}M+92O z2~21QSY6)qYQpoi3~zm_cq{dR4#l$RWZOj4lb^Jp>={?cG2yF0@gMq z2ZVR9%Q(r3HH$&puRO4$=fKzfuhbxtWk%*ruI{(SOYOetZg)V5NwHM+-}0s#bJf1h z&cr)P`>~hKlqn2~e}l9~^_{9cQu4L2H~d=&i}+6GQk#uSYib!|GcfIqCgRz{y}B{E zU}x6wi55lT7C`zdC|M?AcBJLv+qCNs{a;7FD3QkBy7i3RH(9!G5IkH~AGkC`gIYvKcLYuf+#yI#_ z>eqV~6^Z#skiU_?0qYyBhke`^$E?M|!epcVE2eDMk)ga@>SFzlyPdu z231aH^1}C41_`HanTxOXFIZ1u>Lhl$d0ukz6g?Xs;Fa3_Aj0KwdA<&MWUST$$s%M= z%RQ=S*OnAs8acj8xENIkEm|M#O!~!mQ~alyaZY92rx~cO9YK0?b2N_>Ogib4>>($^ zeUBn;ej!XIj+_*^1ktms;2ZafqjA`$%a3$;zWzFD_NPB~*_>mk$;3)@R33?U(&n(#nyW3}tM)q{uNtq< z!a{BUCBSQTxr=C%*J@<@NrOC*2>WB&src~%^f>Zd#<#z2+)N8|)KtcY#LfdVWg%Nn zs^c#1MALq~>{9-9zK-_C*b!S$1qlKnB$3%@c~*{d)+HE8R6wPvTbcx_=pk+v^-)pp zSmfonZBr}bJB7_w-q3^DbM9lf&_?RzdeS*4S=eODvM8N=K=6a{P|H^T=>#J4iS$l+ z;v4Tz95re=sOM_U?kbVs+x#T?46D{3s7b}w zpE;qhnTE}Xg-jXPJlmu=DmvU}G@tbVFVHOEX{5nLDPyaR1Yw9_hzFUnAB6t)^Hxsd zQgxF<3n9tF70n4L0{kz!&q^;7X54yrSmZPugPF4UCoF11IvkqX{7glh%6H8W-MnA> zE5>!v=51ibk(>xDb%-Xo^QxnUzAep+S?NgEpUid}C8?s7+$4wHyUz?-J)~hvtPgts zq;U8}A+50;NN-Lm>l*!#;qF%>Paci~QjS3$pwEp%kJ1Sa@2;T#Mub?jn2!NRnMV%VMfg(pdBv& z5qZ%!;~vl4ldiEC1RKIRIYVQ?*1>lW?O-aSq8xl7Bh_1t zx{LZf0KV||`)%|5t-hefCzZB~A5kx%j~3ai8j}O2S{8DoML-?cC0B$GH2lq_o)6bj z&Mn-a_rMFrK27pZaIZ`*IV5q=k_>6> zH~0+Jq7&T+DL4nP+b5Mh$HSJQAl=2<%%L@Vp#D1fBJ8XTMBo-EZ|eQmq`_BIMdG2L&iNwbqBNx zxnaHOW%szPzyY|%NjQb9+LB>@pi2!&;}%Ad@`gsDUu-&rDw0ZtUi=%%zV8ZRdrscD zv?%C`RvLbL+zu2`O$BkZabe_C12ozec< zrpy=Q2SAG$mTusaMbaCCLSUQug4V4`yk{{R{rWky=7UW+ZJUpdz|fU{_E_9hOy2|} zF&{JnW4}Iofxf{9J$sQ)9T}7iY(DFtZ6}$pygkl;whw1~{|?+@z9A3d!dGyi zi(w!#^`EbDIx3oAcP?SUBTsgRcUtM(iU}{N&BM>uiVCsc>sbu{juayQYS8+Txp!K9>CVU?32trcHhEJF>NC=; z7f>djIZyTQ)M{hC55s!>^~WqL9)0C&SD28qcn36_K<3l4F3YoBjBTTHo*zL6MRK*X zG}*~lM`LAaISI}%0ZbY(*2^b96YZ~Oy*2wUd6Wn@Rv$m1;Z7Q-wiNaHu}t&#?R!O+tq-9)Q`%&LL5iS?QykD zXa?sB4KoeUTgNyGu>qOitkdlf5rYU*zd;ci{2p41*x9dtMbnX~d7)VR+SR&p#eaWe zyGoZ!M~0m$JHB+JvYORq(OdK89p_eC7T%a3u9$2vAC=Q-=e?>`M^o%kPx6N@geP0E zI+=!Zka|gUvkD~eMvJU4`L4@!vcgR1tS5u$&NEhcCG4z9aWg|xb9OX*6 zD)#VGx0WUQ)7Es0vFY-1P9_#FSAPaps-8*hDzDqc=#)9EJ?%9V^I}e~N2B~0;c8+a zqgUBRMlj{wLd?~ITtnO1Tfh1eVf@ufrhqTA1;0i%hg685L}R9nn;~42Rqlc5D&OV-DxRbUMp~O44o8w?2dJJ@2Q2<+ogC7Lud9Lh zHPM+2bWUO*?CkR*bUo=C9`NyAFWe7bYy=Xda%|@9R~TT?IJYx+Xbvz=Cm>w&TQ~6;+Y+pGO*5F;&)8Ji$;TR34nG+frBH%#v z;6!VvTVxdv>fPhv@XwzrToDdOkVs-|qR+V^?r91+cpw}zL@=@sG%sl~o*)_nK$Z*A z6#K}&lT{Vo3o4aD)z`gpVZ4H=t;9uciK=~fLGA8VN4%8TAfN^nG`5mTOMV-l3i$R6 zBU{`wK2zES_bH|KyS}%IrnIN!`HEO6*sBzVhQblGFQYqSbN0l4v)AAcajJM|E+&I! zG_^V@V9HV)Yn^%YLgC&wKyKdyv}#CMn*;653BTAo(NW#lk5HMipRHIW@f+Pc1c$Oc z{o07^AIJ6Y$LS(?V<(ulS;dFzY!vc!?cxcu#;y@v%HwWzrsTL8{qsL43gz;D04^~O z@mBk8DeAYPc`gR6FZS23P~EknNINNQ3#yg!ux>~Tq^x+!o=&L>8U;aX15vzDbFq(D zdy^_yl1ows>IKvb6n1&kRZ6kQ+l5I@5mF>6W{2^RK0HSmStc0&z-7lt%xsI6ZV}z> zNYWqX42J*trS!Y;8J5Q55Bnu88DLSmvh#NL8V||7Wt$emnP>v6Q>Kw0+F0cd3+KQ*GS4z| zD@`yYaOls9bWgX9y}(xs-t|GFH09=hu=dlUI>g3b=M`f=J8R-WeeWNHM=6!8qh#6?r!RFB(xqk>)Gu1E5-&U@HS^_mXj9%xwi%rQt zDJ@UK%=oCw8N(F>il%KS!vk%=ohWLCp)(uXCNS}I&gJ-u&k>Vd*7TEjY*cSf-D{Y! z>~H*fGhy92c}BD`H*O)|K(E@8yL8xk4C;r$qNOomkaK#A$k-t>J~^h-gg#MlC33xe zO2*%i2aMWBS_)#phm-hI;vXY*ct^Rf$}y4dyH?nQb1hMQrOEqu?3%y&ODFW+f5-hK zsOD;R;vmQq@V831ndsjzU1E;w6{Ga;jg*F^$YdvRP2iR@C#yw~ke($@>(#tpy3J;X z@iAb=-8blibVcKyG*yRIO_!%IGf(b8G~*JreN#BYPr!*6F|uVhTiXhjM+&a=+N4{r^h_f;=#ZTA%6u5 zL(FWm_~X|`_G9}Vl~d}tB{rjpd0T{y-yw475x^RMSa!k_jFm6r&y*lI&obkpiqyk6(Ha!9%Pi# z>_N;>GRC5~h)2v&H%_6r2uGx&sRMz*TQoAG;2<{|;Hk_X1nzZYCore0LxE{h zWGQjAFi8HawjDJixfX|@xops!qOx$HIdL^JXzHBVAS?-O^}Y4D#^7Jt518qB1yE35 zzObQw`6Bvn-4DteImmtgZsY3kuOf&ljdu-v3k)A~H=|^;^j^wCQE3Xhb;gcab`ZN@ zwmf8|d5CY)VGjd57YfPJ78aDR!U0R#70uRm6~pQY>gSRhmQfh&^vjtAzf;_!KcjoR zeqLRe`a_J9a(n%GZ@u=u_P$#2a{GMW>3JvUHKGr`O@eOEF(YP3g4M^k+J6RJvJrVdT0sq6RBEgc%`|$PUwN$dlRcozog=bl?9G+i zgE8{^=-L z#1|#(Hb#-OJo60E(F?yR2xKhkWsOL$Oj6n%1jv?2S2%H|ang5L@^GYdw}uyr+61Es zA1E(yM2w(!VsA*@1lb(^&QA~=lK<4Ev!}|Qrh&*Kp)#RfP5WwR(wUgSa#!*jXX2() z^bZaH_wj8iKzIXX7w|Ws4~uhtM~5AZGZp(5eJ?jgP(6Pd8e2=(TEfUQ`<+{`ddiuO zY_+sx0idJmumiY42T?KaE2N*%H&f`#^GwAM12BcVfEJIHYheo*%>N(G-hs=`b!)Uu z%C?!ZZQHhO+epebQ?_lVY}>Y#GVaV;`&2vv#d(j@$&XV1)nfZRN zBshByopN*WTIx+w;FEPdXJ=`5W{aHN*UKQnXy8tNUp4~Iy;Nb6! znWfwvH+Q|ht2p9HAzhSQncc)-tEYf)c;+`m=y9ob#$mHzVFKW%rWVndN9IgzttnD}SV0z-+WjQ1~Y7_9mx1U(C zNw+E*`m*0FiZ-tgJ+#FX_P%{)Ip682{J?MGvm1aL=715wV~)-~xa9VSaM;9;p$sDd z+0}`F7;HyJlrpT=!zO+0-U&W zLJ*zHGG8wWc#j3{0+ko=yZbN-@(8MgGE{u!^q19MaE`tK=usJ*qTR!>P$jiH^(B|p zNjuC{)CLFiU0UTq+?3qlDngVFv<}sP!fapOa5OBBKvBqKxKZ049Sj0Afdws*g@~y= zXqFODhgmTbuUeqhQYUbqUraxgclFHW(U5xJXOOzZ%yqUs?1LT*Eb)6}M{NoVOyn9~ zInyP+gHW7e?#b?H=SD8&=>NLABN)->UHs;&UV^oDj5x%Nsu|RymMflpFXP^GguQ)m zEE8y=iGaRYAi_vT=!sf+gT2gBN!~+pPAUMG_tunDYldjEnOU~U1qnZ&qZo33r^7C@ zio$dSqw`sa>JH}3O*N5Lk@g$EO$Nv51nfbU&OncHhc54R>~mt?*?lsFH!PEr`2wqC@iseQD?i)4xhH-wVYe{2_d#;YT2&BQ2X8f9BT z^;ju%XDo7HZrmu7gQ*Xq(83iAY_eAnjm`kAdUriMRqqxTVQ|cO)<_rUIeh?r;uR(} z@#+VoEAYe|C+Kv$zr6WH94XwTNaP8Pz*K^NYdyN5*} zutAsHw|@CIwVSgK*(p5Wwfw)&G|SlipIU`~-kLsa*(Pk8paUUPDi%~D_QebeQDTcw zBqVZD@4?@5MLg8wt=cAC-8Kr6I@mw3uK_4?WIj{ukMYyN!MTRJ}Rc>m-t+0xm17jC+^)&}vaIs9C*SpCKU6};=Fj1QW zNy_&y9D5zL#xEh(GOpXoca?!lECFoW@TJ?O~OlkHI1T29tX<_@-Aq7TxbdEqO{ z(5Yt$H%3NoMqOtdr9Xz@-3tx9s(#v~*3n4Wr6XIxP~0#$7rs znekemrDyciwI=`{51(vm4)z5z0b z5(|;@jDpJwocb9J5Gn~YDhcB9Ijbs#_PM2_BI;n)jL$HGROW_H9IX*XXOS$zxLZ8{ z(_8#5q))y}hIe$yy;UhzK>;Fh}0nBcHkDCNjwlz?}K*B}Ka*^Bo8bv5` zrSO6RdFhvckLf&a>Jb(lv-Wmd*8dqdM1PH&o_~xRNDFA=e;zlJJOHkF>+|h~><4s9 z@orr(a!cMe6KHmM>NJ9=s1QT~RF$PXL*Wiq7ef7+!pjI{3js9;tQb8JHQT^>a?J!( z^%_){aj3LrG^#YiTl{eQUv{?^08T`S4fi<%z=={HTkOE9e(^_0a{)AxS|W3-Pry@0 zN#O^aqFn}~jen}}Rg8hf8-OZ&ax)IE+D6ONyUAUbOtSkX$(71nbAOu!4EUL9kiqGf zhM+QB^(>vW0V($0^qui8xKIM)YUHzpDt{6by&U0j!#J<6eJg*~{^hCzgkNtXmr47E zeKRl{)`9(G1BDCKlh)x?=886Or3ex_#-@$hU^;DUCVhf;0Bir~WDj~HhxP_xS?r?m zG#Kv~g=`m`OSv@wq9n1XO}KJG>|>iw2$9D66u+Vd@MHA;6gR~@gqK&&oMzI93wK|joZxb`kqb~` zYtcKTQVrWt7;?tnr-u6-1JrjP?F{s%3|~e4o|U&iS=q;IS;{UIleH!|g{=8|TgL#P zk|YRcmL2{`-+>E6_(EeC1clAa@ro3@OTk$szUMIr z@43H_&;p972o$@D(D^RHVf30I4+vxy3u5KGt%|+B2yr}dJm1oiA@2zSFi0g966YC( zml>ew1^!WnCqBt}D3jdtR!Ry8<*vDzS30HxU;vNvtkD$=iwG{x&_h`Qa$#imQ40=l zn1-{}g=N|CVCy-^qXI)PrL7W`Psz$fY}e_YuH`wV8{T{x@I3(s;Lc901R$EE0>jb|>zjAn zR~H>eZ+BPt-+`VGIB?kljt6UkwXotjMg7eOFTMd`EYPC`Mso>d3~Z=A3@(@v2Fk@k z86ag_K1uiSGSRf(n8nYK_#Hx$QGret(P&L3 zE~*u@>sRe_Aj@(k>N4s`MNuV8Y^JH809>yZk>fwJ8VRi2!}1G2X<3Z(`G2?rHOH7x z`&gMW)@?&-^UgPi+l^3a7Cdwkv4VhMA)-MPWsy58&IqKbNWaT7-mUAfMUNZ<`;nTd zX$Kf;OaglXdWsPgI)@2s5keL+^E1IcxRo1Q;-&x>^=8MxytsT#bAxJ8ye_IE)q5( zG8z}4Hod8%@|bbh7tJjlhHJDg1Ps1lpri(*7S}KRnsobah_i=9&d3Q>Mqaa%s&g^V zfL6(kanlvOe&KG1A1F0cB+d~vwt8<07-1;Z6Vwcf=J){E&T6 z@D=D~ruHHheQnm}jGL33lHX8V!)Or5fm4{rRO9)2 zF6xW%nEMiTYVo|_Sa3M|R39MPjcwV~%Q=yzWOb?j?x=XwAFOIsuJU3WG3N#A-eo=S z9mXb9d)!hHw}ef;Y@(OK9pR}iBA2xH&eTx{L1jwTYpYcCc?3{UO^S*E_~* zp@vx?xf&p)eIOO|ZrXG@g{4KHV^p4a9`&mi6zPmFKSBG2c|;TZc}wtGV|(T6BD)~Q ze!?9TCbEjsn*KM@#X4lyXD2ORboND% zbk9YR7dbW>p-*V5&4|9y^3q|qTeU%xN5|J}steq5>f2zMcMM5bd7f+*&UHz%Q{yVHVU$gwE1f5Fd`9a`X3p;+6i)sKYg}j6aK%|DJ@DwR8Uea8YV*SsG5R zVo8u&QCVx=2M~{jbS*QPa1ZS@2KtAtzSANc;I?r)Qk_ zF{}M=@o-J*l|W{kJ$F3_KuwvtA*sj%QY>uj*%=F6vKowRRa zVV>B^Hfc4E7TFBHy2!C&C7)rmE-Dt)&u$%-#`7DmREqoMW*iK^EgtfmY%pA{G$2j2 z?&~n}L|i-=_E&4b`UM$HAL{VkT`Ydp?KQ9iXm6on0oq%{GjF{R5|kK)JkwU{v###K zsuP37_=OsAegvXg8hQs5v#&Hx*)xCLk|N?{x;|0|-Or~05u8B5wD`(+0-<}_0+R7b zzKij$h2pa5rm>CN+u&%wQ31*LIsjrFZ+t*j>?!gm84utf{~*>kgxsa_REmo!8!yq_ z@H+6GiNv5MmlRcEX-2L|=Z!htw)cMyy>K$5soJ!3HB4!`z>oAQ0KxikoAAb$^RQ?& zdoBG^_^%gre@Ejh)}{~~52NoRD~>q3oGCMJ^Ir(QX*`&ibYRmql><4wubbm~ zprBNow1m0=w734zE-0w$h%G*jg@c|~r{uXMdrscDMqpsonaC}^&d&hE+LP;b7ZgwK zklh2!rUXl5t3JbWY4C;5a)XF2uCo^&tce2vLJqY?(w)pjq87p>>~*y+w_Dw}q@M6L zH3wGI$M#fbU)i)K`~e{mWpeBz2~UHPb$0URPd8-JpAQ(pTjazoumWiX5vx&Ho4Ar9-5izA_rwjC}hk{Nbj)7JxnS^2mBJFt>ty%>@$^{T<60U%K?VcGL}wvNqQ~Yg z8|UtkI&2a)CGr1_SZ8IgI+CHze51_wl4py@2{uMHWB@t5@gKx&E>0v^i7hSpQ_Fr{@;be|PFBkO=KK73$Ll7cE^Wyp@XJQk zRskH3mi%?FVJHodXj3|34LUH%)@woe&gAShX-ki;B)&bC&!?%<_f7NCTJLNbZjW^A zCBe?IM3k@Fi>%ZD9~%zS=j9kf zASn;S2ER`hv_Os4dd8Gthp9T!Oik%u63|@(8=Tuo&SXAUP0XpFV2z^i)3?@Evw@2_ zsCC{bIP@+ucj88^b%-9LEhx8s{2jBWa+Z*y@~1!ONEV--6$g!s;ub*nUVp+ zeT#`CnYku#g(0jDteh$UrQVp%;@V$p?4CAcg1Gq9P3*)B6$1vh;#aU}x(%m3o7-cH~;B z>#WQI(jUP~2ipLYOuD(i5a9)rzvbcXvM#3ElM!8yG0PC+?++U66{4m(I||N;mzQIN z7~^d0ATS!OFt*Y@Bh_?bbD1`A&G<8Agi-vLHT6qY%@_^#(T?lVrXBs%D83CgBOT1% zD+lL9rE)J$uIlvhiSQEzJcoEI@Sh8>#by87kTr+i(oNmZ8SY=cPBt8&+hm2kez=;>rF_BHM?`#YRpw^~-c{S!Oc+b-2il8OY z;8JeaJo7Z86bR*JYmE3R<(wpH zVf_7&Nfx>Gg)15{g;UJQIhQ>BtTqh%x>H8aF$4bSLH>?jCh;-Ui1bB)TDSX00_S|7sf3eE#C3@chdXIaGeX zZzd3TFp4b*>xs&K8cEdKJiXPMRD<5jRz0u_2~p;xb*>fmhSQ2pyrnMLbks<2$HTnN zIRR3De!Ik#VeFDl9mIw&$)rgEFmanBf+Ez-=fCY(OQy8x9smPk0x%%B|EmEZ=lJjQ z;eT89|FT$oYy97qeWH1=aL|P0HU#2-wpaiqqYwS_jX7o`ozF4g`eKqa7(Wg2&yD%S z|6E_x(|!D{+0We-f<2X zntBYbd3mSF-^A}Y<6`wUHV~g|YL*^rMVU&4c)n!{teUTy3-!{lN}=5X&-bWEXEL%O zsx^@@hW&stIxChx*B7mc7S;Dn9(CUkG;Spo^}8E%TH6{u^j_R5U*RhIWnwy@lkMs^ zG$ZG!TxGUbA;dGX+ezr-2d&c#kwG)&xtRRd>4j=+ z_4alM+HaGkq>p%`1whyps?2zY@xG<{^Nw=ICbf<{6^=eCRsVA|(Cc8EtQ z_+#3)h~@y8_Wgm{SLy*Z4?}xKnk$=Uk)MsvU@*89M<;d_HsOxsM`!M!ZXjr0-)xfN zyBCPQJ(w;$gZDJJ7T6j4OjSxX&(m_0VFJYuv{6U70GWnmpVHS7(%(5gs{P~kqTh>? zS4eh<^A_q7YMrCA*Rb$^O#83_OOw#rd|lTfYmIN2J~u#eq|&mff8AbOuGzL40l(un z9`F1y?bE$9anu8yBxrXEy{9VV%}YpM#z1eh3#CA(UK)4-o?D?$K40|VndF*HCc22z z_%A^Y=3r?$hjU}>RthuD&T&pxPK)lxZnJcd+~-+p|D0Yt_-g0CXE|-_;e+VOzTlIV z^ScI$(jcObl#aDSyi2M=RNrDn@gp%v@35qiq-D9|+JT5;iC&ZLTm$44pqp5mMH;E9 zL!nG4fadq>(}wviW^W9AOUSTjhn@Jl2l@;jpy!cc&JVaO2WQ6T7YGlWL>RE<=UWG{ zn2XuP?T`}QV~&{~hG%I*J9z}iE2K%X_n`jBE56aXn3VYeYm= zR%Da_!)CfiYMs0?;rccoO#`>S8wlLYX)t6{=i8pphbybRBg?gFa$x3oc;x!l3yFRi zpIIsUhO}1az}7Uj3di9nhZgQojcj|KXpn;IpmAdFxJ*;e}*#;v#7&#Q32`{3UAmqf&hJj6VhU_53LldEwZ38_|Q8vd@T-+ja;Uj{BrrsrR=9 zcot&rFSlKdw;sPEoTq&G_YuFNgJ*wWI?n5OV7iEXzKDc}v2!}cHmC_cH?lN&8_zGQP`*F$IupvV(JnCOtZ3=V4Vu?ZKE>yy$iuFGWiSm zrl&NXk5l{bjy8hY#7Ta@0YZ;`rT>MoUN?GXDFQWis8PHpqMqg1nV9f(!2#4Fhd^}| zeOnLuG`3MC)$eP?DM>yxZCHUU%}dDF{SEB~QMFe>sA3pWT|IoLvp9$W6*?&JsfG0X zJ4LF7CCYv%L47&smBxF zE2PueQwDBZ_v)cMReXr!9S&bP;yk64!8(b)UoRllvt8#FC(HWFOS?ojv+Bduj|dy| zb@9>LH(i2`@#P~y32-WuBF3I?ymG?a9#v+Q#c9QwWhugaUHlR2rE=UW2>0$1ap^gs zB0l@O5yGu@8Uro|d%yd=btlm<__r69n7;JAQ1VoOW7sx@mYNAny)7e19*XJ^6zz0|6_SDl{mcBt?Cr_ZTV5fwl5L*n^9)}sPD-<|0;lOU^l@K`E z48Msrve>k*ZU-gwW8RS#aUS(l)({g^ihcM+FwzJ~$XG!(hk?OLqtx z3oTC}_mW@y)n!&2Q{2nvUH-cg-^=Kv*!1Wqx5kAtnIeVxI7H@Z<>zhtr$kTB2 zck?HV&-rUW?NJDNa={+YABz{U_jCIf6trlSyg7tHRLjA4tVP4j-bkT+8gAZ&YfRoH zh(k#wv==oK@^Y}u@N#KyAthFZL&}bFigrhHN~t&2o=D1D=juBVwRT_-YQk_XEa!BW z91Z+xdNU9Ne+=d?_7pfAL&!PiY*IfqX_h-FFv?#p;8NSe|`+Y-w?s-rL4dcPp8+2=b9e3kv$ca>nbfalf#sjqnPu$LIg7R z?EQwT?2R)K;CsR7G&s~8aSzhy>v`+ytgji>P1@v_nERQrgy-odK8;I~VQ5`W6sMos z*)va&y51Z$8jUfngI{`AL(f(4M4DDt+Sil++;h#XNSR2+`Hh3*LItWAqPfzgN4@(D z(vE(kVA?r#^1@0+b_!J#7*=uEtS{SjMusXV;iT90NuJ&18X9w7$)bX;&RM5MK(GAO z3|p;UT@U#VQzo70`71&2Qp~lRz#Ki2)%0>%LXF95Uu`JQ5lC7u+a zJTq3_AI7WZIOBBXXOC1lE!I8gVKkRNt9Y6j@a7cYzX7A25OCCsV_R^q2;6dDNm`-~ z(cN8m6^!pYX}3zLICm@Rc0gkukhJSpl5Yh_LytyOwCk@K(5sK%h%d8e!Enj?1^*oG zE5(S=V8^Pj4dj|bII_hM7_T$bk9Nd5|hNxYO0DZ)7u2i09FxG}aBb*Qvl9WP3(jHEO$~(#l z@pwCCVeMXsB|Fd9^jX%}T85z+Ghz%jSku4TxMaIW^6pw%F@j#}S>ax{*;3Z%w{;c= znGA`RBuV-XBI5dbHR0lB`0+dC?u8zrm7NF>H!==?9BL7vHJOYqleCCv7$IAatxZT{ zj9IS*RFot|e-ofmjj0wt*+WEPoafR*5OdM-O92+uVV`=x+=foHxzvbb%%uB3qec!D zqDQx7wWWV0z<$6%7&|LAg?*4Y`AXC`s&P227911}V=c%baw9B{He+kc5l5`4XKqVY z6h~~q*fq~xMBz|2QTY&z)809%a+tJw&(%PiY@LdezH3m|xOvb)I<9jVZK(FZ(a{lCWJ~iwS9~j|H-&Nh9>!zQ9kW3QV%)Tbm>Y=?>IA$~zlFX4qX-zu&7e+neK2 ztP3B{A4p?pd(jo7`N&l5^5jR7wnN9aqw#_95>{TNNMb*8MH|a%Z?(~;WXftu$s;O) z&0Z~SKmz^Lkd)zf*rU%<$IdX7u_0)Wm{jSyjkQBtTSCQM8BSPt+gm`=pBrmnQw-Q> zhjg2^U+qD&`jeKY2H(E?3gGQtpu}D(!O<=r-1THZWGkU;DAj0@m>4i^1Yx7XZ7xAu zFEebB(Ft8!IA{^liQ-x?Z78Sl{)OyT+Fqk#GsL>!*S^rVT$Rp-JlFG{)#C_;zV;^&W}^wOWP z*ED&$oDVl=>mci5Eu{MAan6xKwt$okDm4zOj9|AH-d<`ic^cMG1NRV*px%nljMy9~Av=7`E(W*gyjT41eT%kpO#j>Z zVB`(R!g#PcW1nsHXeI&8Q=%UjqoSI!w2amv3*Ono?Vf=6fwp&J({_QY9r0*Sv?)dghlI%6 zz4wGLwL`@DzSnT-((T+7km88Rz+Z;S*Z)e9-(<^zq;i(vJ9dl;60>R2kSBJPK%Sp{ zJ8lM?I1?laMfq&k9odPIO=t#&%j$|>7Ec1dr=(IvuX3PHe2C<4)9P2*s??P1rmyRU zfEAT8`i)cFA+e5u;39C9TAEqSkQp1sJbL)`T1?);Hm*G zqpQZzOHN9K_EyDLlNPOrnx=oDCAi{F!i}bCZ<%J={|uIw&u?nGKRM}0jp*$2Uw>i@ zlnGTl0iG=W`!=AWiJ6J}|Ld#8|Dg@YN1fU*&mY^Mka1&>uAShLywL&)@h|R8CDR6< zlK{X%mm?j+oEh2NJ@$_o&++on=P{!`f<3|t$`r-BVz(_A!PXpt zuuxw$Ae;mCPJMfWOyQ-iR9^zHC>a$^bHjvd8Ti_aN)0V+N;6b81^vbx_YX0i2-Y7l z-tFsOVmwNo`b$qMxuJULzAXfeywXQRfu8Vf=yBYAsK$nxxZZAI}5k_tk*D&Y?oI z{|>(gp}pt+(+51mlv*w12DM4^9eIoR^;aLzQ>sa$Tq3@alVOD_A?vR`pw>@m6uR&%fIu)DB|@lvvxfbIV=HAygTf3(=E#qwTO?2hK}|FRFlEokZo--DY@@f zwn}+@U0OpN>xk;-0c{KNKx$n46%%Fe@Pdo$Nm+3>+Qqb)itRTgP7vZ~XLZbru&E@z zMMM#(erMHh-_F!UWP8%0Bi!|dj)qUv0gs+Keq3Zr2GkE((LlGHUEL|1MFurj9oZbl2{LV6s$)0Z2E$aS4~B#f0_GAMk+q5>fqFS{9>< z=0q^z#drhZ3uQVM<>3*Z~*g;u0Fx}WKDCA-WGznAZkE_ z&Q|!oBD5dV&jhjuOAg+^kbNjbE*?K^CCsol2hS$3dY|u(euX&0ej(Iw8)B*xaSyH; z&_IezXCD;003M1AFh(r!Cqm@In#0>?ReY@j^Ok~qkCeYwxK;>BMggObAQ<1FMDwOG z3gi)fOA&`ImuylYVelJ)Q*qF?EiL=DOJ>daf$;)e((#71HS}ju!t-ZQGRgY_!%BY- z65uPWHASBUEv{tCIPDO@Zil>9S1n;j*qn%SSp0LHR-}JwbeMNO3Z$4|rW;fCOSM;M zr8I6V%h@lX*b2ASrV%J|g^P%g26EUhjzgql&8fKR`#T7-mC+!;Uyl4B@q0(4X zO^o{gi%e5;F?4n`_|J!n|0|jHJ+4VJ@sH(dY2~siU4{fOSHQSmcA@u%VG5XB2l45+ z|1?LvkY~|`3s`*U0WKPV7;oz_(+l9D`Stk>h>oUQ^AD)N7Te48kU{aPFyss@q1a;2NZ(>bjwNPa^zmz@bVte4>bRd#?kC zt{eievs74dK03|PqieSvwqUfs$D#dn1uS8k!i#LKL$ao*P{&*%z55caG}_iOv`rx7 z+5w6@LOTxcuC~_6cN3=#QYr4M_{Xvntv?`a0Bm_Ly$i&upbO9Sgkep{UqYHzU|a;D z;mvQ0XF8x#sQ?%jDEQ81j!0dtGF${`AG4MwI1T{Anzx4h1;f@-wqBJL$+UAN z3zB+d=MFhzdT!_kyu$S1Hq+zK-RTeRx|Z_GrvF8zW&bmo=KXJETJ@j&=s(Cb{53Go z*ncL|L^Sl-{uh}B+I))s97%H7^K+^EWw-edEwl0EHY|E>n8~;uVVasMxK@q7LT1W0 zESAk`oz2vH39?1WG6{bH{_Bllx0KMukDz`I+&gvszsa=zcz?iyXs-t@DD{c@U^q}4 zw_8lrytB6-UC=QL?qhA<9+h`xU-wY&;uO8ZHAWpkch$H7fssXWOBeuVLyjHP-JIdL zcKO10y7g`s%R_~CZyQEDhsr)4IC)RBsyyZi1jCGz0Whq{0whlYYH6ri<*dKa(gTaTOTw#3mthhHo}2We!Q`&^ zCpo?Jm3Z+t-cJ49(b+3Z&B|F$>HOsI`upP`n|p3RUU(CY70VALPkgT;Lqe`2;A>dH zjt&e`bZTeFV_|bJL0Zr-XC(aaKVVq;cXYuR#5DnTnoTdocN6S5zS-}DiDg17g;7#z z4CF1+@zn?;DiT{;Uo2+ecZDJmW;oJ9EC2#dKY)PK9A<_cK(YZmdc1<+E%U82oO!|+ zl|o*Da}B5YD%si;JK2bEJdivdB4Z#gu>vp&1(uSQn1tsTpsC+z1Q*<;7TDvR!NL|Q+sMtyqxfF7RrYf72R;YWK^$-9jC%JGc+(RxSo9l@K1#VM(PB2sEJ40e zNlTBv>A#uo#xQ2BeT_z}hm`TW;+Wz&^6-?Edp&z>zWS22{Y);sKBbQaf6vI~PnEi~ zC_^Z^nH0H*@o+au``Ld%s+Oe_BUsu|wil8$0epE0!4;cT+3(AL^kDi; z5bt@%%kcwq{+{RLwulDKU%o+eR;_6+$g)_xv1TCFdyfsIuZ1YdeQ9$IZ3nz-gV=cX zhQc!*Q2 zm&6kJ+kYMn825-kTmEQb`!-|mXCHdkE#Ypl4oPqfC@n%|Q@<03;_2MU? zD3pzUYp3j~^9Vh`t`lc!?Q1v$VYSLE!VpNTUQPS3Rh@4B+r3E49l2E^;Ef-S;J>A~ zRhMO|!-{{0)gQvFXu7xhDqaU@+DZ*3TDaX614L4+Rx35@^*L`dFWy`ss7n658j zWSspD%x}?B%cfk%dZW;CVNr3(FbNU^&^DQGTdHcZL~md2blp1szA60>2+}(=D?Hf*aecIf=6W|GG5#S9Nt#aFLLjr?4UrXrRC0=^2#@K;e%bYLl1gzpP zUQ2&bx~}j$0wN!5hT+EVX6``Xhty@hR{X+rQ^#73;jRL%rLtycb=CN#E8q(2N%zav z5Ihk15InFuE;#I)`1d#VA8gmAU+Qm?;XZ}E_!bKJlze=vQtiN(UxO?kq%Phi)Ag+G zqM@E;zWf?^UwxbGz5-T{=EHb0`a(5*I+T6?Y9`{j6!Ysi=GV&&8{bFO#&;Biw`|H! zJ>R#rn9t*@&(rZw5KtN_rJ2FtmJM?9vIvsiB-M4xbwjnmOnZ}8T5@yyUWG9k66>CV zk3||vo!-Ldh>VgwT109pOz~TdH-(LAoqAR%ZiP3@fzAi za}iceNU`r`;69gVWEASN+3Vue=k{uaojh4J2@+&zmE2?&WXgs%LkKe0Czd_q0w;9W zY;MA(^w#Z}4C(5OXAlZw%~;qMEmZ8SdSazNqGV@h$V<75)Aa}S(V&4E*HAOTW{83l zDmuwcL|VMMbCjnzEEf(s8*%u7Obn-wg7K0X9%NY+i<3>5jPAG#2 z8CWM_dX8xm$~WMmLE|Y_)1?QI0x?#wyUj>?Gceb9YPfbUdNS9H#`3RUllcZOj zt$ge#ALbbi#HbIT+cmmj79%&Wl{u!u2Gx*SE}wc22aQ0I7S0uZKe>Zu10H_UDO~aN!VGKsP0^*p(*}KT!U-6R^P~}njg>c1T8H(bnQ@zR^;fF zCZ$uw-}#oh$p_lTZs~YCOdde-jkxsJNYfrm?Gs#&$@?XSw68mhEB_@PLm#8DOzkW! z9-*vXNCE>DnW*QX7x1XMYDog4i=X?>XqtE=nd-QW_`qImpcR`-qES#-Af#8a+b}hD zos&8zHc!->ZAFg^`wYF6y;t0A;z*>WYa?-==k#NSW|kwNOp{t6wqdDoe&_5=JXh7@ ziF&RUMVZfF|1-O_A<-nu4W2b)?bjoW3SPn%*&1(%CX*P(o{Xz92CrE}8wQp)sU=Diq#j_}$1qm4MaISzMaJjopMK#WDZ%rn!h~gz=J=YD&WPo9J19^S4x7yLc{@EKv>BEt%Ah7G8jn{L z67(-K;wzJ;NN^OQlrBwFnh#w$&EZ&02P_vDrxKqfBj=3$!b}5YG=_sMeEhk~ERUzpd+sv`O@idje?-yK@ zFE0&Sn&$NUhDNn+bMw;kC9~M?-q)G0vQG6n zcwzlQP)e)gtfDTME;Q?sOmlX{!fV>oXW!9f>XbM4J!8XMM6n4A+2#tZq`hvpEats8 zW8l#-mJUH>r)4v430Ax^zeg#nf-AfMbNc0A^F2l*-laRS%W_FWwnY@)2KJMk%t@6! z*@|kwBcs{aT-P@+MPAR~f-Ydqg*U_*CDI;Qm09BA;z8ez$28`lD-F(Dfz61bd2G(W ziEam204HEPJbx{Y0{6^@Eh*Fz5l#9yIUll}Loh|{6laF)H@{t-cDW;H`nc}*dSshX zXR_q`5XK0V?+4%9*<&m5-jg5VoW&kHv^*OWe$FtJin>i#tkvI@<$lHAQbdyGkU7hX z!`)!MZ(a_onW$H(IeSV@+j}Dm747tUCc9g%`LUf2B2kWt#H*1{qQ@7>gw!?4=MS~BzLoB{Gnhxqu59? z`SIOYngZT9>O;d)=5v}Ik#rDfSR}gZ>&n=STJGaPAA`6M2Gx|@BY!YfZf&WiY!0xnv#A_B+?iIR6T_o9H|j4Z}-n# z7%XaQunxRz-NZtkbkBpDDJ@%7Pm*7pDQYkDw-3$`BadK5UO!At1<>ScK;hI*4xzV= z;8Srtkp^iatpmO*)#xXF`l*6sLE!#SNE0G2huZCDN*7NDF z*1O3`Duy()1>saW7YwA zCRxg?$sT-fbvBQ+Bm8c>BAu&952S^32j)zbEKex(@vMaZS`9&BJAQ)G@i>KrG|!J+ zW?AhD2ypt!F7Buk@Dp2K(n5y(QMJ|JpXuS=mFyODgIA){uQZH~vQ)B%p=7!b4Y`31 z87fN@jy5X-)U;=K1Glsa_x`D(Cj?K0(Ji?Te$0Y^e+KDqTLh7mDhUs{=U+xyk~lC{ zL&BLU&&J>lfg8C^&`FyNSXpLAm=DkU)>Sh!57weII4|09^If{Q*`houQ>&4QI$;lp z@;7^D3AcDN4l&sXthCk0r^FlMKhX0kUvBve+#?K)V6yZ+hdp(gvQ19}x zJZj=Pe*I8}PB5+#D~q^sh(A#x9x=1W%0piX6KEsp@vh@WIROmTEz=+qUe8 zn%dFf&4u6792xkH)D6oB%O*)-8*h0Y&txBw5v64UY@1NoqT_n|v)s51r;pqTBaV)Y z`yDHg_u7%ZJf z4y|qGON?DF_g|`dI)U9x_wJh9YJk<63_zhpZFI=jAgiQ}P&Yil_<9yrA3Hip-HYJn zYbA_m)|{A3?4-qi!RPzpFPNN321>0Ip-tz{Y6RU&Tp?n!b z%<2*3j+pxycJ?zY{Tc}OiO}hU#Po`a`mCBUU-wKfpQ=9khAH7!Key@-K>XwzMC9pv zIL$yZ9tOnu?4ljy{|WrBKh^6CquPqPU&Pm$YJ6qUTc$zrU3H);{<6kUS z18VPDI18x0bks1^LWVyE9qNn;WX8H z7xVr52iq67y$JHAy&co_NU^*p)<}SQlsrB)qPM>lt$=HCfx#Ow!;xRJSmBm7lOhDXF(b zKNhFPh!EvI({lA5pCa=%_DcqNP>zo7hzHVAE5&uNUXvbG5xNx-Fr{m4J?B+%9I~I_ zhMzv}8tvfTE%<*}d#5l>qhw9HDp6_Mwr$(CZQGUhm$q%&wr$(4WMxjywR=zh-Tz7N zYv%Mlc~4@k6%o%9cW{$kP>Qih0V^WN>HhU_lE~%xF9g`cNqeET@_^|>WWWU`=i-Hm7qzvfcq6bk4%1c{sq@&)?eMQ&|B0gcqs>O~~+BHk)X(K{NgmEZsqZGq?^nI|QK zv$@VLr=A_E2XWKdl(*y+P@lfyOFt%u;Ggy7;&TnCskW;=IIg^`DsF(XHpCAIK|R*h z5=bYmj5>E#ak<2wtjOn;*GN^<2{(@By5@jtWAdrZ$3(l<<}h1iR~~Rp{uM4(bUl49 zzU*xQtL(8*TKikJY+}e1C=qd{9Makt%y!uIhf7#4REvDX=*7<*1#l0o-HZ(mOd+1H zA2iJ!&rD%*i^;Z4m4Kf2*|5K@f%8>;jF-Z2xrI7m3Q38s1t}Hpi3t8&LD&_~i%Ym0J}@qITN9;vqQ;$W7%)`F8?E0|bYNH`B!IoQGfLDS?kLyIz%J#U zaH<=UnRXV@ccB8k+qF@%9!ANG=!346JP7GW9kGO7uG9Qk>RUHqFrvC?Z~_=sXIA*d zCAu#Xhb1a)okA#LbDK=s^xaKj8eFnAq((nQ18#mPAA;;()#gPNzrrk_;w}rgNCH2( z1AURrhzk;9+0~&mFy53C zMfHk4$Rh!KVoo^JnK8nuJ~;wf&Qnu53^e4?y2Ty>hO(KF%D`Z0X@(oaRQ_buE0Z`e zU)5w>Ce%n<#W&6ogUU3inBzaHnpBi9d5=#fhiq~dj;P_uc-V-}*(sI%hy|> zdBD)l!#JmAf8T-#bIYD^1o3tu;k-|wT3@2((MlCl4yEKCw!A(UUwqGM{{k$pgA5-|fU$N#$TRIaUVS58t%H@`0B{63NC6^yE9qOMo)cclXI z!XkSk8ZC!uT+Hm>AMT|;D>|g#Zh<+NA3vo3hYz=sfsKixf%W$gxSg$mwXB`7iPOKl z;8n?1N(=nRAv4Ld`T`1_JD|dC5{Iy;2$jk~uqNcYr(jtmI^%}vQ-?sOr!n`D(=UB^ zyD^U366>6-NMu@9(isgUUfFIt3C=BizP|6UKMrCiegj2fmry-ou+9%k! zFY%b#E${#{@qp$j8Ktam;;nphS$c7RDje*El-X4+`W-!z{PdfftdVwYXD+=4UF!daHO zSs_95Q&_U^aSc#+?Wsn%Ri}iQS;n~DX=Qu-`xJ)FRVda7wN9`F%446+|M`X81(N+W z@cK+#)VLRrCW;4ZufPeh7*L-Ke6&Ef6v%6y-o|C&gPIx-=x{QU8Q4$h zgz%SIaYiS92$e-;)ss7e`7u{#Np%cSytL+IrEUqvt+|}lw~DSo$8<|1 zyYpqvl1vlp3lK?gC?XY!TU9RtsDf}MG&gYCXZA}|#hLM0(HP3TzQ4Y+ayV+r;PQ-$ zWAp0FihChv># zlZE=g^9JXl99QIjlsW$ zO;w_n9FhR?$kzg@>ar$$r=Ege7+!?1NO~y@x}~Hyb~BlYb9+bHmf@Y&D`7Z# z0qCBpbS-1u zF`B9Lvxpl>odnUL17H)detI=sAwGE1HQLMc=Cig{g>`u_(@zTVmExd73h7=bbCxevlS1$vHITuH_1A`~ z^q6Qn6OyQEVu+xES(vf4FXa4z`(k4s-)GLr5yq?YnJVE>|w0h2raZPRhLZnafOXB z&o~X@2KO6hc{ArFK9~z=<_k}Lw~w!%KGMP0exEK_+MB$s$wPS<{rUH%pab5%ivwI4 zPaz+%>xjns*d?#*E>ry`*FxQ~18v?BeK>{hjI1Nyz!mcZM&|jvh@l-CAK}2)>M;UG zDDy;(^;dM)>$Xr!hci}*#)_2Op0yx*s30SqJ(S2H!`;8B_6+)|Yi8g-ev~5r4_g8M zfL{Oay$o{y-5>bJ1*T@DjJ$&LwPj!sYJiN6G9($LfTX_o6bJt^kQ+oK8--;nl^JK% z0HdL1aynvF74d=IBFqMN-7u+<6~Y3SRM}#493d-8mY9>4Dz2|G?@6Ol7_WPLMwkKLIsdmzAz(Ct-^v`f!G4@9lSf8oz8mZAf z1+rfxVcy>SgMo;|*awfH-?VJpv4H~i>p>$z+!XoWMzY>QgGh*|4))@M35XHw?t+7C zB*;#;T7=p0wE=;0k?x}XlCZo~`-MRn?Cz3-EhLpkyKcmZ_YUfV77(HFuc=z1s?7Ye1|zHS?(f(6^Q4z zm?s=JgBOUH4v9CnFl58Tm-}3P3*k`EZT>t8J*a1=S07_dD zbfxJELPqLY(p#t*Z%er|TPB#EvuZj*h#VV_xwgyn)-*OS7`!c*s$Or*ZtBl%ftr4) zt6Mk*Q}R=>a}o*X#~N12S7!FiL7kXvVsMsMbNnr)mnt29gA_D{c*^~HEKj|^*LqL# z^7Pm8EIEG0%8^mHkz1et{2ON4s%3=U?K^@TFKuQJ(u9FAg?~nxbCHMteZDb_9Tj6c z&Ak%x$k*omZxfrUR*sTAvZi#Ja=FUiySx?l^IekW>oMZ1JcGm(Zjp-gX|o+RU_!JM z)0nRqyMZJwvx|&3teKdUl#NQpmoHF`bu+uBz9!N8)8v$Sj_Xq*+*M+u(5AmHn;&U|1eVbJD#dK6g76EQ(+bMv)x5+xp4=_E$j zi-}rtsSGT7+(ch-$Js$oC?+%;AA=aOnTgY6Mz5x>4SOUSkA6da z%4Tn8$3>vK>A0Q3%eh3iGBtI}FnMeCzZGhjI-DF|gFk^CkSrgn$(-XLKPv}CL@f?W zHp#%Wm;+~tGK(K+v%^hi#W7kwIZ%}*O=TLR=BBC@8y_B?{rO%_!z?&w|Cb+li$Ho@ zYw2d?WAW6Qk4{d;c^?Pk#!ztB%Y7#t&e(gD3~$ef^Uw^BZLv~9c{s*FV;JRwxT~UM z!w;CPQJ2+)siI#kf7DyA?d?HU$n~svVpAft>23;I%JaM_2%rQueLX#xV$DsO_g6 znYfX?rtEtm(sFfzo04$d8}fp7bzQ;^LyZP_e}Gw?$-0H997h8?%vE4<@FVN(+4>wa zU4K(gL~OXjo#jRo0$dV)IB`LR~I2)4ou!ZMK){KUVb8O_sHf*g8@er>T_cnf8pS?+f7KIS`Gh zAL~DUjMCyBuR^tzno-|Vux+Vj;V~)_E322!im3aDE)zD(D6{@88MX=lT2$O$*%MtX zfQDHO$`W1^k!Skrs+8U=Z`o3i0XZR8WCDK-j?&8@`IkWQy(HW-MDYW==z}V88!dMp zrag2TpRQVPpk^+TZy*0taNbYo#}_~RmuQ&DfH9PP7>x)YRiA+?)34QE*d{3G#!c?O zS@r~$!;P%@a1EdL4UPwlsDnl{kmYIxpzFkLiMjldE|IehiQGfFg3Q)Dz5`vh?%6(F zrw_vw#%%3|zW+1oWL3`r@E2#M!EX<=d;u+3I|?$`U4wkMqTjNUG;y!97NZ7A#a*d% z1T+r?Etsd37`wN1R>&?;7#P0M9T+~?>lS65hDF8z@3ka4!n18w(Cc{^iyy(jP+K#g zBE`N=b+>@8i^s6)H>ia(qOb85dozsCu|Y>*KWnneK|A;!BjpRdKPG1~&JX~?+6;Y| z5_XBPGp4=Vx~QAo3mawt9*3o16yxvZLc9U{u;G5wLwL#za!Ih8@)7ch_Fldp<#afe zQ&hCCD~*9nzQ%Qk;7-ffv0JPhTOyD*e=1y=fo~9{cN9+IPnlNfG}om1W3xmWzL6ik z{sJF}BM#B`$7<~1JG<1rR_}jo5976YO~!pjhcOcsyFo^G0@aNV9L3vH9YdA8N+m{0 z4{L3(MDwc z9pKia_LHc48G7PTy*F{8eDbk;O>FQ=JIotH{Cuws)M<}$B#5;Od}Yj{23<#)JLtaF0ui;N5YM6 zTFiumFL@E5SS6)mZMAtm&+1YkuWtxWX+Bhxq*{@$YIW63*WB0?`Lgr84b5bXzqb${ z|D-+l9_Z_BZBB(P@eP8aSv;ovVWH~vV?#0;CV zx6;9C@t&5u`~W*QaSB|8yYNr}X6H+D#0WEVFTn6}&@G}4sw1)vjN;Wo?XH#hCKkkt z=QQzvx9uUQ&{uIt2~*UdCjuCAc-NxMT{PmQy3j+F_F1Ao(D5}hlc#1M6z1op{9XRy zu@_}G9h_c#ju7q#Kkj`9-uTQ9idPdDrMn33&$tLR7Uf+Qlp3p4{{hTzfBuj=RENqp z=$*QghVN$dDhK&Mr|qv2%8fEj-zqUr9%_m!lW zY~m_9c7yEHO~2&+c8WaEW_B58|Mm*OU`SvBou(-R3}=iB)yWTN7I2%gCW%GKFpN{x zL}J{@Gh*ru4n1~iCejw1w#~70mdZ6@Fj!`=RkH3&304e*EK;w-L()MF$+Wq$CLL-r z^-0M&o3XDO$5(J;N~L2>N=3VSG<6Xr)mF3a&$~-d(>IHiLm>~km$2G71q#&7fumJN zoAygXA}5Pu1;t&Rl($2Zcs#LWR#!0}gd0B*`u`*;dgZPTik2_vExksLA5S%tkD@8$ zun3u`h##3otN``ap-o+|4p~tT>F}5l4gB>ug(z=B8qbZDcyXJozJ7)jQ;;=RmCQsT z%VpZyduXe`$Wx=IeWK<_3yjLPY#*ksKX<6VFpDzRx@IhjZWxaaBT9%6z0gbG(K)+n z&LI$UBoCSJRRQug5*;ndZIGFRL{EYnJ^v#GTYR$y3PA#K3I+zmS~Jd*cA>vX_>Z7w z=CYUO3?f;L)?zA*i6}Sqf|ComEkqh zCC*{;EDr zxI2{4ai_Ei9%?MJ$1H_{L%Lk{QUKuFo8kNn=s9l%x-d7&b|j`J&;!rHDi@-uOi8*U zw~4GAqn7~QDfvrAPI=#|Ftp?Rfg`5;AvD5wM~8xU`oi{EaM1ID=f4Is>%weQ0DyBy zOe;MDQZG3mwS^#nR8PG(W3)pJwh*F0*Nja;k;J7yCf9Dahf*1-TCIkH1Dvi>4xvia z4alX_Xy8tI))(r_=)aA)`xXwUp3(Pu!l@v;!Hka)>WMK+5(uJoIeY(Hsprl?^kU|X zNo+r*dCMfc*fdw-UdLV#(VLS)`8MyO0V^}aj;)cLxiEz3q6H4Y>%0ugt7MYeZ;w<0 z10BeTYy>MjC~KS-+!fM>?1l9%D(y!GPZdVv_)C=ZbL@nTLeo1lWUe9;4W3E=Cd8pK z$wK!5l9+1Y}zr_A^rz#)RK!PF<#%LfqP(B zB7vbUTe5spiniV}`?099fMaUZ0lpr`qG5w%8t|;${(is;fB8b*M2RQ?m9^Y8`%zOh z2q|(jUJ_^20XexL7=4}`Ca!7MxQOK}(u3XV7zyNgWG(J27C#}lXF4pnMLyCl8vc;U zO1>4wpe++WqBoDyTl5+y(0a1bl1Di~@{Eo4cnrYNHycQbHDU0}P)#rSd2n-irw-j4 z4YVffpo*_WVWoW>HId~I%z`>5IHnL78$1R3&h@{ayVP1k4b9f3@A z1$ArT{j1j0dapDnlSTdssB%wx50LWpy2w?oBi9WCQ2N`U;Qhms>X*Ihu3VlPWS7$E zLWu9Q=+2{P&AF_*T5?mtml9VN6_E0zsym3XFV&g|t1-dfy^Nz7rAmkeRmn&=_#Zu;= zRUo%U3_Z&ITp~=Ex^MOk`K5tzewYTO+^`u!$vDLIMOs$jfd>RZ%@Hwcaoi02@bjje_(bXRc!%Mf$FbyhIYZT!PKNa*Gz>y%FO*gZPXQ1$~ot&UCPtg z13YRS!8<2;XDA}w`GKkXO602v{i{d7G|Nn&*J6IxZxPy8>{7Umc{NR{^A`m+yd=T# zM%Cqo=ehH?oW*$2c;XLWNcsif)r|cX*gsVU+-VJk)#zPQ=sjhewF9a*BuloW$!tov z3bn8-9w9kBk+yV%tJ_wkDd80C>yGrpbC#iFwdVSjPzmn!BmqF%el)ZN6;K}UrtN-E zTk4|SGY&H%GRp@Kt(lJyU>2-C@sF%3@ycri3WqZZ2}V_oWasE;*0(Gl>nlfZS*nv%1=ww9lvidlKyRuPw8l-nFuE7&0k_LiUo7>@BK#z^2d8oHw zN~y*jDO6xCt_-*%Y{0EqzgqI?uJIvnjQzWzd^e*VHv=JBQ%2k1m@tYhx%JzuNt5oC zt}afI(;y?6t~I1fCfJ4oDZjzmi5wXMt^UWbMN&sx1;&$IbBxc$%s@~m*>^NZUPWzy zvB|ix#r1Iv%54AziwPKg6G|Y^XZ_#Q^nQ-3R8x|~)&Me;vxErbVl!Dx55@u%(r2?m zds~{n3=P012+a|44*FUoW~kO?Mf6?~RP3`)?8k-lEx;!UvsX@BK?6sVgqdo2zdm?b ze5U5yHUQU_17Mw%^WBwD+&!aSz9Q5VVtgwUzy?})qprPT&J!7gFIhpNYIKA+&Q!d4~hYJX2P~J zW&d8VmduVFkvcRzQQo^q)YZb%MW&EB4LN<8&mg#YWGvE-zM8%%_$HL5Vaw2(6fCMf zv`(CCVG1 zMAW?PBU&v`k;YKe)K{C!pB%A2k}!Af5dO8yFwC0jWBl&hUw`-Q75*oJ&p&`r-`cCZ zf%A9!-jVp9UqmfzjsIKysZu`s-|EjNYpa7%d@~66vw}H+STHCb1-=ML9DH6lB;@%_ zRfA+RdvlYk`2^H0Cq98lJ|bMN5P_)^x;Zj}xQ&OInd@KU;mZ5Ua1i;x5PG1lM|MF}Vp{np_cm60og|LY~s@4QnYw0pQGn z=KHvJHV7;xhLhz8=vt~TW5gYgDO#{%e}`3>`y&_`_tdC+?AJV#;W zi#Fpji;sQ0elr-X#q6H6KmtjBG7H-C1A30Gx$@@4`{pNnkUT8U%AB3Wu*osb!pCGG z7szu8QNw3X4RM#VDp;Vah7ahYrX> z+nyhM(SZw%pHYx>mg3SYf3n(2wrXoFxH8*v)fr=RL~FZsKYosB<%O%f?a&DFLA=4t znxPv3TUnSf!D}Yl>Dp;#gXj74^?9b}2YZkmvU4!OtuA`azA*aMX`GN9_!B!t z8862p6?*JL84dd28T~Xq%CH8sO&`7Jp}nktrhsUlIXXfnoT2nI+Mzu|m;;Ai{J;@C z=52RSd;TskC>|sT0IPV<83Or$Vq%)CF+MqK9*Kk~hyE#}D8Q67M5?`tBblc^=1@}- zl*-PyNw9wzD&0t(bWAxZN%xTY2Z`1&hwu)To(tJ|;%U770Rw)Z>{rOoa4ouCQ^krM~@CYOMF>263s2Bj%|#XhPnROWUkJp{JyPg7*6V9>%u z)(NCFJFVh^jJBb6wADRQfY&h+#t7@Ax7-l z8&rY)i;Jmy;~bAVR@&etNawkDI(thD#{%sGQ|xG4_XC;~kn{WYhDyqieVPmp^foz> zW=NG~AB5N$Z8kKPVQET9+q2} zhg~N}j{fG}waxQav;#=>$_dr5R(!chl9$3jVEbiBG~{db5h@pw+TB(hmy&mkt?Sue zC3w!2lv4U$-c5uom^p^Kxy}U2%UQY=y(cJtV>&7U(a&sywUIN*uMf99STzoSI81*$ z-D#J1tsMTSmgxZ|xEmt18yA3qS`UoY9Dq%HsC<1jkWHtN^HM0YMqi>ypRY+v;(gR& z_lpL*^qRy~H#@(aP=c3Ltt#^}~9=sM3|x%G*m+5Hx*iQDFrbO(D8 zz^}yb%LaN}+az??3|i1fdc<%J(k)BK8u?@UIH`|+T* zc=L+tTx;+FL3r@sPEqdzJ6Gxl8y$yJ*m^eMWNr!m$sD8RYm*d^#rR8??U#l1i90qV zCxS5C&a;pC;13_HK>^-lI0NvTdy1VgbJsoWB8w=(fp_1;mM4$xMT_leGzYAM7u=LP z;Of>oX{0M;1z19Xb9lS!7Kztzmn^+d1bjtUG5bU=CdWtyztPm5fz#y!UZ#ay@Rh_iM@gi^c)4)k9vOkW1*!6Gvy(!@0Oax5u3+P|(VLVhd_xEpSNWix< zg#Z8Q9{$JH`-B}?WCrBnp6O|N&M2trh%XU%V}kQAQeE)7kk)XA-_eMOc41p<*NTiA zvYeocE%%~`GTuM%#@*Y0ThS9^kt94mWut^7} z!aC3mLJznvoZ`f3l&J`nH`7DkQ56MgjCp88Fb+}++2DH;{fTarWv!=9YNSJ_e0r7n zKO zQQZ+=j;N!)mXH%9@k`6xQOdDlklqqT%|t9C+nlL#S+1Z8t>3;vCLyyZPxx9`w?sb` zM^e^M%CP2imi6;Mnu)AK;PCV)R2rF9c&b(IQ>)90aW3|nPxd3&y>Z~#JqjHST{mY> z7Tb|lbju;lbQaM?bd_0V9YamiH0|MzsTY!qjtZEB6}|?Wg$@c&Tf~N{qOPh0lHhTLdqk5kdwPqRGrHO zR9Gs2xveWG(qEMFR!OFq#h=xCoJ-rerE+Z<$X!b%+r`wS^96Lh9QTCv1!2Db_Q2~L zmeCJ>2Ug>^4e&o;9RIV?{a>b$e`KHk6;xS@8@5Oc@I2er^|2vEiSJdjsD<#5RPMX^ zgu3uPn(Gm&N$vuUthVQr9l9P{j;~a@r67oSyk0IOpC@O#m<~7IY~0h5MxK_ z&X2AV`r{fJ%38z6miV{DUYYZjWtGmu)eP$C;D>rh)=5BaWC2K*;DCy^06UeUOeNLb z*RO6H=_14TrxEEo>j=tz>;snuE2gH(i;&Nyh)^y^Iahng6431ITVc7J8~r{jZjQAW zaOw9*;vm?BrBA^4=AEV(W_KXgL5#G^k@8*>?w8P_2M6X3eo~IOc5^Yl`)!E=(AJn~ zub-WhoZ7kptc+|c|DXfh>6-95YeA6iESuLmyx#devQ35TbiU9A$B$iTLr+=W;_&;! zXLJ0*m>wecme!cz@xU7J7-hp{e_Xf@xj%+g%Ut@{<*!>DlZ#}7>@zz2&PZtXahp<; zA(cAE?TQt;e_{Uh+F|mW^#X?Y@xvSMKSJdHdvyGprAQsZ8+i%iOHLfU9zzgO5#AFV z2ob0iF;*NAh!vaE8a6B>;E#IznCZT;S=LstaFxy6a>06e6?s4tF3o!RVgN*Feph4B zv(MUl_s#pW)w>^4Lx#0!$|241vsc58*F*Q0Ten?z%EA|q17t7chuBcoPx4l(-@iJ^ zen0Qq-mRkb^0aCVb?v+P{U~3}-REUE>Hs!e`6+;dyeP1#7x292kJyO^7cnA4iHkb1 zc^kHR3(hMa<>^iTTgaW6|C~XRG(ft$%oR6GNIt|7ofRjL%g zNm6PS-fI zJWu`H_-b@~&VKn1tK#WUyoc2qR@=f|*ZRb&L;vYfus{p?T&Zox9u?`#pIVQk(S6Xi z+d)+~q_&mOvTlu-2=(^0a#3eGo8nl*^%JlmkjLNXtMv>9kQjF$JFWmNPY**?GBfuE zd&lV6hx8^E>J^L+C;2m7OXHCWFQh(1F>( z4$Mt)1QtYV0O=#R(xnMRvb8{TnrQ!1ngs{aS^POZx%sG&y%!^bRQrC3*Zx+Kg!hV5 zzQp_PeJOcZHe6CGvdRskHIIE}=M-ZGDwHH_2++D5NVw;YR&5NRK(NEhPQ*v~)AW z13h|!ER}`Zh4tbKOitRGg${msbbtxRZ?;ngp7jE;j?nch+$~o=56tC!1#aHF{RB+j zU^M2=Fu95sXt?*aMjuSsg9c36At*}TlKltF?%#CfFYzIC2UL{Z<$G2y?ICu%8Ia3a zCNj!*G%!2EZp_|jJe0cgw>;i@18}!OFns+nEFZ$=cSw>AYYicFR z5$T8G=91pZNf=A|bsW^`%7p3>N4QzEFtW*1_7iJkm;HiyOG&w#gHnl1iT`da%J4QQ+JZBFPf$DlocgQ ze2y3?rU3Ya&?a&nM7Hh)rY3TgK3GhIJZh78DKHyBb-RNS|vlarg^Z^BMq@KG{4 z4t3-unpW4Ej9+-}W>3RzoXe4%ah5HdIRLWZ%T@hQFQq|Ben-EqOOQIhB`fZ6wLKmY z_PA)+;T|Y0_k`md+twhCCJ{c{($AsMs;SXZOkMOBLFR0*&cNfrC{pWZ-xS_^G+rdl zuUx1Wxp#^xoo^hndZosAGV^~V2_e{ELe>HrWXK6Y6IJPWAWTN)5s;(w$Cb1xdOdl8 zr#F^T0h-!^mdPDhH6jUtsu3`&;)61UP_G?n2a3!kb0}th`e*cz91+bigeiuQIZua{ z{0uCZB5;Lsz~+6Zc%Gv|H$(xwsZxh!WO>)SH;o9$Ewos!#K}={tdXt(jF90vqK}g2 zJfZ1OWe?hkjf-vOB3L9z3cg&NVX+i`R;U-ufHU5U^syzui{`^xTP#kWWu4=^9`T^E z4;`sPMF2+C1fW1#VB`LPT&))GK|-0_9hd0>GWf{}(!xt;E|XBx@6?Yju4rDtJfdLM zBX!t}3)DqEuS#8fPar~1#J{OEQ{dDw(YkOJKZ^vKZoitb`+`sn<9lLR_awE6)uvzb zvRAfTM>>VxL+8@v4q3DBro5@X1S<0`50e5F4Q(|Dpk zcI=F=(PJ5Isixo>^93E!{_fRf$tav4djH%Gq71@OD%bgGVUlvIv#MZ&>&oy4^e{8p z_HnjUn&gR^!JTEqN*H=NDWZ>C584^>ya*A&vn8gR^|Z`)c$adxcM>6Rn`szd5c`{t z5XWew5AWs;Ej`m&r3;jzH1mpm0%P>k4;66d5_~X#o}*R(!m1J$Gj?>}ha_PkJ>zw7BD_ z&EXV9wNg^0E+Xvk4Qn1ZyYN?;^N=UkH#!j*g^PSK+tDZ940>u`{adC^ikkxckgT1IW zk{;G_^GQ~6B^ZHxRZ=~WiuOTvhC^vjIWgmtTkzqA0z)4R*`i&W}Bes`0k*oI#%9u+z2@G2xo>mvEPVxZk=?x84L7;88yNEg>S@}wCV;(a6=eIlNNJK^t1I%C8A+f03G#eVL6DmLnpcq;e(73T&s^wSn>5Ag5f zH?i>}f|}oESuWfkKREv?TqL5&+DBJ+?v17M-4w%_xLEU7jt)gFf|=gLY@y=LpmI^hrkAJz*v;S z-Bey;L%aq%ypHVkV6I2PF}Z((4c!``;LY8}x%Q$l+!X5W5+c!Da@hds&aX-XSRw-lu@-ShCkW@D#ium( zD9nST2cZXZjb@s2x^kkq;HrI;){QTURHf5lW7ZlRVBI6l)JDpg*T{5^iq@P-5)FYO z6z!;0c5G>`K~Y8)DPn%LC zusD@HFC9i?OA6b{)OJlLV0D9FGN5s7aC6&42!}EyHRiu*o)b*(`vLl%pGrFOO2RlX ze@JVh9SR1VU5Zb%nz#-em^|@pJC|Zp9YV_*bGHi^CLfHiVl~8~O z#(IgYWsL>T>tPUTCQ8yfcO(UFj&R*2wBCEpL%+trXZy~S#yky3#ftdTP0W@JB2`ey z8u1LRvFqzj>H6OU2!l*F4_2!)3@+*V8I!^DHW~?H^cpEJfF3(~AU8@u(I3}hvs^`C zSZzN-R;JF{s^N=71*{+zvqZ~6JCdX(FBWw56O05HN5+QV{g>=o+@8TFhrU&sTB=nA|8U0*A@?{-{k()Moy|h@j z$9=n($)iIjkf~j{xl&Vj(cT_#pt7lb(-vltY`lUQ`ePx>)XG^~H$#x^R8&g0X)=}( z*70{(Va9EMV4ExcGoJu>zl9u_C?P~|o-Jk0gKD{=Og%4>w|+>WPv_*cz)1Ea$k`wHH4OP_Q*e$TG8U!Ef+mf1s>p160HO(D)I$kSO7gEZGP%e>P3gnB`(y za|A~ysoxX2QQDJ;P5E_zPRs&iGOH3x=0c`;HEF8*mxMR_JxKgEUY+%=et)7?559^| zKvO;la7oOvfHB8|M(bnk90$6B=CT<;eEmiNiOv*Q68J1Hsco!r`L)y_eJ%m1J>UR# zZcl?bd%)BojcArum7BO_EHBo);n7VPa|Ix5H-BxvBJA^5lOyv&hvOyVKT|rfL);xL+?&7oOR|Cb7{Nkt$lw=ge z{U#sn%WU`?``1}^OZ*A1UbF3egkrvqGE+a2i*rwGC+r1$`V2?JzuWU4Zga>(BQxscP#15o%wAoTdJK zJMx|V8^Rd&*BJR^1xNAO{a+q{q}msMC(RGhbLuOvFwGOq4|tncyE``!O&v6l64_|l zPBiE99jUW@weB z3MsWB4-ic-2eW6gXlupBA!L^pAl@#u(-IYMRDDJ{{)@R?(>B*&euaJ#_@4oKGX7SjGdWbn+?*+nJ9q95X@O zQf5sc$AEM$31900Ge?n=DPE%oy6s22rUQQu?E_x#e|YewCeUp(yvB2yb#M0H#;xt4 z9!6>r>va+qYv$OMhf3UvLnswtqD`@=^LVfNU2_C);F8)EjEQQJwt?z!ydh3}vJARo0P9%-UPVf$~rTIgF6{0~SU zjwb&-A514LCuC>lXy;;2ET$-@BCo6=FY=G^o_~NjX)l&-646@1VQ+oTe@Wud>oE@Q zRR$GuYJ-jKmrPU^6*fw(?Cr$Ih0U42Tz+#owNU#B?*gq7oZcQgxo6tCT6%o_-XU_s z%pk6RPK%HN|0uB{cE*(NQm1#iUzz#C3y3NSt;$4qH>y==p=C1XdB!)6yJ=p&civ1q z%OJc*#vbd|>2Rr=q_*?Vh#F*+2kK-jlNhWfPXt!WWf6KHgw-@=`rdUoHKlKC>0vm= zswgHicUJE(>TMVX-8EE#_7(2dua+r6O1e`YKny(Oful_BmIY+AF0mnrQ;0()2t=HM zjTG>#I&!Bto7@LG6|A`anN;OpPx%yr5jy)I1Ba7)&F5h(Ez+i#H^44QpjN3DWaH=E zddAozD;aVOhbFcMRqKl-an9&$jpwPQL+21UWFiQ$sIQ}M7Jp?*y*f#uN9@Lt5lVlL zIE5fqpl6h~kFIn~54mVX7`6}@Gabm039{wS&dA{ry}ZRcVmu?dYDGFegdfqVFOif^ z7uL4gAz)E*JVgIroV^2+CQy_lTJExK+qU(UZQHi1yKLLGZM(W`+qSy4CT2HcXWvG= zH}5apIQQO@Co@mpk&p_o^+D_I;Yn~Qp8G{X(V%Yn;+)}}Xv zoeYw0lQ$yvz4s0ZnmI_&ps$F#1xif5{NL7f{zpM~@Adr)ga-mzCj$cF`hRc1|8E8T zpJhC)0pp{xl>VK|6U!4TEh!8cg$O`pP47ZPBn4&-LIm|Bgrsk?3Oz)wkuw`fheEFQ z)!49W)oMlP{B7H8HN9-wy2>6HrrJbzEv3HY>TYfA-n!VlwO+Ha)S|sn{pkO((K=e;y5`AcU>_U z!fCOipsx~dpA>6dl$rkUhWP))Mtei*AY(&sZM#rM+;6uS^+sMy9_+dQ=_Y)WMBN-D z#SjE3#Ec$Iy1RA}JN_$;Q6~!{n209;ztz7#8buzn2L2Qscr7pbY!&_LuxY#Xclnfw z_}5EB+mB<|75=l5L=4jUPMT`*dwjc=^o@?|V#Vpp-sRIDs29!RPLk0a%&`YL9nVR&GB5k;GG)v4~=8$e(79rh?O}mj>I&V_TDL^L2a|Tw(R*FW(zP)sK8F^O_Zaq2bN`d3cvNF{e{z>0g?j z;AC19nZC9c_(Rh!qJv5oY^dxCNYdm_~ zWAk!qy#TCz`u1?i9C)RdXL-6YlBg*hlh!i{u90ob%(u$>RaJT(#58d@&7`~y(Z>(| zRgESwS(8OV3P5aXPg_XLfX3mO;GAwyoXFwRfN)dePVQYULB5iWQeiS3wMa2JO2|*5 zuCCL5zFuCFAtyKv_gOD@Rsn*M!?a!u$x{_Y#x%}L^bAFtGhyOt$F{YK6Y1PvMAg*u zdjtu6#`gSy_@jO*#*>PR%Go(-E>3Rr9d{BzUXz@!JP|)<>N@%OGco*lCW;LX!d0$& z?e_2fJ1iQ7rkuhnTHMR(y#0~Jx=N%#O3IpJ@V|t+QwrKA`}Bz}0gb>PV!RacK1F?u zM>;@xs!EC}$LTCR-7=4voY+Xu845xMDbGwo#lw@fF?E+KKnQeYPd5Z| z-|8)c?Pn3$8L>%N9V4tx_!Wekqs$btXGfZCn2%!Llpi`Vc7ehalW4rn@xGi0>OmpI zz9hC$P8vg<4RJ9~l(w<2S%;f3H$au5W7Ao}V0*Q|&$kxsCShHB7E_uuVNL10F$u9_ zN^U^1MEd2{JS>b;f*l67ULi?REPIl~{5R+PFtJEd1ha0IdDz3SkuELM5cs~ZNK#~k z;nuNNpo)qLMsi3Q@)GT2Y5JH)JCC$I^K57)IQz5?d9qanlW29B;k_fDO*s)g$bzC} zOwW=56~gL#l$Iq8o_~ZjT)hk}#}Kq*eP01UsaWuU2cmbW7}lBT)^F1+EZHcTo<4qA zb2<+zn*~;p;vBACoSkM^OQZ@V6?>m~g9#ffV+KT=r}j72Nm-uqyg0Ajbj^GgPa2C= z*i%6~#VIRu6%o5+uv*)uak$b-uymrgl!Q*9%WLB(s+2G2OC-674eE49+vYA>All#c zMsA?RVXsX-AE_1i^tZ=q+35{!oR$V=Y?R8Rv7L08C)dlpDkshJ$wjOLv2v*tTn1d% z49-xiZ3}2++SJ=5YWwoB{7mDgYJ0tepYzeDF8TbsG6p$xhcnF~y@9B*q9^cJ7iP+~qdcCYtUV&xYu!awp!GHMV#-scxRHt5LCey|JsO)4 zo_DBM>r}v}&;w-XFO*ow$EjV=E4gWFXb1&db-l_0L;Txy7 zj(ry|uhQ4zKZa*{mixoZYRm>Cz^ZB7SUNs{(=}55PwtTvBPK6TY#$HMPN7x5UW;cIn9c7P2vj2$+8(rx zNe4bVAQgrfeCFgS6x_tAS_&J6T=Axc^T}IvYl%GR8vJ3lB2E{=D7pya#heQ#m1h38 zalHs)>k4<0F$bW0 zizlD~63Ch2p}ck5_Euwj)!sPCiFRYQC3?l$Mt29rm9{c#okh0O;&9#yiaT#rdjh-P z3=#<$iV2q|vEc0_00Q+%Rnz1@jJzUg#e1Ch>!RLEgk}iDz16Mg5*Cb zGlfSF+caPWldLzXAZ72IGS-0%l|&UuwQ{t}{#{;GFCBm)o3C*gb< zH`W-6pJ$)priXzouz@_O-!0?{ap>s-RvFti5>iZEd^^kvvq!{$-4n+5LLl1JT|w~w zKEn{pZaoCL<_mFZsH`*Z*!)E59ki{z@3My1k>&G#}V^#NnUy%QVwQ+6frr6Rzapx4H_KPivfFXp=K*JY22 zt3PxyaM$ke2_2(9Fh|k>C$Za6J=dw-0N;%2c zW3q&R0Me|&92y4IP$X0BQ5sauN$!~-s=_Yr-_U{6BX-0eJr`PB0&IL7zsD5#L0YqB z^$pAg9cb``xmN!E((ph{t%2?+9PTe{vw?)e*D2oJfL)OQdHaZ@GBMU|qegC-+q_2k zDz%T{CNG&)HLj;~s8^-AKnGmRlb0rWZN~Y*x!QQ2c0(o0++zkHl|&PEfuAee(w)?9Z(EA^V4ryh7%T;Bf}?+!=p@>y3`yk@E-Y?4vSO7y zTc*Q-=E^RCvXbzOdMqP%kW z^@Lu;>V3=P1f`Io3V(&Yen=l4abHrl7MnOpb6EUbtRobxxw0MkfJ-|iD2G0|aGejw zKTf+;=DM$ZX=yQU)P|FLoI@}#b(gAhQOqvCkI(TmA!8x!-8bwPFu%SD-?4d3*19S4 zisiEmm|lq^PxJvCDX%&pzA>*tXF`MMC)BHY;^JW1;z<0R7f$xl-$&mOLeeYo>`8Ub zNok{M2g@Dw-K2D1_^eowu${VVrm_6lEogQnfHvU?q>v9n^U=qP!Lcw49m zHz~5}$@zkCTPquClZ`TNXUfScwN|$C1{E{BBaBuaTLPgR;==0aOEd z?gZl3&sqfxD9-K{p{B}gI|StI)(?4#f@ErA7uJ8)c}I| zC;y?u7uB2}SGJ+QFw1w6+$Z6Nq(+!6n4h6&PyK~j$#zo_$l%n{>f+$Zlxa|g~V@cmNO zcZ++S)_9j=Nyk9;wMqZ1mFr>P&h=whX;18>GMx zHK0xOL?@yuCQkq84i14GlLJoq2 z=8r5=ZWKP!Vl83U7?k%sH9*n~I*|wO-u!RT_5Xou`(k)fX(4{1>wmB<$A1^sI@vor z|4*5q`o@no8}mDRlN}01IA}EwRiTV1uYBDKE&}9thOUviTSN%u$FTVKJs<{^oqq_ue@5IQKO6 z{&WAL0LpY&ix$xfJAKZ>4t={d!S`|y?NemX;Y(pP86pD3!5BS8ll$J~cD4yPj_mvDSwO zHIm@vuOS7v1Ln$7dLPX2AupEF(vW?oaRk6!G*G95NiArEECMdyGwdKFoSBJOs=}x; zHgoVVlb`-5)k>l2(*m5fkl#daGz$iC6<|Z?q?uOdswm(JbCTL22U|?NCU?(Lr|IXh zlQ7T!ut|@OQ=%kP&(sk&l~qSnk(W3Omz9irHBw`R$S~I*VV7sX$+=}0ZLf>LRL%-a z#Fbi?hy&x+x4=ZpH2uesTld+UL3l&G?bGj*ECk$9BD-J5Mj80x;=0K+&;gaP$ zWVpyM$s{zrU=7{QRK>?Hs$|==OB;{=j$uPQ4Ij3mqj=j4Xf@@eFKvV1x=Gmo3^x4g z`UqXoZ8f6Y6zb7jMHE>w3c>;#w&2Q6QBg|6;K%TpMxu@`;v&2_G*)M>9z9YEK2e;y z&WYw_0?Y74Qw%j1<&D%OZrSAmo{9AM2`y0D1{f69NR#kJ>b!~pGlY9!oq%@{@2lz{ z-eK#RQ|kAzgdCGG^atXaLW>O*u2S#Aby4lpb$(Rw<86x9ONjZxNo4ErY-1Z(*=E3V1 zZ*jq7-w7^)Em@>WOJgOz*i+*xPj5&WQ)L-EPuWYt-#(|gO#6Y!KP~t9HLYu5tqaE`8ymDm$h~f z0-FQbl;ctqe3OWv5-;@Xmo&9Z;YvYmaDb~vn%Tc7T-_cH?Nn6`wsHx`;5&XnUg5^2 zxTeskl+c=fzeK?^$K>JDJxJL;OOdK)^sLKsQXRen#OD>czUkQ>LlAuNU(akA9$0pS z4EvCrc7nJfenVIe+roZfUyz;P<~!%qj^J6wlxb3?lHjG~S-JCvb9Qa)K`nipn znC+Mlaz*hFeA1J1vk@h3G-<`5{x5jW5e0s)4+I(rsoX6oCjt;wh#wYXm%d{Ne5rvF1#MT2OHXmoh5tWBAAe%E9B5hR+BV;R$uzaWPS1-zqeA7O|<={?O z14vW7(q$_;V=(Xh93++SvrN{{fM}C55T>X(kw4Su&>fVB8|sQj*tW=ME$rfP8J5S; zb4)nM;(MMWM(jX3n=TAQEeO3qhJ)erPd960JVJWI_-z4~9?l#-1wO;D)cfd|NM*q3 z#)M(ArsfqxSzFKmK;`*$8NI!iV6-FG+CGn%HKc-v?VQAM7btik*a&w3TWU62i*9ns z3Tu5%D5M1Eswl`T3O|n`^<`yyNQzm<7s;q>xgsmHX0`UP3hg$)F=*Z+sR}RGY&^H| zkfLb>E5+szM}kvQdej}Quy%}DOKRr(zcK~CHln@sel*3@Kl+CM588(RGFTP0bat_{ zGj{o(e~{{p{el`QU#;cAkgEt3tBs%pRvkN(fuD;YlyMzAf2wM0?Tf6g$ zbw(RS&bXKWo*Snbb`P%Ao2ZysaB4+;H6iy$)qmF;V0)cY__QLT1KLVNa$h%D;72N{ zkvUuO`iHsDh9}$)uve^;`rh!kY2)n2J;E!OaRb*?JU@81hE1@Q{5nx!7F%Z=DVk|p zOizb|B>5ZvjV0UKn*DBwsU!d+-|6Jw&MS#X(yQhWu7bhcg4XC$Q2ZK=IdJUvK2x5V zaL2MntesHUz5O*8WlVJ%RFkEy=R`Zd#pAi7X+A$K8yU=`({jsJqrn+g3DHx}7|8|d z3iHrL&iEltjJ5t?xH~$p58zPQ75q}v z?x@#EKjn$xlgbXcKu7O0%H9U(YP1E4D~)guv)|EqQ^EgWZC{JtpA~oF-34iHVF{cq zWL$wH+5vZ$bt?9uPGTL*5x?V99o=9`n=}8&Rh{BLh%^smq6fl;gO8M;PxwM<>+~=r zqAPnzU}LyMge*V#F^OzgrDkqPJ4ZQ$e-KjfqY0q5_54}>h`Cnmi5&cd#I(8maf(N5 zWkq&O8zVq?3WOk!`ZnnZg{Tp#Q$?IB!+xgZ@$XrXKZ}^8_my_hxH=y4#Q8A)9FJoI zq^_b*=^DP%GG*sI4>JQ`;!J&RmsP_vBp=u$hlnTOgb_RW9Ky+;Fi}ux?rY46S8C3< zj&Yo@ul#LBwFEb1!f3 zll&9ceKK5>jkpdeE~nVt!s_JBrI5@^cuZ-0>_R)kHga}Rq3#?{H*Qh|Lw)E#aIS$W zrJ0SvO5#+aN_F29CjhLl2o}8lxfJn5+yc6&VLO5_121j$`t zo(>1aZu;^Si(m8yQolwVR0rFX(q?aqL&m{M5>5sA!%0inrbr#(8UZ=4QhA;9c#~DA z#mcRh=hbUez(S5z^EJ!5jpyjoQX(z=-(p}1?o|;q*})N-XlazIa%+s%W|pehY?U@D zN8Q-&Q4zW=Qf9+(IBXKjXbpnpM$BA8^Ptm5t1lHfhU)`_6Q?)acQwJBsCKW`;WY1ZgHfGE`^2 zHyLSo>u7w&Xd6ip>u+WaH*}L@`o{TUXVb+x_mW5%;+Epq|32k+9&;b$tm<8G#?-oY zg}%6=I_OFe9J5YcX!LPl+ns6*$sR9ght_1C<>L=27`&p9T34&Bg;1rv z6dTL%xGg0wCEsd+^BV|HB+#h8XGAatsMNO=7lzsE|FZuvF22f*Vr?$h2iH*-U1^2H_2oZtGM6<;aBTQtd#o$i{d${RREh`8PymLN2=pf8{9 z1MCj6Kt50GHffqIb3^!pWQaF;_7rNM;@b;51n`Dy?3`nWD};`bA&ChbE;m9t$QOvK z&J=^^0|8hqeIPNF>g9MLNa!v2uy@(Ko{J^*^N(8WZ7-EHYn^?YKftd5Rk!eodSjAX z6HT1ko_)}R^(3-q_X;WUEyTfHI)XdOA(9x>&BV;BuDLjOf_SsZ_eRlO>!-tA3qt>Z zt3Um7`bS;kFq}KuZfZ7rjghRQoJj^fVn$)i9MT-CBKt=U>!JUN@LwMz*_gJA^B3-GvS#m!wQW;D2^PlXq*b3ik@}L05Ev1j}yCfaQac8MVY8YBe9;OX>fidiY%Pu3w3u)8*QK8(r;tn$rxG~C5< z9(z_n5qAptecXt%gC1hXl_Cj^g9RD=vs+QMo7KFK`v}+GwW%jX9=U6$bxK-_!9hj2 zVxw~&aU^mRSVEBuV_!tPZEwbc5(u2M6P|4s;VxjKu!F457+?$R{~@X~BYAPykwprt z;3vQ zmL9JHkEJ)iKGXCB33)-zruqMy1b<{4qsRICPOcY#)xxj@qJ_$VoWp?AMrh^tHgUVClM{=!U;()}-u|t_k=wz+oPvZ7-hh zf&^QB=|D-NazNTZyRoIiA0=(q3#@;o-u18u z@M%hI`Gg-=(zh7koD<#tX4&}bpQ0IqHsO{ZIJ1@24UeG?q_5lz$tSpGAM-5j^N!~U z`s8kmAkpl}2R3)wCG#;Ggs2GiT+kkX?sgnddgk8dMjNd<9OXYEm zHn)-p^ad)vx9x;WU9X_tq!9YYBltCx)9pb!CvYp@V3J;;t#Q2nc|=%VVFA@iR)lJh zkHC-qos5Kj)QZh901^;|T%8=qt!CtD-ro=_#QPgvD-tzrQo7*5$8y*jT%TTv z9O>NiI)bOn3-Zb5sROP@AK&0hpVv=p0hQXke7e&yA=F+1jys*HRBE$~0a;;Y7dO+m zDJ$JMD>Pq`D4Is69dKKrVF8U(qDVLUg@`jFASgStZE3TQv-*an*#6X9h5Z+AEMnS73UgLs9PkZmOUfRB$BmJ8xIxHqN7&2t zLhlX#SGi}(5o94uSN~LaKJwOT6h3}_s?F2f8_TJnfI7shvlAl8XStOD1-6OdYAzLpx zUCnSCyR=m+N*aEQcedir|A9g2f|o*xexfaI3IF>TRKnEG)XC7r^nd*zmInWk!F%n^ z@}Nh~5C;YsMGFC!k(@^;rGXkaYa#hth%(bv65sIDgn3E^(t?U!<6^!eGLJ5AScRrd zh*1ft5~5kF+CovQIZS%%T&>xz_+<0NjF}~!5X^5X+k2+_rRv4!q|5x@%Pl+*ML0kA za))A!_5veF>)kC5zTe~^H~#WNozUQwUC~~Q%O^aJXPdG}B@Pz)(F^vq6A6mD^(!dK zjp2TYi_J(k?`G+qAIJLv@V57Cw2cj*@JJ1xov}JP)D|1x=|E$%|IjUmugaL``#Zjs zgE>^fqeWz}klXG4kqOU#{;mz@&Exlqe8Vn0T<2!dqcSgV^&Xbm)eLsd(p?r#=`Mqf zVrR^WYggYw@3}|(@a6JdLQJaXI|rV($3^mwyt(`Am51H(H8f-eY(6uCr2kkldItv* z#zc+&nf@t-UBc(zyp%pCvl?Ob@(SXuc!o%cpzZu~+eC?iNIA!8wYEtLgfCn%a3h&y zmeiLlTn~3`@1hPmZ2DeZEtP_nWqipHC9TsXSb_oVqc;f>i^BB_dmQ}4CKt$B^hwC{ ztci0zgTe9&@`reB-&!aQzX`HAvY4`V*RWK?^m)@}LBgfpbkF;FfWK$dF7^q`D=`Y# z#C<1<8Xl|2wxqJ;dg5hoyEC*IG$%RRQsRe~dIFp3bTrJ=EbI-4fMNqpret~gc3kWV z`y7(yA<0}J!f3~Nxmmyx<2d85Tx3i)N$AAhKcq5^#)NT+W%Kt2D%EIw_kD97#3?8X z*i({^WKq)$r;eB4L9u>lCT|0Euz{O&FX{hQrCF#!4*vrD>^W`b($?Ua0Ab9vSv*U zyVMuLWL>V|NABfzy9$$6%Ri^vuX7rz{=(kF$vJsRF5Q)-%GQ>{TxV|td$`QjZ~z$O zfp{hSfJO9f9PUF{cD&v)7!_zYCY^&vPIrVwDe6ZkM!d6J-R2Gn z(J1UVNHO`|C+d^%a96fpj8_~cN)#Z#oMpygGXWWiDaSRg$^yVyXf~5el;I_*#prW~ z+pIG)L|o02nG)hs(Lxi`F zE}?qbv<3y>E(bFqo)72(mAy}KLG2Il0vBQ#zu>we&Ac)s`bPSCXKi-*x@z}8K?(Lr zK)**uLCo>^`dDwF_}Ol;_&II~a(n|B;CzM){zcY;zq=>8LiF^vLi7y2sO(Klz*Y^I zLi`I|f`CVq+dU*j^NkkrSLv0yabi)?!ncA(d|w*v%c%28;6Sv9IGlh^%aoT1M#NN! z+t`n!oj6Jwt8R57=Adn!nWoSLNA=Ce6h?y=;S82GP?7hd0!4augGqtH^@r?^4TK3M zXMCzNc9e@!5{ouk4Uq{9MD-jXW&X>L`Kv5sRt(#lI~7{gFS&J&c9Ja!XD8vynF^LT zn|wG)=E1qTZJu2bu7V@btak?yt00x!3x`At>ohq7K7vW9=-Ep0m*w&hqhT4KHn~ngh z+atLck_Gk{YrWRcsYFE@*|kUQtSM^65t>bp5gyDyu3yVBowvlS`qb4dyJdTRhyg$< zyKbp^31S;9QD0P_OE?Uh#Q^Q@VgZME9bnPQpq?%+@h5&l+q>mDPB7CSBNcl*Wm_)a zr^LB@eD~E`TPw9-(%CkMBNWdg9ZtMlPjYcXP2US^WlA%sR4z#9E5Yw<-XMMLkCeFo zT43;(n|P+72$rVioakUxLBExpvE@TfUaF(on2$hR%cjo1N4PRpWuKJBQaEcoucN*A z+)W{P1s>3^kS@pFcngwJGF%Y)TRORZ(k!6uMbXmq$&@OX$2k7np2C-$45TJ1#VZE} zI}Diw>L9!q5<%?(UX7Y1CU9PT=%a<(9pSeGxX1`{wa<_ri_psy3 z%IR+CvFBDay$!)ZRTYCHDXPv>R<|9~<0IHKsK6NMm9PwIzc2^Oy>3n87`Ej^x~X*G z$zO-pjdm$&hSQ+pox>-qGMBR+c=l9{LvBgjjhqwCIb2xJ-r89c#UK+gNJ;7BHmraYQT&yt+gc-#09;bDS4IYFC zF5cMfEV?ry$*ET*<>8f3XB@fuADfQ}ouX8BhePcA^_3OlVBWPriX(pMcK#P6{b3NpsA-1%?P&<d=7X`|4-t z?bLDZF~E8d3P1W=a_piyO+3&SlA?jX(_!81aHeH?jNafW`*2T~y(BCo;k012fpU_` zA4dwG%&lniRuiA@#}M(`C6z9%ipDsY>Jhb7sb}Y8*UwSxYv1uonA^N_8#;G1d4Ip} zH8RWNdC8ZSX=dG2DZ?4WV=2Qf)8XM z6MSF^-cXn?NRQp}9`{^kkZ>%)fttj>8ixs|P(0HZp!d*7>II`jvMEEUD(V!bMu_Jm z_3P+vBQxhH-HgILAc*ep2c*cgl9XtbNA`qGk?GZ}9Kw|%rIS51inC%TQ3>ePLE41d zA=Ifk_Nh%lYgDfXTqn$QYA;7+Cw^&Eo9&`E3oMt7?y5a$Y?iNsiFa2`p?Ep0|liwA3GPO~BF{*rrXkq#nAtQXCbHDg@ ziyXlp%t{*ofP4pv85Cyi6G}M*WYE(S6EMq?L>4q{DPn7uYO)TC;8r`6gnttYJ7TPv zj%~4;a;buBFOB0*m-w3Q%#I^u?hP7Qn50k3^HEL2(n>d4M%sR{HZXz;9N}o>%LNfc ztNc={C?yeOMe|N_pA+T4U9Id2CY;Nf;H?Lewm2S;VKdN{dm5+_a!amvMVKpRRK#^5 z&-HtRvcwotg4B%6(gC|p3=WITkw$Kv+&4p`DV<)WR+A1F&&^{cXPHY#=gs&X-shxa zpr0dH+Ronr9eLSdz2zo-e&BKfHBL0jjaFGtuPLv28~mDX8ZAko&t+L#e99E1A$@e0 zQ;>~07m`?_`Awlde6ClIyK;x++EkcJQWKR@kM)I1B7I7MaH}UE(IkvMT0*kHC?%zN zlxNgA=qO)hi7C4W#wQ9VuUwZ~T(ZaIamyQV3qtl_$NWyF^w0EMUN2vvSE_$YmdH~U zn>#`VQwVCB=%}TD%*+Xf>~m&(m)47w##DPRKgCc;SljT*Q5%WiHqxOU@Agg9SOT6L zBf9}TJ!*BWu<_9wD|}qK6j3mD9V^(Td?3?g%4K&`uv!e-0AeSfIUT+EnmW-z8AvtdA!UHrRCFmZdP1 zD1;hy*GQXD=!dr{wubV$NgXWQSW5ZfqNgFa**PY>&tlemUmvcNRR!?boq|8FJFUe_ z1Y2j)G|;No#dM0e>R>FcC==ZlxTRle?!pX6+-bD*A>igO`ujj)1uHuYqA zRcPGJ`IcKISIrH4H=+p#h1?ME7T~; zy^N()v&So*`N}Fdiq)3VtImOxmKLHb*mPB@0ZLD1RV_&FsEUVJZ_|lEZAOd$EaXyq zm5ATkhiwo~$QY=P6mgX#9+|>GZtXNz{|E4DXj#|${s{zrLisT_|2Js_fZY%9lCd{5 z`CrKE!d*oj?K?XiV8+~z$Q+C)*=O*FlqImf2~|1(n#ocmGzeKLQ>`y^imd(2udK{YVPMQ$2Zu;&8D|Gvj~I;2o23i z{S3eN&BI1555f1#{s)jC~fuccQt#Mp_ogu&LM+caGKRM^p@rvo8gq+=iAutXtVtN~@i#l17elmjfP z*hlk0i#QPMTOG)m$u}ONE<%Q0%mL~9`$?iM; z@RhqlsjfhR$vqiFdx*P~00CooC>sFgv7pX$+1DQ}-a>?-h4nodDcMowk$~;mlKu7g zsB+B|yP|_m8Dq(d){TcGwZL7W!9@>EgF%}F8M@;2u3^)H&OyeFaR5|u3X{>Zb@sBG ztDqn+r9zLU{Kd0c;Fqj=;#&5c%o6*Q?q<_pX-D%1y+Fat_(iY^WG}4?Y^>{rDLyOx zh$P8@-RbU%^2u%q*wsHjV?IZ@ckTttX&Fqb1bH|~_Q0i;N9z4q`yvZ8p`Q%yQ^-JD zX9x8GGTF{a6m6!wHNGx^D(EJ5wwWU~`tfcVfN<%@WxR}(GeD^8O7O(N^;C;GeW8$( zv8h;Pavsvqs7DIN9J&;5@0sLFOZ;B>8H9f(QUw|8I_oREak$N{nE+c+`RCe0m{_xb zwoTRjVpBF)s@imESZA=s&~R2=o45CNm3ALuu^OhNYezR*S+Uhaeap#E(v$dvj%{0f>5{^V!9VKscWbyZ z8+hc;jD%XdUNJQMgGqfk6DPs}WfE*G+lJVd+R_vcXrg&Bn8*OSwOs&%Bo)+o(C?kS z-+l9yqJ_UB(@c2ntrTWM{gz`(=v_4uH+H0LhD4yf>Ov#SOuj%dKGMqJ=yMXTh_}rf zC1Z;;oJ1RcBREinA)tLC5rE%OfTuE{4dtSJf(L!qzH022APdpYy5R{POC|Ni;HX!E zv$G{jc3WG9Hr&=9IuKj3vArI>%=t<2FT|MB8(*XAiprwxO>e{tty2bz5;3PXg;ZnpA~OjWM`+JT(8E%o|B~;SBq;D$?GhC z5YIF?8?_LYlqG-Qwkv`-J7#7pu1xv-O*J=SB?VUV0H?`LE&T^&?csyog;-|VL?ak% zIYoh`v0Cn>TQH+oEKd)_&$WiQT+jQ%|1;T&+jh0$MNCz7Q`oa7FHEqNldngD1a_&t z7U(3Y$jB{}qG+RiUrBjLZ}y&qrM#->PsLI(Uj0#LcKK`Opf{GU4xB1L}l zrNB;$YvjdUJC}Qo(m2@+b5?nu2^4`7i$~r8W!8WNF8F*}>PCeU(d14|MzV>nB~Mx% zmqXa8&lW4_^+)nZu~G*D0<~!h@e!mP{(Eyr<$>N}{^L42KbU5v?9HVI(nvB2AEV0$ ze=*P?%8Xt$mY8B=j|z>CKxCO<2tEaa0DouQe9y&BrSt5PCC_#6=5Jrvr{R$k%<;Xv z#1}aN5n$NggoA=a>ywy394ufRXP>{qYvgJa-_J--PEHF`Pm4KD^Eu86ogXJSzwVKI z?>W4V2p-4%9b#{dLlx?zX@|Rw*sh(saEoW~k85wh-h7iffO#Alx?km+I9Z9lPKNvJ z{z)UO+~pF0=pl}9O_n+6aDD|ig|1#uD(XPUN9CP&+lAe3o`B6qtcI5HMKgb|S)flE zr7BFG!Tt_p0{b+75XPAB81O4_3MijFfvj}|KRj>&za9luQL?aQU~)#gCZQ{Jc(bI< z5Y@FW>fwI;3#R5H*gaY9f8>%NItCA`ut$h)4EY3B*)$(|`$Us;pE>)-qWhQP*wZsc z3Bh4O61^Z*lEfKMk0Qu^1Pu{B9h4)@B68}Nx)s0LJ#+8B&B=X4;8aK-_HYRCs|v7S z+U$XewAR!O+8)gIbP@Du+?yFZF*WDW2L?1;vE{LFL=GhI9W2@~u?#X(SXHLj46!61 zrqu3XL-&ebSe)kdDN}nICfSpSFPJ^N!LCjn0oq{n2HkysQ()RucTug{+GKZ;*b>?4_NIP%6~ zyuM)T4P8uiwQ7bj9H99~Nzs?mqwJExD-C+XW&nr8!lupehKalewX=)H5j^j@zbVU| z5x8X~jnGp7`JM{1J@ou;_r%un9RB`4RIN?P184^~iPeo>ACTzIeFU1k(30B1L z@EX~_?K6BQnTg ztk%QRMlV-cn(;YwY0+<>TjuU5|1Ck|oLVAI;N#(TP@h>m*E8nW86ii1HO}-SDO;bxP08h*PBVf=OMML} zQJp%dhn33ywkJm`9II>#uTMD}4u}w9r(zeuL_Fp0W#V>JU3rmQ5+wIFTPUu^>(rv) zTKJ}xFu1lk8Qe(dJk`j;NT8XToxaGwz@aPBp-Ae?I@auBYd*2Ut%UzR zSk!JH*~=vtW+0T8iNPgtdrfu1YSrMGcKpkUYBH8X?J3W*SNaLAR4gBZE5aqr3kF36 zHjY>}hv*WLM}z*`?u{m2GV!mom=&?5T7j%fMzRLV)-lr1-VJ^cA{kGNoDsfwia1Id z8DD%#39aFC;^_&!QZg8vZ}6l1e`UVgKm;!Op@D#mae;u?|9{l^f40uen*SL&bwlgf z$QvgL0-J^mMua0SR0kgh`I{UMk}M2F*q4-HuiwNhE1giWNvphCt#U~VhMv+&85Io; zLIkx?&8AxSvS917K=-m@t6KNee_Mup<8Tt>*0=qp$9d*!#%YG@rE4$w8;=)k*9G@( z_kkaE7oK6_4jk(`X)lsLE98p*vQL+Jldj3zq-`aZ(9NoC<`>?}2NO)^rw0(i#BE#% zgQW#p8^Z@9R;Q`QYGl2K=pa5^>^md;%jCT=v-eMu{k@Mc=XLxZ04wL-ag}21gRKZ` z-wzAk!b3|W^Vcumv=BFfl6@?=nfI~zJc*!)VU_~evJFLBAGhRb6 z5)8HM(!2^ZaoQ+xWB^Hmj%84$EXVveE_6@Hc^R4}Iy>Jp8%1}@8BnpD=%#ARd>c8Pv%Yc}yH6X#783oMV(%~^5dMnwQrbh2uE(VE4`$>TN} z6GLo#J~OUu(|MFbaXT402a1u9DW$AdQgK*~^RXg=f1>UCe%+kp7M``5GTTN58XZ#w zdtO`{7JCBNA*%|>M~RD9@~)?Iby#v%DIzTvv|NTsrACKRe>0<&ys8|eCX%zI^MV&k z&wvI2c`+d>*^bm7^WhiRx#UfQxdbc`wXFA!Ad$T(L+Z*p<2E;VVzZci(&Z|20WS_W z@N#2gC80*-=RKOJvoip`??~nip1_|NWl4hOOdT@S;wg|^v~V_xO5&$ubTfan#@;y@ zw;42bK)Fd|?qQ7&S7*^<+0T1un~oO-T#@jcaMd0>=dfCp*nQYl)Z5%XT11bk5T9d` z;UlGg;e_5w^J~z3Q)+#czhdJl<2{zs0QO%0in9Z@Ut?xHvswb*Qo+0Gq87;3xXHdecpQE*T$&g1r{RfeodXye$% ztfWWMwTpb6G5c54&Fi0W;1P9I9PrZ4D@MnjwgIVf_XOHoy{5$2AF$!=1Qs83N}iSt zzqt4i)GjpYa%bVh;?No82zKWm{Cdp~Avk+w@+&xy{ZujB2coF1a~LD)=9NAZovVj3 z#QWL3==up_f98SHt5U}AHa-xseRQ;LvKhfjZ_zUFL@xFX-Mf5;(=~Vs4dME!rsp5J zbMXp#9I5}W>_FZlm2Qi1Y3>gHUzNi@#aBzJ&J|H|w6)aZ4h$AXqQc=Qb@~_yEI|Cb zjjKXhpM%Now5Q=Mtt!|ejW-ND|KS}}IUK8t{DYg<&=~)HU$}3ju^W_WZvHsuFOKb{ zTY}(tK9tB}-pX{;zD#OIWiFBonuRa!a1p-)YN5l(zpt**msp z0=8S472CFL+fK!HDzYN+fQuURwbS3qj%5j{b}AGabsQATIXU^AG>4Z)tUBu z)p%DIjU6|TO{Q(*_#LjC#zq={7VEJl5P0d5c{wsF-9o9u2)p#L?baf4&z?@#oKW<} z7nntgzZHfLIvzeHFTQCO!H!Jdey;(aM{U-X3Y(U2hTVjns1O7Ei#}?UD-9vo=}3Z_ zc9)4{Kj&)cFKZ8MsbSrJo1!iWj!xHXvv-t#4~)T}BJX4P1Or z36qPx)6-5b$LYay(Ikskz$kELdIn0KgaNFYV`ReC^;DDRF$&LObkWJQfaveKE}5BK zn+unU^k3G579cfnV{z~y5<)6>&x5L)kb|>;#&}ltGz(k_D&6biC6w?3KI=h`mkLCJ zo(A?;Jjmo;*w*z8c%LPpC=I3Ukod>TNA|3|I=l}~tm3(V&pevlb3q|!3h*tUmq|>; zlIKCIK`yfsv}3SMs;KQ?E8uJ4j(Wzj@;r*XtHHFvw4Ipx%5|m~U|_7cJ1QAU&hWp} z17^@-%>UwGQA{ZL{iTHDX(CJ-r<1L^iFYjq$KiFF=;WL6*Va9YDyr}jsm!61D^D#Z zU*)t%z;z4U`jQ8zswK8tY{p8IL4a{N=4{Zna7H9qDpij{*un6x#?SMK%MBY)CF1#*f5{D!i?szN4~yH5fobkr5Ddn99L)P;+C+nE@_br%cDp+T>C+Gw6b`bnMZN z*OrWrpGoXm1~k)7R>L<${fDy97eT293#2ld_F%!FO!%!$fY%?=n(@B;Luch`XJ=k; zk!vBE;HRKHLGV3Zvl{;efBCnjkFpjo(XXfU(SF4s( zOOUrcbG{ghd|QHZ+GG5^w~ZR$<1O@36!MuF^67eWy*GBP{}LEu^htXGkgNvq9eScx zyTelP#tgax*X~Po1(agme_(1*9rXjLhW$M!ZK@o17oYBHwB*XgIN)jq$FrozXrF^z!vTP(AZv1I8;qi4u$-9{GP7 zX_T?BvG}hziJFZQit10@OoA!E7?4tiXI5dtnl{&IBLKI^3S;{)ye`np2+Wp<|u ze3Bsf#{%_$vo<7l$(|@UlCroiN}jcfGWa((|I)1{aB8AtewHY5B{sDbb#3`xgkYC$ z^Lh#cAv-8J@0pfqCB5#f<0Z~TCGc5;kynq!hfCv9GRdhAF>jmoT1qdm!h|Ybt3?`K zV?9N5%o3}$M<;;UrtpZhI%JJmXN}nh*irK;yydN}DZUnFXkw-Y6|+{WW%IJdt;lO@ z(?T(!oLi+TlQrx^0rTO7R{oCz=16s4KQaOhTEIiCc}<^WB}Rp)^1H!eK|Ak3%I$lg z2EDyi5;YrglPp@^=WSB=V8mhrk1nu!$Y|Sgsmm(U*7bTl_MW+s1ks8f& zy;Fmf*-hpbEC;&HbIzDbZf19pQEqnUtiBWM3hti1k`Ow)jaQ|-PXSQ1QK~HTwtH-a zOj=L|*qpurBGSq&i+?c6Y&X&?8cui{sn50oxxj#A9=x^NqwCAQVK^dH+M2Vny^g-M z{7H#Xsi})Lpl$!i_6q36eiYflV+HP6{A+Yibl$U{uwp(nrcs}IpzE7mZRWFUo zjhfA5xHFHR0)R=R8Y>2H-JJrDl+yRf)kb7?2w<45WbIi5P;09bbY;<}@sl72-C%QL zeG{2K7G)#YEnHx6CB6q6d5^T;HGNC!r*pGscyF0$2fs=3-vxTBsB21Y$;~{mBq@+i z-Z{6AImM*l6o)uVVXzdv!z2x!feLy-D*(8oub7}q?w!t42YA<}Aez`L?i}t*MVd{G zLCTd}N@8uty0B1De-NTjeZ?L}`R#cTbD)1c=K4y9cX8(CTsCMQKXS2nLu% z01X(|#lFZ!K!ntxRTP}C1iSErLBSOe)t5?^otY^fu`0O-xr5G2%n`dsi8d?n_ML(Z z0VngHp&BF$VDlG^!$eIJ|KpWlNdb>y6ebxl7l!x9P}rp(;TqOQx(Pk@7D->orQ#R5 z$0zrdIKPPUrM&})6n)e3i#OQ&*MIZQfNE05d#Atsgn5gB7Sel8SYCgb!N+5>wnV5LD{jz<7x08vn?Y?jE%Z$o?@4&WcA0;_0*s8c)F5{xOT_^*mxP3f2 z(;*Tkcc_}0|LAza##UtyKy9n;5E_AJt8yS`!GAEIyRbtyZMKCO;1$$=Qq-;vf9+g| zpPM@G!~7n|s$V7gt25}(H=RFkU+8T7mgur(V9;H(Bz7X7j~YMm>o@WI0rhjoo|rsZ z3DJ=_!*hE*fqdDi)1DN^4>imdw@(P^ATENT^EL{HxqtOaO6Aw=5^=`DncFr8sf8$)%@BM;s{Vv8pxf zB1l`(BXD5K_bn$}PQZxw%@VXE#zQCcyI6F26jjfX17X_Y3IPp5{dGlHmtjGzCN(u_ zX5und0f7|QF)-tCLIQ7pY1;T+h7fIZBygI<{tlT0<(ARQyV|ktTRlb_)GWRi9=6EE zW-`k&q-g4r3-Ny#51k`MqAWl=K$FY}5l8He^aNM5BMK%JBF8{Iy9)F~b5K1rBM!xO?9iqTCdQ`4S?-uaaQF*C-h=90mf)OPia5{xDb zTndKfurtU`7JxmG-dr7IB%6-EE1VOP{Gd1OoIO6er8~uT@&3<*CxnE$r)CewT~TOD z(J>~(Zl4TpXP^}=ibrkPfS<2sXMPrIJ^s&3eg{xaWpx`C6rD_l0l9|aJ~rG=zZ)8> zC$jFmG3O~#jCdWHKIl(;I4^c%`^5M*8sT1Z(-{1|3kl)4%Lr^yj$=jHgih)4q0$$!`YW(g8LI_s zB3fq!6n+yvDph+HC9}cRNc<#hns{a;%w$V5rH~1_uV#A6&5}^}7-n*wLUeLS<%lWI zAX%u{6V3JRV!bK)AYD?RaS|+>&2V}K7aa^)haD%=VlMT5o#p9jjMN7j3C)LY)KObfKzo@AHM2C&RaS^^9NQogUk)2D+wkalgcyy2 z;I7|h+#%`c$V_3U2WEs*7fF)-_jWwug?dUQGaxJ(W3QruVdR5+)*5=w7vhdA0U z!B=q~O?ndanL4h|x&ke;KB>!whodXY4=SYG(zvfJb;jGwKC>T&t^&QBrB~i;e zBrRmqnEWsy({Lp#^xGf} z_T}}1(a?vWV&$cPTsBkK=E{O9D`#K~dnK_O)1@z^2#mjA@6;KbdNpVCO<0CBL2v(r z#FHf~=aePFnS~MSrr-19GNZ{9=QVtj_e@Q zrxhqo-QH-+E&SnG+eZTxkM3?vz~A|-loz9@s^>yAQ|8T3md{Sc{0Hs_|)Z z-2~+d!LSC|C3*~&f}RAa&)vBIis;=9!)UCi)wU#hQ0K}NM<~#bnjkRvg0mBB!yc#Q zKu;&+FP1fH%;Dr$Mu4j|ga+ixV~?=(=bxcrLII$oRcy zw;p;nnemcuqA<2N^Cxd?9Cb!|?^0%wcVjWq37*o+-3LyjH@QDP6Sn5){0TM=$DZEy z@LE;|f&-;b+s36E)e9WYl_lA z+r85#{NUiof*sn%s2Hh(GFxhd18-o8N0DxQIY;}zQ-xUwAT&iPe1i$Opv8lDU7+W7 z;Jo?Qf%Qz#KYA8?)t$87Bu*1mN4gPJnwMLD|06=WD;fV#(i_67-hw9E54||28H9!M zw8~`r7}azq{C;5zc*p?OuBG$7VQWUZG5hw|KBe1%ntF}jpPDz!l2=rIXPWjOW4>Qd z$l71i%VFyx>u}z#<_w+3T(fbZJ>i66HOaNDMqHNca)Kt?F zzq#Bq=WG&#zEwDxQIhP&AKLs?_e%Wdih#g3^e!^L{BA{vp3d(An3ba|aa@jrwq!v) z5!gcPg0^Y{ShxMQ%02vRE6QEv=#&Eo8A#_PnOVKo5>pCUbwX7grWM&j%Vp}3tdsR- zX+1(U4qc~aB75)rs{_SOQ!gW6<+FkVHP}YmlvKJxTlJ`nC)w*9^~oj_YdIX|CswV?cfi#+5e^^9ZG%9z#pIN+(TXYIR)EQRR^xF> zHI#NjnO7(>Msc;Ep~SKSFoJWDy^aJy3=}Wty|Gzl2C~N%=+1U-RQ|s_9Ycnri{LefWn61roZS7^P5DZF%Aygp(iPJFrhL;WI0m zK+5YE8*=ZFuLNsn^r0;l71$H?WZMY+CgD^j^^!k9k>&v5OoK)}!P2Ind^>?ZK zU@E4F2wWSdz~q<7Cmq9U}ldoYyxx!j$8!?ss-6k8`prsc)6MHC?-xO zdjI>4LlY?42l}I?Z?XP!HLYyt_+Nv!|Fe$PH}yy#Cm}_uQ|$B4mm}{ZyoM?u0)@jv zVMUhKvLZ;AaOY0PI+4|=TGC2g@21xHt5&V6M2B8y%D5k>+tl2tTeVfSb!n*Gx$^vB z`Tek+Da8x}+CRJLcGG^6^N{N_>v(dI{>^vK>7RJFBfW_%%4Q4q2dZVVP9$W*__geY z?L|Oyi+->UmZ#Dl;<{>J)IK$$^XQR}n!jd$Tl&YdspZWahV4x)DYqbiH$)o+`Cz89 zi6P2&T{~b_|2qUC=#vYrQ}q&RA2dqEu-2O(Y43rV7j;M{3Os(e=NJA+T7-nkB|-sI zEch)Ze(GL0p)hCO{x~7}0StA|k2wl;DD^IBDhGa_X6CYznU`d6#rU-&aH5~RvX>95 z6?IXR0l;w)pC5%4D=JF^s34p(f+u~PQdn20%lqS%-WYQc9xxVR%xrrazI^0pT1SyS zQ-v-4NK+O|4gf*oD}J$KCZ=*CBtFbZnwRHysRNvkpIU%P=K||*;W&C)|2IHBnimo=#gc zTf8iYtfjN>dxo$(U|*r<&ePV~kJ-o$lY?cjjW1Q?M21My_(e978*Qdc?BROp=I~&z za1?9so<_?MFfu0&R?#o2WC2-4hDlJomuLr{$3mf5Vx$4JgQpg@#vPaoB9XB@f=&9z zhoy%DoxICa8@Qc$*VaeCheiaXaMM&&mU&j26lF{D z(G)sRNz3j`#JrZVmPzUA+$8s)*{1@tk-)80$&k^^#v0WDV<89CilOcfmO*yQDqa|$ zjPjkC;fFo4$`U!4pir5Z5uI9<7I2XkD_L?Fj+@c3a!OCWe+{MOWjmb9`oWRfn z^bFSk>GJ=hsfXoRKT(-}>H=Y~ePb{*C1VMN&~UZWd}uhd$O?+qD zQh(ETzoAIB2)VVeSwDe$llIiHf6TJx?ohLpuhk(tdnW%|Q0tWLw%rre=CzcKTfDLu zQtQp%s(;E3?wFkR2|%{U6enG|K&Rkd56X)W(5v0a!+j4#VsD51R{hi#>E(QN9AbOi zHH53(7l76h9(J*mp5;7)r;rPn_t37E!eJ`RO-+<(M9O9k23Gidt;mrvASw9+#v+m} zi(-AcO_(qEtVu?Dfx6f;jeGGmN0tDqEqi}(=M;Nw7byC#B9X_`k0_-jr*hs zq2>}(z`?ZKnzIS@<3-TGopBZR>`=|&Xo9cftuedkYw{|_&a{P!^SiF0`ZBSl9O0o3 zbm^COS=;2U6kMD-g8#<)>lLJ#?CX8H;v+;2ZWk~FFw`FpopfRQJb8_c4$PM z8EVe$fTH_gr1&1pikHM5!mK}cM4#Thhduvjy8;*eVcRck^nCP4CK#kC@(vUSX6|;c z3KERUJMzAqyPpvd^96AMwAb=<`e;ryktYNFp?6B>idYj@jme=SZCe<##(?f8QjKT{ z!zW^wmo+%hX=`}tx*%(4KiEE`=5H;A<#yraTbhW^JyUFcThqCcobg7pwFKNhUtLnZ zU2Cpa!5(zNn(1!Oo-^;#!cS+LB&Y7G*ucWheL6+E`^6OOW! z!30$8rNd1A!pfr%yCYBY(8TAF`?rU(DU(KNMQ8Gq67Pc5s9=0|70*#YY^Smt09^Pk zjm?(!B1w}d`fPc`_~7`#sB#0@CAQt}9GJJ>McC7`CAq%uNg*uMU#dWr%O{Weak&lJ+}m}e>8g$iY_4d z14&DtAdQOSbP~vBf|xg|@2@$2z_$?XW~dl^#RcEQnAO9J(Vgc4vDZDAF&GZWDJDUS z0d`H1$WDN%PH?e1h0(QTOO%yHP)+gVPBdOyPE0${%bqxIMCPs6eQzMbDn?yAyv+zo zk|P;j4=RH4jB3()#8%ScKw5k1drb*z^2x?6{X|t!(r}U&9&lO=2Gu_;o;5v}+WFC3 zm`}}#%FK||JN!-$Ng^x4)WW@zu8{w z0pT!hSgLKoJw3^?&OCIhLX&ohe)>dT8*2lhXT#Y|@9?g%Jer?34c*i!%W>~QU#hE` zsW7*gKNx6v=v!#-p=A%2uRj#hV0=kOc;f=GLAR++7vKK_@I$s+sz~^=(T)9y3jL=W zU1fXg|GUvuwUGxXp!!~eU#OiYK|P?Md2etR149oeDB?;t_&bD%A`2uru+XQpwBVh} z&+iBpkVgoi==Z$#6PWP2RXSnD)2$c196fHHaGkWjZ2EqDeUbPRZHAK+3Jb9ZkUFm& zcGv@9bTUGU^pm~WNcPtS#&gnFDmaLV0T)7#s5tgaAZDVrK-RDrGmDK@VwPB?sJHD! zL>N54tuWgdZ6D=eac=Wv95%wh;sIuu%~thM*>fvv=p1eW+KuDMRGlYEn*VqT&xjaF z)De4!#Bc4=l@gmL?>S7u1{neBIQ1RrkPa>CoeelQl&oRHp%J7H8%>!HS=L-BFG}7G zh0jr2wBazJuZ%YBG?F+%xr(KkEph~gf|L6>Y-ebe> z)dfnyt=Q}K!@()HjWhluU|q5YmB70G1X;wzGZI*gHhASAtv}WhFo-y2xyEA2F@o6C zv$ZB0tRGNLO$F)krWUnZho%~&g9EfpGcG>wttj-t4vLmg4zrE58zuqEuO@p>24g=A zYGHndN^@=j>=eIC^^yR-B>BES&>{-o#Twjd@ptWpYKS~7%~TKY=I%R< za#qePSVJucQy^GTrhS2rUtx{eCH!gP*_No3%L^4h>mcy72wD%Dh*C(af_f*PXD>_D z3;6<{uLYIfqERcV5!n$p1D&@SB7T)otvqQ#^oul~E*A1cWwL|_dt_zJD$bjorv<@V zDS1I`-bUfX3V8^#ZYFiqk8=+IA;{yGad8>d@A>;o^@0WE1M}f) zl11z`Coe-3i2pv1H)@Y!p-=L_pGCZ%T?B$^pZ(}tCdB=M1`*ErhMOU0JWhAH6J^W3 znoz{c7(pL}H0h3_+|i#jf`x2A0cgh_c>5nVipKgWEy14yr-1zbtJ3|yfjR&8p_6xU zws-k2O{f0hg`$e>E1zJxU`*Mx7y+XTgDRQW5)an0-cYxtl}FZOo*&WB0Wi*h50Off zvFyC|nOE)F0b}UA_+f7e6$cMIRK3FpTsHav8g9~Ii7XBPFir9^S#L9LJr^H7CsTf3 zZcjVETK2Vm^+FMY%Jxel>XBB=2NmHub45)Mp$%Y0A;#=cfmw|E@IsoWEv3i>G{-5o|`-y`Gd(s3F5hXNV%6P9X)FJV*8J( zcSfsjcqTI;X?~W*AK(0_Gc%S}?;fzIT_lO5i<-uf0MTT<%%}`Zb1>TEK$p@Q?Hnaa zua1C4rQG-BE>g4nvljdz5Lw zdxF-P(GV+-kwf(aBNSu6c5$V5n#%hEKbUDxu|$9 zET%=$5zI^LJchiVO?f)eC-SSAwC%7HSXo-i8Ott9pk} zolTzrvmGg_-hs_fzbXxLWvM#g>qA|3!-Yb)YaHtALVsGvkRG_R`zXukWG+^(;?o%Y z@V#i46Z*8CsrU>o#!C{mFC&sHI`j64d&nbFJccM=#fJ%dR2`rI)TZ{D{b&J8XpH5C z=meE*)QN*4k7}nO){}9Fx_NaGKFhgI8#@W`BV2T@fPzIRYaLJ=1ZVVDp=WEpeikC5 z`jJ+;g4QGz+s(1@gN~A{f&J1yEg|;$zoq)XC1RLLb$?H4m490+O6kr%8#g2d8%+Xr z@|p~wML}cQ?wDavYcp^moF6N%S&TMq`tuv`*u+IInsGBc?M2K`9%ymbBE%r`UB)`$ zpl2##m4FIE73i`|ZTjlE&fa>bE1eum{}gmr0#sONW$mX-{H{Bqe*<;W3{qz^;DXE; zc=&;7qg&{T1P7ia4lA(HM{52#Nw~H!X%(e1eX-H=2OWR`?osHQ8i#fy&2fY`7&N_p zraOqy+(Rc4gK%E63^ATxOfg6 zfLX!S8>_ExJzfNR-#IyWaxR%2rWZot2)9j=x|b=wv@#Q7gKISgY8?`2>Z@7Hof5+R$|Qe>EOp;Jg(|M&W}-8CSDas??aHj;Qof{;1V5}~UR(({&2`0f;T@q!a8eio}L zErdATOo1gb85ly#tX}%IByP7z&0b>b*GlC&y}=XO;_N;>0!HTgggKnj{T7P`OZ1ZR zD(x9wPYX?3=uJX>zeEf1i+q>6@%aeW=uiP5dx{IwBDh@zYX=+L5(LjWKXT*lH?A;; z(kq%=L2B-}*Dd$6A@qkm`bRdQ&5s2#KR#;x)%|?xB3j;d#+p%HdFCIaM&rk#;fGB_Qq8hz zj~!tL&40sn)6S*FJE6rJsOlbN&#wjgNPX@ZrY7mN9{JpKV4$UKJ4=M?I=?pVgWPU8 z3{3aOIcOc3j8Z+4l3;|SCMz;Yci5~Kbbyu}TF>e=n|1DZ-`XlOvQaj>$IH8Ml~#v# z>th*fQF{T={(=iO^616uwKN%(P;Xuk-)e2R>|x$#?h@ZHxQ?UjE1b zVP`wX|GU_^_$h5b{oW!aCDOzX+y_BYP=qzFP^&h)0ueG4n)~Cex(&_Jn(iv{X+%}r{=o*ufA z0@Xc(RWU5l?!xR!>(Y=CRYW&uMH5MmLWZ6s#^sXI-Pl zi4X+b!g+|xVVG3M&1i*GprvWEjh6n_{A~vBph;+Z9+Es?a{-+BZ@uAKl>nr1?2jf1 z8BfJ4`VDhNG3LrqmoW1DK}Crc1!#=R#~a&`YxtBj##3NMYNUk-)dad&h%Y=(5mU$Y zMj}es<3*ye#b8|pmDvOUC4z}w9@XZ?d4F;oRfD^8D2%N*0~lhZz<#kM2R4Q=vM(N^ zNo{u`lDoq&g$wyfPLXz@%o{p_%@UCH5Od8}Nr@3&)bP9gvH~4?%9Jh=X|#!UPMxl? zp)iI~eYekAkLr%hgqr%~jj+yZZPg*jv7$Ud)fIOrs&8hA5oSomX;)2?KB2GSwvcQ% z&c@m@Bu2x{C5R_FvIZQBNaGy=VN{}&Ci_${v-nP17%*cdcaLXDYlD65Z=!02b zO0>%S&2j~JlA_2`?w&^UN=pM9^J4tdKfeE+x{cjgon;LjaGHN<`qU7wuE8eNG2I*s zMu~T=D2!lA>YT?CvVm;sKoNpKx;Y$*`iK^(IDlV7irXi_I`7KD*gG+7|jl~Ok5!dJyy zdYq=>V4Rw>Rpa$Pk%#XfJCx;gUP#qW;2$zv{ow!v3*z{t7~uQ^Of~q~xw%=QdzC)d zyPcD*6WoUtbO#uF&HcX6V=PwBzpHrbH_`z*Z|EVf1()>@Y+?(be{rG!Ue}r*k!|oLyUO~mIs9f;F8m$ z;}h|`)$oZe+%2Lkh9LR|ibBfA>-e+Aa(9$z2f7fi%g5wY`P;hX#{&F%-a?h+vvXlMpYk`CsgOkFy{B8tRhT&(NCBq!dT^!dYKpi_Aa4v zj9Drl7`skszoKj>#>EF%kfOFXh%7?gX_Bmor&tA)oCEW3kn)vnOXBA@EadC8_LD(i z_Nm|#-VfLOl;uX1iIcf7+F)XdVUGal2jb!*WU&`$aEIy2zv2_l0&7#t2p-D_ioYPd zml5_=1eLTurKTasUBZNqA)PigHCdNfV$uCLP$f+(i1QR@i^{f4s@fuj^=-yRxN?a& zK;S$=9gjJRT<~%Jg`AQqYmFe@ z*jOFL?ipqW4t2(G6Aj;`+uW3a-D#A00>v82V7%C~)BXmj&&M(Z0?n4q%a-kcS5~rx z)NE4s^?zu(N4_^w&!2}Q@2BnOKeYrXJ3E>f*!&l4MO*#n<@kb!YUd*4C=YAVst)lV z(QXcGu0}${77{bhU*{$rWz3M-pYm7mvilD5xr}6}QSDUwYqiwu>2a4o>vNiWs&D?0 zofrw}U{O$KI-Abo<@eEZayaYf_lDL-;H=;I0c^^PbK+Rk^rW15nNJuY% zID#g|uXdOSDYtK#Hlz~!5K)Bcr&Vo?xDq%PREH>pQ3c8Ez!!!iWQ?XI*JrOBqk3DS z1jB3woh5mkEwf3R<{g7kC9|2*3atrl2Idb9x$id+1)Gke+H8`v|6{R;`%6L~a zNGFk&Rq4`CE*6;0m>qEe*jW~kI3Hot_MV7`EHs-I5?x)3k$;hxbZn=|s`E-s4rK}M z7MW;oQw8)EJV~7J>LV=KsCVC)b6WZQaK}aY>A=`2=(Oc%kB^I;q}GV%;-?awMvT?o zbhaP8U{Yx8oA8qp$^K&5_U#_m zXD<@l9b>7tT++~3Q*CiDClSV@TXvp?8(2=CXAq}NmsS7mHq}y`IF?aElb&x}==p37 zT54%P!f)RBi?7ox+fXY_7Lqy3cKM!XvE+z25-LucWcD%q z^a%!TDlWtFWApN)_kfFIWM$+EUYt{r?LK}{xUQHj@aJ7PknUC6-7;d7G-GwvG)6Gr zenB7(I8J*wcMaCMAZ_rJ`T~3ixJy+_{$ZPd`Hm%>s%sg0m9`_;`ew@~VJvcs8c7QN z{X-l1^)t(P$VYev4C4r*t@e_}$+(HcSy=BF2KGRKQGX=KE`Ph^ea2g|9%M{Ek1G;J zOxcQvFciUAoS2AG_^Sv!&;7Fq=W6-l8jffEut8A~7#ZXbGo3e1sd2~c86Rc#4L_kE?xq9MsX;hPJWuPRR|3v??hGkC+)767o$f#e7e5dDyiywW1G3p-5QEHhh* zzVR~#8IRQTgnuJHvvd}0uZw6#!r=Fv%8pCMj^ETONL5D+{2u^f!n(-`_{d6R)pkYEZFm&lq;i(WHhVAZwiq zfjsu@CW1HFWP?N#$-tx*D?JU{NBiW%GNlXphWPJ>i+-gVo~NG~UH;FE?mra*sJ!l6+x9}uTr*zqm;^7oQYyjCXOX= zV{^F%G!^GJnNY}Z(6n!Fc^5MQ_JG~OT!Z7aOjNfOhYhD&|+-$vE zckgswOpX3v@NuBtp&FB_ zVQP00hde?S%xveP5esFkmp*ms% z+flgffUtX22<7l88|Faso|GWnQM<(>a2=o=)^?ZeOJ&&)ZI&64Bm-rakhd_>;xxk= zw_hC7x&qz?B8gw9$|{F}ty;YfE14G6m^80P;>%(}HZWpMoU(hd3R#O6$XQehw`a|k zZmQL}IqLQA9F)Xb)(iM!U)+`COFnq8*Ar!HskVN3Qqb+7(C0yjN0BG1NK*WAEG~b3 ztaxRdkQ8CgT|x%I28g==p0|j`)C|~aMah++fOQO(U?ZL>AcM*Vg;jEdmS&}nNMyCm^X1hdJTE$$#D)Nsy z-3tFfAuTIVfaz=YV{hpcjD9Djl~6Phe! z`78f(xjU{GYL@yPyKmlZD79}80ZrS_%*?l7ztla{qdw*c6n--lI==ueSMiR1TltRP zw`4aVBKH7;+BaMuYSG)UsUmI2B;fw&S^3WDQ(O#Hm)bXS61t}BRZz({v~mGGs zQXt|6wGV~-{B(mWO0S?xo!7Lk*|NK0D_A{C^Hy=J#`q;a;cdT-Y@_YpE~^nQkE7c; zPFG{7M=6-Cm724e^Ll4$dY)(OloxVMR`6zHdOKfs_mw(>i9U0Eh+zI4gY+5XQQ(Lf z;$XB)^@Mo?3Qfw37kYO8;Rw5ze$_cOW#eVeCU>2XdF)OK<6pIMW47bR*onnr ze`fK(#P@VnpeYaDRxLVfv6qAT@;k%o*z^#|P8&@A#qGeseKG^RT9V+P=T7S+=YiSI zxDid=At}ATHY*R9nJ5QCPUjV!f#Zs>*z1K}h-Bs`8*w27=;J}3D+UnaA5G+%4chX8 zud=v`K%+HaIK&R%X_rS44r$9O+wTNQ@YjS>5=aAZdC9^;j(5>GH+dm)`@j|*m8k7e zEbS&`lf<%sPFN?t!BgAVxxRDbjNmh++qA{^4H|4sCwdXoADZKw#^vmsCTUwlvugNi zEY-gj$$0TAT&@^+kw90^wY(ndy@uX`PoA8$T<1AzDxZ#YIR20 zKe%mVfw-4!I{mL5S7Urq<-iI>44lV&HD}r&=ej%JzCrOibLIF1TV3aiC8Rvyk$7WW zi{=7wbA~EcLmzmb;Q2#e-0zul_N{~|Y8-F4JMNJ9V`JaY?-+1L{N`zRPf?amF*cY8 z&mB})_tQATgqVU31_l88usKL(^K>(Nns?;QG%roc zvPb_ew#N3e%T4$iIw)$#x#tk?^h(8_bS*YyuXdQv{G9@!8m7={Z*dEEf`$lRk!?~x z%0#GUj`Tx4WD0}H*Q3|lvoRLOL%TXH-bOAG#bdD`P&^u9e{?z?!hn`akZ=r=yw3tj zNc589zj);d>G$r~)m$)ES_4hrZeok){nZj zNDQTYGG`w8BA|^9vc{kb3WFbFLNbC$Yi19L!agzDwjy~kW^cddBF(L{UQ-&@HR{T# zO_}`e4CXRRo&i#DdnD1YBFl`GufL zAs6FWMQz#g`b}oW$W|Z-hl%=Ak{%;<(pqGoq^l`jb(o%Z6RoUvt6@QqCN5W2JyWPB zE{9{J;m6A$yIrO&Y?^tgLdj;JT-zd*`|AHao=cUlZ;!Yl;S6e14>V4l20L`;Oh*Y+ z6*kr}gklZP{7o8mnR#Dfiv+eDo;b3$_Y`+jKAha_76jrMw=A1=?tN=EA5yS#vNfiZ z+Q}?Efe>ekTCO$i*FB{yQI#H#)T;hB1WJb`7CH5-`n3A2h$Dm%tDJx@xH?2h6cBtr z@Qv@B_ozXNd$JzFqp1#<&tnvf(*2es zxvL^MK(s6R1c6A9wRs{hhaX6v1iMfK=4gpS{3Y8PEZPKC%8UX%B+JZ*-+%uzBWt75 z%8vzq{hF@&^-KDHDI@=1J}&t}dZ?&k0J)yGQ#sN-&%p5Nz=Wg;3F$15lH>^gBSiyk zCc^F)ObAa8GLsOk3yc6!qn!i4I0qZlTM6lUuI{L)Hg$E^JrX$S=3q!PAqE@$`u@H> z>U!VqczkfW0bbrb3xNFsU4k#>oPwp4d){|)3{lUbnwB0mH43JheiBa57(+ZsGfKux zxC(U!IObr|z8kXXPe1QY6}R@HVLIJMcE(ZBKN)4T4#HyZE74c&&1=!?4i`s$ri46=P(YbooDpjUGY!QcTSwY2_RyouB|_tD`u zed%50i?rKZa8thqi>7=uM`CgLj9^3;5Cf6YF`_whEP zht(&jNq&9Awrw6!s?Hh_t%pU|>|}}S-;ujvd*X)ZK!S55-HKGRhz>a!O-#ERQqc{IT-KANiYHjbw6y(fDf zKyUq8AH(2(G_fOl>&{S#vI7i023BFJIS(!A)M->6a_`(82)x|!>HFi#_=`b-nA}xL>g`;HnCjHFnA)Q^ z_Nf1}&L7nT{!<4iUZ(zBe+Up|hLsNmYw$=Ry^oN%l1|l*?++Ne@upKrUUY@(@#teo z|4bl%aL$v&5{oBDpsJ55>TzksFdb64E*pGm}K?Ixp|66urQ zW)fh@CU(t=VB8*X>n(^6ztZ7r+Kf|u$o~7qmCTcV96!Q%;@9VEf5@@$sw9k#B}=&M zbqKdl#w4_JPcX;Qoz5-xO@rt+&Hd|QGMCcpw<80OjAC{A4+w-5b``x=e`%iw=Qv^Or`#W***Q_K;10)@ zFpeoNw(JPkQU0 z&^eftilRQQ%$ra#*0lL&^_ID&GwarY%lOaWG+BMr72mT+LT_lZE@!2=k0hMd z%0`c@xg+H4SW8*YeEb672?|1}5xlh2x6YLOSCSM<2&fuDV-{3Gr=f;dDkMpxg{6Uq zrJ#>ljwi?vLPDAvL|qQ*`q(5C#FFXHS5K*<%|xhxN~w*6r+`|HrSMxvNg2BopRtKt z1Gf}*1;Jj^Q173OVr{!mZ-AgDrw$8YE31Zhc5p@D5_+ChdA)%qsR~9ZEetK}GD6xh zZ(CP%;hBc%V^dS#dAlVm!W2J=oj956B9AJS^=tJK?*<#Zp5kCOTxVukE}r}CpeVQC%3a6}Y_k(PN8Onc|;yYt-TsV*j^ z+$tO>dG0s5d5ZF-GqhlAAtA=Zd1SI>H5r(d*~n??QufS|{j9f|f7g)&coYMYmb;V=>q zN+VPl#aU4m4q%J|))idA;e%AjZu=i8d4FSCzyV*I_NarX-cusOAE*a--GhX#^cjR` zlrX9d@5&c_@jBEStZ@k01^VS|NRi3I)9@((5yHr3+0#>HzM6ojeaM#54(tEMezw-{ zv;4skk%2h3IpejTL5wEyZ~ySEHt~;Lei3dtuD1=}=8d--ROuPd`{>%;z+~e;z>M(l zxeUT=Ir^Bw3MU^mfF?(JlwR8cQamTCSv$Aj^iistj^;~s*S5{U2R8C6-OM2a{28$ps@&V*N`8 z7Znp4g<=ow>l7YY*r+Y4oMsZt?gvRSZmDPPxLmYo$5-VmzJ>#bEwNw2iy|Ds!73RW|3PJm>q1Uo-C-G-q8$>MYQ|L`z58$~y~no;y6U1*hg|eQ%rzPD zr>l&(B83k_C8ltM73sziN&m|xa5# z{|#2H;0o~76}?D-PTuqUEA7Vp#CLeveIS&ogPJ4$2|>*rMw&IfHckz@qK-NkjpQ8( z_S)vW`d~+ulo2&Kx94Cw8&7k!_+*j`0jk>E9DC~I-PE-ZI4l{yAR=Mm?Vc6zRHe|d zS>07l{Afz57;LL36LF1jlWkqO$b~=rgUp%;XF^Ov!0#+2EfoA{dPfaMdY{ul{|T z)Ok(oemR=h%ZOL18g7V}%o|lQJffuQ#*NBJQPAp(As5yUccPkN-k$u+czri_GieT{ zWNVRqPA4QI{QhUDZ+nK?;x@G7gI?nEs! zX0Az^CDB&vjGG>lU5OA*u62RbOthQHt8s*=Lu#rE$dxY`&{_LnETri8^_i%I6Wsm7 zmZq`wTpYI4GUpk!f$@pl_;M3&@KGN3<+bhf8pt-$hkQ=-^6Mq@^om*d&R^=RTS9_w z1P;>8G;R9=geLR|Dk0+s`U^4z&zoGnt4YMT`NU>tQ6{=F3+Jf_U zx&*)Rnj!mbOqZ82oM}DnCSu$us@pbUOsqpF(k#o`vg^Zf2yNj7QxmqhZOA}E@ z{Yap7pa8sbCG0uFD^86)?Y!G?$?;+)5ZSZHgz3HC8s|IdvXSnCwxP&@aIv$C7hD68zS;QjHs_S z<$8X(4vY1re&g=9sg&tljA>N$TqWwjYepYz)5KgEq{{Hy4?3f8k~9>z4Lcm$c%OfZ zmA9F)6}$N+Sty~>Nl;XiXLnm_6tvZqbXSVx-2FcG`W8@%1Snh{&Q8BLiVSa5Gt*)2 zInDc4zal{^AZ5;6#Xe781172-XmP-H-9hyDhCmS@Kwnsvpm*VGsynz)tP^|h8~ zw3`-yq8np?O<=^o&RZpTTjyRXnXSJ=1T;Ix%}r+4M=$Z+agC~vRYp8AOuB2BJ zsH5cQtD}|ciPo2l(7UUy85NW2=xbv%D*8EWY~;SNu#r4i7tGZ4;ec4ww%e>NuHP9{ zb@lc3!vcldM*+p1a|s+{nRL9EGa_mA_P2c^n?`$Ftr35=3b)B2h+#*O>VPi1phQ(e zIlQpyyK5NL!}rK$u9FD(!or!_*+#Bo=8YzSoyjW>nPlIj9L|=JWcI1<8h3J@sbn`P z&l-0#cX~40p%=8+$!b|0q{sj5) z5+twd?Q1%)`Zw0Slfm{W=#kw-)bi;vZ6upFBl}7oOZ(yrtf&m1#|td@#609ow7@6J zAY&e@T;%6(Ol5xKF>`G18K#7sR0a*FP@i~0j+8%&zpE$G3Cihbpbs6q$Th^5ZClyR z`9IfuA5P@-&f}!kwS#G2>wSVDJ(m({n3D0~h5h+7wq$H#$jSHqN zIBtZFzFEz2^_b!!=Xn$RGS%d0u&VGG^m}v_4XH?_FiV-BIBuCz#dhu}jg5DtN`4Fg zbW`rACX}eg&ykcIH?_6mq_<>-nt7y1@hPU1&N>U>P|TM43s*9Ex#DMCXN>24 zlQS8X_1&T0iEo_d7y9A%5!Iph{=pM&te4mbqh(V$S`gzmEHmT{6uO#8iQBcr-o*jr z7iHaJ>JEdIV|Sp7cAM^vzm=ck@yo>-mklf-`%u*`I?y>U#;4%)?Glv5M65wiaX=DH zUjK`RE#}5CoaBdE(6T~%g|=e-3Aof@Crdm!h3JzefI>4de-&qpz%)!M9Z_49y-o(9M?8EQ+CD0d?K70H zlYejKAjzGsUeX16{3Uyf{0~aPB4r#}tdTKtab+=X44rm`nO32TmSPz<(tb-KW&v7r zG;q*B<}l2T)$koPGzpS(r|iEmuabcSCi`$M}7OUAuDKM^>KfXbYt@n^KAw{&bAXuPG}MK7iq`N2i|aB`2}i(b zq6nI#F})@dwKkG^;#nOp7Bn)`p(uvK2^TC#WMHBBRq3CtuCf&Eb2QiTB01%f%yT%I z?6PU$Q=aPPl;I-&F|hEd98Y>XqrF$IX@uDrhvbqG!mfTL^ZSGoHzbjfVa@Q`o~eLv z2g`6xqvoP%!@kZ4_)up};p~#p7~QFxY4f?s;UDjbc7!=K?gqnh*ys89mxH8$GWzql zv5VR6GWu&QzG+7bW!?GpD{E{fj-{d$t%Bu@2-bt6ly>1N<3)XJ7VZoRSR@2X=Kg{E za}ZICzgg`D=-~%~SM^yLf?FW>5@D?KpYvculHH66ZEg zaP;s=OYw^}+9=Iyk;&;J7zg0%ZtcS;GD8MIngAH%9qI_{wbAS5iP5s6YUPz1?(IhH zP;X@N%V6kX^M53E@(0gDL0)j8X`S9D`R)h*2q53QTcBq#BwCSanbQyO-<$ZIJ|s?h zOzpGR^Z1LwQkU0ukn6o$`|gD!a; zO#ap0B}0C_@u^kiW&H6dBbX6faIih4p@6H`?+GcRQcDsgU+BYjBstMWVEnV#X*tGvVdc1 z-ki|`X18fG75v(CRiuqVt3VqJ`3m^EZ1fF4{?0i;eSK9H)sK85QTA!MR5G4FX56Y` zma8&*>6|lY8+rt(+L)~r>Q{qJf!+DO0o`{y9Hb-p6^+u#^!v4)c{Ag`GER@X)8Hh? z)I0h*gDn$p>(0;Zc=``NXuqE{S|x)}i~cDFUl38I|OJF(Y*rlAQd0f`Z1@?vV6+E*># zt!3+4f|B|^j8j07eOdIvxY;pR{nLw zQv@^_;EbJlqfzA<8YMbEHZxUVo;F4-QWsIfnk*4<)c0vC04#?ro2a zZxLi#@+&%M;1)wSfKE9Sl5qQA#f3deO8QXfMVtwv13tkUfDBQ+Z+*K4x(M6ia$I+U( zk#Sonv+JqwIebAB<(9f5-bj0Gi9>f}UZ}~@OP=W(o|buNE(tCG(R}OXe@2%xIm=D* ztFT8PE*^$KTdIuO$+~c)O4T<^@qg&j8%?qGXS60AAx^~lR}Fes2cFO~*PNUy^Q?_o z>eGE=*mYSSjXxj76j|taHoEJU>;oE3C13tQLJ2-MelL9fIsQAGqNk1dJGKe&nr+6jc>TT7m;kk{2Rfm(Bl#rQt`#m7 z^v{10h@=1Qr$3MaX}$FRns?05oW0`dh?x$BK0o-f<;Xt=mQ90u5G}QZ<;a!ffNkA7 zzuQ65UTr|-Bb!{SDLns;@A#b{joa6W$g3h-od3)ZRGiB<8uyL!YQeuSYQIL5a^b^J z`iI9YL>0{p>10R`7kmIjd7EFu2WJ>!`_Z_gE(RKtZ!0&r) z8U%*+S_m_Db+EB?dGMGd*@aHd`z^-LF~Dd>&z9Mv)#K*fC-G9dg!lXTJWCjU^Oh%K z1Xp{W$rL#JxqlSQ5z<3L$nU7J-)863=(l-GT5!j8K~?zoaZbgVA@G*G2@wjl3b6&w z3&IEUaQF$q#{VO#p_LqG6qw{-@kaM(7P(RCq1a&kl!~4`+3T<*Gw*+OKz;x#Tj2t) zKh=Uo!9Z0?5oXX3C)7k3*hRzoucV)o6FwpwPTCM0Y!6QZs-iUX-h}|O&#E;@I&}ct z0bK_Q-k~gH#StuVf(Vvle+ewNf)Ku@&$>^<^?Bd|=oN64J_~E#3kaeN$d1g>z7_8b zH>2D$>rB^z%9yk21jjuEAPw(UdEmyM<>E!H+OscYdJt>IYU6FPa<{h0ACR}8+u`KZ z48!sr-*)+$B%Dd849u+d^evFu9wcRhUY;;;zdcV~d3ABB@d;Kjp#)3Z(3`)5Np z{+34WO0Vt`TVa?iJi>qsCC{QSi@3inW3u-fwBp44v4&P$LC&`$6x{-&=$)z!nsub! zl>XNp?bV*GwIxD#{cqh}sj8!Nc*}udvjVv}>z9_q!73#eIy37z7Jv%JmAok|@7@eE zTfO6yl(0jx6VZNzhZW_V!|eK*?!vx|z%Zbh;X`>|QVdCt<`I(oXp)+GW2XHLdq zYVmYZe$Z+FXU`#eo+Mx4pCTtOvWy~E@{Yvb=$a$34NnwhQ8NkdUMWcOg}UODbwTsM zsZzIuVU+&kk=v#^NmO)y>n3H?OhZRT+{Bi+;XX;6Xttt6FhV+(ABi}*I4rZ2)9B!2 z7MjF=^y#ESqk|^Qw@B%v7t&*E_zU*T>4yn|Ig^cdTt1Kjt&K{skh88dLLYucd5~dc z1VOw3Ckh@|;&Y=zK&-L-lCa>M5HTQJ-Oc@A7EhSC9_dH7Y|<@`=$@;WSE982Tu+N- z(6eQfyhRe?F2xJd7d%0qJ&;KYh;7!3nGvIw;N!oYS3qi&zBfCmjquyEG#Ma)Z)Lrp z$oy!w5I?pWJ%^lp-6z2E+c1L$&Tr8BmY)vJZiJGlH9sAKRScW8hu2#tHUH8+GkiEh zc~LDh-voK8UxH*y22JdRI-~w~#198}6yGkWOcd17MYyIUZ}^CvGsFjlGe%}1{*R_o z++sjPT5v2iqw^>ET-;7n31I^`_^T({40{?;7=3sKj+@?2M4TBve4&pyS^mqKBrA5X zF*HUItqZw*C|CG}qd+uc3!x<@-|eQ1XZC}G4qD7gFlxR-{xPARi!9C?yf$9W;jCwZ z0G0TO6y52j%jsq!b{L6a>NMb58nrzXnlBw{tHRo|Ara6M3cY2|$TPFhKou#>HOHG1 zP~p%db8nposrxOxc+1g$v6Ba@Xzl(2IrsjoAgaQH>lbtozAX1QcTeXf*y2myfobkw zNi>|J{|1N$^8x`s*>A%8tILB@ixW!OfwOoYY_gX)f0WRzKR7LVX*FGZCwEvNjW(j% zgyFZKi#MTO3}igaMTmdD2ykXq_kNJ1(7=2E2~a~mWrm{3z7fXrCXBruyAkg6EllI_ zi=c3xV-}U5KQuh#giFU{UpO?9SL4b-ky-Ej=cL#?C8pQeyGM+u+&Cfjyif8{dYs?! zoBnPW!zYSUTo}{pwN;7Mz-0OPp+Au6gd2L-E9rNo359K2zuh}%RRE^OSy3QeUwydf&( zwc=NHHtvB`E?u+^vqV-Tz0)k*+Uvr(e(=1CM_1lj$kiSauEN5)us3Eqnb+KV^1QZh zL?fE7aPnN_U7&FD{N~FVE?{NxT6IZU5|3_eawZ;$%AU{mK$+}EP}OUH`_VFxFi%UE zqOlA$fxDL!faV_oAFKRHHA!8f7)zzdxs^L#Zi_Y@M#tdFleIQXJ3}pbU$UlK4J;{A zm)NuM$NB47BmfZK!}BlC{XxN&-jkkhPGpx@^%mQQszP@>lr&6pw-65G8oGPuh%1Ez za?x=n7!wET6SV(UYhh};hh_ducVob;U9<~=g{?R7E4h?o7iRCPt0OaOX&SVt|HVLk zdU=DVZDQV4JslO>@VWLi&C2ATBCvrmO~Ye2Fpc|}4)!)^@RnSG4+k* zQ&UmHSfFZYVmu1j1OgMWX@@Bte;v3%2G3Y&n=Pl$tM}%_<=>WnHo=V%u=t()bVc10 zpN$s$xF&^I3X*7I-Aut_iCP$6bf(Cn=4H|T4aBVsU>_tr3`b8;?dI6X%yuD38tFZ> zMwQirKvlGXlf_6}S@V+-nJ5`szPWw3VSG(YAHa+Av_!z&Acc$-NFVFFcpgp+TroAF zw<_8-med3*LxV3%vYB{zZ+t9JO;}Nra!=HOnLtaI!})vD=8%3~5>ghB7V@u$x&0}> zWIHaRZ5H!hlKz?ES^(~NS1@5D@E=O3AXaFf6EEJf!GD|K1BNvC&@~58FK80|`nQxX z_!9AQSkdo$SP9?8brcxJze?iF(jq7hSal*sc8544eJ0mP{bPq9>(Fw0xn5|ehf;x@ zKB%!HJ^SJPp%meASV)ljyi8ydLrfY($GaekAw-VM^k@@~@fxwoMyP7x+I3o+;fBaV zIJZ!~{xBr_o|UNf23;Nmn2|FM;V-!Kr~-RVb*S?3dc3N|yIjn_!uXbxFg<+_8^$2ox1ul&`i`uh;CBulRxdIe7-)~fEEa*qSJi;SIO=)2^u zq}!$9djKoE?MgF)K%1z;6UVu??|3f6XZ?%0nA&n`6zWGF#Te$GO&Bi7nsKcafRtVb*aoESc1#J z6`<=V(nv4659m&itRe*b3x@Ah0MC$4#>|FEWHqL%7LB(-G$;B_UQO{ON`t29O4<}P zi_}K<1bYSEso(E&zpE10Tki)~d|zKR|E~WoU0)7=Mpbr7UNxQIX=nAjtdbTYi-x(n z3QI|{nS3e*jB&E133fv(=a#}_#O2At{0iB;^aGwpUZBtRj8!1~B68ya$mE?q)7yA^ zYWtWkWK-vO9{K}ABn%-xWah-BFGfX?(Tc}U$@x?EhaA$=wK%T?LQ|0=ntGEL2Xcd9{N{6jErCC(;ho{0<8MN4x5|H;ktOMT!-_D#9j!y5lQCyLF7*B zN%sqND;mAC?UYoURK=RGe2WOZXLI1F3|+5xSj-I>Ek^=V7rGVDuarVsD+IBZ+6RoD z3`$qLc@_Ny>vbS@pvgpPD@MNU%RTwRf2HH=nR(@Vd-rW;rTyy}-SO-BclEa~V!3pn26Tw2pOgTSmJHyTscly zE&OR4-hbuOLjvQTK$bwszkmkU82>=~_tEn-f=jl-gZQJDo|20EUlhmygeWH3z*76*16 z@_>ACQ5Ty~%?oWTxk@n<@E#J|55BPqg0L<>;-@zXxIz7}p)Xjm7u>PFYz3-cR^a{I zM23Nk1`Uz{=E2Z6P>4Rn341sF0pDS0tYXx~VXTuVXRBf+h79r1pg{zPnC}M~_3>t5 zcv*A^{Ys9EYn)p^;yKV7lrD#YQ^4i3{wDF;{-@C|erkjP^C7_xS-Ae2v#&@j;`T6~ zc#@ECy?qBSZs8~-i3nlGq%`6f4CNsUtt|~L{k*U+$9%mg%!wf|$M!TzdX&T=EyY5* zfsjqBv`R4?WNVcQz0Q>5{h^{dfi z6Mpi4m*gk9;SCPA&>67a*AgOv%1Fv$P7+XCShU8a0*j25ss*ToE;-7?4(a76SO0cg zWXJ^bPLr)&DhZl^7_oZ{)|nip2xB_(Iv$2IN@szI<>Q$}!f@)*1}%tUO{20o>8JTP zTB7V6eqJd6LhS5B^=te1^T6ByKes45N1mTs`XLox80q)!j3!4#5Z-ygzq-c%4r#;N ztlR#{{MW~Jb&TXAm?b>vgyVv4-mBn<5rsr!kj04`F(jQvHBF?G4GTMHRf>#>1gisO z9u{d>)EYFnG3Ai>g321Ey~{rL%RNkc58n}&mzsS5cHfPI!!6W$a_a!cxn&63nMX(G z8c8!hv6(i$57ENrKk8#C3=CJU(lu6Oy0!mp@*x z{G0QJF{jz6i;TbD+l9D)a3vhS?VtEI-?DS_hi9%&-#I9KjOPx>?uh;socvFlFm^%n z>Th+We7NVyp7^5A=QRHCbhAK<1j6eO{Qa3(iA~r~Q8e)gagaMTPH{$M+fb$3Uw9Pw z0Wh~vXK@m2Ku#F0ghFC(cHt6p<3Hsn@6i4YK6SeUd`?ah#?XsdJVtdu04&H+?IETT z*$BB%E@^fp7G zgx`ru973rN=T*SS<~_;54;q+nyFY$)l}O>ylphpo*T}*8+KZ1g&B@;zwA*9b%$#UCLJ;a6iA-7f@wS77}t%a9Ges|l#c`H zW_A>T+0`>vGvv8pdpVB{@t45bcEK^90#7X}spHHwGg{eKW4-1xt$sJS#RKU^6Mqy5 zQO%);33gqSqkj+QL9A~)+Wx*xqg{llm7?TI6ESmDbys}E^NW+&RhN7{ZC_M}t z8kP#5FHG#OSk-f3PB!;W>+pYX4roaH!Eo@Sl zf4s*RA}vZ!KlQD?QUT8%b+|*eYyd#17ogN{tJHTj{|B-VI(EXuL#Vi4W*)XilAx1{ z&O^PpSKk!p%@pD|%-=|U4I0+e98ffa0@%W6=-houI})QFES^V0sj92FY1h}Ei_dyK zy}4+n7__Hisdm1;a6>EnJ9VgLO^sGWY#9bCBX70EBBXAb4tFybFD+#=zd zP}FeEv2x|kqHVhwdWxHS;V2vI_mGO2*!(~*l?kKmq^ZuDziDt0K&ujM!7J+_SN~eb z*+L+|n$M+uk-~s8#}W(*l9p9MM(1U$CP>3$mt{S&gx1PWQ$k_8y;g)iTiZ}qM1({v z*504|_h4)svQ6N{ME#T&F~IK5;FQMMp!5=A;E0G(+%0sh9x}*8o>Sy0_-AUXGO5e3 z=@JxkOeepUca`@2$I_vRW`o|zbVFMGEYzU37&h=Uf0XUKx%JI9l4P#E4;*I#B%~iD zTH=h__kF{xU%0dIoB@K9C!0P`%|_4VMtk!nFP>T7v;vev=)`Z$%Aq^nwh~grPO~gb z>btzTwY;5Wg!_>2loT>hAK`Dc$~F1@R+uCO6)}k($y}^Fs;~k6RTW=u! zM1si_6mRf(5q>uP#u8!ab?NUq$l1sqGI6_~tQ|a0%Ng5xDSuF6u@fhyCn@Pfe9(1) zK^QMi#GyqR-AF$O*Id&;Xxs#I5K^*efJH)LZ;@t)wRY)m@ASZW`438`c9hDR-_!@p0e=Sw#z_Y4|b)M&gzvW*DU!mwjRA2}au%fr|V^e>D6nmdg(V4PeW+4Ri117>i;z%Y>~by+kh`}Y+w#eSNbYRH5AsHN!{b( zdCXZ79Le97h}Fs1th#*lV@&|fIk7Hg$fD~C17!WVONO+xc-+k4dE8$B{_Ut_x5z>% zvx7e`@Xwar+N)N7njMl2nfnJ;BXu?ZuR}_FK@RjGP~6>rAdzH((ir$E28kk8ZV_iK zjq}D`L68@e064t7PMZmE379chYam40C~|&kewuRJxx6oqizpTB4<9_oZcyZ>LsCBx zu!Rd{QpT&Jh4Hj2N(rBulWoS_>cQ!>+kR0Rn@xO3TbCHzjCG15eb9C|>l#6`qNIa5 za{|TgjfY}G7izhR-BiV!YaBNrjO!J;9Z({TAU#))RyD2yFM9h*r6EYDiMVn^wPIe7 zT>QF_6T*aTW#-q8$~)1Yqv%0fjgg5RYih{gdj(jbfrVi3vny(2bXh6@l@A zYfFN=7kDkx?B0friseJu^^obM`_lMxK%3uHLRm@Mzrj^voqFwv}zNP$sbB3Xe^O?BJ?88KM zd6k5f1;>pERj2>ZJFy>*a?MKMBL*IZJGAmbi;w?`N;YNGO&JJhhBD+LNEw5v=Mgq> zrVXGmCDUOTFxgw~)ju2q7F7n{lzlPO<15k+;kDs;k15yo@WMilFF4fl{zgnS--Yrd z#~4G`AXYaL%n4j_LZ?qT*-O9k;g!()lR6d&#A(B*NyR)M_k?C2m2r&J3EVj1_DNdj zrQ_I@xwHCVXVbuW763;tg4FdG<`3bCW$aKw$1{frJtMKGaR(Le*aK!UnB=ypqz|`@ zQaU9jlNIX}>`)CodrA{GhR9$4wJP(I=)uYzrmNpEtqhv8p$UBLQ;w ztRlQ{?~%xFTf$J0@vIfBw`D99F1N`n;vp}+>8cr6kOYwhoH_JneKPwNOcAXYwP~^9 z5LE{S@!&Tv7EV?7Y`WEvj}#0}$64Y!rZZfs)2_!^#Nx(5Ur-5$B$cSfgBrDqf2S*z zmWb+@7X5IF56Q}DPZ@@hFMX_(ZCG7pegg0lYBsD*lzYjmZ$+SMPat*8V#U@}p&+RUV_1cNG?}#CwyP*<6$6 zdRMNWN;acCP8?QK=GFvYuUkc^8dsEwAtk!B>=b||raB?9JjMp3Il6vQ0lW{D_FKMQ zb-anmn)T_2wlFt~E&IGD{rr=r{I@=nzev69b(EU1gCD&hKOy|*bwx>F2|*&IHm(Or zWkYgSf8PTcTj4nNP+uA^9(zDvMFl(^YH@ybEg8=0!v`6$VVRW?IppKOz%K3~DUuk} zaf76kgKz@iLmL+39m=?2=#?b~xY=OX%jO(fa6l0&>S{%z1ML^VUNiwg*;XjzWo2oV zI>LUg@8Ns1tXPlDByNqn0p%v+zQd7g}=Ni0n$wrb>L-&i0t!mkS zE|#QPrIPp9Rb%8!k@%|uUZOH$c~ro=^D%ZutXTcZsBgVhgTIxQT9t!?^Q>GjSbn$n zTiW40(sIPKvo`03TXEh&@eCazzh#}O;UGBoAh`D+IFXv;Ts!o1@B8_ZlTYj#@`_F{ z6I}MvR;ALuK={pf)F~NI_~f&xP;W*RTKKKSxI#BYabxu? z^ZbKLNZDwblp+3+QBbBqU(~9Nc=8Qe-8cEjEGJN3wG;R$?r$DW+^6hUaSq!LUyaI=t6wyVY9k;NPCs1h z#OhOEP8(Otm{s4Sth}S@NQLr(Cc953!b*GVS94ePi$_!E85R~Q+hANUOC^G2sgvscpCg0f$nWr zDs-HS`!yc7{&f^u;+9AI{kFsCcl{p91#?l4UtyCzb7z_R3>h+*+p0I1A@0Mx*V4gt zO3Q`4+l9c{@;9}fJ(r3~`2e4D7SuV}VnOUSheJL32>B{uzyXdUFNKbyFdD^heqmbp;>N!1e0G-3%_Uysa-wti7Z9c;xDAbupJ| zbTn$541KlmDp%TS^(e;06b9ayF_&eamhaD|py?|sHvYHE)H;R zJ>V};^F8+6c*BV@o6$iPvqnY~FH4yDV*`%B%m0&`{U;l_|KXGUAiFM0a>COAo7B+J zE8-7&7E1`}%RCu-Hw7U%bE-7qE-$_w{c47x#@ffv6V7q=;K><#&;g5L`>W_R*zMx* z=Lmg1r~W*-SdpPoPVP6G=;A*MIwJPMaEwa7EgK;a4@V%(4r*2b!yIFG3BR* z)kVbd$MfD=A(Jl(AdYO5up|Gjx8ZL3(KfhRDKB0=aT(|wW`Pcm^9evkslK1$F{*v* z^F!L0#Ovq6L*nZGORXa~#P-9ox{8GXmO{io0EG4_u=_U|?Jn(utqdgjosXduZB9|*983D=(`Ms%DL zh9yer?E|-#Q)}Vs-lb>sUEqwD2!XvDlMw#$y3FDcP- zfh!h*(-4M&8I^(=?(g7;x_Qo!N_W*wuR0t{IhdAI2yW|4>6k@JZyH=5B}5GqymGV& z;e49BDGTTjj!W)L8hB#VD&rz2x;bVM<22_FbZiPsM}=jdkx)*pA&^>DL9gIQcvEl| zLbYxdLuAnOv@o+9Vx^h@kY6&k%J*5q+@(vrAsw2uPEXN%@X|V}Dw*ucXcnb=@miAJ z3tK@}x{z%hmXNL!0$^o{r`7?n#&m%bqoZqOagr+3>YwsqObEW>T!mOdC#Dmj!Cy#f zi(e1Qp)37=#vIB~+U|NavpvyF&u}ll<)gqk#M?L7G)=@Y*wH}iwG2z0uu?#0uel?0 zCs0D$sRlFm#n~e*?EPG?4}M2D#kn7q?bGJ_ zZpK8^O>oWD!eMb!O#z>5_xP>uXYYd_N#B8Bg6l2P2E{&g`-~NlH;HQzy z2cRGy@c80;5Pi`fdHz$Hf|ZGpt)i{bzi44aD(mtnN~n4z7!Wq#v_bj#$q}g3aL&-s zXi>Nu4Mdx%*=ki)A zi$S*^5NzjrG0VtKb+Ayyb(0BYFfm}AX)iY^F!)XVO9`M@`jqjZK?T=98|ly?Nno5O4!6J)$7}Hr|Y_$&iYq%7+EAjt=deNVP);# z_biy1Lb?%WCjA++g@BKdtqm+B0D!mzWN7zfoWC|P4~Kq1ZB5p-Vclib=&b&PSDO%- zH2$xR{9y(IMY;eQKQUSmF8CBtGML(|XesfpF zA#PIP`V0``2nqes{3Ip3!ltahH(G?ga2hWX=D~`5l4~284SG^I!45hTGQQAP9k0ah z6~ZmFsrMQrd&+|B5HF7(mGmQ?mow;$kc3fcii&)4Ag*(=5xuafIyZEtvr%P1<3(S@ zAo2*_&+1H=D%CsIm1-GX#wOs`{ls0gcu`4_MzHNa8W2;w}=aMhfs zv9NWH_u-mieSm|XMV_!y5!G4W;QrNx1C&&Ca>%ey^*2lr+ z_3bx9z;{Miz7SOV*gcxc*n9xbmKjN)H;6}KYpO8rGm2+ij|c#W%+dB|!jYv$TV7?> zwyLg*hTVa>_+x;bRi$jLy`U|{B{Ru*MPzI8PsVNM6)LP7Ky!3$4lAZ?V}a&!Q*N`Z zzz*9KOQi~6zQ7jh<2{V&(&81FhYP_`Nlw8qI14{{m?D*bmbv7U0t0Tl=oP6_EGFz7 z1D6;|9kF&JdN38oBN`}rT>X60+Fo#JCB1$KPYvQ!s61{MqfA=%Bgrkl7)UV4E1i(N8u?uJe@`Pt`|`ud-ICxChFFNk0y5 zFfZru`?U(+{!I1$-S8VgtZl|Dk=N&sw~?U1se5W=u}PC1(nFY$o9ZgSgX#`voxSzE zrpX9e%-aSQq<`*PV1?bX{o8J=d*2)R8Yk#{p--(*!op>sqN&dgN@1+Wr1SgUzAEpR z>#Pfu`CWJ3ap>HzDr&wG~UnzZ;+1sJi)|c_j7VBg+I9j|_5c*vq zPbQ7BT^ryhC$x*}et+q*uOSIv2xuk_Qj){=YQ0LGzcC0zM%}Z@d9*et> z;Ny|H+~ksDaH`ytq{du#mbu-vx7|22cKp|mm`zrb-Ja)^1D!f zMQ=Wa5bx1(ML4C`6bc7VJ}1)48m4=T^$o-FoUgzESpGdy$i{FrjCcD60q;@3zjXt* z2M*Km?~hn7aQk%aIk^THf-%*JqP)>?{9k!Mii{y9uE$Z+!|R;%d((;#*FT}HaIYhH zoev)IZ?`CHGOk`y5JsGXXqoBVkF( z+;Cp0wj!^h49t*4sa6@v7X28s7FMZt5pe~#?VUvUa#}FCofc6J}`H*Wsq8id_x?fTI6a-LmH)8AEH{dN^nC1@m_pD&sZzc&&{YH zKA6C=)mQ|YuA*v&>%X;W z=Anfik9O-XRELhfK`4V#m_*1V2+m-og_nYY%ja{O)c~VM`zLjYB9kh!c$A-$ZOM`< zic7`6Fa0C_Bl65^n~$~@ccK)bBX#@j5C8SG*YnNW@9uY)T=3uGwGhK3nvuujZQepZ zC=PD&arG1W$Cbq*4WY^q0VH99sr@uYQ6pZga3bZ|aQjIcnMQ^EBa%wZAY571Kf(Wi z{J3-GMCn^a91_OFfLbe4;;Vw@oy}|#Ptca%=TB!5Pr&+XJGHbL{;&(0QIuY6kL|84_yCn;Rh+j9Qn!`3hD9e8N-`=m%*;&E z2%9c0em$+imU1<*F3&FKGaRSn8rjx_gVleExPtZDp;NK_&85AFOY8g_haJ2G7)bW2 zkzQ_j3!;tWS4xgZF}=!IeHclG*+?f19A^Q<{N?MjB^)Z*JXLLdVjJT+L5IKK=+K5ZGDLh3LNe(#)$l_QMYrBSf0Ni2Xl)1!YP99LSfyt z!y4b)=MKPCDHFZcCW}2S*HAV|qx(m~Wm@tHA$vhZUMJL7J*;`SLD9q~qC(}~EWX!v zOZs#ZAZz|NsMJ1rw_qix5EnlpA3rmH zRir_#4k<&AIcoy<+q%f-d%C!Fa$DkPc64qrn5kX57u5r~<}}N&J>IP!yV;amgFZwZ z1cQ)3#$U1dKunDM)-VOI6t5lXnHK@qv>rf!rxlpC3~Q9@RMwBVis*Co`t>n$Ts{_E zUmmb#hPxs{XF!#wYM)Vy3MomDHvO>W7ZZuJTj7}9hpQvjKO|+;6fe1iEWtO<_ZFL0PH` z;kYZBncjgM)j(`mz@Xsb*JDrE<}FH&ikL9S>!XTBSsNV?yX(Xe{^Lyg($HiNeN0;B zoXpaX+SpF00+XpUmo%Ubk4M*FT{Unn_2$Mctm_Aj!E8{CrY3V*-(E&)Of_u<$zn4(~Wny)6{%xFvjk8!`ep6Q1%z_ZdFE~#3%r4M27 z`<^R5uxVbuL*9~jxny4kfR@~#d{Vq(r8*|I%AwD;i^~2M6X44}#%UvfTnN#3LV`m% z55^>954Qh<`hcKh5I3O@c9)8L5@S(=cS|1E-MH*uFu4iteOXEzhcHc*Y2sT}nA~HX z(9EIv!SU~%1X&j= z*s47;nWu0B7qZaW=G>PU7Y_(04~c(QtSsI!qywgX8s5Br(B|BwTDygduAb3)9JngrST?ZqjLNk(XA6 zMjtSo_PFS!jI{=En*oS3(h*k^5iwuDWSMJLnkWg)icp(qv4P@7F;fX%Dk4&BaK?SK zDy=b>mE%1>U z{lU1_Qh}C{S(%Z>>pskMq3Dp!(2J;@+a@)_{3`5R*rSZM_XZDlI>Rhp34Y)0HQzFc z0rb{viGb`6K|_RA)v7##6Z&Yn`DzonD2!GFe6n?N<*JESIKRR}K@bSupIQY87#+8Z z9(lZ2;wlrI=5cXxk?p(HssfZZHo^YCkwoii4HsFaVa-y*(>E?w{(1X^g_X_lI4>Ae z^JHv<@Wb)!CF;#`T0j2ub8=|EG=<#C;C1)m^lhOGWpZ!z)wp#;d-Qo~45QMA+~&kQ z-v-8vx90ClnY!u@hH~@w4Rb>{Ne(07E;@m1&))#`AOwdgRx0*M{rg>Yv!1ZsX_0Nv z#-1uCqL_@-=lGJ9XPF|o2-M+1l;~Z=`j^P8SF=NUBB^iZaIfEnF2#lp;D#j$3|%z! zTwbjz(h_%nDlryGyU}U6u6-&LPTVr*X%|s6#B-n2AeMT~V=)fnB{R~h^S)4~qWT}; zXVAHpTbosuCru^8YCyq`(^z0H(Ho{|9ARJSl7)#!m0N0*7ipp`ni_dM%DC5B^Epr} zqt}mACM&c0q;k!Yh|?cKJ9NJh&&5Rho|3-M!ZN?5oyXCd7FXnqR(lrpx~0}DqjrY3r3437mE{j$VZFX2~w?| zQ=OI|uj(X9etZ}5C-Q%M?~^HA2Z*oHoGrD>cD1ve>`1EQKYQgHYgOMu%`$MM%3?l^MO(W{>5!6_~Dc8$0n%=%hvw1|pz13I6Q^OMcai z-t(06Z1_dU+VA3dHr@Ise_=!TZyuA*b;n+(XAY2FzHjy_25Ja@!-8n~{T=_nAV`FF zsbnUSNYx4R*pp9okJ2I5;znZ&xk1Q$=6%LL&L=`(m$v@+2?l#xun($ymbR@T9znks&Pxo-Q zaq@w7lQty<%^qO5&6%a-R;RI6I0TP zV`wG4XNBK#FkTu^TEOeu(s$$6!#A3`sNW?^J{U_%mnr-Qwf<(qY2vxatd4=j6PR|l zH-1n8CrwLa5o`x5W}C&3&$QcMubTBqJe~5ffE&|Z8*#}6*y9~PjMSX&?sY=%=ezh5 zwh8K)Zr{~M;*(7F!A5-LmbIH{6ZMyiKZkw?zWbnQ$A|&7Wac88{)Xo$IZ;Z0H&Nd+ z=axQW{~ZXp6TI~}N<^LcO636`0Odg9tF}rAgKk-}D4z90iDlH=|7*>{7QiwyyB8`5 zULf}6M(Qv&#aJzPGxaFGvv6UAfYre5;7x6r3H>&v?^ZbR7o4VN!Kj%@t!R8??)+SN zezFo46vP+BVW)8bJ@5*eafE%RBDkWs^+SN8@cV*BOXRIlEy)bF4c?n_MCTsiAMjPf zLOTq4*ypTX@o`M9?|a-|wPok*U1t3iq)Gur!VgYEr{_=8EMG(X``^-j$gg9WmSMkr z8^r#P6I}nP3I1IUc=@U>qJ3OVm^OESh0#JFLr2C=2>>ZRlNrGaYcY+Vd>fZT1uXgS>H0os2vJ9*gm z5%MJvxY>0lx`N9`^#D_f;t8hgLAbAj^>}e=$!e1TYznWEh>-Vy`uMF-lsv;8>gVC! z?D88qch~y#ga-5D!YCifW8uRoAH|B^h%ocgHcFUs()Rzz!-L*a;w2v+_VfH&Y6?z^ z!t>`P+a?&lX(SMczh)7V?NbQZZ*YF+!pceAwIF^ye)0UyFm?mWbtfa>PnPOW*Y9wE zO4a?iPqyl-1<)|{R0C|k+++wVnRHCr&KBIukzi}h&DmGZqcs9FCnOgn9!=*YsXcYf z5)*r)Z5|)J)ws>XE3hX`%f*Rm2W%rx%Ut^uH8*X1)J=G2)r&k%Om!BGn$-rQ;PI6B z)X|#E)XAbsbC~y;l!L1ZyO2xb(pO>BrC+bs^meiX_lNn9y z*g9v^&bjlSUhZ94;(hXnQkd7GMCE>z(c{Nrdq**Xjod5%JPYc!!ltaOm=Z=|qax?b z`@$$o)mWElS7Hh7vc^sB|Q%+ zGS$xART>rua?j1FW%aj)fc@@ioYh-!e@0Gce%(BpDyQo$s z2cC*Xm-yz@s#l>Vu#@Rw+&yuV%!=1}WI0wpl+hBr?m@pLMjYz2lDJ$@!mrUl^Oma( zdz+x76lVS@Z~w!_HHcj@cmH(tPR5$2s3*LTMKkrX(-BQ49x6!@5I}8Tk`6Rc6AM7= zF^kc_%s?T9ry|KK*-OChdKAy+%0ay*S8-KyB7ktSV!>uQxR|-~U&Ww^UYOfY(nBDU zDzl3|1M{p;=iL$-S}Lc%OqBvFl;1cf81xd+&c97VBKkF0ck*SW50i%kHFloWcMBL} z;ytO$y+|OT@2pm;qz=r7qnmV{y5?pjCJvrprx>RLV0%V5%hr=!kfHq&Njm{9vy$(d zI>#rY?(Dkokxn@X-OD%J$k=d|>OTFeDvat0TPh@qomex zXDqERonk~*1a`@~Rnm1mDw3@uR1KTmumEt`s@2bAd&?w}tV|D|x}zLY_E&2GX};8f z8CmTf#mO2c()3=_XuxSMMvIRjK8#|Fltq$WM!R&3^-vuj>rRt${6Vu&w*UABoLkb% z?8XYn$)C3?&IJ70y_0)U>nRD!7^4ugDi@qG=uc2BD-+Jpg;x$nBUkin7t@b6yIrE! zqPzTMCQGlA#Rx4FtDbsGr?!N;ttgkrQAU(OZERVdxS+Tu3V-O*=+UM@d9j+GcH;~8 z$hoRE*a2`&EleWk%V0-upnyw-SZ~1_7mtltUQ9wGf11yDwNj^*1LxB(8CZ-bZ^t6I z@(=KvwCHTHUGWuc=%OE-?deX9|V0Hv>jEH)u}J&?8$|7Af?4uS*YPD@sOm?mKwad_hq0_ z#B@F+ElU|apW)#FzGW<2zuf@&Eao6fH{N_d*$wPFTLo@nhePQ%-TG!7xb0L6)gQL? zBVU!Vh%UYBM^_~8eT=!;sjN#r^)|r1kHU{SA?Afo?{)2#5^y zTm2zl{NRSiN8PY(-KIY8zrh&_Cm z5u2FNd_c^yj7eCUQH-nx!FA^a5Kk{qPUN-djvIfl5!w!%-u@Mu-}&!wjhC(2fBdxw z1bd~q+6Y3T@<#aK?k@UwIx*`B#Ox~_f2-}S1jf(ASC;Hu7Upt${6>cV za)!Jg?+VfOkhl&ST@mDc#}~uF7lBdO^X3i7yoLMm72fsh*lQ1b+?FV#IGTcCz`EV! zJAFSz9pK%HemIGFepc3m+WkCizoWuyXwqSYStm6pk~)@9;S!mQ+N0nAQJRI*OhA0x ze3hO?9n3TSFg8^&ER`!D#NT%kM8&*e0_Ml~7n&YcDMG!^HE;8_QI1IpNzn#R(M=cj zLHIi(wMMlaV+|UKgH}CYouZ*0=9Y?@URt~^#;31lOgng1(!X=HBNN5l>jw_cj!EII z+40YeBSvU*y*~{YCC|>B;0@m=B;xz?63vl<7ef(D+<`(-tAO-f5zGe4q}~T?|?YSD_=iX@`o+jTRC3LQ6mpT&mxO*(Jd_oO|st~;aJy8=~kMkR2j#O;P; z0GDs8EBJ_|H@Vb#D-)&)rhEnM`l5xMMl9=+j`iK?(a)~vXHp#tGpB6(-SQ_?ANT!* zSq@~)Q~7iS&M4w@9`4}q3AzNj4u#hCS$V@Qk!VrH=srZX2&FSrV*E&zMNV0@eWqxi z#d6|EPpBaylz)s*vEW_|B#80KY>y+5`m=}Ll zEE@A0O!rTo`SMn#>}fT@F_a&A>Q(h*9^A)hn|ax=957uDm875I55;GaS;F!e)efvQ zj;F=4SD}0CIeX8HD=&xBoEQ*9Zql_Tun6RcF*Q4*_ngwY=)`RSj?h` zHQ}B>$Ix)c@@31t*-=)SHs@T1n=<-$!FYxC23DNRe2($M{o)~@wgPv<5c*y5wtHAWk4qiJqW zgQy98Gk3KbsV=yu&K~tWice-`I3}HRMMYfq_FRB~Q^sM^vm3H?W6A*PmD3M8H-$(! zbFD3+ohWMTAy3sPYMAMbqc(sn4Q^{vxN6Mr8n{SbE>#M(Fh`Mwxrn3&B;(E7@A5%M zjK<(c8%cQ;d#PU=?Z#EzgUpIF`jc$%z||OG>W>#Y;w6511NPlYRXI#?T#*#WvN`%p zAya>AcH8ed@8Y1hI2h-l;YWB3<$e$R@4?brgG?mJ-@m3rK>yR*s(&68NT9`5jvGT{SJj zlHFt&Co~F?7xhbsI(QOIBsE+U&7CqNpN$yN8;hnLwPtfn?&%$xLpGCRszkeG9M>0P z3hCET=n*7I+StlZZ_0nr4}eZu>hW`kEgwi1=ESYm&0$6euq}fqP?xYeMOqGfL>&m= z4G{D*(S>JRdC57EP3pgdQ!B{2AU8;2Co-9b)ju^e=TY(A+H@)#!#up2XWCzKBcel~> z8peP5$^S?IB2LL)$oTS;X;8j>llgz}lih61To{B*O~oCY#f(fWN&oBhe_>AyshN z73#U59W*9R>J4U3qhE1KPC^qk5kzE^4CSq;L-twl`^;U`faS4U?k%W&xem2JyjJ8P zwGKM<>#$uL>Mi(vK3~1TH|nmu+zI*TwW(cEVWH{VOHvbW?N|lOy2kVO>S z_qnL^vNV%I*Xc$&ddLQeVOWZOtNvNO^;1Q}J962UJhfhwbLNRcI1ce$)x1#EJ?9du zbp6zzX35bO90e*U*L7$Sp~GA&ERpmwrhcxx*dkA57R&oNs!gUreh*BIei>Tc@bTOV3S|_GDDTt4LOR#YZ}CVy1-fVff`Hye(Tz3|g%3{v0S8wH|uf!q)7L zl0~Shbj%(zu6+KywHKBqOQ}9y0uCE(3vCILUd~GtE}~MW zJSirB(I^*Ku7nZE!nY-{7SV93wjUsQsd>`amoTfO`1JWtA?3Aul%-8wq)a4qSd)=y z!`sY1=;Xw+IujL*voe=19rcPV^pm&`h2wM110@48mm~%5> zr4iRwKz{$+-B8Z_UG8GFcggoq$C_{SfW@}!OOsVRP{o6*Y~ot1WmA^Csn4ei2gPb^h^9;7FZ7TyKf*nS$MsH~Y z#j$juSzlZ#qVQcw2cP)$t3Soe$9qXDRt(*UZI6q5{I>ii8FtaRGtcC*t26Lwhk~N3 zf%tF57*)|6t-0*;A2>rSSX&n})UU0uvR}TJQrvBmChaz(@A21ds$v)n zNQEQ$Wl7LGFoPfy_hvtf^>x16_a5( zj!Rvw;y;dO8JLdq{IUBL3f{f2W0d6No54*WjJ8$>i+9q;a#62$(%W#-k5s0Ygng0k zr{Hpk(SGoNS|QRgT)UI~!&Q0N!gXj;#6YvBZb2@szp+Qe1>Dl#8o6x8C)ozStx|_%7rqp@PFt0!>OCn71)gPrMhN@v zl_jI)XOTcA8&tFT0WO!39;R6r7pC&B=-qMcK2kTPjgJ*&{I!0A*Yhb70CCsPKtI4Q z?uY`O9ajGC;dag7KM=qz0<_IE+Ib%c7YnMZU@j6tl<>sa2tvIo8KAv3TKF^JthSsuARI;brHEHagP&eO$48WlZcIr zViSY#nnd)SQ;z}n1+m-gR6*)@RwDhRN_2uX)0J=Pks}5bO<^p?Zi?VB5*!c#!X{V@ zESLrfZUG_^3W6*s5I2aePsQWbGO?SVF?Luyau?SO`uijNK0M$)b0wTc^qSh_zz%SV z2C@C8@qo|xtxL2bQu}Huj||zU?p9agz!oMH|KL;d5O;e-|lm4=g zIYh>Dk(j)W%VNxI^P^8o6{ic!G3-N(eIoq31X#!Aj$ZhZe-B^skL^E~0RNPF*$H+w zs7#2%pKT?Vb{aKmI=>0DX>@Cho6r745Ep^(mKu+j`K4YBc8=)Rn|f3M%TRv>1rhWh zkjX#&&}j$`MeY&lTR)x`X!|hj1opMl5n*LcX=dX#2Kj`e#;oF6*Pmo_m26L`wMi^p|tB1H}~n3 zqgo077a27sc(fvmO6$)yrk|;TxmzzFPae*1lFr!=jbrzbhu+XCMdso#XNA#!LH@l= z21Ik?BEOWP*4ON<*8gjnh&b4~+1bk(Im(!MDLT8^o00xsU;Z!M*rciDiYtNE$u`eI z*QDKCsYxEZ=NWEm^Ee{C$C=<*o6L!hr4?}q9!}P1zE+T8?oH0eSC5ArItXo;P)qhb z^B0-mfHC2Bnfg6MRP;Aof8fY&s0(l>QlN`7l!39u8$Fuchj!;g)=llvr$xWh_(*{8y(5>j$Ywd$)qXhD6W1>dq?8S(W}88KZ3>;OaqgKUQ?94V$s zIW(A8U8O+_7`HJG8tt9F#;fip<0rn1q0xS?&w$^2eM90FhU zRT-qA)<>I36n{m!e-L7cXU7YCBcYQm)PlJ30-rf&3hprlnpJBLo=j~F zCm@?E-Z>$G%e%%dQ(mEO#8g(AK(ELToAM{m9qF;^+6xWq49$-D^ zNQH{BqMj1-^J%86G(E+259yXJw*53||0#;O+S!E%J9A7FX6B63@5CX=|CvP$%w7jT z3c_C@WatN-okot2yj>M;HV^GR$!6j@oxeeJNl8rm;oUjuttFo>+;zx1mTpL%?3UkX zK0aAVu>%NE<;b(wGaG$7@9p*O~(}*qlFcDJ;qP%lTs?J4(ENv zV<^R7tJyc7J?D!@Fu;%Z<|Jyk`Tq0+J6$Q_>|_X1)mLZuOPAu+?D~K_SKVEtM5b|H z4cZ58R`4{sZG`f1xF41NomDXhF^=Afq|8m9(e%T(s!pk$E;X|Q`nH|h1)vpkdw z>g_B0G1!Vno$8#XjV}CZV~B74PF_S)bbqp4NL1EhoTSeG(|JtKEbALh>0C0FdG0WW zmrNQXvR1?AuP9lE`TFO0yzvWPEQ1jpIqq@HODYBbHb>MG&z8ddoIC3G{Mb+xKQj4G$r>Z5$_(P#URN4k4X~`Zb%AAYa zBN@chuXyL5pZOOwsd@Mmo`Z_xaMUHabp~W_uc+mjRC#hHlc-B$_K&RA#kDb=O&FA( z(C1SkYEYNmnygkd0#<>+tW~K;qL#7*oi0#zo=G2=2U9WL#r7lVi5zNAuasmn(_MYX z-vVOy9JdJYe8;NP4kl{+3R?ANG70Ec{l)O-DPT42-}pnE8)X&_9vm_Uy#uutphdOK z>x6cAKSrnCz>XY!3~cNyv9EuN{3V(#Yh2pu?NwXQ>mAD5F#O>$WvEhutXwT*^9QhV z@Yo^(Q=z-A1}9mEWqSqpD@n^1=>!fC^g`x7lG8G@B;Q0=IJHMlP*NfCebwP+7kKa} z5R#;5h`xUC4H6odt9uv-Qogs1R|?xWW%S1CE?aQR#@e_DPG2zE-~KZrE`dP0wc$IH z*I&LXEo;VFIuSb}Gl;4N%lq-cpag0E>Dc}#s2gb8;NIyEN$yK!K=A z*`GwDPsRdf`6mRY$8XoLmq4gP5ZzA(AZEjW0?kKM?0uE+pAW#W(;Czbk;9 z*fSxquWd3Ih;QF`|Hlg8pL($!E|;UvOqVHL|Bu%vP(iFB zD^o`f+dC~w%|%CsNYpDEN=r&++GSmelTo{^0CN6|a={3{!J_9OHNHH|hZ1ENHJ8?7 z=vbop!!3P6EX~L$I-Ru-62N%KtIkJE=8sgohFuyR_wqqL-b-kWe!IvgOrvnnic!JRq&ElOCgRFAs5GH)Pywp014S(MBXrFP3sQi zd|-vp4*K5oaPLvGhZr%Meh+!}gn-aMc9RsZ!2gv;nIYPwP~cO@oLGWP`1Cd(lT9Fm zrj`3!8WN%;jzK)oRah4jxx~qiuq&o0u$6m6iY4WXCw}0BtxX#%KAuEOG6TQCO9tc4 z95P?f>V|)N89sPm5WdEvDnBDNA0S;((&G>M@8ww|__ET6{_WdO{%_w@|FiP^-_lgl z%=T;7;$H$))XdS$-qg(A#OvR7#3r?&zvf{U1&$|8w|zBXCwRB3zIohzSf z%)4Cq+?$Vzdd^-yzda0RzFqM=Z@7F|bYkHD_5BR~<_N-1(;t8;2-FRvgrh_hawMjv z%##!U1Hybnc_NtJ&FN)29-?-l97d`kt~SIA4#T(|4*sq)axW%Ndz6X{epja&IA0q@ za7b_2iv;88NZR1YQYP1{F_M^nSHF?bharyLA6>%bb%N7nA7h#LPazDHaGZVJ3&2?oF+d2u6`V zDdsYd`fzvdN}MNwBw*W-;GUkhBa1!~+<=-DO>rA6d3<9+&weX%1lc*&Z|=XMdHBLi zZ&-zKXA&P2d20%;;}{-QC9hlF<4hj8!;DXt>IAN`0xXiV9L`L=J(tE(P|8ifs5Z`6 zWUf{+1g%yx&Uwoh zy^>>gC*(8-6AoOV{Z(F`;#gR)wS0Q-uyn=)iuXu#Svi%5_6Vc6~K8O`yBhlkX#~8Mf2wt3%+~%TJEw9o=4v4G?(@7G?;1~y= zl7iGoUm1)XT*sMH7O`bX*%ERK|H~Qi;^E%-}2M&Ngv$omdseX8mU&9G%Qx{Dt5nUAX%Axc=3)MDuQ(2Lr&=7 zmZ(FMOL%4>T!|yBq%m~(g!Uony;56+A;$4|59^hmDMf_!1It|UvMAXHCGUJ@n%qj| zS2M!25ldZR{kKViKCFDhGxDNGlg%F1!dj>iR(Pn(tIo0_uX`kV=1f^SAUdE-MnzeK z2vvZ*t}SPA+ek6iZFN2p@xc{qaaK;nv%2Q{4nbCQs28{F?8_mXU3OmjlJEpTFoLIcZcblRV#}ols=LUFPckA`F9~PJSpZH|M97g; z(u_E_sfeah^HXTat@hrn)iAO{ADNf1WPyH3p(*3Bnz4)~nCtXta^Wky92r@)nqu>w zTGhdJyJ7T!e9|NpZiNO&flh6>44=pPR->moqA9Js0EmQ*XJ{Z_EQ^-b zsLoZ%nCPv0Gz2_S{2f;wDIn=D2QC7VLkJex`w~9W6bquKbQ<9^Cs z{Z;eiWhAyJl=|j;t)M=eNwS3cNU*bI9!ow*J)WqW$vJ(0tG_IF>7|aDBVxqZd_|1e z8@eB!q{3Ba%GldapUP*n&BlCp`58*Ct|JU`do0=u|{b8ynKPi07+@y=> zaASdN+#V`wlmxDN&bSPDIYv-U>Xz$`lBzOW#kA3lHt~yHJL%e@A(#WFf#1>i0SYI- z68bi6awtqU^7qmu^YoLF-18XP(@BpOb`=X5=U}L1(WrAm*3~uwEmSzPkiyh| z_0>hVXYV^v=LB2a=ewx!B1~)Cf^;Z`WbPQDG4$0`0O2>N_Bqtv(e-IfQX^7#ikS6_ z_YN_YZeiZp`z{zx*O>Lo_B?K=vAQev7Cq}Cw)^#%uheg`JJ9n;$C=~?`8r(u+Z{?h z@vc;F-#j~`1Qc&WJ%b}SnDu6J=hGtu`tHp(D)!_&8^{vjPTQz?p%uzJVYjGFT40X9 z-+tCb2?$}jq+RI_t*dt61yJwfQGdW)C*9IxT_5f8V+B<2@p;xn@Od&cIT`3V6d5yY zmGt>JHbwbSUk$&Ry@x~~+_1y~vMvI(i1ic1jo^d3*h#%U6+m7$N5wT@J&I(hZTA{$aJrtl$gfmH$ z3h@%uDFv1FEtUK2_U=40fAGxfqCPzdEf8607TbO~pS(m0D+@%2Cm5QYt_x4Q(BGd_ zHXciQ(vgB-wdt>-=)01_Sw1Vi!y^E@_Y@ZCs;`Mcy~*@weE~M72xqlwfd8ue0weaF z4h2Bcs#YAA=aR+0qyQFo4$wol@gb22D6RLCD*oYT!67 z%Y2g^S(E68m`4x0wisL-6*N2+M_AQWMXGGIXvj_MIMnx=2II}XwO5MAse_V0OY9xk z#(6+rrOj$PSx@l4DEp=$&4MV+F59+kyQ<5!ZQHhO+qP}nw*8l@y3pG*I}tncw)1xL z=38b)#>w-^^3=A^Ku23@)=YBs=xQ*%gk9g;J1ht4ua@~@JVvzjGg66c7f6i}RD~;! zrfTl4Hq@leQ;5s}C!IoMagv#YMTx1xy3_&5qZJJTCVb^H0|O1__}srNJCht?mbII2nS-)V%_=!P&nAth z$QZ{_M+bHEawjrXmV7}YUTz4%Tx$k@%ym1c-x=6W%ONTY$#YFh(qyO?pdC!eTl$7V)SK|n)@PxCEdwe##>;OeU&_M#BPFlr-6?;uX5dP@I5p+rWYkattM`M)hE{YlThuGrhCGO4Q(^x?FEAY z_-l&y;Q&M~1LEYw%?7kS5Ou_v(uH7sFzPTYcI=8*e6XqKQH$~Airsx8!53~xWaF@a4Sal7Mp~)|$X0C%6dm97| z56z=oZf}MtdloV#kRhwT#^0#kJ{l>Z!>0NHavlr>ONb|9 zgpxKSj|(E_h2{qQ)96}o5{wVPX+XIjH8voO3lqp8x?P6o49!SpUTlY~E-FBAAWyjs zRU-O!CMXV^r%`;cy0)A8=uT?P_4 zw|_&RdOHE_Av0+!-mXC{S7BLsXw{i$04U|?BqMhD0H+OV(CWpEQqvLtJD05nwCE5d z@~J{Okcp6pubBgQ&p6@(8wtoC*mXUjI(FlV6%Ob3N_#b>UMO6}3owc~(iw9>q`5L3 zMhv+ja*N{q)(u!3hrZ0$vMkH^5!TeV!QKVoDo(>%1x@06JxkKjkjFpmgRhrtMW4V$ zv!x8{7kh~>9RrL^rx8Cc5x=&S{Qkr?evX4LSi^%nzyE7V%R6qKQ}NJF*u0~YzhFuy z{R)$aha@5(IXPy;Xdf2Z0s7~ms}Xyi_0pebB{c+$zYJ_G~Nxfr2y#p#gU>`cfGP_o8m@qxdx}6gyXeITdJFI0)`!h$~ z6I*qJ#|YyururbqiFO|Zc^BOLaBf!8vM3l1BjA0@MNWIZFnHx3(mev_K3NJ3fOsdl z<&3&Ig8>i=ZvD=T(WRsq4Lu5jx!mspcO7N8{B?Fy*vUNf(z$hn#!X+D`+!}a%RI5Q zLXN?aDO)!|r~^EmgDZTE_-fR=t#`)CSo7Rv6ARtQwVJ~{n|(be+flCWS{)XQKTR?$ z{Iug4;@o-cy&gEdSE2B>$p;8$HbwYay}b%teo^hofs8%vKTK3ydfNT?Ztz36aXu#q zczGMd+5>hE>?>H%wV>S?C22GWZl93lOPtrfDbuq!#^=oR{~mfC99c-|7|}Iys5nU^ za6(m5mJF;kGn zS*9;8VL%Fa!w{IZ`xJj8#bFD+K6xHKJN6d?lYZO&Z|Or-=D-EpBPt3xZV_s$e z2?&y3&-wt(oeszlZN@z_)BZpPO9*HSkf9**P!KF+vRDOc?7|UUl{1EEU;`mkA=8OI zkfC92w*z9x_`M5a2*k_IZW`=td&NC)4+82A!st5;W+cMW9D*Pme!*RJzSA0mAsW9- z-47%vK-~cgza;J&vGcS367hv|Cq{nY_FdZpH9zDZqUnL;50QVkOnJysanP(0Hh(Dd z#aB0g0dVm_8h%I4i}xRl2AJ+detS^$g{dFlc<}W_wjXJ~qxe90Kj`v?+#`*CzEk@J z*dOncf&C>yI~FKq{!+jWCXj3f7*}EmB&Hqom$1E8x)A+nZ-s&^#Ue|5lBWD2>wb<% zA;BBsD4`(L^ip$jk}7;P!o|~!I*?Csexk%$iO8u|%GM)1lX4_hiR?s%#c?KcNf5yJ zf%Pv`o;C--d0^=g!Pg_fEu$$_b%Om!jZqgA?ZVV6ab(Jy$Cr<+^KxH&2U;K~K|2=Xxy0Of>yIxXUImcYP4MU~Knse~@^8`mjVv6oOM zm|-cMN~?yccrIZIQ>!XfteaLkJqT-!>!S^6*b|!wVIRPGBv-9&mq1Bh4c#N+(?$oQ z7!Y4BMJ!VX5%Zam?iyVh7K_EF;1lX||CVeP>f$){G6K6SK7L}KtTkSgXO3k$Q!!rm z6B0e&HhsKh{^&D)^mSJl^1F}Tn)xl3O;s6UgmpZCO6 zy+52O!m8N3T=(0KrFvLWM^Kc!Z|k;zRVee28`BuJUExG7^?K^mkRfjk>Y`fI>0H_ieUSb?NV4fV1CHN)HTy@%x zGTxsIL)|d#V2)7}&T>GYvTwRjaedlcn+*TGJ|r+{h$rIT?%BO$pH)e~UxG#{_K(1i z`2r>GqM&gZmr?QYr=9*u+kZ6fwO4oP-D`a}3gPN&+TY3iO>a!oKNvK_3Qc_GQ>2MPXaE|8b8=GHfDb#)X2h~8^ zN~p_fT7rhcGDsCKirUt<%Y8MqzF!!n`?;IDyi9=v0)9K4&UajIdG|c=JlSu5-^Q|k z*^T&rYsB^Ps~l%D-#wAQyhR)8!{ZMGb9q1_L?0qy#ZP(6$hVGSX!V>WDP_y8p2s$m91|yQ$B3JlA`jfgvYDz#F=W0mE4nhQ+^ogMYT5W}wUv~D6`U}$ zHIWq$ooH#mDhGReoqetTE-{fnkL<0gFw*J;bgM?0oeN!hf;I}2(kg6qhK`gLT70%H0b7};t5ref9vW*a4pUu7mT6Yy zGDhJT%89pew2?Azrf77rMDZ1RccC`5C39t>jnXptpf1u4Sx`@dqh(BF9(`7)-Y(Jt z*rP=_wxzO+eotAbeYG~Zz!YPfANh945{(%Ji4NcC-Hsh4tW7K$VDjhUeyJvnwQcDr zsj}Ev)EN0pzD!;;(#vjl33DS;?io1^A8J(_D|Tpc!)cDwRC^w&#@rqWht*Mr)S;gVpUC165#s~Y2S~NLsDrN5} zGCULy@j305x2&zVPVi+^WEX(=R%oG_EsN13p+(W;gp!m->^a&Y zWmjmS6L7@+5?1PJ34iU0ha}5p&lXc*THNws?`$>Z7?!ZbkR_X(#2WULmfB%Ex8W*6@@()BBFYS8qaVFzTA?;%^0VVzx|4TMl2o~vZEe+9VY%w! zEJ!;3lI)(Za{bW00i{|VjjzZ_uS`AS3EdZS3vJbq1uI*dgVqzsH)IzZLj)!;h=pZO z%?V@Y27gYb!!yZGdoNAcT1juUrldT0YoQKM?kPPm_s|~Wf!-U%0=qvvhu#}!AVrNC z(9pFXK~|%gRvTn^2n@;?*o+_tYpqZQ(D+amK1{7;1nC=hhh)(#IFs%;P~nCaQ%e)N zs|ZoOL-s2g5Yr#GBb&vOn(J0wms}|i-UGbB{0AV4QA)e03`;dmf{twx@fnONF|{6ujG|D#(EL$`DUi=fc3Zsl675hmceN6ik!?o%Y2H!)BF_C)9y5pj z3H6UY`4b;9H7yOjXFPnzX1iJDnh=QlI-qK$_IY8ClTVVRAGd`1PQq zN4#s*qqsd;yc_nNk;zVQn0}GjbG>;;(SIyTF<&e`JA)qEuDs)Bb%_c)zJIkWChhTg zi8X%9Q1c9yq0bxWs+K-c8Fcgq3h30y^#re_!{dtr5E6FR;a6_9ui3-O+gp_62?1%Z z^fvw&&9OtJ15i!3lyXvmh$Hv<@nAe|j8Kr}Xw9>zXHSX@ip(geVi?Mj`9>x5(#z*p zb1~1Y6V1TBJ8BI|uwf(5$_aUte8*Ky7~uTda)Uwq$M6lY;0X70C(DtBPe zJHTbU(>uj{iPn+b2HmXqk<#ZmzL56nKyvofe&F=O`2q+(l=4AB?i=C2Fe41m7Vlv= z`cs8;5A&YiUc+||Hc~#rrdNb}{h=n_87=RB`Kx5&7cPAF`!ulZf zN&O;5WXp6|-AEwwZkvSn12Xi1(?9pLEQm8gxjMYkc zM$;$`BpX+$&8XU%KsL{r$`NQlHeVCZ8QZnsP4b6xNY7Ngr9t)razs;RSFBlvMLTKga9SBG!xRdsN8{s|5t|22*Y=D~glDa@x_C{|y zbJz{pCH01@cBZSnDgGtW+MCDq3BYv^3uVoI zlyw{6BWA4(_r(my`yWL7UQ{0}_t4P(4 zO89Cvc8GMA^ZGr0t>@zjIez|`lp{1Ygk7Tjh&k;F=z5f7G$)#gRkT>2Wv@;~R?00DK=}4w4tRJ9xe!%p; zufAy=XF{M*h*{t8)iYNYn8f#sLgIO-{h?^?5WxSr#-6-*IJ!f#Y;ibECak5 zP_T$JnGIHEGkzTtcKVCXjlk0aO#X#oBG3-mv0!TUa!vmxAHzW@^df5svI zDFyq#bl(501^)+8VpZqAh!TD2J(Dxj?1lm3#ENYRp_B##Pz5*IkkW*Zl$@jthJ-V) zEGX#K)v60>j>BjTMvX#?T?Lo}lwcS-*LJOKTV3vo+gn<<;QL%}y_u6HL|6xJ-+lMu zzRvr)b8ho|=l*xo^$fVaH-x`($b7Kp9D$E?D83-8S#!=o(ay~W*lW(lfiX^wKE5<2 z;zhCP*8sp27o7W%7)Qtyw4Hm^vexnP@b$Z@ml!oEDaQE57xb5)42u* zK5*ak7qK)t9mqKEjZ(wE<{`TnL#jK`%f%}K)Qr=r;2)}%Dy*WO0P6v?d_Qg>3BUPD42>7o;kovPh zYut{>!1c$X>x`Kf?7f2^D39W>Y&?%-5c@OLcFsfTp44&sWq=#S%m5v>-Ks-2%D;3Q z76828R|LUR=e- zNG-&BV|?blITJkN_W1ka3@H9O01BUHqUa~5lxxB`{QZS-yFO7{p4*Uwf3&vz;{jHl zYiK%e6~Kk>o!6~@FdqHo5lPU0aLQo)4$U)F-E+c*e&xCs!J$!W?m3h#r!lS-2B3qZ zV42l_Jhp}C8zw;j7)OTnI~{euTf_MW7qEg}BR$dt9sW_$pHOe!p{<3g(?Ww4k+Lk= zBdB1gg`fa)g{zQOpa!*)V=oahQe&&rG7mlbOyO3J>mZYK zt9%Uw`nPkfTsre*{N>8mY-|`wN>Px3)jvu6kb_qgJc)+}=p|zo_%vf)3Nt9@B(JSz z5bd#`o^K;D5B~PbPr%JQtgg+;@?$-5w~R$vsnP3}29guQO6PJ0?L7jBpRYskhRXg* zmx*!uOv(P~uLio;-@@h(fi%w#pzqm3veaSUv}aF1Fd5^W^II5;Rk!UgP zY7i9>_~wX}2)m9Tt0fEFD5T&3w?d40B|Rf53PH6MYw6Bx>!tR&o^`l9aY%s0uZSHd zOIw_Gt4ugIGm%ouN7tSJXI>9Q9CK~UGYg%S2%&TsJE@a=z~{-LE(Ab_Fi|OF;lbB2 zS{Fe{*(4}LZE5JX!CDRIi0gTHnImPFS-B22NTN)bD>Lt%79=-7(g;~kE-q>cj+8Exrt&laTia4RxS8p=GLD7_sj3yZaBchp1 zjMTSV0&03sbNe3IJO_zXN&}-h8}p1D1*KsY%4wwQ+M?ca)#u9%$g`b0e;@1OX0d_7 zfl|b0XY)ZAOdheV;|pQzxh5V%Pi?Lz&%a@p{NyYQ1#wq3zgFfKm2ow|sk#l-4GGP4 zwCA8`Ne11Llq6AxIZpnDL*+3gl&?>|bt}?RrzDJ#i*;757$nGD#e`!MBlA`#T}(Wp zk}MJpe+LCkBqkIcl$rh@c?v=q9#nAR6Ul4%N~s6{yDHiqFQ%)PLc&r38&2a|g=|$44Rdn6@|I}ILeGhG zU@_gEei^69+|p$y(jb6m499a(j2{1iETaj8FadMp^&qIV4BTkF7 z?#Qf)tbL)%=1gnJY(%$(Wy?seX7$lmD{|_By=pwsax9v3{jF^U8-x8FgH(=koe=6_ zkT_C>NWJ*5>G>&cL{1EJbmYKtr-0n+VOC zbK7nfW{yoAY(0V2@+SaRYVP-OmFHm9=}#Bs`Wkbz`2-V<5Z5te3Y3@;?M~z!sBAl0 zd|6iCmWX-WY}Y8SculmCC*Gxa#kvqomkDkq>#SB9$+A~#+M>6ef*7}+6;>V*B$l6A zfuV2D@Gd6PFv={q|3FQwds(AVO~7oNhNHYtiDhNk2mAoL_+Vx2@pWW!6+e^`9=C$| zFB2ce4TG)Urel*EWKhY>tzZ=LwIIGqu!>J0I*}8Is}r*-!YQqK=#e)9sf zg|`l9j6#wUQSqTU_YeNxf@6XQd2YWr{e?1e-I|McvL7r%iS)^h-IXSL9ZvRMt?JGU*@7Ui90Q*n+kok30EO{%t{o_C)}emPz+wsAcb#^H+9i^MI44%EA?uWvLO)>zFdNMAI}0I{2z;_ zhX-3e`h@#2vUGFtHP7LX--^1=hPr>2)*_Zg1cHnxT!IUueH^)k`sL(Y$HlKb4#IU0 zVRfG|Ocn!;5LGQGoTLZ+SUB0fkv*d^FfNQDC`CBmNUcpn9+NvPjU=O76lp=xx+yZ+ zToj4Ju8LH0DkGa&nPPT2L}{PpqJhYCVKGRyLRdV@yp0KDb7T!gQZ9{1f5cIWT^!>8 zS0U&WS4BE8YonO_jmD#$U7Yc|);XB<$FT2+hC@^m6kQnuClrsyNf|ZKR6AMJWsNe+ zu1vxDH1ABw@w#N)=5{!o1<{(cTQ*GM7|vW5Di6l^0%ynLk~B9)${4FM_CVspP$kqm z2nZKPK8*?((IZB>I)wBktw*p?3UszxP5gH*+uZMRC6Q!Z8r1Yc8BWOHHLz=0NZG_g znpFA%L^1-dqToV;>--l7{~fO&zt<7R1ny9$@FYQSN*4>Ibv zxMIIeZ#$j<>1wFb)f*F=adPj))Uka)m)1R*QV0lkg~r4`+&k={P&OTBHXe`SeV2x_ zE*X8aBTLKC*W$lzDUJEY4$1%4!tc&Ee-Jv;*5qrImzI>wm24gJ$n_5syx0f()S__JAw-P%Q`5jwkqn6!Kd3pWZ_artYtN%)CR~PkX;zyII2kGf_3Prr)7DONnQFW~x z$c&xFFIx`JD_7w6q=a8JVR@btanJU*#$qhzti3N8L%)Owg?9SGvoe7pD0j$TuLAGE zqO?1_?wB^nS*uSV!t~4bKi~&Ra=%$=lP zJ*>*(aXZ!sA}La`loJ)Wzni-Q$HfDZec439?F-3Fs?#ILh&sPO8cgVKzv&oq;I|LN ze-->D=(8wEJAg6%y+O3?zU(g$=v|R3?8{?_G5bZ;s|#aUuw?&tsK{&}RP^)r4M=GN z)=KCn;jN+s!_6n9Md;|t7Rls&nYp_bC?gWHA?~Z{K*q=210of~7*7$yS{v zzF%0tZ9!e*e$LRGpmmwE1<9k#G|^dv)78|bmU*ngwm~;|0Ep)@jQh|Wl7>)!c)>af zTY(O;T)KuvtVat&4*RaeJ4tKc_(Z#kor^vUFIK^MGe(P!baiPxn$B1LU%rt2T(ILn z7tta$5!&TMa4%10jD{q_26GL*3HPqpS%?MYE#~(}D;;Z*slE^V#et{<&v`nGbaI}{gc}$^|NYF$_;3m?Wk*8y6 zv7{)RE4W*t8CD%-rff4T8Z3VqmL=j?r(W*f3q@s=ov#_|jE>cXWSJRD z^WD*vVCKzCQ6F7VVXBF5^Pv&Xz-c?#_G$j>kwIP(1C)$-k7zKp-K_?F!wE^`8Ef*qLkMX#+6Prq#S;0`o9RQEyIBfAdKn=tKB{0>>%5FUx?+9RqBa{i#( zqj-n(e_@_7hTh>xFmIILH!5C(cBYJ9GFZ?El*I=`rnonXWkY(V@Vr&;g_2N%ey5)f zzdMoemG8pLPW3!=JVD{9){(+?6J!Ex30in#~|m#4}dL_Sz}t9{_+E2(+w^Thf= z4#-*&_0+vS+~O94j^Zx!;#-%G{3ndNW9Gko>s~aHB(4EMY-ui3!PMk%PrMAPYJza3;Ddl zh(A%KYI}%L?^IH>vC3dcT|CJ}&Z5O2dc zAd#+cs0&5kr5)_(5C}kc56hbz>AQ{D_{2l}r1+|aYxLJ1aiDwQvM}Kh-N6 zHvUB47CgH;>U)5!#je`63F4l*-eV=XgnU2eODL-NWF*LzdZx2ABR_Sy)k@*hc`I5^d@0FbE!M7K$=pLu&%kofpmC z1K}fNaj#5S8jCNe%@@Yr9@Ms8QVXZ-jb{ruZBfm(gt0IGa8J_R6v0?v;zke~^nc%2 zu;_Bpid|y}`#YrQb&nA15Tm0-w&mA!t0+{N=}_#V3Ldn5e6|}WQ5atZ-+_hZ#0&$; ztdp{$KxUN$UWNq61+$#IKxGSIm3w>JzqPOJVjy?(ZNH!4$EU7>Czry zHdEFo!1jnxgcW`!cKca9`C^$=98PVVsZ%nG2Y6b*DMO4KUdg! z*^kO66lz_wh3Px_CTKhPvIX~(M`&5pJHo)8@3X;Xz7A=Pa+{Lqj_&G47N<#GY#HV@ z>pFNf>Lj?JYs+!E*~-8}1tebwSUPrS&H{VnlEn<$oVa10*@oiNd6TAXyWa58Bf$@L zVTOxO$XpwUdIkqMGVxrp< zsigWsqH_$A?cpX7jo0xrLs?%kH49K3uAXJU)@8hU)g)r!OhvIZB~Bz(?DGM8)i8)t zO~0Ish4M=XosWgOp$J2}>>F$ETS|O+S5mmY`*qAD4NV-?-EY&}^x52X^LEY~Z4EbU zP{em`(tC>KnWo!`act9*?KIsUG4U?Z??R1xm6+B=gdgaR>un}TmtpD-jc_2I%<=(~ zq%gOS+n#J$!Lho;^QetE!p^too5H#~f0%UG4BWcXn0n^%e)>W@lcmVm=kadS;$$C8 zdEz`Kd=C#U>5B@0*82Yz*pvkb{~>zzZA~l%^-D3aytcVhhpTx>Xg}c5K|sICAZ_&DQAz|M`*ozpGXeMzXLc_vNmdHJ(744 z9Mz*9(cG~1%JZY!s}CKjxgqXVsU5m-hq_m)9m2bj?$roKd^z4DCcvx7kZ%5B8A2qy zj4QzR4Y6)T5VzsUZZ#Bt=}?G5WEEt)AIS|cl8ZGxz-9`4{qS6Ru*;V1P$>Y}Mvz?ww% zNI9?Yz@?-TfaTas5nhm0daZytkdZcXr7(;mG69MA0(ox!!x=w59Ue})CG1e`o@7o? z>a#DrOB{^`_PC1E5YbV|DAAkc^Mf^H<9g-t=pz=YPc46>U}dRMOPf?01CpLZkm|G< z>yFCGiym-C{lvnjc+$*@8t+hEsNH`UKa{vKMJCXTD~?pj zO!IuR;8wATd;eE_v+&7`5wH%20+|zudFL$ZSCr{NHErXtUVV~Biy91EY7xQV;=hvI z%C$7+ZRu`e*82l6UYGjwHRVbe(@N~*@8_Nm|VePb%eQ6*^DmP$xb z2LynuC=y1sA8Pr584w4S^B`>xCX(4;Q?Cdy$j7E& z21HPkL9HCIFu9$hX?*6EB$JCvN8vY+y6RtQ7@hX^L3A-w(eVM2R`#_qft=SzJtAo( zqN8N5s?JJWRa+P<$WwsXr|TZ72d9+w1_>q=dhy9Sowzpkt0xx)J;LHXNY8g<)7Ny# zQ}qjpJ}HQ2(~MaA0b(B%wF7O$A@>r+kGevF9Mjzkg^cE?c~)*hj7Ngfse1Y+6RasS zZ9I-`t8JUFzmm#C!)w8AiX`XDg(|g7o&Xdx=L4CV5_Ez#7LyNgQEhyiZ3K5-;PabA zGQ}cRTw$}k(NmPUuwbW9sU+C|iivvHKi?sK{U!SX$=c5$rJxkdKmRRx^gnzu0rTb2 zcK_1+f*Jo;L^(TW7f}xfQ8z;y*Z&vd*QyEQgSy)J*T1`&WzuYjCs7pf#FP*Us6+yJ z1egdL7AQ90w{7kXyrDsJ^UR%?k-kc8ZSRU64SyT7n)miWtA;Hbq9u?v44VC|t@V8? zU2nYAzE-Qot@6FQ8ONr3Q}XlK+jb_)xzD+O-}*=2eNK+&dFAmh^hF`CcrU^CC=B%@ zCa`#KqQmM^jp+KwH5BN7=Jfnv;v(mY3{ngA2UHAGjxxM`ccUB*IxhJN4oZW|MIW+Y z`$_;AEb0eauyWKKH9)QsZ!`;Z-jfmak1TIr^?@ft*|GQ0!##B0{iw{>Z(hHO12BfO z1M#B=0_dM+PzEo>Py(9w$e?^`&guhmviExCFODB^-XC>dKYDxyDvwUs{s9KGi_&*A zKRWz=B$9qr01sF_rHAU^Jy`lU-#z)?5)^NskCE$!zab*^ky!OG1spxJZv-R9IUYTg z2l3E*#1m=wkzmCk54Ak4cD3~AY(cKKinIOIi3(eGE@3SeGfPnFrtu8+Y_)UTbTuFD z7p{iI&GaSvi11d0T?Lz#@2v*8eH1eXNW-wJwd}%)73fiQST|NWgYS<{Bqk#2h*ee4 zfwc(NY3OXVa9@j@_9%i{w%{4LjkRJD_R+4c7i(-<;tAg5Em=KR3_PX7kd|ulQdZu9aB8$8 zHNEWxGoGj$Sr<~v`L)p&@)?O1sRqkSAR#c}#>tv4;j(A8UhJidXz=7S-GVHjc%h+6 zw$&^`=DUC5a=QemRf;s%{@Fm;iqevUy&x_t!3>%@43oT{oJx;$uSPVS$4WoZE|TB| zCu*n-MJ;k+k%Y2IBRo*2$4PD1uvtSi-@-?ko0e5KA|>6qeJn`n=9SH+DX849)Q<7$ z8d>Y)l%@oA+bXOZr7@^&?CX$UD>H$4M(~NkWnLDWG}IC^2}qwqwNmWuij5e|*18;3 zc*YR&wvOcAP|&&qw=<(o2r0FQU=;d|aE6Fc$kenqc_hfv3=TComLfuXx1qqV-Nc>R z_{Gz6Yo;h3aK;GX&?<)n*hGh*?9RPv(4)`0pIB;?7KjDHikW2-=#prRCgY&uoTF=g{sVTSkL5@2*L1CV(6;Vzyr9YOP$X zHETE|sXYGXgfk4hK|G^$00SmT@8?@XYHi7ZyB7K3MT9@qH;)thnc4jJF`inB1pgW% z^dG<71(O$TL7Q|JCg~@D2y2{5!02|Yvbv29Y4PcC$ijwC8mACXCE6G`C9ygo;lB-y zMcT8BiT+Z45g~*O=PZ_J|8e^X{o0gM+$?k@7h@UAA&dP2z+cB!cf#)`kLD(GlX&yz zM{mt$oUUiXDUz$7;kKYhSfy9*0ZZ%Dx?ih_*e?xxQ4SQn)U*S4{kv4|mgvxM98x`c zX3K|aSgJ05lB3jWlpoB$Lf>{Rs5R02NB1GqV;XK$8nM=^-n$QseX6eXO|t}bj~Cqm zbyK07R3}Q4vdk>=wzPPfk&}jQBd&!~WGYiYMk%GxhDuU1t7aGkqa#d`WDKbfWms5V z&Qp5 zW}s#fJRpZ!%D9*G?eGq)d>padO1@%f17JTB$rKRzK%$thKd;wqV(64EeNxJnSyC7a zE~#b_{KhuGAS4j7z3KN$_)~HCL0|76ts@eV=r@5caH24y0BlJO&!u>LLx?tiH6emtKJ zt7|SHCeTQ4*u@H{5Hw~5n+w5#Zh8Bqo9|}++`71i;Riw3_ z<9CyCCZWZY^cKi{i_G+MrNmAcCt#_>Ix?9Wt!z1Jva@+{K6LTG)Zfe9lY}51?;v9ucfT1mP8@a#QBgfgusBy`= z0ANEI`;;HB193_!wN{#C%T`{nk~gNYDC$eOyHhPw0$%yY(&M;7AS{WFS4nLd;)QhT zlaV}5vel9xY6WJ}7uC4>lkpC6j6HNgg9r@(E$G z%E&{W%-0mClc~U*^Pz#BO)iBTzSohjC&d~-WPbtWex5^-o8xN)#FK)56s3IRq3yy8 z_?+QI#RFgRYy73g*7PA?y&%lr_`{vnvK6@YI>9JdWvYWX z`Rz0ROa+#WrMROZc5R*{PcuUEziB*iCZ5=C<9s!p?%RwFgzWT{?%0GUvF1{m{V2KI&8IR4=S^qkgE;9X zPQuC08imD|;31ND`!bHml`A)Kb(^ctdD>q(m=|!0YF|k)^Sx zY#0LvtywD~8S?*t(o0hAQbfj^M#lTCcsoc_z4UpLb~*&%Y*K7mB3>Ctp}JTStyrNAJ=(#Hi~iqVq;__6OO{fiH%yZhYbRYCN_s{ej4x>$PkAkldZm&Xr-D zVwd~1)BV8h4e!0iTi~@D;mGbSxMSQG@~w+Ne8(64O%k7%F!eyt5~+`pPO#nu4?nSP z-0LG_B;NHva*mEAJ=ir0c@5B>H;N=V2{50~tlNW_y()K;pht7|}ewd~yPW?3Zy>};e z{iN4JcQ2gdr27HlLog>9;aK*g`n~u|wudZ#bo){IJ^2g#r$%N-dkXbSHZ!)L1_lt< z1bI&lC*n4_91c}8tRLOY=%K;VFVXUUSbL`++q!jIFjv~PZQHhO+csC)wr$&3Y1_7q zmCn4m>+Yzk|J)lD`&8A_jNZqXF<-tuKeXOjQ{Og@EPk;sI4Una3r8iQ>wmxdmBjEx ztqli5S=$!XT8*dsicAESClsH10BOU;sX)j#p~JJa&ni$TF;uT?;-z%f9Yeo;8*_!a$b&;fyQTH5rsT zd?!SzPEb&&Ot2O_HrI5^&WC}kmAJ={1s&@Ko7Z`7N+x=Pk(U-Uac@devc}HOWMHyw zJV+d=L|>^WSPsm4ZwOjmmS3+f zd2p+n0gy;(PbV`BbT~ke5rAe+IXws6CBk&Zogt-E7J*q7LR$>-_)Ak7rzt3)DM`8` z$kQBSOZurP7SraW7n9`D)pp})v(2i`1$<^ThIS8|H|ewxS6Vuz0Scy#ytM;~R+4QG zbVFnz+ivFfgFhT$|Lg3irlld9=0Six@6cAH;$J%6!diQ%NI5U?gT$})SJZH!Q>*s+ zgoDCx{FuYL z2*asoP6jrP*2=JU^DF1&j|7x4yga_m3b&P!DRVI`lzuH{wm;5@e!44^X<@i%{aVzz z3&&oRI~M9D#Cq}+U4qQp4v{;``%>avOtDtZR%k`z-wKsG$=lXV0%dPdsslJTyPe@E zb3-1oN>gw{Qu-|G9-^TRx70umw2o%{fS z5itSLkrCP2{*FD0BRxybU6?nDeWfBu2GpQb!5~Zx(`zZ#N9~z=p&pL+$py67bfcrw z$|9%K3);(Nghjc!)+N@t*$%U!+V4~=qgpe@z?DVHB{b!_Tq$^7Z5$N?a4HpAv7=kb zPsN7evNv00Z#LW%hGR*Sl&i5|zvdw(q+XuQrkml?#rc{4Lj6(nelFt%W83(AWIen@ z(L=}vMtgxZVohU@>>M_@m1X6y&d7Rh*^~2G=T9+3lJo|)H50XtP-TjZ(%;?DA_@`B z5(BHie5vFhXty9K9jmMlPr_D9IIrTh$3_|8qZB6R&bOQ~E(z(OA4}i>;GOLvL8@+& zJq#fe)Ky4D_nKYS>!=}YL&*IwhDo{SarjHk7@12{rnPVmX08!WH+~%JPd>bm{=<`7 zIhDJP=JXOpYP)#}l~Z@9Ba>-#e4G=`mi`lATm%H_ebCd$M3-FV-a~{{G+y; z(y_Q3DJ5^niGvqFz^@H+M8j*U05o_HL48Vt;37wi5H0nmPiCP6sjpnIJghiJtjZJc z>@Omnew}HY=Z`d&pzxdZv@xjjAidWccsQg4`6=UbBh9}@Y~-x?YbiYcmfl#)W;L$|3!3CfkJT$sQZQVC zxnaRwa2k5gWguyP0C9jG{$1sF>>F2U@y4dEK8i;+eSp#>BAmf=cuacqL3!|KbXCS1UMwMdV`xKS+*CZj_nR|>l9LBy`VCZ`hqeFlyC9bzE7F(swZRH6a+?So-fnZeqQ zEJQ*g7FabVa88XCvkM;c48yCQRn$gWsjN_rb<9i@R|-aqa7Gk!})j(9A~yd(iuzhIrXj7hHH}$Cif7-y z4legH^u-`nk`isllU^^uhfJZjgxBv#CjXd=O?}0C2#JU`1jwt}>cx}>%t&fn)<-nH zi7x$4S&H!n?4B*`Xt(5i_5*(`;mtkpRd0O=KlvLn#cr5<86}gPfC@5h_0p^*^xGi;#RTs zZ^I1ZB~vzzskEtfI*K^2SUiZQ-XJv^`(xD&A`6z5n!M`Al@$QI`C&62MZE_0NCOAM zbG3miEnbL>6o)qn`>`1GB-Yg~{nLENePgdte5p|40`)H!Hk!YF&f_gL6$~Xp=51Z> ze~a`k56^r>nf>!i$<=6@4kb{`8xJs z(6w{bC1RVLkw2ePJL7HNq3&>hz*V>cYWl?v@2IC#?1$DtDdf1t{dG_&%K|s~9)h_B z;%;Ah1>bS`^6y)Dsdyj&G&Ah&Ul{t9yY)J0gBZTfpAHldakaRQqx(Y7XWcP+@o~Bn zD_baXqxFS*0<_`n{>WqB2KrL^g-LROS&cVb$HM1wJmhG z<;5I1crf?e&3u?Vwekld$}&kJDD zc6O$35x#kS0QJlYBl$dwVa!-lVfMlQH8`GZ_delD{d~Q+a06iBo;E^v;+Q;97!-sp z=DsI4t4|wH#KlmK$~IG&Dj23mE|;5Eut8y}ONAz?^v4_Ok97-LiKh^B#$c37uH83- zF5mBm74l@UCmAR!^lo3XU@P*a!Nj$Il$}N1j+q_rdLMW0swE7GKsrT5%QwH4!mU7k zJ_?;h;(ZQCDIl-m%87beKORQ4xPaF15O+OdN{XEJa=>qcd`VID?ZUC5Sbxe%$kNJU z9=BU^gX#2-Ymmvp5J7X$bdQxUC=8o$?5Ejn3Vmw@d%Vdl~VF{9sUrnUbwj8#; z^Goi6EJk(6yMTzsH%{YSNuIX`!`Vv08fTt3U(sW@q>kES{Ncc!8S0@D_+gMGXaTtCuP{3_Qe+8;HJ*R<)z8g^E6Vr)|stTsA!3>J|)L# z3%LyS6G<~s>l)8O&&)WKvdl6@gn_wvRy-NWz!`i*X)Fv1KHQJ;Dsb~$8pq0rj43Yc zmj+$*hrDtCgkaUt3qF&0!ssI(%jH?bmGC2$X3p`bp9lCzXb+!F5=&yKoi=&Gru7*E zx}&U&P5`%_$}aHe(n4bssL&QAE{yu+LxxXa3rr&^$8SSHSeK)%hqBDO?2jW_K(bM+ zi?`ZmqVEuJ$gm1c3_J6EYBiwOmd_)AF5C=n&Y-`BOlAtUL~h0^wnyeCGN+VLph#*I zD>nxX8^i6V4I^J0hRlCtI#9R1LNn~z+cn!`Lyi>($e*MYvtkHGUkGTyb-{4__(9Tf=?Fi zpK$FzG2rYx?KyDg7ix^W>P8K~*1z!7zZAukcBO@tLcN%k{?0)-)d|pV{oPY*6dPj( z;VI>~RM4WHVDIQNZd)DGJhER|eg0S2%4NQzAX}{hYlDVsIlf`21p~O%rkHDk6Fm6I z%J9NyDH|ue<-8F;Mr!>&5_~;N8A=MNN{+ZXWZETF<8N^N#5%f?a_&A7ZNwVH zU55>|Ct!#(zSKt{E`#aG{VVbYr=Fsb-^-i^;vc+fo-!}-L3_W*V^?(dZ6iDL#}(_fi`z z2~%N+uiiKYo9erz@D@~7gUJ>$cvBiaS(s*4sxj{*;O|q{l+IEc7f)F&N(-2QpqNy! zTV3N)R_j9`Zr7M(s)6ifc<+7SyKzXwVYqV9<%Y>Ivu9EVCLmWBCoR%@numQ$DKggZ zLhBO@^Jb&56lxWoy{e-TtNI>lw8k~Wv$v%)Q7yqpoKYf&OGOCU>ED!%q|$uHdG^TJ zWNF|}B=Y83KMK;-?W&x%BwB`Ju+x*l0}~%a>)OD3;RCnOLn6g-$x&*Wl~{CIMnhcg z`R=d!>T=vCBHx@ps=XPA7wa3;2CdyAsV{y!`zHOWJx$8QNY}dMrpH6sGBmA?V#wrN zVh*zXvQ8OZO0K+pEMx(JKK(MQWqtV$`#dHSrE2T(g(9GODZj5WL6{877%&n(GCAf~ zA~~Q^8!`{eISX+{r{cH`hv(W#=C0KyrrA{u2@Iv^5NbcDM8o8Z_}}Kvz_&}3OL5F@ zAp9~uv9^p6-h#@7vf5TP?ba-0H(jT-s4#X`&0E!7ovLiAO|Z&_&h}3@fRyYh#RPg8 z1gd!K@<5f^bD;H#NygH;8EWJFal81#U~~Um3hTjQql$@elPz`TVXTIJiVYe5mV?Ib zbjf0ZUG)_+-e>!hWnWSyOic1g&P1VQ?#LpyGfBIILT zXrh*DRGNU30-iMaTax|e#j`@)*-=Xs!6o6t22IEkVkSAzS)#g|sXN1dyvRQ?@Os5V z)nz}X!U;2=^N2|72gHM2|~-W za&ysuETLK~OXaKTE>;$c!?V<=vZ@31IvFYE)a}HnFrMsBVhBMI`i0VE5IftBhuASs z)xhWFTgkM0RDC>+0U#BkbD+xAi%Zg+?uOJ!CMlRaWol@bF)3$BPo+; zBT23y!#W3csp+LThAuQ@t8gjpHjBf;9-^Bj+<6-0t~Utk?IsRH1k+K0+F+)SU0Rd{ zAo#m2Tj=9p8K&O?kqZXo<1l6B76ZCED3U;5TI5J~c7A+I-c_hN&n!>nGT7?Owk_BH ztEeb@x>scO$AsKBYtkM>v=C8YILf|FAJlZ5X=GQLH@g=TWBJ7c^*blJW3pUM^7S`K z^84Oe4F8bqX{Y?`W^{Vgo$W{%HEAznj+*y|!m?lSbE6|#)Nxl`;@-gy#jHjWA?pu= z%APP2zyDm$M8Nd6hJ*UcgZeU<+EO>3ghM1npY`Uf8k`>O0Ua?SYos$gxA4{Z3MEO| zxM&URn-(|-0pq(ni(hxF3$}Pt8QYY7Fnyp++_1-p;74jKm43J3{wGlf!l=9>Z2DvR z2D#=X;01Hh6A;E165ZPf3)g@+nVz{Oa|nX2){9)O8lukZF)7}z#UxbtfL{rZDbeEv z)VK&KS+RtkjfOQ6G1`YtEVdL*JKKy*c7xNocnb zZK-r39E%0vZLK%Q00Ir7CvyqAC1K)=`i71yy7d`3GABzijAJv1N-%U7z^bwSAvN`M z)eoMX$VF(zG8*bHgO*7aD!ALn&C2_^r8!ek$UAj-evn=rzquT?xn8|Z!ye7VvWIO7 zKj&)X-0G=nGu6PuKqLKE~KS2O`|5XP$e?V^qBctTrjfq9Qb?_hM z?^3%&Gd#RJihLYF-wVjm^jqCf!lM72IVduhuZBq{Sl9`!8~`yb3n4P#8Xpi~BGz*= z2=F$QdoYvbDlf-^Vg$adN(Aw5>XXl$7=(>>kO3@HI1-JyFR3Ot|2jZr!ViKdZw@xi z$AIs5pX#roCuo%KfixI&vNym6q<}^eHwanRCUs@+r|;_zi&jh%q==G3Ci~+*uy6dj z8K{JuCW?wTwo8O~BR)l%HP(T$EF9S{!qYrtX%ntti0Wpb-8M>H9-d*~#7AlyN@smN zFsPH()Eor&E|2?aD0`Q#SYTwfcT=F)+%igSXnKed2S2nHL_QsC-KIWVMdT^bR#dvI zY$cXPB;5!#wTI{=EaPnKUc}vMO-D+btW5O!kxjl69bs<47IE($9a@(w-Y zMaUJx4^by5(EOM#D)LT>nhb}k9GgM&H#^M$rDm|I<%s2a5b#zIi!;@_IoI-Bp!l3@ z(*o}5BjWrc>+(Y9DnH){=^LH4NT>shkN?GKY@6=g9;pejtxAPMCnKI4rIkbKe0XQ_ z)N0A*?^-IiU98PYH*&CBnIodjO6-20@zm84%9j?+-(ZV|++{I;gwD0O3t0CRo*cPr zZ2OVUWx8Ckaujcd7tVQB%Uz99-_TtvXLjGuq1;t{P`sAD`<%{e@4DW+LV5=qGXij@ z`k*v9{#uk8_OfR0HKMb@fi;PCmC+1It!s7_==kHAYu#12^64(%{<&E9?JDjL4_pUx z7kf{;>m8oa=IuJ69Dc8n^gfN)N2~F88EsCxLpu;zk;Jxd-bI?jK7+a5;Yl0KNdsb= z3Owf@UuuitaiSHhi1&xo*oODAgq41hxxRj4#}hf{lHTa696Ikz>-*wDBRT(tD@Y7t zh1gFhkdIjmbR{= zPTL-3FJrh)D)0w0)f69Zfi|Hj)Tv4D(EOWDYQmLgaA6EP#gYL7+v}vE2p-yLzlRkzs<%;PpV+i+ZfzKc!?ZIoQ6xVOm5bY-E2hJ~eAxT$F!_j_9 zTx0JV6Z0dLU>CbV%aBD~q$EP#Po~0Mv@Sx?d6Z ze9&_TItFU7b%Cc59q8WNC0foRsb^!X99xg031sqtEbVWTrfY&8jov6U&g21%^%G|i z(DrJ`SSD(ZVKwafDYwvRY>>zQCgPZlKZq-eC9PXmkDn)=i~Dz^l)F)w;uIM}<$a zF|-Jr#YOte*?hFhwY{`F$~Ej9zOKhUd0&;ejK9xmk7mPh)9q{In z-*6UuMOhfBVEYfv-7ITEQJsi4wCA8bqw<+g6bPr^qcSe0 zEcP1$tqGxR*#ui9e{clE86>T7y|*v^BrHZ9AOW7SQc61`$3!%7Abl3uPM zXjsH?eO^2#RN(;ZE+I-;{C&*WQ3?MMlwh5fH_cy-gmkRQTNqC7;Dp%7H~YnYc9*dm{UDp}*GpgU zBkbBzry|)A_i35M6xlBx>M>LO5v0!3}c~m%9gpj-;G|iBp}1 zQ~2rH4vCQ%GIt7J=G*+Lgd92%8lCCO>SM23*gw!}M7wLJKm{VdphHb+;>kUu;N z>DeEfBH3#c{2MPahrR?lQymeTaI>nqHgOn!M6yZmf{(Cf*GpM<0-S@$u#Y&`15$@c zjJy*f3=P^z{nTj1#3RKhHxrQirJ&x|;*|HST5Ra!i!dQA>@I=%(p&ncn5DMlO7_)N zSV~EX8UmYXCCH7}CyrZTl)tP|RN$T-NorDEj7UBuRpIHpuFx8KUObl&9m<^om84c7 zX}^GA;XKNf+JG9fwBqD%Cek2J-VTUQ-iQ|8QA<`p;I47p|1j;+3MZ=JWORj$6=C)! z{cZQhSb7H?)kMs z5dD%~51MMiV4F>DORVX4Ths46!|W1Y+1ESe^p;*ZP=dvp3lP@Wp1ek&MPW+_bpl#3 zz!k7VCL;Gu#u2#N!zVgju~#w$_DOW$B|D}b5i+-ZgQt@%h|GW3 z>gqY*(B@}*nDvGdy(`^SxAi)5=9cRT5%##%L&# zrQLpV<5sUgA8HYINVDWlQk@VzIWIh!QjF2MmAn6nx5p#9kuppDg81ib<=hY6*Y3yj zD+V3_z~Miht+?1*TNoKQoBZ1;{(lMuDVn%BS~!~!3;#boRdzJ6b^5|~sA|_5smb8;|>$U6sF~9ftV@cXUNrC9FIP|#r zu(XJc2er_Y!i2<+A!mdmnVM;1iO~;7JwK)%!9-As%8mR1bjo_m(*Lhjxsy}Mr09kd zMr3ZzrSm3ijg3>4_WCwWYPz2=@7!SzE~)Ei~w&KQQ5Uxm&ChC@3}?jn1#`iI(CGQ;fB$BI7sERUNj4t#I|Y zcj6H9JC$D-Q{pSP*ehh!dwKj8)6$S(wtvMN`i|S%0A*$&Whv@Rre#Uw4VNLP({`(M zbhmw*mDX>|GM4Fb=PW1$@`%wOsyay-<{7!y|JYWw0&ue=l@QCde03DHN@!{1b9Z#1 z*Ea>XjTFzTGOay@Fo`^CaneYB+Krcjr)A#<(r!9%6EImeR)vrO+d7&IJ9M`Q)F{%b z<5{E;a1zv>N-??Nha?@Vk8Q;qFoZZt3ube0F~}D3U%0sc*|xYp|Ls6_LTX%kCUl?s zr%rOOkSG8YC)!<6|BL;%mi7v|D*1WYfFq3DnL8#KbJpS=bPh_J8!Bx60S9;O9taNI zeo)`~ntO1J5_|puNmtb#3Qp~QQ_PCJ=BO+QVIxG&r+tI>Q16)p{-y%}i6_kYnoeFBrUw z7vNmVb3Uc16V9H#RL&m79QsE-QbYfe%FAPV`GwUYIRhmkoK+x7i+oo zv3h+o%80U4E!IGK6X@ci)E^CmAWPFAi`GDoM6nO)5m$qvN&~^=vnxJoB+K?`jo!th zryaK^VkeT5890>r=?)DD#?%U?^uO?O%&}YBS*c?l!H1f*sN=wm7ex!Q8QT5`jF8zsTsc*>v)N1gRQtb*f>4VEXZ#W)LTi?mK~{hW=6 z1mKQQoLK16h1iZ2*$4L3IeaC*T}K+@KX8d?NV@ zL{gzw2Q~8!GS*8hGFtd8ph>AM=mA+L=Ns$N;0jH=S(0yKQBVQh-u0eTR4ND?@pk=) zRP9T`yIFfmBwsq~9EL=~hNZ|YK0u0nQsLXec4K=^C0h*q6xak;V^l8BkW1kU)CcsI zZK{qSDUnOd>H>^*YSBe7c|LBZY+k{Szx|`XZsgSK2$Zp^AasWN6&91^+dA&Z zfUTMpHWdRt(d1cJFZ{lpbY)@<Wr){4Gq4 z?GZXXsJz3E*(Jz2)Ow&6o6gS#p$oUezr!&@;RXV}1ZOZ^A~g=J!@#PGhg;OW?DH`X z8FmGrSq)+YQTZjhv@!YlYF{+`eg{CH6Tq|@P~B9W)0)W2h?&PiHao2gkCRlcaQWu+ zWaN~cwQ=l3bBq7oDXx=q!}6iBnEddu&uHm~Uf4E?*Gf98#VdN_I(YxMPgkSFiZ^;7 z*6+%Ua%yQq*-%`T-G{=_(`C)NIhWOE5yTSIDItnQ_sulJ~#1_9lz zohuwMY#+zI-6`ed!CVZMhh362=GC=jK818sM49cr%BHC;6djDBaeSmbiM#2Yo$8r~ zKIFk4aQXl}InJT^fp&9X$MSDVKWPSjq20bq&=<%*vmaFdfkWgEN>=%k0mc7+$$tOm zeVKn@XFLllDaGSL{N7Zk8k)wDVZ?^scM%dGfI+~$GOb1!Ti2|OorWJO8R(hf`*62n z7&q2b0pn7qp&pyEcGpdXS(Iqz&jy z_A7ntfh=e&+Gn@&)GwAfTw0AiSWDe1_0`7Y%P7wQD?JrZLvdbv9)eRH+~C4}aUR4x z7`&#Zt>bntT(+H?m1|b{3A#3>&Vd9h7iu6vY8sw~6cuVgnkOi!)ZHdKd)J_9iaQ&_ z%55a!E;W~x8Bmz8@eYQxL& zaC-^s8!E-KUATL!{Hh5!!8I>PGTDhr|qbxrE~kJPI*t@hwoH zP{i1ah-3Z2qyZ0Udg)I!7Umb+fkTMCi7R_%XU5R?v;ysHUZvNAug|7yppcW&RAY4B z1_4zfok&?SMM?KP$<5b+k*c#kzTrPqJa7pqy465f)_8KzyGxWpU*i0iUGA*Wo&H+0 z3rgRL`bkf;nw@8VgNDrlTnpu;S>j)WM}R%eS&SZpjOhb)IboVM?SJ_&MEmV2{GmCdB+6+(E3jUUev5>b)FXWis2|Z@UYS`C5wV}tG5{Zcb~dgk zsCZU0iavzYkSyBS6DlZPc9MwkTLbG+<(n_s*>)FyO`Rk-7 zJ(VAk6hFoLVY~Yz$LqTN?C&a! z-;YD>OgdMk!j&Cv(hYCvz`Rx>Kb14g=v>}AB=u`plRcDEWG5>`wLzd#I9E4t)v=Cd z*a|5mT03D9IqQkXQP7}u}Lot z(^b$hn9T1VtdZK^GY??3MWD+>nJYud#eyJ1L|se8s|p4G3>}^Q4Lo8)~<03Q9y4G|pkji=0Jji77?k0ahc{*%HG& z=dQ?f%yThLHp4D4vz#W%LW!g-20FUs#zeQ`&TXiI}8D!=cp=1aYrh zKD?$`b2J3r?Zgybw|qA!93UvEw32mJv z00?@o_Q^?qApgK@>EXEtXr#X3di9K^^hi*H3{4J@N-?|$Y32fpeYG-OF>KNwpmrHA zG`?p0&o6=fn+LxQk#dKNNx#VVzk}FNLfir!UAb{}=|-7<_)bH8dMM*NVvKBeZ1s0L z4#nYqySboWCi^s9Hv2m-vEgpFdPq0>JV;<`mAg2_(#gf#mn$)0=$vXEGDk|j!GA)B z6U(8G*i>2|(kU{FGp z_bioQ5~l{1z?F{1%!{~1!>jVU0SA=H;IS!-L^IEY_qZgAx+;h3oL=}l)PwhZ4~JK0 znpUx@+|LLuLq~qQno5R_-+DI8#E!V9n+I+nQMf}AQ)Ki0z7#pEbCXm`vlC@h`MYKe zdm&>&HsPplQZ02(l*EasQwI&5 zuRu(vWjn*mV#GjBce(Y~3+b_p6}Fd*TSk%a9+L8nS@ta=@hBp2RCD7X(m>uVuriyq z^*g{7IeM$=nIgb*EnIoILM8+R&XBDeF%GMUu$Q~BMa(>ssh}jc`lDn))d*6t47Cz( z#dd6RGkmAZ55Bx7#)mz$QjDOScd>iT`0_s{zrT`!UrbRls zd?>^eM%h7kCxugokWoaCs3&9diLvI)Si0xDK3SPeQU%4L8SM5iVYd>y{>Rq^FG?rX znXxs-v6jNI#V2UB0g!Z-s?kXTTFG3On5^W5Y-RQdJbqZMkovU1b^z~>dXS(GBeb#| zYv8BMZv41RswqM&n`o)*!X)F~NQX ze}bzfb4O*-pld5P;Bjzw;(YxLtVO*Fd1iRw=89>Xk-lOoZ|j_K0@#Q=+zJ;^@viJP zE@-2L{vpGxFm0utKslsT+M|_(bb0j9LQfTp8kl?)#Pt>-4YcCdmYo1r?`_r3m9E3_ z9)n{O+;_%_`S--#07we*(%iyk;of^ZvR2~^6_r?!G&U!1z}4Jd=9P1=H({A`&T502 zwKJcZV>(i&%dNLOzDli|`)DEIT_(OA9 z)0)_T#E`QQAtvsHZFLLDvEA>9(l=yDPisK;*W#*2bo5&iu5dDW(Y&5(Jtpy%p<^S~ ziazi}j~~2hXlhX(MXh%5q#0IsJUfIl#(V!18QfS|?h2h}JZOlip^Qvk*jG%nk!rxz zWHo8;wwCl;5Ilezwg)N zk22dah>$;QmNyzHS}35$AK0u0tkUbDRobkS;3E>+tH7O!KuV6bUo^mEC=#V zp)*rkKJ`Yp165?C;DaMvE}ghVhV|bQ@zGS^2~-S#YI7E9NaQQ{czT1v-Vi2jDH?m6 z9haWhYr=&)`~x)+Fq{}g zML{6I@*r|He>`4@NhCh>qJa~FF3h>bw^^J=)t$}B-peLFF?oG|xLwfL5;2CVM{Z}D zf+Ue{?hqI3_qWDy%Sej4X!tQrOG(GKcJ{AyuCJM6D==|#XWT9jejk@kzYF%Yp%Qx1 zeQ}gj3Kk^pY>_X;4AN5r2{B%{7d}0F+3;3#&O{}^!U7CI9DE^pPbS0%N`>qA$r$-w zl*rRp{pY8&7(`E_{8D+#ff8?mcDT08`P$fKRw*H^ACX{IygCDWqt=kyQr)&hHnzpg zZ%XW*^`SX2KD;nKbh@U1x|VoBCjuYegdW|=x;uAZwlSsun7mLwr!r9a#WekB!~kHm z5A>BkX1FVHH>^F4ntGNCo?K0F<~5f6X?_9p;Nkveydt4%R^Ty~TPXD_)zbXPcrlMv+;@^1QvuHW8Fw$(|E_RQyhSDh zVNyp!!TYUIh2zfE)futy2TR)IyWFLka{*k|7heX1g`)EAz3b^RD*P{>o^g1)AyGuKo19l&G9+t@Y)UhiX2OOJ2U4$|dltm7$oM z&2#z03Rs~f>V6GG!FbFOW>Ob3ifsc%EeU*`ex+Ixb{}cW_g5{ED(eB*l2#R9S56&v zM(b>VAiu7LHx22`4Rr?;Np#;_fnC3Ub`L~ta-(0M+p@<{^&y%0a3SyiLWh88zGtpP~K0LN*~mm zvbAUol(MrsVxDYLQ8W&BTS)E2ihv3NwB~b~ex2^r;bqowa#DN42LOGbLSt({9OW5B zi4HXo>i~je035GOj}a0;VUd+WSgcPP#W_itrbIt8+g^ahiU#Gi%%E34*pR}piJFnl zS2u`&1|6@maA-I;d+(8B20nFKVmrgoB)o9bY$U_N%v5Q+Cdk%U{)H3LSI|6zfjQqX zwgM^UrEP8H1*(y;peoAb%A7CNZ4JE|%2_X4OcSrtjLmUkDLYd^$~T74XsND|Enh*7 z&RweuoxE_348%n8%vUde=moO1x-)CtKbzBI{;i_B?4)hpLc%IepwEwqk$@7oA7Xlu zBC+5+rc5diD>=tNy_041FnLc#Z3vCSUe3g9(%wBRM~khne?D22RgHIs*DkLEz7J)U z4n1??g%}Pz&}qKZ``yvI_)Oq%?aaQXD7rcR!=Q4aILb;ILc9)jZpNZ=W*X}+wvE^3r1RJcofjUWMVbxNM-L)ZWi z4ODu2gNh)YOW1}*NBH=L9FetHH3Zg^&&o?a1Tbd*@sK_OHV8qi5GVm@AK|Wa+7InX zABhe^Ss0f5GNG_IyYA!63E=|Y6PdP7(HzSBkl2hBJ02g+$tmv@(XpkD5bxAVU}i9u zADhIQe2Qgtn}(ePu8eu4N&gq+((X173R~C?<6{u2Z3c?{9Axtb1p5In#_JcH`!6`p zUvM73Dpdk4c=kpOm^q&$uaYCWeqFNt^SqtO+w4yNaAuxi007MX(+*zD(ayzQ&_l@X zN3%rUz|p|k+QeGP#QCSIC;mSk30PbIbNhek=9|^D)R5GWzetIpgX;AmDc3FK7k~42 zEmr*23_~*?DCSpKR3$Z#9+ov7P7e?0-m$v5(t6))_AY~SsYWfXIjX4rB>6;X-boLV z0>DzWP*a^Y$?bk#d*ATNbvlmi_4NYZ1?Z_9cFlq@0(-NZjYYG%MND7>?FToYk0O9x z*)@q;)JHZd@DLexM`vLtJLrPf=Ob{Z3vxh6HSWR`1@=*Kmw8Mb8aGB!W{_CIkERfq zGuB49Os8)lpb|=TP!TLnUD`VSMD$EZNer+2A(~h^Xw3}DT!q+_nQqoGdTO%l%WltZ zzG#e$Z*((3OgMKNEGfVUF^RIwEM(DAUM$OOaS`k(TW_sv4wC>QEhmxYvzBMB>+`10 zXECst#34#(Dl^t_cBG+&3+zP%k?L8*8YEF-J1H-88**?=#pgX{mK+m9FS)Qp6s=g< zIIZ!7%Z9Bl)Jlc%AY>6}IVsOLhH#u&w&M%5TuyY5QP;@ns?=8Ab(Lb>`PTM{X8)Sf z6*_IQ*qGB+Yn^m&;OFe-zk4nq>`vb}QjX;;A->d241c5`!cMhsqYlW(_uPybIF#?d zA0pS2S%A)uR9J1+4$Wzd4j2LQ7+3&PCa5oGeq{RV{SsRhAn(S()*&l@B>fuQ2{lfU z5&oQ+lVTve65X-NW72IB4-VKVsA^-+yy5sG z4CEUwxQb<^VYE9?sJbm-W#;_rGlon}G_+4f09gjowLdOiNxwhAfT|5N!~Rr{I0g_c zC#oypqK&{v^AEJ`xVzAwOn0e28}4HLV=kg%e>&SJ^1ZGsy>C<&v=)Vo$Mec}dAjHh zfI#r}i6Ao38m$+doS7CfSE4Bp%3s2c5#53R-G%Cna%NG1}u+k<9mu5H}BdYNh5d3Q<&RZqkpnZ)RpFcYhvd z)Oo71rkoh$dhf723~ttvtdda|=B5G2%XwQmMXY$-XP8iG7j_>5LD7&n;z?B560li+ zs6S#sWB!!zu~q>`7Nn1Ds7qWxxWcsg+f@F#IIF{QVs$~M_j+_6qoBM1jh`#>{z|{D z0qMFvend{xWTGCue+y5Cx73o(g1BxpcEqx|)DWOdZ3U@1mx(K+wsmxkW-_*?5nLa1 z^=%F2-<8i~s*fC4vvuh)y$@^->6{y-Z+zRF{;d z(GC_xI*9`q=mjU=zePhCwq-^zM<_XHTHnYXb_ua(1cCul%DCbTnkB?E-&vJa6%~!h zaBxU$^%h7F#SImez321A;1;&E;&$p{w-SflpEFAs$n%aHk^Kj$o7h2rqmLmbW%!MZ z_lX(e!}ex=?jYH!)RBN_6K2GxcdL+f+$se2C!a`!Z0Hmpc5aO`i%357%p6FR+$9JV z`{<2S5pq37LCc^z(Mm@ot@D*n7Ro2@mP}3Xk_=)|2@!NXA||@&XT*hL?H2@pXF=u- zSgWxJ;y1YczWM?XAMWMOsmK=zDzG!i+mJp*-7=Za06-x4w!9JR*U|Y~o(c$bAAjMC zGiyXunjVC9wm0(e4N)P(IJ4ixcl7cukzlZSpr=;A@fmRqtt^t2PM)Zn zlG=9ILUW1o1AFu&Y*KVPm_0w+&{&A56 z^%>KW-5%|Dg$OwY)9JBI zEM~aK*wj6mSA6e7zhk9N$mO8bZn_U4VvQWE8l(}#Xt0h<*gIkAOwOai} z8yuXCL)#8m<&O8|oO8=5(t4~4cv28QKo2l~d}=T`Noh+UD&Ky|hRpHfV5OtZv$buv+;Om>Ps5)|Y4GXfYS2I@7u-9m{? zLGyyWUD(=l&8d5a+hTA7NIdrUucAi5t0)TYTRB0sbG{+j2zQ? z*y=e0Lbd5Q4DC@J!TpTa5oJD1wUFg=jLbgi#1V4sL;*XKR~(gQlycj zQs{c>e*T4?Q|695Cy^>ZqXP@wsw}raDS08vJomtPRe~TIwq5^a?2fjFqZu+QEjPsm zS78N2(`32kHUQHE_;pUIFb_Rs(*$BHBs_eo(;Ygh`Au8V0(VyM6^f86x#I>|M%WhE z`RflSLeSY(N9gG~t>GHa;7USgnQuDw3nY!l;EFhVU;f5qaaF|N`}E26>Im4T!+3kP zHS-+BuBe^bAGdLgT!_}*NTHga)65g&L~_>;BqryA42}bhK;)NP5G7-JZnQBo!^O{|G4$;r_&Pc%-qXa=*mUdwO8agj%o-!xi!t z-r>H@-5sq?H(tNXjW+al!d%!j^DRlhFVbB!A8!pZRK(jmP%Y?%ov zKX~W~{3vs5+BVr3^<4hk8Oc+Sd(_}Eo0(|%^S^Xf9Ma@b&=Ciatm}I1u2_1YVCfTk zm-Va**mW0-n;3F?#bxVWW>a>K@hcvn?#)W=Q9KG_tz&QC8a1b}pOMxU)xWQQ=C@e2 z__7qY4JM+rc2QviyG`^JX4}C}kVFVxHG(W|OMd<}p+V_>H1`ewL>g-o<89_)TYit@ z6pl$Iqpm28dA#;TR&gVkVMx1*X&TFfkH3j#3d+Iiw&1vPvGSW5rt{|;SUxlkt1L8A zT!%GJiWEfjBZQRwIybJb?tRtheSfgZayN&u0feo5v4O}A=++GsAp*=n>#sR>Px zgqnaQfa?Ih-+O&4<|DTi^m znrbKBQt&iAj=YvnJz-D8tew8j$5F)Q8a#(yFxPr|RuHIiWNJvC;Z}rLy;^CUIj(@L zIiuFR-V(FZx|h<@tgNX5sd?W$4-WlQD8qH{nk z1VwK}B?6#MBpUvuA&;H>P2l1}-o58V7; zu$5ieG6g&Ip3AzomHs_-cOIWH*X+vUtg|Lg9sZl!poe8E>>-d@zw|!j61oo7 zwl%OOSCr-Iw+GQQDN1HY>}lx&Dq;fL&_HcG?df0M&GD&Zz8Ha4dlIH8tJg zjCBT<%MhWj`Vp!DOYi}F7yj1)iQ4|AOEeKNC$}lv)faC?yh34Rq7^j*8{X@g#Ewyr zT|ROLHnUCAKKEza=$l63!6t@f(e-+GJHvGe($lObeICO7iGSGNiPp9ct7ln#!VfSR z!uf15JYK*|u0G78P}IEtP7$uZ&7586nlgSpA|`UT$&h;Jy&TtF=yPq?Jrhxu+a&Q* zW;VKK2zbd3!CmU{@hlQ-sb6%28bjEyglFNx7g@g>jnQbaTGmLjIQ+q_JOR&{As(RI zyVR-9z3Ef1<=`5D4LqZ8!BwEj2F7_v5Px49&zvGA<0EuTQH>{YfVS=iEg23Pd2)9t zl_}_mkQJ9Brl>rA!0Jg_UtPW?IpSMX`S|8mO*pF5Pl6^#xZ4qvI%-e0y**ZT~gVZ_hu(cv!~WJo)Fx~dED9g zf7XJKbow~wjbjx~oYLE`iM2^_T6BkzF8wow7bL{E=}(s;I!4pA6D`+;G5_v37*)`c zTnrskV$1I7K7;Ho##ui;6R~K7L{MIsrI1|mjG;*+7uU*s-bt26cyoRPWZNuvxht#l z$ScV8V7(&068vkvokkCPnesiCpY#pk@c&;0|9_&U zDw>MxqG+E>?BIq_NP_2$Q%NjrWO03jB*FYpemhHAIOK(~$tt7a$VsHCDG;yZubJIW zWXx{TCTsc>%vpbn5I^9(u6rnWsecf|MZzx)n9p3jbUUnWHoQIg^n4)b0c|($;1&;@ z3>yM7>p#bt4Us=LqN`8d8O!u``ho-J+}@H`jqaBDn&>wsv0}cAIsoT{%!QExPk`t! zXicL@rBhWnM|(SqwtQQ`PnHxHYSP0ul%~SPMOITE6_TkZ8!!fnDr+~5U`>zIpU=C* zmrA$`Os*D}V~XxY!1+lK*D!ovEbUCkXsTf{8jGfd)E1-h%@k==DbxYL9@3;sbCvoS z!eu057U60Nu=&-URLBjb*x*TW`KPAgr1+__x9fvcsI(ua50B@6#t%8#!o zM`s6=ZO~j6F)ItXTk2FGL?Lh9NXT1HHjVOr;WS`puf-|48Fpn1r+IqBt)8{Y2UK)k3c^{4{o0)D2CmKT`6fR7cAnr&L+&d4>W_s7#v zCog6I$gmF#3#^Rp+0Ex$;x<~iV;`UbbO;a8tv!MWYiviT;XHymbawGEHJ+mG$})FT z>zS(VF5P!WQ>|}*N9{;yMeXobGGwmn9z!oAvb`AGm(_rI;`s$C1KWeSQ6P5;`|T=Z zB{IPMnP|exB@9IwRPc+PzzY)RbT>+Kj5bYX@U2o47jX zxA)hxVQ1G;Z&@zrw-=(#;9ll!!MWA)t~`Xkbp25x{5oqJU{TX0=!zF*x#4BeN#v-h zD++Lk`$W>(tl(s^A>ShZ)`!ewF0Y}D$n;0@g-6H+bTWeV$z@0yf7sRG&=gtqq-Y-g zGYh;x`jM|mK7ROgZm*zcHU-ME?}^#TtCins1y6=BaQ)fWo%c7ZXK0KT4lt%&nD|MX z;DNWyaIm;oE@uh-E%ge*yObu>Bv~`EQEEq96w-hNjl7uNGpNbDpXrWl<9q8H{a1)C z@q_bIjxbAYlnTLopXm+#U_qai@QL>)G^Wd}sOROW9Lu27^y@4Eyj8(K$gz#f_CpWx zfcNqv%i1PeD6+e0l)}c-*eT;6#af3ee*tQ0zYxy=iOLNWc)X-594xbSDKpw2Oglww z1)6u{fyIf4ZzD=;bWB)^U?o|9AN+hlm#`D5Xaiq8Ibk&49hty=%BimCU)1_ar-pK; z2xF0X7$l+i>_ncAfIqVGYZKIoMDU_t$#{-*k5+l-+D=h|SA&)3j)j#ojBv(f+^o?c2Il^2%ast zX94D&Fymgt3}D%89Ke0dhI2R!3cbREW#PCp{>8ubiH4-@`7ZxCXrS?*zVxhZog8JI z9KXN6TSmWC^exSd{vqZ-BJpp3m^qkzzy6Qwut`A${C9f|fEh|8>lLgcF!xr9mWBeLXc15DD}Nz410= z@B$bGDixwCBHu`YWCC??7fF51A)JH4%mSjA0+{GBwU|i%eHa-iC2k4k7Vm`Kzt$5n zfsd8=QaHPnX;3DShO3F$&?S|D*b9pK+&N74yVU@;>xVVs*)|K1jKx$*Zd>kq;E6>S<1pG=j6zND(ktGX)~LkDDj_<8^LuD7aiL}97531gg+imG05|U5!tT#_oKB^Z_L&bBr_NORUT573MGcOK1pI)H{Y0A4Pi6h z*stv+f_cF8qZzNs5X2yaKS*M1hb6W;a^S^tiE(cssoZ`Z)Q|S7&q`wq9+oA!yuBgo z1u)sfSvgW@+LYwd`V0cId2>S84rU(XelB>R?YO;b|F-|)_&SJhA+C4ncv6VpQfbLi zWnx8~9fPeOKKeUU!;RZ`vzhA-+y~v`G*rs?ddv5@Qa3&`Apc9UPLoSkarEHsceRZ~wL+Z(!)>eehLjbhr|74q8LRhA6A@<0y} z8;ON^KxMWL%ZTbXUPLK~YO-GHsbzU@t@THseh9Z!MB&(Ai9k;bVxbFx`7;y&HP*rn zG#|R!$Mpur%9oENDoDpnfm~sY~};UWxf^PwZ`#h zB15KuzC!{WD}CvOMd=YKV>)Z;GKc02Dex*;15MDHNquW(Xs^NEc#RIP^f4WgdD{mQ zNkP?8gF2I>o}p>;tYz02PI3EoT_>TR2$OruGuWC*2*WQ(p5Oj#(*^FmjEe6LT+P{I z$b%GWc)T?P6L1BG%zOnG_SHtMCq)38KlNF3$qZ6GlLoRf3e6tLCilBD&}1nVmz|CI z9+{C*7zz#6$)LHBcz4GU<(`U(8`PV2v8n`r9IfH9Fx$M*#GTsk9F zEv?d)fx9sk(GiUV%Dz`0_zRUVl=D*lWMv(pR{=A{% zefZV11DI+Gu{Ucls=*&h7T6Km+{ICsvTI9>t26D+$rvqNG7~cA(O?uQnUshnQp^H* zLE}untQi4T^@V~`Z4c9mZ%^I(!Z>DFiO z&E)#Pv2jB>h^@sE9n7r8=Z7A5wiM-e*3jCDVkj<}=zGVjk}WFD z|AIh0Qdd5ODK~5@#@#HslFSh^NsV&NWmR z#QGX`FBJBD>W6(y+@E^_1~Et< z81G>M?@>Yqya*A-xX4wl+Lpt3dsJ{?RSd$vg3Wjd_r$|J^utvd&g!q|Ob0gx-mQL5 zi@I{f#3sPh85|rw;=Z9Kvx{y{tps0Yl3v5`9InGh`k4^lg6yB@d9v<7Uy$gKo(&DG z47qxmqF{WN)vkyUP=*&oJ|(Hz0*FRd7e(OwhbxDTJn&hfy%&y*d+rIHy!wt@%IK zu-MVf4HE3cFlB2ZCHyub8D!)92T|KC`+j~)x2NX|>;^2u?Zj2@XGn;-@?xDe{{S&5 z{(0$XDRZJrmmX3JB;y#~X?S#`(M1T{QT1_{?fRiS2~n`+&$sp4Q2nOE5U&i#4Nep` ztF=24+e@#zR~;JBTPX-y+qRKf@6SSVKB|~a1mV#zqZ~M{Y%fw?>>hnr-3;ozdayG* zN&WR}=wLx9cLN*SdGuZ2>-I)P^A`jcZl1*PY)kXoxI&!m>3o-u?m`M~+*YFc8JEb8 z`jYqpg-E@02q9~Zzeob@#qskYJ{jxkk!9|;vq~V>qR~(Wryg+#5tdgRMB(LA#tD8i z7uT8K-!pGs=EB0mkUmIUg5b=Rlm6>mM?%(nFaT6=B>;_n+U|~zqbM>9z%sZ0#+?u( zIA3zTIJA573LiFv?BgN78E~WVo&a^3CFeAHEu^A>hBbs*klJ|%qkQ286TJ|o_qoV%qC$?h?wy+y;YRE zVxK{4BnqrEwzt7ZnXDhJ$|he>WA5Tx-zD$E*V$BEH;zxJc=0*-;#ct@*e&)hPf02! zw3HhztG>D*#J9dz>~(BWT({<=KKK*a&`O~)1U8XgPD$^o(Sjt<)GhFaERdp7aJ&VB z_>Y5yQX97&#D6V)?M!ia5}<*AP%wdj6#lat6kuj$<|u3H`2FMl4S4^zr~mr;e*wCy zR3m)QzQ+!(jM*Djvv|Nj0Cg0k;yNrmpd@;*jD55G`OtH|Xyh4VOtq`4(=x6wrXsNB z%4mP01ml{)G#lj@81u2#Ko*y}F9!IU5H(+C|I$oi_+_Cuzu3}W?WfV-O@5#_OS$s*2cUG5=*I+h zq2R~g)FQgM4u%MNuEvPnVkL}_UfdHB$TAEj0RA#__8w4*dZvrb+)$9^N%c3NWZ&E8AQ3;?o^gNRYO$0wIlU)lo;_!4(&kL}+>I5n#?#u%aLZ=QYty2mU= z(mF>ENXfPh?EO_`>xs6LZ6*87`~VhR__V2%3^mgX0l&?K;VYNNiQH>xDy^ae|JbUJ zLY)D#Dj7R}o2jU0ZRKe`B1iD8voyp5VVvr3wyWz2?6MM~vBI`iM6yZ$Y>`25|cj5h7o8mk2TF;yZSOIprX z>s#D8oyG@&u|%WjJWDz!PA}64UgZb;o-+q7<1N%k#=t-tY*SvKL9(sdwG)S3Q2T>Y zv+$F6RN;{LtP(lbI?zl_wFdxu?}l?qHiB3VT_(`pF%b#{z?PVjRGDR9MZuCLAtBH< zOkE_AfKunu%(9_j_=N{&GR}s+-iv~z+9I%7nCztCURt??^QTUz3943FZ>@yU5zFGhjq1)vLfP-&1y@< z2&`h|CLMT4&Z81cjHr2Gbrqs!F@o8|22LZOWvIsqY5X)1Xm-UHi~Vl^i0&_7^crbC^OC&RE^;r^`f4eqZczsb*~ciQ<6wTto<6mC&b+a z>BHDFcfvQ}zjCX`$aA!UV0RmYz{29EoI5%Paa{|Ih^OiB8D5;+-U*izb=Y#MXsXVA zvmccxyddH7c3A<%AEitG^g}eB@Jz-BGOgR#!@}v656HRq{A+vcb-z$1FN}<3n3z-27>k$mE$5R1*R7X9AXfd)9r`E$>pV~_F@E)rwo1%ZMd?lI z`zkG7Mv-5-Kh-1bm-;IZu~>zyEcQct<^wGa=&>2MTw|wF7K9!3`n6kmuwD5;RikmY z20}z({`ecc(Hjl_I%zPnR)H~jliP!<-Dg>r@iB*me}l6AZDuUN!jCS32x*pH{La!8 z&ZDq^jQ|B`XKHlIW6TGq$p16P8Y58Y-KDCFGr^0181l3CzA5v z>&pO$h(3m!7pINzL|skLk~t0m&AsUYTb~a+>W{+ z4`TukOqEXJczy$t)N3@i^fX{y-u%nDuzKl^_Y%^5UOJJ7m zmDy;;pAATUkwZq&rKj;p<+dA&j0?esl0MhHop94uVCjFQ0NGKga2Bgc&j zuj^m9l>Gp9k|PgWMRv)V-i5tVrY?dYp6!VI!NqX&nmq}v1ZHUby|u6vhD*ZX&x%)Ho?Ux{i?I`ic-En%LyG z&r;}Ui|@NS*=^$0&<#}=7oI5qbdTSijfU;kA#K;8msiKoOPD_^t%y3OoP$;cmD6&l z1!lIpO8uVyt@{z7lXAKEU8BH!6UI6J&r-mDwmpD256Ib6Zmo!Dns1sA*dS+SLwsFj_224Ss%3iyDY`ZvZ< z(M?@R+g*`Bs>4RDwSAOKW;cRW`^2I^n9d2?&;%IGA zEHGlf{s$TAj7c^)1ygl}qZyY}#C?EOzb5>ndOQB~gY#$(yo?TFOxUf6D1AzpEJ5Ej z=i+>P#v`PACN05tH4^q<4i^qJ@?UE;amB@NqDvNCcF zi+VEgW)3g)1a&YePxLNR=`*ZXQZNZUvk^VNjsr^EUC%C1Vh_K-faK*7dCpsosuA)&gYXWo6|Kiz2v$cY$a6MDptQNn0A z^sgGhx%B%Fxcw|h2uggH?E+Td*zGD%8DzI2|Be#&bC-b%%YX!CV#?yqsjZjMa-j`d z!7O0NnWbF&mtE_9#hYj!{M^*b@SK5t>BhA0hjp8WeeuJ*OT@IFhjm+n9z5UqU_9Op_Z{g;v z?KzLLASEyzT?smW6+`jZOJP*e*!woU%@@nT1K()DDztY~zyM*=#-J2qMKKSd=}R0O z56;4Eq8lS3PanR4^xuApowdU;w^5Xtm=mKLtOf+>KULzXN`PjlP=<#jb0|q*#jaF| za-vmevv|i-fUZV4`_gaj4^z64i{Om@^7AV$b~;Q9OaF*%Dq})r9+cOUC%8$`MG<&6 z{|X;=DccwF>vH<11zRp?5KY99Nl^09vlKR6h(VT;Wm>!@r_8wyC-=WFcbT8tOBD&_ z%9mxQ%yQ~R`jFO=b+r+^Q$Y5k^y<=GKZfXt?){@mY*LfPM<_?N3uy$v-po{X|84pB zxtA#jzZ<=b-xK!&|M}%J`PNpl{2!()a`rZ^ZvU}>RVvyJ>%wTf79DX_RFrlL`hxayu))^@u^ON^dHy25ILXkgZ9qRyWbBAXN8w0R^5xiw!HF@iilqncsx z&N_)XlTr$yZ%aJM-5QKLJni5 zeSkI{7TwdMF;>~F-qfCR6F|mEs1H&WbMPJ5;li}mYBWV#JTF1i<+wGF!1PvVL9_VZ?%()5u1_U04t z6V>*4ghA?OnX@j0Re8RMtY~(cLJ|m+o`t)(!)S@Do*?DdTB;i=n+*hz+6;f&W|sq{ z{5ck#Ets^-oh;Yv0vzx8Z_M3Ieq9DJN@mNRescBo{b2~BgvXvVZ4C$ywvr2osfzt1 zJ0(C-lxUL}{7B1I@dVFdw&d$cPF4*(mhZx<{tdIqCBJ}2jK}ILXi{(bN8-aFt-$*L z331~nT|W=wB6nbX)(iM;T8KE$&k-oP*;L`k;fhxAW9BF` zu5$Ir1kz0~;x{m)YMuU~SWl|LnAz9gcj18tQU`c>>Rw<@+-|Q(Yu8r6dA7@zq3P!C zP4dcG!_*I-${Hi4$vKja>Wy;Ovos&%_CC9USF&(EAx3{q1|59#Qj0+&H)6JI?jr+{ zd;At$6IzeBpUKudQCkJu_{PMZ6e4V*vyCcmEknJJN~&>_1p!s33;h?u?V{>zLli-e zz{8~TLY=djqK4Z-R_g8OYGj=EjXH$u=8!Y45g`IL1vM?RiZS{=y|%D_N4`GYW|iZd z%10q6n^*nP%?r9kbE>*n0Dos|T_b}Xlpd$kMa6;J?r9JD---UHX8ciw-h0N<6o3CW zw2hg`g--1|bQFEV!D|1<_m864KX|9cX5Y};|HdAwDbeaX6GIyLTJNyGwBsj00+tu3 zXg=J%$*Wn5MWKK$&V#099aLRzw@T5vPPd47tpY<~#DTB*kQ-htkjH_tX>=I_92pxQ zvpaOW-7hZ_0A&-S28Wk$WfPVWRufhY^wy(M69P^Sw33i52!)H-SSy(+GCbDd<}SOk z`a_~eI<$AWl49_t^)9#&o|-Ky)(^FVCFdT~ix=z8Xsw-v7aX78qOCZKFWE#k>!oXK z?yJ>18$an(FfOBMar`~PEJW(z{O0)rl1r^I5Mh1?2ecGWGCNzYdaVv;7?}%`{-J%s zi?S-G!0lePS#DLWGiURfdtWPp5TaJ+sgI^j2xpzLpFx#j&cK0}t~^orZb}3OV#St5 zFCNXiMER2?ev5BgF-&d%wPlnmIM>_c4(`>$U#tW3f$XQ&O|Sb4t8q{Bq*a_s2>FpT*82& zKGc%cGU45+(b@Q*WD~K@wYWyT)U)o$zTAQ?Ql~E! zDu@vDJxSCA!^VLBbxBx!=B*7^kPg2TJ$wRcucNP z02B+oN1qL&b0ggi`@*c`L{^>KX&^61S^v;D%qP^sr=6jgQ-*t z+aIdvBVVd6MNNlJd#P(tIFjju5?1Qbcm2{<>|y&HeN6@#Yfh$$KYyHRY&mvOJCm-` zHG<>7cnQAdUWV|U0Hy76`jHXR1`3847c;=YZNCg+@-YvH?QCpD(P%dR9s%g_tnqqW zuiig@gUw0rb3I#KFna7dWkXi3azkSjg02+d($AH+FIs3Bw=WbDCHb><8j`0JOh{B@ zXR$+t;dC#SqiM_v?rJ(RW1NaQ#Rq}Pus=A_bwl&9qdjelRCGiHMB)U-(%Oe+GwQP} z6=qIPCE_NrR|`pt4*~Oy4Wb(C2?k`xP|K{A)M7YN6DIt$L;zrY-ydraMS1 z?+y2b&T4Fjb&e|aoGfIxbH!lNU>L=cu^{iY!nol?X`9INXu_OtA0Rqdq0Ll7K-7=w zxE5%STjSpw^&OM+0x^GQ3PtE-&#J@KA(SUW9SLRa@XOm_Ci-rB8JHYq(kmGPBbZcz zvlj!IjgBz`QR*GwT()bW%Vop>t0I!6zsLB49s1`;Rca_sHsJ=6%f}1Lxn*DnmJ+Dv zSc62`EqxW3#G$0vm1H94EyT0tBHZg^*(Z(0ZSM=WczQrg9F{9=uR%7m%exU{+u->E ziTkBtsS7mS?j|<{)A#bJc^zwvlY+CrBhc!)kb^(i&4cpDc+FC=#pF;Aj|wAa;+B<@ zWNVVtk`y?U?f@$ixlTRhM58wW< zyj3-tb)b#g#pN_Ij!V^ zH>B_cBbPY%LxwnfVKV437SBxG`FWt+%AU#ts2oy%%7IjFS>a{(&T+c^N?0E96bBYK zDt57XDt5nR67{i&sk+N{H95q%3qc}+F$>QkI7)U&J11kmbQ|4+2P#T67jPF#1~Wf; zL&I&DjcgR2;XE*?$9ttEbI@h(N)1ts+9v#kA_>W!R`N|)g)A1~Rk;=*Rg2pb<1-&% z=NmAl)KKjiix1*zmjWIIf;+5la1Bdm!rf*dHP@G1`3bc4RVmJZ(H5Za42h%*nuWS7 zuVJQF#s{o7-;vMO+&Y%+9@N|J$R}`?ZcdqrvaYQ)jR4h2yTw+o7@1_Sr(dEG)oZDT z`H>j&MGxr9u7PXhHdY0O*$X5>is$_626J28l~%9b-y~BMk$nUDZ_P+=Eq2%g>OC0A z^m0YoL+?uWE6_z&qZa}Vi-}&mhXpg#4<6XLkTtH^*`G5|?&WziY>l&%&hdN!3>=&6 z!|U|^L#Wq;5Bz&f&%Y^d2SozuUD=#xn~-68OOTJ4O%G!BR6en;3U7^PMnoHl5M1LQ zFa^2CV6adg@=!@`*!f+`bhkfYg>{6aJ3pBIaFU-JF}y>UzSFRL8l?ri#s|2Q)I~%l zbFW14`USj!LP}qz$T`3}We42P&yoJ2n>;eJe}G%MKemPS>7GZ; zG5smCE14Z|>w-iEIkz8%IG454Be;ofO4u3mniow+TC8;mcRwwzSV;NK#x|5#Jep0X z+>NSuTIEDOExPjJ^CcbJt`j`kh14Rs@Q!Q#0$X}aZ-D9p{q{V;A^zn3Np*Et<3_kG z=KdxZ(osu5MSGXk`a>J-f~|fw3SVOf??LJvKb}Yl#SS-Dz3m{Nhy;>y>?2W&7GeJ< zg1mhfOL>wg!i?s0EYBZMpf5O}NN^w=65lN3X7(v=`fX$cSy{4f>2p?*#^QgFeC`H|nY{?2*W!BUne>r9Ea zEQhMdC;VrdV)?w4vsh~&*wtg?`{qI`B|s`AFyQ_|etnjOnv}JR*X^bJWCq;i4D4a@ z%DuhINtvy41k1Wf$js959uzs!e_WEUdnKOyl#s-RDN_z3di)C7(-yqN&wD1#Yn7z% zmZtC)qVSfY;QNWzqX{x(2Ui6xBJMr*C|^5elx5J@h-uVM{gkV%6=1*H-ua$=T4su) z&DxDy;RY#vUGRcpEiLX=K(8w}xh_LMV@W{}SFv4?_62dZ96^Q(4av)B3*W=28 zpKmL60`}6_k`^t;+C_Tcy@F*)*;_K>_hQs&hKOeuvmf|i__6K$`q#J~#E{8-@~uaw_pQKa_5Uxf z|3^iA=;7k2wOIEx+4gkR)uEwm@w*0g6x_06W%PI5?|m{%u>Q^A6|{-FH65CoLCNYo zj-6Q#Qe{7BojwK0Cwh>`n=J4rlEtiuVs?OP3`AB!v`&I@1YWt`_!zJ-yj0SZ!Vn%ZDW&5x>;4IH3ZC^cNg{FQ`?!AVfeR zr(}8o9zJd^BhTPb9xzU?$UNEvAdDtuyqXvfA0s!LM+;(PMa%de>Q~Z@_^+6~IYByu z`#L5p;($7~6i8E&m#Ccze_B@Yt=ZwxN?@C7ri{Be9~+ zV%p+)dC0QLh$=$Pa%$+1YP8{aW$CxZuFA*{OqJ#8D5kVWH50D#a=JQuS0^v0sq&MX z*Ol?z+Y@(nY`zM<^iu0esmWoW^^YWrGYfVV5mgqZTQtz-J^(N;CwsHKa=|2puk^Z{ zVn;J!GaVHwSpw^2Wz%94lPwG43X(Qz6uSyJ+qe-zLf6l3E}a%WYg``J8!NKyXEmNB z4!xK%0+mp#Ai)0bh`-+rB(Zg6L*2N~+3RFgHj|AiWay8I}Ib|nm zL$$2tJU(PAQ(vv4A%2c#qTtK0GuMIe_Q7;he@Zp%7%^1zrbT#o`HKiVNvtNi?%lAa zrn%r<-2gASCkmll4AVcA#yCucnmAbPSn(Q)n^3ujNIZkMRhU5m7Cwe3c29Kz^=+&M zNP3QITxx+dj8UiAKvZkjgKi)UeL;1>sl&|G94-|_`OwITE$VW8)rzIjm-yvn z%q0_J#=`qojDEuq>8op_*u{+Hy!dlcC=$w#!=?t$55n4=j5xa4D0Hz{@D>)SuP&s@ z{OOBfgFlObrP@I?e?);MQ>Q6e#&Q@6{xKhe#Bn;%M7Qv*Ic)9jY-Uv9)-pcO(i_ue znFDRL5vgpJpKKez<{qWes2BiiS8gai*Cw)Y>7lL|)7}j&26Pkk#E(Wxv9#Q_f z1WJmcghp$0+KzgZz{Fp@>fP@^I}WrnAjKYv$M8gh z_qW$YxzRmqP>Wh#bvd*KcndjJRe2<3S+tIiGn!e+nLg90%ybTFWQUTq!k}qFtal(j z+xLYz;r{ndzQ!n*dur(L(2zk!?pP$6ZUYLJv!S-9s_hc_8Ji`Tj7}zQVi+tC9XqnN zVhbG?>zZ6Zt$zpR@j-es=0oLg6H$x7Di4}d%WoP5n+Be)=JgOe1IgXIaR*U)>Umba zzA_xX%XEw4Ov}P0S|%qIWQ-shAE2fx2@1R}sf_@D$x<1!3z1L>y#C|yvQ;ax7nex{ z3Cjr@>d*DK#t;5r7XYSaZhQ061V%0ioTI%To`hFY(W3x;u0YAMN(~yQW<3}TjPUOW z_N1)*LL0L=W-=;;am31UYw@}Jm2}GaZCj4`;bqkiQMZv) z$3Q$YqOdk@o5Non(vpD{OE8oS_nU@~tEKhUAKV2%gEwV0jOTv|)Q ztAnUhu_3eyWTjqWF!WA)dK_Npyk&9GI;>K4J6GOvPpS-&hpswhG4zwFtdXirwj>p* zo>97LyYi{P$5KibRSjhAH77Hg6Ro%Yg@QILvci`~6@iS+HKPwEZoK+q6&gI3k5E@TmY) zH0Nf?A1Ec~RDLTLUgcL02;zu%VHOd5F<4T#B^BA^tbANDqs*Cu&Azmv%rgQ$jIp|h zILmLWW`+)IYP)D**=%6%sysDgTkbCbQD2yt*ZQXg3jjgE13b2`ShK^gdEit@1c5Sy zmJuAV`@tKw9ng+Av7aqshdG&jQ|i9&1q&vbozV>dw(;V$Irykocudw5?B#%WlK5uy zY91zltcPzohhwz^maH6hFIQ1J;n`l}Qdn>to<5thL_nM5AK%pUtWdg`CA}WLi%~iP zykYpuXR4BI$)%;jTBD{QUnQJ#GIEr~ABS?9qt=)ccpaYivh9Nz^B6cy`9&m*Qa2q82% z8y8HN$&+8f2g{tzgh_j@#Pd+UO>1aJrR(zcI|;L}1WTE%>lS)t)T=HMof_;6d9lW< z5}qfQ9Tqx1K(^W)eAU<&RZgg@8m!tl`C*IrnOEF8h^nqJp?ES zad(l^8rSiNpK1|v^;{t_KnXrMzIfjfyd9ouj;PcfN^B=?U#HH(o-Ih1n!R<3I3|BB#OaG|p{?jO%meJkKgN-??vLVFy70Ef%TTj5g__$Jw@Y{>M70B@Bs8O zK2_V6abaIk`V7R`G4_QD!^4kV5?IBHM)MTp)j@d=JU+jFVb5)OU7QI?p&D~Pp2g7o z5KGpxp^NtSI!`XH672%Wcze3GDZ`z0!9%2jLasfgKYZLvr-%Ku_bO|&zp5-9;-205BjEfYzuRyCGpY= z6d}a}N+!LwF(rLU5(6h-{5gS(>R3~GFVDwlT-ffs;KOZL!z!?wHelTjzPH>D+7Htq z!}LMfErt(3+P=)zdCMmr55V0y^CAz}+LPgQyWNhIh(!~wSlZJg3+SCd+oY*A#TKo} z)%#rSIv3#%rHCgnz|HsiHuq*ww${Kc6Z#k11+cqXkF{$N41Ym`PYv3^Y-{AnT3aIv zDR3oI7SPqa(jn%O9MQo~urqJWADH-8gC-7<@wG4r&ij5lv_YnTT9ePS`c4=T(9YQc zPv-{V&;>M@W(Oxg#xkF1NMAy{vU=rIgVL{`9&=1M?Ysc5g}qSs-s@$Hvd>H3MC$YS zrRRc2xZi8FW3#~BMidg{sTO>U9${??{__2@9C9Sg)q+`9Kx_8%hjpUP)%cjsTL;<= z_DxDf{e`ENsM?zGN{|bzJR}=$cva-5oJzGN9WYshbx~ZP0V)ifO3d&Bs6bt>333G& zh^fl@SFCjj_YJ&Us<*g{e=2beDE=H*jH%s^d`>KEx-z`P6q4>vhrM;*R~4il_$6Cz zbU!uf+dZ_~Ps%-tH{QJxVA0&yCPHg$TEXfxO9#j_Q48@2&>FBUQ2sb@5>7`KoHB1p z1zv^1E091bb3ZYvKgo$@5>4c{hCkadNu2!$z061iDN(TTp-F8ltTzroPPCnMLt5P$ z2dBs>PzfCv;w!9BKs~bK*^Ln4y-EjH#6+mD#Ue7w$oWX6(L_0DS4E1R=4!bq)s(c_ zLxQh!_v>5%I{6cLg)`Jaz?rw;c=_5TIN z>4+G2iA8T7@V1%X<_fbg*BXRNP{jX!`x>iTyUO)gH0oIAA=Wj|KT1CfL-9+;uE`G-`k1Yf0TH`Ku@{I+t*O` zq)U{1kXrew$qmOu64muD*B!p7L9W)#YdfLOSB(qng!uBo!yNblHEGk2we?;2BN9G) zxkIM(U!MxSGJ65Oa%EFbW1z}&?C?K=JQj|5d_RM!_h9J?dK?0Ej&C2RWb^2e8^5Zg zi}8(@BFOrJss)ic!HR7Ov_=qdFlRqT0|2(KB$xf=p!nHEIeLqt&!4^PzysnztCIAS zk~Zv-fVhDK{r=_MPC|Tep>7yI3NzQMHMYv4A6wmpyVz-J&>+hn*}FyC&&*O!VgO=; zpvu^`tY}Jjb>hZNbV>UE!rD8n^3GUh~L;DaodD{au?u0#wJ^@!^-N zO%q;Tz=TpbeJgum`rzRaelB&31DotV=tOY!0ZuNY zS)Isqf^Si?q+0&D`;G9N3}?xCaR|c`8uIn?oSIyT*x9G~c!M50`aOW((Bq9D;te3X z)5{$bawkT40HxkoXpbr2O#!@N*c>r)Cy70vD}?Q9dPz)C%UA2W@wT8nRr9pKL{Mr@vl@I(4z3dgd!VX z3@-d`vJMH^$9;QuMNOCvS)WTcj=qxCH{jJBGQGFJms^41W}@ttY>k>75N^1!0VW~C6&Lp7XS$csv_>CwLoyd$cUD5+_ z!3%1kBdw(HC~dTKPLNmTOFTX0UXc@>Wn{54I-@Q0YUi*iPHiTtGhypMug6Q{Rloc8wMvxU+SJ@a-6gs7z8iud? zIzxNkje%NyXyb0C5OWp?YV;gxXvppA+JiJhSd!S2oQJXB1ybUe@SihtMLB-e8hqQK zth7Vyq2RrQP3Gc=tgL9rPtC~9fqM0A>eoSw<;YBCu?Zyff`Jf}rnNJBRpco{p$beSaFhKxfvA?=)5qy~h2AyXz3pk@pv z3was5h1))7(gXm~rvtg1`KU(+%X?15Lbf{(<(5B|EpbsJWb)Rk^3d@NH*Z`8VH|s| zoY_5C_zC)4j5ngqp%lV5ZSni7qvzk+yx|ur&wyG{PRHt{7f>}{$`;?CoWLuM=NwVF zvA|{d!fJoGF?re#U`|#ACfz`0i0)ZYx}R?hT@J)y@h{)bx@iX=r*7`&jQzKf%i{+C zYoXN-hSGL>sqs`Z!mfAKRyfiT`EiGgGcc1SE`+uQB`)Nl)0axoIrvE0{ljFAY=ULy zH%Dhi(A?pGA)802P1a8HOg8!*$JAoN>3>uGCyX(>B`w zQ0^(eQJer}MnZ@V7e~<7h<6UUr7j%@N(JPbIqRv;2_#zSNVgz@-1^V1WTxBFc8 zDP?od6slATZ&sQ-6S0&=$mkJU6jQTCZy=k>xlVyZ^T7ucWaM=5? zVOHkdyX+m#95eGj1DEBOxL4W%-GNEW*#CoFa01GSR0+5*l$5BslBnKF*H($817 z8W{fRwX{Z(>;~PF2Z4F&@(WGFnqyjWQt_6@dR91+q%0HaKd26$$r~z^V+C{Gpau_0 zP^JFd*J`WMz=bKB^6d3B((u|6+LHOMA51)w4nYJ^aR@Q-999SsFR@{myqjGqL||k# z3<=UFqQAYxK`0Kc?1w3+ni6v(=DYrXV#&6)>h#DbVsZDHgt|3eoaz(YQ`95RRnK(^1P zd~GqrPOqaoU~R+aB==Z{tL^Wjge~vR_^6#z82CZ4+t2uhH}B*-ibbNr zjM>x^k8+!I7rAW3Z)2OLydN?_FV+(m!n{sU!H>rLz2Eth@X<8&vIbz1Xb!ZF1nxN`Gc#-xQHKb z@as_p#3GF|(zN2Ab2(MXxC*6Cacn-93g)F6@hJ;SB(GXwn>_CDp(SuFKg?#8seM$& zpj_GNN4jQ_jAN9QdU?}JtwfK&^dmO3f}Erlh|9KEI|i%5YnNnCe}zc@i^Q8>yLWe@ z*{s(Cue1vvew5>j&&$!iH;hl&c1F%_{)3l}V$TT33GoBt3Coobej&QG(NpeBsjXpZ zdW?b`(ebU}b6e@g6Q0Uo5?%=hVo#EZ6A=)@PpW`Tn-j4B=46!dn7J;Al;}P2Dq{xz zX#8xTcuh2#@SSlJeZi-p0g0X{xtJg$Cx;?|9CN-r!m7K&sjVi-RQ7D(NZ9FzJX&8a z8>}4v!nib06i>%|NzN5zug9o`0MRazx@fRtMj5kpi!H?KAf`gZR6L{86fdWU?*P9% zK~lkmfvB>3AQPe=hTiO&grb13TzQV zi`LySSHNowAFnHrs<}{?FPnCC5T2te7&hoE{xb6I$=mF?gpvS)=3FfTn?-t+xH;rQ zkBuOPpx#yan3OYZ67hcEYM)!<7K0Mt8k96LVo8d6hF89%!$Guh(cBjT!Feo`7j?jH zqf>;eKi|Jm7-Zr)6nQBS#!2ytFyqlD=ITY-0w#M#BbSu5E-)Gfde!Uvn^kctbs4ur zY34OMyd629Na)y=@~`TN;qJQ?d^$|QD^b_XvJ@Sp)(qr)HqPw4)IS3T9Rlx%6UC^t z45$PD{2G8cAnnv5%K=zYMp24VPDLqSQ#7U`bV_CX6bV%%qA{UHt%_-1R0PcVWVHnT3Z`HY*Q22Isso7m&a)NK@Xd5wa7C4OE0cC9Ju7!D6FBQxkbfj*5!fsy zD@Rkw|I*4X@~uLv^it8jRDL}31xQ>fqF3aWnF6#n3!V&*c_+SY&S8|~Fb^9R#JvIP#kvf+OMHz!M7u0IYjoMRlOg5>@s;4 zxJ>gsmvM@H%|sWxjz}K6HHvxZQ!C&nP0olLReh{&m2=b7D&!_B7x5=eAIY7<-Wp&1 z-zGL{dD(Im{aExA{h0Wa_0sUm{nBzP{WPn-OhOh?RplO&7xi$PBw-swF|1|Sl2k0x zRFiib(Xg6TjM$f``t7o9K&$Sf^IW>)O>}IHwb(E@mf)kc&v}Wb*GvwyyrO~64GK#i zvu4LVXCbukvAU|`qPv$$hY*+0hvIqXDIOjf-`?LaKzzh7gS<_K6uK>jX1g(m5Vn8( z6W7j#Bg^_I?@L(4-S^Q3-QzL4-Rsdt-YwX~-p?4sKJU>A-DWYf-N!LTKDAgz-q+Yh zKHtn^K6+UDHn|R2oBe6}@T_EOB16VrNeqKI*2*i;(dAD>M(te_X$H%f!|Ui6$xk6e z5l@Sv$)6zmBwnQq^IIkp&Cl3Q7kU^@mo3pAPh3O5??y)XpCJwni3Wch8Dj=z<%}Yj zM%WiIt+I}ynWpSZFr~8FL;;O`>PG|7hq6J4E>ldZ?AvUroXgk@W|di-_~!G>UgYj|78pe=kCNa8NOe6y3S+*u}y;QgQ=JYFi!!jOKllHF}lPduGAU6nehUs zWA8U|&sFEZd)zm4PY&i`iz2$&@nVrFtT#4K59iRvz;40X-_`b0c|j+K^H^qxu{Q)E zWwYu1(YipCulU2A$6+GuL7ij=Jp55|@%S()7x_wuxIgV>J?93{KKFFS7(Jz$Tgf$S zVcj!WqPU#po*?6bnV|S)k0AZ0`pw-qA~%JdEfmPl(wL$1><@(^L$Ma$wQxJTzdYHg z^WYGU3>8wZ45lc_H_3^fpZ|j~^uV34LG8yFI`?A?75(4v%l$w9r57ii%l=Rghrj2m zHyczDQ3kjuQd5YeGRq<`NQp?u06!x+FE(p%<7t;yrmoayN~h=&B)34`&~(!V+>JU> zqELdF7uvE@-)^+o>2!R2KH&0#qk$poWLN6q1{(vxAx$~tG{*GBY#zv9$n!9XsN3-_ zSF~M_Nf1J(Ugr&%t&Unyfc#c;+xH4e6rmqB93*UwhrWUC1(XT+`8lPz-G}=#2jB?j2q0VfWk%HWmW;?yDZ zrs10aN#W4GW+6JO#y#?a3SuMxp;sAHOlbdyLBZH%(gIDr6aw z48#z|ejMPs6lA#es@Qa#KY`)dUp?HoQVeoB}S#^D6Lo;I6iC>s`rN7Y4aq>2i*GkP*& z_sK^yj5?+hz!0}4^o||$n@?R40|3AGeZo>AE}xAQYc3Q2&b3BbMRbdfVlc;D5X-zV zCHojRM|#ogp_uU!X>sqsCcq7;8OPXR(03cHydXA;t-l3l7ctVe956I5N7~MSRpP7odpYbun|JAE2V_+|3;^8Fk=wfRkZenf! zV`Ua~v9|s>{y|q;n;88^q||>M{LEeaUuXY$tpTbwKSLJ?zEiFRE2^5M`7#!3a|uar z)rdC!z^))M*oo-maIF&Jlr#;_fMlu!u?E4u+Fd&166WA3dhb9?UzGlwkGYld(h^F1 zXU)s%oK3Umr>n28usUkI%090z0q9XM#aV(6gkt|nb zF6$b_anh;%1Drw>X6M=ihU(sf>$FbC-I}ZtP9^$Q6>T+~p)a;nLXQ=@lIq6chuF%F z%@&R^IP@dXDTCQr{+!C{Qu!^D_z zw?UvPqL>+=I^e^|p<&-k^?ZddF*DI>o zk*l_{q3(V$l%6K#(|A_~Ka9SpVKL~T!89I;X$EmrUA;%Qs`FdnT(2czd{dbIVC{ku zQn_X!8Jy0U0g2KCqZC70{V)X8Ra2}pWLbOgUd>5j9#!<9D z7z(T|`yDdl%jq6&|I3e{!Ev9Xz|T;yGe;oyl+WnTD@;(#^{$@78A2(@iaX{P7>weJ zig0=W#(E#SS0tsWVT#Q=a{_Ft4HgH9xY#mRJ7US-l9?gOUhFiYPPF`?tT_P+e|R5B zwuq`5tO1cfJ4geMcSKwL2%AJb{bYUL!2jL21|8n@&HnQSwS)WhOZI_^p(94i|HPOm{IR49Glmg^KQ1~Zfoi_~g#85bv01h( zUIMgxp9$%`>Aiuh`8pvH?am=I!E-P*nqBj3f0<6_jN`5`dZpnGa`$ zVhg-D{E)j5)Dm6^(EdUYMQFve#v~yr4+M{EQr&-OKlzVe!Hjh@>~| zL&NHUJK(g3n!s8F9Hj?ZOr5u-*<>(gx8MfMWSK~io1WEB>jW5)#+(AGC1c_u=Q>0~ zImPzk9!ou&emvFYl1Ia!Sc;-cbN66kP+8?!xO8wbn`5H;&B?f#dA|A^bBD*%d}xK+&ZI*b%;iYi*E-l@ah8De@V7U@mopfwsVx~j;a~Lb8Xq|;Ja!mz{gLh z0sLB|HIEx)%{vd8nn!n`Hf>mpz)rt+jnOkur5@#Amq(zr*XGSQ94*eEKss)PBcnzN zK-65oLgDn~BB6Bq7FyqQqIm2c{iT9bbo7hRev2rKF+Dzw#$zU@s;C!NOrM~bZ}wei zcXO-h$=6w$6ktGCqe3Xob`E-M*e~v4u90EYLMh|sj{_XA8XIGEMm(x4PxttD_5!@= zO&S?{${3|k!|W=F;;o{$Tu#lLoJz)#=u$hjhJ+x)*DQ+3=pUEdF$~6N*)2GX%8)$` z%>+)AyvSxch9#FxWHm7@ikFdPMLD0H?<1TA*rt!RA5yhSRE}L>4W1CW^Ys=;MAGTwR)9crv=sfJt3xV0sLage~`i# zy-RFUNi%g(aF%Yvt}AL>uN=QAOs8%_z16HBw9EI#OgVH}a-_&PO|B-G1OC}O4OvcY zFvp&sH)T3lQ%|E%EV~Abe4Vv(=CeI}fL-JlkwsA+dsIf1UGqPl)H$=^)FeURFy3Vjx6sT1-endWl*S5r3Ob-2X|9Vw z_HVSd@?7R4o*l*i@R^nwvG@Nr<;pD${Z0<*sYi6u z-*uT6sz67JyaEvrV+({z8w5oh=}S8&OaOwK7nvX_4>h3=IiY9G(kua15>Ft=LpOWS z?|prpMv&7F42`b47vAf3VpOWJ!x9_&*8jC^?h#;A__*uYhYw5fvCGfkfBpS>F9UNt z+lx`d7%HJDm#?BNq&z@RRAZzlO-yV07okyCyK#Uq&c5;*$KXMWNM=q-T|ESSF zQc=dh(dtJy{;d9EHupbw|5>W5RlSsc;8@?;B&}qB6a?h02dUtZ`IV|wr4aDBBM~65 zLJYOmGY_z6|8#J5U|aG^@(FxjMTcmba(;H`^d_N^H$FS)KD*&PAAG$& z+x3972B?6vAQO1!@c+IVn2qfK?-tJkBZf3)$mr#X9(2HmM{*Ar%V6D#gp@ur80$@i zBw;9EoA-QN>5UH15Y%3^WpS?=v_pyTP zNEqdCCGCjQ`TNh;mGA}yI{G%w-cWg!3WK;vkz=^hIG9?JnsL8Eawvtbwd(I^#~gCN zKFtkTrY_0APM%)g9pcvLP1-e8`b1a(cvFMe1$sRzu?+y|aR*w0MyJ{b}mNihi^v zLF$ArAu)L1;5@+@tE34*ImOKb1{3!yraZK0ZdH_N+tbvNj7b;uaf^>V^ecFW=4@+Y^ zShh?tyrrNG?6-FC5|Ub)d^0|kOUcveK~!Y}it#*Mt29-+1uV{h4Ou<=cO23c9)Yhi zNWnzc?$mduk=P733HD2{9y0F77{OQT;_b#@CB)P%`n=8GV`yoQ(b^b@X`UdV^}NG; z8sTOH+#)>$>1g$UQPA(3|fxb!WRmcXc1;NVK#huiYc z&e&V--ffj4SJ_W!1zvOt7@NM8fl;>b@5lk1pKMYK64n>+`rQ_K;upkhuFPhLFMdao zC*<~)cw_BS*MoK=I|t6l;;R*aOqRx+d?Dci&;X_-e1bD1t}~{NGg#L`)cQgk1J+Ke zB9t5@ZeNc|SmW;=e`at@N2J7!moQA%-*`J%HI0|Fz9}X*lpQWXq?7bb8IFmR*BdcA zjr8U%?5O&sg*XW~v;uthp7xmJuR71DXo%ncLpUQb=Z=;u#IIj9%c- z=dA0TQzbg?Cw6~W_bGlxm;hxAnO3G?(N^#|1JK=&_)Jj|Zu1?Gvrhq#pO-H@brdBCKFbY5fP8K8 z8DN8}h+5y3Bi5M2z6n7hOS5q;!=r%K}J|QJ7JrfkD8I0SS z-=?5IwgfV_oF`0qC9>W)1nh)oH}eEkn!2!tiIV*(9yXBklbegQm=5}FogR+qF8*Gjb|s%6kb&6@ z>M6-^9_79GCTJB*XZ__e3p?(RkfUl0fh0_9FLH_pQ8mJsH@mPvc$vkM*^VPOg!$n? zw!l&x;u@BHW3CO>vgeNnZLJ*+CTc0XGc;(X;iD_joSgM%Q(~PivD&Fz;)-bAf=ZTD z$V@gtW{0suSN}D7#?8Ywuu_wKAX`!fG$yt_vcfoxbv<61#8|m{@1VN?ITQUdRx<8f z68cE?l)|qV&fG(LZo3lQ)PJy2wOERG`d`ucYg6h`*@C;3&D>X=^L|2S&0OpOB1hDE z^5;z5lq8xO@3T_gjMT7`*z489HsBTQ43KtC^$05%(Mx(yp%HLp7>eeZeG#1%WhEi< zLqOd6Yj{qAx!cVVXFYXbr5zpe2SLTz5_1-ojLB9x!Am@c8v69W@i4VFx$&Y+ELNP2 zre!?I*>El*_Njy6^a_AvjDxJUBl-9e95FsiCC`J~priUmz^2p)kNr{!vPZ{nim=oB5W;dwsy|COF_a;{`8=`4**{ z&i<1~ZnJlu%-3HD+}C_hjNN2!*j;qzg9y`ZuW8LeLf1FrB?5_-ZYYe*H_{CDBU-rU zWCg-Hy#ubVSSg(Bay3KTWVmA&uufZ_v)49 zZzGLjqA$>kHon62WAPKavBKRDN>TGjmI_~HR7IjjUPU{lyW-k!R=Z@-B9K9%NM{-h z`CGVpZln2{6EyVGaUBag357&GH zS$0;PCO`Bh*3bi_Lsf(RGCJ!m6WBp%)X3H>^weH}NXqm)ze%&4=>=U3iY|>vRhboA zhJ%i7&1bZL*=cDbIeEI|eRH2>mKex5$ZSNx5XhUD?C{1aR~%eJ5z^Tb?x7^`XbV%3 zVua5)Sp3jx#*ie}dU}Po`Jb)&B^1f@(%;q2x!ilFo(7>{IG^?0iVxvzv_-E*E;Gen z(ZitFH<`T|+|fL)Yt$^NLG9SuB`2Z8Lqt|>ifoqcQI%8c1K0ahWqH)nw}G+LCaQl# zHGjB#DDbtB<5|wk<1|50v*BdF0{yHhUUa7wFj@1EfEKOoJL3~l2}cm?+RI--2fB|S zlVtKbC2)lDPC*0T$>jg-7HU3I@miI^>MlvSJ1p-=RPAN^u#i%F|G+YrfjymZSsj*H zV?n&_j<(U7%-no(I>29JS>pQLfm_Vn&S>oPib?j%ydiKcn=EniWdceHAr`=5iM(-C z66wcp2$q_vIAcj)k)_60j{Z;&Z+VRmuB7izvK#}V3pEYsmRGP5_U=(uF!V9Gf}mG1 zn0GMuxMK)?w?oYk2c6b(V+NLc({$o&2J~V!PkLR~G(t0HM%}6Gj5zF!exgnv?9qqG zw!bQ9YsQ{?xX%3ouO0c$hUMpl<)-n6?-I){N%v4?XvFFrfzhSKd_rQEre8n6>OYVG zFQ~-dRxuAm+jG$)*P`%ll}DJ51+~9~{LuR6z142IWQ+)c+qY6AI>qy@8zef_>4#)+ zv?*5LYxWQqz`TPq16>3{>S<4caJZa2Is2^Ab$ac>?^_`D-PFUjnL?+XuHd=f`3$MQ ziWpj)jvv)4Zo%3rVpkdS_B-*|4C9{6wWbPfFpRlZ{qEmQbCfk`r@D>p0^;b>Cx;i0 z$AY5^&-CPgg@P;K7)Ke6M4U^1bDCdX*A+&X>M+ph~l;8aYkD%hT_DmXC2BlSwmUxLYXi*0|O%ah~o~@3I#WR?R8?};Evhg5M>2$$I|xK zEjiB=G!pb`9T1y`zS-T0Y7bSl0)?gT5>9Q#sEeF@|M#yfy=P}--5+W5hV=ge0{%ZY z<^NAX^XZ0ii0Zp_L6*3gEh9+sFS$S-_=@b$co?ipL=pnBgg7u#08w7Hb*QmEQ^(Yl zXo1b=ugkDCswLG6bQSBWVj2qTa0)}4dUY>Zx#?3m9v+_3FR;%p8lSE7P14cOL=+A$ z#(Uq*ci!u+>8|T+?~iRa+F$B1av$meVISdrclF5%`tQ3xY2u;>BmF=)edfR7c4mKd zN7jxGFQamo>^SfRx(ohG^i?(z=7pnX-xH(x119&%fG;;byesJ~RP#|1Xa{y8 z;=^6J!@+Tf3#Q}yr_b%v9gpq>;hzGZ?!fcGhLq#(bMXf}Nm1%yf)SK%e5Ef1`iJF37!baviMHI#n6F`w}?Y9;WF-kB9Lsn!@OLZ?g5B9Xlv?xl$ zvCPJ(*q6aufh?E5+Fa{(2IE)b5j@*ONYJ*lR z4fEZi(t<7qrKy}XPz$Qb=M<@MYlVZJ=C-Jp37A0eA**{8H9@xM*rxZgpU^NJY&nNF zftAAln?NPI%X~^t8bskPHMl-kW@g{~%g&KkA-=yg(kV^;CN}ckU2V{-HE+iU75z3K za(hn*^(XmPMZ3=e6@M3=igq8g|7b|0C^GI=to&Ulz-8HjMwdtpz6J38i-xSuG=tHI zJ^PRO*^;OW^^!rcIfPSQ+Cr9BriqY0rD0;q>}G}ODGt$?47E=PS2ZgRW!GXwN$d}7 z-8EqPQ3;exXAMwX*vXyl>h&n3yvvfk{1Ny<0zDEXV>o6u`?xF%>qH?qCKet2FJ@`7 z`Y9s?t5_yzfbo@98;`cD+mu__K9Ux1YorOj)FuTN_4lKKnYnH&GbXhwm0osmznA2r zs3}Gw9;IrtvEVTdh6ik-0_ybo?v|xifA+G?d-^p!yd72wvJRLcsvTI4m+;dWi96=4JL@g$nh9-K>KiURsz{BoBIPaub2pH39^-yx!giO8-5_CDt$tN> zRi6g(t$k))@V1tMZ;HpuWFwYU?^tLT<9oC-(5u1{+#3UQ%NvtF z{htH%=rmd7Zf?CgE5F!jBEsu#Nv#iX9Fn>uPA&PZt+_#_L8&yxm}Pp&XBrKGeFKct zCq((_S1Y<^qacptQJ+IQ@$)yF952SUrwct>P;kX$rXz8Vvc)d195xob@b0s3PuR~6 z-f+%%CJ$@Q+(pWWYCjsR+t&6a#q+<`z&JL%5%b+moPk>)I)S;AJs`{F4?)q!YG}1` z3-7o33m`W%MOWANc*Rr=@58)r`6=x4H6EFNp9i{}RjOk{Er2Z#R2C5Wl~IKaa{S>4 zn$GQ?gJe!Z6sA|ktjOqXV7%Z&LNQX~jmM^)uycpGk8Dl3M!J>c(73drJw6R*4_hCu zsj8;tJGJNcGzrcf|D1Y+G#wY-^sN*x z6j=2SB#!kjqD^aN>yZJ++N?mXUu5xy4^zbrc|#tu4#WwBt9ClpE4G zui=VFiKoX|j{)1OW-L#V<^+dZSl7nC#ril-?a|-t?*wWd9CxT^^X6=92|4=H<8Doq zVK>TWw14E;=CFB!hB5;7DV`Gm-w#=Tz!j>AKd`c014TV?Wr7K0R!43{m}LX$dAL6k zI)U0081f2Y^F3jT#O)_v}y&dBGFb>gu!EVUd86iiPFz3 zaCT_04s1rSC<7Ze4z&>tf@7T#m*zLYf@$Dr<5MU>c(2Q?tW>78U=t@;MS<^2uv--U~T&0kB2AK^mslPM+hzuc2CaWpge;ZgkueOKoHUjIKb zqy9_CsI99hFJpdR8koYth=~NQ6{;2*4gleFB3!ylP!JIk=>`$n7{T`giH4=0rh`}_ zt&>dhSX@QSvZBvJ+ZbWV5C^Z2)uZFjB|l9r)^NL2yFOO?@#7i#ZF4a7=e2@2MGfZI z`Mr1e_4sh)`aI4K|5DwbHVN;A$_MH?JQ4Ea*kdo-m4|?Yfgq>MN(ybV??xBIv{o2} zIE1jHA2|IhAI5?czg<}1mK*vmR3=z!kB2}!Z2aQO;p_j~#9bT>4{a77>M>kC4(c&Y zJ}wNQ7n$(Sa3WR?lmTYW(oOV)mx;acOFYc9rMs-}I}10yAO>CdUKEz@+)YDZ3g*RD zGJPTx%Ozz8ongtOdJn3^@H>&jxqSw8e740nYj#rCa^glAg7e#c`Wbi#%krE=(c2ya z6Jq^uWL2_;{)YD|o7-Tc)q*a{oMEz=U9 zwQyWRi>Rd~nu{^7l{ldJeEGP(JId3#tgaff5El{R7NinnIZ}z{>xq^ii15KGyb|46 z-x(Z6;vm^9;M?f_!8T-SPv-2(vo&gCHqBJq z>!Bva05*)dyu?tnn%H7}%-EcD?y6q%F3UQ!o4MM;v|*SlOwday$W{7|bb<=%X^D!m zI~Jf`T8LO{$zHS1k~L$mI2?D2oFh6fC*Ob0xz(G=xz$IBYqwtt)gpa~vvA8aYc=4S zy*z-0%QsBt3wNk`x}~hHJ&+Qs7tvSiAh}!OV9&DJ?4jJ89P4KM$-K3Ai``ke z8y(x|t~X5kp*sA#!4-~}Q#9mFzKkKCH$i@C-;L8Z?iqzuwo9VpVnKlVD4I=Tc=$zl zw;B7QF;=d)7oEP(hS^0+W;DlvYhLo8f}%*pZ{}9)Ln8n2+j)q3L8o6avUG$0gLgbu z8leZN;5NXS=oqlcEqQC>N_DE9;QCagOi>2`8AYVWueDgrH0b%-u2 z!j??s>zn+2IgQo?f}62q;K~7>-pG?Mrif@p)SBjYTCDbQ^O94z2QQ-*``_6sSH(L{ zBh*H!UVX<3ONWEFSR$_+Cy6A*%Q5<)u)G5nICtMSBmRr)+C>@2v6vgIoEoi+(vJIm ziQN&kVsN?vfvnY| zbJ_pGJvqF$hUmDZc-mp-bU=ECtnPT#J?}GnVf_ld-sx#Y*W2~4;BV0cm8#~i)r{c< zV18DBIV=VUS+emxfstDzDfu$+T<}wV$?D;*K*n_Q=9ikfdWTO>Zbnn^>&4AjK?gQ5 zug6djn2#7lCxz7)R}ex}_etAssJpYg>Vf{xM5@^?MZQl!~? zu?<5&T)NnSQisxg58t4pw)Kr<@1HAFZ_S9xBh#=*(itWzoEg+kP>he!B2ZCd7aaZ# zuf^&9nBE<~);sm^%Gc@l+YFt~5o;a)yI|#nT_NioGcBsn)3XE-t-r{wYV+rSeEKVBUJ`Zcy-JW-(fp0*UccOt$l$$4(J*Sr-=ip4~fHy|T9sGKaBkLdZ$C!IO z`9G-_9J+lZ-GAursCs*?P%`;Qj%EjSf&asFwY19glj(*_~z(lwjHel zi-(!st{FjLzx331HQg&G&D#Uo!$XKr?pW-$unaKnqAM2%9D)M)osNkP?GS^6yFy7C z0k-mAR_(omt}}yAlSy|UEpu1J+=mjIBIORk?+8x$tNrs@HLxPu8Qu>OD0EdMD^ij%D57Wh#@w^y}N6%Pvtw(O!Vz$`S>`T1Y@7t;Qu`V(sO zOkTxG$yl(>*I@X?WAT|AZU`g!`SXpruj$lNfu=n5J8#W<D3#sXsi5qGPD1-!J>P=8lP^KAS=pZ)`=A@;NA2^NLJQou`2OH1itTz9x zMEj4x$f6XP^#?qLXyftP7poer4cn9T9Ola*McXjK`m+VFe*V>_K~Jk%Np@WT1*5jB z<;L3X+=B~S+KsDB0or)1^-1TbRfnKm2J9xKXIL`q=w5mrIGA zgLP?;QLA+nA%oiL6Jdjb1kKqgRj-tDv{*33T!(fLu=naS_IU6ZIZvAQM<<0l`)HVw zW^d))=_zF^K^}#P0oU3~Hwey0(YjIw7<>-%c7PAki&{VtN^=XQ_2#Y8*WD~JZ2o?PZS4vM;tFFP~h@8Ou7+bm>6+O(E5#?`1fqDdH(M*ToQobvq|)pWH*8x z@l$#`1B^l?_%631g?^f6VYWf5e-x1DKZwB-+dk507Xv0xqlYUl{>}tD;@eY%R4CpP zAjxx>vjU{nXlfn)i;p@&XrAhd&>X%JK+cP&bAsM-yosM51jBSi*mn7aU_j$ z?~ZccMR62Cab%6+bTwi6mIJ>AJOySJ0EZ1=K0^QR%AH7rzGVdQ*RN>GU%%x3-#rC( zjv@v|=Km$#|8w&{Y4<^OXm{nqbl)D1L}^BjHUy*{0>_?UDHu=)MvCweQY;|YK>t*@ zQ9>9pCI{2NJaxNp_m-xV(2Di;wF|0_!6k4CEwqk~)f5%0E|tpankA{0_GYTHrKOAa zY$sE)e?^chK2N*5x!2#*Zr?rYxx~4jHwgX~j&8I|`w;|KH^z9rp`~{>PTaV?Oak0d z#}5Y{Jg*`M8t#PQlAUQ`dV8cgx^O#cKp&j`Y7ZZa1W-4*?VTl{wZCsNe{De&pvava zmj+S{WC}$0X$dIrUvht-eHV{kKV0Azd$IX{!x@6=!tC%Qe|eF5NTu?D#>YLQ9)m2N z^b@G?R?LXCd*k$yM}G4ONDtwozA2~jo!*%(zQ+N1mvC~IXUft7>j}40J2NOlc3iendx%Yn4#J zQ8CNr-$KdAmr3*7De#kl8z&HOwGk#@aH-IDk)hd@L9m6Lv#y{{#)n3h`G`0fv1`s^ zScNZyU{mkW#JCphX>)Dq(?t%VBiX@?%0&qikG2I?G$7~Ywkq!EAyl@2g|Nhzbu|@R zWaH`~>MoBY?qy(%s)p_(ZhD3Js;JSoQfH#x+B4VlqS-a<12=MF0rsJ>`Xhp62ApE` zD(ky$N&p%Mif-w7Rcc15EbF)SjA(jOlUWf~pxx??`pSAI2e|6n7_wxSW&Gkt@WcB? z5?!jhqFj>d6>V`8{2ivx7O66>@#;zXF<#123nlcV$qUvnc2H_fB*ZbOi=$1zZHSNB zGJmw`(pZ#S%Pm|EWtQc)5h>`|4?5PEI;-#S?5N%|XwT$BKb@ zV1_yiDWEG^5=D?8MvFUQb*-YvDniL%F4tBf9o(eY5XHM6IcVfcgjNYl>F5ykS18g+ z`^@T#jW(aPuFy6YR^(Po9KA5-OC&I~EU8*SpKE1cOJp@w@?;5Y&YMa(QMs5t+FzDF zVx~D!7Cm9mJxeZLdnwn?@j>(F_%!qr3|J~X(v)#OBGdW%S8Nm~ zj-}Lw0%mzRcl2(E2;E5m|FR?W86$u3yQldLQ&akZ`%^;8jMsxb5Ri(GJ#(8x?wBQ0n)G4~2c~2cYzT`-hO)xc#I486|CU8EE*e)fp zuSUxBPLwWryOc&P-Zw1&&w!^ zNVDd*Gp>-VLJ1iwfuUicG^BVNWvRFbe@OuC@WJU?W3745IWNYAM9MVd@iNAZ z#MnuOBW4bsKaMhKWTvT%y3VB3pk>qxXAL;A_FVjl z5WTb78c6mAe?6lrCW|*wm{!P6$EEvbT8=BIn~XbXPb-dO(xLjh>((Me9eequ>q%Bs z&W5)3CMK;Kj#F-$jt6zhe8P4YjaRdB$W)_7-W-5fYA%;?7HbW<^@<)j*J&Ay7h&P) z#;1+ADyC_%l2BhrmmzPb(Ok%w%f;Bl{+j7QaPC1yOjg=BA9jnlNZDKDoxWC*rgIE2u;o;#$pKT$zlDJ9lWKC(PH zJn3H55v0yT*I<;S7mp^nbmn7w7|ViBCTJpW3qFe6Vc?=v$;yge>Kd;v4?h-s*Rk8f zNxq~QM9b!6bV9f1u=Au^nyWtz(hfPSaYpIq%W~P6%7@wE9zAq%CebakkUbt=o~)+S z95=Ji8`x`bcin{5K%~4PcT$YZcy*>3trc-iUweaHHSnct2P<6pZ&epeWK_RCl;9At z1|o9~v3s0-FrHgUj(c<#<2VO$fuJC19d;|cE#wLbtO z3nwRgZ`(XY^Z};@#s!jfvtkyLfSzEn$QCH z`=50#{DM)GPPMYNZA%*Tt@!E%(&onGJpi@05qzmQ(G;et+@vM6Iz+>yka(6O{ESEPxr&!J zTpB)D)$O#D3tIRHj9J=b2iJM&a%D|;56s?f6?Cy033mcZj_SYvOJTGD6%p8i6b-HukRXS$)klj$pk*ZqQ zEE#U<=o{jWN!mSRg`9mD%d{?LWKAa-yA@M1`GcL0W`(xW%EGu;F@qRyMx?c+^OPyp zZ*{aifzhu@0DpdP zwyKTl`i|@KKO7%7n>3>g3m;r**pP-=cc z${eD~Hs&_w)BiB6z$G0^RKcYJsw7)!RM$CoR3`V~l-PS9=wn&Zjk3+82*jKu$VbM{ z?2fv85Q-{Z^ySF)l#bk>;9VpftF&UAq6D*-1*pWWHS2nz@I>{=XT~i@VqMgfNm&?Q z3{QbG4~b-|dnij_i55Zzywr~NvNzAiyL*^RP+I_ZnTd!kZ)*0@3H}sh9As5$#ZV}X zhOOtj7FHU?Bl!vhAw!{Z-AOyqad{vWy&r0o2z`s^5hRwq5I zl_BzeBp5LklLMl!HbD>Bf+>DUPv;&~*~$c0sGF)Yx`S%zG2{ zkY3IE8f~yEV?3=o*8{NkxK;^Wt@1v>SNGgD@v3z~+e1_Lcs5Bd_8mVoLagifsJuk2 zNHi-0Ebd64rtQhw9&E$XkEG3;;waU*V4Tf}1Ur^IEeqV<6noeiybrs~+dzY$U=m#a=P z&WBg|lUUTM(j2hqr?2fInx1G(pt*_Uv`tJkauBastu9OZsT?oyt1?rrd4QkTd`|Fs&3XvqjyIC;eToL3s!7+;tqW5CHJ&WdU_8uOE2Zqh|beJzox z0=EQj&?-MrkOM-#b{$HMLj`SPiFb0&zuL~83vj?el2yI2XH#aP?OIW%RS>IM{_t~C zX`3=8F|9})nYXsM<26*4go^2$s6UrwFu*j!2M1CCoxzl18#j#4MJD9N8tnU1Kpi z2siaaAk9|DjZt_8Er|N%z~E%i14Rn1pdHs#Ga$+(b0IkQrLD@b+A!5J0!!qHry!Z? z5R;`BYZrsXQ=yH;*JKWXlg$&~!c44}QiU()qTSPP2ve^NUKtd3a^CNUM7Z1*!Zx_w znt5&cacMEc^(;DCHxYFz5_vhVK2pqO+3lhthWVx7d(RnzEUkIsgUg>;oUPAEFvYB! z#_3!=I_@aBi7-TV=4f-7H6vH`)VEbx`#EjHNBjFoV$?}iwVfOnBPkrl;?zgu$cNJx znniFDPLzg4M+UmAxMY$%h}#%StDZ##7;**6?7~Yz@a=j}B>NUF5->)9{IW5$%_b}7 z17Sj`ikgVYBI;QX3`Oa5Mq3A!G#~dU^>V5p`K2(vTYI$f-V*&pC+mLv6kv+$9$(xI zJu`Q%_z|}#O(doM6MnIDOFeS%Y=qP!CyePKn1cqTpB+GT& z-EfPszrEUcq^VK(U3nsQPYTpQsq|CIDDyX1+F8ytq_}AXnLX51y6<1TDLlc8;JO+w zbTn^!I`5a?zh;RxGSkXZecIx+O{f%Uro|>+I8c*j#hfqMH{>N{tMqr)`~)6X+RVs$ z`W`Rwp7Uff{|4{xWS*79Zet!i#u&Vlolfe3p0%}2M$05VuB9H6?d(S zZ`l27h|qB<@&V<@uGBVH7n`q-E;Po?tDK}hGi*Lvyl1C%#VZ!X%|4nW`2G(dAHlQ) z3lk_15G)K35ZnJ3(dfUl)28JAlRO%{lNUr&q!YgXFBpwRX)PHIA>t1qxdttwAV?T~ zvvH;6xDC@5)LqB77;@8pvqvYdC0itEu7>HDexLyEGieQ6rY|EiwmZOuweu0Y+_5sO zk-v*kxnYPxF~J-3A9v%Jk5Rbc3eX>*vdPu-*n1{L;^@*11gJXK5Ypw{t;2s9c7#&F zeM4Rn6-sVF0%b}MmLyYpp(oPvN==NwS~M3TOS_)N8uF?WT4u=<%UN0K!P(|pvsIYE zes0%Q&j3a9_R?xiLALO!lxjgsqwmX+BX($wS0q`%7gVj!)xE2CN0 z97Up21CXSNJeefw5>id+c7i;$fY3%$&^J`zbb8HsQ5MToeQPfAb{KpMXMtz0-Wp)c zXd={2qQPk)SAROq0bDTA2A~$MpXVKy62jDTKsaU=r%B~iv!_p*kuyrI`Ck2%4ZS23 zE3HO_;#PztMVbJ!A%p+U=iswR>~ckRajh`FRq?%F{iU;<6~*9nI=E)<-Gr#L;!}>d zrZ%6D(6}<{7G+<Y2US88x{o>FCYB=FFQo*w?yI^*PyEj9k3Pyk-SltLP)4#yN!vuciDs03 z@gC4!Id1ffS}tfe-~e?y%W$*QMihSCT-ol|YIQ&Z-S+THihpBGKeP$#@`rqixl& zr`AK zVPH?2jsWTvs(Kxmsx|mRm}#Bq^j&JfM%4QT*rP?%;`a*f<0`acv)SO9{f@74>qTiiiycFmcO=Mj`U;Re@^&klD)86DOWe&QfghPabn@$~{p^xIuxLQaXX&T-KV<$rUMgO%q73ug}cm zsLJrFZfzRABC(#z&1!B;nu|1^S{`ceq-b;B(l^U#*rb@)JZ>(~JcgZGJ!BPfT4rwJ zF*b`H7&H4S#ektIL2k4_hrKR!e31H(n`kdkbmC#TRXo(jF=gM7bAX`%2y8Hv+_%SV zLEl*f4B{A*G=A(|#tzzDi%aWkPmdmGY}3+Ucwlk#Bh=DNkWgyQW3M+CqiJu+Ewmp% zoxq(Gmu_^M#6gRk^tE51JzKCj{Whw^LoI{!$L2oGtrHr5yn>-?RpJsGmeDXZPs*9I z9x+}m*9=ffnxUAWcN66=f-71lzmO?Zqc&mOw5Q%c1X~g|z2zE$88~a7LwrlOexc7) zw;o*HC0(mm|CIyqH}XIS*p~~&n+ep!rN^;|y1U8Gq@`;%LVkRp8?_?Jes|$wRVCD$ zpAf2 ztn#xtyKe!DseZ>sbt}_9utnVF$Y*QSmkod!ZVuf9J<1(0sM2wzqYGfzv8i225*Wa z#{|sTM!|*hK}v5@dvps|K(4>caiwy9BV6)I@Zyw|>|cFF1AXyl7EZ^b>I_%lNy=#v z_v;0EwMVkR^_Zgi!0bTvz4cV|f{%f{Hb7Bo`9Z1XPUmJgft*u0P|tNxy@3{r{hq~f zi`*R-8g-OdUXk4;K&;pSwSAEO&QJX&eZvIid%+=*LO>PMKJ8O?D9A;v*~4kGAUc73 zL;}>~s7bhX=U)(M|4nNGUN7q7=RzU;h!l*BJqGO5#9qI={S|#hM8VG{WW)-E5DmE7 zP0(NC!wC;ExMv$338l5I8fy=xggh|QTUjnC#R10qG4`i>N55$;Z zE63m2X`RJLC9+A(gEr#xeu{*8e|%o^QVf{o{{a=wiiAV!2ZA;Xq)0VFoMcQ#uY^S7 z3btO3oKc7FCeEl)?-0}SLz+-vq1B81joB*$vj>}hz^>IPF)Bc~*e}uvo;Gf)7g!jZ zn=kl;8#xboEI}c7=dKWDK@cV=z8KqqLDcCh_DwOL)ES+yGCy`VJ{Ay^ddqoX{r-0( zf%GnXrSb8qNOxLW?(0A1y-|K+NS%;CKwm$Mr2jDl`u}aS{I89%OV!#<@d&e9mUmiW z{wLOtWl^+9IEz+r^;<1BlwX~sv56m(P!9{R0ZLFt2B~|#*msPxk!om@Uu%$*r+9*c zkBJuBRwez{YNhFF=F6(X#hUMBV#ly!#}OcuKw4|kk3BK7w1>d0?fc)g|JJ|nFX7dTfr*NABr2~Y=tOUKn z&SfG^nZuW4tK1sGUl045O}o(}+d|#t`nL>@{5*wrv!Qlg2_(fTL5}eVI;ecYhaN{F zDBdH(Ti5|g-NgDj@R-A|Md!NsoxT?Xd}F}^3jyNW$lsv(97KbOopi%+KmVsnjzK{A zkm{Rga<`G>0A+aM|I3U6-S|@B!H&7Y^;+j8p8__L{%WJ#-x!OgxS{aTuI%mD z$|Wh`PL2mD*3sWi+D(&3v{r)X;7;%~=Tp8NviThp@DWCqCAfew_(vGj1kEyBr7c4Oc2pQ_PFrc6dJzF%vAt9 z`@!7SzV1E~Pa|0;pOpfPc^I*?1P&(`S7^w)7)aaKGg>wMQk=GJE=#hKr}DdPWoa6H z6vrr4cshm2my){p)BdXaCZn9cZcxlQr#BEETfnrvA!{+gN7fvZoLz-s?c5eaI9G~sV|C2=)PsIaGYtmF9RHaeV% zy}gU66%lN*Ds90FVTxmkb&f9Z50b`B%uFOA?ESxZG09>hqf*eWcwNl)f|K{`_tj=! z07%PgAdngHjC~58{AM!)+ZWR;tSZM9P=l1@TKj&8mLIacJe^9uIy2vk0f&lP9ohwo z*WnPIU3N(+;YN!;w2syw3Eli_k^h8_S0n)>({LtJXRCjOj@Cdddd_Y;dQZPQ9pNr8 zHG5sxP=xHAC7lmZdeUnZdW?QRU=i{rRa+y##QubqXK$;F_(w-T)qkx;mxUZqmF0wx za_C#tI2iXJLqoofPtf9z7|FK_5_fY#8flJ<5B4H+loBHM5?=zP@yHA!V4Q8okXWVf z#@5Xf$X}o~H*9OS#c6jly;M%X?aV1f)IeBq-!j7)iSxb`F}7$999c2)YN`%_4~w{b zmS2f}zUHwYZu64I!%XSj(jY`SnPL}&jpMS%rS^cV?7RZX=JjOOURBJFsga5x>FA$J zw9qnk-%CKId~=jhWHhOyvk|qtLnO>s#Hf%d6DL(*>=E*Yw6DZ5G!09E%_~LfN@(ceuxmF)_XMOJwo4%ph=J$MaJF}Jfs^t~> z-fO49T+Kn)bbcGD4Zq2Cqad*$JcQDY4%RT+nde6qBv z3uxp*4(DWd^w%8*R6r_V7^K${Pzz*bV9LxP^-X9BZ_HLYq5c=M9@aV{WUgz~2X)-I z)v6WFHDTb!%;7z2#lN1X^c5wCJ5=lwkTXAfi1hGDJN>vicgWiFc@0h80e3_4GwHff zoO+o^Jsy-7;e6SVrEWZ9{k=7?csvS1td?+@QUi|aIeX;zmMEa~Zk$um7g+kz0oz+8 zroH$KFSP-K9SD6H44uv}^;;1WlqE`_Jtf|lxh<&6+bd=cHRSB%{Vq~W!!GQn-bNm-i5R(2?`lImFVWE z4fv;=n9BX*JI86FVzNLN+Zrd%V8>PK(=`C+OTzu^&hN9W^S^Oj$`UyURM@?*lG~D< z&-ENX^*48#?s6JuY%fl;ftmAT@F6pg|1ALz86p})oC+ynN=l5)MBq-198XI{%*=YQ zi_)JQpm^Gh(>daB3s*rcRIu4T;&5SRdf#ioxv(rMOy_9>1+j=Zy{X0*srXcowbrcG z`k*mu4nNMjc7ik5;1eQiW2OMpLg8GDtep%ek8jiCiGl0msv^z(td~|ZdDF%kN^&gS zLv>)MyQb(9Q&TArT5KX*d@*tQJ`+7|6|>ooAfT(ve4121**$$z4<(05(c7L0aydxd z%yrt56?smp#CYfb_^R>V9^kVfI8*ld#8h1J$~XGudV27%I?*ANDAEf5*Yxsx>YMJC z`3rPw)K^N2^<#DrGow_~*gRtXPVNRkM5VR3ZH^L>;U-VShmt(X7WHYT#~Y3Dwl^r) zsdpIiqcul<)*{bQtE2O_NG^x*sgY0}#Hfr>8kr_nj+SPmMYc<53X3Mi)r}HiSFP#U zF))=UAIM~LF>$+Mwmk*KX%ZbYG)N25(XhT?)UAXzX;&XTD`a=<-;iy97Br)93`id< zlpa}8bUikgv5dPLXqKaKjY}xO=^wKa6{pZg#Lz{pVOw5=xvkI)It)CUo<^L=^%*jLxYi1wP0Ni?jQwu{;% zq(+bQ5#)tdB%gun_Q32O1XFE5%Zsc0u8cWJ>6LVp7V##iKiQ5r>A@SZ;Pwaq_$KrX zli{x}(1VDVHOz5bf2tqiB*YtcqNot>Bp>3#%;2XK@MM7&lmkPLJVt{(*n^Z;av|bm z$eXIcS5)dZweK4e8ydxh_-%dOp5|u$U_JF+3kv+e!+(G?eIE&>mVbkMzqKuXo4w%+3xR zP$7eH|DZN9B(Y54tY!%BBQ^$o-T6#%j(hyPP zZ_8qJCd`d|WK`g)9pA zv3ENwMIS{i#osVqaDz=#Qs+v=n1L9gkMTKm9oiD6S_0hDzd=@zQ(ISoqRWWUQPYS} z1f(3jdNlvk7e3}+$fGkgk|dc4h!~e=r7KemRzr7w+)>g+MWsCy(eT{>q?#nBN?Hu{ zw%jk&N&}4b2K&hwe+^Sd8us_{KqrhsMJf&S_Iq>J&IX!5ugp@?YYkmPF_(W?4WnMR z9XYe%XM&zaa@b{pv!%*w7NQAO^#(1!Ez(A*B$3MvpdA;z1UIIu@ftOYq`Lr;LHCz) zo&&K5nX*&Mz;ZA^2e4mV-xs6J!Vc#NIbSbR<#K-I*(9HkXs{OLE-LC0bjyY1W}8ww zHRt)2IAi#VY86*1x2Q_^OrQQXcC;uVQ74V;HU79+VA{ z zKR(DgJDLgGy4w6dY=m9vkX|TDKYguM{jpmFgMWY*?VvM2*>J*d-F5@j*)Cs~<;Vq)olyek{l+xf?TQ^lzJvKHswRBllDSz9VJdfLq z8Vut1bf&#+zjPmbd+&JeXl8r-i}MCL4mUqC>nqZs9)R#}g-Q9a1Fu%HOF~yY5OQc0 zgf`Vd{Qy;W9iez{hKx2TdwTv@x6Aa=zS(!6FW85qByyo{9tFX79F3rNr3Q5z z-I8#4sn0))hrNu>cpi;^6zacFy@sPZ-o@hJsmxNDzIUqm`r!Y%4fXH3Kd6A}J{}^x zOjwAK6nl{pf*;+x(BTfgIJ}K=@Ir<-Y9B%6&fAp$Cdt0Eg13|h{##-04BnWEcDW47 zP(X#SoWVE7IwFS)+74REn3Wl)&d8kp($KV z{A!}Sn;kL{la<+J>!J+;f|I4!g~+-Ijb5rBs`W%O*i-7FkiDd^lpMeK?qL4m+883t zIl;T;*{C#Wq+RY(C>gtY7WCP&8r1SzfEzk|NtHR|ko!Wh!WM>9Oqxo5n`VTIM62nC zWdw5@cbtE%+h!_zQk@CY<)A?uBJ5Zw8%$Fx&s4h=bQ5?H#AKVAjW~ETKDFLja%$1Y z0f0GLW>`=|P`7Bvz#|k%)e`aS?2tR7K}AA}M}TRvelOf8P?^iy>^Th%-Z|-<7-{L} zesRhrwj-%c_CDh(C{oi7f(tnj>cX;gET)0+soD2rj-K^&07Y&3he(~$2E>x5*u20t>!;K zOkE_+lE39ao!M1K-5G&b`b3y1dqwoA+;_)8N_8dP$I(>&1mjb_W1_e^c?}BJl{BZ@ zRi)G!tWo~_)nlvzA5TT2UPQ)B%CzMP*`s*p`6)1je@6!XSLF`9lbW8yQoaR^aZ1fs z-QX0G!r}hxvtt{$ye@~h&_;a1tNTfJ*=6$*HM!BZnQ8K2?>71NL zT;eA_YRzsBW%7YGNJBW+xl$)(of@P^GbRs+^N=kh;kAP2KAK1p=BWGcvnOTp3exBlZlD^a37I`(GbgQEixs8ZCOfh&{naiZ74RJTKxg5 zJ^8GbITjb4;_#Bs3u0hku=8$|BWu1a)5xukh29HsRzc6*3(SY=+>h~*bB@?+Y&JOd z4XQ?5v3}ti2Vyc8dpId2He&Ov1C6Ny4-`E$<@t(M!@403djgobtjrc_~ zO@t^Zr@`{u_N>Jke2Wu`8d7jzO%?wM>od(atQF6mK}Ux#sIMD|j?&f;6*Cc;9GV7h zh1cpzFt8lC@CK^fq}tbUgcxpSobK8+?uN0%AiyG@2dd5LDrqBZftWE;d*+0d3V1U@yAU;UKmrg&?d zspV+qZ%jb+(&0w3V^~$VDT&niwr3=wtIob{wrVad zLjGB_lMc>mxO{QutO```kJK4{FRz(GRqZ*r0Fz} zz(Dxvk;Y?B6NQ|9y&9MYr~YqRL&@dLqF2Uc=1%@&+8fAO-*}5G7L_P8c6uia3Z;uI zbz1EO10AWz5lC?!_ykbmEk0cZzj4v$k}pb7OM4P~h#^%E+A5B3OZAt2XghBrw_c6L zkUfe!5N-uW`Ev)lwc0=lKg4o;1kxbweNz0h#+DGm6X@;iD{x}69^d#ogGCnci;z8D z0fZc24;F%8C%%q=q;Ca&DT#jAT?t0d3KFiEUAwNU^m2rO z)^rItawBa*t_lC(-Tty2#@Nfl*qM&>a%k|$yuXkaht2okhw}z`;bJ&>-j&+v1Hdol zKChML359yG&Z#PKf=6rvo!N6=>@_rOK~lz{>j&H3xMdhg8)ae^Qm}H)aoR?G)`Xf(z# zG()oK!K%aVs83hBA=-WyuC6xZt=0(O){SRZU~i2P<`nvg!tLZ>_YCLDWTpCJp3`$f||H65?zB z0q$*?B8KW-NLIEj7r;9TS^jwhH)ic-wLu44dI`zUCG!S0Se#osh;)5)VRtVs0;TiV zxyB^G52Q@k9@Ki+VFs7XMwh5h$?juuX9OD(_KEk|S3!rwKsJMGo_Vnd zw58gTaF{!XRT*P%Pn|d^XxgzDLz7;D(znu09z6(*ey4d#+5 z{Zq~(^#Awd`@d!gs>os}KEYrh`rg2c@1jyPbqR>b-d_ztGW)7&q-FVmY@V;wKH-{ADWDxkgH z2!|9YmJ8KV_R~=K#Onx#9>@WEBfoWo5y4~hn4mRh#(04tH?cKqyC`#4ck$S z(P%cXETpbDX{t$-q;{CvU{j&7g=-ckp`!)d&(a67lsp*7qhLm(j;D(4t75(INOd08 zw3a&(RTNuEG1V`C&C6t~Be2fcWiUOGNmEtagUFr)YRuMKX%Rq`*%~y`FUm=rXgL(v>WCS#k?Ev^$C7XH!l{K9in~W24E)eB z0B-hN&BYakxo6dh$tE_*(mt3E5@}w_#(BQDwap4k{3 zWw7LLIcO;`;`d2>6mB7>DSz6Qp=fvW{b%TNbMbR1TVb+jJHihjI#Y>G90+A+(uB5# zofMnj^HRQGz)4TH9th>IAmDvAB~AI5HsKCK^iO+Rdh3Z=%DJx8lIgAJScGJ@`Axm> zyf5yT(6N&5t+&heE4w{?TG(KQTcrd;fG2tDuc zfZ^|Q`$w=WH_5vi`AO{R0^k|pF?li(reJ#qyKqKkKM6s_s{_%oBd{uyMVG65Q<~<0 z?eG9NSAzwXa<>TX0x~X!pbQewg`!7cgS#iT)5@R<ef$R%Cjg>|^5rK|Xh{tO zWcWYcDF6HFFZy3WLQyMyCnsYY;{Qt#_}^duKlFhnHApX|qo(g1Ptz{PhImLV^w=e0 zBk5ItX&`l?K7`oXj@{VJK)*3*P9}lbb~$_VCWWD*Js#_d=DFpUL^NcD#qyf{`Iq%> zjTQ^bt#wsR%c|z(bEcb2Pg7Dz7*gPEx*o3=&6yXz?;Ed^Oxx^#mmx7Ase&yYbCD%1 zjk}om*+a5kud<&h*nCU3OrKie-^;hq++>D5iFGF|+28)ybnSQX*mNCtRIpTc;V@n2 z12uQp6grLvEsQtLupDkeLWtW=2Q;t+x6SB3)lMv*@LzwVwh{M!s@nfL9B9KD-kQR; zxy^+1>?+6Ic2j>R>Ju-b&<2#SwwFDNHyjb_Wmn z7pFsEAfdjSfuY`!9X1+cp^%8dK}+%T|}MBVh|cMy7;xjBX}HvV>#{ zCPck^4oxdno6BD9BzLiIBt_GmRJ0__Gn#J%n}QP?7;6i37u)g{kC$9Qz=1ZDAtk7x z#bLSF&}2)yQ_hn?yr!s(sPkpXb5mO{1B8_FV`OOWmb;Osn?(<4PO%HrM2Swl_KTCp z(Yucg9|&;j&;CnAC6lt*94BKujZ-_3Z}(ebXlY?sd<)P|FgGDYj259fzpNfg%U;~0 z+=C3vZ>Pz#&I^AwVpovQXkKCYHucbVzVFD$}ac-0rn3xBdw=A{h=ELJ%pap;x&H*z;jP8^-{vr)oHekFg*RSH; zh=hVt%q3gL#>Q+f;GxE1FQlM`{c#bk&!boD)hcFsZ<2DP2vGCEm5Hrqi)-aPrOLvr zn+u6#Fz~6fT!$*FWSlhJ>dc{8#y7ip7OauVA=^-ry=qP#B0}9Wu)v(lK`a$Mj((I5 zA4e`ct3>goV?C7r%IH?h$v1sky?qSt7r~)7~Rq*JfRuo`A_b-_Yygo zVqyajGAong@d1^)M3M}HjK3oQz54dGV__NAmxkyz$Nz{4vvm|Wg-wrTib1z#3$R06 znrMbMB00Q#TAOGL5^4!i(9o|D=%{kk48s4VNn*^!WUiadTh$pQ1zQlzQ*x z@Dg$xvpz4HX0@D15ST41PZ3?36B+#5q+0Zlh7UK&Vl9U|&v< z>71z}Ek<^|bY>OT9f*cd9z(Q$nmLetP|TT?EZ>!pe9`DXy<|?pa@r_4ZOAf3K#kKJ zR5kfnC@cB1 C+H*U*dJ8k-LIdjImVzTDXWS)#1TQPnB&S68x}Rp8m8%hs;%=Z*sPZ*1aVNIqFc{7 zMb?rjM5{QP=FpwOC-MVPBq-L4%nGbIwWXKLfzi;V1jD8-c@O!UFB*laH%K_OC3P3i z7~3~xR8O6XW|@*)&L2tCxero1N%i0B0_{2SDxB6uR3{j;3l^H3vL3zN-Pqqj;;FyF z0V@lzoZd+`i{>qI4=fZm@%tte&$Bd_df&P^wSo5qqcQf!>sZ}`k*YAMyeyi&68dG( zSG#8{cNm&NjT8z092DdT%lcjk4iPT%IS3lN8inWMsn4{od+{!o28((Mn;Uf>)w?CG zeV!0Dn?!c+8`~??Z3G29*w+(zeFQIV=X)L_u1m+3_2@XP9m(T3Y5FS-r+C<4t|fVA zyf8#d<7NmL^6KNYRTu2PZkYxLnZ#gnx7OK6dsw8gncpUpAGnabQj?FC>W6AakFLg= zp9?Sh7Fa#>a#8TsCS<*MQ#`1a41U_1M5n9MXXJEM~s!RG5|;f)y+864Y5 z)HFPwV*{UT^)bJLf-wG$#eqRJm@i}y7N?l3o%}1I4!(RAcGZ7yc zB0Lcz1fl2Z{K`~TpUCKkEan(L2psOCn?tZ!(~D_;{9^hT;8VmZ@wrxT*9J1Kw`2j4Lokvo$}NTP@qStJ*K*u9+1Wor7OchcU7|@k z&rOg0!V?a?PTkmD_ZY@4WK5_tYm9?w;%eMsr@pS5l(_t7*Wns`V~V+hs^2%awTo^G zU|O+)BiIPW?5^BE!dbUP2@wl_VEz3h5>}ZRP0r!fE}cyu=sURJl&rq-GReo;EwKGY zNX5#0WD#R=I^f=+#-+n9%o#&MRZpAYHuV-~M{e(iU@bm8;1VltWCHUWsSFaddNLAt zrQU+y^4K@|DB59lb<)kjFRgPmk-Ov#Olyw-0~{d4qya?!IWoiihfmkXY!{F1TWl6e zFwt_M3e#7G^^HJIoh{LOxf~;fHXD?hA3LR8dF2+I_)W@egf)8z2}de7-#6tn!87;% zrxISM&(MyRiVsNDdlzWab5H4pZUdeZNY)*`6UrZM!2WkzZ~+7sM1`#yGRW2~cvdhG zjR1T*%*w&&8g!4B;|rl!w|g?XWg%z>sP-s)JFF`_)apG?a0x_EAY3M86XKvjxc@fe zzy@qjQ+2}&%fJ(cgOLS#cz{lmw-FedALgQBw-Xqrb{Y~%E+DG!9H_kL%Bz>ne zcEWI?^Qu}7Vx?ib-MXTA)`Q!ieUH=0FCL4em764O>Pt2Zn;y1-EE6+oUK_X1ZQ4jw zWe@u3zSH1rkcB@@C?Xujq5!sG2*)TbY{%@llynu!CKS-ONq!%+U!0yf7nfZ%$lKaF z#f0arIo85WR;LKXcJ93Q_Q0R52qk(%qC|l}tom?R_Gr?JwMDG8NjAn@)<^YO6H;r< z&-H;FYwy>_8J@nL9jnLLgF|j`8+X54c8Rq7W7=xaQxNx{D0`Ka!nBowD$LQU^Nw1h zRrWmAL$S#Nu_+?A8d2Hgao83Bmw8*QK{i8GPGmL(Vs>0SVKobyr@1k8DZB%?P((H}*7!H;g_Pw>oHKi~*8c`fQ0vSFV^&zqnUI+JlYx0S)bFs@Gwya7#N9 zm-{YHAgVhOts$y*JGh!1NV)^uhzj@M6oBAt@(6|%#oMEzl7_c?q%~Ug5o%kcQ+Lpx z5YDiClbkrfb9>R$X;)& ziXG#{9mA}zt$PngcQ2@LYSApOci;%ykX(s|5f%gR)~d;>PggxblcN7MM8E0{uymwoTeC_6b>Ov*d% z0_SRsUjE*tp!&Ke0Iq!;Ruz4G!Bx3&=VUtTw zFQ(OC#GO#((HHb>s7NS|3nV3Vl1Q20W`He|#G9H4W`JsTGQ!h(u4!ucRqK|mY-4tq6(2BsA!l?G2$&b2$+`=k)QiA9 z?!eMLZtnD|3;Zc70aI||#Bigj>r{05#>icVB_z|BNjg>78kpyuvWAyz0Lm&d8W_f- zAZnJRX)n|V)?Wv%L^w}3`ChmNZy20GgX%>vZ4{>pWjuRukX5Q!>fW7X4X9dN{F+4K zq3DOAdhwTw;E%ZuJqPNgH;pRBOrIIxOlw{IwB|#E1`^#K8{B?^L`+&s!1p%ZVtLt{ zPuA9vg3Z-^?TCwgo79Qs>RcG2PfXt0UDnpIh;sJyc9YM{Ky+>jM)m~)X|eLcB4y6b z$;)Ptpisf0_H$jL>q3dakgL=fheAVQ<+0vPQshrGGifj0Ot<6~Y+$w$&|#V8i|Uru z#mD*EbX@qNqmKKOD-^Z!Uz>oo-7C+feZY?b?a-Bn%@MD_V%MFE>u+*f645sQe~|W$ z!JS3hx_75zTOD<5+a24slYea6wr$(CZQHh!PI7bV?sIGJed?{+@4esFT3^?kHOG9$ zc%EP622BNv2Z2|ldgAdk^2j5&+<^X)-BXM=LjDrg>nGRX$mLKQO@#^HYZDK6Eg#7H zU)N1GTGgJ#pNKbXBmHsLOzb(zs4vh|1|tRrWU?xk6uKbSp~z&{e&pMK&M1#9 zt|T~>)O}DjzlgWr(dpo5?F3yRzpaW(yDAy49etSN<=DVplX5$(E*RZL*^eB_);pdZ zVmuu}@i#1q`_Cu?cqiB$0$4X9W&_z|e$M1Uy!Kq)(N=vrCr%wQ&O6l(@H{!-H~5bG z)(-}MGqrm~9`HYN$zF>0dW*y0@yI|=l)@m!P7f4+9>`OSR!;nmW=QTBLz+vle{D~4 zFv2<{Q<1H*<31#RmW{2;{yRpfSivPln22tga7TeTmhKRD2bWpVAyZ72JyTldFI-M& z8o`k?RF=pjohi1z=;{!rE)c(vsF+t(B;Oc1wg4if>@45f7Ca@}Erxo=lUBqdIloBy z^3F7==q+thMmtOO=vdZtR~b&1#h&Wxy$`6M7hqOi+BI74q)&9MG+4sj;Q_yYt4l0q zOI^j=;nUvL_qrb9scL=QvaIckpYkBC%sDUsdoExolvvdW=c;DN2KJk{Cts&aHQx@d z(hLwJpTNHFFn}!vn`x))aeRA#vh2I0D9YaAr&r0sO=rjK&zBH&oqa z44&q%e;83xv448PIFBu6v7I8rK9J}T3udZXUEO4RtW|VkmFSGNyh>9I# zn=}v!8YX4n;^_^soaA6g`w$VX$Lmk*`y?`Gw%rQKKq-P7THtNxG!&XGj@hxsUrxJ) z9%%jKq$n!9`NaEIHs0+u0_)x3?vHY=@ zf$~>5e-aD$fdfF0d8xh)$qAdrZK(Uza|5CTp}akKMqKrkHRT92^YD!CQ&aa@^~Loj z-XEX0$bXQ`FCh#VBH#>}B?A$A8iG>k*61h)GlE{}_*G|WO7ueYYGkV~t5<@Rr_joU zZFyDbv4b_%`)bUtEp2;11q{DkfvQy)jlZhtL5?NI9ndh`3|`@fLPA1aPbgJ<#Y-=qqlmLaK*65LAQTURj>EV@AU1C-Dt{MrB`Qq$0on}dn z;dsiUYSk-7@nl*HlpRIcnO9V+szbh5d))~qKu>4Jg^Dp6frE6|SnRYb!imuRwL-oD zOo`zztI+oQ3Bjfs4yo2u%&<&l_@ohfk7u~)q?F$C1jAPh&P%Vk*YM&QUw@ny>ES1) zHBWH;Ec%3NN6Z@m1hKkgg9%~Yt7c6e;u&DlBwPfCsHW8^H(?Pfl5Rpvltmr->S0my zIogR2r3o^Ho@RZ;3_Q%I3ToMA1$9QtLb1n^t{dP}G0 zB7j%%Wa^V5`U{$31i@e2G)Gqb)EiEgOb>0!l`a0CMXR`}pnxl0Vclnl%2$Ji%b{ol z4`jS|IO@;CfzDvI-x4|*CHJG$g?U<97 z!w{EIxVaxpe`Y#sPtt^crT;3@!sp7Pg9XkscCEI2z*_09x$<2v@qv8_h5D-q8dcUK zKykx}>Y@r7RoX*HaZ>_?t>QbZxN}T#(*jjd-UErsR{FD8ap$2?uZ?#%3UkBfqA8+H zE9GhD-;`WVZ-w_Lu_?s zWE?yyA21O<{vX{LNL-AcAOe4Y3HF$Y1TZotBJ6VjowWe&uqI``a#A zS(BFgsgrYOlUC*C=God>nWfUxSI72*A<|E=giReU*Y0PJs%Q4?gWzo54=TjTqu~vd z{I46?jIF0R&Ic^oPKj>j6JG9=Pg?)a^_*?Yhb+EptOs5&Pu-bo*f)PZyk8F@{-1_3 zBDilUJhA!SPqO$QLQ@}$IWnMACSaa4`<}REhIccz6SzyN&vpJh{XaQjY3NpNW>B15 z0o3iCD?5opOf54z1n^SD&9F32ved~ftqJ2?D0`6FG$wcm_JO^Lca?Xtgc`bmq~h0f zshpZ=gnuw);a{$aPwW5ZQG57Ml%OsNwrqW48n>&ZN zVL$N5uIXF?dnr>R4BbA)MCry9=W!(-n85aXt0HrUTw03hf3^LNN*qGHl_f|CBj&iw zoRkDDjodYSq$(-`&I}!J>(lBLH6Xx&B*s!HipNqJMh_+|mF173g%Ig4-B0}sKHwU2 z-&jn~uCBj{U$pmR!?X~4rG+P60aX>PTU5avWKax>tcPhrg2PXvV8oN66spVAil1}> zOPn&%1Yk-4go4D6BCup}d6kvH|G?Vd04hA^8gW(xqC8#7!WYr!4rQC~kb{I2@+-BW z3bT3-L&FX$-1lRk`b$Tz%kCE{$Q<7!{T3^W;%U{-MwKa5Bh=_@OSA$!)wz4H=Rj0P zkQuix%(iL_%;4rvJ~K1PYT)0b?urYTpX!A?1E&aUrb} z$}^$i&2pfup$93{(5Gl(gwj=Fa$k->(>^Vfwx(iF=zR)h7@p$;Q3Zkj=;SLh7>2J= zzyXc9`2tNRd|}kr~aHRAS#p;zb^%IPU{F$!SYBOHGgpsES1|yZ%D^APGiQ!ISnu)B*&SK z6^c=1TMIO(AGWt}MCr7ZU&e)Yibj?q1pykJ!rV?3BwOK}5>PwG#B;n2Axgg=_5^bb zuZyzPGlzeMYO<|e^u00`I~^ttSItA^gw;0Bqth_COZ7+^HMwUe*_3#WL{=?YVSLpT zP64okL~Rvh&(lh3Piv7l+mGzP>dSG1Su>r|w7F-ml zTHS$`?22hl@j$FLNMmVQMv12lqL59)Wj^%~hX|-n@dzy$E}YXO;jn9csERof?;1O( zoNG^8#{8=k-+argc6IkrF}iFwb5MQDOrkTqobKU|-6m)Afz@ezw_NxP)@gd<6#pH& zA@vDH$ec#Uzh`{M`sxvH9ebqVQQq5}+99Bt!D>~hfZcC>&GhP&dkja$>z%Fw&(}d0 z3t<%-((llh3}d(7w*T=v#iLkrNXw&nbbD8eWSfwnQ^oE1_Cr!gKmkAg{RJo<@8CM^ zZ;VZ|Mp|&;YgUMP3KV}A32$FCLnBVzF-g%Ia8KBv!i_z8AX2126a2GVS;)waCA?Ly zQ^Mg)3BdU(6v)vb(ojMq8oyhB=o+%{LKn_jR#~C_Yg}@62Gi#nasOelVezOCo^mPt zgE_hv`jV6i0r4VEz1(f7LQT@dBUfaY-nf1+$MY~|uHBRh^uvpv93*4Vl~$P=e5!c( z-Z(J;tmNIi-{{BS3{H;I*{2KG`Pkv9D22sw?$D7g5srGTZ>jzXx6wRT5qW-}URn9- zSJ-{wtbRA#>Y(eZ=zUYk+T9i%4s+w(lBqri=bhuvj#s0 z;p7}{8f9Z{0+Y6CL0^BN8wFD~tP4^Mn`Jr%PNxE9H=21`m7!#WetdZSP|oMfwuKuo7h}n!>_5mDr$BJC8AIqnmX$K2ZGCSyxuLT zkvPA8iRCTIqZE%4#U1>b&by$h!A8{X$InIcWci4>w2y2Jt=$Z6)SWkJILlsa4$Q0# zyF$!M^UuAeS*|XA{4W5mx3~nb&Q34UvvL}6VZ!nv4%DE=0%dW{hAA&98WDddeJK`2 zeC|S@qWF!&vB{pK1pH)vWF8rL6e6gcNSn$=UYkWMM@Mm*9Vz&G;Ak$;vFzW)KQlr* zq73M8Ds3=~cU^g!(K8K_1ai_2_Q59`9r%P_F|iEHTBz>B5j&A^7)p1=0H{GSX~AD! zG49BT2_yYKjkyLyM_Kbbf8@8Cfg7J2bBSp-2R95altRy<>B7$O?7=2!&xRBlTI0*P!Q56H|0 z1OZVjlkGHMe;h!gEU1BgC?e9;wAz7Fo3BVD0wo_}=TK}+Z5l4BO;Ob*8C~RtastW> zn)~e@=k?wc99(#D=>Np0Ton=ctOaL~F44VXS{w-st@;~zo1w4=yHRpg^ICCKCto~KBY zDlZLP;-jAmk_WM^JU>-G%e!pwwaSkq3+s2lv&<7Y<$7~JD@4zcwifX90ycUfbn=!& zpVv8|e8AM|gKJBGKDEn|RNJ9^#c{ccElMk#wEG92$)sfR_)!bS4a$VK_$7iN$H#CPq`f}mR z*^6}Atc(b4HC~PxSb@Al5d*i(ywipIb}i?17sU?+t4YWcAsl% z?&kd+khldaTg>$KTp$)?&WwNzrF*A=5gG+vQ>hFj7$RCvK!{ynWr2O_p`~q6k#=cPYB!$H8P5~DzjuT01elS13@7he%)M^YZz#R0 zBm>v&ET`pdrln=TpAP&E*BEnN}CjYEJJ#KK^=egR4=CI?|9u5A-2X3;YD4as^ z%Yb80_AGR=Lc<EFt)cP>owsQ#2n>4}jB6pXC>(W+^edgrXf$tvWgB=AB7nFQkfN8afnZPmTJVuQwAB z#(faaa`_RJC1MA-AYQj7$y^ub6uPKgQ(j#aS9N(#YvB5>YMG!itKya@Zkt$hCpCIM z^^_xi#EhK`LrC!E{(2*y%+r~Xe>368^a_=Itsbe>7Kr%{sy=~L{Y<}cwbpyCMr_#H z)ptv#JZ;t#>fRT7z-66z_ki*A$$53>3I6PndyApXi+(|R!^k~H?h5>zXutj7x!g}q zeRW)V!C89sbh^o28O^ryJu(z1H%?Zbwz33DSBFM7J8xBO&yZ-a)GXT^Gg{l^;&Tre zZ}fUHb#Ik-k1cOdS()3~3tc<*NVz&7bU>Gx4Syx`B*CkEZ&l}wDn1K$N$ZLzJ~wX> z=Jh{+_`Q6d-okf%n7X&`f~-Wiy>LqZLPN@yDZE`gJo+Qs%4PKOWHSUuh98$$o%wZ# zxw)Sx96-YVMw6D)>6c&#fAj}(iYKrhxp5qE1snm=V~qu5u6VY5#;TBvTfkc&acvVt z9=B0k1OEC8_+`_s5{D8h%L|+T*goa{lo2r)g{2e9xk#CPESn#XD6u7kS9YrdPs+we2D zt|Xr;7ul7fEgw20@A>N^|L!BbQHG&zt*8^4+YQ^wF|J8oh{qqRA&>&r_Le>^+Y6-9 z*1mp8@C(ER7Zr0VK+}@ETT)oqm{!2JNzp2}akXJPEB}L(ea$^Q`J(l`Pk+I zCgwgqHCv$k5@c^w@+UoBX#XhAW7D=^#016~$|<=3^y7Tc>r@a_fFG`Gz(kYvwt1AE z8r+>ADeEj#mPCCF?bN=3J!D^lrk^ZCKN6xuO-43!(mf)9jvVOdUrE260`>)(@^p9*4F_sDmh3;)j|vO66Fa z>Z7*2tN(-GwSytt{f;LtctNiwzp?g^c6QmF8?47N%~YLi z@@S}vpX9MwkNF69FI+s&G#oI9^YcWYGt%VI5j=L*k3SaH+qMtwjcl`&mUrwA#R-Zj zLe-7Vj#lTSxNgZD{wNm|*1WHraAzy#IXr*eCU& z(uOC9;KvXkgGzzzk7ftD*8{2jtxFFBB+=wTqN~;e)gM`BgxzIsF%mZV*xXj|>zer^ zCfOa|o#reojP8k*Ja8Dy^+z2|eB&##gCw2s22cZmhW? zA8J($HPBdCLS0GziAbx-(VamTckCdI*3?Za;g|elOgL;SP1)ZTzNx&tMWZ~yP5Bo5 zHh)b+-LlI!9Tb>Z4%X%u+hKNwP{U@ah9Gjh(d<^5Hi)`VMJlz@MB@0S{7RB`P7Br+ zn$$FH5wbvX!Ta~$7uIW7#!;hm3252%ei+rW<>(19E!C9 zjRdQURGnkwybj^{SB6sS=nQ_xE{MHK9fm6x?Ay0A)w3*`Mr03;l@>0M_{+c@-!OBM zWx|yn${>sBV7-)Y>Bix_)a)Y3X8cJT0e%ck#G!1;>s3Y`3Afn`n6Zup5$th?{x0(f}~L7AthFf+th zg>n^*Fv~WZwTL5R!X%1I^jj?B@tFBuHz1!MipajS#ZJIe(aggXbLtCCP&*`6jj$Ak zJH=jqL3%$(z{E)rp5~>~2=OMCx45$>51nlvbjr=w-n0LK_qu_x&?hWTCk$wKakH;v$D_ocvY7du0Rhp;tLM2*778WrFThB6Zg{wTqtX zwhv3DgF)bRs;XcllshgyVz}L8I&0jy>ixX>>wP{O=%V-905CkfchWNtnUW&@JJJ+Z z85?n8U^avt78ADKc7=@)$%NVPP>%)v8)P30 zO1}r?$X8;*q=_QIa~x-rg?hHn&rRCI6Wc?2OWx|C-fyzE5;zStG#%P=%*<823k{{N z=ViuaeuSbfg2NimaBu9kBz|})Nnw&XeJ^E3W!+kI+=A?DSk1qX-Uv;t1?9dxKl+XN zwlr3_@>rrMAh}ap=QK0n9!I_8;WTmofO=ncZ0&MeCl9>wVYDP?idbVmU-`R{aG1qZ zTz9|}$+5s;)Yim>o~j2_Uenk&Cbm3*YK-kXWrzB(t2t!{UT2)ED`(}>pQ66hG$BgZ z5X&>ZpgdWbz0yRg7R^4NKhx0X&wNUk$=GEQcvUu1S#i259d&S`+KRo~5TD=Pzt~V2 z*kO(ewICA-R?*8)3_*Os5mjt+BP}^A>0yI!c?fFC*N={>uT5UwSS-IbFe;($q^OWH z%t6{c+gN#`hITY>N*;)Ya1k{$%Xq}j$b7vzk*jq>1X!Q;vL8&>tF~}GKTGO} zp=~_uN~d~M3S5=3hp*l(WSNwzN_5t=)?Jz~DwuX|ZdzQ?g(M;3lv77;U4ybw3HZwp zM5KY=E|j}vdnxI|Pw7&09(jnWm;}qXJJD_$PAxS%6lUuoT35J_U*O=dEAgwVVxRJ* zEU5bQg`=w%wCOf>_EyS6BKH~2EyJ-w>)pocEvP66-N6Hk_RIy6mhz&^(&+pps>ETp zB#MXw6}PotZP^x+_S^+@d(jq3=Hvypt9l>YC04={7!y|2x0mLx>~J*5QS{)%6GiOl zOX$ca3h(@_eXjOkUX;!bGi>+5Ev4q{t@&SL0i$2enT*P7?TDpO+k1{UDz|AVUzvRd z&ZqM^F`C11{f(?9It?2O6u0T?3t8hP651k@&?vlT9PHC^%H&njRrSi4+2Cdvg3_`> zcYgSLW`I4j7Nbc^?a5r%GBOKO92%&3X%c1@V)6vX{${=nUw_}nwL8wt^nx@s)Qwsq$F4HkqA5 zrCR3HcRi*K4^1h}>gGf*8F2n13*fRjWFy`nUHqta$L_?(hjVxm2_+2PR6^FY#WrnA zl=xF`yqnUv^GfM&GGjC44Y1O(uWPPNhjY*Rk;WreRq7EIY?TjAL9Iwsu|<=Q)d4MJ z#uAvW7< zv*U~hXV|MF+c^+ECsc4r zW0_Apuw>C!WTZO(s*^2(5W>JGPEVYB?28W@b|zRmhdL9;Xswzr8%EbsedvgFM%JFI z`nSj9uV19ew^dmoh^I__5&#Nl>fO!D%x0V9Sxx!SU$Z+#E{$W8nq@?hBS=k}0tF!s z=Z+SPdE+iJ(P(rq0vf&C_$66yKDXLMFIxz1`ob5~4!>%Ve4bx_!h(5jqKjI|{7RGS z_IvMF6}0bJkF8)N+Q>uEteEwjh_0`VC>Uj`&o`mR@|ELqAeTCOcw&Zm0DX1qYe zKmn^Uxf8K1ik=(4EjTS8DvF$-9;P@W&smDquwdm_vLX)-U+65_OT209qJRE{GCb1u1Q|1>Xc*{mUI;rI>r%% z3qY#3nF?FO$RknuAx8BY*$TgM_#&TUqo#83SK6#|w{@KxDMZ)N%vbOgYlCJB5Fgjc zSIGr6aMn2E=By{fuRA>R6P{m_xCiK>C)tKA?Xw`zQUahQr|Ws@@W&kbbbEtBT(Ib+ z7+@KO2@lzE^)TQ6LI|gX@p#il)>H^QARCOyFb3-J+9H|su-BT^3A~ixJu7Tr*Hxl4 z=S()qe}Vt&%m|Z-*>Lf9nS1bkSpUZmz<)ZN|GUc7_^xtMhQH?5lm2T^q+=WE!S6zz zj4X!>ftdsq0weB?#d5gLcy)8aYVf6EC^|p1Hk8Qg2XDgdg;SsszHA#jqy5y>7whpv zMrw9<*Vhj^|6os9Ioa9hJU?<&+YJS}N&-kJ-zxsz2(25KDs{X|`WCUfgKR5E^}g=)}qh#rxO>*Ti00@H`l8vRaO2Fos_1!!^zBnsx3#s>cG4cOGou3UPX zh;i;s8^8SnAh$%t)X|V3P$cz=%Fchbn#c}$fF0NN+%&8= z)QPBcM7K}N^31Qk?tv>#X@x9vBEF0b!X(v5Hm+Gx!0eMQJsFR0_kFrai0ipx@dZX@ z-|31(x17^{jBE2qFvrql-LDVEFN$o({OHAk4Z5fe@rRDt9?c$6^fx;-71(^Jz+0plrqvPjs^Rt08z_&=!jd<(w94HTD zLCS`=Dcu=%@ELO;2`mAj*%yZ>LHsTKvBDF#$ReNUNw%nub$UElYJEb9dlw`I-XZb} zZ{$-n*8^&$nd2=cljKZElkg1z)B}eV0ZVe9Wb&+{7!>lyXrD@rw|`~`NO}P*(eKdf z2ZFBkaJKbumk#t>LE61h#_&^b8o)gpI+|66dfGkKzb^e2Dq>kZpdUZdzL)=hJT9f+ zU}vOfspx8BB%o*SAZBH6WcQ8l{>Ltm(~|hfhr(^p^sW9Ksgxo-6oFEMajSJf@YS67 z@4W;1Icle9>J4VQ%aZ+Y-+(?!0LF=KvSa|11nU>}2`+ z`#nSWzK)}yDmZ?2{oVP`3(%zy9lmjzAeToWztEi`B;F#ra@*!bG z6zz~{dn?lJ0(Qot4R>Z!q>CoPU3Va#5X_BeBYsw?Wn~0Y(Hmi8!r~b$6^6DULdJ)U zvH&3pdl%yQ$@eJo8poB1Itio^adXTYNkksCS(qS99tLRH&#QZSHZu0x>-CbbeN~#H z$qy3dYIS$V)XO=_31sDbM8`tXd=6+I9(oN)KlACHqo!8wV5oiU9~u6c@L97uEy>@H z=&FpLSBW`(@YK*C6)I0tnCFpIGt>jOD%00PcUK1#$|fcA^GKD*ro5aTJ|zmP zklR1&WsQwXW{Yd%OZnhyW*X&Ep`J=o zXhiT(IxZ>3V$f>fK;4au!fj3xW(Aq?aD}0>rO5E)Ux!*!>Evq%7_tN5>UhMYI@Z;I^Bt(M|1~R*qt6!$5R^?5xPkuBO~RUz-pzd8pVPrEmW;D3_#Z9 z(c(2mQ4~;F(k<$hpUmp*tYJa54!QOE}6iGMXCz) z<;kt!6G}#tGl-pqkuD)e;epQ830ibwUYRS3Q2JsQDo@D^qXiPK9vB4V3mc`jspch- z6aUFOB7uuB?Jx9;sT04@RtXa_(WVs^`KcUiP>V|KN%SQyH$NDlW9 zS^6b`d=!1HxFm`|3}`o{R$I-EM};Tw7phz>Kk+XdL7+&I8NoofCDCF52;|_FL8dgK z+*%m$l%lW5wwB2-CW&jc&2HHQ^;vs;lk~|#tjdKpc|r<%LVf1sA%KXF3y$ z-hYbIww(vNMqaSUs;f^H*_o?ibrYb~e=u^JEzUHLLi{@Qpj;hKn{3i9#|Cq9HS~pc z1F6|&^r*kTl8-{|<2yw4cXaAEe$DAJVgs8>n>sqQA-|aR!~1a*#6lQ{f(*+P1CZ^L zC4~cgZHh(x)EISh9jX2CH$YYL#^{llDQt4MQB1MIuu^o~S2uti>}8TOSf{gm_^%<3 z-g6G|q@t$`DXSmp-A-i&NE-$~Zh&EBugmbcCi<3f3;nDj%U)=j3v!9Y9Y1MD;6B)M zO%XN@Uv0#drNC%izczIZAl~2X6Ruv#8R-tZ&|m@J){NG*z$FO)4?dtzOF}@?ZNf%Vt7oTyEE*aQ!MzNsq-R zdL8I?n!w$JEOvVQYZ;)klz+7LT#avM&mpQ!h|3x{SkY^C;!`MTud;Z|;84X`%w|$` z6vu;J*QuIF-b+w&s#(&dNaY?YjSC6ll9nC$JNQEKE&Utypy=) zFH}6F06-r_kv_=U4|Iy z6g@*}1IB4nBx#fF_!R@UY{1MFPB@g*X4SpNwLm7j`V~V?0k!_e?YgVR9GOyjdqf$d z=3yAb26UL2gAdr^D`B)d;T9b~to(WbG)FPI8OBahumm1n)Lavj{Lt5KwKN7^->u8* z%uHJY>aXaWMOzRoZ9d5NC@@Ds^q*2VDF}AXdiwDx&a`k>KDc)$xSN?n_-(= ztDFsW$O?9Cd__SWQ*Nfj^X{DVlaVx%CIguGh0sP)Z5k>o!NJ@YS3SGD^TJw*N}|?MogoS!h{#&^&F840Q|zGx^+@0klvYINJj2c$qR;KiRP8PQbV^{q z^|UAvr4=Vcp}68#ir3BqOL50N z>)ST`@UxRHGTuRG`g|Y`I|Kjd#HlG{Dz}y z$V<~d+J+jjiV&4m7SgH!6HuJmlA5GTSvgc`xW}R%Y>CqE#DUj5KiJ}5H2~L!%W2Aa zRe;kHp4GGYfbBFZ(UPp)uk?U+RqR}o&NBe48S}Jq`kkEHiLP+zAdiXP(>5SG!1hXI zk2Dw&7mQ-AG~mHnYm6n=labH0HKBez{9bYBI${*M-IuJY5P9gC)Y$pMmFki31*&UI zwL0$TCgx$Zh$@-~z=BN5>$&l|Eg&N63N|UU`n&qFsW`Xukn;dxuHc z2LJp^s;K~8MRrUY;O)%MshCW!bRE!@!M(F^+9>^1bkfsWwNku!d2ckFoN^{wz&SswMf}g zLE$^(35}mdvl^B_x7@N4>KD=>n0z=%NF%wrnM}a8TPpXlzVi(Bqk`Hw91;qo)%rW4>RXES`1(DABxJmy1p`HHnY%KsA>86-y9J!}% zZVLgL@I!=;lZf*Gd9q}5e={y`ZKbhqR04tH83S4LnzYC$a*s%JiE-B$T!%QMq{m}0 zUeMrSGKQUDn%$w=K~!Kg2bsgR$T5`5d8*A{wt5JvOkDswh+Z*<5-|0}ySdLO8;K)$ zuw254yTw^qx_iKh)T_xd2)Trzm?Z5*3-N+ovzU~S0=t9eWpE@iDMD%)L3I11gF18;}KA#9m@8$c_&ci<9VmVdDzZ7+W~vw(GFi^G{N#fW|+nhx6z* zk&0Sdz_NomI8B4+U3e);*W9d>Oe>K@iWrMFUzgh1n^G6s?DC}|dW`5gkd&cD>DouQ z*0OxgKnqXGsN0UMD%ae`XN9hvQ*^c$hfnDhiZqWK`AZ5aC-LksuUH`Yc^HMZ5e4rL zl^6MHLhKkC+VCyuDeEz=lN|R_m z*>{ZRW)CIgUGqlgJ-bKW^7_)oZV+nswc722ofKJ4= zJC7PeveHEpTaq%s64HzA&l^(ACetk!+iV+dIlgljn-Hx3pS3tjd*C?tg zd_O}oWp&_mGwOTVz@e&W98n}6@8N>5JQ_>lD^|~iHSz(v1es+84F7xmr;H~I+cW)+ z$Fwd-cj=G@hMZ~diH;K3anC34zuuph<|f~0zO@hU?@Twg|8$G10IV&HWG(cpr~*%TCQ|Pw*|(jizP)|C-Cp1~5bc<%E0TnP?`Ta7AqJe`I9xTd z4YK;8ml85(z(F3Rwtd_7!-v}2rl!Q5UY%Mz`l4L-?tXfO_}sS?*iHd%{hT!TtwGy| zSBNTeFA8+I;C2Hy1-s>4{lHr|5Tr4l!^n6)^PDuhY01asaw&zUZhM@+eU-}xOPDDp z0NRD#7+kyR;l}DAlm%l^xYjE;j3@DANE|T!RFHmcS{XJ0YOP{?ku(fsk}y4!T#ulX zbA}4-g3)PsJ?y`@16w+^@6eDqDcTx!*S0tm`mWF3AJjC)&itoOX`1G_0PI0|*Ir*k zb+3}Pz1E1Vz73(PG<^$AzBOS?*C2E6GWRkRymX{V*FpY8VOV%f+^=`{-0^mzQTPVz zv1oqw_!bLRMRAulOfAq@}k?PySu!ZJRY%(iyablTo z1H~U1^|ukc-Gc_k)gPhb=&_fj^Eo0U;ULw+Me_s~j4E>Nq^s!SDQCp8h>}9Oe6R$) z<7M50HV6)!6#sJ2Z}o1PdNbD*jbLzQHYSB3kj~YT7AxQcxyLr&{u@Y9rW%3N^E-Ba z67I(jrvL9XX#L+|^R)l9^!`tL_CL-S6)_tWC5%sPV+L-C_!tbjNU%PVQ6jn=3u(Fw zU7Md~KQW%A=Tey_9HP0oiR93PHf*+gJrwR!;BWW^8{LQaiS*wbn>egGh|KtFuVNrR z;FG6&ff|EQz<`4HmGtnf7N8XL*XWLu<4rVQ z+3zG{2i9YZpY0^n)?~ltdx8Qki45F?uTNR^F&Q!IBYY*roAsTBnCFl7gOJ;ktJKwB zNq3SZeo9mqHIN#xwV87|+yAyLIGL$|A?3u0GUrHA=D^t;ds>y|kfS@?m7ROjO*d!z znJ1M~wtO;gPp&d`QuXa{WHqPct+Hx;e^8vd0ou#^qw8;)YE|BI2}>+Q=W4S4>)95* zX{~c(c`U#bv)%sGs{eGMT_K}XdWGA~Ec+_OPBZ19{ext;f{J>L+kkM0jNzw|-Ep6> zoTfb~2j*A=``8p&!B;PsHKW<%)|P3sSk7UYPnBceoDChb72reia%LT+G3b6DR%NqY;5@m6u^dgZi6!~ZZ-P&cqkJzZXcG^4F80I>B?hAO^skX zoFn=|Sq0mDALfy@L1W~);+1;%b6lJYbtruie65hhcVn8U#_n{lpd`a(O;J_FOj-JO ztYq&aqY)e=cb8t=CXUn8DZNvQHk6d8AL^Wc81qVQ9_F|bQ;ytD5TzF@XA}>SUb(8T zFe+uOc<(%g^bRKK^d39tbh!%wdZ$PEs1Y{pEnL3yV+LnZMO0T`2D z0}Ys5XY8_uXZHJ@H;Ui1O_a75r#!k3Hv<$Tb%|<|Z~cZAqLN9LrbJ)<#`yJ^ZUXxc zm^$2!N#rZlCeB#?RVpli5o=ffGZzC2ltaCH{jG88^^4zMU za@5gzEUDGlBQ^?v43U+UfJu8zT*Y8D^AH1b-M0D3e%nw_!|8E_gd0E1j2ynLb3md> zQ4@$e9$yJGiB*#PnL2WQJ-+l;DklH)uVWb z0M{fEJ*rhHa-^ zl$zFR^+sEwn^>Y!*!^O~d&#|-wvNMFDMm0*J@h|%m3;!ED~?lnjF9U@4cZodFTWY*vlC_lxYRy1;IufFoEJ^C|oGfGDe`c_|d)$?fy_ z$Y}VZd_^3(!?ZTDr1NDQ&Q8SSwzf40&GQ%mNp!|awI+gAkCfmBXTeFQ+`4jmdCdc^ z9%S!O-BlapYb1+Q0TCVmv_^&{WTZKpkbM)5hB0{IBCEhve;b2F0f-#QDpo>;H|L|~ z3Ok!$Ik295wvOpNbRt{GDI0wmZ`dI_%IMd`3!?p^j{x6~y)U`sfR6-c zt$r(#+o%BT;>YREPw_^BE)fxYyw25D7%o7QPWUyL_i3S z45w37!WlV+>_tB;vprUvOarI$_5ssuhk$;=ZAB%Um+k=Uuu$9V)7_5N+k^CZw&q8) zHXH4(mj?}AlGtZk;Cidkac1ce3UdiFE04z8GpIANYmJSB4-@vvMuoN)q8ga5&m83{ z;trL#kSeuj>EGdsCDzS?`&Wl+L8~BJobo+3XBxuOGisAaJ0+#-_JDujhVW^}3WWR? z>ngX4Kf)VH2V873!kfF*jtXIsXhB=JSyWLUuk=hLvjku0IsT6x;^RR8k*u#V7WBaQ zdS|rL{hd}u`?iEam-x$46Jt#Ulb93`@{aopj4=9B|?GwHErUe{xZV+KtFJ@X(W}dyIDVeQ}-C3pq z%BAI>IM}=?a^A{i&Q${LcC{u<>86Yki$Js6tD;{=h$L0Vy0to@tJA4!jJj>}-P3A3 zmKu87h~rb+)ET~ z`PjB?|6}LIwr$(CZQHhOJGrrKCpXDWZZdiGW~S=BnGdh3tGcScb)D+o`>eC}Z*`j2 z_x0xgOaT{^tf^7P;>Lj*ek7+|ltfDDXtPURb}?#+?@qF}y&HOjfDhdY)J8N=iG6)| z1^qoT7&_H2M~4mpc@z&00fvJiKuI|(+LyQl2~$kcNwbo9EE%Z~5p}?3!P2BR#Bi!s<|31U1%&88GIVw^bX1utaeg&d)Q?b@t9T z|DOAlb9VdGFYt~w;MQ|6#?4z=fbq``Z)}+WZa)Gt7-|wrX@5i!fSrb{w*HW@zKMf_g znICf*5(P|Jh-gMR_WTU6rJKvD3N#Ukj<>KBo=Hzh;^ZlGlVFkM%jGP}OngX90BaRU zTv+}%(xNDCKFeF0+j?gDg|N$3X8Mfh;Vfy0jAs*{Ls9dBn6orzF<4_c5)3Lfc5TZg zNW{OEg&x*WDJu6wGr3QSRHdHrcNUL9dvaf2KoA2rOescD6G~1&sgsk=d-flkTxpu- zHN&=#dt-;!2bRwnD1zGNR9#SVTALIX6ZRsP?4*cTn3zL&tcDOvvn4`ddBTPg5%@bP zV(os9v19nqwx%ZEXs6^@u^|IZT>!MAWQ}PwOFBTvSkVN5IbRS$vWcG*XvvGU*;n)2 zBswwTGC8F3n>b4;Q)OtAgMY1#QN48Y{{aL%Zd}I*-v9kbUuAKi!@}Vy z6$cBN*0{vk&c59;6wR*Isy$H{Yn>_Jx76BlxR8XlR=V-uXLqB=A!kMJ$2lB{!iYS_ z_Cw@(6OH*+pVxMTL=Fqz*i84=X>(#N<@-4hw2R;gWRS~4!!N21F;F*Gkuqi3E^d2M zZ+({MlQrdP&yC&-EkH7B-G!Hj0hTn&7@q>eDnH~N^EN{C$Vcco@ z7X-lv8eG3H&z#N1*d1aQPD2SbOOx_JbZ_blPDPZuHI7?|dOxe8Q> zj`V;n{4sNsXW1=F^OJ}vJnHi{HN^cO!R)44?USTM7Z2~AZR#H0`GNkbPZ(&g7*@al zO27z9z!1$HX>gut!wsSTQ1&yd-C?b$Fdp=JNbhZNXfTes7`TO55uP)Uy+wOOcPiNh zdJD;9Y?>(+A-q7U~(1k^MwwL>QXZDR>ld7>CoU7(Va6;(5ig(b)w8bm) zX@=}%y#-&>IAMfRm2uF@&Q#CEyy>JIF5d+|de)J8^o(&CqH~m7;{zpC+X6*;HJyg0 zzHe!tEBxYE)--Zitz()<>PQ-tCxoO3;U=|sveKU}f>`FVlED+V0YYTT2$!%Q z>GziN7c9$JZaB_2)tl&WJ+Ira3ekedI%=M{)E%?B|OZU<)KqHOTF|1T%29FeRU7%-``rp_Gko~W+u&L zx!YDI#ydBb%j`|m-r7SFH0Sh_C6@LwSeY@IiuYu(1j>ePSAs_o$R}}i7UHXINxIhP zEiMG2hMLw>w^2Bjq^q`K%WCSZmISiS&Mj!!D1#bvZL=oq^;5YLuGDlisj;PAUP_ZP zDYUvOD7LbZB)#GuSr2V0I-IjJ$&;qFA}zx+PQm@dmr^(zIkHvNWA$~}QJb4DAPdH1 zN6eSX3f<^ES2ZQ6y7f$qKK&=%*6s@cTuHGG4asKuwPnlAt=PGcqbijdiEAbs>9J5r z`L->Rr<3GF$@%$Nb85E^wd1dUKWv7;VWy`;{Eav2gTVnE-o*iykXV1UV zGi_Po?b>5aPJd0^0&GLWCQHZH(fjL7$UkxcY7oG&PX<>7jx+e5vfa{N4 z$}C{JoY#D~t7=SkY)t3s)MY;2qwB0Hg?5s2(r+AQsX_4;mD#e$iE!AW<8L=hF$Wmj zSkBpJ>`pS|(mjMoM2VkD@>CwkW{e?Iev6=@mj4p5F;R#^Z0vis$f(WRkGlZ>&SD<> zeob0lI?JF*3Y6^yB-a@>S#wt3&wOK`FRNEbO7OJW*mbn&u)%6&)*6cr;MbXAud|X@ zQ=>Z7?!TE-%~E3x3N6mWgNV_YnK4V_;^3g6BwJ-G5cCl!)$Kx9s!0uN{Hcv5hY^h* zYYqbAR^s9@vuh)3`YiGY?T{U(z*|_1NQu)D(V!cP`zf6lX0t1c0jb>V1)Adcm##jj zdrJ>gmu$~hKS+M=A0+=!4q(!3i~aAh2-~OH_+7%1gXFkr26x@QZRjO6-O{}`zW+c# z+|TAZPT-L5#~j@khv=Xc-|WpkGA^l-$^s3zxDutaD_9}7>8Hn3Uc>EmV?zC zmc?(skBmc)Xo`pxHTFIeWEq3gX9FI;l%zaXYRP!56(e+H->NiOl^KinENL*o=^Su2 zBXQA~6sBWn9yYD2nR`);jL`{bJ+U@E>99^?1B_*jDzSZh?acJqkuID4F*^}F@EM~t z+_eds!;%j#Tqj>wJxFWX+7aVrsD7=N4De=t?m(UGVKtl59l^1a>f=hQ&ExKz?ie>m zkF7$JKQ$8l)p`5KH5_=VBYXo%Q7A5Uqp#kDdG`r(8-Z`67fSi)7KL=db;rG-a4Bb= zROUH5%tbnAsvpy6sxdk97Qpbtxn(o*__8E@ujV;di?A22!?OZAuRNT7LFyyt{qAHY zYdB9D8oZaR#QvsBR5^4Pt)w*oX1Eu+^4ZwdKzOV}y4vuMU3~oDl(2!98BSdUeh9x4 ze7??51#SqpGTG4tYLmz(5j^-_-+6$*xEl#)i!IJe0DX>JCxVvEf@HAOCQ#JVB5oB} z?vAIVCMPe!kxOM2vjUGnp)xN=iqA7f(IZ0-C`+2kCmxMGY4cV3w>sv^e_Ti1&#BW# ziBr;;G*WiQJw+zYfpRJO8`{Y4Sbd1mQoVD>cH)6~dkP8z(n#5rNkg|RWNg98Ya#Bx zki?Kw*0nf5bPI1VXe`%aC=k_qNHZ$$&!Nny_3juOo&tu#vp;do(i!+Ur4;xlA zTkTa8a2&&{#u(JW%CE&2L}7ojf8ALAR!Odx!#M3+IEH=y3aV0sj#JSi5@A4moV`YT zf06=F^#-j51f#%#^N-hbDHwnuBGYgN5LJd#p~HcT4h}Q0UKSP8I&nuLbM#4H4I5K)HC%w7GI3hyA~nj&J>5{9oMBaV0nsr z(4Rx)3{7{yv2XYXWzH0l?)k4bXv+iL{r(3Po&6T|ZS#e1e~^E!4s-U~C`WzTK`@j{ zFHqWwjiK61M^XL&zZF$|MX#s+&yya1$Q4aEu`As0C7NBu8lZ^bu&gU&>ypo>jDH|M z7YKjI<}(`jowKb#i7lyrnKT~OVi2_kTz@f$&_RxJoYrDg4a|WczaX~l-Ev-_AvUsV zv8w+dJmp?7J&v+-Uj9bW#UUS&M-{Toc}3a`Hv>Xm(I{8S5b{-30w?B_$XlM}$z!r& z`gt2wrpogY&kcDr*2?JaGzU`PaFu>!;%Qqr&-0&mg+RTlz2X%?nF)cdJhH7l@(p&; zjdNAW(3cQB>-wEYjZ9uQ<>~vG=@TzPb4<$vf$`eOO zR&xd|@RW)6wU7YoJP_|~aPHm1Jhj*POzXqp%C$rv3GS#O?+$}uq}5A81lgtYB}r7I1$S;u*-OBksVeAy(9?a7Vz zp24)?m;XitOk`?sko(aiG=GkHnExl8$p6(M{*MZgnkXv|0zeAe8ybQMBJw*}Og4av z)dT27z_Xj}?arMH3}Vko9Zv58s1CNF5t@qSsq9k~t#w-=$|x z14~l0rxeF7?48bF%m#BWlYOO``4Mh4s(psTs9shx|Ep|h!o zp^KrKrL*OayzsxJ1s-S*m6esVZ4MTDW>3usI3^(?A|bUPI>8|dV35EBAVUDu5ZRN2 z7zK+VGjudAx=3vlhWamX#q226LMsp>)olms6|2QA^=^%;zV@~gfwMjiZ4Bm$@7GL@ z*B$TMja$vTVe{|T{b3-7EKdE=RD@qHJo|Pvl0o{$wNV6+8j5G>s-G1{;=Q1~9&w=R z#qs*MNsIA;CN?8%lUO4ZejzG0a$$PwL)O-+I%fkf9f5>rk29V8Lg*aa z1p=1{pV)dSaOL|Hel?1YEb}_o1N0t|QR!74nV?}sb%*o#OmK|m1+*$XOTb(mE5HOj zQRa1KJuR=8lT0A>4!ksJ&HlzBL9_64?j*R`=qdEVw((lY%Ue_P= zAa93lX>Qw(?C}%=yH$JUfu9`9@$cOCLD$E&Tc6H>oO>qYAXR-9fLA;s2(#ZF5~BKD z2b>P$RK6WhW_b?>#3PA$WzZe~JrbkinxOJbQQ_BbB~kK#YgvR2f_TmXf_$|HY_ls( z^!%f}Gu2Q0@2}lBMkXtxD^Wjx$)7~XyJ?!fqC-Me-m1fDn!d8be7ybfHC3NsV9dk! znrDvp2MqcC+C!e6D_x#1&Cyqnw?6#uqC=)E-dH{wn+VjOExLj-P$L35_!1JD~cF#{a}vjsbVyYy0_>ANJ!lKz`50 z-(mlu@LSmU2l+ki{B2xeTtWieqFF!_LW2h89osFLe~ZwGyfiiM*xmt~Ev%=1pyPf7 z7E(fEpSy=|g>VTaeW*^E1{vfUdMoB`+pD*JUQ7GT&iWU|H;~)Dc1!(MuO1S1nsyr0 zZayR~fPh480!@0`6_o-t)ug|#jciD?NgUBj+a>;??Qx_MhcPwQYXKhy%$s75X93U3 z20<3~En7&ieT~qHL2u_nfv<5!G@@S$h97hD-|seD{=DDZd@9J+ z-MvlD-0ulWM-)6$ybRo6WV4#)Z^DBwP+FOL^^oQ7+p@CUx#cVbCkoJ_+9B>#*!D9< zniD^kMNf5aKcV#6TdH&Uh^@_`u}FoC2x3(^*x%rwvt`|HF0V8pij;oqB`%R=CVX3D z)O>tmJx=z@5y6Ea85sAAO;8FU5>J1R4k3iT8A7$!x6hZ~V|t&}#mtWE?Wm4wOZ9fn zh;7(OK@!hVNEPp}l92F{ygIwZUK>B8ZzEQ^hNiNdpEna)AVHR)I2Nt7ure+m6kO7~ zErQk&mC1t;K>H0_=CDjyR#z@#iR}aUbbcr8oaRqpC;SjeN=K65kODRB`ooVv8Q@`+ zS1Q()x};*TwN`L?zA~Lgy90^l&mABlc^cv9Y^2NGgpgBPX9$FYGFApMHo(* zJm)*=9AAG$yK|FRQ53?Wusj(QR7&D>8swf(^g$Y$2%(jsPU{J1CXpQyYtwKR+NKEq zsLz)#i9zJ4hPtLSmHpUP+byby-f_06+rLHaco4vsnj2$l5_O1sgWHoZl8;jkz#~Po zmwq`w088AisE$bo$)W5@>FrvKU`f}ucuCiGN4DhVBY`r9Fuwr!gwr8FP2BwCYm`YX ziz<#*aolgLf@puQU9`@^ReV(TEnc8KZ?4a>SJsD?Xc!Na4=bj72u(<4_{J&y-E6A3 zwUuot4}>7~9y1KUP@5zaD;>{}D$=Tf0fXlmF9X9)=Rt8@trE*$8?d)MGz|K(vXRb! z3Z>d18yXggoUn~V(L6p&HYb^EZOl8Wh?SIG&mpn}#Rcf#TAap)@gZ|}3n0nKF8ezy z#FIHAG+MH}EXa@+r(<)y#9YrwHXA1~UYWo;LT}S7rzJ`acj7|76y+(p=fZgvq$>)= zaWP_9#L$DSx^s7A@Zp1VEBb3~27~nYP5l-dY+AobMtw}DJeF$WU7}ge$6w5dZg>I1 zy3^7XXk?ws?8=uF%fCWcSydJ`A4`x%aftO0qEyRpF{|F{IeTe87hKZ4@96sQFUtFk z^_#DSk<5Eq%B|>2x5_5%*82Fm=YHIg%NQTo#;OPl+3}>bsm7ciY3z##4ocb|Pn}$^ zpQodbXBSPM4;+|$VxBNKhIhQY9wwD7bFK)y*Y#y_jZBYq9HA2}>}MFgXgg4ul}1n^ z@oJH4WnFsClEZz;;WVDLVyf~7Kpq(K2qjFK@}*IfTDBTa3H6COp@|HY8;}EMOO`L{ z=PS#ipYWD-0TX8cYC(HBp7L&7V)enn;}jd+0q-rI6s0Xz$G1 zRKQ*OLE-Hyk&gq{^$tRu%9P$6?20_1m3y%xrpeCRUBVK}(CMZEo)z56jlLUF>c8eL zOX@+G>ahc!?4e>hA``cJdd6dpw`JawE zY5gYa8~k?lpzo&#(5-4{6?C_RQm8QMq|kB!{HAJRQKU*@S(Jb~WA#_UXqJFG`NEv_vYOqKe1;o=F@gbTl+{sO^v2%`p z>+OL|d6Y66Kn8A2q;2A&CXb*RU-mePSM?Y*x+q|dl|O!Ll5Bn_V^$zBr7#kCCSMv& zDK|X;Rbh0#p|QZWmQevGK2>qSI123@oYtNfsnGp119+%*Q#nb4mtaG||e zHI3EVmtgJ}i#b?upg~%XV9`6>lNrhM@{ECInQ0fcteP<&wuPF3YFg%>rkQOFD1@?R zS`1WHfTGOOhzURvU76J}EJJEmNd3`SK;0tCGenks3Pn|BdBg=Ej9O|`7G~^#Coraz zxfQ5r7bBM;tva(Ll9?e}mo?!E)S8)LK83 zN@#7%6OzW@$eM{jJ{HUM0F~j_K%PhvtrSUWjlw~mH&CTB*sEO{5n_K94I27~#=sNo z*FLWx4!Eg;2Mab?)AaL1oi}fiVh65CvJgl?H;N_*WMWNZj;;-f@r~q6*7{4hlxAI| z6|*>^^j1zYr!>l}N{1Lev3{wEYVPdeNdK4&70BK19(+5=>@z&>9u765&!+gx00};8I#?V)Ozd4_A%6%^4(551UT#0TQPuBW6Z(-tv)K^1vC2 z0Bf6tyEI`R41M{0sWCT?*fLnsj)7(6jgG`1b8;1{Ln>QyK93INn26+7tc&uJ^%&)X zy1-Yipipye`x6N<_q;Uy@4!hYC>0WmifGJP>a3n{u~Z3W#>K!|Bw;PQUG1*1;w529W+YTK*Z;ti8f> zgUXoPh&((00-x66ORhNl7*c@7v&mT6hKq}sf7j&!yDJ4mD4-vI6H1ZGzk?um2U=R^ zXz!){I;8_9<87%EqHakTzu$IAg|{Q5(R?7L9GX*{+#-dqjK8jWSd_$DlwtxxTbyzDXHCfgH)-dkRlx;P`gCngZcV+D z+9&P)bxUx^CdzM6^_c$XVn+}B&-&j+_>9<$*_#KM7sc$n71s}(vZ+-Q*h~083_gHA z)0N>@#v~1(PKV_%Zt8Q>(lTjp|D1DbV(p5gz4G*RQJLpTc!@5nJM~ znLAEW!ZOJ3&(*9wqKpTpogI??81r;jYr@AgVOv9AwK^l`7*O~<>ZAZWxt_T?4Gk*= zSbej&OR?i4zr1s<)R7w^WB%RHP;6nd=1^88o97Ad&!a!*j_@e$mt>1NQt-Fjy~X~b zcF5O-0f4yIS_UDAAl0<-9SaA?@OLKvXw+&vso8qJ-eXU8)~?{)vH<_QPe@u)RIb=J z|0W~EE#A~k$_Ad(I0j@)T6lkSlX8H*O!)=wnMI;o&M^F-*tBklW90qX30sj=1{^%# zpQp;+Wh)pO6w1FwXtF_Q-I1UcG43`6)rh}|p(Z?W=gu^TOScwT%j8 zXW{)_>(sp2?JuCs-fvMwtd2M;|&Q z1&^rMluG`bm<=mVkYoB$&CYW)#Oc%g)gfHw#nkooNtEM-M4|*8J>H2D>u52w22t4L zV}FCNJP^PW(}8zRut9@;e6;Eu(NFCpKlJ2mhkS{7fA<1D}HC1=z|0Y3&gEVx?c|q_H4jI1KvCU z@iaL98HMN_mvkvUl-C0#uP9g8fEj(%pisEkUDN zCM>5A#AwwFE5B8F;!B#EqHe&L)=lk$iQ|W;MW_4LS$lT#+ zS9P>-;nWd?+7dLBbg09L&3QU>5Y(rN6P6)5Ap<;SC<(L&>tyV}wXYi8%2lr+W=|4i zFf?a@lZ*0qm?m-@wA_(7OSB>6Fc$aMZoBkp7Mjt&-Y;>M7yU6kZ0_gIEy$Z zQ$;?}VW`}wL?`D=%DW)zOu^=knYL`1n&9PVmvqu5wy8&{m3kFB8aAVqbt}aUA`C@G zEAb0-D#HNOqLy>4<+g}9Nz+vhNm*j(<=sp9B#p|t7m$W60kf9tUe~j;lg~P`LC6%| z#}4p@8dYR!(J7ar1er=)Ita>DCZ!V2R&*;nNr<(P7f^%#t*S+w*ts#=g# z>RH68%coA9Y|~Pbn%q-ZA<}%J7gCCh=8WcDXdD}cwY4 ztH$rOOuZW9+?&KyB+9tyj$XJi?v=I*GX1tp2ShSx$L`5h-J15{lkwM(BGDdtJ--kr zbJ3KkJ%Cz%(bSYybRhg4?qoun3qEiFmI;V$fb$cp>xRxUV6h2~Hbk@u>~(;KcE$$J z%1>U_1C}37Jr2-5Q0>6!3$otV?m$4c`vtol;?@B!AEdjE>2^faf$nU87#F(b0J9xv zx)4RXFJ}|b?Et$Sn?B6v0e?N*>p-}R_k)9 zS|5FO;JY-EeMs&h?SpVfb8(M}AmM>v86XFt7bKwYwhK{g%DE~*d%{dao-Ae;kbZy; z^B_z1N@#gZB)bDFZW)d*MXdtJkwPg+Qqv8F;)-FxGXW0d{1O~?3#2$ zs`XB5*qB^brm+7Hw*pUZ7&=49yl0%q)9{sVR z5^MWFYBTPL)6Q*)JX}SadDjit6!wg%z|`U7(BeyrxmSOSP`vb&k!Db4=SV5&VL4+R zh#DG0S*mo|FA+m}-2t~K=Vk8*=jHAg=lyY$C_A&9V0fm4JpF_Jfd0d)XVR#PIYzGPlYvg~!@S3YxZ z>@;QYF90Ro#u3Klc8#!tz^?36%HhLO~bjNKYji`-UVu8~ESu9hybQpUEsBW~fE$BOh(?XPn`A?j$7HK2QSKONIs~U(!@EsVF*#fq?8Ufnqz=xVkKOP%q@`yE+$pP zY|)w+L6xdQjS822o)Qr$M_d)fy0}b2!JPpUq%MIN>{uWJkCC()sg~88bd7CM%rz|1 z3^i5oi+FGQoq87gJ|EeqXQL`?gNE2kn)q_hu^!dD%}UQkdYQ7UbdseTo4dDDOh=Xr z41EPV^@i#0C;xXPJit0a@8u^pC&zaj0=7jS3 zi>|Abt&qnxv{Y0bcxjLZrh@CH@$iRs9;~UT&IbHl4`SQU)3W_r(xZq4@z;xppiDrT z>rlgv@G)Z;Sn<||%yq$<9>_C;@JyKV!4N%2bpx_!UmU}XmAu|SmB*vf2bw)7y92fN z{CeQuV^<4-JOEVtA-wI-WCxD9ku3+nuHbL~P`@73wIjVApu3^z`?ZePQ@auNO-IDt znAb!52aY$2wxhj!AiogALv;_R{?TxfFRt`p0rUkm9r;F*NGT?GJ|-~??@5nl9oGe_ za9*`C^v7>)Dp$wJ^Dt|wKwo*$-;ydr*wA60B!~HXhA`ypf94TQt&mDmIRLV5`$rGR zQu@6U>26A3hIg$fx$;!+y%@5@6uWcO$7IcNU3pB@&pf?bXi5HHa_q+CTMbsjsM^G& zyt#yN#Rp1PCWk;BHDkji$>YSR`9$-aAj|ibmF;xFW+k6lalMO^PD(W_hYGyyzFrlV z_wKstKTfI-*P6=(Q<5#~C_l5FbFw=H<%TSax`=cij~GAhVYgz0D!hBdgh0k*E<$o; zzf|5oFIcWQ*asgnyqQ)QYU9@oD}aHSZy&<3HFZLFe-Fn-Ymf&G+GbzTKy0%xOp!V% zx`T|-kIK2`tR(`TF8$0I9)*YH)Jk1v<-}czY}$ z!gS321`UGrTx_Bt(NgoGg92Tp$g?t^p`hQ6ZO7Y@P!hYFdWbm^M=J>n;(PsrXh>4wx) zJ<&{N6DuwG@*$p0prQg1OTAEEz7%#+7QIz@tZM4inoy@zF>_C(Ajq?yip@W4E5BTRiHZ&+U zq-RHHF?3DobHZfgU=m>;@6}z?ariWiJnp#(T{Rj_+0+xla0I@YRprs|r16XJXtDCy z^E|GT`X9M*B1xj$dJs>O_TrFOrwADGA%rIp zl(!2GGNqEs&iNd{sL$kUUTFg~gz|*3_9<)9OKsY2gA~%+f{nX1qc#;n!(xLEoJhjD zV)%-Aqfd)YN8-pFyGN!yl?|ohys+Amrl5=zU7J*{l_{>3GPFv$I_0zmX5M!>HZgS3 zKfHctIawkXD|UJdop|C@|CF@TW|MWGcn}5o$S>IM+XlIA8oJPHw(BRA9hgecUv1SS zlQ}wU6lLQ`EPt8&Vmt@|E@C{6@>8JGVbaZkOEm&gi4JrK!~)44sdNC;>N7Z%is4FD z8A)Y^n79TZeWoy1Rzxz{RdF|GmeFM`S#}i>Ol8Z07+q+aaFq$Q2p!v9d1l(Z4D4*p zj=XKTlF6tME=ILfqAs-*iYS6&icG?l+W^yF&3;gb#d(OTy^FJL{Do-0xh-IXzg+%2 zY=bUAn0R5Z?ZN_=Y40bB9SL97!Lg%lY8rzqI_%TC9HXU~vGqBV8wkCgVK}?_*vM{} z#H{8{(RiolBnSuyq$DLBN_ilVicY0yTTriMOqOjWvWiPpf?msrWVv|6xha@+FT*9Z zB3dU3DaS2MAk6Ojxg{~PTZY*4T^7Q1ERoh8Fp8MCg_4t|8d^@r+&4bysLd&6GiEl> z^4L&CceRJ4L;ldMe`5reHRen~(#J?piVpxO+mk@|KwVRNR639}++XJZ?ByWpL) zhxhas61P(wC(E)<>8^}v$>%m~6S}Rs75I~*2g8WmNzq3T`4@`ZpQVmc<7*^#ymc6h zJ1n`AW0tP=@SFwO*1xV7arSSH@|AKn4Za;$Ahx{FAVC_-!ETRmz*F#k;v!P=Vn|S^ z9dSJ@G=za`)0WyGnM33b{LynrE)6Oy3eaxD6G?zXjo{HmWE-Sfe&uRFT90_qM6eHF zI1=a(&xKo$f;&X7gWVh&aT2u+7$2f|=;=U+kIo;!J972V*9E+etlTHQf&0+35l2}o zqOTcdbr9$pQO2Qg99cNR%As^Sly#t=Me}uNY=^BJy6G_Zg}NTf)nL9B$Zto>8WGl^ zemGWlAl4(l9_1F{BSwL~^>`rZV}#$cJs|lJ6O7r9k`|?V$iQL5M+1IHLL_k!nTM7K z81qr8hg^+{pL#m(+C&mxC zqUhgk7Kh%`Ob-ImiJWHC$_q^*igeRx(WRn|nscfu)2>m&PT@&5p1&m1v8@nGhS3|a z$`y9%6kqBl=lH-?vv`8f9Q-`uniV*cH8+PcW5iG$Y0Q~%urqBsPH5-=b~AM9`>Z&c zg#%95a+GxJSe6?>EEXk07Sj}TqnRp5XqxPYbU5UL9@Fm{V$^%UsPwoF?0fhw6EV&+ zV1C52j({A=$b(BGX*&W}c@@WnEyg1IMU|cKz|>s><(!17=fGVleXN3x5|lBl2og(f zmZUP41FE!X)?7;%^WeUF47n&v4!E{cx53pm&bGRXyjmF}YobQBi1jIwwlJ~*HBliO z)hwG@z{aI?jE*&P)9~9Gs1%-XrOqlVTpZ6Vjr^DUlh_OGE!gCnCHu& z#P^N>v6%_BQ-OP|Tz8;9gxfuENzf&}?#|tLd?HW@QG!+i+BRY4Ugv-f zbtHNc$SFjxcLP;11#vzUohd4;nwUU5Q@Wh8n5(isRY|o;=wkaqO>BarrVtE8Tqqge z=|I7PywM^Cj`-(ecccCIoTFD)}6{}?9ehi7XqnW9b+DN;wU|P zbKfl756JEyUDvijTiX@L?j)Vh)2GP4u^#lsV`Wn9>%R(iWQ1mWoD&Ot6(H zca4f!fP5yjUn!i>lntkRCTPFH=@6R~`L){V@QxHZZv|Zt=~PmLtcoxx!yt=fAhAZs z%5RRQsr<-_&&-Iad?rTHImC+thf)B)k2qxg6j@9giLjj~%S*&WsFE^UiTyR^#gzn`EBUGc4n_N*r# z7KiGe9m#~=?~ZGIpBjlL{RGOZgD$3^Db#0!BqtymRW>2$Q=f&MzB^-cb$2iUML!)Q~U2VEI!qslAJ6dh=wmWrOE3I~})z99yy)2nC zG6p0KI|6C*;Vfsn&U1I}etXV-dwX~8Put<~0>Jxd7t8mch+qQG1)u^;_d$@O9*eOr zZ;e)ZO0&~do}c==GK3`BkUpRqy8T0>=r{r3fMj9v{8A8en|kSA}Y zIPa>|ezAKNZ|d;9eI?$4Bb0uWcSL3OC^BlhWK@v_h? zm9CRe?1Xa&=#XZESw=@8?z~6n306{^3O#n1<;JSbf+$d>0s0h{N~cMatYT#zB|2qk zQvjzL$XSmya{jL=M}nI4qm^}5hJox6u`+KHh(F#Y6n^z1Y8K4WvYdjpmi<`*+R0p5 zGw3N0F&YhPP;{%PT?RtaX1(h5@)SXZf6={M3I}!wu3@esO=ggE_?@1w(YTV3^fIDOmR^^-A1ntxLGpS-9)_ zm`WhRs+q~({{E*O<51A2jVMC0=w;k}aHd9LNV<&<=M1BcU>&C<*5gfEz*V>vAhrlClKBKt~$J39^(E*9D6 zHL_yj;54?Z)lL?6UbcFExvpy}dy#@r4;(Mdozs%*PDN51XxqlJn#c=3(lNHV=HLT^ zBE@?X)4_!-b)LKH)WNzk2~I;BdAb>=tLt&nYYSCt@KM>sfI_HHTCHThTrr#2UF^f` zrZlLt+7)BL7n^=MITy&IVxm@$jDl5&BSF{b4k9b3k)FmuG6#`=nj8XzIxdF_WvKG_ zoo9U&z4o1dD)N?*NlU+-@myH#wpl{+M=RX^z5E+PKJevny-thJ@&A4E{r$n2mP{9q z8<_$lwy4LK@*DNIi-muPi`tXCH|ihQOK6R~b(G5}$z(;RQU%*LN$ZPO>(S|rAF0*& zSdsx}R!Pw+X*Bbln?sy|)Yt)|bjf%Wx9cg{GNP9wCoN7S^eW-%n30%-dhoFJ(Lrl1 zyN4~bYQlHAqM}SGVl<-kq7`W#T%`fCKPzD4D*HXh69vJT7FQs!5`Q|AB~gLLk~CRD zMs(Y7ME&aESBdy66JjR2+zgg2^cX6|)Q)csvLwZfH_@m6r{H65VrXDe;?ZJwezVt7 z#|RsjqJnw`<_O6{$7r?Rrk3s-&fp}trG|`iu%Z8u5(<%utFhJ?UKtUebPzFjbmeI0 zmD70X?;fpmf7{n3NDW zO+y-*m+U4fK-$kSrx(ONqBG*yY~7G!lsw2-xjt=_6b?+xz~lT7Ai8R^JF}s#cI}iM zGtD1aF+wOArb;J(=6LF(+i~dd$Aju2ocUm=v6LV@;S{PV=t)ECKkNLEsa>ghaos`zqUtgQfU}W{8kX~#int)4^Ej37TIG{o*Z=)Qi;?W3{ z;g1n1!_%SD%GJndmWp?&m=#;)(-d#zD+$nWw5(MJ?hNWFhRUjUBe+WpbtWb9t`XR8 zPmdE5E>VZS7C}ujfPZ69aN4d_>T=1;Y+S>%PbV^i$nf3IdDlIHOrCEm?~NI|hi%JxfHeY+PHkXA;E}XI>|~Tzj<9C6lz#rJ9;OBCkdR%MTd0i&9oCA>STGd@+?8Ay(IYgUi4GdRXfW=o9w$PU4^uFfTcg;nCZxE)< z6xuy8RD4&m^c(EYBW^+((6oW%JXA3V_s>TOch7{3VA zX(s{oBcvD<-s~I#u0P@Ld(h3Vtne}QV?Y;kd+Tg zxN_>{*6}0h)h*tdDJC!sRVp_lhTm3K+cM%P9bGXY?}lk~N}4Ooyn{ZS4rBqIed3IZ zKRL_j6llMT`V8{bGD%HEW+uNCwU3;x7~v?@F_NTX-uya>fMO7XRObLOca~B=pDU*! zu>=wfh!&IG2qoH(T(v|{W`xE7W6hNB=!F-Nft)}Kn^iU{ni+u9;4PN3x#6usAJCL` zx|((*-gnRnMWOUyW@W1S6%T6KY3~`g11Nlu*1vWdu5g2T_M9X$(&s!M2@{pS_ zt)oecCK?iJ8yzd_#hk*F9G_WChch>U&ZfmsdWMyzD1H|4ZjnEVf!~J~$YDm7)W~$_ zMk2Xp<_BdyLqI!yHqU}!l?h{TDqaunX=fU^oyCZ3K`^Jf%9odvmSEz1D7dp2TLxw*;smg@VZT}<s!Mso1hfiVlxKZ_{I#W<3^cf#+`BkFmgiBInix7QE467)s8@hmW7O0k+bALmSr`D zSUsz$hgpFEBeFy!5=4?Ls`Z##7FCzjSkW`PZhxX?^MX3Se6QtY=a-h7KXXh6-9w=_ z$88qmO883~+q=wwl*1Hu6^psO0WbA|x=_jyJ2FNb;p?i%rJf_VbVh3u*I5y|wC+1{ zCG04LM2(S1xn#CDV|5&{x{O~C>RIbAWsO&|#%tL>k085uS>1X`8$Pt9x?$9t$=$Y1 zBe}elTq*3l9cG44O-$3qa~#MioHkZl-Xt&R<(k?xV&acbR}<0rAY-rg9aIG%+-Hg6 zT=%Kp)Zh*{Nj!Y6kjft1y@Vf-%o#|gf9!-%kPX}={;eQxtGNSoT50+6 zNzaohTf0IyKpo7v68HtmvEuYq0-U49x~F;jjE;EdGN{`TXP+x_8sk)6yUc3t7t9{Q z!V}g|P<%}XW4|69%5Td!S9Xh1s(UjoO748)y4~B$)jpqV@2ulIN0q&Gkn#n3;A??5 z&l`v=HveN0`Q2{GFwu9uo$xZzf0Ulevz4j@>&FY}FybHS%^g zpS-lLj_^z!M^bP3q3~=8G=|GO>t5x&)JK6oc4PF<{SS})Cq7vOoT|GN%=$z{EGO`X z&B^ELQ-0fZR6%M2LN)x-w)h@U`eW)HnAz+$y4jV3TAk6fcl#QElmA86Idx|OHQTy7 zwv9JVI=0oZZQHipv6GH%+qP}ncJjuj_t`h+J7e!(urAgZwQ5z(`Aom1$6Ib%L<$8? zX@I>dvv}30Y2=DHHUE@8tXVp3CQn8OrA^_Y7fiw%SJ9UtTG`8fH(uE%xWp%V(U&J$ zncl#PQ{lE=+~SVHIc|aYCj?+L{wu0Nwi-fp*TNbAwdkt zS%EtZPokCK})MR6}mpL!mQ9QoHp~@F(|0%@IbKWPtug8A$%OzXh6mZi-r9XWze+JaIw3r(iY21(XdUnli7oI z^2fPmwq{n%pSW}+VB=ONM=aAYuRwwZIc^%L+E~|pVCP@*A~AU@u49_EpAUx*95S}V z$Xik5ZRiz;Y}RpEw-Z-TV<||9HM^#3LzUkMrk{F41&){@|HQmRZp>aim^0%(?vXP$ zqo2SA9ztToG2t~@mHS$n1hLkjk=D==2|=$xdM^o$_d|^7Hl6I0j>GrOHAEBt4zJ`M z`q7E|gb#IHgZG#Z;cX)_r{B6xXbp<;*N=F``WWQ91ZT1$mFCCk*M-v*D)9e0PKBnx zNap6Wf^?t~3x&$6;anz7y%jinL;S|1ra&U3jib{9HCtO-l2XwH$dv0M_&v!rsc=^X zRBL`CG4XS*gvHR{6(Dh5Qu-0Ch%-FZKrvmklS@X@sYqudy(jVkd?sW1^RwswW4d5t z%2px%W0;Qo3kmpN#z`U$E*7o^)+V+_CPLN*PEICH|Lw2ZQrB`uT|wV6ie;&<{K1Mv ztcXGp=})T#g(4Yr#r1Q;U!WQKIBhnhj%5_|UR<1np-|$1I5s2!G7fALlE`9{#U@t+ zK1NC{{p-1g|MOH&ttY)%@AFLUtlLfWASD-6f*(qEGRx~E(`jb=ebdYAG4%W6Sm{TF zJ!O=sI*y~kVBCtmcq2Hx$lN-PWIk`@UYNNU3m*CvBYbS+a|cc~)ZQ4w@tzk>HjGg> z)RQyTQ?2D4Tz1uv6G!#ltETXsAg%78)c(B>JUuKfpKc)P{ynIHJ=2d9Jryj0PGTBe z_`N0uV)rhLlH0(L1)OPfcOik=tw!k%ApsYo1zB)QGdi1JsTnM^1vM-@=4j0qK^ifO zWKNX49ZK}6`8NpV&v6>cwLEFm#web;a`4#UooNz>@YHiYrN)5ehL9vCt_IIKbF>Y# zMKd_BXHal29J3IH(1$f)wiHHAD@A@wG!%)Fg&cO6_AKf~GQ`u+C*d)+hHzcpon@Ay zowNzl=)X3{G}9VM6V$CY!w1x*GPvLmwei>UHahl9K7!z|B~FekddDU^&I`{OKiCII zrqED`W1l>WtAd zY!zaLLcnWuZXq8T;0ub8_d+&Q-DvG2lYG3R_i?vbdeDudh$u`QaA z_K)umkTE}(q4hXw=Y`%D^={{1kVk1uZP2eOxsIYO`5+q~d#nwcm_D$x-k^-HTuCXD z74I(06}trNIkQ!xJv|3rgle@u4L27*l97ihDj73hyabg;_f-o&4E*#TG#2Y`GGuovq+SrDkj~x@hPBTVohYjHrm0OFs2u@-(Ignkpbk=zfx=@W2|>w1M&fA_)-LcapyS0?h=Bzt02SPR#!2jqojh9mAXek zg>+w6x3Aw`TKtG?>iq%ZRv?kOdx%UZN0LsbsNElKZI4vcE7j8TP|76Y*&&+IL&f^z#u- z@`y%@oi;*M31RrX-2yl$nR}IaM$(GQ$TG3>ZJ8TZ6}^IfNw!L54O-{$s|ST2(Gd!2 zybnwD9>)Q$7Uo6b<-HDPY5sM`x?&X&`3X=lLk{-|DL2d~tcwY3e!E@<3xLjg7K$tvo0cUwW zX1o!ZYze@oH|wTOyJNqD=lR3BI@(d?{-dK7<9#*)Zaq${L%4lbKxl$Bq1>p^$NzN2 zlO`-)b-U?K*JLBcPGXu12_*yfW;7B-zrvyO8|L^1dn9n1y40+rwuX!^KtyGA2`{7f zN(Bfvr^;F`t6vwm{F{X#fEz7SM0wLkXYYWls?VLp9oy|1r{irUSqiXf^7p6uj087y zGj0p&vf?8{+Z3L}{rW_YFSI49(%Ip6Xl+dWu*A6M^H5FSC)he!z;nlMp{RvucxgXC zpLL-f>j4|{y=lJ=wUnTXk+p*n#~+StiJ69!Yy?)Hgmm8$DB* zBVs@}KvD7pv&i=ES~TRC2|}E3J_G83KL&tMoZge?s7zO4SdiFW)PQ1X3Tz+TgFpHW zA&|9MAber@XpFtB7up@p`8;#+t);x&xuFxrGp(SI%{UDAsl>Ms5KkS<;*|j z=&V2xq`~nzqKc{lOc^5hez+v~D$d%5{dj%#iUq-cm}bxQIt{#(7$xn2x4M zqc5N#+>@aRPPQKYd=mF{3D3*gi{y|ts;Q^e)Mx6F^wk_dXBHn}Yj%yzwWRiOO1$V+ zuck`1{OSg;hhu#kIZ8#c`itVNCCP9uVLzU@YvL^?i8REl4agApL7ea+X<76CTNK1- zSWWYj{ThSaRwGv_`!}&>4Q5r!9A4S8M4597i(CUb3cYL5nG4xPWDAG%5Jg&fVzS-B zH^@QHK%kczaV#ftn2o%l)}=wk0n;{V12pOcL`x^mDwk?iwn~jz(qjY_Ei>z;bW8h) zxjMjVq8Q(5THxdB?*Zkc_49;ve?v@3Eq4R6l;||mVe+-kb)IpuRLWC}H%K(h+rDkz z)5w;mm8HrA_!NjrjY{rA!Os@qEXxnA$X}*?SF1~iX839Zj9GdQUq&&}4gP$2gQ#fl zFlRv8#Lq?U>g9OBl3ElnYE?UbU;KJ~<%D)n@*J3Q89si(F$P`>=a$Ryi>>hHp~z)w zH<|@h2{+gYqAS9Y`g>bo&wqN?d2%R(J^yj?V*mNqDgS5hx`?}hjlH#r*nc4n{|k!u z-(6LgI<&jCn#)%#3$xwhW2bnG^@QxD1m4KKc=IKPLtv+YMty;L1{sMK5m>fab|G|u z#{f-?jD*F4JWO5@+Aos{(MZkQAqc5@Wc>y*qN0G`(5Z@&LK0fPwqAm7ciu${<|a83 ztTBCZuDx!yoo3#0oUXG<{NBoDKvt=O$aRAkEP;Q*<@Tk~uZLT4$QK#PLK?`Y_mm)t z;k?)Xgf9qqi}LbY<-?pTZ3J6-Bjm74yo00a4W=w^pbGGmL1i=DnxIbHH3smr)hYV; zi8y?svhoy!)TFBi*OcvPfa=xkKVYL0c?-d3zAjMl^;qnS@|1*_E#1U|s;G^kzg4!oJ;KOSuf7)`vJe zfA#Rr9tFhLb}xVGY7t&N>UU{~UwJ?OfnFd`?dgLZ^jkOjyVnl~NZ)kOla%*uwCgA1 zy!*X)tHjDvq3p@{>rpy142KG?jKx8G4liG3pV@1_>;7&7x## zdNq(N>t%i9>e!kb!|u91QM>Xy7M|i#n(YWGW>R;H^Pie4s~UexK^m*5)|9Mn39wz4 zp(s6l?SzVbA}oqTHPON@C-7#eX}@}f|8KGIpNCaeR-l8c@MvtCkD>9916 zo9jFwf%=fCQd&9YeBJAD$>-|b^KJbNe4YsEGK$PFZ7Wa+9u{<5TVajkH`KLLK};a> z)#-E$z)3Om%E)1Q(Y5`MS--^aoco`aeaDyb1kI}`Yv+oBSXOarj!DmyG#R6kemln* zrLd_i)2UbDkeFEgoU^0lhgsyjZ1Du&+==j}^505tc>M8@afNpBRy~RoRF;8c zRuh}Y?uApIZ+YWo8JTlW?I|oVfeF=A^MM6wh3@0w8EfYOl67X4FJs=iKqy&-=AoD|R+AaWHe`SS!dz>V62tN`E)LFDO~++8dY&%K ztzM|GCU&^1bJ&g;DQImt+1nO-oI4YJ|Flb1ZMN#QV6NE2mN4Nh3Tore2y@4p9HDNJ z+A|7cEg_cpVV$nUS|r|+ICh$)g3JzZBVMme@MTq!eX(FLpX$hTPFL~XdE`qnMaH;A zCf>YsYbXnA;4ik+*Kw#=ugtX|B1$HUG_E6(qS2sY#kd}_q8((k+yiv33!=pkH6_!a zMgrP00aWUF3;yMZ>@N+}WFcA&b?(Y-9E;hKZDsOHSdTU(Yg1U!bfP0NlJV-6`%{ThU9dZSYibawvla*t zEbz0PA1*2JLDiwrbdJHWO?T93y2GDd6R}1S7XQ(;8-pJ&yEs|eU5s%#B1^U#!`KGH z-^HcXh-Lq4+vV(cyvE0l?pS;x_!5mIo(Hv6ad7DFK5vVF?U+c0Ti6#Do&2Y0E{P^y4zC}sG6fXnTNr!6` z3AaX6V_<{uG>q>_@mIR--z8D>#4$@|iQLCA9=b~DlG7$%#$|$*q+rDs_pq?lr-Cla~+5&gEg+k1seTIhC)}PmO_xoA^U_G ziH7}EUPNZ4Yoa{su#_Gl**B2fDF`v%VHXo(IEKtgrOqC^0V%h@Y*Kx)+QkxVHUzQs z71U{R8M*NDt?m9KPpNYNmv)OS%!WEP0T-mMP2A)itZQ4gh`(fUVJ6p-I z%0Jd!}0l!%z2!0O{B% z>E%Mu9y@um@BPH8tiJH1d|1#yTN=ul`(SoEK!^qZ&zAm0f z5YNk5%p!h9EwfoY-83`v^{h{_Ji3#~?7;P$$+wAezFWJVmb!pMNf$PTnFM2I`e(5? z4YSUA@{DABAhnv!8I4EG>Y1&My!yAh!7bV2$R%-{lumZX+40vVE$}$RV^cBWrfl9w?(KQlR`nH4S~Mg+?^>AY&wQZ-5? z=q}GGa9L&=48l~T=GLGYt9_B&8{SMZJaBvNsL(yRosr01{}$~Cse$BQH$pDY{Mj$6g__?MP zNFMOmELw7NveI&gKQ< ze`EPR0;ptL{irFiO05QhcLrD;s>}(dy#HLoE^Z5uSS%d@OXS$(2sbf0J)?3SU~TNN z0C54gP3^xIU}X(QKbe?Y#+;R0K~sgNg8tB(`nG!Y%W`}rU3OmRu_{5DzsQGT%j$cI z&_`i+j$de}+)UlsbcVQ~a8Qb@*=7%bH(+IpGIAmEOR&~P`rglF_Y|{1_c_#uEtHbgCfN;I`qYL;ndR=5H6 z>?)`!;2mb~>o5PM68M6A{e@KEmOS?jckm5%J@VjAqDwdw=`K%ohg8(50oN&F+yAd{ zbqa}f$0nSI(*{?Pf^v-bsDjJJe|U{3eojJ+7@+PQRsgj}yo4 zcXCH{DeKMPv~w8NdT;WE=??z38xq=-;~7izE~jb5%I?STbf6BJcSm4{ZBL&z$k^>_ z?M=}F!?dq}8FNMNr>HZBQ3+~lk=cKKFuSXVZfpdi^RZ)Y)jJ1>E^jeT>hKStdsfDO+U?tI2tn!j6Wx!{e+tKF!< zzSHGi;jL5O>dW7M4D0d~-8uYxzxX_{&qWbC2Ni5HqQ6!=I|hFORPF+8l@D6#05^FZ zevn$`da!)GgMO*uKn|<*8>Ysd5qrN>7mKLrZy&CE#M9c?-7MfQzyQHM+80#L;krlu zMFmHYogv=Na3R7WQWmxt|8|HTMJrC*1^caQaz-|1dqllFHtb)-NzhKDk0)3UA(YZ2}7yt_LAYR+s=FOBQnYn`KA4umlhz7oJ9|vAth7;r|{a4>T%S~FjI`) z-Z``OHw0yO2#$9a_yMPOs9pRUG5#&S^?+M=!%dyNch1$Jhw~of;5y>i#mM0^h}38- z)_2Y!qp}dB$F#->#1m2eyc|F)d=hbC&gPI?)EH;;goNuxjW*!OY(3m|I$Y)O2|2$x zu>En*Xwa6-?HQ@(#VKd^0EeQ)7veuZaQq@SSmKyJepvDTukbNx3tJNdGZSS;3mY2? zTeJUwk41G}oFF#@B}BIy3!~S|+4%Fbf2# zL|T$%x7F{(&Ha9&tJvGg&3_ZWUh*xB#w-ka@^+$(0h;!CknQ|&xH36(#E1s0Ss7G2^DHM0c*pz9q=zJv}ug-izP%?9` z(gv={JjKTgW!hy8^f}~r?pY;g-pwhp2Tq5Kq9HVT~In?}SP|;~^JW{%*k-1Kt zEHUsNix)gAQp|@%nJ3|u55XE;JX9vO0n;aAkFUfsK`H&6@_I1CqrE9UKZiAw!2EkGF?&0?%A)2(M_GKXPt{u74B)?Im9f<6Q_bAihv0h zT_PkMQuit}AX<_NmtN%qnXc|sPJE~yk_GZAC>Nwixe9Sz(F0DFR@!FP)fdCUV2F5Q z)AY_6VxxvjR^D>EyqKAnj#nWo+HDIvu%pUcC+Ktj&(lJxqt*wbRSf^N8I`5_h2Blg zMMUWO1;!iq9b*mWBF^(jM?S{F{bY@qSzl=#(LSQ=MF#_vsY|~@a*xZpv;A18uoSV3 z%@*NAqsgiFWy-KD^?9sV{wG#55cHzbzP5UbAWE3)q7q*9$0CUV^(GFCk%X2TjY#E~ zhm;~Jw3CTUB1r2Qssbj-Py!DgDpmlLbN#;RYQ#{gJ3=BX3{$ksNwCO2q|h)Xk}Y~w zU5ov3I%}T&uWep>oU`-6YcYOKZw(}QJxs7Fd%Vh%cO#oA)y^GBlT$Jsd3eHgzpJCzvkDjzSJlC`=+IE z5#>;ATnoFlU^v<_fD2u83!O%Xq-m~rQZHEvuj(ES8zwbhe?1tg-XS4~it${QQ_t(~ zM57x>stjFR%!=V$72daJJv+@dljXgcuxg++t^|tb#2)1w6$rBc=CDnWxDK1L_-|!D zMlgB?TamtKf1Jp-Hoyw=o>NSFI^gWX=cPtC#8++554O-VjLqfp;sGGT0?uAQQo zPb~%Uj6u36Ggn>9I#I?F?>xA#z`to|?`q7;6Cix4KBSpki;g*JOX)|E0VYwVZX+w)wySG;z^A^B& zI0%AF*&O|7Q_5eU3YknZn9_+sjbYcUhCb0sBou_q#7bx($HT-u4}Ir!5I+=wM`ssG z_R7c68dMIR8(KASY3eUl9k|JE$AS{h&J~}lJa%N)(*Kj1QYS#Dh<34z3V1Nj>!a!3 zOhhjg>fIIy%_3SfBPp0%?Tdp0n>$vUN=6xqF0F##lDlz4IqG0V+5pX?W0DEu|%^XppKi0@HGHTb?;h_vTLbbOcUg z+QuO=(=So`4l>HoEG_>U=MDj##_e2N4{g~T)>D{v#git8ngiD?Za4&|!Q6^dCkUGa z)|Jr%h__Awy~2&OVKWuANIdvZ;^0#KW38 zrlE>_XGn6T1O5xeZm9fmp1M;u-v;itoe)RJYcnkI3*oaF2WZgE(K{R$jaBz#?b&B1 zXes>p1iY`m8%}2=!DQ~D^%MWY_;|;wr7^D$Nk|{!-2jANQVY%p)QyC-%!?L;U$h+~ zP+=3>_iu}tc?yi9G-|!GDHp%#?Lp_tE(K8Hr*%%>dz;xV_tK6Y?ju*%+&35ewvb47 zRZU~e1Qpf7eq?%P za`?`Vp1afw#NW=ObpRyGxSjY+)wx(nQ6JXZ8kvyICVt89M}mzdO#7}M`b5O>RN3Rq zwT)R|jd)JRc!l}M-vLov_`(~i5fW4KzXffGMe9B;$js=gN-uX5s3tQF9Gk{f2vqBG z>wA6x%PnIXAdqZ-I6xA|;YLs>hVNnD7qX^Hz0c$$SU!wk^wpN*#{h2-DpI5~6Rn+< zg>_U9aP|`NB#5m9DN&@sm%jpN&&V(|YjtvUPhV|fl}XvjNgmYi868tea%Li_PE1eq zGi|eN&cJ0)X7OQd8qwroQ+0tH{htdu)$z$TKdl64=3@x@lqEgCW<3YREG~(U;DZmQ z97ZdNLGdzrP)mqP)0sZ2oxioN3qcmFSGXE@4`a(-46;fTg z3!0!EAYe(Cz%}~j;?_Be|9Wyk6BC*W3Sw{Q(#e$GL5Tf>e}v75Zx9xAEX##%cczMTk& zSaNe7?ifleEi_AO#dEnh?%lngu=W%8+y-E&8W%Gj1^`XSKYZaji_LF_Y-C)7UMe!hb6ClA-O2cCP*@( z;CB{Jh`Sa)!HPN)>qsg*iE)vNB)22V?@*&CoRlmr!sBumqN&V!c zwBBBE*GYc$smY@dS?W^SbxYIVGW7`V-4P!)qJZGRfLr!2qp~yq?S}>AsLIg$n2t-F zqbeTZjPLvruNuHZ<>+urPET^2hds*YSGi*43 zY6CWzt{r+tLpkb%m@3z)^l$NkIai8oJ~m5$>AbLamZa0IA1~`Xc;g?81sKa>DF&E% zMNO%RU(yq2rt+z_6#^}@Rg!CAfk{i^{!5GEgf3ID%XWjF zxZ||BArVk=Kd&w3DKQ^H5fon|%6(>?2+%emi(Uk_eEz!mhp;NY%-(J(*CXACu=29h zr$|Sazs&tgqm0+#bpr^Vp_K2M>{WBfp12Z06t1>`Dvn+Vs9!{v&44+r<>XGI0 zp6B+REv8M~4E@N!?&p?(8_1Z9vbfva)^B_K%MY?&q|MxitJm)9LL27XhiM?5rCW6c z*lcZbYqsE}`TC((n!WG7lWWZt{ZTS*kf-Yx1me{@Y1%!o7h8{nSIzYPB^VX(c*CBA zs&z}OL71v4dlQcL12-ThNB5*Xu!@K+7W2NUrm`p5RgPga;qR2M0) zJchmtiM4*Tmp4u;fy2TZg)X&Kw&yW6c1iK|SEQ>x^|c&JDIFQbulJLk04^Qmfn7L} zOTuV%hq&V3<;+qu=Xqth=L50x7VeL;YmldzW(e~f-r3g^6^sv#2`7V-P1s&CH3O}a zlZlM#FSBhNVmFt3n?E(=Y7zA$dr=cWVjhl`Q`^qWZcZGZ6;C-rH4(fl6=NDSr}#O< znF-gWar@4Fe~|CG%G|*{kiypmv5mW@Qds=xT3+-rR|UDz23N2Jv15kRjKa(_vs#pL zV$sZ2q*k-HUmTYyCzCfRb*Tz`#rAQ{*vg$+sVgpl@$d3-eQ-zIxe^;|SfJIK$!R zP00e#!#BQ1h@_FCQD}X_IsHo8okP1_m1E-!{qC=EJ@qEA_se`vfYgjFrZASxfGYcY z5aJpv$w&WKe>4&LOdBBvaBe<&ukz(w7u~aoAm|1mSl}lXTA;ukAzZiE&dtcKUTA+V9$ZL3Uzh+^rzrpF$Zn{R zfju4;9wLMhguyMzn*#)>%=!`==%@1}{K(P^(Q?koL>VoE=c#wj2_Y+CFYV)5HDz0Z zE{&n~uIOMh5459e@85XT?Jg9YjCqh9Z;DHzzi&~RsKWHO=&9`*vqF(MMW%Xr8bBKy z-?(J|3B}I6NR3sYiR6sn1SZ-Q()ODUHq1JEDW)%Apc z!huc|Q3R*LaMO6h_VQ4=?1;lY51Z$t^r9|j=!E@x0~7Wrniy$X0u8=-uNNO@rh7IT z&8{SvDdHT1Y$4U-yN{|=oz%GKMJqQyBLNwvkUSR|Si5^%fKy&u;^d+0#y**$HJ zhM6R8C`?)YCCr{C;+mX{VxsJ)pjmgXW-6eUM?t0*Ga%Vg_GKa-m%Q6m|9&$yMk7&w zAs`1?PAgQlLk?WqFS6#O8c5cyizHR-=|2-4&0ak9VwZ)TPewKyGDldJh_rWv)p zzr@@Iu?nT)X@bNhsMq{TU`#W~&48u@bX`GsM(s|IxZ0{A8S1rmN*bw~IO zig1tWEpG2`H0|7F zt$TT=ObobnNWMx{A9z$)@z^=3+`6uudBje8X8vqqda8 z{~k$__S(%V4i!KJa8^EJ+|w?{(s80wO z0-MvlQe>Tv&*PL~l2E1!nXfq2uc;qzb}xcIuSI-`Js)q`#NWx$djUs}1o>ooVTYi#dZWyRhe z%nGf8t8w1wqjZa3b+c>7Ctwci^VMq3GJ6%fCiS$|DkMd{CDjR{Xgq zQy*x4c&}}+Q7R0L`P_!QZ`N@oy}B2ervskvb_Ge|3Gc`ezxkrfA?CG2Z->rq440_A z8Axux$28v%-$|Q4w-lNZkg>k&nVViTkd)2D6!PKb#%}ezaev{P*1xN1rY-cm<-T#k z*pR(!8*d01zObdOip_tXinDQHNWN+^n`aDEMaa5oW*B+P%txBIHk>m_o|gYPxmp+x z^m-{&DX1wbf6jC6w@lcgM>}CHTi3?XWp=PRp}^I}l(6h8MRKR8o(^J3a7!FREWq&l zV|rik>*HnK+c#?ZVB}2DH$l^bz{L?!X8`h)+!sc+4pZf5;ZbE#mi~eLRK_=ETY&YI z&rhj{D~n;sLw4#2V&ou`T%8@S53mI0#kq#Hk+Y2TPMC)0O`N{9 zCGMTtGU}m3a2RGUbM^BaZD8Y5fo;p8N+B~SGhoBW`dGXJB9^gSfpdvjQ5DP0t^`?y zgm=CL%!O0r0eSED>`~?#FOoNBRsNDhQ291fM5`blm{7McDiED3sYmMM-l{RvPq+dI z|F5|cg?Zp2-jcIG{m!&KY(YBm70^4@o0s*~!nS4fzome6{GGruIuIiOV*t{j#Q9UV zRm_EIrl@H=r(65}THiZ0k7Uun>yI$D}KeidUf(L2d0-tCmfIX zQP3Hzhj%z60ddt$p-X9hXqoa1>LH^hAavpY$^(z`ZMl>uK2{NEo@9DQ>Jg<1#7fcx zswc%3(4$o+>8U#dXvRkzvLs2|>k~^#c~to167{@wX>v;L@5~*#8qbxp=|E+B>1X0b zr0cYU#-le26Q&yq5erNwN==iva>;aSgD}q03g#WpEcV*w(k{QKlnAKx&e8ZpiW=Ml`}u-X7|_86Y^0Bv`{&qd==UF5E_D4`3rO%HJno8U zkt`pkp;}~&sd%W6Uu-1DU;bJe7+_rIcnd)73ds#&j`l zm(nCn!2~^p%tlan*Y~whgyqhn7>1{wL_qItUDgk-0Cs@{_5_&y~0*hJAQMlM+Q#xYG&1YKGX!8k0pmD0i5gbKCoKq^t6I_ zj$DaZi(%5){(AtN`P2Vm^{W-*IqY=+8Pqz2H&-yDg%beYJVYEIU&xG*1w(AWzW~(E zjWh;Jw`P;+=9CYaV-6Exm6W6S0lT!Cd)D<6NfU!`7}wp;TGyG87moqD%N$$t3563xeYea)&Q2MG zsXfYK&lVbE@~jSvuVlnliKSEe1(v|xUt+F6fterOdC;G(<1)I8$u6k(Z-BgbtStQ7 zI@l&GX76vQJ|qX;!WO|wK>T&l+x0a$`<1_W@$YTnz01j7@gM^Zyv3>?<uQ}OzQ zw(c9qiR4NuUkj-?&(^j#_|l<=HQq*Mc*vI4M3F15tkCGfzOX|D?KK1H1q`N* zX~U03#EHx(w$0?nG#-w1HJp99$`UYp;@z@Uks(HmLRuMMZ3FZ-9-hmYu)TyzsSEWH zm3$4stgJJm12kKQ*#Ahpc}izL5`1ht5tvP2({SKJfAlOruK$J(fggK?sLu-)?%}lf zlJnm>#EFO(D!M)w17S0TdS?mcF7WlS{5^kGs-rx5@CSvZ-CvdZa-}uNB{;=$bNC7; zOYFo-f6xRbr`#cqn?+NuXNhC5RCV~NhR&bZr}*jc7WI)Xnxs>EuXLC(Mu_@)(H*?z z?=n?%2aqLN?u40zFOkjagPBk2py3rScxDO66H<+NYpGt@vnF={`$`;4zYIL_hKBjx zE*x9D`_{x^f?v<;b-(+mTrTLLoa7Tjt^`}L)eh`p5H{rq{e7FBtN?%dR`X6cQt zS9M4Ks)qB=q8Xu8y#Vx_dxp@hx{LNkT{!(1_c31MgVelB$Dzh((>Txqsv`CJIdaXq z%oDfk zvUr_fAKP!UFnSELn%RG~0kRMZ9)!vOyA?O@a4f#H-WRKnA9jmCf$dUz!2{dv!aGx6 z@jXB^0S4G-Ha&>fr?JUjj$`ELmfI>EyCWN%ZvjB;y%7vQU9+ztJLZnX5=rED{Mp|v zjb6cMfmBhq1Xkw!8;RhYaZ!UDO3eu9=$vCSmM>KP$n#3MlVWe!6sixrn2=A60UC7{ zUtQHV=APW!$j`Xig1A{AnXkry8c-!#?_dSpSLlHKm3HlQ1EweOdpg><=2kED_){2L z>A>xkH+ts;k>yKXli&_=jgl&ZzzDdw_rQkBk7IWdYvPt;{z2)NVAH@;4pSWO(k=fB zj~L=<2N z)1hBkA^sgLRbzPc3E_BIOO3Lq%NC-lf`aT1m1{cSyyA_;m`z{Whg!5qmKXgs#>7!5 z7{W(o^P^W;HQr}Wi8ISWJm%|6(FQTtf9>sFmKS41E5lnskD(M2lo3xFhKlMU4=7uG zH(#Yd-=7&GAc!*c93jbx%578z5F6?TKh82TV<`{aHyu*kwF+J@cu*hWz~cbDJ6sNZ ztao3(GO}=`*PgqM49|=55n%Khb;sVMj++t>4MVXuP6rdWNpbOU`%PeAFsx(70AL8R z8>E9202ODa$!KzXi#hMN58mM#`E7BjE5(fOjFZaN&WUgsd5o>&$hnPe#V)u|uuM$v>GIhv3agj0?rPUeak?K!A(&X`INzWe+E6qdr#yvYYCQ6Ry*zeI zGJY#1TPBw7DfFt+H*hRx(7mng0;T@VT-<`Dt1Q*C1%0(ODz?H`=n(JJrx1(=7ed@5 zRUEK0u!9>V&cgdTP2J7yt~yv3fm@LUcC--;5EG74QPFQ72L^wJO}s2$AIL_4ZC zymY3ByTtFX5t<+duZ|)9syk9odGY8EmUG@vI&Pl^qU3I)86i82OY1=iBPW5dw?t2A z85smsN-#|4vh{_r<|91rp^kI8r_?i@V10)_nSE67v(`}hJ8*bbqWHn>yh0_C*#kqR z4t|cI40x*Oad-b-ru87s+etpVd3K^srPn{7e829jHQC09jGlC7y)k*Y888Z{RfAv= zbN&`?fG}^Sb9B<8^XXK8AfLgX4_Z;LQ5!p&S|)0&Nj-c2!^7C~-TauHc+9r3zVCn{$-Tf#y2-&Vl_r*tAwW=M5IUxTPN zJRh~$q~f^1sX}?^#cyxvKp6G}Ce)tk5CB#Bj?no>9nuv}24PjkCQyD;hK$uU7Jup| zw6z6ElIJBfjxs}EA8YVZh6RIC^u7X{681tLe-YV-Uo4#hFt0|)Binxxkf+SBZ| z4^6FH^t#ad^iA;|E1mbI9EODaE00_}rS5}yv3okzzj$TaebAREX20IJLdwf4n^!$sH_j!;ec$G;y8KOt9WWkEd_cA0k);!~8|T(I89 zJuqez>p}tY4~lc4fh_v*-w@#9!KO6pFy2|Vxi4mroVMIH4*bnCfD5LBuiaTOFy_u2 znhT=`n#-1XY27?D|H2O{yps{|Zdy*A>w#P9e|5QUh%@1J_lqxX_<_uueMig)ITA$} zx_sCPf)lyd*0PYw^8Zlw4osp%!IJRU)*a)HZQHhO+qP}nwr$(CZR39PcHhP#cE5-h z(ftQbM_1L!%*refG(*D?o(@|;6S~Ca2tH8=lMynbE~+#`-2MeW&@`lEfB6Zh_cLk@ z0lb9Vo;;P0h?E7^r4bjXy2t=EcMSMEMR!`Ldom3;*a0nZM4q4mu5bgc-~+Bql{*KG z@lwSdi{Ev~a+miJ^c4tn^bm?|oWysP=j;VaKhz#6ZHv8)n!Oi=UEPKt3eotWbF+dJ8$nWk&C~ zo&IdYnyox!|8myQYfwRzchgJWz)q~p?RXh!m%M(@(&ktM*ZfhfhQ$`ZbCAe$F@N%B z0rtMI`pg?Z&&Xljq^j#*0WZygEJH^L@{I#F^!+;GEK#7BWWt4^mu67PBI(}UWaB|C zn&IVSk>xrQ@>V?jHJFDxxo=R)w#nuGm-$({&mV~kK+sFpBaD|iWq?RMK zzK~*xfFda~u5jC08{*KgRF}Kb%{|Vt$)NG#&{GbOxid4a_)Y?u?t?>ST6AMcYr#4q ze`u3}_PjE_AXuG0yI@$}_>g0pJ+`+k5TQ4!&)2Qj7c2EfW=>z6QFZ^0^~cRfE)IZg z$lhk^6aj@6h&^#a12HToF-(^?pWE};XdPjeb9W6;;Sk4O1@cTXuk&J^UT^TnzB)pp z0d0gmg~A0k(VWC`BaCjQZlrD6GDNhM~kw9W!q;_dzGi_cQoq;9O{K`@&)Sh1$FX)j`|mQV?eq+NZbUt zu{9ZZ&VwrhZwE@8w=7aq0{(7yaiXy?^xTwwssQi0FzAKGJy2;*Sz9dn%VtEe&MD6Q z*b7}Ce^GElKJgCXEja3Rp1xap^s9)`u1lO5Yl3zgJT6qW5S}q_5_xR1r0>*a^Vx2k z%yqUTo^fw7Y}jbZJrCkbBI!g6oRo5LHY_FS?5Gz_)ZU)kF;ZoH z+cr{7UeUO^D6ph*SuR^PSp92Jx3&yvt!!vR|3VkkalagmlPjKS#@JqWtKtDsH}}RF z<&N%AHs_A;jY|!ikvd0s7lGx8VIpP^$x5=0Sxp_LU)OwnE8f!by=-vF-vV!r^$m#4 zx$_d#zX?~-$gOwhLLeQy9Mp>X6Q#ViZ$kI8I`x!h1m~405xa;yr={kNG}5bI?-6e7 z|9O9n8M(}Csa)4Vd>u<}4&R5?o&F=9cplG*I!3Gk+%=0@a+~ko{`NX#ld=1oTj&9v zR*?uHIS2Gb&6G|hF3s%H*izYIHXbh{85xv<&2BbMP`3+!qbUda=113vk3-5*isx5{ zXO6PzC&r1>4`%_$gVrd*L-Uu{#T%C*aB|K1$gVzZHv%K{ie4GAHosI(zt51YUOG0q zz&NbXbgv$*BA++kc(!h>F=d#g&tPn&35SzsylL$~3yrfqQ|5fqpnNMiK8tRe^n6^0 z*qkXjhlp6&ws`oYinv=*F@*4o9ebZHo%TFzfJzkQ#7; zc>pnRKa9X`NG@Y)uq>XtDeSd`xC>sQ(Ocf*Dzu>z+jtoQz?o=!9Ne#tf?uWZoFK{4 zHeieg$UUMlCR84;A&VHWo%^`z+HS=$q;W@_-7@pIN;r*@$RISOWD^@q?P$utXEW}2 z?37w`MKh)g5zUerr~K66Na3qFzml8Z-$r9)ZgMZhmi|bIz6QHGPrTE~FPDQX z!J)5zEsL%!vHR#3xUpO6boN4@79zQOj9>EKU>#uA3k!Dw#gk^eqbCOzU4-#6oakd+ z$UO%g>{HbQdnq9Cjn^57V3cp{g<)wBjFbY-++{Jrctf4+TdXfY)?Mjm;9u;+VV?+< zlS$H(S*yCdayX#7=m=hDm=HPayRa{k(lSeHqcb-ko*a^ZL#7mgeLAc*FwP#RlTENC z!JkA}?VnIA%PK8EN)rh+5OsV5S#T$j!EDsu2g{N5IsS@cyGqf@vF%d2Jjo}V=XdfJ<=T-s}_{^mVKJCORS zUma$?f6PvWZCy6vrxtMVu$i8Ry3F0=z_U@=QBKKUD$Z_%XNJezFYh5DosZ#d3^#0e zTf*3CgfKPhK<^Eooou+>nCNetT!Y&wUsw{TzndXIK8TW}FH;EPm_i;A^piz>E?*Cb zg$v)fyl_sZ$eQ-|ylj{WB_s`Al-jQlO&&BCL`EZGaK4-kt=T_yqdW)LZs9wK~!BW7`%EUn|6>24khMTXKGt5zU?NtpI<>wJ$tDpb01ds`_%qIt6#tGFUtCV3thG|b9XfVMOkI- z{`=ypBdwgQpslHct+O2w{eRzLO^%j?`i;dFs{D?QDR2b&Qxl!TkD@gOk%K=%!iba* z7ZsrDxDUvcgpRnSDXpqF*X7Z30}mh>#bAWtAmC${n&+m7MH2OTJaN89QAX{xEEAZ-aaOX)Py3E@)$ zb4)bSDCM+bSe_SFmsd3_olN`f3ci8}vRnv|$HA#3pz&zs3Q9VNWk|7Her;lY2w~tf zrYkB)*sNrEY>qQo?=PH;yX=g)Kkxe_elRBnV)$5soDI}w{gD(p5&Q#n5s0XTh*86N z2Tm}DZtBArfb3ZPK@cZHsfiRbBKMfB5*)fG!n=&tk($N};la;|L9mFru&PF@7-fd&WU+s?n^l-3DsL-F0@!f4J+f<{J(kDs=DBWr|)5 zUPOj41SbOx#>(;_(PggZFL#$_qasw$Io?cR=G7amFmh<2l6{wA#5*H;6u0ZWl`u(X z)~c>Mg?(mFs9nuCcm4Jv4(YLD4wM0po&o)+v8Br17v9F}W78uuMQcDAOG$9G*jhPnBN)n4b zLEBw{^Wn!J^AkmgA$)>I&7)QU+<-(0Z19)sUg$ql#tKk8$4u73mPohx*@Du4GH~J- zv==H7H2`b1lG&Fh<^(}q9+e2OuiO04! zimmqMw54@K(`LAXp^HLeW6z;MGu!aiGGuH)puD9UZ6V%qH14^FeewH!hY)-Qha9}CNkyy-WAyNSUeG5@!K@loR=M< zx?4C3M#{6nu0n=SgkA28XTBYo!PLu?BN>3H4izW#_XQ*c_Jk50bBRmk!__WY22dyZ z>-)b@U;j~;i3&_^*P#Ibg0KJpnEuN}>AzRyW;JLp#buly8HdcwqEzT&>2zYgNH#}XMsTtOVCTYaZE8l?MFJemtwa} z>`NlGOh>b?)f~{UYOU>)9NDmMo)2-${LTtG?q;o6zpi{)ZL{IE6V91fx`W>jTi*{O zz!bPnh9E9&Y*+Ddp-Nm=B9OTDhO{r^`}1CJcWKx!o%R6N4rg&~a201xhFrL=h0waQ zqsVRw!-^De?GRrfAU=$E$ahXWC5M_0ZXycoRQj_MpR>n@?p=8Y!l<*5z8+L19%ArqgTVc>J)-l2f^A}>0 zO0nnRVp4g(zOGW-O#@-W(_(J!*3o`yWO8yikfX>RZ1n>0#L-+uK0$s9MWjyt2_qe1 zRqT19&wCt%2+WzlNVsY#Q6W(jfSu|1%(8~t#~;Za_~~?l{u@F1@kce4{Z?@Q&G|cR zV>dlL5Is88#mv9)W_@dBAnt5PECbU^&LY~*9+imDOQz(r4L8zwGI@zJ z;YxDH_ZdaJ*E?SXwEuL8>_?)7tSPJ>CNODJ^16Nj!E^+*oWWB_#H5EI;&l%h6|p2D ziu*`P(3BjnC7_#ktUwW^oruaeoKS@x8h z!uta^2F&OoqO5PFnAr`2EoA=`-i+>g2`R1zvF6iYI+is zu;WXq;tH{tm~~Qvpk<475Ru0Y$=Ec>f%*FaRLgg0K72EchK4!|Fw7YA?ss6Yyh2KY zv3G_mr@!$@%?(Yf5=<6vkUJ{(F(J7JrXX4Z-ANyk%wO2i6g$gz(BUT@TljRhg`DLf zx(C*!d1PqsGk~3KP2JZ_OdQd##v0p}UrEz&~aCvTuzMuS9ROm_HG(%wMp+ zf6!gIWT@kP@(%DYf5?0PE!^3Dg%WU!?LoVNKk>l+8&<}QX|Oe{eD)`?S}TC;lw*n~ z#m1~btsu#oQ{GR2X5CEQIuYM%`hj4YGmS1`LO#+wg3emvD1?N?V$uS1pz{V8T1P*1 z%pO0Uf_XHo7pr1IK7X2DP--AwMlL2;Pe>}6*2Fv~k(7sGq8EZeE#qBp76}ZW1SQE% zyR28_;@8a{xT4pi5qn&alohNV(6$X?1Svf}$}xN{VSt9!U&z8@cy^<}qiepD-?2ox zOFdRu*gD9N!DkR0u5wF`{+l?!#e)C_<-de@&A_-tR2o@5aZR4NYJ0Atv4(d%V$f@i zqM0RBq=`Una_5a`BX*bGzr=MYw9IG|QJl8`DCq2udlbE*v|rx5Bf$^FP-7K$kkBL8 zo~3Un*l`v^Q;E}syDrXWQP$ELl2X;Z1u=nX6Fa$m(6F&3P2P9*L&pecw1&qCFvx9c zRLJLI88zvou{NP+2-dc=xQP;kuIJHMPgMeLJn4So$1R|J8>p;+7pQC#!SvHcH=jC9 zx7(sLL;l^2Wd$9$52s7UZlO{wZE02=E(Iqu^8TYm$+<^z^W|bD4C;XXH#8sH>r>thVeqIy3=U7SU&cH%w9y6ScDnAcZjQQD zO`P0JCIgj#vm}Y`1wox0|BTw8o)>y~)qj7YiY5vlerNavYPisD^ySDViXt94mLa(` ziG}RN(!^RtXmj3GqD8-K*B*e7`tsxTHv92%bFL2!Znqp48cYY2%LBHfv;ak>!r6-syVUl1wmmu^Khm!N+NB7&+KS zAW=Fgojoch=#iinwvSTf47&f6?}4SMRb=;bh^!$Ymwh_%Y?gVmj-xobmXShQ3rXtO z!qDRri-NTlQ$&Q#+(x-9XeoqN6k_UKbq7e?+ca)|1lJfJUUtJ=-DCHHqGj0c z>0i9vXLQ5M6vD0fGx#oq6^$hfz#1-Lg}B@%Of_6>z-@zU&qGr)l5K$85<$I2Vl%33 zKzW1qYEbG1_!_2KC)qXfoLNN1?ZJw?0^xv5>I%hT*u?CZAd|L594~88IkpS-@iXZ* zx8WRloiv*v9(xNXHN*0N%?V*2q@XoCb{A%P2Pt7V%3#mUGp-FRt(<}u>GZtqDouRR-oO7N*eV5!qcw-TG);HOVxu|jRf%1!C(IIY@ zt@MLSejc=FY0fTqLrn{qjGY&WYck9q@_`1~1g1saaUo5X} z%lu?i4i#H$2;fa=1fHIVPu#N`eC4B{Q+H^3I$r+5s{(!OLNjh*`HSK>{2^S&EDiRN z1w5{3s`1{r(m0UH6_&fe{rer6S^xiXnSZG+QNA>SehD z`IvJMgcq9T!h>`Z1qyZE)wskz+h>V?Em9x7#IGgXpC_B`tk9L( zoXf5DxbOG(CLX}f=p)ldeVPDXiqNgUSww+*T1mU{d{t3h1T9fjar}S>Da(+vVq4O} z8&d4hU3?I={-@f2DpC&J)5g}ql18{@i^*ALWFw4Kc)sVzoVDn3c7leZJJL@mJ<()H zB_|9Haz+h%BFrm;@L&Z+MR|&An}LP0@Fufmgu0@$@$gZ~1^NoGkDMA6kLs(^uya#} z!je^K95p?WNfeZ6o7|F9&()2Mqov?>&AgQEJQ*@&a@#yxniLyn#n^w|Lmi|;?U|i4vO3eMT|f$=h`0c3OX`z0rQYa@3Ml_WfW7sp?dUAD8NlrkjGJ8x0}YKHPVfo#BANpAY&1+ugOLf8DJ$<9&it6 z4!7SraFUvC0jJ8Z>{Pl)Wl-nMH8-u8=rA<*P|gx+vCIfLUV|UkC_GkMl3vX}yO(;Q zA6!1WMqRzXbzQ)nc@t$%r^(J9pO=cf-sOJeiY}V%*w--TsSQVNAmvIC4Z_9%ZX-FA z)H*MPVbh7N75r zt6X_q1B(G=Ml7Op?LIWOk{H%C7V z)yo4F-^j)d|Lh!!8=16WwJ~Wvu?V+Z)Sb74K7&o0bNGx>lt|X0&C!Hm)9lcqNX;Gs z@8^2HT#>7jWZ(vzYKNhew{u4EO*UFiyNF%P6tYMv77_~O@L0L5x(p@?gO$TX2CYf=|k`JoZ?EcT?+>XR*YAa?@tDwB7qqc z-eDrioPCF^s4>20FcakG&ldrQZ{E=chY(eD!p9e47j3UtdvjFe)95~#lrMToZi?|h zqAy+|;9FH|XJ6l~m^E|U&1J>n3~oVZ4DQ9UJpo1jk^W~VGlCwYSHvvP5CJXF`;c#x z8;|)zRB(=VNDx;=NFpZ}%c{RcME%Aj93|7&6$@axyX@L#}29Bdu`)62S9HQZBa8TH?GIxFM& z-P&JZAqb%egoJRyFnK^j0zWVTBnUbCaxRkCqyD4mr)e0Hq=l6d3WyqIC>lAXzY^3C z=?LWptk&linw@LQG%dRSf_e|=)2Ip*9_)9zPjjwwo@dhk^?Y#nt-JAtg})_$ zY$JK82Y3dyp}m6I`6BpSk&QU7u>-;DrD|QnA#stZ!bkVHacM~o^=)&sN9-TVqBFG+)L*}G#&5Bun0<*~l)q={01STA=Tb_a|>< z9m;MAi>E8NHX+h)7S@PPLR6RH3wQ_0*n*bA&9ZUkKx7m@ZdO1+>*&@`tRD_5QdFT* z$YsDGM0It$(A8U4sZh~V5za@cBt(oPG6=yjmoPd{&D4H0tu8+3aH}Mz>*#No9&s84 zTU58U3PV&}K!|;40lS^*j$Bf>Mx$=4knD?SJxXE z#vqn+k3U(J{N$DiIJgA0X%nmA+GSQ7PjU!I%{TCajs+RfJqxNCVi&EV&C`}E^9n@V zw%VwUcPV7Xl@*$jVU##%v)-_Na+*;3)U0CJ6f4H5*u0;oaUHKt zn8|Rh-^Z9*4TbvPiFvGJr(3F{YDf#j4rf9i7v3rP7#a znL%g%hIW!>g&#>9O$^96^RvqDUl?V^v>hLVLGD<8QM3c4yoMFU;@<#o)|10l*?YfW zf?lk#2s#40Q{g@Dwoy_mvcygztuhl&ZPoU|i|_I0P1(x8RYpToInhR<6zsgUL7Y-B z#R-#&gYi#78_}v&Vc^sug^97W$-wW|RfsW`{1Yu(f*jtKC{dGlO8u}rxu2rsWAa6c zg1Z=S^={W#R;R<2tqRXc0u_-*oa0h(q9EBJ<7zssyx@Z4tVYEj2qKMCwQLWyi=y^) z|0ol&S#-!;1+(XjW1h8QPiWz+hSZG^>D1+K&CwHVd)Wq=i$aFOV8_uDczavXzi45HB2gAo48BCtvjIhKD1c|K0U}ff5F&--#MC3(>i@2-JZRn`cxmX zzE#J#-gEk&|4BUJ@S#2a;vdDkZ;9dEpOso4pwdlhF{ma%?G3F~7}T9HP4)n#Tb-<& zbpAq}1b)km={2n6dn~@W;jkIz@V^Un*Q=ihslrgT(qR&*C||YOpt-#QMCJIhlu@bJ zb(r*B3O3Yu2Id(7x)x#JjB^@2LLRXZzb6*-s26j=bav1$HQ%TkkDAE2aaBql2S39v z6;`1+9?dmH3E#wX>$ciV=(f z^$^$4;dM)k0$D6--m|!=Z*n(^g0yW)uf!p=V9;U=#TJEq4|>?YOv92MRqAhlC`eNh zcsRwlelE67J_h!MJJP-Z-r|m6)kSZNGGb8xS`G}h1(>kLe^}K5->n1oT+EZz9w9=V zh3Z57s%MYy#bwrtxwHpEk91=pe!0J@tG*0w zFLh5t3`22+{YLkP)sdM;GFVft*65=iW|CZR!otpYd2o^!UrwF59X>}ue>MIbjJS$R zG6fmUE_i$yiYY-O!umipqWimR%?__ip^8=Z3Miubo3DOaP5eUn=DRTs%|L~caFs0B zQac#yTsnI9?@nO{yhi>=@KKE-CS{`&d$jQHb0d5taq)6;PNUG#U2`LgI0|-iF*96P z!2qnA4{fW@l>&ko&^#d}ZvP&Tka-OMcOx)ZlIj zsq_^J_S~ToY-r|NVK+PWK(Voeu}<;B8y~zulB-b@u26Ki`y;XgwrG%@ ze$zKCyE~qQH&UKRU&BmvXicd;Zi>3fvUa4#x*V3bvU?Boq?&lki~)1EaHK+uVZR(n zfEo>RKuzFc>mpd(q%eRD@zZHyVT^h&hYoQ9#W-<)p4)&4EuO9rIog2t&-?DV#!W~d z$^jd)j=>seQkEPcbFkD|#ho0bie{zW0C5bdoKwo@>^`!8K2-A1*8pGKWh$1tfKgn* z4bH@hA9T-60@s4#%C*DR(+IH-=*_ zbMl<)kGOf9uJmD@=Y8Pr=b>}{q4l7#b;vvspBQ=ET%KsLhio}~A)T4Z-|rYJ#}mOk z1_m(tQ8S&8S+syFhy0pJgeWhPYis~?$4y!)2tnr7nkACcSfJ$TJil&s5=`C z6e%jw%IjI~8C{E{AQoObn7(6p4>&_ZcI9XfuZA7q-)#$lHoVZsc-uMyj88*Rtn!Xf zjOt^k(aZZVCgNJoV96CbMSF1rO7!!LWeUbGwHJl^g`l{G^t-AI&sxGFU_b{rhdgL~a#i7eugt0C~evQ1b zi;0I{TYNIM1gGczS%zo~i!Ghr|3q4E|EQ{T`&%-Wv8&+cQPf`lImK3qnq|zUwffXm z%6k3!KL$j(03<_5FaUr?`2W&?sAy+p?)0D3nTn<=vMS1t43dMPoX{|oPJy#JgkDlb zbwL?=l^Fucu%z{6T$Dc1R8%6Qz-{%gif1p^^Kj2|F?_RR{)I;M?6;`5`DgYx0zF_v z%D_arn1nvC#PN0kb_COl0e$g10p(b zj|6NBY6L;_{u^B;mlig1jgKAXS?A(VVTM4=q0)r6Wg(1ZCr}!w5xxTWalh9?J@1L8 zM#yD_#X*gMg+@sBY_W!Og+?5pNvcgL%l9rDqr{SY#HRp0yIh+ynbN+U5(j7Rr3vDy zEDNF!s%aIRMV=aIr>CNtGDC}ky|Z>9iCIpZ>Atm^n-;$B#2#ri#vg5p#BEg@M=|MGGZ7N{Dif(i3A00ujDe`~cw+zwY=w&;6Vi`K zbZ4gp7bCt|%nHRi^P3e$5ufqx1WpXyDnS;!l0 zgn24v?cg^~{S)Rm)Bnb26*YR|64^C|kErKV9L)DX@-hG1Wt0xDxG@!OADT}mxM3?r zpSx#Am{Pp)BqK9qxVHm;U%^O-jp&Lf)g5JPc$^{B)Gwi5kj1W5l$;pg+H+^3+gYvf zsRKTMPBK`N?oQsF!SKaKv51*|a>?qx#?Wo+rxrE-;1iaq$^d;mVg5u$q?*YJ3 zydkWDy37RXz-><3lgEO%IBm9=k1jcN&}od7!?og@bO1(|%TSS8+D{o{F-oQ!zKQh{ z*+5ifvziOOsr8e#kZkoP8@aHK!fCXjgccbA$Uw_gUfeg(MAKVa zTT1c=6HIB-%poq+<=QG3>NC(*S)%9ngv4c@qIT`T-h!%4O;uL& zQuriyhG>9w;kPRgcm+O8b-frsvyMlDXf0GKH-yR05!OkTRr*?%$7f(bU*Idlnk&uH zfI;O%Bg* zl9>Kfqqc)@uI5cGYhpy;`%{i>@<5z!unjXVA5s9`oi&LMisRiD;S;q_RCB0AZjVXs z7}k;SWfcBJ)SOV{gIXCztQu>_(wC0;qZZNa=k!3y97ugnzLX07N|pGsRka5u`Npro zEmDNBbof<#W@*2eV6y=L3tt~{m0PLF84TuNn>E1kQd0Lr|C&eXKHsb$6$&;R(Ff^3 z^S7k~OC0B)tZ{~5Ba=7r^pC_eU;Bw&vB@>T$Vt(k%x@j5zxOGkYjnZUt`h(((@@ey zC>ao}rg>CMrP|Ys|Nf8oEOaU`090@Q0BYp_=R)%TO)dA|8jGy6lb!SbXfLXtj=x!2 zKQ%<2)b93u=vog!Ng$dn@?rv5bZoGbU4OranldM(B86hS65^86qvR>j}h0~9qv=k*w1Xo+Mnm#M_=$d zSYOFJ$k+;k+7Nr@G<&B#p@6}06BN4qeuYUF?8?2eeR|j$Fxvj|T^Jl;W+{s5QD!M( zx@gj+$wUh`iSi${L0KfT(M5~nq!3f2H}V|iK}SrVls@UZbnr7H6VvSw1fOI*VFs_a6QCLXmtjEQB9O|B)*0 zeHcz4L5$7mQkc=Pn8_r=6GcOjm!c+za(hD4O&JN9ajSXKlBS0e;rM&=*(uE6D0=Gc zrT&M_IzsKi)@>bsl3=eoJvAWkTF6a;5Y_rTgm7GXZ~vAlNW%p&`Q|u@Pt?hKL`GW- zpZR2<%@l-n!v)?}^Bt$;_^9*u z?(9OK`fisPIIUEDA;IBr`g2^C5hyM5>9r`sr9Gbe?k@DlQEAz|8;!u2CW`Yk)^Q`4 zST@i_Jem)g`O?vYqQ2|N$K-Oy_U*sa2+Y(oDytl{bw2f>i!)uRu}fns#t$1}g$ceN z6plRyHD%mVG^-fIU54oclJUpym5blKAlvOyx=B`pMd#%bTneR*&crnpo=W@+Si!U; zpm5b(O?(D=6qGo{$L4R~UrRc6ZNeUx2k| zZm6~M^&iNxdH*)lO9tGvN3qyy4yC(_58ROVkbkPQz9aiG zKv|wQUHCCmVq8%#Jl%Bl9v#T4_JMKmD6V4f=*LwiXMYOR$HY%gkhzKu6}?F28q9iG zF5W0Vn5NFyN`-hvF?JMHI_A&Z0Kh?RzINfA7v2(zr~*|v&}(5+?YdSD0ee)-hu5D@ zDqEa;CMs!xs3TR?HkT08R^w}IAO{Ueja0%fYmTZXG4I{q{}!E{YEQtn720r1lGQ2K zK{VSBLVMayLE%Pn$_RUy8D|FbHST{3K`Y?fJsM;2en*`~AR{kEGyqvLIun(IigKo? z>EtsC;z8dBU~$<#flgq$yRR`fB!KoDCsgw)Qj6tnrVx)+GgAMg=^0=~sPLtJK+7lx z)Wmh7A(_hdGj-MRgxjz&Wg(xPm+@*}z-Kw|FMh9q(BeH?8;o;8`3AqebO{f?L@|*R zhf05SFAd@G(QC!Z5E0A`w?cHVH>iX1#;k3#a%`?6`Q;sc=|gbzvSh9c(Sif?!M^Z zsR~>I`(J-!CtowI7J6AUafgnz^BMI&X`2;wPCSjsRT*ehrmakrpk8?&$Kh)DUFQfk zu@rbhbUsGZQN)NTV{0IY7GaNm`}aGs$MDpU9gP^RVjDE;Ux@%`E4{3QDxi8s8+!^o zuQfbw0$T@RAr3Hs!NDl;he3hJ8o)SULzUsvP5I)M5Tvn911!@XM~>6Z-N{yShnK~~v$G&f`N7Pg@XN<34E9Zq6f@cY_C)Ri8IM|yZS z+t4}$?g-)$okYh!0Mc^iXuJb<>`@FKcIZIUJ9OE3su{%#ytp^k2mYR0(gzMJ5{U@3 z*1Tl3(Dda;Gr|KiIix?7o%V>kq&*wijs97G=OT$>7SeX}3%__pVnUYK5$yh~J)znd zWZ0hMY$*(ktVm;{-&YfRWi}$Q%r4FK!)-ygx0he}>B)^w_H6^!mLY!^i8QJeW0afy zm|I`;ivgz5jmY62aayQ2QG znsL9X?Ei=N$N#m}Iyo5YTmLs#kpF&@t5)7t#8O82w~c@V0S)p&UtOKB(h?Hq0&TDi zRvCf>4I2bxyb%Mh!{V9{JbmNwT4SAc-yh4vK65vb*!A2z)5Orsc+u#abl8riM;EP9 zax=9P`{?cS%W(dCdrZ{@j51^n)|YxRfUqT{PZ#*JGoMN6;noU&yZ^0Ipe;S$l4q{y z@y5qkLwv7~g!jC6hyo@jWxx@B33JXTmdRNu0M>H4lS7P@X?T|Vvc+ewTJ1oJvTf!~ zA2O75y>Wp#=+bgEo-yuivmAWB$-HRu7IV(I#Z1`~r{(@oVur>c0V)BtQ@5SL{>Z16 ze_$Dtc1lwV`OXl0+`ZhiYG&7}L00=n+lE20N&lx&ctm*{4xME4`*VQ#+k>%A+?ck$Pphg4D4Yrvq6szx`X4M} z2*G>~^Wa;)zk!$rH4Hip9vX8sIej!r?$k6k@aTJ{MdKDEX08C!VHx21W zvTVhF39$Z+v@&}|Dcy1!1xBLHkFANlML6he{)u2Fat35Q9(NmCq+m9AMtW$D;#*8| zBVGQWN-$J^E%#DRsP$+17*x<>oWv3--th#?WBFBA0GsbI$hp(obxWNXC0NSM7$zp= zaE!Dlf})}Vrq*ms;30ADtV6;Fb+JP@YXeeI;8v$(UNB@FN6UjI;FZNaq6@Aa){8b; zgb*M)7_M-Wy<6P!geWX82NdKl(FwNIT~(dsBP!-Ba+a*=t<0YDo5%Df`>W-Gl(0_W zw*F2!R?A83Q#78rt$_8Ti_6<*d$%XqhwMkm638m3kz!`TD!FP?5iBm8N22M)uc)Gc?5gK3w+UFB-Bx8;_&4<=Kw$D^eqM6`^ zV=H7U3q?2z9*U!@Zk3Js5suT!8IbQJsA*9EkJ)mRN ze&MY_5FTQ4XjOS(#LFcD%q0-6t^QMlxTa4if}WClPp?3{v5Q!@uxoFrSwn=vAp0n) zsp}ft0(iqQ(KMSI!XEJ-8pHyUYUT#uC*7DC%P-phT|hC>zT>3=3;^)8P~Tz6RqBVz|Cb8B-aVeZytq4eF#VAt`Y0?r3d)vJ`~Wp5 zG8730CUqufhbcSEX=F-c(RquG`bh?L3RArWR)+OiiOPok1S8XQ1hL}l3ib>_E@ga) zzE;`*C7qNhq0w8XWO|U-ESI3cM1?}~ps@Z!dh;^q&D1$Gm^p)RWdYV>2VqfHp_JR? zfHC@xZr@2s%L$}l=d`){v59fzky5)NO69=eGHJ(s)?_p>5k$yM7y!n`g}l{zrx?^f zQ2ng2EXq@8_i;;ghHO|S9#pKO!MR895aJ5=eJJq;m~eF#*aILzOM7A6RbUYKt~*(Z zOMlkEVl#Efkpi_Q^q4DgHIy1sYMRrMFIDDAs??AlvG!N1pS7N&1)}Hb=W^HOi|lEr zR&M?DngH?;e>0R|P-11;5lKTtjaKY_@f3(WoU?$8#~v28yMN2-meb{TodPb3tGO8^ zx(*2=@>?>8bTM?;)$vUGY#%8`avQgFO-oX*&aUZfQin7*7IC}WxVfgujFq3vFM$qhD7(pMcN?;uS#)Ij;2&F>!TZos=5bv)*R1i zKaed1tL)x;4v|mNxe5P$xMW&=W!~-{*zES)Y~b$YDM}C2;Ao%h;2dP-#`|RC@G7Bg zx*;L(lGHHQ4~|fbDDs_|7p~=JC!A5+P14=zGyc#_}EOFl6yTF+ybaimy%#id&P zM5|vV8krX{Y_z>7StW)>8An`^dInf+!2MAbPyrhg zoeAXqKL!I{UsE+60E|G&T0+|kZT-~B&h7ypy^pH}nq(o)X;my1)fY2lZovPhC|{zyi|MOAfeeL|8=mmSWSxbgSw0u5E$S=kFESyX5s zq6RdTshA6u;@2)D0^;JsZ+O|+o2r8bfA7`xCVQL+LqhTs{F~`D=acjFYv%U#n|s#l zvbggGv`#@8bUtR@VJpHHZZDGGjoxC9#od=YjP_91=m`}V9A=AV*A@M(qU)dXnJXB5 zNFZPoxIL6eAX`wU09#Zia2sh@berm!Ylt1*&9BiH{>h$Y!^vgCNk0Pa0HV8>APH{D z^KQ~;E6|$@Lgc*|z%HZ#IWFRXE3B6+oYzucFXZAa)|>14OCUb(AbjxdA;HvWERfsZ zu(YvfA|EO&UxU3;updG{OQWBD03Wmg^)}5CADUR*vBOk;uHaJ8P)^vdQh(Ely{|{i zw`Bi+40o36&u#v@=x?_GK3IMK$fjoY!k|2KX#Sr3QoTLJTVjV{pXP-PN8|+JfSoog)3% z6GRH_GMuU6Pxh9w&r98qHBrQiz>v7B`q#~j^>C+!o?|m=?M^1rqj}An4sWg+f{MJt z(6Sz4`pfr@DiFihQUc{h#L&&N;5LUny*jmf(L6C{RfO@=_BF;(=geyY4TGt=tr+=D^#UAlo<}em z#m7V3f@vO!2SB-QI06I<$*iM?+*Uy|e?I9^8VkecP1ETdk=9ov4I zSi9=p=f#WdSw`ovCL+EIGO$N33Pb$C4L?EIQIffe0`(19gb!H?NR6-;lBD60x{Z5% zsp7rawR1F<@R9to2T=Uej^>^JA7k$nW?8gk;Z{}J+-cjkZQHhO+qSCGR;6v*wr%U? zIgQ(WZg+q8W$pF4_Cw4WBmNkJPU5J7sJOzYchsf%D0J1x5Du_BaBtpNtl&~Kt?7jA z8)eK~oYrTjj^K!G-6H^t_jD_B=FrBTSo=t^Ts`m<1L`Af-i$2S*PD;bB_5>p_pcBE zmq#{{3peK+`e7H#P}*2|0EMXqm=LWP`#c zPKMNVtAbhUDpC;7*}in1HtO6CTM5<;JWpw-Tu#5O%ztc8(Alz zg{NLKI0~GwK}{1$kjwJ9OCKm2LsG^p4M?e>L6dczlePtm)&`eCE88f<$CXMk6aF#D zut?z;P%18~DK*)vCgJgy&2WSJ^j`|xt1c|c2(;P!DUqazWIKdI!U3iQz|ZlZOAYi; zoKU<2D(h8(N*d~yw#el|A!q7%l!GmiKza+Bao%Uc>OHxl%YNll|bv~Gc+)@e`kXcxqnQ(c&zmK##!x|LJC zsm}0ahBU?6!;XvYpJy{PWi-BnEx&Rgz^!{)_U;RyWhAyO=wG{})Z+CG3bfV%8k|Tf4PadQ7ZVIMd^n9evDeG#o#)xg89paFI0~RqCkwQ>#79xbr z=6Z#(MUDlvpqkJcQ6}*|KhU=Q9mC&cA}$vUIR1=MAj~1q^emi9W9=wGv4e`?Rz&0U za>aj}=p2-9Zv~L(248`nQh0^d#3^@G%7WmVca2Om8f9c z?*cF=5*#P_r1(ICu=lMH=!1ysc`64X^WO77nw-gXx+5$j*E_FA#i|wEZ5!S#Af#cJ z!}yCb-T<=>pmN#kg__CKcqq!+GVCEt5nZ2Ee&>^*J1G7vPj+0VOvWOQdqfO0C1xSx zL%Hl1r!bL#dy?K65wzFRwC;wOUFKU`T75D?wPMTW{vJ@sY3SzZppwoXyZA#mFkwyL zaX(SvO>NQ~VewSu5Fg!b5TC)xd%06$fv#FdK?vEbcYMZ6pj2aX4-GuTwb>~0y>*0R zM@54ZX=LVdIHZc&=p)6ZNEVa1jL^ICxkzyF`6(O$AH(I0Q=IGpI2};F)rJSq?V6e^ zaVe{{DNmrk=mWv1E3tvTmaqjn1oEdoEGP(b`FB2|Kf|sr^%T3;%!H!CY=LVd`1(u8 zBGGKPh6Za4pefse*Y%?QXn~46@dTc7jcPZnx={kHVUBe~_lU{4wXdw9MFbIiB2dMO z2$U1_C~M7_iAY_-u&hV@X(u#!>V!|{>e&lw@0eLe-GB0asL~XI9z9&^bo4CnYO`%~ z#VYR?ha|5)+@Qus5M_5c4th+JMlfZFd5T#a)`Kw@lAF~#P<7>+c2S(cL>Bbd%W`&} zo1POqQXHptD07IP%U}cE&@q_V6v6y^5@iatGVxP8N{WKfj;NC?83VZTQ)WBi@RbJ@ ze3FZ>NiH@D(ob2K&YbZdXh1V^lJRnNmc*p&iP8m5;uCR=h{We3({sS5Hpl(4hR*iD zO4;k8h9tRJ8JAPaQA=M!(P!7@1N#Mtu$4l?hE0lQylxy^e6l#Ji3dbS4XXB>m#KB3 z#3+xXd@GTDTjmSzwAsiX2)gALj|`27kn8D$RQt|=Ptwd+*R9J{VeuOW)9&gP=12B; zHG=K26^jmCD89XW@o4R2l1;<(!?TI0tG)7>G(*{T>(5@9ptpq*pXm@d!qx5Sqnv?3 z9+*>m?^{$_B0gV{y2qS1WuN^ZJ%|%EoStch(=zdR)7BJGmoxkaICIrYyDG==*HFJ6KR93c=JKL zJxU9sd~O_RRjZ~6aqD~~a{(>FngViNoqn&yc6Biq5Jue30V&rgi@+_}WGpnXJNaZ^ z|HVGU`-rK3$omg903q1iArn^0q}S*db;LWqw&W6`_Cp|VH|~MM?-8Pc^q}C0 z({#?KoVqEVw;a!--@ae3nBD5_V&A0fR;>Iw>K$Jcxg#bz7C(I;Vto_~BblM3NJ?6$ z0Zw@BfKor3ffl_oee8gnfrWt={R2@B(G63$b#Nf_g?dp(RY%4FM-!AI`R&QWUAEG zc=bYgW9OfmtGolXPvc-W7YA*8`m+=?)*3T*j~1Pk=}iqFbF&%hgd}kbFE~=9&lF|F zwBvEP1)0uEO&1@ZwF){}#+93M%Vx3Hq^qjeoP#7lNL^wi+ytq~CQG(MC%t4?XK~H} zq!xvx?k}js4Hd;q zg&$3km^xy+=#DdA8PsOdfWQm2dW+qyE{kOBxoS@%ZUp;gG->;E{x^u00?MkUIt%VO z{;nFz#4PQ#QVH6)b!bM-hJ#k4l&3cT%=?03)`sHjJmAHGlz6M$Hb`+AR)kAXYIVy) zN;2rz%dAFfmu4DfPGbU0>!iyL#f0eq!tNSi_n508n9TEX$E<`f<0GaY$awZBJ! z7cf2|2GCJ^1^)C|bk|jB-%yahBAfL#^sV^2-nOWhu^t94>CJk#tQx~LI}KWM6eXw$ zz3DbbC!Jh33=nHASqRNo8beKhCTM5UwQ>b|bF?$imfvek*T~QPg<19F-uN8qu^3wx zFa1JiE94?h0+#Vz!bfKv+cN`gt$M}7i=O$`WfT@y9`sUr9~2NP(o5Ed_q#D z`#*X4?Exbg5bYCN@-bWSmheMtdB<3kGu+KsNc7#*^+_x< z0)I%fE8ut@OZdKPu%1zhbFAKxdp2i?6x*D4z%*CmMXr)Ze2skJfu;0SL*WKKq^A!5 zNNSjUgGQ1=t##x`0kc{vQZ>Xu8}jcRUnqf=IT%-Kq!e^^md(SDHVJYa0DGUW4g&tU zlbVr+o~wQd_^F0w;R(Dm&$aqC)>V4V?#Z=Mkk}i*=Y+3y4n%x}cp<+Zi2i;3r~Os5 zd$I!22NH~nVD9zZ4wDFdvhZ8neeeqfsoX(V@ zs(-mSq1m%j66q?Bdbe|4)1`1{^I&N7oPNpRZgKT}A%_lk?u`*zJ9LLzw(Xnqq`m&S z4Uz#ZB9K)to$2J1D;L0!tCjyVnr8HOLQYNo*G*#DSZP~U; z)KPJTrssJ_vtx~B=sULOB(^aJEYUTjMpW?;o7*m|G#+k2tY-f;QqAHjriLH- z9OTDBaIYYU_|+!|6%Q{t9JQFYg-+Ad>JE`f!4oh-H$AN;{e-erSfHgPE>qqd6c&^R z3Dtp}jR;L}=s_WlI-npkjO147ybrnD3 z9#J***5u(*1ID-&i%9}k4}u{vg+@{Evb2u_HD`? zGW>6o#i-lA`I-k{jfw?6XM3^)PPb&yhB30$wm0pZINF6mW&>SA9rMms8vgHVyeK(X0Cvu1BXd@MC*`L}W_&ewHx=wmn)V zDjvC*7lgIO4_JG@xWs%?53KGPR@_Tg*H;w7mmsjq{1MkEJ%W=e3JY^U8)b+)rO&$& zG}{{{MqfauLIjI&+Ki}REJ*etO4$xz3ywk+DX%5SiM1HbDh?T!%zKR@jvdq$eMSTV z!7hPSJ<|oB?iQo!2>Ef3!h2$>^^AaDOMhrhbu29e!E$N+2ujla#JQZ;Bv3bJG@N$+ z_AkKt?2r$7{`d588{x+f%KzgI`JWNw`iB2e{3=y7UH^d+ilmrOxmyMHkPzT&od-~l zG;2xp$QAbw_VE9OU?~ze&#Yw{HXI`~X8M)onGsiDJzLa*FdIbFPkte(+Nxq9P^W_G zeX4P>b>g^om4zN_Wknu$&#!rOUN_b0_2y9E`*~FD3seIJMSspG6NoMd-a^*6mxbrf zUj#CRXe82S1uc&ZoQ{OWDdL}pM+_qyW-D4VCT%Bg-s`Rfzn>REnmHX)%*2H~K*B{h zT!)bpsn3{$x(@=w8>$FeC1xw{EJ~-n@^p-#jjFR>`#`63sc7 z#AO|wL(g|o<9AvgT?^5)mjc;qAdW4Fl1~ZJB3E0PSGg_6t*mQEynl&cR2}f(KF4jGf>kGF1hd33bq%}d{I|+6!&3nT zo}NPOk&4juDT|?B039K-W0ORpJkZCd$?0jHy1}XAj~iA?G)mMrqu~E&JX%^K zDIsq?e5z1m-(tMv1l>etQB=ug+csBZY0Q`$U`bCP+7q&yb0LzJEZ*Q=GItD|{~pwP%#d*#p6 zR9Kcjo9twUJ8hxO;&y-f^v)i2UU)99M8r3$orK_cvzF_mBn`rJ+^RCaTGg>?SF`_n zSaUFOez51-G+jdIUl8CX{oBbYPjZNc-vihkr26z3#AhVDwhYec9kTkw4ZCac2D3eP z5B}8|tLLXi$w_Gp|2JDmy*YoK64Uf;apdZ(HR#Dvwe1blW$0NwlgGi+{e2KybTALm zU^psWtv$o5I(r3`R|KuS2@VZC3%PkmGZtHf=}ly`)GNCu5E18K1*kgbE@FQ0=rWzuzd8S1d5nVr%sg4o@7|I|M-Q8#O(mRGPBxaj~fNtl#lXNE4~Hf4wDLjy;s@cm~x1^`1i*!Oz$8& zrwj_(WxgpK9^z~8B+-dBf|TAXQj~2YGu+n4eXcK+*GCUJTeHTaAcKz8ozD{z#B1Xj^uK~InOC-L%V?}aBw8`0nR5Q@d@y+< zztKfpi7#<*-E~?TG{iEDgG|9a%kT)0f)rGS17EO^DiE+!+0Effw+gg5-~=N6?*6q?;PDN-+kUctAs?Zglw#GoDWGABsjO% zXHvy(o#ta-sL%-oZd5GV5@{C6QrgD1$G>|{x9R;82jwwbS`RKzh0yyhQjwE{0sKz( z1tLR{F^Pg-0-ZfHw{r28c7?uwSZkfGrl{@PcC@bmkT>%bU4rAbN@|IDD{;->I zSEM_(T#+xZ|K9a%k))$#f4dhDp#Atk^M9Q6(#B2>=7z#9`c}^WOf>s9&1gbZ(@JRx zpgnZgeAn!w_pS$N^XKOhT=!4JHRgVMIud)jX@lPKdOL2d!98f#@$qH7TR&h!JQyPH zYb%M_xJ4psF%1O0SrlRg`Es9nUU>rHGL2G&908SdL;W1@@qy|48Dz)TC-!5mRJ4|x1u^<{SV%sG4$3u6xWP|50U02tK(P^j92M$eLO%ka|M%P zxLLE|bQla;soA;)6?{}$DRU7*-8hjnrv=HXi0l0jhQWKI6=^F>l2sE0PFB zj@tTyDKtimZ@%wM(dvQFmZ1=E841UlNkLSIF;)r&!XQmp5%B2Xifa+p?_U8gZX6X# z24Z|y^;C>AIE)G6iFjjE1p+>YK#oYaMbQ@2&$AVNeTD;0<=)UJea0w#F3s5yO$Sc(b^clU%brb7&$?C zZA2lXn_1fY@kHp|!j4uGvB2xm!0Ixb*zgXL=ohw%eryKBkVt12P!m=>f)w`q|liSRQ?umJ)ZrHql zH1GUO^HA?61?AZrP&Xi&G10(#N0p&WM5~K0u(phb-3N)GscT1U?myebQe~Af@>%2U zayLYYRMC9LAxe<*>t%ROS$&!qn~OJ4?+UP2$852#q@K+?+}Wl}Xr~L@-W9=o`moWr zLP)XyNGn{>qKoqA*O>R=n8(w=({o_J`hctMYuYK`V&uIAHBNBhj+uxTBsFxg- ztYOlj-aU8W-kB@mMGv&u-Jy%MX=q1ZzA`}q&)PxMy9wkltp|WjhGn?}^8_^wVBQwx zj|DOS{qdPNw--v z!0Dz0u;u*qjAF>erwId;CjKUTJ+p*pS_w-2YGE)UX&*$7O3x!H@tZoLnu(iM9=YJ8 zEg2ljE*xR?cp(rfYEA?r<@Q6WY8hb_z2~tl%EuZP`@U8S;T=uf6Dyk~ykI<9ph_20 z)$VD1+)DzZ@KU;guMAQ4SlBuK5TVS90fFl}K#Y z2&UCz3&w}zkU0T7BwNr$3Zp1ekDOfsNytauraDewk}%#@Ql$OzApb1o!)$_~Ivoe9 zSF(p}o$dXrU19Y)uL1KLIKAYXtNh3fghe})vYPmQLUv~9%=2bvl_@V@f}bJ zJtP->M%Z0WxA+uMy|_m@@-2}xlB$GrS@<5^naD1}h^d>Tgl-&%=Fp zvmVuXoP1NHXkF3|vkg=?OMm-y3I7roNKUHA|Dt()$xMjNEcucew@wmZ&gT+1*D;AB z*##i;i=ep7Ge=pxT@1fo6zOWBSfYQ8L{Vu8-fQ>b@QzQCSZitq| zeP}-6Q_$owNUGp9`HpxE`=ZZj)W>Zdj#Ia>l|vW)ER14zI3yESIM{O;hD3V@UAC0w_U%i|7Gy+*9|5Va4ki**x!=a#LF zwa5Cc5b%44=u`a9*I~@pwz7NxV_pPnn?*Yf%I#|-tm%MDJGlWGKz1+74c0asVbio= znAen$dq2zZr$}a*P*olCAw@h`xD{=D1?j3W)`d4E`4LP~BMBt-nt(`pM07nhigk=N zf;_fjBfkOXIE&SexZD7W63Y02D}{!i80+%qa?)=y&8{T7qCz%DjR?l#LbkU?3vyJ) zm5ND)>g6evRO$oNHzh+4^egqjvHE!vWykbAltVMq#$-zu9?-kmy5#@^us0QhCi|+9 z-F+S~MNyQwXBX&U%<{X^*viHTB}ofbv83-BqTk85QM7D^;J!E{U5` zxGIk=B2$Y#?{fj_{m!uTVgzV&RQfZfqDkr}t-+=BRC`~HI&HP;eK+U(*OitlV`EiO zzV2#>yXL$*mi)?+S13Md?PTf-@TW503~ z>d<6e=-_mu^mp*9zkL7uMGx9L$tCGq-hTMD@%~?;75*u2%Q+bTlPCYi%bm-Nc|+>AR->fFM-ss^ zUY6F^9sou`L{(y>u5>LI93Vy7Db$D*TSJ7y4)P-76+A@2L=~w`xqBltK(UDIRjHTF zpTlAi!q^uVW_qSVUYO~TJFZwHFM+OHm%u>jL7mLlQpo8{DNB1YqYO%}Nb3-CSs}kC z*|tD!Q+ccj_2hiZ%PTU4LaV zLuX}2CF2qD7c}dy1Vc0{VU*OBZr1% zQqzzb#(SiB6__==luH!jDNlVgh{o_<<>t;I@qwK4#?E5T{<@G0@Mx<^O9IRx$}ugf zC@=wAj>B#UyIP|2kIHFL6X#2JeNDIOAN;O3inE;YLHqH zrlEI{X1x>2@JA%n^Le_k&{MzUBv`|fN>wS^@sh-U4@+EAoK)R-Q%n%h+O1V-HkMp8 zBJ@+^^M>sFAVL=yY*>t<;rPr@a-?lEY3%QqXStLI1eqI=^_#QxRcnKM_xL^;li969 z*0AYt+e89xOeT|>=wlDs<=saxGb&Z>rJhldK$gctej(CYD%a5=wl~;-3tmAN-P=)!#uXeWs;jWDJE((-t;N{8&_1-z~$lL>~- zGy^-0X|S+r$8^rAoE{i&=HL?>p8$rNIigv6@r-|OjTnsdK)W!dsq&GsJhli_p z?1+8!2gr3MP&Rz9K$3Xp=d+SQbI2KT0Z<284-Di=Ala@E&-pCpBkt`7Qs z(@epU>%9y)goetX0iyWFnJ?IoMJzmEnGE~wzGmEhCT@MdL$Aun=RI@+??(D8! zni+%6a$#P~7(VA6FCWqF%tN1XZK(sIpoX;iCp*$8WuSe zR-MEpqf;l7G!PBsC+{u5L;05c6okPPLJaOlKG!B|Vv#u}>_=SF`rGE!=jE-<;nvsZ zHwA(Jo`)Aq53-jQcas%hSCbx<2S1<=!f7W1UYy59dk6(dvzribbR&RF3;7^vaQ9bD z8wDtl9$Q#mFDY*uvXyQwM4PNB|Ccm$9fJ;ng*LP!12lROG-U*({BmzdK3)tudR2f< zl<EZZIrrn%T*#RPnF&f{5d-LspQ+E5(kj>*)&^hek*;g^LSI>duzJk23ZU^H^M^rf3qju>DuJ zZ4TFsa-6s+^Xv4eh0X3P{`4dbs~W)I;@e7;X1@==kRzqkNe3irxjqvM9NYXT8InX6 z`A8{IqVRv^X_onkMs?a?AP?z76vDi-S0qAd?q9V_hm&0UXO}}KxyG3JU@xaAwYhSa zc^yy1{MA^gJ%&}OJ-n%eY)%*f31GVy66LDY9{FU{UcC!s&%-r#d>Z;JHZatjyNl#i zw#(>6qhD+*#>g13oOS}6nIm7uDUKt@-)}0#S+xu0KM#1i=w~LiNux18Zr=VZjPSvV zyGf3y4ofdQt9-BYJ+0FfGwc{kd_bCI=F+ zufc$3DP3E4wOf{s;G@rNkzeKb-)8v)yP6ZER@v|f*Q*TwuuU1ot^igJ^($D%-xgc^U~!F!gR zXx!LdN@?31#VGB=@$&mvNQc(MspVPSUu=C3JI}oeMD~sjs?oj&g9q|W9b_@O2ETxN zc~hyM&d|V%j*TZ0tP^sSJY^}^dM6Y$vw8@AyJm~Fu=^pqO$Hx-O#~{IK&&2g$aVsD zgY4}H-t2p<6=p)IENq6b6WN52Yb;s8OrnZVnMYkTEhp>=u;}cHfT@k4VA<@q(Qu8> zJ>mmo5~O|{;Ap5Iv^9iPMKaeco3zS=ZTu=D^=pl$b{#QUHpL-S>9&78M<+;?)kE&< zMCJZTi`6&B^ke`$wXMGE+kTll4KvfeudYfRL^>Uk7=(Ds{hsWkpCPL7vQ^9M47MiF z96qVn!5!qe9DntN1gBhkhvTw=p)J%tnf-2lhT8u$YxgH2)cf}BGNBe;el>QZ;o7zs z_(x(rXC!P^zHRSaL5CmDs{`_6f4Bf{;SsXTn@u#HC_9RgBio*-Cp)+NERFG7i7fVV zYnYBf{TG}S@osIW+AeH@eikYS-U~Fxdm;pxlC)+?1kt|t`^?wB>`isQwIbhs&lpt@ z|3{nX|F=wAxI3BI+R#e;^XEU)YQT}3a{347a!g4!;Oqw zaeu6@Xrt+=7anzcU9DerYz-aVeQa{RJ^F6BA$0>irv~(;Dd_37B4;{3 z&lv&Ldzb=8sJNU&y7)6WNcwqlx9P4M(=Wiu|z>6b1LJ^R+7CrKi9;C z66!9Ktc494^F>2U-~O(K5+@ZHD85FK>cs8k@q?;)?)}wDMWG4Za{!T1SGbGyJ>6;H_c9jq~=&pB>k4(!*4vT7z(I;+h z=#^<3sn4V*?$sv}g&DJJBVONXwjUZyFQSL=M{?0ht?}-zxzD{H54w!9Kq{i~T8;9M za)iFg$zH#Fa##4BWG9PbH7O%?MxTnmSOF)GcWAv9zsjbXR1g|^8HK+iArB2T<4Imb zTE``}u9H17TSNF^`q65k`}q0d$7x(}OP8wehbLUbSektk6#FnubW%H<3CyfYA-GxI zuN9E!vKry|UI1)7fX@wbFUgS}!Bq8xwxmVPkRg-t+9%zqr&>crl47)64p?3B?u5U7 za8LTI53I5iq6Ep))A4yFlClbWk$Kpte+VICvC>Wo=$@`gAUK*uAk;UvS#ttN3qw?X!PcR$UbICN?6e)cOg> zpGpHZ*ot=9V%(3$SN%rkQu0ZokYfqBaZ;vVz8>;|QjlC$qe7|IU_2>%(Pee0(ib6* zJO(2#T!~0!_aw~JCK|Sw9)e6_nlsK^8!Q)++1{Zim}t(lcd~jKzLiOs^@JNwfNysu zMT(u~Ag=N|gfB`)R}DVoJq%Rdl6@L~3NB@BLeO>b{fo|NugGYmxTZLFdGd`aV9ZUD z?HyD0=E1mp_sUJP*T2#kmizm_HCMY~&IK^Dh##+J47|NUZ$uqcq*l5szEMlM%A{#f z6c!7qe+Y7Yw$Pt)V&(6K3~x+|It3Ag|IU-kh zkO9<04L5}~GKXT==*;q|OhlPU5%wmwtMGY{e!;o+863@QPYxD&v36oaLs~E|Ip@g- zWRuoeYd=zr7D1VS7M&xpkh&{vd`g;zTx_JS9PIM zxg+9Eo4?D4)tTUsdj&+G%~g1f&sx&P)y@23k$LpAaOpe8-gl3;c_{Jf7L~BDKH3U- zhlsv&{CCDSZQl=$5_k9mqtXPf;1PK!bqi8a(UDnt2B(nstOK*8_vQWm!eGkd0mtLM zJGSW5>8anjrgku~`Z#^}tTI!CZlRo;1MKag^d3H1W!*>$P&xZDc;O7M?!hQBwG5Nn zAgk=bh^rhjmc?juoXdSLLnYpDZS;Qpz~=l>A#6h)dQyG$L?XC)P6@29c%s5&sTjv! zVSN0nLu8ZmQ}i@2cFNpexI;+krwRR9^vpLY#$*$$c)Efh!Z*J*d56D0wd#Q6QnS00 zpI-d-F|vi9_dXh3#j!wM^|$U)y^t;+jD|h0MS4fv?gx`P!d0>A@=B}%Pkz6cAb_Mq zIxe03GED@N#;#C2$%KojEnEYDfKF;5tW}7(bBER7uI3$<|4C&bOfZFIf#r>ci~qw% zEAN@?_pekB4Zw_!>E^*0I0XHW=mdiaxl1_yn_Ug_5}cs5t$tURPG)+Wny~yy`I^cv z39;2OJH4OrYi37V>v6t&K%LRp<(Rg)*^;m^k^t;Tw)!xzBQy)Z+%&oeXd@6Xv{Op$j>^P9F-ebtw*fT>NCe(&{ZDQC2|F`LZ4Rmx~*!EmATnqcnqOMj%=J*Z%zp30F1 zj!>8G9=kY^h93L!u;O4k)yBpu>l|ZlHhG6l(7Vhw7@z*actlWAq}70Dbp`oTyYmFT zn0jwzU45rh=?KV#V0_FEep1=gc)X?Rx)>V=I3NhgKB2w)qP+%^!4#yC77Bl$sZlb0 zD`Lc9JjdP!|L&z9bp9f)``rnA3h!1Cx;hVoc*YrL&p0IkJW?LSpxvyC;$_#Ow) zEJ6*2!~htC76+tR!FSmzv7X3dsOx+3z;W|709qdX`M6Knw5)A)7%6c^EtG19T@9@} zPPJUh&p>&G!29yM_R@)wl5fs-IqvUxZ|L7kc0sDU@2CDqlmz(t7lTgI1M)GoylFB{_kh?ACQ#HfQd#xcf)GSx&^ zL$j$Z@yTfwpskK=)XH_PL!%aV43A_~p=O*$NL~)x2B|aWkApFcXF?z8u$RCJVqtSY zlTyZ^vNQ!tR}@^d;HQ?g#MC>9^DK%wGT^%eOZSPNCnVzG$n;xFV_z67ythJ=umVkm z3zSV2e(9KHI0Jxhrt_p62(%FS*UuCAPZDR9xNY9sjZT?v1^AnpszP;al-|%s=+Hfg zQ3OTHROHP|W|KlRR8N<$%cvf>GAln%0rY}_7jFK4cnzRB_^?~&qw+SSnH##VWyU+g zJT%PH5Q<~jP~=7GbA92?Q`HK8FAk=;qXX4r5Q9J`FbrRFzdkLJ8=4UCU;6$2om%@X&59!NxIB8a%jkvxcJKeFH01kI*+L=AAc>pkDt zPmz^S1Q!e&D21wLZJ1>Aq5EosY}jn%#XkPU+$A`qhbr~k3;6wihRpwel&t?~ef~|V z>R_56nnS1w7EoidDug2b6@dYt{8!B?R|D9-e9kFDM%*N11h^!kdHdI8+kp3gwmrmr zPX0rr2iFj-&o$N;`oj|mM7?={q{C$9+hyBTr>C#0J7d@T%Z2L?6+^89yUYmYX6^va z>Edk}Kq_CgAa4S~Uoa(LeKIs91b&Jre2^kY>B|{9tU;a^DajitY^r4kJLGd!dd#W- zE3}*lk^Npayx(AbY{-LQ0d4!|Hh@UG*oZ!{@@7;~ijT{?An?5TQ%n;it(;l|yy57FwWO@qP`ua55Es(6MvCJW0+36COc`Gv;A zD*X>X9NDw=^|O}w+Rk6a=Uj|?1$*2kWqn1(q3sW|i5P9ZYI>ctr^ppzt5pbB=qoF{ zRz%zwa&=Q^@j23Sv~r$&vPV&+k)}_bCZTR zTUm+VYwx4t3x8QCX4j!X^nXGKn+K5BIQ{+TjOXUzCNC$$kQjT{12&buEsESZ(Ab+_ zN4J-FnNuNY%cW4lEEVBC!;USLMAsp=1!nmuuQxCK(^(Ms+R#JL5g2yUtiyRk5?xtT z?{s+&twa1^*$jQg#1&`?;40p6zlWsy6m4f}_d^AIL$k(F7Hs~QOzi=xAShC-HF};+ zmu7A-)lhzeNuG`5rbS={tE|^Werkm~)a{i!*zM)p+%Q*r1tDSCYR#c)#Xg$a3<2R&MF=ArlG@W(gw&rNu<`o6 zY)(>pRx5k$wdfIUCQ6lVY8Ek-o<%kC8H2#8>~MEM5vy)Is=wEU3}IWNZdr(GcUx~| z-M3|#MWfd5SEH%YE#d*Uhwy2NxsOIboyYGN!5}JZ_MG~=K@U=8Ls#l!q~-V36pJ>& zG~9+3Z+cD-{{9S+5!!7BD38#Ne_s%CmX)nEpfV{RLh4|;R9qY)CaTF)&K>YK?4cuv znpA%oVU(-Y-E!QVD>O{lR2`cxQ|UmR7ZQixp5^&)sPduD)^U2({6CbvQ+TFf)+L-u z#mO6+729^j)*IWlZQHhO+qSKWQE|nU{OReQ|C;XUgKtir)8}Me``&x6wbyd*M)A)( zk}JV?yy7DvLhhc4sI*vF)TZwoO+u=`bizkudOYzG7UEl>fvB#vooj+{nst18RqICU z=R|nCuc%fYXCUy;+d-G7IB*jn0A#AC6jGRVR73bb)+#Le?O`p&5Tf?w(C3o`%|V^mEUivE>`cb__sao7z84VPzO3Gwh(cqc1|@Q0p~ zrBEm)i{(TylMX6`X`Yb~78Z;Sxi~pyyawALM4LAJfR0Mg*i)2?*Em%~5FzH*o2P&m zBSujp0jHlh_>~*?m0yc`9NuBJ9XnUx>^?N3cbj8Rv##0H^iAqsjci(bCSL|w2Dd<8 zqME|Z4tpB9**n4Vg9@X-0-u)r(}FjAIDC#{LsIg>J6|6E_s5m;ZZ1(rrbwch0JRXQ zp*^hb=eI--gnKD{f`YNU@Ll-oIxHoO%y=PW$T7~bTSU=kf&=z}S&jf*djg-)?}Eb$ z^{5DuWVnULV!}B1Xq$FhvA-3xey(fQBdH4cBzgr}+R;9v1<2JysAM$qG zQ}py6oGC{@G-!WQ(K^3AL6g#375w{u+@k)&G%;5JYVP@~H_t}}0b%?f_vgxA=4Su$ zM@m+M_S#g&`|QE}TRQG*c#$G-y2f-$BC`oTa+-(7-4U2vY&^&4^5cH@p_h&L?NjJ( zR;?7&g7Z)wi5wjbJeaT`#$R0uoXhsM59byHt?T;xjgi-`lt+c->av| z9IvDK(QjFlEc`sDi+0%rbXv9h5be9CV$?f2<40utHY2n?TM@2iZlyt8ld+pOHvI2x zynau6dLY-scnpRBMRkbpLBKTkXPL9s8TqS%o` zE!b}9eq6}g-?ZphbGM&^qpUn-`zY9a<@<`*@Rn}kgGl@ZZfb+Xd|n%|9*5MFA7x6u zeW3z?SPRU%1Dc1I8?1JIveeO|yC4=*wL!*m=UkBw(s*C}zFaD3(zS(nic zxY00cEU`tJy=#^ZG0p*FP7P5?aou_%bJFZ&&@1uZD&J-`Iww&%2y>dI9=CR6nHz^s zrICPWttg@vmGKf2*FHsGN$pS#HJ2IPmgT;%?Rzh7O5;Fs!8te=6~9U zcd{}V`Z>Dd0~ z*^9`<(viXi%6-X)0#Z*@r7%Ggb(E;>U9$crGcyic5k-En`4li2nt9{%-|AIHr!XIS zIylpdp-Ynhi(GVY$z2moxLC5+uLCu$8Y$`OnM@7piB||QC_5%amZ)g!Tb|KPaK_Lo zAzMtf7Ks!x24xLVHk>$93hOZSKE4A+O4gAax*e`0Aor5Yh3vcucv6#R%|u+z6e>VF zZ^}i>_*`CpuLitC!h~6V3u%>O!-mZ9vMc!)j__iaj`#AM4!UsOayZK@eR2x{tfdYk zK?`Hlh()?r4yB!B1@j53^6oC_4Ed?nkuesoWM&g#&D{v$iZ#S{JQun|26mF-KJ^1C zx9}wqF>X3k4tp2{uAUg%IO_s(XP#i&^ENO|XDvS&?4II-&F7eq>+@&)o>?oZYg#<} zCUZB9QG}bQ7=mA7*gjPU9+$!Me;KhBwioEukEEz5TYV?=jo?(pygN__G(!fm$D7{K>8BEo<1OG3|1oPVZR0 zlp6|?I~JTKt?5|vU%{yZ6GLV~6Kq)tH&u;}6=svqwS(g%#cT&YShvpFDbyaJjmIQ&9kBFj`m;5Jj%)ts+8EWtGmf`*L>6G1(>^-BjN6fpfb~c0YPv0u&Am_Q z0AyE4xAm;-=WTq%lI&Z=4EChcuz&)ZC|eDF+i*NntkLhb92Kju{Z(wcG9-dpgG9?P z=_Gk>YjPcizYIfP-K6ue4RNXi6uKHFCB5CH@gt7gGB|6QdOj7?M2O088FA&WR0zHewxurvL;4cVpE*PIwzEll}eDG zHQL~TF255fUwSF8qA#uD;bCG*>61DgvUY50m6dapdGiOlm`ul*U{W%PT*flQOV>9> z8d`<-NaX8_S>*EQ4UdIAat&jh&K8y^c0@v5q*OiCR%wUT?aVt`_ug2`Q5}hZCBmjD zf<_iHi|9q6ph>G1#(0C03=yn~OG`nyjFFQo+h*1=f5|cgx7v0&)P{6*@jnoqZ=Wzt z;d9;{GWM>5;|$?er{5#&F_5u&VQ%9L+QKY@L01k8YCQZw@YV*pfisLnM2()TwPj#K ze~IvUAyoP4qTmOYRdPe$7~M9uqw(>(Wp?78Zbx8T3(=PX(4O>s#&DKW*j}&$a`=S#zdMsxW=tMx&8c&H9Sg#jY3Z}rDepzf#ENtd&RBR@=le?tVae$2%N^GqRQ zj$D$-d5;N643=RJ;bOq!`L^)+K+P(QZTV=SX9w;D#c@0Jw4`urw9AC6d&pi9!j zCD6krP$P0eR`9W3phd}Xt7tz;F@8)!hd&Hb!=tEB>pPFiZ^rH+7HnRGC`|T>DTqyA z36<^YEZd<<<&<#;sNtJFVCb7tt?jqHV-BdZIH)hLyR)nu)$_zUlBC!Nw*A}+C^krJ zQ@^sSblVMHY^Njk;UxBeH~FPCb#VQHm|@SRq6c`ouTOZXgGtqAMGkt<=Y>c1fMok+ zQ?eN--6WP~A;&zgV-(RnrJgP^a`HmN2#FC1eu@}bubkyCHSA!8Lc2z%Q7>dQU|jG$ zH4<-)6nKjIrWavZuelY3Z;^)DY7jdy2+5e)aoUEuxi=^%T!+V(p&&Q08mMJCaF&T%g2ZUYwS)pO$po_z8KBuP}KO6_>l_e zKBg*H>)1kxO3J&hGW-V8SWrRm?bv4-WQKQYw#0`8zJGR)Z4}xDdkDtA<<;OZ4)kwqZ4@PBl^6*e6i({ae+9AuB0wA&H3b zIXrR3XvViD+?a{r4Rj`RQrSHVW(f0~nxao;wV$>ui;yRa$0{3L=`A*e&5+M0Y+cZ; z2h(jRss}n+l|E4O-((>5Ip|F8WC~ zhAB=9A5;^2fHN7x$F2yy-CL?+ko78ij8ie4vj^_kPE)V;-egLr%rZG0FWz*7IMWsI zug*|4zn#VE_ixO0vq}%OM@>9=ue-lBx5cA5CpK7M**8N`O!IJ*n(_nl;uaw|rxUu=^OCgQ&aJ2QykkKX-W9HSa@47mO-xr7GSDDH^cr z5JlugORfd|cjY2ahyWH6C&x$3;Y_=*&*>^|If0?fQDXK-w*mmI_2v4on^YN8 zSBOR7Vrx!1ZM9m|a72?h9r~xSWGd`;IJA}13SN`5VH<=e2zXDHAf6aL34$!@L6iH{uW=@@8?g1{X$6GInQ$Y zF8kv_wPTyS5$n>P=e$h^o3zx^V1|k1GO6AVh$S_rjA@)^R=$Sz!aFeI+ag_N+EZ`+ zS;m+7loLHnhTi5m@9BNz_QvvObFsfqZq0u+90x_kl$Z-cZ>=O+$_HIvz4u0O@OTn< z`1T!<-wNcKX%xHWM(4QddpJg3dhSn?3*w@uED5#4?g6zqbkq&e@QkeQ>?USP_Gn{7 z{S*Ab*;TjcdtNz?Y)Q4SG3#8RRb4^?T$bZcsWg2A288S2EzDw^Ix#fdQ# z>xxXc6sS8i4jXP1d zjmN*IZdi=8Ji`7vwpIfKX|cbsRr-ak|8oBHZ`k^0WXivTOTl(Q9`(b9@}ew?ieYQy zI8BxoeD%5N)?dT`#R-9iIiysoJBq4F#ytBC>jM=fdj!+hkAK3=CX$9k$|S?nW9E!= z_bJ=)?QJ$z7X*KBC4XP`8}-`qIE&Neeoc($+Nz!I;CBe)W2VbLJGTg1kKKJPqgzkI zMkY_+pSr^k8gHNnClqI&ewo6(9XBseqd@#&aHGbI$~lQt9e1KAFyzaUFWG|C;^W+p zCqHlpMvv}9^`C>!mrn~_#RTJdls;NX_dVi1QG^eYT#d%>DFT!DBfHcdGh#@Lgt*{k zSV(>t&*{(Q`Z|eVdiwJ(-oyzoL*kKFPlFdCt%u)~R}hfIQ+MPn)vsUoj+xk(>=#XL zbA-(x9CnGEGDND0I?41p+_tqkSkvLsfpViS@+b z7)7Ew=g)mVU14Xa{DU@O8i86BPpd$?hOML`=#Pc^y}1e-!UnXQT;UAVJk|J>jB!_< zzT=Op9=j>RWB$kOie1v8ARTPFbGh9M!nCT)C~HO7WDa zWE&?ruDopWpA`&>>fhPd^7L`O-V$X}%T|$L=aliu$@ECe3wT&-M3SU`&JT}LC|VVV zvU~p>We5wogBD*zmHXE){9g|0|A{h1MRCQiN@naNs&D9I>)`&c`t>Ea#!<%fVY6y5 zT2BE5$H~hT>`YpyE@%NIf)X(xZVDM?bm9RBs1x$im2^8UCBCHU4A%!*ceBS0*w>=pK9CBLO&Cl=*Mr4gsXQ2VP0^SVN<&lHAFH7T_3tWzAF(y^eq30Ry>Y;i&Rt4k?MMl zxpG)!Gm{+yHmRvZYm$a0O-JXw7V79n>JzC}g^c2aDS-b(gIbh}H~ZKOQn3WBZHKGT zJjFEAUmIPjt*kUPil6n%Cn_)mnQX9BHr(mozB$*j+l#dxCQnVF>g|Aiw6&V@W~++CJCk!^w;~Q%Wi)=qkHO zqG!T?w7kJPcjkPWb&?L;J)veVCxD&=cXmTyfwCH6N-t9cx}yid!}xEImLVn8k*b=d zN!fP#e#R*C%(m6Bvl$+q%rL+M>XO1MRP!Z0u&*icCn7Gje{_88|E=h7b# z@P^$pq>9X?I~!o<{d09&S@EVB0#-_pI>Q>lh0v#rFfuDd_RC}2Qf(w6ARvSY7qGf- zL9vhHqO(0xxzrNE2f!C(D3aqJGi9&s7CW~Duf)dRXWVow4K~3@wT@hub=<_7Oflty zi!$Dw!0OZ@ZBmZp)wIU&P^N6ZsN%RX6l2gStd&U8dHSBImbr#z)N16}@9{T#S+#bM z``o5KShcT$^&(xDJ{(UMzlmi0jtr*cdUMNn{^4?{x+&;U36Gqty<}DS(t1NxlU%xY zl{ASeWpXHBwZ@xP_rm*l&oMU4de*T$c+)afw?-Xv?~?EYBK^m5)zHm_J?{Rc`Qk;k z%cvK3TB{`URgpG>y0m&PIYaq+#)Iv-Som%sfdvWC6TY-+v$~-$;sU#xl;#?Q))k>E zVCGho8$5@_i)(o1xEu3ma4|T6ISUJyl(4%C`iznsDeH(E9Y&V{Brh3YhmXOL0QoQ% z@s&@D`r$XcGKP+A(n4tZ+R2WD7XkkcW{6Wi#Y`c{HAl3Y=YZ_0Idk^sm>G+I+^&Sf8hE?;6kN$?*OoX0p;fYI0D`4i@pR*wU z(MLmS-wH%EoqGfd!Wa;RUHja!2VZk5L@cQk&}edrAo>=zXNN-jqT*4ip;tw;F1%8k;qREW z-Nz8}D864wcz<1^Y$u7JU^j|RvU;xXduHF9UvF1C5kQXXiwX!(sOSuV{b*>=(nShk zkCpRGXtb*y@V?8#!h9;ig!mI5 zIB#Nf%`m*!N}US?at|?SdD&!1*C0D99hFc9ejvLy)x|vc$hdh98C#Q(GZO5T2 z^9*Oo>?3_@)dx;07PvrDli$d(w+3P5gv*8A$qz9o@c>AMp4a=CoM3lTmwL3w&kfgS z#nw-_jvD=$UQgjg|xz3sv?Fv0j2&~vu5Ee2bf-X@)POI=h;|AS@SM?rZ zZ!J4q<2%p4b?gw|S`my}J>dhHJ>?(cd?KYW8qU9Y#GEywyt1FNiu2<5UXCcC(G;}e zjd7s}pgU$5 zBukb5`b(q8_xrztmv))Q#_bEdM*nm0GW~DxYAP=NMEyufDFN&oM@nD<&kcE$}RaM{oEn?wV)&Av<9*1R3Y6YAEg`~Z>A?#ztre!F4jrcQqn z0D$n*St_~Q&qsTy0WJ^^s5kakjusOlCTjH}XOA^x+*cEbg1tqSvj5I}VB}I(=fI2UhO5xRUT5O*0FS{sFnq7jk z&HSd<8SfFdB1~bF%OGs}8_^GLhPVr|&^JnVaTs=68Gh~$etqd%SZU`|T-thIL+GJq zQAeB#!knXYZT4KRK&`;}`l%s%0hK;<_wS!0`G_Z`QT6m=4Yd^Y1T68pbyGct(SM^r zamg;3fNg&DV?7J-tRayKf~jM$h@}KV=)_Jy=w*y$qL=?5P5`#RyuEK8CnTW=2nb+Dq*taU!qUNQp4BuBZ{^ULbfJ32(s|Pg zj~x0=&^LMmsb$iN1VxG5e7L^8&dh9ja@m@z+XLzrl#N}{b~eyYi_vN?me~n^#kuBG zH!RP4B%z85H@z2>>!3^Jn`Y9;MB~_BDhGWdUDDt+Gj~NR{D6RU$;*lkSLrW3T)F$m^(<0V!0v-$KceegE4UKPFiY9B9waIEv?p>##n_<$~m)QvR?} zB}cyh%Oa^nxHvAQFk*3;cd^gc!b?+l+&|nA2#N1;>WP^*JEt}mm+y)70r#hOyf7_< z!({E-s&bUCrYR`bot`-<52yf zv#o%X_!|1Rj8WY^lGP2?N_)Q+sb;XsBc#gMQE5b+Y8{^ZLYUZ#XR4l(gNx@7j9sP7UEp-<@`Ni3 zPx*c)m&t*`ve#^6sgkc~_;XJP!D-nr!lb;oN3z8*Q&!c9f2JE0D_SDCw4adEeDo@njtuEN zZv*jFoZd}V{=E5>tMfh?>3i{v@KL$XS?gC3ak>^Z+WqTRGM(-Sm@idq#^n;Onn;CL zEooDUi6-yMI-Pm}q-E{~9-tA3Owh=@b8(?6hhk<-tr*2{@v5@kcQNUvf&(ij1v%xa zm*z7Y$|3^LFB*hFEv(B`Qz*-47meGi<_^8^AdQMG;F_7_@N_Cfoio6k%Tpm_u)&QP zJ9QDwT2cNII*!31sO8u3a+r-K5?(!s9sJFA>GTc!DWx%Pv=jqtu_=MXg(;inT|U=u z{p4@Cy%xP^cPAv3=W&ms~5P}%{6 zp2StkWMe-uW_N`zy>qQ3R>|eWm9lPzoTm75*BQi^$dwpYL0y2Jk-&6N&TC?|hOA^0 zHBwkU9ni6tWU_RPl+?P0l(5<)cAzqPqgR`J%I0a9l3)-zy@t<>qfIBQ-Xmq>Rnp31SO8h5tcPZleikZ zyn#7)J3(T;tALs(POe^$>QRXZ4X^1^A6psA#GzP8IPfiLX`(36n#Cel&xBtnPD;N2 zUGFbyx~9;=yOVN#O)gbDwc)(cL#y=^lL-=$p|c89 z&rt@CpwK&_`o^apgF-cEkMb4P)aHqt8}9ur<3*B2!}?Di&r&sedY+RYzy{0o*1B|0yo94Le4XU~J z+x??m4LMT!DZ3}}WrM1%6`cDmr6`w+lo)S*$YO{6%JUy=Tka!73^Gk^qX|d`*8)Y_ zl!W-@LWOuVg>6&=uH^^J8e4<_)J(rN4*&7R?>&ddp9va{Fl(6TtF$X5s|1)Q#OVz; z-Ju#_VG3{c{Sn7(5M6;m&qfn;dy636<-v^>p~2TBzet~unSaht;>DEFFd^cZ6W+yf z@aNg>yRc{8uLpU7K@?c`ZY1(-B*VaVh!0viRPjf$->F$Hzmazui;Fx$7Ik|yor#m8 zRAHM-Z$M(lZh(HmZoqm%Mk51V!oq7H0^b8_)+HlgEy>X_A3CfTh)#zzjj70J0M^Mi z0qgl%Ix8KHIJ|Bu(kx+DN*6Hg3vn*(5y+ZloCkv6i5RKtJS|Q8f_o3!52wJwf=h~N zz*uFt;MOU#QD(-r*Spl@W?3S`{Q0<%qF`-#$V&Cbd5V$;OjjzR~KE^fGFT(A~M zw=*u;jAFG6$8^s24|sWDXiQZC%;!cx$TbX15?JHEWZ0EtZ-90coxbk)_;@ltWr<{4 zCv%}=`nhB#6iPDlJjcnJtDl;gg!DrD%QWj$TRLPDLI&Yqvy#ws*+|CTkI(>Gm1u7~mo&=(?*N&H|#re&f&BLgxRqWU|hxLi4uO z+VCPC$|?SCMe#>VTKT(jhLBK7MUje}w*}MdO&+^(r_>S~M|vFUeJKTRdi}i7Bv^uj ziDF<%ctS@LzT#f22bFpIfb^;h`prxfag8h2pKJ|Bv5pF4uPbQ5Xlk}U#@sYyk;pD$}Esj;Zz>rh0J1$>H8WVqJK7NJ$=FHHEo z6{n%;&7S352jCp>p;j=^2~aev={Cv`B8~Rri~AdM?J5#*1vGh#|7cc#xKPz*d&}CV z4UH@{Xiq?oVCN2jD&o(vh zWgQL?YxOxwjMEprTT~I1`!)B$Zhx0f4cmxNUx}ltthMX;QE#~rdVeWG&gH2MB%8r3;Hp~>AjmiUhzgA^0k*m4E$K|-!U zzonCIk!(cTt%Q=TMkCwUW{XE6Z;{19uEb5fZh&1;o)IS|IgVu`3iAiGxWSL|DW;@L zgFZ*7T{#tteYleJ!FG>CV(IUx(J<3pdz=5u-!56d6TbL%qeDZoyFJb7@*|=HPK^F~9 zJ~sS4THw}{MGb