From c304a92a1ce39cc7b7e329fde0b97f93cb8caee3 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Tue, 8 Oct 2013 23:42:34 -0700 Subject: [PATCH 001/567] Move AddPartitions into TopicCommand --- .../kafka/admin/AddPartitionsCommand.scala | 127 ------------------ .../main/scala/kafka/admin/AdminUtils.scala | 57 ++++++++ .../main/scala/kafka/admin/TopicCommand.scala | 22 +-- .../unit/kafka/admin/AddPartitionsTest.scala | 10 +- 4 files changed, 76 insertions(+), 140 deletions(-) delete mode 100644 core/src/main/scala/kafka/admin/AddPartitionsCommand.scala diff --git a/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala b/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala deleted file mode 100644 index c74d9c224565d..0000000000000 --- a/core/src/main/scala/kafka/admin/AddPartitionsCommand.scala +++ /dev/null @@ -1,127 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.admin - -import joptsimple.OptionParser -import kafka.utils._ -import org.I0Itec.zkclient.ZkClient -import scala.collection.mutable -import kafka.common.TopicAndPartition - -object AddPartitionsCommand extends Logging { - - def main(args: Array[String]): Unit = { - val parser = new OptionParser - val topicOpt = parser.accepts("topic", "REQUIRED: The topic for which partitions need to be added.") - .withRequiredArg - .describedAs("topic") - .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") - .ofType(classOf[String]) - val nPartitionsOpt = parser.accepts("partition", "REQUIRED: Number of partitions to add to the topic") - .withRequiredArg - .describedAs("# of partitions") - .ofType(classOf[java.lang.Integer]) - val replicaAssignmentOpt = parser.accepts("replica-assignment-list", "For manually assigning replicas to brokers for the new partitions") - .withRequiredArg - .describedAs("broker_id_for_part1_replica1 : broker_id_for_part1_replica2, " + - "broker_id_for_part2_replica1 : broker_id_for_part2_replica2, ...") - .ofType(classOf[String]) - .defaultsTo("") - - val options = parser.parse(args : _*) - - for(arg <- List(topicOpt, zkConnectOpt, nPartitionsOpt)) { - if(!options.has(arg)) { - System.err.println("***Please note that this tool can only be used to add partitions when data for a topic does not use a key.***\n" + - "Missing required argument. " + " \"" + arg + "\"") - parser.printHelpOn(System.err) - System.exit(1) - } - } - - val topic = options.valueOf(topicOpt) - val zkConnect = options.valueOf(zkConnectOpt) - val nPartitions = options.valueOf(nPartitionsOpt).intValue - val replicaAssignmentStr = options.valueOf(replicaAssignmentOpt) - var zkClient: ZkClient = null - try { - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer) - addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) - println("adding partitions succeeded!") - } catch { - case e: Throwable => - println("adding partitions failed because of " + e.getMessage) - println(Utils.stackTrace(e)) - } finally { - if (zkClient != null) - zkClient.close() - } - } - - def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "") { - val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) - if (existingPartitionsReplicaList.size == 0) - throw new AdminOperationException("The topic %s does not exist".format(topic)) - - val existingReplicaList = existingPartitionsReplicaList.get(TopicAndPartition(topic,0)).get - - // create the new partition replication list - val brokerList = ZkUtils.getSortedBrokerList(zkClient) - val newPartitionReplicaList = if (replicaAssignmentStr == "") - AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, existingReplicaList.size, existingReplicaList.head, existingPartitionsReplicaList.size) - else - getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet, existingPartitionsReplicaList.size) - - // check if manual assignment has the right replication factor - val unmatchedRepFactorList = newPartitionReplicaList.values.filter(p => (p.size != existingReplicaList.size)) - if (unmatchedRepFactorList.size != 0) - throw new AdminOperationException("The replication factor in manual replication assignment " + - " is not equal to the existing replication factor for the topic " + existingReplicaList.size) - - info("Add partition list for %s is %s".format(topic, newPartitionReplicaList)) - val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) - // add the new list - partitionReplicaList ++= newPartitionReplicaList - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true) - } - - def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int): Map[Int, List[Int]] = { - val partitionList = replicaAssignmentList.split(",") - val ret = new mutable.HashMap[Int, List[Int]]() - var partitionId = startPartitionId - for (i <- 0 until partitionList.size) { - val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) - if (brokerList.size <= 0) - throw new AdminOperationException("replication factor must be larger than 0") - if (brokerList.size != brokerList.toSet.size) - throw new AdminOperationException("duplicate brokers in replica assignment: " + brokerList) - if (!brokerList.toSet.subsetOf(availableBrokerList)) - throw new AdminOperationException("some specified brokers not available. specified brokers: " + brokerList.toString + - "available broker:" + availableBrokerList.toString) - ret.put(partitionId, brokerList.toList) - if (ret(partitionId).size != ret(startPartitionId).size) - throw new AdminOperationException("partition " + i + " has different replication factor: " + brokerList) - partitionId = partitionId + 1 - } - ret.toMap - } -} diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 6560fc668e8b8..136095589b1a6 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -29,6 +29,11 @@ import scala.collection._ import mutable.ListBuffer import scala.collection.mutable import kafka.common._ +import scala.Predef._ +import collection.Map +import scala.Some +import collection.Set +import kafka.common.TopicAndPartition object AdminUtils extends Logging { val rand = new Random @@ -82,6 +87,58 @@ object AdminUtils extends Logging { } ret.toMap } + + def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "") { + val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) + if (existingPartitionsReplicaList.size == 0) + throw new AdminOperationException("The topic %s does not exist".format(topic)) + + val existingReplicaList = existingPartitionsReplicaList.head._2 + val partitionsToAdd = numPartitions - existingPartitionsReplicaList.size + if (partitionsToAdd <= 0) + throw new AdminOperationException("The number of partitions for a topic can only be increased") + + // create the new partition replication list + val brokerList = ZkUtils.getSortedBrokerList(zkClient) + val newPartitionReplicaList = if (replicaAssignmentStr == "") + AdminUtils.assignReplicasToBrokers(brokerList, partitionsToAdd, existingReplicaList.size, existingReplicaList.head, existingPartitionsReplicaList.size) + else + getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet, existingPartitionsReplicaList.size) + + // check if manual assignment has the right replication factor + val unmatchedRepFactorList = newPartitionReplicaList.values.filter(p => (p.size != existingReplicaList.size)) + if (unmatchedRepFactorList.size != 0) + throw new AdminOperationException("The replication factor in manual replication assignment " + + " is not equal to the existing replication factor for the topic " + existingReplicaList.size) + + info("Add partition list for %s is %s".format(topic, newPartitionReplicaList)) + val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2) + // add the new list + partitionReplicaList ++= newPartitionReplicaList + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true) + } + + def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int): Map[Int, List[Int]] = { + var partitionList = replicaAssignmentList.split(",") + val ret = new mutable.HashMap[Int, List[Int]]() + var partitionId = startPartitionId + partitionList = partitionList.takeRight(partitionList.size - partitionId) + for (i <- 0 until partitionList.size) { + val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) + if (brokerList.size <= 0) + throw new AdminOperationException("replication factor must be larger than 0") + if (brokerList.size != brokerList.toSet.size) + throw new AdminOperationException("duplicate brokers in replica assignment: " + brokerList) + if (!brokerList.toSet.subsetOf(availableBrokerList)) + throw new AdminOperationException("some specified brokers not available. specified brokers: " + brokerList.toString + + "available broker:" + availableBrokerList.toString) + ret.put(partitionId, brokerList.toList) + if (ret(partitionId).size != ret(startPartitionId).size) + throw new AdminOperationException("partition " + i + " has different replication factor: " + brokerList) + partitionId = partitionId + 1 + } + ret.toMap + } def deleteTopic(zkClient: ZkClient, topic: String) { zkClient.deleteRecursive(ZkUtils.getTopicPath(topic)) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 06bbd37a5af1e..ee940e9efcbd2 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -78,16 +78,21 @@ object TopicCommand { def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.topicOpt) - val topics = opts.options.valuesOf(opts.topicOpt) - val configs = parseTopicConfigs(opts) - if(opts.options.has(opts.partitionsOpt)) - Utils.croak("Changing the number of partitions is not supported.") - if(opts.options.has(opts.replicationFactorOpt)) - Utils.croak("Changing the replication factor is not supported.") - for(topic <- topics) { + val topic = opts.options.valueOf(opts.topicOpt) + if(opts.options.has(opts.configOpt)) { + val configs = parseTopicConfigs(opts) AdminUtils.changeTopicConfig(zkClient, topic, configs) println("Updated config for topic \"%s\".".format(topic)) } + if(opts.options.has(opts.partitionsOpt)) { + println("partitions can only be added when topic has no key") + val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue + val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt) + AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) + println("adding partitions succeeded!") + } + if(opts.options.has(opts.replicationFactorOpt)) + Utils.croak("Changing the replication factor is not supported.") } def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) { @@ -182,7 +187,8 @@ object TopicCommand { .withRequiredArg .describedAs("name=value") .ofType(classOf[String]) - val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created.") + val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + + "altered (Partitions can only be added for a topic which has no key. Partitions cannot be decreased") .withRequiredArg .describedAs("# of partitions") .ofType(classOf[java.lang.Integer]) diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index 09254ccac2a4d..115e20305a154 100644 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -100,7 +100,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testTopicDoesNotExist { try { - AddPartitionsCommand.addPartitions(zkClient, "Blah", 1) + AdminUtils.addPartitions(zkClient, "Blah", 1) fail("Topic should not exist") } catch { case e: AdminOperationException => //this is good @@ -110,7 +110,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { def testWrongReplicaCount { try { - AddPartitionsCommand.addPartitions(zkClient, topic1, 2, "0:1:2") + AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2") fail("Add partitions should fail") } catch { case e: AdminOperationException => //this is good @@ -119,7 +119,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testIncrementPartitions { - AddPartitionsCommand.addPartitions(zkClient, topic1, 2) + AdminUtils.addPartitions(zkClient, topic1, 3) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1, 500) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2, 500) @@ -144,7 +144,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testManualAssignmentOfReplicas { - AddPartitionsCommand.addPartitions(zkClient, topic2, 2, "0:1,2:3") + AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3") // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1, 500) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2, 500) @@ -170,7 +170,7 @@ class AddPartitionsTest extends JUnit3Suite with ZooKeeperTestHarness { } def testReplicaPlacement { - AddPartitionsCommand.addPartitions(zkClient, topic3, 6) + AdminUtils.addPartitions(zkClient, topic3, 7) // wait until leader is elected var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 1, 500) var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic3, 2, 500) From b998f5d8790cf6f4d079cba399f5c5ea8ea43e68 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Wed, 9 Oct 2013 10:47:18 -0700 Subject: [PATCH 002/567] commit the remaining changes --- core/src/main/scala/kafka/admin/AdminUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 136095589b1a6..9a8b0c97fe137 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -100,7 +100,7 @@ object AdminUtils extends Logging { // create the new partition replication list val brokerList = ZkUtils.getSortedBrokerList(zkClient) - val newPartitionReplicaList = if (replicaAssignmentStr == "") + val newPartitionReplicaList = if (replicaAssignmentStr == null || replicaAssignmentStr == "") AdminUtils.assignReplicasToBrokers(brokerList, partitionsToAdd, existingReplicaList.size, existingReplicaList.head, existingPartitionsReplicaList.size) else getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet, existingPartitionsReplicaList.size) From 7c920e97ef9e2db80774d6ccd9b85e3934e92645 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Wed, 9 Oct 2013 10:54:09 -0700 Subject: [PATCH 003/567] change comments --- core/src/main/scala/kafka/admin/TopicCommand.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index ee940e9efcbd2..56f3177e28a34 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -85,7 +85,8 @@ object TopicCommand { println("Updated config for topic \"%s\".".format(topic)) } if(opts.options.has(opts.partitionsOpt)) { - println("partitions can only be added when topic has no key") + println("WARNING: If partitions are increased for a topic that has a key, the partition " + + "logic or ordering of the messages will be affected") val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt) AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr) @@ -188,7 +189,7 @@ object TopicCommand { .describedAs("name=value") .ofType(classOf[String]) val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + - "altered (Partitions can only be added for a topic which has no key. Partitions cannot be decreased") + "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected") .withRequiredArg .describedAs("# of partitions") .ofType(classOf[java.lang.Integer]) From 5bcb41835f58be142bb6ac7c3155dfc163a516b4 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Tue, 19 Nov 2013 17:03:39 -0800 Subject: [PATCH 004/567] Add auto leader rebalance support --- .../kafka/controller/KafkaController.scala | 67 +++++++++++++++++-- .../main/scala/kafka/server/KafkaConfig.scala | 12 ++++ 2 files changed, 74 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 88792c2b2a360..d9d47c821cab3 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -28,7 +28,7 @@ import kafka.common._ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import kafka.server.{ZookeeperLeaderElector, KafkaConfig} import kafka.utils.ZkUtils._ -import kafka.utils.{Json, Utils, ZkUtils, Logging} +import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} @@ -36,6 +36,11 @@ import java.util.concurrent.atomic.AtomicInteger import scala.Some import kafka.common.TopicAndPartition import org.apache.log4j.Logger +import scala.Some +import kafka.common.TopicAndPartition +import kafka.controller.ReassignedPartitionsContext +import kafka.controller.PartitionAndReplica +import kafka.controller.LeaderIsrAndControllerEpoch class ControllerContext(val zkClient: ZkClient, val zkSessionTimeout: Int, @@ -112,6 +117,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private val replicaStateMachine = new ReplicaStateMachine(this) private val controllerElector = new ZookeeperLeaderElector(controllerContext, ZkUtils.ControllerPath, onControllerFailover, config.brokerId) + // have a separate scheduler for the controller to be able to start and stop independently of the + // kafka server + private val controllerScheduler = new KafkaScheduler(1) val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) @@ -250,6 +258,12 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg initializeAndMaybeTriggerPreferredReplicaElection() /* send partition leadership info to all live brokers */ sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) + if (config.autoLeaderRebalanceEnable) { + info("starting the partition rebalance scheduler") + controllerScheduler.startup() + controllerScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, + 5, config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS) + } } else info("Controller has been shut down, aborting startup/failover") @@ -456,7 +470,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def onPreferredReplicaElection(partitions: Set[TopicAndPartition]) { + def onPreferredReplicaElection(partitions: Set[TopicAndPartition], updateZk: Boolean = true) { info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions @@ -464,7 +478,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } catch { case e: Throwable => error("Error completing preferred replica leader election for partitions %s".format(partitions.mkString(",")), e) } finally { - removePartitionsFromPreferredReplicaElection(partitions) + removePartitionsFromPreferredReplicaElection(partitions, updateZk) } } @@ -493,6 +507,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg isRunning = false partitionStateMachine.shutdown() replicaStateMachine.shutdown() + controllerScheduler.shutdown() if(controllerContext.controllerChannelManager != null) { controllerContext.controllerChannelManager.shutdown() controllerContext.controllerChannelManager = null @@ -731,7 +746,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition]) { + def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition], updateZK : Boolean) { for(partition <- partitionsToBeRemoved) { // check the status val currentLeader = controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader @@ -742,7 +757,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg warn("Partition %s failed to complete preferred replica leader election. Leader is %d".format(partition, currentLeader)) } } - ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) + if (updateZK) + ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsToBeRemoved } @@ -898,6 +914,47 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } } + + private def checkAndTriggerPartitionRebalance(): Unit = { + if (isActive()) { + info("checking need to trigger partition rebalance") + // get all the active brokers + var preferredReplicasForTopicsByBrokers: Map[Int, Map[TopicAndPartition, Seq[Int]]] = null; + controllerContext.controllerLock synchronized { + preferredReplicasForTopicsByBrokers = controllerContext.partitionReplicaAssignment.groupBy(_._2.head) + } + debug("preferred replicas by broker " + preferredReplicasForTopicsByBrokers) + // for each broker, check if a preferred replica election needs to be triggered + preferredReplicasForTopicsByBrokers.foreach( brokerInfo => { + var imbalanceRatio: Double = 0 + var topicsNotInPreferredReplica: Map[TopicAndPartition, Seq[Int]] = null + controllerContext.controllerLock synchronized { + val brokerIds = controllerContext.liveBrokerIds + if (brokerIds.contains(brokerInfo._1) && + controllerContext.partitionsBeingReassigned.size == 0) { + // do this check only if the broker is live and there are no partitions being reassigned currently + topicsNotInPreferredReplica = + brokerInfo._2.filter(item => controllerContext.partitionLeadershipInfo(item._1).leaderAndIsr.leader != brokerInfo._1); + debug("topics not in preferred replica " + topicsNotInPreferredReplica) + val totalTopicPartitionsForBroker = brokerInfo._2.size + val totalTopicPartitionsNotLedByBroker = topicsNotInPreferredReplica.size + imbalanceRatio = totalTopicPartitionsNotLedByBroker.toDouble / totalTopicPartitionsForBroker + info("leader imbalance ratio for broker %d is %f".format(brokerInfo._1, imbalanceRatio)) + } + } + // check ratio and if greater than desired ratio, trigger a rebalance for the topics + // that need to be on this broker + if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) { + topicsNotInPreferredReplica.foreach(topicPartition => { + controllerContext.controllerLock synchronized { + onPreferredReplicaElection(Set(topicPartition._1), false) + } + }) + } + } + ) + } + } } /** diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index b324344d0a383..921f456be52d9 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -229,6 +229,18 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the purge interval (in number of requests) of the producer request purgatory */ val producerPurgatoryPurgeIntervalRequests = props.getInt("producer.purgatory.purge.interval.requests", 10000) + /* Enables auto leader balancing. A background thread checks and triggers leader + * balance if required at regular intervals */ + val autoLeaderRebalanceEnable = props.getBoolean("auto.leader.rebalance.enable", false) + + /* the ratio of leader imbalance allowed per broker. The controller would trigger a leader balance if it goes above + * this value per broker. The value is specified in percentage. */ + val leaderImbalancePerBrokerPercentage = props.getInt("leader.imbalance.per.broker.percentage", 10) + + /* the frequency with which the partition rebalance check is triggered by the controller */ + val leaderImbalanceCheckIntervalSeconds = props.getInt("leader.imbalance.check.interval.seconds", 300) + + /*********** Controlled shutdown configuration ***********/ /** Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens */ From 09e807bda103fe632e62312f036b850a59576f36 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Tue, 19 Nov 2013 17:07:43 -0800 Subject: [PATCH 005/567] fix merge conflicts --- core/src/main/scala/kafka/admin/TopicCommand.scala | 8 -------- 1 file changed, 8 deletions(-) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index c3eecdd2cbc0b..3c08dee334640 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -79,10 +79,6 @@ object TopicCommand { def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.topicOpt) val topic = opts.options.valueOf(opts.topicOpt) -<<<<<<< HEAD - if(opts.options.has(opts.configOpt)) { - val configs = parseTopicConfigs(opts) -======= if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { val configsToBeAdded = parseTopicConfigsToBeAdded(opts) val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts) @@ -90,7 +86,6 @@ object TopicCommand { val configs = AdminUtils.fetchTopicConfig(zkClient, topic) configs.putAll(configsToBeAdded) configsToBeDeleted.foreach(config => configs.remove(config)) ->>>>>>> eedbea6526986783257ad0e025c451a8ee3d9095 AdminUtils.changeTopicConfig(zkClient, topic, configs) println("Updated config for topic \"%s\".".format(topic)) } @@ -212,13 +207,10 @@ object TopicCommand { .withRequiredArg .describedAs("name=value") .ofType(classOf[String]) -<<<<<<< HEAD -======= val deleteConfigOpt = parser.accepts("deleteConfig", "A topic configuration override to be removed for an existing topic") .withRequiredArg .describedAs("name") .ofType(classOf[String]) ->>>>>>> eedbea6526986783257ad0e025c451a8ee3d9095 val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected") .withRequiredArg From a4cd17a37bf0f5ccae7b8f6a404c8b8c0de4dbad Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Tue, 19 Nov 2013 17:15:01 -0800 Subject: [PATCH 006/567] some more changes --- core/src/main/scala/kafka/admin/AdminUtils.scala | 4 ---- core/src/main/scala/kafka/controller/KafkaController.scala | 7 +------ 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 05fb5b5c3dc8a..8ff4bd5a5f6ea 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -33,10 +33,6 @@ import scala.Predef._ import collection.Map import scala.Some import collection.Set -<<<<<<< HEAD -import kafka.common.TopicAndPartition -======= ->>>>>>> eedbea6526986783257ad0e025c451a8ee3d9095 object AdminUtils extends Logging { val rand = new Random diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index d9d47c821cab3..476ed86f1ddbe 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -28,7 +28,7 @@ import kafka.common._ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import kafka.server.{ZookeeperLeaderElector, KafkaConfig} import kafka.utils.ZkUtils._ -import kafka.utils._ +import kafka.utils.{Json, Utils, ZkUtils, Logging, KafkaScheduler} import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} @@ -36,11 +36,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.Some import kafka.common.TopicAndPartition import org.apache.log4j.Logger -import scala.Some -import kafka.common.TopicAndPartition -import kafka.controller.ReassignedPartitionsContext -import kafka.controller.PartitionAndReplica -import kafka.controller.LeaderIsrAndControllerEpoch class ControllerContext(val zkClient: ZkClient, val zkSessionTimeout: Int, From 033872b316fd5a68d7463138d8199fb5d821f41b Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Tue, 19 Nov 2013 17:38:13 -0800 Subject: [PATCH 007/567] commit missing code --- .../main/scala/kafka/controller/KafkaController.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 476ed86f1ddbe..e2ad6822f444d 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -114,7 +114,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg config.brokerId) // have a separate scheduler for the controller to be able to start and stop independently of the // kafka server - private val controllerScheduler = new KafkaScheduler(1) + private val autoRebalanceScheduler = new KafkaScheduler(1) val offlinePartitionSelector = new OfflinePartitionLeaderSelector(controllerContext) private val reassignedPartitionLeaderSelector = new ReassignedPartitionLeaderSelector(controllerContext) private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext) @@ -255,8 +255,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) if (config.autoLeaderRebalanceEnable) { info("starting the partition rebalance scheduler") - controllerScheduler.startup() - controllerScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, + autoRebalanceScheduler.startup() + autoRebalanceScheduler.schedule("partition-rebalance-thread", checkAndTriggerPartitionRebalance, 5, config.leaderImbalanceCheckIntervalSeconds, TimeUnit.SECONDS) } } @@ -502,7 +502,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg isRunning = false partitionStateMachine.shutdown() replicaStateMachine.shutdown() - controllerScheduler.shutdown() + if (config.autoLeaderRebalanceEnable) + autoRebalanceScheduler.shutdown() if(controllerContext.controllerChannelManager != null) { controllerContext.controllerChannelManager.shutdown() controllerContext.controllerChannelManager = null From 55d77c67c236488df2b9f6bb59d99eeb645a0553 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Mon, 9 Dec 2013 22:51:41 -0800 Subject: [PATCH 008/567] Address code review feedbacks --- .../kafka/controller/KafkaController.scala | 87 ++++++++++++------- 1 file changed, 56 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 6cd58a4f85d9b..522e6c76daf92 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -143,6 +143,22 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } ) + newGauge( + "PreferredReplicaImbalanceCount", + new Gauge[Int] { + def value(): Int = { + controllerContext.controllerLock synchronized { + if (!isActive()) + 0 + else + controllerContext.partitionReplicaAssignment.count { + case (topicPartition, replicas) => controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != replicas.head + } + } + } + } + ) + def epoch = controllerContext.epoch def clientId = "id_%d-host_%s-port_%d".format(config.brokerId, config.hostName, config.port) @@ -465,7 +481,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def onPreferredReplicaElection(partitions: Set[TopicAndPartition], updateZk: Boolean = true) { + def onPreferredReplicaElection(partitions: Set[TopicAndPartition], isTriggeredByAutoRebalance: Boolean = true) { info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions @@ -473,7 +489,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } catch { case e: Throwable => error("Error completing preferred replica leader election for partitions %s".format(partitions.mkString(",")), e) } finally { - removePartitionsFromPreferredReplicaElection(partitions, updateZk) + removePartitionsFromPreferredReplicaElection(partitions, isTriggeredByAutoRebalance) } } @@ -742,7 +758,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition], updateZK : Boolean) { + def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition], + isTriggeredByAutoRebalance : Boolean) { for(partition <- partitionsToBeRemoved) { // check the status val currentLeader = controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader @@ -753,7 +770,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg warn("Partition %s failed to complete preferred replica leader election. Leader is %d".format(partition, currentLeader)) } } - if (updateZK) + if (isTriggeredByAutoRebalance) ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsToBeRemoved } @@ -913,42 +930,50 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private def checkAndTriggerPartitionRebalance(): Unit = { if (isActive()) { - info("checking need to trigger partition rebalance") + trace("checking need to trigger partition rebalance") // get all the active brokers - var preferredReplicasForTopicsByBrokers: Map[Int, Map[TopicAndPartition, Seq[Int]]] = null; + var preferredReplicasForTopicsByBrokers: Map[Int, Map[TopicAndPartition, Seq[Int]]] = null controllerContext.controllerLock synchronized { - preferredReplicasForTopicsByBrokers = controllerContext.partitionReplicaAssignment.groupBy(_._2.head) + preferredReplicasForTopicsByBrokers = controllerContext.partitionReplicaAssignment.groupBy { + case(topicAndPartition, assignedReplicas) => assignedReplicas.head + } } debug("preferred replicas by broker " + preferredReplicasForTopicsByBrokers) // for each broker, check if a preferred replica election needs to be triggered - preferredReplicasForTopicsByBrokers.foreach( brokerInfo => { - var imbalanceRatio: Double = 0 - var topicsNotInPreferredReplica: Map[TopicAndPartition, Seq[Int]] = null - controllerContext.controllerLock synchronized { - val brokerIds = controllerContext.liveBrokerIds - if (brokerIds.contains(brokerInfo._1) && - controllerContext.partitionsBeingReassigned.size == 0) { - // do this check only if the broker is live and there are no partitions being reassigned currently - topicsNotInPreferredReplica = - brokerInfo._2.filter(item => controllerContext.partitionLeadershipInfo(item._1).leaderAndIsr.leader != brokerInfo._1); - debug("topics not in preferred replica " + topicsNotInPreferredReplica) - val totalTopicPartitionsForBroker = brokerInfo._2.size - val totalTopicPartitionsNotLedByBroker = topicsNotInPreferredReplica.size - imbalanceRatio = totalTopicPartitionsNotLedByBroker.toDouble / totalTopicPartitionsForBroker - info("leader imbalance ratio for broker %d is %f".format(brokerInfo._1, imbalanceRatio)) + preferredReplicasForTopicsByBrokers.foreach { + case(leaderBroker, topicAndPartitionsForBroker) => { + var imbalanceRatio: Double = 0 + var topicsNotInPreferredReplica: Map[TopicAndPartition, Seq[Int]] = null + controllerContext.controllerLock synchronized { + if (controllerContext.liveBrokerIds.contains(leaderBroker) && + controllerContext.partitionsBeingReassigned.size == 0) { + // do this check only if the broker is live and there are no partitions being reassigned currently + topicsNotInPreferredReplica = + topicAndPartitionsForBroker.filter { + case(topicPartition, replicas) => { + controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != leaderBroker + } + } + debug("topics not in preferred replica " + topicsNotInPreferredReplica) + val totalTopicPartitionsForBroker = topicAndPartitionsForBroker.size + val totalTopicPartitionsNotLedByBroker = topicsNotInPreferredReplica.size + imbalanceRatio = totalTopicPartitionsNotLedByBroker.toDouble / totalTopicPartitionsForBroker + trace("leader imbalance ratio for broker %d is %f".format(leaderBroker, imbalanceRatio)) + } } - } - // check ratio and if greater than desired ratio, trigger a rebalance for the topics - // that need to be on this broker - if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) { - topicsNotInPreferredReplica.foreach(topicPartition => { - controllerContext.controllerLock synchronized { - onPreferredReplicaElection(Set(topicPartition._1), false) + // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions + // that need to be on this broker + if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) { + topicsNotInPreferredReplica.foreach { + case(topicPartition, replicas) => { + controllerContext.controllerLock synchronized { + onPreferredReplicaElection(Set(topicPartition), false) + } + } } - }) + } } } - ) } } } From 425af9b4a7afeb21191c33ba6bc3f20623a3f0b3 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Fri, 20 Dec 2013 11:11:20 -0800 Subject: [PATCH 009/567] use zk for auto rebalance --- .../kafka/controller/KafkaController.scala | 48 +++++++++++-------- 1 file changed, 29 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 522e6c76daf92..74e2ea4faa8e2 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -21,7 +21,7 @@ import collection.immutable.Set import com.yammer.metrics.core.Gauge import java.lang.{IllegalStateException, Object} import java.util.concurrent.TimeUnit -import kafka.admin.PreferredReplicaLeaderElectionCommand +import kafka.admin.{AdminOperationException, PreferredReplicaLeaderElectionCommand} import kafka.api._ import kafka.cluster.Broker import kafka.common._ @@ -945,29 +945,39 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg var imbalanceRatio: Double = 0 var topicsNotInPreferredReplica: Map[TopicAndPartition, Seq[Int]] = null controllerContext.controllerLock synchronized { - if (controllerContext.liveBrokerIds.contains(leaderBroker) && - controllerContext.partitionsBeingReassigned.size == 0) { - // do this check only if the broker is live and there are no partitions being reassigned currently - topicsNotInPreferredReplica = - topicAndPartitionsForBroker.filter { - case(topicPartition, replicas) => { - controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != leaderBroker - } + topicsNotInPreferredReplica = + topicAndPartitionsForBroker.filter { + case(topicPartition, replicas) => { + controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != leaderBroker } - debug("topics not in preferred replica " + topicsNotInPreferredReplica) - val totalTopicPartitionsForBroker = topicAndPartitionsForBroker.size - val totalTopicPartitionsNotLedByBroker = topicsNotInPreferredReplica.size - imbalanceRatio = totalTopicPartitionsNotLedByBroker.toDouble / totalTopicPartitionsForBroker - trace("leader imbalance ratio for broker %d is %f".format(leaderBroker, imbalanceRatio)) - } + } + debug("topics not in preferred replica " + topicsNotInPreferredReplica) + val totalTopicPartitionsForBroker = topicAndPartitionsForBroker.size + val totalTopicPartitionsNotLedByBroker = topicsNotInPreferredReplica.size + imbalanceRatio = totalTopicPartitionsNotLedByBroker.toDouble / totalTopicPartitionsForBroker + trace("leader imbalance ratio for broker %d is %f".format(leaderBroker, imbalanceRatio)) } // check ratio and if greater than desired ratio, trigger a rebalance for the topic partitions // that need to be on this broker if (imbalanceRatio > (config.leaderImbalancePerBrokerPercentage.toDouble / 100)) { - topicsNotInPreferredReplica.foreach { - case(topicPartition, replicas) => { - controllerContext.controllerLock synchronized { - onPreferredReplicaElection(Set(topicPartition), false) + controllerContext.controllerLock synchronized { + // do this check only if the broker is live and there are no partitions being reassigned currently + // and preferred replica election is not in progress + if (controllerContext.liveBrokerIds.contains(leaderBroker) && + controllerContext.partitionsBeingReassigned.size == 0 && + controllerContext.partitionsUndergoingPreferredReplicaElection.size == 0) { + val zkPath = ZkUtils.PreferredReplicaLeaderElectionPath + val partitionsList = topicsNotInPreferredReplica.keys.map(e => Map("topic" -> e.topic, "partition" -> e.partition)) + val jsonData = Json.encode(Map("version" -> 1, "partitions" -> partitionsList)) + try { + ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) + info("Created preferred replica election path with %s".format(jsonData)) + } catch { + case e2: Throwable => + val partitionsUndergoingPreferredReplicaElection = + PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(ZkUtils.readData(zkClient, zkPath)._1) + error("Preferred replica leader election currently in progress for " + + "%s. Aborting operation".format(partitionsUndergoingPreferredReplicaElection)); } } } From e88f1acdaac9fe738787ccb8375519293c913a37 Mon Sep 17 00:00:00 2001 From: Sriram Subramanian Date: Fri, 20 Dec 2013 11:22:26 -0800 Subject: [PATCH 010/567] some more changes --- .../scala/kafka/controller/KafkaController.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 74e2ea4faa8e2..8017abb61def5 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -481,7 +481,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def onPreferredReplicaElection(partitions: Set[TopicAndPartition], isTriggeredByAutoRebalance: Boolean = true) { + def onPreferredReplicaElection(partitions: Set[TopicAndPartition]) { info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions @@ -489,7 +489,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } catch { case e: Throwable => error("Error completing preferred replica leader election for partitions %s".format(partitions.mkString(",")), e) } finally { - removePartitionsFromPreferredReplicaElection(partitions, isTriggeredByAutoRebalance) + removePartitionsFromPreferredReplicaElection(partitions) } } @@ -758,8 +758,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg } } - def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition], - isTriggeredByAutoRebalance : Boolean) { + def removePartitionsFromPreferredReplicaElection(partitionsToBeRemoved: Set[TopicAndPartition]) { for(partition <- partitionsToBeRemoved) { // check the status val currentLeader = controllerContext.partitionLeadershipInfo(partition).leaderAndIsr.leader @@ -770,8 +769,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg warn("Partition %s failed to complete preferred replica leader election. Leader is %d".format(partition, currentLeader)) } } - if (isTriggeredByAutoRebalance) - ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) + ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath) controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsToBeRemoved } @@ -973,11 +971,13 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg ZkUtils.createPersistentPath(zkClient, zkPath, jsonData) info("Created preferred replica election path with %s".format(jsonData)) } catch { - case e2: Throwable => + case e2: ZkNodeExistsException => val partitionsUndergoingPreferredReplicaElection = PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(ZkUtils.readData(zkClient, zkPath)._1) error("Preferred replica leader election currently in progress for " + "%s. Aborting operation".format(partitionsUndergoingPreferredReplicaElection)); + case e3: Throwable => + error("Error while trying to auto rebalance topics %s".format(topicsNotInPreferredReplica.keys)) } } } From 1a7048d41bb56118d4f31ff972618ebe13007857 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Sat, 8 Feb 2014 11:09:39 -0800 Subject: [PATCH 011/567] KAFKA-330 Delete topic followup - more tests and Joel's review comments --- .../controller/ControllerChannelManager.scala | 35 ++++++---- .../kafka/controller/KafkaController.scala | 20 +++--- .../controller/PartitionStateMachine.scala | 41 +++++++---- .../controller/ReplicaStateMachine.scala | 62 +++++++++-------- .../controller/TopicDeletionManager.scala | 69 ++++++++++--------- .../main/scala/kafka/server/KafkaApis.scala | 5 +- .../unit/kafka/admin/DeleteTopicTest.scala | 63 ++++++++++++++++- 7 files changed, 193 insertions(+), 102 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index a1ee5a7074121..8ab8ab66bda1e 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -235,18 +235,29 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging callback: (RequestOrResponse) => Unit = null) { val partitionList = controllerContext.partitionLeadershipInfo.keySet.dropWhile( p => controller.deleteTopicManager.isTopicQueuedUpForDeletion(p.topic)) - partitionList.foreach { partition => - val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) - leaderIsrAndControllerEpochOpt match { - case Some(leaderIsrAndControllerEpoch) => - val replicas = controllerContext.partitionReplicaAssignment(partition).toSet - val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) + if(partitionList.size > 0) { + partitionList.foreach { partition => + val leaderIsrAndControllerEpochOpt = controllerContext.partitionLeadershipInfo.get(partition) + leaderIsrAndControllerEpochOpt match { + case Some(leaderIsrAndControllerEpoch) => + val replicas = controllerContext.partitionReplicaAssignment(partition).toSet + val partitionStateInfo = PartitionStateInfo(leaderIsrAndControllerEpoch, replicas) + brokerIds.filter(b => b >= 0).foreach { brokerId => + updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) + updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo) + } + case None => + info("Leader not assigned yet for partition %s. Skip sending udpate metadata request".format(partition)) + } + } + } else { + if(controllerContext.partitionLeadershipInfo.keySet.size > 0) { + // last set of topics are being deleted + controllerContext.partitionLeadershipInfo.foreach { case(partition, leaderIsrAndControllerEpoch) => brokerIds.filter(b => b >= 0).foreach { brokerId => - updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) - updateMetadataRequestMap(brokerId).put(partition, partitionStateInfo) + updateMetadataRequestMap.put(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) } - case None => - info("Leader not assigned yet for partition %s. Skip sending udpate metadata request".format(partition)) + } } } } @@ -272,10 +283,10 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val broker = m._1 val partitionStateInfos = m._2.toMap val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, - partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) + partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, - correlationId, broker, p._1))) + correlationId, broker, p._1))) controller.sendRequest(broker, updateMetadataRequest, null) } updateMetadataRequestMap.clear() diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index d812cb4121d7f..8acd07680ebf5 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -433,7 +433,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg if(replicasForTopicsToBeDeleted.size > 0) { // it is required to mark the respective replicas in TopicDeletionFailed state since the replica cannot be // deleted when the broker is down. This will prevent the replica from being in TopicDeletionStarted state indefinitely - // since topic deletion cannot be retried if at least one replica is in TopicDeletionStarted state + // since topic deletion cannot be retried until at least one replica is in TopicDeletionStarted state deleteTopicManager.failReplicaDeletion(replicasForTopicsToBeDeleted) } } @@ -443,6 +443,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg * and partitions as input. It does the following - * 1. Registers partition change listener. This is not required until KAFKA-347 * 2. Invokes the new partition callback + * 3. Send metadata request with the new topic to all brokers so they allow requests for that topic to be served */ def onNewTopicCreation(topics: Set[String], newPartitions: Set[TopicAndPartition]) { info("New topic creation callback for %s".format(newPartitions.mkString(","))) @@ -581,8 +582,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg // first register ISR change listener watchIsrChangesForReassignedPartition(topic, partition, reassignedPartitionContext) controllerContext.partitionsBeingReassigned.put(topicAndPartition, reassignedPartitionContext) - // halt topic deletion for the partitions being reassigned - deleteTopicManager.haltTopicDeletion(Set(topic)) + // mark topic ineligible for deletion for the partitions being reassigned + deleteTopicManager.markTopicIneligibleForDeletion(Set(topic)) onPartitionReassignment(topicAndPartition, reassignedPartitionContext) } else { // some replica in RAR is not alive. Fail partition reassignment @@ -605,7 +606,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg info("Starting preferred replica leader election for partitions %s".format(partitions.mkString(","))) try { controllerContext.partitionsUndergoingPreferredReplicaElection ++= partitions - deleteTopicManager.haltTopicDeletion(partitions.map(_.topic)) + deleteTopicManager.markTopicIneligibleForDeletion(partitions.map(_.topic)) partitionStateMachine.handleStateChanges(partitions, OnlinePartition, preferredReplicaPartitionLeaderSelector) } catch { case e: Throwable => error("Error completing preferred replica leader election for partitions %s".format(partitions.mkString(",")), e) @@ -748,17 +749,16 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient) extends Logg private def initializeTopicDeletion() { val topicsQueuedForDeletion = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.DeleteTopicsPath).toSet - val replicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter(r => - r._2.foldLeft(false)((res,r) => res || !controllerContext.liveBrokerIds.contains(r))) - val topicsWithReplicasOnDeadBrokers = replicasOnDeadBrokers.map(_._1.topic).toSet + val topicsWithReplicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter { case(partition, replicas) => + replicas.exists(r => !controllerContext.liveBrokerIds.contains(r)) }.keySet.map(_.topic) val topicsForWhichPartitionReassignmentIsInProgress = controllerContext.partitionsUndergoingPreferredReplicaElection.map(_.topic) val topicsForWhichPreferredReplicaElectionIsInProgress = controllerContext.partitionsBeingReassigned.keySet.map(_.topic) - val haltedTopicsForDeletion = topicsWithReplicasOnDeadBrokers | topicsForWhichPartitionReassignmentIsInProgress | + val topicsIneligibleForDeletion = topicsWithReplicasOnDeadBrokers | topicsForWhichPartitionReassignmentIsInProgress | topicsForWhichPreferredReplicaElectionIsInProgress info("List of topics to be deleted: %s".format(topicsQueuedForDeletion.mkString(","))) - info("List of topics halted for deletion: %s".format(haltedTopicsForDeletion.mkString(","))) + info("List of topics ineligible for deletion: %s".format(topicsIneligibleForDeletion.mkString(","))) // initialize the topic deletion manager - deleteTopicManager = new TopicDeletionManager(this, topicsQueuedForDeletion, haltedTopicsForDeletion) + deleteTopicManager = new TopicDeletionManager(this, topicsQueuedForDeletion, topicsIneligibleForDeletion) } private def maybeTriggerPartitionReassignment() { diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index 57c96b5539f20..c69077e6f4dc0 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -51,6 +51,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { private val noOpPartitionLeaderSelector = new NoOpLeaderSelector(controllerContext) this.logIdent = "[Partition state machine on Controller " + controllerId + "]: " private val stateChangeLogger = Logger.getLogger(KafkaController.stateChangeLogger) + private var topicChangeListener: TopicChangeListener = null + private var deleteTopicsListener: DeleteTopicsListener = null + private var addPartitionsListener: mutable.Map[String, AddPartitionsListener] = mutable.Map.empty /** * Invoked on successful controller election. First registers a topic change listener since that triggers all @@ -167,8 +170,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { assignReplicasToPartitions(topic, partition) partitionState.put(topicAndPartition, NewPartition) val assignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition).mkString(",") - stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from NotExists to New with assigned replicas %s" - .format(controllerId, controller.epoch, topicAndPartition, assignedReplicas)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s with assigned replicas %s" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState, + assignedReplicas)) // post: partition has been assigned replicas case OnlinePartition => assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition, OfflinePartition), OnlinePartition) @@ -184,22 +188,22 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } partitionState.put(topicAndPartition, OnlinePartition) val leader = controllerContext.partitionLeadershipInfo(topicAndPartition).leaderAndIsr.leader - stateChangeLogger.trace("Controller %d epoch %d changed partition %s from %s to OnlinePartition with leader %d" - .format(controllerId, controller.epoch, topicAndPartition, partitionState(topicAndPartition), leader)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s from %s to %s with leader %d" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState, leader)) // post: partition has a leader case OfflinePartition => // pre: partition should be in New or Online state assertValidPreviousStates(topicAndPartition, List(NewPartition, OnlinePartition, OfflinePartition), OfflinePartition) // should be called when the leader for a partition is no longer alive - stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from Online to Offline" - .format(controllerId, controller.epoch, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState)) partitionState.put(topicAndPartition, OfflinePartition) // post: partition has no alive leader case NonExistentPartition => // pre: partition should be in Offline state assertValidPreviousStates(topicAndPartition, List(OfflinePartition), NonExistentPartition) - stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from Offline to NotExists" - .format(controllerId, controller.epoch, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed partition %s state from %s to %s" + .format(controllerId, controller.epoch, topicAndPartition, currState, targetState)) partitionState.put(topicAndPartition, NonExistentPartition) // post: partition state is deleted from all brokers and zookeeper } @@ -358,15 +362,22 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } private def registerTopicChangeListener() = { - zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, new TopicChangeListener()) + topicChangeListener = new TopicChangeListener() + zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicChangeListener) } def registerPartitionChangeListener(topic: String) = { - zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), new AddPartitionsListener(topic)) + addPartitionsListener.put(topic, new AddPartitionsListener(topic)) + zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), addPartitionsListener(topic)) + } + + def deregisterPartitionChangeListener(topic: String) = { + zkClient.unsubscribeDataChanges(ZkUtils.getTopicPath(topic), addPartitionsListener(topic)) } private def registerDeleteTopicListener() = { - zkClient.subscribeChildChanges(ZkUtils.DeleteTopicsPath, new DeleteTopicsListener()) + deleteTopicsListener = new DeleteTopicsListener() + zkClient.subscribeChildChanges(ZkUtils.DeleteTopicsPath, deleteTopicsListener) } private def getLeaderIsrAndEpochOrThrowException(topic: String, partition: Int): LeaderIsrAndControllerEpoch = { @@ -438,21 +449,23 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { } debug("Delete topics listener fired for topics %s to be deleted".format(topicsToBeDeleted.mkString(","))) val nonExistentTopics = topicsToBeDeleted.filter(t => !controllerContext.allTopics.contains(t)) - if(nonExistentTopics.size > 0) + if(nonExistentTopics.size > 0) { warn("Ignoring request to delete non-existing topics " + nonExistentTopics.mkString(",")) + nonExistentTopics.foreach(topic => ZkUtils.deletePathRecursive(zkClient, ZkUtils.getDeleteTopicPath(topic))) + } topicsToBeDeleted --= nonExistentTopics if(topicsToBeDeleted.size > 0) { info("Starting topic deletion for topics " + topicsToBeDeleted.mkString(",")) // add topic to deletion list controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) - // halt if other state changes are in progress + // mark topic ineligible for deletion if other state changes are in progress topicsToBeDeleted.foreach { topic => val preferredReplicaElectionInProgress = controllerContext.partitionsUndergoingPreferredReplicaElection.map(_.topic).contains(topic) val partitionReassignmentInProgress = controllerContext.partitionsBeingReassigned.keySet.map(_.topic).contains(topic) if(preferredReplicaElectionInProgress || partitionReassignmentInProgress) - controller.deleteTopicManager.haltTopicDeletion(Set(topic)) + controller.deleteTopicManager.markTopicIneligibleForDeletion(Set(topic)) } } } diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 613aec6f40db5..5e016d5d2bbef 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -40,7 +40,7 @@ import kafka.utils.Utils._ * 4. ReplicaDeletionStarted: If replica deletion starts, it is moved to this state. Valid previous state is OfflineReplica * 5. ReplicaDeletionSuccessful: If replica responds with no error code in response to a delete replica request, it is * moved to this state. Valid previous state is ReplicaDeletionStarted - * 6. ReplicaDeletionFailed: If replica deletion fails, it is moved to this state. Valid previous state is ReplicaDeletionStarted + * 6. ReplicaDeletionIneligible: If replica deletion fails, it is moved to this state. Valid previous state is ReplicaDeletionStarted * 7. NonExistentReplica: If a replica is deleted successfully, it is moved to this state. Valid previous state is * ReplicaDeletionSuccessful */ @@ -115,7 +115,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * --send LeaderAndIsr request with current leader and isr to the new replica and UpdateMetadata request for the * partition to every live broker * - * NewReplica,OnlineReplica,OfflineReplica,ReplicaDeletionFailed -> OfflineReplica + * NewReplica,OnlineReplica,OfflineReplica,ReplicaDeletionIneligible -> OfflineReplica * --send StopReplicaRequest to the replica (w/o deletion) * --remove this replica from the isr and send LeaderAndIsr request (with new isr) to the leader replica and * UpdateMetadata request for the partition to every live broker. @@ -126,7 +126,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { * ReplicaDeletionStarted -> ReplicaDeletionSuccessful * -- mark the state of the replica in the state machine * - * ReplicaDeletionStarted -> ReplicaDeletionFailed + * ReplicaDeletionStarted -> ReplicaDeletionIneligible * -- mark the state of the replica in the state machine * * ReplicaDeletionSuccessful -> NonExistentReplica @@ -146,8 +146,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { throw new StateChangeFailedException(("Controller %d epoch %d initiated state change of replica %d for partition %s " + "to %s failed because replica state machine has not started") .format(controllerId, controller.epoch, replicaId, topicAndPartition, targetState)) + val currState = replicaState.getOrElseUpdate(partitionAndReplica, NonExistentReplica) try { - replicaState.getOrElseUpdate(partitionAndReplica, NonExistentReplica) val replicaAssignment = controllerContext.partitionReplicaAssignment(topicAndPartition) targetState match { case NewReplica => @@ -165,45 +165,47 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case None => // new leader request will be sent to this replica when one gets elected } replicaState.put(partitionAndReplica, NewReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NewReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, + targetState)) case ReplicaDeletionStarted => assertValidPreviousStates(partitionAndReplica, List(OfflineReplica), targetState) replicaState.put(partitionAndReplica, ReplicaDeletionStarted) // send stop replica command brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = true, callbacks.stopReplicaResponseCallback) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionStarted" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) - case ReplicaDeletionFailed => + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) + case ReplicaDeletionIneligible => assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionStarted), targetState) - replicaState.put(partitionAndReplica, ReplicaDeletionFailed) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionFailed" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + replicaState.put(partitionAndReplica, ReplicaDeletionIneligible) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case ReplicaDeletionSuccessful => assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionStarted), targetState) replicaState.put(partitionAndReplica, ReplicaDeletionSuccessful) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to ReplicaDeletionSuccessful" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case NonExistentReplica => assertValidPreviousStates(partitionAndReplica, List(ReplicaDeletionSuccessful), targetState) // remove this replica from the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas.filterNot(_ == replicaId)) replicaState.remove(partitionAndReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to NonExistentReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case OnlineReplica => assertValidPreviousStates(partitionAndReplica, - List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionFailed), targetState) + List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionIneligible), targetState) replicaState(partitionAndReplica) match { case NewReplica => // add this replica to the assigned replicas list for its partition val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) if(!currentAssignedReplicas.contains(replicaId)) controllerContext.partitionReplicaAssignment.put(topicAndPartition, currentAssignedReplicas :+ replicaId) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, + targetState)) case _ => // check if the leader for this partition ever existed controllerContext.partitionLeadershipInfo.get(topicAndPartition) match { @@ -211,8 +213,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(replicaId), topic, partition, leaderIsrAndControllerEpoch, replicaAssignment) replicaState.put(partitionAndReplica, OnlineReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OnlineReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) case None => // that means the partition was never in OnlinePartition state, this means the broker never // started a log for that partition and does not have a high watermark value for this partition } @@ -220,7 +222,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { replicaState.put(partitionAndReplica, OnlineReplica) case OfflineReplica => assertValidPreviousStates(partitionAndReplica, - List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionFailed), targetState) + List(NewReplica, OnlineReplica, OfflineReplica, ReplicaDeletionIneligible), targetState) // send stop replica command to the replica so that it stops fetching from the leader brokerRequestBatch.addStopReplicaRequestForBrokers(List(replicaId), topic, partition, deletePartition = false) // As an optimization, the controller removes dead replicas from the ISR @@ -233,8 +235,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment) replicaState.put(partitionAndReplica, OfflineReplica) - stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s to OfflineReplica" - .format(controllerId, controller.epoch, replicaId, topicAndPartition)) + stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" + .format(controllerId, controller.epoch, replicaId, topicAndPartition, currState, targetState)) false case None => true @@ -250,8 +252,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } catch { case t: Throwable => - stateChangeLogger.error("Controller %d epoch %d initiated state change of replica %d for partition [%s,%d] to %s failed" - .format(controllerId, controller.epoch, replicaId, topic, partition, targetState), t) + stateChangeLogger.error("Controller %d epoch %d initiated state change of replica %d for partition [%s,%d] from %s to %s failed" + .format(controllerId, controller.epoch, replicaId, topic, partition, currState, targetState), t) } } @@ -273,7 +275,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { } def replicasInDeletionStates(topic: String): Set[PartitionAndReplica] = { - val deletionStates = Set(ReplicaDeletionStarted, ReplicaDeletionSuccessful, ReplicaDeletionFailed) + val deletionStates = Set(ReplicaDeletionStarted, ReplicaDeletionSuccessful, ReplicaDeletionIneligible) replicaState.filter(r => r._1.topic.equals(topic) && deletionStates.contains(r._2)).keySet } @@ -304,8 +306,8 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case false => // mark replicas on dead brokers as failed for topic deletion, if they belong to a topic to be deleted. // This is required during controller failover since during controller failover a broker can go down, - // so the replicas on that broker should be moved to ReplicaDeletionFailed to be on the safer side. - replicaState.put(partitionAndReplica, ReplicaDeletionFailed) + // so the replicas on that broker should be moved to ReplicaDeletionIneligible to be on the safer side. + replicaState.put(partitionAndReplica, ReplicaDeletionIneligible) } } } @@ -356,7 +358,7 @@ case object OnlineReplica extends ReplicaState { val state: Byte = 2 } case object OfflineReplica extends ReplicaState { val state: Byte = 3 } case object ReplicaDeletionStarted extends ReplicaState { val state: Byte = 4} case object ReplicaDeletionSuccessful extends ReplicaState { val state: Byte = 5} -case object ReplicaDeletionFailed extends ReplicaState { val state: Byte = 6} +case object ReplicaDeletionIneligible extends ReplicaState { val state: Byte = 6} case object NonExistentReplica extends ReplicaState { val state: Byte = 7 } diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index 91a446ddc8aeb..58f1c4274e9f8 100644 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -30,8 +30,8 @@ import kafka.api.{StopReplicaResponse, RequestOrResponse} * 3. The controller has a background thread that handles topic deletion. The purpose of having this background thread * is to accommodate the TTL feature, when we have it. This thread is signaled whenever deletion for a topic needs to * be started or resumed. Currently, a topic's deletion can be started only by the onPartitionDeletion callback on the - * controller. In the future, it can be triggered based on the configured TTL for the topic. A topic's deletion will - * be halted in the following scenarios - + * controller. In the future, it can be triggered based on the configured TTL for the topic. A topic will be ineligible + * for deletion in the following scenarios - * 3.1 broker hosting one of the replicas for that topic goes down * 3.2 partition reassignment for partitions of that topic is in progress * 3.3 preferred replica election for partitions of that topic is in progress @@ -62,17 +62,17 @@ import kafka.api.{StopReplicaResponse, RequestOrResponse} * it marks the topic for deletion retry. * @param controller * @param initialTopicsToBeDeleted The topics that are queued up for deletion in zookeeper at the time of controller failover - * @param initialHaltedTopicsForDeletion The topics for which deletion is halted due to any of the conditions mentioned in #3 above + * @param initialTopicsIneligibleForDeletion The topics ineligible for deletion due to any of the conditions mentioned in #3 above */ class TopicDeletionManager(controller: KafkaController, initialTopicsToBeDeleted: Set[String] = Set.empty, - initialHaltedTopicsForDeletion: Set[String] = Set.empty) extends Logging { + initialTopicsIneligibleForDeletion: Set[String] = Set.empty) extends Logging { val controllerContext = controller.controllerContext val partitionStateMachine = controller.partitionStateMachine val replicaStateMachine = controller.replicaStateMachine var topicsToBeDeleted: mutable.Set[String] = mutable.Set.empty[String] ++ initialTopicsToBeDeleted - var haltedTopicsForDeletion: mutable.Set[String] = mutable.Set.empty[String] ++ - (initialHaltedTopicsForDeletion & initialTopicsToBeDeleted) + var topicsIneligibleForDeletion: mutable.Set[String] = mutable.Set.empty[String] ++ + (initialTopicsIneligibleForDeletion & initialTopicsToBeDeleted) val deleteTopicsCond = controllerContext.controllerLock.newCondition() var deleteTopicStateChanged: Boolean = false var deleteTopicsThread: DeleteTopicsThread = null @@ -92,7 +92,7 @@ class TopicDeletionManager(controller: KafkaController, def shutdown() { deleteTopicsThread.shutdown() topicsToBeDeleted.clear() - haltedTopicsForDeletion.clear() + topicsIneligibleForDeletion.clear() } /** @@ -117,7 +117,7 @@ class TopicDeletionManager(controller: KafkaController, def resumeDeletionForTopics(topics: Set[String] = Set.empty) { val topicsToResumeDeletion = topics & topicsToBeDeleted if(topicsToResumeDeletion.size > 0) { - haltedTopicsForDeletion --= topicsToResumeDeletion + topicsIneligibleForDeletion --= topicsToResumeDeletion resumeTopicDeletionThread() } } @@ -125,8 +125,8 @@ class TopicDeletionManager(controller: KafkaController, /** * Invoked when a broker that hosts replicas for topics to be deleted goes down. Also invoked when the callback for * StopReplicaResponse receives an error code for the replicas of a topic to be deleted. As part of this, the replicas - * are moved from ReplicaDeletionStarted to ReplicaDeletionFailed state. Also, the topic is added to the list of topics - * for which deletion is halted until further notice. The delete topic thread is notified so it can retry topic deletion + * are moved from ReplicaDeletionStarted to ReplicaDeletionIneligible state. Also, the topic is added to the list of topics + * ineligible for deletion until further notice. The delete topic thread is notified so it can retry topic deletion * if it has received a response for all replicas of a topic to be deleted * @param replicas Replicas for which deletion has failed */ @@ -136,8 +136,8 @@ class TopicDeletionManager(controller: KafkaController, val topics = replicasThatFailedToDelete.map(_.topic) debug("Deletion failed for replicas %s. Halting deletion for topics %s" .format(replicasThatFailedToDelete.mkString(","), topics)) - controller.replicaStateMachine.handleStateChanges(replicasThatFailedToDelete, ReplicaDeletionFailed) - haltTopicDeletion(topics) + controller.replicaStateMachine.handleStateChanges(replicasThatFailedToDelete, ReplicaDeletionIneligible) + markTopicIneligibleForDeletion(topics) resumeTopicDeletionThread() } } @@ -147,17 +147,17 @@ class TopicDeletionManager(controller: KafkaController, * 1. replicas being down * 2. partition reassignment in progress for some partitions of the topic * 3. preferred replica election in progress for some partitions of the topic - * @param topics Topics for which deletion should be halted. No op if the topic is was not previously queued up for deletion + * @param topics Topics that should be marked ineligible for deletion. No op if the topic is was not previously queued up for deletion */ - def haltTopicDeletion(topics: Set[String]) { + def markTopicIneligibleForDeletion(topics: Set[String]) { val newTopicsToHaltDeletion = topicsToBeDeleted & topics - haltedTopicsForDeletion ++= newTopicsToHaltDeletion + topicsIneligibleForDeletion ++= newTopicsToHaltDeletion if(newTopicsToHaltDeletion.size > 0) info("Halted deletion of topics %s".format(newTopicsToHaltDeletion.mkString(","))) } - def isTopicDeletionHalted(topic: String): Boolean = { - haltedTopicsForDeletion.contains(topic) + def isTopicIneligibleForDeletion(topic: String): Boolean = { + topicsIneligibleForDeletion.contains(topic) } def isTopicDeletionInProgress(topic: String): Boolean = { @@ -205,26 +205,29 @@ class TopicDeletionManager(controller: KafkaController, * Topic deletion can be retried if - * 1. Topic deletion is not already complete * 2. Topic deletion is currently not in progress for that topic - * 3. Topic deletion is currently halted for that topic + * 3. Topic is currently marked ineligible for deletion * @param topic Topic * @return Whether or not deletion can be retried for the topic */ private def isTopicEligibleForDeletion(topic: String): Boolean = { - topicsToBeDeleted.contains(topic) && (!isTopicDeletionInProgress(topic) && !isTopicDeletionHalted(topic)) + topicsToBeDeleted.contains(topic) && (!isTopicDeletionInProgress(topic) && !isTopicIneligibleForDeletion(topic)) } /** * If the topic is queued for deletion but deletion is not currently under progress, then deletion is retried for that topic - * To ensure a successful retry, reset states for respective replicas from ReplicaDeletionFailed to OfflineReplica state + * To ensure a successful retry, reset states for respective replicas from ReplicaDeletionIneligible to OfflineReplica state *@param topic Topic for which deletion should be retried */ private def markTopicForDeletionRetry(topic: String) { - // reset replica states from ReplicaDeletionFailed to OfflineReplica - val failedReplicas = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionFailed) + // reset replica states from ReplicaDeletionIneligible to OfflineReplica + val failedReplicas = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionIneligible) controller.replicaStateMachine.handleStateChanges(failedReplicas, OfflineReplica) } private def completeDeleteTopic(topic: String) { + // deregister partition change listener on the deleted topic. This is to prevent the partition change listener + // firing before the new topic listener when a deleted topic gets auto created + partitionStateMachine.deregisterPartitionChangeListener(topic) val replicasForDeletedTopic = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionSuccessful) // controller will remove this replica from the state machine as well as its partition assignment cache replicaStateMachine.handleStateChanges(replicasForDeletedTopic, NonExistentReplica) @@ -245,6 +248,8 @@ class TopicDeletionManager(controller: KafkaController, */ private def onTopicDeletion(topics: Set[String]) { info("Topic deletion callback for %s".format(topics.mkString(","))) + // send update metadata so that brokers stop serving data for topics to be deleted + controller.sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) val partitionReplicaAssignmentByTopic = controllerContext.partitionReplicaAssignment.groupBy(p => p._1.topic) topics.foreach { topic => onPartitionDeletion(partitionReplicaAssignmentByTopic(topic).map(_._1).toSet) @@ -257,34 +262,32 @@ class TopicDeletionManager(controller: KafkaController, * the topics are added to the in progress list. As long as a topic is in the in progress list, deletion for that topic * is never retried. A topic is removed from the in progress list when * 1. Either the topic is successfully deleted OR - * 2. No replica for the topic is in ReplicaDeletionStarted state and at least one replica is in ReplicaDeletionFailed state + * 2. No replica for the topic is in ReplicaDeletionStarted state and at least one replica is in ReplicaDeletionIneligible state * If the topic is queued for deletion but deletion is not currently under progress, then deletion is retried for that topic * As part of starting deletion, all replicas are moved to the ReplicaDeletionStarted state where the controller sends * the replicas a StopReplicaRequest (delete=true) * This callback does the following things - * 1. Send metadata request to all brokers excluding the topics to be deleted - * 2. Move all dead replicas directly to ReplicaDeletionFailed state. Also halt the deletion of respective topics if - * some replicas are dead since it won't complete successfully anyway + * 2. Move all dead replicas directly to ReplicaDeletionIneligible state. Also mark the respective topics ineligible + * for deletion if some replicas are dead since it won't complete successfully anyway * 3. Move all alive replicas to ReplicaDeletionStarted state so they can be deleted successfully *@param replicasForTopicsToBeDeleted */ private def startReplicaDeletion(replicasForTopicsToBeDeleted: Set[PartitionAndReplica]) { replicasForTopicsToBeDeleted.groupBy(_.topic).foreach { case(topic, replicas) => - // send update metadata so that brokers stop serving data - controller.sendUpdateMetadataRequest(controllerContext.liveOrShuttingDownBrokerIds.toSeq) var aliveReplicasForTopic = controllerContext.allLiveReplicas().filter(p => p.topic.equals(topic)) val deadReplicasForTopic = replicasForTopicsToBeDeleted -- aliveReplicasForTopic val successfullyDeletedReplicas = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionSuccessful) val replicasForDeletionRetry = aliveReplicasForTopic -- successfullyDeletedReplicas // move dead replicas directly to failed state - replicaStateMachine.handleStateChanges(deadReplicasForTopic, ReplicaDeletionFailed) + replicaStateMachine.handleStateChanges(deadReplicasForTopic, ReplicaDeletionIneligible) // send stop replica to all followers that are not in the OfflineReplica state so they stop sending fetch requests to the leader replicaStateMachine.handleStateChanges(replicasForDeletionRetry, OfflineReplica) debug("Deletion started for replicas %s".format(replicasForDeletionRetry.mkString(","))) controller.replicaStateMachine.handleStateChanges(replicasForDeletionRetry, ReplicaDeletionStarted, new Callbacks.CallbackBuilder().stopReplicaCallback(deleteTopicStopReplicaCallback).build) if(deadReplicasForTopic.size > 0) - haltTopicDeletion(Set(topic)) + markTopicIneligibleForDeletion(Set(topic)) } } @@ -314,7 +317,7 @@ class TopicDeletionManager(controller: KafkaController, stopReplicaResponse.responseMap.filter(p => p._2 != ErrorMapping.NoError).map(_._1).toSet val replicasInError = partitionsInError.map(p => PartitionAndReplica(p.topic, p.partition, replicaId)) inLock(controllerContext.controllerLock) { - // move all the failed replicas to ReplicaDeletionFailed + // move all the failed replicas to ReplicaDeletionIneligible failReplicaDeletion(replicasInError) if(replicasInError.size != stopReplicaResponse.responseMap.size) { // some replicas could have been successfully deleted @@ -350,7 +353,7 @@ class TopicDeletionManager(controller: KafkaController, // if you come here, then no replica is in TopicDeletionStarted and all replicas are not in // TopicDeletionSuccessful. That means, there is at least one failed replica, which means topic deletion // should be retried - val replicasInTopicDeletionFailedState = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionFailed) + val replicasInTopicDeletionFailedState = controller.replicaStateMachine.replicasInState(topic, ReplicaDeletionIneligible) // mark topic for deletion retry markTopicForDeletionRetry(topic) info("Retrying delete topic for topic %s since replicas %s were not successfully deleted" @@ -362,8 +365,8 @@ class TopicDeletionManager(controller: KafkaController, info("Deletion of topic %s (re)started".format(topic)) // topic deletion will be kicked off onTopicDeletion(Set(topic)) - } else if(isTopicDeletionHalted(topic)) { - info("Not retrying deletion of topic %s at this time since it is halted".format(topic)) + } else if(isTopicIneligibleForDeletion(topic)) { + info("Not retrying deletion of topic %s at this time since it is marked ineligible for deletion".format(topic)) } } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c56ad503d83e3..ae2df2014a08a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -145,7 +145,7 @@ class KafkaApis(val requestChannel: RequestChannel, } // remove the topics that don't exist in the UpdateMetadata request since those are the topics that are // currently being deleted by the controller - val topicsKnownToThisBroker = metadataCache.map{ + val topicsKnownToThisBroker = metadataCache.map { case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic }.toSet val topicsKnownToTheController = updateMetadataRequest.partitionStateInfos.map { case(topicAndPartition, partitionStateInfo) => topicAndPartition.topic }.toSet @@ -568,6 +568,7 @@ class KafkaApis(val requestChannel: RequestChannel, partitionMetadataLock synchronized { uniqueTopics.map { topic => if(metadataCache.keySet.map(_.topic).contains(topic)) { + debug("Topic %s exists in metadata cache on broker %d".format(topic, config.brokerId)) val partitionStateInfo = metadataCache.filter(p => p._1.topic.equals(topic)) val sortedPartitions = partitionStateInfo.toList.sortWith((m1,m2) => m1._1.partition < m2._1.partition) val partitionMetadata = sortedPartitions.map { case(topicAndPartition, partitionState) => @@ -600,6 +601,7 @@ class KafkaApis(val requestChannel: RequestChannel, } new TopicMetadata(topic, partitionMetadata) } else { + debug("Topic %s does not exist in metadata cache on broker %d".format(topic, config.brokerId)) // topic doesn't exist, send appropriate error code new TopicMetadata(topic, Seq.empty[PartitionMetadata], ErrorMapping.UnknownTopicOrPartitionCode) } @@ -621,6 +623,7 @@ class KafkaApis(val requestChannel: RequestChannel, } topicsMetadata += new TopicMetadata(topicMetadata.topic, topicMetadata.partitionsMetadata, ErrorMapping.LeaderNotAvailableCode) } else { + debug("Auto create topic skipped for %s".format(topicMetadata.topic)) topicsMetadata += topicMetadata } case _ => diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index 974b057a88056..dbe078c0a3f87 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -296,9 +296,8 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { def testDeleteTopicDuringAddPartition() { val topic = "test" val servers = createTestTopicAndCluster(topic) - // add partitions to topic - val topicAndPartition = TopicAndPartition(topic, 0) val newPartition = TopicAndPartition(topic, 1) + // add partitions to topic AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") // start topic deletion AdminUtils.deleteTopic(zkClient, topic) @@ -366,6 +365,66 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { servers.foreach(_.shutdown()) } + @Test + def testAutoCreateAfterDeleteTopic() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + // start topic deletion + AdminUtils.deleteTopic(zkClient, topic) + verifyTopicDeletion(topic, servers) + // test if first produce request after topic deletion auto creates the topic + val props = new Properties() + props.put("metadata.broker.list", servers.map(s => s.config.hostName + ":" + s.config.port).mkString(",")) + props.put("serializer.class", "kafka.serializer.StringEncoder") + props.put("producer.type", "sync") + props.put("request.required.acks", "1") + props.put("message.send.max.retries", "1") + val producerConfig = new ProducerConfig(props) + val producer = new Producer[String, String](producerConfig) + try{ + producer.send(new KeyedMessage[String, String](topic, "test", "test1")) + } catch { + case e: FailedToSendMessageException => fail("Topic should have been auto created") + case oe: Throwable => fail("fails with exception", oe) + } + // test the topic path exists + assertTrue("Topic not auto created", ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) + // wait until leader is elected + val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined) + try { + producer.send(new KeyedMessage[String, String](topic, "test", "test1")) + } catch { + case e: FailedToSendMessageException => fail("Topic should have been auto created") + case oe: Throwable => fail("fails with exception", oe) + } finally { + producer.close() + } + servers.foreach(_.shutdown()) + } + + @Test + def testDeleteNonExistingTopic() { + val topicAndPartition = TopicAndPartition("test", 0) + val topic = topicAndPartition.topic + val servers = createTestTopicAndCluster(topic) + // start topic deletion + AdminUtils.deleteTopic(zkClient, "test2") + // verify delete topic path for test2 is removed from zookeeper + verifyTopicDeletion("test2", servers) + // verify that topic test is untouched + assertTrue("Replicas for topic test not created in 1000ms", TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => + res && server.getLogManager().getLog(topicAndPartition).isDefined), 1000)) + // test the topic path exists + assertTrue("Topic test mistakenly deleted", ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) + // topic test should have a leader + val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) + assertTrue("Leader should exist for topic test", leaderIdOpt.isDefined) + servers.foreach(_.shutdown()) + + } + private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topicAndPartition = TopicAndPartition(topic, 0) From 8014b6ff694d07956da10135508d98c625f3da3d Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Sun, 9 Feb 2014 09:16:06 -0500 Subject: [PATCH 012/567] KAFKA-1158 run rat is not needed this is documented now in the release not part of the server running --- bin/run-rat.sh | 35 ----------------------------------- 1 file changed, 35 deletions(-) delete mode 100755 bin/run-rat.sh diff --git a/bin/run-rat.sh b/bin/run-rat.sh deleted file mode 100755 index 1b7bc312e8b42..0000000000000 --- a/bin/run-rat.sh +++ /dev/null @@ -1,35 +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. - -base_dir=$(dirname $0)/.. -rat_excludes_file=$base_dir/.rat-excludes - -if [ -z "$JAVA_HOME" ]; then - JAVA="java" -else - JAVA="$JAVA_HOME/bin/java" -fi - -rat_command="$JAVA -jar $base_dir/lib/apache-rat-0.8.jar --dir $base_dir " - -for f in $(cat $rat_excludes_file); -do - rat_command="${rat_command} -e $f" -done - -echo "Running " $rat_command -$rat_command > $base_dir/rat.out - From 962b5475cbccfba7dbc424915fe517123f5e5203 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Thu, 6 Feb 2014 20:21:47 -0800 Subject: [PATCH 013/567] KAFKA-1241 Better error message for underflow on read of struct. --- .../org/apache/kafka/common/protocol/types/Schema.java | 10 ++++++++-- .../org/apache/kafka/common/protocol/types/Type.java | 1 - 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java index c10d9ef28db01..68b8827f3bdd6 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java @@ -62,8 +62,14 @@ public void write(ByteBuffer buffer, Object o) { */ public Object read(ByteBuffer buffer) { Object[] objects = new Object[fields.length]; - for (int i = 0; i < fields.length; i++) - objects[i] = fields[i].type.read(buffer); + for (int i = 0; i < fields.length; i++) { + try { + objects[i] = fields[i].type.read(buffer); + } catch (Exception e) { + throw new SchemaException("Error reading field '" + fields[i].name + "': " + e.getMessage() == null ? e.getMessage() + : e.getClass().getName()); + } + } return new Struct(this, objects); } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java index 1bd8ce4dfb7c5..f0d5a8286380d 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.utils.Utils; - /** * A serializable type */ From d2ec321a03e654552ee364f6572437f233b0c226 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Fri, 7 Feb 2014 09:08:33 -0800 Subject: [PATCH 014/567] KAFKA-1238 Move metadata req/resp parsing into its own classes and avoid updating cluster metadata if there are no available nodes. --- .../clients/producer/internals/Sender.java | 25 +++--- .../kafka/common/protocol/ProtoUtils.java | 49 ------------ .../common/requests/MetadataRequest.java | 25 ++++++ .../common/requests/MetadataResponse.java | 77 +++++++++++++++++++ 4 files changed, 117 insertions(+), 59 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index b274e5e4376b6..3e10e32d1a932 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -39,12 +39,13 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.utils.Time; - /** * The background thread that handles the sending of produce requests to the Kafka cluster. This thread makes metadata * requests to renew its view of the cluster and then sends produce requests to the appropriate nodes. @@ -300,8 +301,12 @@ else if (req.request.header().apiKey() == ApiKeys.METADATA.id) private void handleMetadataResponse(Struct body, long now) { this.metadataFetchInProgress = false; - Cluster cluster = ProtoUtils.parseMetadataResponse(body); - this.metadata.update(cluster, now); + MetadataResponse response = new MetadataResponse(body); + Cluster cluster = response.cluster(); + // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being + // created which means we will get errors and no nodes until it exists + if (cluster.nodes().size() > 0) + this.metadata.update(cluster, now); } /** @@ -338,11 +343,8 @@ private void correlate(RequestHeader requestHeader, ResponseHeader responseHeade * Create a metadata request for the given topics */ private InFlightRequest metadataRequest(int node, Set topics) { - String[] ts = new String[topics.size()]; - topics.toArray(ts); - Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id)); - body.set("topics", topics.toArray()); - RequestSend send = new RequestSend(node, new RequestHeader(ApiKeys.METADATA.id, clientId, correlation++), body); + MetadataRequest metadata = new MetadataRequest(new ArrayList(topics)); + RequestSend send = new RequestSend(node, header(ApiKeys.METADATA), metadata.toStruct()); return new InFlightRequest(true, send, null); } @@ -403,11 +405,14 @@ private InFlightRequest produceRequest(int destination, short acks, int timeout, } produce.set("topic_data", topicDatas.toArray()); - RequestHeader header = new RequestHeader(ApiKeys.PRODUCE.id, clientId, correlation++); - RequestSend send = new RequestSend(destination, header, produce); + RequestSend send = new RequestSend(destination, header(ApiKeys.PRODUCE), produce); return new InFlightRequest(acks != 0, send, batchesByPartition); } + private RequestHeader header(ApiKeys key) { + return new RequestHeader(key.id, clientId, correlation++); + } + /** * Wake up the selector associated with this send thread */ diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java index 90df5d5971a3f..c2cbbbd7f1f28 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java @@ -17,18 +17,10 @@ package org.apache.kafka.common.protocol; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; - public class ProtoUtils { private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) { @@ -70,45 +62,4 @@ public static Struct parseResponse(int apiKey, ByteBuffer buffer) { return (Struct) currentResponseSchema(apiKey).read(buffer); } - public static Cluster parseMetadataResponse(Struct response) { - Map brokers = new HashMap(); - Object[] brokerStructs = (Object[]) response.get("brokers"); - for (int i = 0; i < brokerStructs.length; i++) { - Struct broker = (Struct) brokerStructs[i]; - int nodeId = (Integer) broker.get("node_id"); - String host = (String) broker.get("host"); - int port = (Integer) broker.get("port"); - brokers.put(nodeId, new Node(nodeId, host, port)); - } - List partitions = new ArrayList(); - Object[] topicInfos = (Object[]) response.get("topic_metadata"); - for (int i = 0; i < topicInfos.length; i++) { - Struct topicInfo = (Struct) topicInfos[i]; - short topicError = topicInfo.getShort("topic_error_code"); - if (topicError == Errors.NONE.code()) { - String topic = topicInfo.getString("topic"); - Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata"); - for (int j = 0; j < partitionInfos.length; j++) { - Struct partitionInfo = (Struct) partitionInfos[j]; - short partError = partitionInfo.getShort("partition_error_code"); - if (partError == Errors.NONE.code()) { - int partition = partitionInfo.getInt("partition_id"); - int leader = partitionInfo.getInt("leader"); - Node leaderNode = leader == -1 ? null : brokers.get(leader); - Object[] replicas = (Object[]) partitionInfo.get("replicas"); - Node[] replicaNodes = new Node[replicas.length]; - for (int k = 0; k < replicas.length; k++) - replicaNodes[k] = brokers.get(replicas[k]); - Object[] isr = (Object[]) partitionInfo.get("isr"); - Node[] isrNodes = new Node[isr.length]; - for (int k = 0; k < isr.length; k++) - isrNodes[k] = brokers.get(isr[k]); - partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes)); - } - } - } - } - return new Cluster(brokers.values(), partitions); - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java new file mode 100644 index 0000000000000..91b9d64aca225 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -0,0 +1,25 @@ +package org.apache.kafka.common.requests; + +import java.util.List; + +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Struct; + +public class MetadataRequest { + + private final List topics; + + public MetadataRequest(List topics) { + this.topics = topics; + } + + public Struct toStruct() { + String[] ts = new String[topics.size()]; + topics.toArray(ts); + Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id)); + body.set("topics", topics.toArray()); + return body; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java new file mode 100644 index 0000000000000..73b700602006b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -0,0 +1,77 @@ +package org.apache.kafka.common.requests; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; + +public class MetadataResponse { + + private final Cluster cluster; + private final Map errors; + + public MetadataResponse(Cluster cluster, Map errors) { + this.cluster = cluster; + this.errors = errors; + } + + public MetadataResponse(Struct struct) { + Map errors = new HashMap(); + Map brokers = new HashMap(); + Object[] brokerStructs = (Object[]) struct.get("brokers"); + for (int i = 0; i < brokerStructs.length; i++) { + Struct broker = (Struct) brokerStructs[i]; + int nodeId = (Integer) broker.get("node_id"); + String host = (String) broker.get("host"); + int port = (Integer) broker.get("port"); + brokers.put(nodeId, new Node(nodeId, host, port)); + } + List partitions = new ArrayList(); + Object[] topicInfos = (Object[]) struct.get("topic_metadata"); + for (int i = 0; i < topicInfos.length; i++) { + Struct topicInfo = (Struct) topicInfos[i]; + short topicError = topicInfo.getShort("topic_error_code"); + String topic = topicInfo.getString("topic"); + if (topicError == Errors.NONE.code()) { + Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata"); + for (int j = 0; j < partitionInfos.length; j++) { + Struct partitionInfo = (Struct) partitionInfos[j]; + short partError = partitionInfo.getShort("partition_error_code"); + if (partError == Errors.NONE.code()) { + int partition = partitionInfo.getInt("partition_id"); + int leader = partitionInfo.getInt("leader"); + Node leaderNode = leader == -1 ? null : brokers.get(leader); + Object[] replicas = (Object[]) partitionInfo.get("replicas"); + Node[] replicaNodes = new Node[replicas.length]; + for (int k = 0; k < replicas.length; k++) + replicaNodes[k] = brokers.get(replicas[k]); + Object[] isr = (Object[]) partitionInfo.get("isr"); + Node[] isrNodes = new Node[isr.length]; + for (int k = 0; k < isr.length; k++) + isrNodes[k] = brokers.get(isr[k]); + partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes)); + } + } + } else { + errors.put(topic, Errors.forCode(topicError)); + } + } + this.errors = errors; + this.cluster = new Cluster(brokers.values(), partitions); + } + + public Map errors() { + return this.errors; + } + + public Cluster cluster() { + return this.cluster; + } + +} From 02bb382621ea5079ac77094928d12a129db251c0 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Sun, 9 Feb 2014 15:05:04 -0800 Subject: [PATCH 015/567] Fix bug in ByteBufferSend that lead to incorrect complete() method. --- .../java/org/apache/kafka/common/network/ByteBufferSend.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java index a8e0413681cd1..9305b61ddeaa2 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java @@ -45,7 +45,7 @@ public int destination() { @Override public boolean complete() { - return remaining > 0; + return remaining <= 0; } @Override From 6b80dbb97b3d9b533f60d5c09639ea6c7c5f2be5 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Sun, 9 Feb 2014 15:16:51 -0800 Subject: [PATCH 016/567] KAFKA-1236 Fix various breakages in the perf tests. Make the producer test use either the old or the new producer. --- bin/kafka-run-class.sh | 8 +- .../clients/producer/internals/Sender.java | 27 +-- .../clients/tools/ProducerPerformance.java | 19 +- .../other/kafka/TestEndToEndLatency.scala | 22 +- .../kafka/perf/ConsumerPerformance.scala | 105 ++++----- .../kafka/perf/ProducerPerformance.scala | 216 ++++++++++-------- project/Build.scala | 2 +- 7 files changed, 213 insertions(+), 186 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 75a3fc42a2e41..e0c31ee9b4997 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -50,22 +50,22 @@ do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/clients/build/libs//kafka-clients*.jar; +for file in $base_dir/examples/build/libs//kafka-examples*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/examples/build/libs//kafka-examples*.jar; +for file in $base_dir/contrib/hadoop-consumer/build/libs//kafka-hadoop-consumer*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/contrib/hadoop-consumer/build/libs//kafka-hadoop-consumer*.jar; +for file in $base_dir/contrib/hadoop-producer/build/libs//kafka-hadoop-producer*.jar; do CLASSPATH=$CLASSPATH:$file done -for file in $base_dir/contrib/hadoop-producer/build/libs//kafka-hadoop-producer*.jar; +for file in $base_dir/clients/target/scala-${SCALA_VERSION}/clients*.jar; do CLASSPATH=$CLASSPATH:$file done diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 3e10e32d1a932..e8c194c8cdee5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer.internals; @@ -152,7 +148,7 @@ public int run(long now) { // do the I/O try { - this.selector.poll(5L, sends); + this.selector.poll(100L, sends); } catch (IOException e) { e.printStackTrace(); } @@ -189,6 +185,7 @@ private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) { public void initiateClose() { this.running = false; this.accumulator.close(); + this.wakeup(); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index 108d61e6dba6d..3ebbb804242be 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -26,22 +26,25 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.record.Records; - public class ProducerPerformance { public static void main(String[] args) throws Exception { - if (args.length != 3) { - System.err.println("USAGE: java " + ProducerPerformance.class.getName() + " url num_records record_size"); + if (args.length != 5) { + System.err.println("USAGE: java " + ProducerPerformance.class.getName() + " url topic_name num_records record_size acks"); System.exit(1); } String url = args[0]; - int numRecords = Integer.parseInt(args[1]); - int recordSize = Integer.parseInt(args[2]); + String topicName = args[1]; + int numRecords = Integer.parseInt(args[2]); + int recordSize = Integer.parseInt(args[3]); + int acks = Integer.parseInt(args[4]); Properties props = new Properties(); - props.setProperty(ProducerConfig.REQUIRED_ACKS_CONFIG, "1"); + props.setProperty(ProducerConfig.REQUIRED_ACKS_CONFIG, Integer.toString(acks)); props.setProperty(ProducerConfig.BROKER_LIST_CONFIG, url); props.setProperty(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, Integer.toString(5 * 1000)); props.setProperty(ProducerConfig.REQUEST_TIMEOUT_CONFIG, Integer.toString(Integer.MAX_VALUE)); + props.setProperty(ProducerConfig.TOTAL_BUFFER_MEMORY_CONFIG, Integer.toString(256 * 1024 * 1024)); + props.setProperty(ProducerConfig.MAX_PARTITION_SIZE_CONFIG, Integer.toString(256 * 1024)); KafkaProducer producer = new KafkaProducer(props); Callback callback = new Callback() { @@ -52,7 +55,7 @@ public void onCompletion(RecordMetadata metadata, Exception e) { }; byte[] payload = new byte[recordSize]; Arrays.fill(payload, (byte) 1); - ProducerRecord record = new ProducerRecord("test", payload); + ProducerRecord record = new ProducerRecord(topicName, payload); long start = System.currentTimeMillis(); long maxLatency = -1L; long totalLatency = 0; @@ -75,8 +78,8 @@ public void onCompletion(RecordMetadata metadata, Exception e) { long ellapsed = System.currentTimeMillis() - start; double msgsSec = 1000.0 * numRecords / (double) ellapsed; double mbSec = msgsSec * (recordSize + Records.LOG_OVERHEAD) / (1024.0 * 1024.0); - System.out.printf("%d records sent in %d ms ms. %.2f records per second (%.2f mb/sec).", numRecords, ellapsed, msgsSec, mbSec); producer.close(); + System.out.printf("%d records sent in %d ms ms. %.2f records per second (%.2f mb/sec).\n", numRecords, ellapsed, msgsSec, mbSec); } } diff --git a/core/src/test/scala/other/kafka/TestEndToEndLatency.scala b/core/src/test/scala/other/kafka/TestEndToEndLatency.scala index c4aed10f50ca5..f5d39dd71ad25 100644 --- a/core/src/test/scala/other/kafka/TestEndToEndLatency.scala +++ b/core/src/test/scala/other/kafka/TestEndToEndLatency.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 @@ -24,23 +24,23 @@ import kafka.message._ object TestEndToEndLatency { def main(args: Array[String]) { - if(args.length != 3) { - System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect num_messages") + if (args.length != 4) { + System.err.println("USAGE: java " + getClass().getName + " broker_list zookeeper_connect topic num_messages") System.exit(1) } val brokerList = args(0) val zkConnect = args(1) - val numMessages = args(2).toInt - val topic = "test" - + val topic = args(2) + val numMessages = args(3).toInt + val consumerProps = new Properties() consumerProps.put("group.id", topic) consumerProps.put("auto.commit", "true") consumerProps.put("auto.offset.reset", "largest") consumerProps.put("zookeeper.connect", zkConnect) consumerProps.put("socket.timeout.ms", 1201000.toString) - + val config = new ConsumerConfig(consumerProps) val connector = Consumer.create(config) var stream = connector.createMessageStreams(Map(topic -> 1)).get(topic).head.head @@ -50,16 +50,16 @@ object TestEndToEndLatency { producerProps.put("metadata.broker.list", brokerList) producerProps.put("producer.type", "sync") val producer = new Producer[Any, Any](new ProducerConfig(producerProps)) - - val message = new Message("hello there beautiful".getBytes) + + val message = "hello there beautiful".getBytes var totalTime = 0.0 - for(i <- 0 until numMessages) { + for (i <- 0 until numMessages) { var begin = System.nanoTime producer.send(new KeyedMessage(topic, message)) val received = iter.next val elapsed = System.nanoTime - begin // poor man's progress bar - if(i % 10000 == 0) + if (i % 1000 == 0) println(i + "\t" + elapsed / 1000.0 / 1000.0) totalTime += elapsed } diff --git a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala b/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala index 55ee01b40afa4..4dde4687bada1 100644 --- a/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/ConsumerPerformance.scala @@ -23,7 +23,7 @@ import java.nio.channels.ClosedByInterruptException import org.apache.log4j.Logger import kafka.message.Message import kafka.utils.ZkUtils -import java.util.{Random, Properties} +import java.util.{ Random, Properties } import kafka.consumer._ import java.text.SimpleDateFormat @@ -40,8 +40,8 @@ object ConsumerPerformance { var totalMessagesRead = new AtomicLong(0) var totalBytesRead = new AtomicLong(0) - if(!config.hideHeader) { - if(!config.showDetailedStats) + if (!config.hideHeader) { + if (!config.showDetailedStats) println("start.time, end.time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") else println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") @@ -57,7 +57,7 @@ object ConsumerPerformance { for ((topic, streamList) <- topicMessageStreams) for (i <- 0 until streamList.length) threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, - totalMessagesRead, totalBytesRead) + totalMessagesRead, totalBytesRead) logger.info("Sleeping for 1 second.") Thread.sleep(1000) @@ -67,61 +67,61 @@ object ConsumerPerformance { thread.start for (thread <- threadList) - thread.shutdown + thread.join val endMs = System.currentTimeMillis val elapsedSecs = (endMs - startMs - config.consumerConfig.consumerTimeoutMs) / 1000.0 - if(!config.showDetailedStats) { - val totalMBRead = (totalBytesRead.get*1.0)/(1024*1024) + if (!config.showDetailedStats) { + val totalMBRead = (totalBytesRead.get * 1.0) / (1024 * 1024) println(("%s, %s, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs), - config.consumerConfig.fetchMessageMaxBytes, totalMBRead, totalMBRead/elapsedSecs, totalMessagesRead.get, - totalMessagesRead.get/elapsedSecs)) + config.consumerConfig.fetchMessageMaxBytes, totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, + totalMessagesRead.get / elapsedSecs)) } System.exit(0) } class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { 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") - .ofType(classOf[String]) + "Multiple URLS can be given to allow fail-over.") + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") .withRequiredArg .describedAs("topic") .ofType(classOf[String]) val groupIdOpt = parser.accepts("group", "The group id to consume on.") - .withRequiredArg - .describedAs("gid") - .defaultsTo("perf-consumer-" + new Random().nextInt(100000)) - .ofType(classOf[String]) + .withRequiredArg + .describedAs("gid") + .defaultsTo("perf-consumer-" + new Random().nextInt(100000)) + .ofType(classOf[String]) val fetchSizeOpt = parser.accepts("fetch-size", "The amount of data to fetch in a single request.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1024 * 1024) + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1024 * 1024) val resetBeginningOffsetOpt = parser.accepts("from-latest", "If the consumer does not already have an established " + "offset to consume from, start with the latest message present in the log rather than the earliest message.") val socketBufferSizeOpt = parser.accepts("socket-buffer-size", "The size of the tcp RECV size.") - .withRequiredArg - .describedAs("size") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(2 * 1024 * 1024) + .withRequiredArg + .describedAs("size") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(2 * 1024 * 1024) val numThreadsOpt = parser.accepts("threads", "Number of processing threads.") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(10) + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(10) val numFetchersOpt = parser.accepts("num-fetch-threads", "Number of fetcher threads.") - .withRequiredArg - .describedAs("count") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) + .withRequiredArg + .describedAs("count") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) - val options = parser.parse(args : _*) + val options = parser.parse(args: _*) - for(arg <- List(topicOpt, zkConnectOpt)) { - if(!options.has(arg)) { + for (arg <- List(topicOpt, zkConnectOpt)) { + if (!options.has(arg)) { System.err.println("Missing required argument \"" + arg + "\"") parser.printHelpOn(System.err) System.exit(1) @@ -132,7 +132,7 @@ object ConsumerPerformance { props.put("group.id", options.valueOf(groupIdOpt)) props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) - props.put("auto.offset.reset", if(options.has(resetBeginningOffsetOpt)) "largest" else "smallest") + props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest") props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) props.put("consumer.timeout.ms", "5000") props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString) @@ -147,13 +147,8 @@ object ConsumerPerformance { } class ConsumerPerfThread(threadId: Int, name: String, stream: KafkaStream[Array[Byte], Array[Byte]], - config:ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) + config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) extends Thread(name) { - private val shutdownLatch = new CountDownLatch(1) - - def shutdown(): Unit = { - shutdownLatch.await - } override def run() { var bytesRead = 0L @@ -164,43 +159,41 @@ object ConsumerPerformance { var lastMessagesRead = 0L try { - for (messageAndMetadata <- stream if messagesRead < config.numMessages) { + val iter = stream.iterator + while (iter.hasNext && messagesRead < config.numMessages) { + val messageAndMetadata = iter.next messagesRead += 1 bytesRead += messageAndMetadata.message.length if (messagesRead % config.reportingInterval == 0) { - if(config.showDetailedStats) + if (config.showDetailedStats) printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis) lastReportTime = System.currentTimeMillis lastMessagesRead = messagesRead lastBytesRead = bytesRead } } - } - catch { + } catch { case _: InterruptedException => case _: ClosedByInterruptException => case _: ConsumerTimeoutException => - case e: Throwable => throw e + case e: Throwable => e.printStackTrace() } totalMessagesRead.addAndGet(messagesRead) totalBytesRead.addAndGet(bytesRead) - if(config.showDetailedStats) + if (config.showDetailedStats) printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis) - shutdownComplete } private def printMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long, - startMs: Long, endMs: Long) = { + startMs: Long, endMs: Long) = { val elapsedMs = endMs - startMs - val totalMBRead = (bytesRead*1.0)/(1024*1024) - val mbRead = ((bytesRead - lastBytesRead)*1.0)/(1024*1024) + val totalMBRead = (bytesRead * 1.0) / (1024 * 1024) + val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024) println(("%s, %d, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(endMs), id, config.consumerConfig.fetchMessageMaxBytes, totalMBRead, - 1000.0*(mbRead/elapsedMs), messagesRead, ((messagesRead - lastMessagesRead)/elapsedMs)*1000.0)) + 1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0)) } - - private def shutdownComplete() = shutdownLatch.countDown } } diff --git a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala b/perf/src/main/scala/kafka/perf/ProducerPerformance.scala index ad2ac26411de8..5d399d94eafad 100644 --- a/perf/src/main/scala/kafka/perf/ProducerPerformance.scala +++ b/perf/src/main/scala/kafka/perf/ProducerPerformance.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,18 +17,19 @@ package kafka.perf -import java.util.concurrent.{CountDownLatch, Executors} +import java.util.concurrent.{ CountDownLatch, Executors } import java.util.concurrent.atomic.AtomicLong import kafka.producer._ import org.apache.log4j.Logger -import kafka.message.{CompressionCodec, Message} +import kafka.message.{ CompressionCodec, Message } import java.text.SimpleDateFormat import kafka.serializer._ import java.util._ import collection.immutable.List -import kafka.utils.{VerifiableProperties, Logging} +import kafka.utils.{ VerifiableProperties, Logging, Utils } import kafka.metrics.KafkaMetricsReporter - +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.clients.producer.ProducerRecord /** * Load test for the producer @@ -39,7 +40,7 @@ object ProducerPerformance extends Logging { val logger = Logger.getLogger(getClass) val config = new ProducerPerfConfig(args) - if(!config.isFixSize) + if (!config.isFixedSize) logger.info("WARN: Throughput will be slower due to changing message size per request") val totalBytesSent = new AtomicLong(0) @@ -49,79 +50,80 @@ object ProducerPerformance extends Logging { val startMs = System.currentTimeMillis val rand = new java.util.Random - if(!config.hideHeader) - println("start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, " + - "total.data.sent.in.nMsg, nMsg.sec") + if (!config.hideHeader) + println("start.time, end.time, compression, message.size, batch.size, total.data.sent.in.MB, MB.sec, " + + "total.data.sent.in.nMsg, nMsg.sec") - for(i <- 0 until config.numThreads) { + for (i <- 0 until config.numThreads) { executor.execute(new ProducerThread(i, config, totalBytesSent, totalMessagesSent, allDone, rand)) } allDone.await() val endMs = System.currentTimeMillis val elapsedSecs = (endMs - startMs) / 1000.0 - val totalMBSent = (totalBytesSent.get * 1.0)/ (1024 * 1024) + val totalMBSent = (totalBytesSent.get * 1.0) / (1024 * 1024) println(("%s, %s, %d, %d, %d, %.2f, %.4f, %d, %.4f").format( config.dateFormat.format(startMs), config.dateFormat.format(endMs), config.compressionCodec.codec, config.messageSize, config.batchSize, totalMBSent, - totalMBSent/elapsedSecs, totalMessagesSent.get, totalMessagesSent.get/elapsedSecs)) + totalMBSent / elapsedSecs, totalMessagesSent.get, totalMessagesSent.get / elapsedSecs)) System.exit(0) } class ProducerPerfConfig(args: Array[String]) extends PerfConfig(args) { val brokerListOpt = parser.accepts("broker-list", "REQUIRED: broker info (the list of broker host and port for bootstrap.") - .withRequiredArg - .describedAs("hostname:port,..,hostname:port") - .ofType(classOf[String]) + .withRequiredArg + .describedAs("hostname:port,..,hostname:port") + .ofType(classOf[String]) val topicsOpt = parser.accepts("topics", "REQUIRED: The comma separated list of topics to produce to") .withRequiredArg .describedAs("topic1,topic2..") .ofType(classOf[String]) val producerRequestTimeoutMsOpt = parser.accepts("request-timeout-ms", "The produce request timeout in ms") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(3000) + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3000) val producerNumRetriesOpt = parser.accepts("producer-num-retries", "The producer retries number") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(3) + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(3) val producerRetryBackOffMsOpt = parser.accepts("producer-retry-backoff-ms", "The producer retry backoff time in milliseconds") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(100) + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(100) val producerRequestRequiredAcksOpt = parser.accepts("request-num-acks", "Number of acks required for producer request " + - "to complete") - .withRequiredArg() - .ofType(classOf[java.lang.Integer]) - .defaultsTo(-1) + "to complete") + .withRequiredArg() + .ofType(classOf[java.lang.Integer]) + .defaultsTo(-1) val varyMessageSizeOpt = parser.accepts("vary-message-size", "If set, message size will vary up to the given maximum.") val syncOpt = parser.accepts("sync", "If set, messages are sent synchronously.") val numThreadsOpt = parser.accepts("threads", "Number of sending threads.") - .withRequiredArg - .describedAs("number of threads") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(1) + .withRequiredArg + .describedAs("number of threads") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(1) val initialMessageIdOpt = parser.accepts("initial-message-id", "The is used for generating test data, If set, messages will be tagged with an " + - "ID and sent by producer starting from this ID sequentially. Message content will be String type and " + - "in the form of 'Message:000...1:xxx...'") - .withRequiredArg() - .describedAs("initial message id") - .ofType(classOf[java.lang.Integer]) + "ID and sent by producer starting from this ID sequentially. Message content will be String type and " + + "in the form of 'Message:000...1:xxx...'") + .withRequiredArg() + .describedAs("initial message id") + .ofType(classOf[java.lang.Integer]) val messageSendGapMsOpt = parser.accepts("message-send-gap-ms", "If set, the send thread will wait for specified time between two sends") - .withRequiredArg() - .describedAs("message send time gap") - .ofType(classOf[java.lang.Integer]) - .defaultsTo(0) + .withRequiredArg() + .describedAs("message send time gap") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(0) val csvMetricsReporterEnabledOpt = parser.accepts("csv-reporter-enabled", "If set, the CSV metrics reporter will be enabled") val metricsDirectoryOpt = parser.accepts("metrics-dir", "If csv-reporter-enable is set, and this parameter is" + - "set, the csv metrics will be outputed here") + "set, the csv metrics will be outputed here") .withRequiredArg .describedAs("metrics dictory") .ofType(classOf[java.lang.String]) + val useNewProducerOpt = parser.accepts("new-producer", "Use the new producer implementation.") - val options = parser.parse(args : _*) - for(arg <- List(topicsOpt, brokerListOpt, numMessagesOpt)) { - if(!options.has(arg)) { + val options = parser.parse(args: _*) + for (arg <- List(topicsOpt, brokerListOpt, numMessagesOpt)) { + if (!options.has(arg)) { System.err.println("Missing required argument \"" + arg + "\"") parser.printHelpOn(System.err) System.exit(1) @@ -135,19 +137,20 @@ object ProducerPerformance extends Logging { val hideHeader = options.has(hideHeaderOpt) val brokerList = options.valueOf(brokerListOpt) val messageSize = options.valueOf(messageSizeOpt).intValue - var isFixSize = !options.has(varyMessageSizeOpt) + var isFixedSize = !options.has(varyMessageSizeOpt) var isSync = options.has(syncOpt) var batchSize = options.valueOf(batchSizeOpt).intValue var numThreads = options.valueOf(numThreadsOpt).intValue val compressionCodec = CompressionCodec.getCompressionCodec(options.valueOf(compressionCodecOpt).intValue) val seqIdMode = options.has(initialMessageIdOpt) var initialMessageId: Int = 0 - if(seqIdMode) + if (seqIdMode) initialMessageId = options.valueOf(initialMessageIdOpt).intValue() val producerRequestTimeoutMs = options.valueOf(producerRequestTimeoutMsOpt).intValue() val producerRequestRequiredAcks = options.valueOf(producerRequestRequiredAcksOpt).intValue() val producerNumRetries = options.valueOf(producerNumRetriesOpt).intValue() val producerRetryBackoffMs = options.valueOf(producerRetryBackOffMsOpt).intValue() + val useNewProducer = options.has(useNewProducerOpt) val csvMetricsReporterEnabled = options.has(csvMetricsReporterEnabledOpt) @@ -167,44 +170,80 @@ object ProducerPerformance extends Logging { val messageSendGapMs = options.valueOf(messageSendGapMsOpt).intValue() } - class ProducerThread(val threadId: Int, - val config: ProducerPerfConfig, - val totalBytesSent: AtomicLong, - val totalMessagesSent: AtomicLong, - val allDone: CountDownLatch, - val rand: Random) extends Runnable { + trait Producer { + def send(topic: String, partition: Long, bytes: Array[Byte]) + def close() + } + + class OldRustyProducer(config: ProducerPerfConfig) extends Producer { val props = new Properties() props.put("metadata.broker.list", config.brokerList) props.put("compression.codec", config.compressionCodec.codec.toString) - props.put("reconnect.interval", Integer.MAX_VALUE.toString) - props.put("send.buffer.bytes", (64*1024).toString) - if(!config.isSync) { - props.put("producer.type","async") + props.put("send.buffer.bytes", (64 * 1024).toString) + if (!config.isSync) { + props.put("producer.type", "async") props.put("batch.num.messages", config.batchSize.toString) props.put("queue.enqueue.timeout.ms", "-1") } - props.put("client.id", "ProducerPerformance") + props.put("client.id", "perf-test") props.put("request.required.acks", config.producerRequestRequiredAcks.toString) props.put("request.timeout.ms", config.producerRequestTimeoutMs.toString) props.put("message.send.max.retries", config.producerNumRetries.toString) props.put("retry.backoff.ms", config.producerRetryBackoffMs.toString) props.put("serializer.class", classOf[DefaultEncoder].getName.toString) props.put("key.serializer.class", classOf[NullEncoder[Long]].getName.toString) + val producer = new kafka.producer.Producer[Long, Array[Byte]](new ProducerConfig(props)) - - val producerConfig = new ProducerConfig(props) - val producer = new Producer[Long, Array[Byte]](producerConfig) - val seqIdNumDigit = 10 // no. of digits for max int value + def send(topic: String, partition: Long, bytes: Array[Byte]) { + this.producer.send(new KeyedMessage[Long, Array[Byte]](topic, partition, bytes)) + } + + def close() { + this.producer.close() + } + } + + class NewShinyProducer(config: ProducerPerfConfig) extends Producer { + val props = new Properties() + props.put("metadata.broker.list", config.brokerList) + props.put("send.buffer.bytes", (64 * 1024).toString) + props.put("client.id", "perf-test") + props.put("request.required.acks", config.producerRequestRequiredAcks.toString) + props.put("request.timeout.ms", config.producerRequestTimeoutMs.toString) + val producer = new KafkaProducer(props) + + def send(topic: String, partition: Long, bytes: Array[Byte]) { + val part = partition % this.producer.partitionsFor(topic).size + this.producer.send(new ProducerRecord(topic, Utils.abs(part.toInt), null, bytes)) + } + + def close() { + this.producer.close() + } + } + + class ProducerThread(val threadId: Int, + val config: ProducerPerfConfig, + val totalBytesSent: AtomicLong, + val totalMessagesSent: AtomicLong, + val allDone: CountDownLatch, + val rand: Random) extends Runnable { + val seqIdNumDigit = 10 // no. of digits for max int value val messagesPerThread = config.numMessages / config.numThreads debug("Messages per thread = " + messagesPerThread) + val producer = + if (config.useNewProducer) + new NewShinyProducer(config) + else + new OldRustyProducer(config) // generate the sequential message ID - private val SEP = ":" // message field separator + private val SEP = ":" // message field separator private val messageIdLabel = "MessageID" - private val threadIdLabel = "ThreadID" - private val topicLabel = "Topic" - private var leftPaddedSeqId : String = "" + private val threadIdLabel = "ThreadID" + private val topicLabel = "Topic" + private var leftPaddedSeqId: String = "" private def generateMessageWithSeqId(topic: String, msgId: Long, msgSize: Int): Array[Byte] = { // Each thread gets a unique range of sequential no. for its ids. @@ -213,48 +252,43 @@ object ProducerPerformance extends Logging { // thread 1 IDs : 100 ~ 199 // thread 2 IDs : 200 ~ 299 // . . . - leftPaddedSeqId = String.format("%0"+seqIdNumDigit+"d", long2Long(msgId)) + leftPaddedSeqId = String.format("%0" + seqIdNumDigit + "d", long2Long(msgId)) - val msgHeader = topicLabel + SEP + - topic + SEP + - threadIdLabel + SEP + - threadId + SEP + - messageIdLabel + SEP + - leftPaddedSeqId + SEP + val msgHeader = topicLabel + SEP + + topic + SEP + + threadIdLabel + SEP + + threadId + SEP + + messageIdLabel + SEP + + leftPaddedSeqId + SEP - val seqMsgString = String.format("%1$-"+msgSize+"s", msgHeader).replace(' ', 'x') + val seqMsgString = String.format("%1$-" + msgSize + "s", msgHeader).replace(' ', 'x') debug(seqMsgString) return seqMsgString.getBytes() } - private def generateProducerData(topic: String, messageId: Long): (KeyedMessage[Long, Array[Byte]], Int) = { - val msgSize = if(config.isFixSize) config.messageSize else 1 + rand.nextInt(config.messageSize) - val message = - if(config.seqIdMode) { - val seqId = config.initialMessageId + (messagesPerThread * threadId) + messageId - generateMessageWithSeqId(topic, seqId, msgSize) - } else { - new Array[Byte](msgSize) - } - (new KeyedMessage[Long, Array[Byte]](topic, messageId, message), message.length) + private def generateProducerData(topic: String, messageId: Long): Array[Byte] = { + val msgSize = if (config.isFixedSize) config.messageSize else 1 + rand.nextInt(config.messageSize) + if (config.seqIdMode) { + val seqId = config.initialMessageId + (messagesPerThread * threadId) + messageId + generateMessageWithSeqId(topic, seqId, msgSize) + } else { + new Array[Byte](msgSize) + } } override def run { var bytesSent = 0L var nSends = 0 var j: Long = 0L - while(j < messagesPerThread) { + while (j < messagesPerThread) { try { config.topics.foreach( topic => { - val (producerData, bytesSent_) = generateProducerData(topic, j) - bytesSent += bytesSent_ - producer.send(producerData) + producer.send(topic, j, generateProducerData(topic, j)) nSends += 1 - if(config.messageSendGapMs > 0) + if (config.messageSendGapMs > 0) Thread.sleep(config.messageSendGapMs) - } - ) + }) } catch { case e: Exception => error("Error sending messages", e) } diff --git a/project/Build.scala b/project/Build.scala index ddcfc4176e689..12d84f849f325 100644 --- a/project/Build.scala +++ b/project/Build.scala @@ -140,7 +140,7 @@ object KafkaBuild extends Build { lazy val kafka = Project(id = "Kafka", base = file(".")).aggregate(core, examples, contrib, perf).settings((commonSettings ++ runRatTask ++ releaseTask ++ releaseZipTask ++ releaseTarTask): _*) - lazy val core = Project(id = "core", base = file("core")).settings(commonSettings: _*) + lazy val core = Project(id = "core", base = file("core")).settings(commonSettings: _*) dependsOn(clients) lazy val examples = Project(id = "java-examples", base = file("examples")).settings(commonSettings :_*) dependsOn (core) lazy val perf = Project(id = "perf", base = file("perf")).settings((Seq(name := "kafka-perf") ++ commonSettings):_*) dependsOn (core) From 71e21c6bacb75bf7b5f65dc67c921d4c7583fa12 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Mon, 10 Feb 2014 21:40:34 -0800 Subject: [PATCH 017/567] KAFKA-1237 mirror maker using 08 consumer and 09 producer; reviewed by Jay Kreps and Joel Koshy --- build.gradle | 1 + .../kafka/tools/newproducer/MirrorMaker.scala | 184 ++++++++++++++++++ 2 files changed, 185 insertions(+) create mode 100644 core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala diff --git a/build.gradle b/build.gradle index 858d297b9e8bf..d9d6e6a91e6f3 100644 --- a/build.gradle +++ b/build.gradle @@ -136,6 +136,7 @@ project(':core') { } dependencies { + compile project(':clients') compile "org.scala-lang:scala-library:$scalaVersion" compile 'org.apache.zookeeper:zookeeper:3.3.4' compile 'com.101tec:zkclient:0.3' diff --git a/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala b/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala new file mode 100644 index 0000000000000..faa07e99ccd21 --- /dev/null +++ b/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala @@ -0,0 +1,184 @@ +/** + * 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.newproducer + +import joptsimple.OptionParser +import kafka.utils.{Utils, CommandLineUtils, Logging} +import java.util.concurrent.CountDownLatch +import kafka.consumer._ +import collection.mutable.ListBuffer +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, KafkaProducer} +import java.util.concurrent.atomic.AtomicInteger + + +object MirrorMaker extends Logging { + + private var connector: ZookeeperConsumerConnector = null + private var mirroringThreads: Seq[MirrorMakerThread] = null + private var producerChannel: ProducerDataChannel = null + + 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 mirroring 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) + } + CommandLineUtils.checkRequiredArgs(parser, options, consumerConfigOpt, producerConfigOpt) + if (List(whitelistOpt, blacklistOpt).count(options.has) != 1) { + fatal("Exactly one of whitelist or blacklist is required.") + System.exit(1) + } + val filterSpec = if (options.has(whitelistOpt)) + new Whitelist(options.valueOf(whitelistOpt)) + else + new Blacklist(options.valueOf(blacklistOpt)) + val producerConfig = options.valueOf(producerConfigOpt) + val producerProps = Utils.loadProps(producerConfig) + producerProps.setProperty(ProducerConfig.BLOCK_ON_BUFFER_FULL, "true") + val consumerConfig = options.valueOf(consumerConfigOpt) + val numStreams = options.valueOf(numStreamsOpt) + producerChannel = new ProducerDataChannel() + connector = new ZookeeperConsumerConnector(new ConsumerConfig(Utils.loadProps(consumerConfig))) + var streams: Seq[KafkaStream[Array[Byte], Array[Byte]]] = null + try { + streams = connector.createMessageStreamsByFilter(filterSpec, numStreams.intValue()) + debug("%d consumer streams created".format(streams.size)) + } catch { + case t: Throwable => + fatal("Unable to create stream - shutting down mirror maker.") + connector.shutdown() + System.exit(1) + } + val streamIndex = new AtomicInteger() + streams.foreach(stream => producerChannel.addProducer(new KafkaProducer(producerProps))) + mirroringThreads = streams.map(stream => new MirrorMakerThread(stream, streamIndex.getAndIncrement)) + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + cleanShutdown() + } + }) + // start the mirroring threads + mirroringThreads.foreach(_.start) + // in case the consumer threads hit a timeout/other exception + mirroringThreads.foreach(_.awaitShutdown) + cleanShutdown() + } + + def cleanShutdown() { + if (connector != null) connector.shutdown() + if (mirroringThreads != null) mirroringThreads.foreach(_.awaitShutdown) + if (producerChannel != null) producerChannel.close() + info("Kafka mirror maker shutdown successfully") + } + + class MirrorMakerThread(stream: KafkaStream[Array[Byte], Array[Byte]], + threadId: Int) + extends Thread with Logging { + + private val shutdownLatch = new CountDownLatch(1) + private val threadName = "mirrormaker-" + threadId + this.logIdent = "[%s] ".format(threadName) + + this.setName(threadName) + + override def run() { + info("Starting mirror maker thread " + threadName) + try { + for (msgAndMetadata <- stream) { + producerChannel.send(new ProducerRecord(msgAndMetadata.topic, msgAndMetadata.key(), msgAndMetadata.message())) + } + } catch { + case e: Throwable => + fatal("Stream unexpectedly exited.", e) + } finally { + shutdownLatch.countDown() + info("Stopped thread.") + } + } + + def awaitShutdown() { + try { + shutdownLatch.await() + } catch { + case e: InterruptedException => fatal("Shutdown of thread %s interrupted. This might leak data!".format(threadName)) + } + } + } + + class ProducerDataChannel extends Logging { + val producers = new ListBuffer[KafkaProducer] + var producerIndex = 0 + + def addProducer(producer: KafkaProducer) { + producers += producer + } + + def send(producerRecord: ProducerRecord) { + if(producerRecord.key() != null) { + val producerId = Utils.abs(java.util.Arrays.hashCode(producerRecord.key())) % producers.size + trace("Send message with key %s to producer %d.".format(java.util.Arrays.toString(producerRecord.key()), producerId)) + val producer = producers(producerId) + producer.send(producerRecord) + } else { + producers(producerIndex).send(producerRecord) + producerIndex = (producerIndex + 1) % producers.size + } + } + + def close() { + producers.foreach(_.close()) + } + } +} + From a0939f412d7c5c0e294f7d1c2779ae9321338155 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 11 Feb 2014 14:25:31 -0800 Subject: [PATCH 018/567] KAFKA-1233 Integration test for the new producer; reviewed by Jay Kreps and Neha Narkhede --- build.gradle | 1 + .../test/scala/unit/kafka/utils/TestUtils.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/build.gradle b/build.gradle index d9d6e6a91e6f3..fd519d6807208 100644 --- a/build.gradle +++ b/build.gradle @@ -148,6 +148,7 @@ project(':core') { testCompile 'junit:junit:4.1' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' + testCompile project(':clients') if (scalaVersion.startsWith('2.8')) { testCompile 'org.scalatest:scalatest:1.2' } else if (scalaVersion.startsWith('2.10')) { diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 500eeca2f95d9..1c7a450651978 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -39,6 +39,7 @@ import collection.mutable.Map import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition import junit.framework.Assert +import kafka.admin.AdminUtils /** @@ -135,6 +136,20 @@ object TestUtils extends Logging { props } + /** + * Create a topic in zookeeper + */ + def createTopic(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicationFactor: Int = 1, + servers: List[KafkaServer]) : scala.collection.immutable.Map[Int, Option[Int]] = { + // create topic + AdminUtils.createTopic(zkClient, topic, numPartitions, replicationFactor) + // wait until the update metadata request for new topic reaches all servers + (0 until numPartitions).map { case i => + TestUtils.waitUntilMetadataIsPropagated(servers, topic, i, 500) + i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, i, 500) + }.toMap + } + /** * Create a test config for a consumer */ From d6303ec79e6871c901951e7dcde93d4996b52c49 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Tue, 11 Feb 2014 18:03:19 -0800 Subject: [PATCH 019/567] KAFKA-1233 Adding the new test file --- .../kafka/api/ProducerSendTest.scala | 306 ++++++++++++++++++ 1 file changed, 306 insertions(+) create mode 100644 core/src/test/scala/integration/kafka/api/ProducerSendTest.scala diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala new file mode 100644 index 0000000000000..f8ba361ccf081 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -0,0 +1,306 @@ +/** + * 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.test + +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.{ZkUtils, Utils, TestUtils, Logging} +import kafka.zk.ZooKeeperTestHarness +import kafka.admin.AdminUtils +import kafka.consumer.SimpleConsumer +import kafka.api.FetchRequestBuilder +import kafka.message.Message + +import org.apache.kafka.clients.producer._ + +import org.scalatest.junit.JUnit3Suite +import org.junit.Test +import org.junit.Assert._ + +import java.util.Properties +import java.lang.{Integer, IllegalArgumentException} +import org.apache.log4j.Logger + + +class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness { + private val brokerId1 = 0 + private val brokerId2 = 1 + private val ports = TestUtils.choosePorts(2) + private val (port1, port2) = (ports(0), ports(1)) + private var server1: KafkaServer = null + private var server2: KafkaServer = null + private var servers = List.empty[KafkaServer] + + private var consumer1: SimpleConsumer = null + private var consumer2: SimpleConsumer = null + + private val props1 = TestUtils.createBrokerConfig(brokerId1, port1) + private val props2 = TestUtils.createBrokerConfig(brokerId2, port2) + props1.put("num.partitions", "4") + props2.put("num.partitions", "4") + private val config1 = new KafkaConfig(props1) + private val config2 = new KafkaConfig(props2) + + private val topic = "topic" + private val numRecords = 100 + + override def setUp() { + super.setUp() + // set up 2 brokers with 4 partitions each + server1 = TestUtils.createServer(config1) + server2 = TestUtils.createServer(config2) + servers = List(server1,server2) + + // TODO: we need to migrate to new consumers when 0.9 is final + consumer1 = new SimpleConsumer("localhost", port1, 100, 1024*1024, "") + consumer2 = new SimpleConsumer("localhost", port2, 100, 1024*1024, "") + } + + override def tearDown() { + server1.shutdown + server2.shutdown + Utils.rm(server1.config.logDirs) + Utils.rm(server2.config.logDirs) + super.tearDown() + } + + class PrintOffsetCallback extends Callback { + def onCompletion(metadata: RecordMetadata, exception: Exception) { + if (exception != null) + fail("Send callback returns the following exception", exception) + try { + System.out.println("The message we just sent is marked as [" + metadata.partition + "] : " + metadata.offset); + } catch { + case e: Throwable => fail("Should succeed sending the message", e) + } + } + } + + /** + * testSendOffset checks the basic send API behavior + * + * 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected. + * 2. Last message of the non-blocking send should return the correct offset metadata + */ + @Test + def testSendOffset() { + val props = new Properties() + props.put(ProducerConfig.BROKER_LIST_CONFIG, TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + var producer = new KafkaProducer(props) + + val callback = new PrintOffsetCallback + + try { + // create topic + TestUtils.createTopic(zkClient, topic, 1, 2, servers) + + // send a normal record + val record0 = new ProducerRecord(topic, new Integer(0), "key".getBytes, "value".getBytes) + val response0 = producer.send(record0, callback) + assertEquals("Should have offset 0", 0L, response0.get.offset) + + // send a record with null value should be ok + val record1 = new ProducerRecord(topic, new Integer(0), "key".getBytes, null) + val response1 = producer.send(record1, callback) + assertEquals("Should have offset 1", 1L, response1.get.offset) + + // send a record with null key should be ok + val record2 = new ProducerRecord(topic, new Integer(0), null, "value".getBytes) + val response2 = producer.send(record2, callback) + assertEquals("Should have offset 2", 2L, response2.get.offset) + + // send a record with null part id should be ok + val record3 = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + val response3 = producer.send(record3, callback) + assertEquals("Should have offset 3", 3L, response3.get.offset) + + // send a record with null topic should fail + try { + val record4 = new ProducerRecord(null, new Integer(0), "key".getBytes, "value".getBytes) + val response4 = producer.send(record4, callback) + response4.wait + } catch { + case iae: IllegalArgumentException => // this is ok + case e: Throwable => fail("Only expecting IllegalArgumentException", e) + } + + // non-blocking send a list of records + for (i <- 1 to numRecords) + producer.send(record0) + + // check that all messages have been acked via offset + val response5 = producer.send(record0, callback) + assertEquals("Should have offset " + (numRecords + 4), numRecords + 4L, response5.get.offset) + + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } + + /** + * testClose checks the closing behavior + * + * 1. After close() returns, all messages should be sent with correct returned offset metadata + */ + @Test + def testClose() { + val props = new Properties() + props.put(ProducerConfig.BROKER_LIST_CONFIG, TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + var producer = new KafkaProducer(props) + + try { + // create topic + TestUtils.createTopic(zkClient, topic, 1, 2, servers) + + // non-blocking send a list of records + val record0 = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + for (i <- 1 to numRecords) + producer.send(record0) + val response0 = producer.send(record0) + + // close the producer + producer.close() + producer = null + + // check that all messages have been acked via offset, + // this also checks that messages with same key go to the same partition + assertTrue("The last message should be acked before producer is shutdown", response0.isDone) + assertEquals("Should have offset " + numRecords, numRecords.toLong, response0.get.offset) + + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } + + /** + * testSendToPartition checks the partitioning behavior + * + * 1. The default partitioner should have the correct round-robin behavior in assigning partitions + * 2. The specified partition-id should be respected + */ + @Test + def testSendToPartition() { + val props = new Properties() + props.put(ProducerConfig.BROKER_LIST_CONFIG, TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + props.put(ProducerConfig.REQUIRED_ACKS_CONFIG, "-1") + var producer = new KafkaProducer(props) + + try { + // create topic + val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers) + + // make sure leaders exist + val leader0 = leaders.get(0) + val leader1 = leaders.get(1) + assertTrue("Leader for topic new-topic partition 0 should exist", leader0.isDefined) + assertTrue("Leader for topic new-topic partition 1 should exist", leader1.isDefined) + + // case 1: use default partitioner, send 2*numRecords+2 messages with no partition-id/keys, + // they should be assigned to two partitions evenly as (1,3,5,7..) and (2,4,6,8..) + for (i <- 1 to 2 * numRecords) { + val record = new ProducerRecord(topic, null, null, ("value" + i).getBytes) + producer.send(record) + } + + // make sure both partitions have acked back + val record0 = new ProducerRecord(topic, null, null, ("value" + (2 * numRecords + 1)).getBytes) + val response0 = producer.send(record0); + assertEquals("Should have offset " + numRecords, numRecords.toLong, response0.get.offset) + val record1 = new ProducerRecord(topic, null, null, ("value" + (2 * numRecords + 2)).getBytes) + val response1 = producer.send(record1); + assertEquals("Should have offset " + numRecords, numRecords.toLong, response1.get.offset) + + // get messages from partition 0, and check they has numRecords+1 messages + val fetchResponse0 = if(leader0.get == server1.config.brokerId) { + consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) + } else { + consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) + } + val messageSet0 = fetchResponse0.messageSet(topic, 0).iterator.toBuffer + assertEquals("Should have fetched " + (numRecords + 1) + " messages", numRecords + 1, messageSet0.size) + + // if the first message gets 1, make sure the rest are (3,5,7..); + // if the first message gets 2, make sure the rest are (4,6,8..) + val startWithOne = messageSet0(0).message.equals(new Message(bytes = "value1".getBytes)) + for (i <- 1 to numRecords) { + if(startWithOne) { + assertEquals(new Message(bytes = ("value" + (i * 2 + 1)).getBytes), messageSet0(i).message) + } else { + assertEquals(new Message(bytes = ("value" + (i * 2 + 2)).getBytes), messageSet0(i).message) + } + } + + // case 2: check the specified partition id is respected by sending numRecords with partition-id 1 + // and make sure all of them end up in partition 1 + for (i <- 1 to numRecords - 1) { + val record = new ProducerRecord(topic, new Integer(1), null, ("value" + i).getBytes) + producer.send(record) + } + val record2 = new ProducerRecord(topic, new Integer(1), null, ("value" + numRecords).getBytes) + val response2 = producer.send(record2); + assertEquals("Should have offset " + 2 * numRecords, (2 * numRecords).toLong, response2.get.offset) + + // start fetching from offset numRecords+1 + val fetchResponse1 = if(leader1.get == server1.config.brokerId) { + consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 1, numRecords+1, Int.MaxValue).build()) + }else { + consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 1, numRecords+1, Int.MaxValue).build()) + } + val messageSet1 = fetchResponse1.messageSet(topic, 1).iterator.toBuffer + + assertEquals("Should have fetched " + numRecords + " messages", numRecords, messageSet1.size) + + for (i <- 0 to numRecords - 1) { + assertEquals(new Message(bytes = ("value" + (i + 1)).getBytes), messageSet1(i).message) + } + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } + + @Test + def testAutoCreateTopic() { + val props = new Properties() + props.put(ProducerConfig.BROKER_LIST_CONFIG, TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))) + var producer = new KafkaProducer(props) + + try { + // Send a message to auto-create the topic + val record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + val response = producer.send(record) + assertEquals("Should have offset 0", 0L, response.get.offset) + + // double check that the topic is created with leader elected + assertTrue("Topic should already be created with leader", TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 0).isDefined) + + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + } +} \ No newline at end of file From cdd03d19943d42b6401b255196c83ee32ef6c792 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Tue, 11 Feb 2014 19:29:35 -0800 Subject: [PATCH 020/567] KAFKA-1237 Follow up review suggestions on new mirror maker; reviewed by Guozhang Wang --- .../scala/kafka/tools/newproducer/MirrorMaker.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala b/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala index faa07e99ccd21..0d6d7024ea00b 100644 --- a/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala @@ -37,14 +37,13 @@ object MirrorMaker extends Logging { val parser = new OptionParser val consumerConfigOpt = parser.accepts("consumer.config", - "Consumer config to consume from a source cluster. " + - "You may specify multiple of these.") + "Consumer config file to consume from a source cluster.") .withRequiredArg() .describedAs("config file") .ofType(classOf[String]) val producerConfigOpt = parser.accepts("producer.config", - "Embedded producer config.") + "Embedded producer config file for target cluster.") .withRequiredArg() .describedAs("config file") .ofType(classOf[String]) @@ -158,7 +157,7 @@ object MirrorMaker extends Logging { class ProducerDataChannel extends Logging { val producers = new ListBuffer[KafkaProducer] - var producerIndex = 0 + var producerIndex = new AtomicInteger(0) def addProducer(producer: KafkaProducer) { producers += producer @@ -171,8 +170,9 @@ object MirrorMaker extends Logging { val producer = producers(producerId) producer.send(producerRecord) } else { - producers(producerIndex).send(producerRecord) - producerIndex = (producerIndex + 1) % producers.size + val producerId = producerIndex.getAndSet((producerIndex.get() + 1) % producers.size) + producers(producerId).send(producerRecord) + trace("Sent message to producer " + producerId) } } From e1845ba1d80f0ba89d01d450c066aae90b416ab1 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Tue, 11 Feb 2014 17:03:36 -0800 Subject: [PATCH 021/567] KAFKA-1257 Only send metadata requests to nodes with no in-flight requests. --- .../clients/producer/internals/Sender.java | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index e8c194c8cdee5..87dd1a6fd3acc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -126,17 +126,17 @@ public int run(long now) { // get the list of partitions with data ready to send List ready = this.accumulator.ready(now); - // prune the list of ready topics to eliminate any that we aren't ready to send yet - List sendable = processReadyPartitions(cluster, ready, now); - // should we update our metadata? - List sends = new ArrayList(sendable.size()); + List sends = new ArrayList(); InFlightRequest metadataReq = maybeMetadataRequest(cluster, now); if (metadataReq != null) { sends.add(metadataReq.request); this.inFlightRequests.add(metadataReq); } + // prune the list of ready topics to eliminate any that we aren't ready to send yet + List sendable = processReadyPartitions(cluster, ready, now); + // create produce requests List batches = this.accumulator.drain(sendable, this.maxRequestSize); List requests = collate(cluster, batches); @@ -165,7 +165,11 @@ public int run(long now) { private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) { if (this.metadataFetchInProgress || !metadata.needsUpdate(now)) return null; - Node node = cluster.nextNode(); + + Node node = nextFreeNode(cluster); + if (node == null) + return null; + NodeState state = nodeState.get(node.id()); if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) { // we don't have a connection to this node right now, make one @@ -179,6 +183,18 @@ private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) { } } + /** + * @return A node with no requests currently being sent or null if no such node exists + */ + private Node nextFreeNode(Cluster cluster) { + for (int i = 0; i < cluster.nodes().size(); i++) { + Node node = cluster.nextNode(); + if (this.inFlightRequests.canSendMore(node.id())) + return node; + } + return null; + } + /** * Start closing the sender (won't actually complete until all data is sent out) */ From ef1e30bf5b0aba1758522f8cacd2804d3bfbd4fb Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Tue, 11 Feb 2014 16:24:08 -0800 Subject: [PATCH 022/567] KAFKA-1259 Close blocked only until all messages had been sent not until all acknowledgements had been received. --- .../kafka/clients/producer/internals/Sender.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 87dd1a6fd3acc..d93a455827a67 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -101,7 +101,9 @@ public void run() { } } - // send anything left in the accumulator + // okay we stopped accepting requests but there may still be + // requests in the accumulator or waiting for acknowledgment, + // wait until these are completed. int unsent = 0; do { try { @@ -109,7 +111,7 @@ public void run() { } catch (Exception e) { e.printStackTrace(); } - } while (unsent > 0); + } while (unsent > 0 || this.inFlightRequests.totalInFlightRequests() > 0); // close all the connections this.selector.close(); @@ -534,6 +536,13 @@ public Iterable clearAll(int node) { return requests.remove(node); } } + + public int totalInFlightRequests() { + int total = 0; + for (Deque deque : this.requests.values()) + total += deque.size(); + return total; + } } } From b494cd9de6002c59b7181cefab6854b9a0c88817 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 12 Feb 2014 08:22:51 -0800 Subject: [PATCH 023/567] kafka-1244,kafka-1246,kafka-1249; various gradle issues for release; patched by Jun Rao; reviewed by Neha Narkhede --- README.md | 5 ++--- build.gradle | 37 ++++++++++++++++++++++++++++++++++--- scala.gradle | 5 +++++ settings.gradle | 1 + 4 files changed, 42 insertions(+), 6 deletions(-) create mode 100644 scala.gradle diff --git a/README.md b/README.md index 9b272b52c8b65..e3fea22fe103d 100644 --- a/README.md +++ b/README.md @@ -17,9 +17,8 @@ See our [web site](http://kafka.apache.org) for details on the project. ## Building a jar and running it ## -1. ./gradlew copyDependantLibs -2. ./gradlew jar -3. Follow instuctions in http://kafka.apache.org/documentation.html#quickstart +1. ./gradlew jar +2. Follow instuctions in http://kafka.apache.org/documentation.html#quickstart ## Running unit tests ## ./gradlew test diff --git a/build.gradle b/build.gradle index fd519d6807208..9489914db1970 100644 --- a/build.gradle +++ b/build.gradle @@ -28,6 +28,7 @@ allprojects { } apply from: file('gradle/license.gradle') +apply from: file('scala.gradle') subprojects { apply plugin: 'java' @@ -113,7 +114,7 @@ project(':core') { println "Building project 'core' with Scala version $scalaVersion" apply plugin: 'scala' - archivesBaseName = "kafka_${scalaVersion}" + archivesBaseName = "kafka_${baseScalaVersion}" def (major, minor, trivial) = scalaVersion.tokenize('.') @@ -178,7 +179,6 @@ project(':core') { tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { into "." compression = Compression.GZIP - classifier = 'dist' from(project.file("../bin")) { into "bin/" } from(project.file("../config")) { into "config/" } from '../LICENSE' @@ -186,6 +186,12 @@ project(':core') { from(configurations.runtime) { into("libs/") } from(configurations.archives.artifacts.files) { into("libs/") } } + + jar { + dependsOn 'copyDependantLibs' + from '../LICENSE' + from '../NOTICE' + } task testJar(type: Jar) { appendix = 'test' @@ -204,7 +210,7 @@ project(':perf') { println "Building project 'perf' with Scala version $scalaVersion" apply plugin: 'scala' - archivesBaseName = "kafka-perf_${scalaVersion}" + archivesBaseName = "kafka-perf_${baseScalaVersion}" dependencies { compile project(':core') @@ -213,6 +219,11 @@ project(':perf') { zinc 'com.typesafe.zinc:zinc:0.2.5' } + + jar { + from '../LICENSE' + from '../NOTICE' + } } project(':contrib:hadoop-consumer') { @@ -238,6 +249,11 @@ project(':contrib:hadoop-consumer') { compile.exclude module: 'mail' compile.exclude module: 'netty' } + + jar { + from '../LICENSE' + from '../NOTICE' + } } project(':contrib:hadoop-producer') { @@ -263,6 +279,11 @@ project(':contrib:hadoop-producer') { compile.exclude module: 'mail' compile.exclude module: 'netty' } + + jar { + from '../LICENSE' + from '../NOTICE' + } } project(':examples') { @@ -271,6 +292,11 @@ project(':examples') { dependencies { compile project(':core') } + + jar { + from '../LICENSE' + from '../NOTICE' + } } project(':clients') { @@ -280,6 +306,11 @@ project(':clients') { testCompile 'com.novocode:junit-interface:0.9' } + jar { + from '../LICENSE' + from '../NOTICE' + } + task testJar(type: Jar) { appendix = 'test' from sourceSets.test.output diff --git a/scala.gradle b/scala.gradle new file mode 100644 index 0000000000000..ebd21b870c074 --- /dev/null +++ b/scala.gradle @@ -0,0 +1,5 @@ +if (!hasProperty('scalaVersion')) { + ext.scalaVersion = '2.8.0' +} +ext.defaultScalaVersion = '2.8.0' +ext.baseScalaVersion = (scalaVersion.startsWith('2.10')) ? '2.10' : scalaVersion diff --git a/settings.gradle b/settings.gradle index 74e591a8a4afc..6041784d6f84c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -13,4 +13,5 @@ // See the License for the specific language governing permissions and // limitations under the License. +apply from: file('scala.gradle') include 'core', 'perf', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients' From 84a5803a7e5235e3fa0af7de8e40bdf77ff9affe Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 12 Feb 2014 12:50:26 -0800 Subject: [PATCH 024/567] KAFKA-1233 Follow up changes to new producer integration test; reviewed by Jay Kreps --- .../clients/producer/PartitionerTest.java | 12 ++- .../kafka/api/ProducerSendTest.scala | 75 ++++++------------- 2 files changed, 33 insertions(+), 54 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java index c78da64b5ad7b..f06e28ce21e80 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java @@ -18,6 +18,7 @@ import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import java.util.List; @@ -60,12 +61,21 @@ public void testKeyPartitionIsStable() { partitioner.partition(new ProducerRecord("test", key, "value2".getBytes()), cluster)); } + @Test + public void testRoundRobinIsStable() { + int startPart = partitioner.partition(new ProducerRecord("test", value), cluster); + for (int i = 1; i <= 100; i++) { + int partition = partitioner.partition(new ProducerRecord("test", value), cluster); + assertEquals("Should yield a different partition each call with round-robin partitioner", + partition, (startPart + i) % 2); + } + } + @Test public void testRoundRobinWithDownNode() { for (int i = 0; i < partitions.size(); i++) { int part = partitioner.partition(new ProducerRecord("test", value), cluster); assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2); - } } } diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index f8ba361ccf081..34baa8c6c7a15 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -195,8 +195,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness { /** * testSendToPartition checks the partitioning behavior * - * 1. The default partitioner should have the correct round-robin behavior in assigning partitions - * 2. The specified partition-id should be respected + * 1. The specified partition-id should be respected */ @Test def testSendToPartition() { @@ -210,68 +209,38 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness { val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers) // make sure leaders exist - val leader0 = leaders.get(0) val leader1 = leaders.get(1) - assertTrue("Leader for topic new-topic partition 0 should exist", leader0.isDefined) - assertTrue("Leader for topic new-topic partition 1 should exist", leader1.isDefined) - - // case 1: use default partitioner, send 2*numRecords+2 messages with no partition-id/keys, - // they should be assigned to two partitions evenly as (1,3,5,7..) and (2,4,6,8..) - for (i <- 1 to 2 * numRecords) { - val record = new ProducerRecord(topic, null, null, ("value" + i).getBytes) - producer.send(record) + assertTrue("Leader for topic \"topic\" partition 1 should exist", leader1.isDefined) + + val partition = 1 + val responses = + for (i <- 0 until numRecords) + yield producer.send(new ProducerRecord(topic, partition, null, ("value" + i).getBytes)) + val futures = responses.toList + futures.map(_.wait) + for (future <- futures) + assertTrue("Request should have completed", future.isDone) + + // make sure all of them end up in the same partition with increasing offset values + for ((future, offset) <- futures zip (0 until numRecords)) { + assertEquals(offset, future.get.offset) + assertEquals(topic, future.get.topic) + assertEquals(1, future.get.partition) } - // make sure both partitions have acked back - val record0 = new ProducerRecord(topic, null, null, ("value" + (2 * numRecords + 1)).getBytes) - val response0 = producer.send(record0); - assertEquals("Should have offset " + numRecords, numRecords.toLong, response0.get.offset) - val record1 = new ProducerRecord(topic, null, null, ("value" + (2 * numRecords + 2)).getBytes) - val response1 = producer.send(record1); - assertEquals("Should have offset " + numRecords, numRecords.toLong, response1.get.offset) - - // get messages from partition 0, and check they has numRecords+1 messages - val fetchResponse0 = if(leader0.get == server1.config.brokerId) { - consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) - } else { - consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build()) - } - val messageSet0 = fetchResponse0.messageSet(topic, 0).iterator.toBuffer - assertEquals("Should have fetched " + (numRecords + 1) + " messages", numRecords + 1, messageSet0.size) - - // if the first message gets 1, make sure the rest are (3,5,7..); - // if the first message gets 2, make sure the rest are (4,6,8..) - val startWithOne = messageSet0(0).message.equals(new Message(bytes = "value1".getBytes)) - for (i <- 1 to numRecords) { - if(startWithOne) { - assertEquals(new Message(bytes = ("value" + (i * 2 + 1)).getBytes), messageSet0(i).message) - } else { - assertEquals(new Message(bytes = ("value" + (i * 2 + 2)).getBytes), messageSet0(i).message) - } - } - - // case 2: check the specified partition id is respected by sending numRecords with partition-id 1 - // and make sure all of them end up in partition 1 - for (i <- 1 to numRecords - 1) { - val record = new ProducerRecord(topic, new Integer(1), null, ("value" + i).getBytes) - producer.send(record) - } - val record2 = new ProducerRecord(topic, new Integer(1), null, ("value" + numRecords).getBytes) - val response2 = producer.send(record2); - assertEquals("Should have offset " + 2 * numRecords, (2 * numRecords).toLong, response2.get.offset) - - // start fetching from offset numRecords+1 + // make sure the fetched messages also respect the partitioning and ordering val fetchResponse1 = if(leader1.get == server1.config.brokerId) { - consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 1, numRecords+1, Int.MaxValue).build()) + consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build()) }else { - consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 1, numRecords+1, Int.MaxValue).build()) + consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 1, 0, Int.MaxValue).build()) } val messageSet1 = fetchResponse1.messageSet(topic, 1).iterator.toBuffer - assertEquals("Should have fetched " + numRecords + " messages", numRecords, messageSet1.size) + // TODO: also check topic and partition after they are added in the return messageSet for (i <- 0 to numRecords - 1) { assertEquals(new Message(bytes = ("value" + (i + 1)).getBytes), messageSet1(i).message) + assertEquals(i, messageSet1(i).offset) } } finally { if (producer != null) { From 7e154a36f74ad0ea7e0f6d48b71a5a73d99330e7 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Wed, 12 Feb 2014 15:30:08 -0800 Subject: [PATCH 025/567] KAFKA-1261 Make it possible to configure the metadata refresh. --- .../kafka/clients/producer/KafkaProducer.java | 29 +++++++------- .../clients/producer/ProducerConfig.java | 38 +++++++++++-------- 2 files changed, 36 insertions(+), 31 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 4b2f55650945a..3d180e885a25f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer; @@ -49,7 +45,6 @@ import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.SystemTime; - /** * A Kafka client that publishes records to the Kafka cluster. *

@@ -94,7 +89,8 @@ private KafkaProducer(ProducerConfig config) { new SystemTime()); this.partitioner = new Partitioner(); this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); - this.metadata = new Metadata(); + this.metadata = new Metadata(config.getLong(ProducerConfig.METADATA_FETCH_BACKOFF_CONFIG), + config.getLong(ProducerConfig.METADATA_EXPIRY_CONFIG)); this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); this.totalMemorySize = config.getLong(ProducerConfig.TOTAL_BUFFER_MEMORY_CONFIG); this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.MAX_PARTITION_SIZE_CONFIG), @@ -256,6 +252,7 @@ public List partitionsFor(String topic) { */ @Override public void close() { + this.accumulator.close(); this.sender.initiateClose(); try { this.ioThread.join(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 502af5cd555dd..dca9802c8d091 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer; @@ -25,7 +21,6 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Type; - /** * The producer configuration keys */ @@ -47,6 +42,17 @@ public class ProducerConfig extends AbstractConfig { */ public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; + /** + * The minimum amount of time between metadata fetches. This prevents polling for metadata too quickly. + */ + public static final String METADATA_FETCH_BACKOFF_CONFIG = "metadata.fetch.backoff.ms"; + + /** + * The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any + * leadership changes. + */ + public static final String METADATA_EXPIRY_CONFIG = "metadata.expiry.ms"; + /** * The buffer size allocated for a partition. When records are received which are smaller than this size the * producer will attempt to optimistically group them together until this size is reached. @@ -125,6 +131,8 @@ public class ProducerConfig extends AbstractConfig { /* TODO: add docs */ config = new ConfigDef().define(BROKER_LIST_CONFIG, Type.LIST, "blah blah") .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), "blah blah") + .define(METADATA_FETCH_BACKOFF_CONFIG, Type.LONG, 50, atLeast(0), "blah blah") + .define(METADATA_EXPIRY_CONFIG, Type.LONG, 5 * 60 * 1000, atLeast(0), "blah blah") .define(MAX_PARTITION_SIZE_CONFIG, Type.INT, 16384, atLeast(0), "blah blah") .define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), "blah blah") /* TODO: should be a string to handle acks=in-sync */ From 3d830c9ef1a186cd36e4f5c7bdc2dfef2d51edc7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 13 Feb 2014 14:46:46 -0800 Subject: [PATCH 026/567] KAFKA-1188 Stale LeaderAndIsr request could be handled by the broker on Controller failover; reviewed by Neha, Jun --- .../main/scala/kafka/cluster/Partition.scala | 36 +++--- .../controller/ReplicaStateMachine.scala | 5 +- .../scala/kafka/server/ReplicaManager.scala | 117 ++++++++++-------- 3 files changed, 87 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 1087a2e91c86e..882b6da8a8e40 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -28,8 +28,10 @@ import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import org.apache.log4j.Logger import kafka.message.ByteBufferMessageSet -import kafka.common.{NotAssignedReplicaException, TopicAndPartition, NotLeaderForPartitionException, ErrorMapping} +import kafka.common.{NotAssignedReplicaException, NotLeaderForPartitionException, ErrorMapping} import java.io.IOException +import scala.Some +import kafka.common.TopicAndPartition /** @@ -190,10 +192,11 @@ class Partition(val topic: String, /** * Make the local replica the follower by setting the new leader and ISR to empty + * If the leader replica id does not change, return false to indicate the replica manager */ def makeFollower(controllerId: Int, partitionStateInfo: PartitionStateInfo, - leaders: Set[Broker], correlationId: Int): Boolean = { + correlationId: Int): Boolean = { leaderIsrUpdateLock synchronized { val allReplicas = partitionStateInfo.allReplicas val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch @@ -202,23 +205,18 @@ class Partition(val topic: String, // record the epoch of the controller that made the leadership decision. This is useful while updating the isr // to maintain the decision maker controller's epoch in the zookeeper path controllerEpoch = leaderIsrAndControllerEpoch.controllerEpoch - // TODO: Delete leaders from LeaderAndIsrRequest in 0.8.1 - leaders.find(_.id == newLeaderBrokerId) match { - case Some(leaderBroker) => - // add replicas that are new - allReplicas.foreach(r => getOrCreateReplica(r)) - // remove assigned replicas that have been removed by the controller - (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_)) - inSyncReplicas = Set.empty[Replica] - leaderEpoch = leaderAndIsr.leaderEpoch - zkVersion = leaderAndIsr.zkVersion - leaderReplicaIdOpt = Some(newLeaderBrokerId) - case None => // we should not come here - stateChangeLogger.error(("Broker %d aborted the become-follower state change with correlation id %d from " + - "controller %d epoch %d for partition [%s,%d] new leader %d") - .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, - topic, partitionId, newLeaderBrokerId)) - } + // add replicas that are new + allReplicas.foreach(r => getOrCreateReplica(r)) + // remove assigned replicas that have been removed by the controller + (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_)) + inSyncReplicas = Set.empty[Replica] + leaderEpoch = leaderAndIsr.leaderEpoch + zkVersion = leaderAndIsr.zkVersion + + if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == newLeaderBrokerId) + return false; + + leaderReplicaIdOpt = Some(newLeaderBrokerId) true } } diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 5e016d5d2bbef..37a48005b3e3c 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -231,8 +231,9 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { case Some(currLeaderIsrAndControllerEpoch) => controller.removeReplicaFromIsr(topic, partition, replicaId) match { case Some(updatedLeaderIsrAndControllerEpoch) => - // send the shrunk ISR state change request only to the leader - brokerRequestBatch.addLeaderAndIsrRequestForBrokers(List(updatedLeaderIsrAndControllerEpoch.leaderAndIsr.leader), + // send the shrunk ISR state change request to all the remaining alive replicas of the partition. + val currentAssignedReplicas = controllerContext.partitionReplicaAssignment(topicAndPartition) + brokerRequestBatch.addLeaderAndIsrRequestForBrokers(currentAssignedReplicas.filterNot(_ == replicaId), topic, partition, updatedLeaderIsrAndControllerEpoch, replicaAssignment) replicaState.put(partitionAndReplica, OfflineReplica) stateChangeLogger.trace("Controller %d epoch %d changed state of replica %d for partition %s from %s to %s" diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 21bba48affb38..3dd562ca93c44 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -16,21 +16,21 @@ */ package kafka.server -import kafka.cluster.{Broker, Partition, Replica} import collection._ import mutable.HashMap -import org.I0Itec.zkclient.ZkClient -import java.io.{File, IOException} -import java.util.concurrent.atomic.AtomicBoolean +import kafka.cluster.{Broker, Partition, Replica} import kafka.utils._ import kafka.log.LogManager import kafka.metrics.KafkaMetricsGroup -import com.yammer.metrics.core.Gauge -import java.util.concurrent.TimeUnit import kafka.common._ import kafka.api.{StopReplicaRequest, PartitionStateInfo, LeaderAndIsrRequest} import kafka.controller.KafkaController import org.apache.log4j.Logger +import org.I0Itec.zkclient.ZkClient +import com.yammer.metrics.core.Gauge +import java.util.concurrent.atomic.AtomicBoolean +import java.io.{IOException, File} +import java.util.concurrent.TimeUnit object ReplicaManager { @@ -215,9 +215,9 @@ class ReplicaManager(val config: KafkaConfig, val responseMap = new collection.mutable.HashMap[(String, Int), Short] if(leaderAndISRRequest.controllerEpoch < controllerEpoch) { leaderAndISRRequest.partitionStateInfos.foreach { case ((topic, partition), stateInfo) => - stateChangeLogger.warn(("Broker %d received LeaderAndIsr request correlation id %d with an old controller epoch %d." + - " Latest known controller epoch is %d").format(localBrokerId, leaderAndISRRequest.correlationId, - leaderAndISRRequest.controllerEpoch, controllerEpoch)) + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d since " + + "its controller epoch %d is old. Latest known controller epoch is %d").format(localBrokerId, leaderAndISRRequest.controllerId, + leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerEpoch, controllerEpoch)) } (responseMap, ErrorMapping.StaleControllerEpochCode) } else { @@ -236,17 +236,17 @@ class ReplicaManager(val config: KafkaConfig, if(partitionStateInfo.allReplicas.contains(config.brokerId)) partitionState.put(partition, partitionStateInfo) else { - stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request with correlation id %d from " + - "controller %d epoch %d as broker is not in assigned replica list %s for partition [%s,%d]") - .format(localBrokerId, correlationId, controllerId, leaderAndISRRequest.controllerEpoch, - partitionStateInfo.allReplicas.mkString(","), topic, partition.partitionId)) + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + + "epoch %d for partition [%s,%d] as itself is not in assigned replica list %s") + .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, + topic, partition.partitionId, partitionStateInfo.allReplicas.mkString(","))) } } else { // Otherwise record the error code in response - stateChangeLogger.warn(("Broker %d received invalid LeaderAndIsr request with correlation id %d from " + - "controller %d epoch %d with an older leader epoch %d for partition [%s,%d], current leader epoch is %d") - .format(localBrokerId, correlationId, controllerId, leaderAndISRRequest.controllerEpoch, - partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch, topic, partition.partitionId, partitionLeaderEpoch)) + stateChangeLogger.warn(("Broker %d ignoring LeaderAndIsr request from controller %d with correlation id %d " + + "epoch %d for partition [%s,%d] since its associated leader epoch %d is old. Current leader epoch is %d") + .format(localBrokerId, controllerId, correlationId, leaderAndISRRequest.controllerEpoch, + topic, partition.partitionId, partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch, partitionLeaderEpoch)) responseMap.put((topic, partitionId), ErrorMapping.StaleLeaderEpochCode) } } @@ -345,10 +345,11 @@ class ReplicaManager(val config: KafkaConfig, */ private def makeFollowers(controllerId: Int, epoch: Int, partitionState: Map[Partition, PartitionStateInfo], leaders: Set[Broker], correlationId: Int, responseMap: mutable.Map[(String, Int), Short]) { - partitionState.foreach(state => + partitionState.foreach { state => stateChangeLogger.trace(("Broker %d handling LeaderAndIsr request correlationId %d from controller %d epoch %d " + "starting the become-follower transition for partition %s") - .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId)))) + .format(localBrokerId, correlationId, controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) + } for (partition <- partitionState.keys) responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError) @@ -358,47 +359,63 @@ class ReplicaManager(val config: KafkaConfig, leaderPartitions --= partitionState.keySet } - partitionState.foreach{ case (partition, leaderIsrAndControllerEpoch) => - partition.makeFollower(controllerId, leaderIsrAndControllerEpoch, leaders, correlationId)} + var partitionsToMakeFollower: Set[Partition] = Set() - replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(new TopicAndPartition(_))) - partitionState.foreach { state => + // TODO: Delete leaders from LeaderAndIsrRequest in 0.8.1 + partitionState.foreach{ case (partition, partitionStateInfo) => + val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch + val newLeaderBrokerId = leaderIsrAndControllerEpoch.leaderAndIsr.leader + leaders.find(_.id == newLeaderBrokerId) match { + case Some(leaderBroker) => + if (partition.makeFollower(controllerId, partitionStateInfo, correlationId)) + partitionsToMakeFollower += partition + else + stateChangeLogger.info(("Broker %d skipped the become-follower state change after marking its partition as follower with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since the new leader %d is the same as the old leader") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, + partition.topic, partition.partitionId, newLeaderBrokerId)) + case None => + // The leader broker should always be present in the leaderAndIsrRequest. + // If not, we should record the error message and abort the transition process for this partition + stateChangeLogger.error(("Broker %d aborted the become-follower state change with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since new leader %d is not currently available") + .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, + partition.topic, partition.partitionId, newLeaderBrokerId)) + } + } + + replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(new TopicAndPartition(_))) + partitionsToMakeFollower.foreach { partition => stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-follower request from controller " + "%d epoch %d with correlation id %d for partition %s") - .format(localBrokerId, controllerId, epoch, correlationId, TopicAndPartition(state._1.topic, state._1.partitionId))) + .format(localBrokerId, controllerId, epoch, correlationId, TopicAndPartition(partition.topic, partition.partitionId))) } - logManager.truncateTo(partitionState.map{ case(partition, leaderISRAndControllerEpoch) => - new TopicAndPartition(partition) -> partition.getOrCreateReplica().highWatermark - }) - partitionState.foreach { state => - stateChangeLogger.trace(("Broker %d truncated logs and checkpointed recovery boundaries for partition %s as part of " + + logManager.truncateTo(partitionsToMakeFollower.map(partition => (new TopicAndPartition(partition), partition.getOrCreateReplica().highWatermark)).toMap) + + partitionsToMakeFollower.foreach { partition => + stateChangeLogger.trace(("Broker %d truncated logs and checkpointed recovery boundaries for partition [%s,%d] as part of " + "become-follower request with correlation id %d from controller %d epoch %d").format(localBrokerId, - TopicAndPartition(state._1.topic, state._1.partitionId), correlationId, controllerId, epoch)) + partition.topic, partition.partitionId, correlationId, controllerId, epoch)) } - if (!isShuttingDown.get()) { - val partitionAndOffsets = mutable.Map[TopicAndPartition, BrokerAndInitialOffset]() - partitionState.foreach { - case (partition, partitionStateInfo) => - val leader = partitionStateInfo.leaderIsrAndControllerEpoch.leaderAndIsr.leader - leaders.find(_.id == leader) match { - case Some(leaderBroker) => - partitionAndOffsets.put(new TopicAndPartition(partition), - BrokerAndInitialOffset(leaderBroker, partition.getReplica().get.logEndOffset)) - case None => - stateChangeLogger.trace(("Broker %d ignored the become-follower state change with correlation id %d " + - "controller %d epoch %d for partition %s since the designated leader %d " + - "cannot be found in live or shutting down brokers %s").format(localBrokerId, - correlationId, controllerId, epoch, partition, leader, leaders.mkString(","))) - } + + if (isShuttingDown.get()) { + partitionsToMakeFollower.foreach { partition => + stateChangeLogger.trace(("Broker %d skipped the adding-fetcher step of the become-follower state change with correlation id %d from " + + "controller %d epoch %d for partition [%s,%d] since it is shutting down").format(localBrokerId, correlationId, + controllerId, epoch, partition.topic, partition.partitionId)) } - replicaFetcherManager.addFetcherForPartitions(partitionAndOffsets) } else { - partitionState.foreach { state => - stateChangeLogger.trace(("Broker %d ignored the become-follower state change with correlation id %d from " + - "controller %d epoch %d for partition %s since it is shutting down").format(localBrokerId, correlationId, - controllerId, epoch, TopicAndPartition(state._1.topic, state._1.partitionId))) + // we do not need to check if the leader exists again since this has been done at the beginning of this process + val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => + new TopicAndPartition(partition) -> BrokerAndInitialOffset(leaders.find(_.id == partition.leaderReplicaIdOpt.get).get, partition.getReplica().get.logEndOffset)).toMap + replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) + + partitionsToMakeFollower.foreach { partition => + stateChangeLogger.trace(("Broker %d started fetcher to new leader as part of become-follower request from controller " + + "%d epoch %d with correlation id %d for partition [%s,%d]") + .format(localBrokerId, controllerId, epoch, correlationId, partition.topic, partition.partitionId)) } } } catch { From 5be973674fe726181fc12e7a098b46286a545361 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Thu, 13 Feb 2014 22:45:53 -0500 Subject: [PATCH 027/567] KAFKA-1263 Snazzy up the README markdown for better visibility on github; patched by Joe Stein; reviewed by Neha Narkhede --- README.md | 112 +++++++++++++++++++++++++++--------------------------- 1 file changed, 55 insertions(+), 57 deletions(-) diff --git a/README.md b/README.md index e3fea22fe103d..ec80083b089dc 100644 --- a/README.md +++ b/README.md @@ -1,80 +1,78 @@ -# 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. - -# Apache Kafka # +Apache Kafka +================= See our [web site](http://kafka.apache.org) for details on the project. -## Building a jar and running it ## -1. ./gradlew jar -2. Follow instuctions in http://kafka.apache.org/documentation.html#quickstart +### Building a jar and running it ### + ./gradlew jar -## Running unit tests ## -./gradlew test +Follow instuctions in http://kafka.apache.org/documentation.html#quickstart -## Forcing re-running unit tests w/o code change ## -./gradlew cleanTest test +### Running unit tests ### + ./gradlew test -## Running a particular unit test ## -./gradlew -Dtest.single=RequestResponseSerializationTest core:test +### Forcing re-running unit tests w/o code change ### + ./gradlew cleanTest test + +### Running a particular unit test ### + ./gradlew -Dtest.single=RequestResponseSerializationTest core:test + +### Building a binary release gzipped tar ball ### + ./gradlew clean + ./gradlew releaseTarGz -## Building a binary release gzipped tar ball ## -./gradlew clean -./gradlew releaseTarGz The release file can be found inside ./core/build/distributions/. -## Cleaning the build ## -./gradlew clean +### Cleaning the build ### + ./gradlew clean + +### Running a task on a particular version of Scala #### +either 2.8.0, 2.8.2, 2.9.1, 2.9.2 or 2.10.1) (If building a jar with a version other than 2.8.0, the scala version variable in bin/kafka-run-class.sh needs to be changed to run quick start.) + ./gradlew -PscalaVersion=2.9.1 jar + ./gradlew -PscalaVersion=2.9.1 test + ./gradlew -PscalaVersion=2.9.1 releaseTarGz + +### Running a task for a specific project ### +This is for 'core', 'perf', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples' and 'clients' + ./gradlew core:jar + ./gradlew core:test -## Running a task on a particular version of Scala (either 2.8.0, 2.8.2, 2.9.1, 2.9.2 or 2.10.1) ## -## (If building a jar with a version other than 2.8.0, the scala version variable in bin/kafka-run-class.sh needs to be changed to run quick start.) ## -./gradlew -PscalaVersion=2.9.1 jar -./gradlew -PscalaVersion=2.9.1 test -./gradlew -PscalaVersion=2.9.1 releaseTarGz +### Listing all gradle tasks ### + ./gradlew tasks -## Running a task for a specific project in 'core', 'perf', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients' ## -./gradlew core:jar -./gradlew core:test +### Building IDE project #### + ./gradlew eclipse + ./gradlew idea -## Listing all gradle tasks ## -./gradlew tasks +### Building the jar for all scala versions and for all projects ### + ./gradlew jarAll -# Building IDE project ## -./gradlew eclipse -./gradlew idea +### Running unit tests for all scala versions and for all projects ### + ./gradlew testAll -# Building the jar for all scala versions and for all projects ## -./gradlew jarAll +### Building a binary release gzipped tar ball for all scala versions ### + ./gradlew releaseTarGzAll -## Running unit tests for all scala versions and for all projects ## -./gradlew testAll +### Publishing the jar for all version of Scala and for all projects to maven ### + ./gradlew uploadArchivesAll -## Building a binary release gzipped tar ball for all scala versions ## -./gradlew releaseTarGzAll +Please note for this to work you need to create/update `~/.gradle/gradle.properties` and assign the following variables -## Publishing the jar for all version of Scala and for all projects to maven (To test locally, change mavenUrl in gradle.properties to a local dir.) ## -./gradlew uploadArchivesAll + mavenUrl= + mavenUsername= + mavenPassword= + signing.keyId= + signing.password= + signing.secretKeyRingFile= -## Building the test jar ## -./gradlew testJar +### Building the test jar ### + ./gradlew testJar -## Determining how transitive dependencies are added ## -./gradlew core:dependencies --configuration runtime +### Determining how transitive dependencies are added ### + ./gradlew core:dependencies --configuration runtime -## Contribution ## +### Contribution ### -Kafka is a new project, and we are interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). +Apache Kafka interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). To contribute follow the instructions here: * http://kafka.apache.org/contributing.html From dc5a993e676da4f7042ba1dd9efc034202d46263 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Fri, 14 Feb 2014 14:15:32 -0500 Subject: [PATCH 028/567] KAFKA-1245 the jar files and pom are not being signed so nexus is failing to publish them patch by Joe Stein; Reviewed by Jun Rao --- build.gradle | 40 +++++++++++++++++++++++----------------- gradle.properties | 4 ---- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/build.gradle b/build.gradle index 9489914db1970..58a6396f33413 100644 --- a/build.gradle +++ b/build.gradle @@ -34,25 +34,31 @@ subprojects { apply plugin: 'java' apply plugin: 'eclipse' apply plugin: 'maven' - + apply plugin: 'signing' + uploadArchives { repositories { - // To test locally, replace mavenUrl in gradle.properties to file://localhost/tmp/myRepo/ - mavenDeployer { - repository(url: "${mavenUrl}") { - authentication(userName: "${mavenUsername}", password: "${mavenPassword}") - } - afterEvaluate { - pom.artifactId = "${archivesBaseName}" - pom.project { - name 'Apache Kafka' - packaging 'jar' - url 'http://kafka.apache.org' - licenses { - license { - name 'The Apache Software License, Version 2.0' - url 'http://www.apache.org/licenses/LICENSE-2.0.txt' - distribution 'repo' + signing { + sign configurations.archives + + // To test locally, replace mavenUrl in ~/.gradle/gradle.properties to file://localhost/tmp/myRepo/ + mavenDeployer { + beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) } + repository(url: "${mavenUrl}") { + authentication(userName: "${mavenUsername}", password: "${mavenPassword}") + } + afterEvaluate { + pom.artifactId = "${archivesBaseName}" + pom.project { + name 'Apache Kafka' + packaging 'jar' + url 'http://kafka.apache.org' + licenses { + license { + name 'The Apache Software License, Version 2.0' + url 'http://www.apache.org/licenses/LICENSE-2.0.txt' + distribution 'repo' + } } } } diff --git a/gradle.properties b/gradle.properties index ad7a2f0f62930..447ee442105e2 100644 --- a/gradle.properties +++ b/gradle.properties @@ -18,7 +18,3 @@ version=0.8.1 scalaVersion=2.8.0 task=build -#mavenUrl=file://localhost/tmp/maven -mavenUrl=http://your.maven.repository -mavenUsername=your.username -mavenPassword=your.password From f550cc76cd4aa6a1ed771c477ed8cc832520ba1a Mon Sep 17 00:00:00 2001 From: Kostya Golikov Date: Mon, 17 Feb 2014 11:43:45 -0800 Subject: [PATCH 029/567] Liars in PrimitiveApiTest that promise to test api in compression mode, but don't do this actually; patched by Kostya Golikov; reviewed by Guozhang Wang and Jun Rao --- .../kafka/integration/PrimitiveApiTest.scala | 140 +++++------------- 1 file changed, 39 insertions(+), 101 deletions(-) diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index 5f331d22bc99b..d44c3ff1d3548 100644 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -35,12 +35,12 @@ import kafka.utils.{TestUtils, Utils} * End to end tests of the primitive apis against a local server */ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness { + val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) - val port = TestUtils.choosePort + val port = TestUtils.choosePort() val props = TestUtils.createBrokerConfig(0, port) val config = new KafkaConfig(props) val configs = List(config) - val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) def testFetchRequestCanProperlySerialize() { val request = new FetchRequestBuilder() @@ -100,7 +100,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with val stringProducer1 = new Producer[String, String](config) stringProducer1.send(new KeyedMessage[String, String](topic, "test-message")) - var fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) + val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build()) val messageSet = fetched.messageSet(topic, 0) assertTrue(messageSet.iterator.hasNext) @@ -108,8 +108,8 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with assertEquals("test-message", Utils.readString(fetchedMessageAndOffset.message.payload, "UTF-8")) } - def testProduceAndMultiFetch() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) + private def produceAndMultiFetch(producer: Producer[String, String]) { + createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4")) // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); @@ -171,117 +171,56 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with requestHandlerLogger.setLevel(Level.ERROR) } - def testProduceAndMultiFetchWithCompression() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) - - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - { - val messages = new mutable.HashMap[String, Seq[String]] - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) { - val messageList = List("a_" + topic, "b_" + topic) - val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) - messages += topic -> messageList - producer.send(producerData:_*) - builder.addFetch(topic, partition, 0, 10000) - } - - // wait a bit for produced message to be available - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val fetched = response.messageSet(topic, partition) - assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) - } - } - - // temporarily set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.FATAL) - - { - // send some invalid offsets - val builder = new FetchRequestBuilder() - for( (topic, partition) <- topics) - builder.addFetch(topic, partition, -1, 10000) - - try { - val request = builder.build() - val response = consumer.fetch(request) - response.data.values.foreach(pdata => ErrorMapping.maybeThrowException(pdata.error)) - fail("Expected exception when fetching message with invalid offset") - } catch { - case e: OffsetOutOfRangeException => "this is good" - } - } - - { - // send some invalid partitions - val builder = new FetchRequestBuilder() - for( (topic, _) <- topics) - builder.addFetch(topic, -1, 0, 10000) - - try { - val request = builder.build() - val response = consumer.fetch(request) - response.data.values.foreach(pdata => ErrorMapping.maybeThrowException(pdata.error)) - fail("Expected exception when fetching message with invalid partition") - } catch { - case e: UnknownTopicOrPartitionException => "this is good" - } - } + def testProduceAndMultiFetch() { + val props = producer.config.props.props + val config = new ProducerConfig(props) + val noCompressionProducer = new Producer[String, String](config) + produceAndMultiFetch(noCompressionProducer) + } - // restore set request handler logger to a higher level - requestHandlerLogger.setLevel(Level.ERROR) + def testProduceAndMultiFetchWithCompression() { + val props = producer.config.props.props + props.put("compression", "true") + val config = new ProducerConfig(props) + val producerWithCompression = new Producer[String, String](config) + produceAndMultiFetch(producerWithCompression) } - def testMultiProduce() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) + private def multiProduce(producer: Producer[String, String]) { + val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) + createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); val messages = new mutable.HashMap[String, Seq[String]] val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil - for( (topic, partition) <- topics) { + for((topic, partition) <- topics) { val messageList = List("a_" + topic, "b_" + topic) val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) messages += topic -> messageList producer.send(producerData:_*) builder.addFetch(topic, partition, 0, 10000) } - producer.send(produceList: _*) val request = builder.build() val response = consumer.fetch(request) - for( (topic, partition) <- topics) { + for((topic, partition) <- topics) { val fetched = response.messageSet(topic, partition) assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) } } - def testMultiProduceWithCompression() { - // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); - val messages = new mutable.HashMap[String, Seq[String]] - val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil - for( (topic, partition) <- topics) { - val messageList = List("a_" + topic, "b_" + topic) - val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) - messages += topic -> messageList - producer.send(producerData:_*) - builder.addFetch(topic, partition, 0, 10000) - } - producer.send(produceList: _*) + def testMultiProduce() { + val props = producer.config.props.props + val config = new ProducerConfig(props) + val noCompressionProducer = new Producer[String, String](config) + multiProduce(noCompressionProducer) + } - // wait a bit for produced message to be available - val request = builder.build() - val response = consumer.fetch(request) - for( (topic, partition) <- topics) { - val fetched = response.messageSet(topic, 0) - assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) - } + def testMultiProduceWithCompression() { + val props = producer.config.props.props + props.put("compression", "true") + val config = new ProducerConfig(props) + val producerWithCompression = new Producer[String, String](config) + multiProduce(producerWithCompression) } def testConsumerEmptyTopic() { @@ -294,16 +233,15 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with } def testPipelinedProduceRequests() { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"), config.brokerId) + val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) + createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) val props = producer.config.props.props props.put("request.required.acks", "0") val pipelinedProducer: Producer[String, String] = new Producer(new ProducerConfig(props)) // send some messages - val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); val messages = new mutable.HashMap[String, Seq[String]] val builder = new FetchRequestBuilder() - var produceList: List[KeyedMessage[String, String]] = Nil for( (topic, partition) <- topics) { val messageList = List("a_" + topic, "b_" + topic) val producerData = messageList.map(new KeyedMessage[String, String](topic, topic, _)) @@ -338,10 +276,10 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with * For testing purposes, just create these topics each with one partition and one replica for * which the provided broker should the leader for. Create and wait for broker to lead. Simple. */ - def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Seq[String], brokerId: Int) { + private def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Iterable[String]) { for( topic <- topics ) { - AdminUtils.createTopic(zkClient, topic, 1, 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 500) + AdminUtils.createTopic(zkClient, topic, partitions = 1, replicationFactor = 1) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition = 0, timeoutMs = 500) } } } From 3f0b67b6ac864befccfdd4bb5dee08c0b33c3b43 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Thu, 13 Feb 2014 13:48:21 -0800 Subject: [PATCH 030/567] KAFKA-1252 Implement retries in new producer. --- .../kafka/clients/producer/KafkaProducer.java | 5 +- .../clients/producer/ProducerConfig.java | 25 +- .../clients/producer/internals/Metadata.java | 32 +-- .../producer/internals/RecordAccumulator.java | 53 ++-- .../producer/internals/RecordBatch.java | 26 +- .../clients/producer/internals/Sender.java | 255 ++++++++++++++---- .../java/org/apache/kafka/common/Cluster.java | 40 +-- .../common/errors/CorruptRecordException.java | 30 +-- .../errors/InvalidMetadataException.java | 39 +++ .../errors/LeaderNotAvailableException.java | 38 ++- .../kafka/common/errors/NetworkException.java | 30 +-- .../NotLeaderForPartitionException.java | 29 +- .../common/errors/OffsetMetadataTooLarge.java | 27 +- .../errors/OffsetOutOfRangeException.java | 30 +-- .../errors/RecordTooLargeException.java | 27 +- .../common/errors/RetriableException.java | 37 +++ .../common/errors/RetryableException.java | 47 ---- .../kafka/common/errors/TimeoutException.java | 29 +- .../common/errors/UnknownServerException.java | 29 +- .../UnknownTopicOrPartitionException.java | 29 +- .../apache/kafka/common/network/Selector.java | 30 +-- .../producer/RecordAccumulatorTest.java | 27 +- .../kafka/clients/producer/SenderTest.java | 139 ++++++++-- 23 files changed, 652 insertions(+), 401 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 3d180e885a25f..e4bc972795858 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -96,7 +96,7 @@ private KafkaProducer(ProducerConfig config) { this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.MAX_PARTITION_SIZE_CONFIG), this.totalMemorySize, config.getLong(ProducerConfig.LINGER_MS_CONFIG), - config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL), + config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG), metrics, new SystemTime()); List addresses = parseAndValidateAddresses(config.getList(ProducerConfig.BROKER_LIST_CONFIG)); @@ -108,7 +108,10 @@ private KafkaProducer(ProducerConfig config) { config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG), config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG), (short) config.getInt(ProducerConfig.REQUIRED_ACKS_CONFIG), + config.getInt(ProducerConfig.MAX_RETRIES_CONFIG), config.getInt(ProducerConfig.REQUEST_TIMEOUT_CONFIG), + config.getInt(ProducerConfig.SEND_BUFFER_CONFIG), + config.getInt(ProducerConfig.RECEIVE_BUFFER_CONFIG), new SystemTime()); this.ioThread = new KafkaThread("kafka-network-thread", this.sender, true); this.ioThread.start(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index dca9802c8d091..d8e35e7d0e4cd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -62,7 +62,7 @@ public class ProducerConfig extends AbstractConfig { /** * The total memory used by the producer to buffer records waiting to be sent to the server. If records are sent * faster than they can be delivered to the server the producer will either block or throw an exception based on the - * preference specified by {@link #BLOCK_ON_BUFFER_FULL}. + * preference specified by {@link #BLOCK_ON_BUFFER_FULL_CONFIG}. */ public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes"; @@ -106,6 +106,11 @@ public class ProducerConfig extends AbstractConfig { */ public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; + /** + * The size of the TCP receive buffer to use when reading data (you generally shouldn't need to change this) + */ + public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes"; + /** * The maximum size of a request. This is also effectively a cap on the maximum record size. Note that the server * has its own cap on record size which may be different from this. @@ -123,9 +128,17 @@ public class ProducerConfig extends AbstractConfig { * this setting is true and we block, however users who want to guarantee we never block can turn this into an * error. */ - public static final String BLOCK_ON_BUFFER_FULL = "block.on.buffer.full"; + public static final String BLOCK_ON_BUFFER_FULL_CONFIG = "block.on.buffer.full"; + + /** + * The maximum number of times to attempt resending the request before giving up. + */ + public static final String MAX_RETRIES_CONFIG = "request.retries"; - public static final String ENABLE_JMX = "enable.jmx"; + /** + * Should we register the Kafka metrics as JMX mbeans? + */ + public static final String ENABLE_JMX_CONFIG = "enable.jmx"; static { /* TODO: add docs */ @@ -142,10 +155,12 @@ public class ProducerConfig extends AbstractConfig { .define(METADATA_REFRESH_MS_CONFIG, Type.LONG, 10 * 60 * 1000, atLeast(-1L), "blah blah") .define(CLIENT_ID_CONFIG, Type.STRING, "", "blah blah") .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), "blah blah") + .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0), "blah blah") .define(MAX_REQUEST_SIZE_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), "blah blah") .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), "blah blah") - .define(BLOCK_ON_BUFFER_FULL, Type.BOOLEAN, true, "blah blah") - .define(ENABLE_JMX, Type.BOOLEAN, true, ""); + .define(BLOCK_ON_BUFFER_FULL_CONFIG, Type.BOOLEAN, true, "blah blah") + .define(ENABLE_JMX_CONFIG, Type.BOOLEAN, true, "") + .define(MAX_RETRIES_CONFIG, Type.INT, 0, between(0, Integer.MAX_VALUE), ""); } ProducerConfig(Map props) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java index 52d30a86d0439..62613a3e29a7e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer.internals; @@ -24,7 +20,6 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.errors.TimeoutException; - /** * A class encapsulating some of the logic around metadata. *

@@ -134,4 +129,11 @@ public synchronized void update(Cluster cluster, long now) { notifyAll(); } + /** + * The last time metadata was updated. + */ + public synchronized long lastUpdate() { + return this.lastRefresh; + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index be8a4a399b8b4..ce5cf27efa08b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -1,25 +1,20 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer.internals; import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.Deque; import java.util.List; @@ -39,10 +34,9 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; - /** - * This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords} instances to be - * sent to the server. + * This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords} + * instances to be sent to the server. *

* The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless * this behavior is explicitly disabled. @@ -151,6 +145,17 @@ public FutureRecordMetadata append(TopicPartition tp, byte[] key, byte[] value, } } + /** + * Re-enqueue the given record batch in the accumulator to retry + */ + public void reenqueue(RecordBatch batch, long now) { + batch.attempts++; + Deque deque = dequeFor(batch.topicPartition); + synchronized (deque) { + deque.addFirst(batch); + } + } + /** * Get a list of topic-partitions which are ready to be sent. *

@@ -229,16 +234,10 @@ private Deque dequeFor(TopicPartition tp) { } /** - * Deallocate the list of record batches + * Deallocate the record batch */ - public void deallocate(Collection batches) { - ByteBuffer[] buffers = new ByteBuffer[batches.size()]; - int i = 0; - for (RecordBatch batch : batches) { - buffers[i] = batch.records.buffer(); - i++; - } - free.deallocate(buffers); + public void deallocate(RecordBatch batch) { + free.deallocate(batch.records.buffer()); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index 7a440a3dd29c7..eb16f6d236e07 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer.internals; @@ -25,7 +21,6 @@ import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; - /** * A batch of records that is or will be sent. * @@ -33,6 +28,7 @@ */ public final class RecordBatch { public int recordCount = 0; + public volatile int attempts = 0; public final long created; public final MemoryRecords records; public final TopicPartition topicPartition; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index d93a455827a67..e373265f19f6e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -22,12 +22,15 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.Selectable; @@ -41,6 +44,7 @@ import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; /** * The background thread that handles the sending of produce requests to the Kafka cluster. This thread makes metadata @@ -48,19 +52,55 @@ */ public class Sender implements Runnable { - private final Map nodeState; + /* the state of each nodes connection */ + private final NodeStates nodeStates; + + /* the record accumulator that batches records */ private final RecordAccumulator accumulator; + + /* the selector used to perform network i/o */ private final Selectable selector; + + /* the client id used to identify this client in requests to the server */ private final String clientId; + + /* the maximum request size to attempt to send to the server */ private final int maxRequestSize; - private final long reconnectBackoffMs; + + /* the number of acknowledgements to request from the server */ private final short acks; + + /* the max time in ms for the server to wait for acknowlegements */ private final int requestTimeout; + + /* the number of times to retry a failed request before giving up */ + private final int retries; + + /* the socket send buffer size in bytes */ + private final int socketSendBuffer; + + /* the socket receive size buffer in bytes */ + private final int socketReceiveBuffer; + + /* the set of currently in-flight requests awaiting a response from the server */ private final InFlightRequests inFlightRequests; + + /* a reference to the current Cluster instance */ private final Metadata metadata; + + /* the clock instance used for getting the time */ private final Time time; + + /* the current node to attempt to use for metadata requests (will round-robin over nodes) */ + private int metadataFetchNodeIndex; + + /* the current correlation id to use when sending requests to servers */ private int correlation; + + /* true iff there is a metadata request that has been sent and for which we have not yet received a response */ private boolean metadataFetchInProgress; + + /* true while the sender thread is still running */ private volatile boolean running; public Sender(Selectable selector, @@ -70,22 +110,28 @@ public Sender(Selectable selector, int maxRequestSize, long reconnectBackoffMs, short acks, + int retries, int requestTimeout, + int socketSendBuffer, + int socketReceiveBuffer, Time time) { - this.nodeState = new HashMap(); + this.nodeStates = new NodeStates(reconnectBackoffMs); this.accumulator = accumulator; this.selector = selector; this.maxRequestSize = maxRequestSize; - this.reconnectBackoffMs = reconnectBackoffMs; this.metadata = metadata; this.clientId = clientId; this.running = true; this.requestTimeout = requestTimeout; this.acks = acks; + this.retries = retries; + this.socketSendBuffer = socketSendBuffer; + this.socketReceiveBuffer = socketReceiveBuffer; this.inFlightRequests = new InFlightRequests(); this.correlation = 0; this.metadataFetchInProgress = false; this.time = time; + this.metadataFetchNodeIndex = new Random().nextInt(); } /** @@ -130,11 +176,7 @@ public int run(long now) { // should we update our metadata? List sends = new ArrayList(); - InFlightRequest metadataReq = maybeMetadataRequest(cluster, now); - if (metadataReq != null) { - sends.add(metadataReq.request); - this.inFlightRequests.add(metadataReq); - } + maybeUpdateMetadata(cluster, sends, now); // prune the list of ready topics to eliminate any that we aren't ready to send yet List sendable = processReadyPartitions(cluster, ready, now); @@ -158,43 +200,76 @@ public int run(long now) { // handle responses, connections, and disconnections handleSends(this.selector.completedSends()); handleResponses(this.selector.completedReceives(), now); - handleDisconnects(this.selector.disconnected()); + handleDisconnects(this.selector.disconnected(), now); handleConnects(this.selector.connected()); return ready.size(); } - private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) { + /** + * Add a metadata request to the list of sends if we need to make one + */ + private void maybeUpdateMetadata(Cluster cluster, List sends, long now) { if (this.metadataFetchInProgress || !metadata.needsUpdate(now)) - return null; + return; - Node node = nextFreeNode(cluster); + Node node = selectMetadataDestination(cluster); if (node == null) - return null; + return; - NodeState state = nodeState.get(node.id()); - if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) { + if (nodeStates.isConnected(node.id())) { + this.metadataFetchInProgress = true; + InFlightRequest request = metadataRequest(node.id(), metadata.topics()); + sends.add(request.request); + this.inFlightRequests.add(request); + } else if (nodeStates.canConnect(node.id(), now)) { // we don't have a connection to this node right now, make one initiateConnect(node, now); - return null; - } else if (state.state == ConnectionState.CONNECTED) { - this.metadataFetchInProgress = true; - return metadataRequest(node.id(), metadata.topics()); - } else { - return null; } } /** + * Find a good node to make a metadata request to. This method will first look for a node that has an existing + * connection and no outstanding requests. If there are no such nodes it will look for a node with no outstanding + * requests. * @return A node with no requests currently being sent or null if no such node exists */ - private Node nextFreeNode(Cluster cluster) { - for (int i = 0; i < cluster.nodes().size(); i++) { - Node node = cluster.nextNode(); - if (this.inFlightRequests.canSendMore(node.id())) + private Node selectMetadataDestination(Cluster cluster) { + List nodes = cluster.nodes(); + + // first look for a node to which we are connected and have no outstanding requests + boolean connectionInProgress = false; + for (int i = 0; i < nodes.size(); i++) { + Node node = nodes.get(metadataNodeIndex(i, nodes.size())); + if (nodeStates.isConnected(node.id()) && this.inFlightRequests.canSendMore(node.id())) { + this.metadataFetchNodeIndex = metadataNodeIndex(i + 1, nodes.size()); + return node; + } else if (nodeStates.isConnecting(node.id())) { + connectionInProgress = true; + } + } + + // if we have a connection that is being established now, just wait for that don't make another + if (connectionInProgress) + return null; + + // okay, no luck, pick a random unused node + for (int i = 0; i < nodes.size(); i++) { + Node node = nodes.get(metadataNodeIndex(i, nodes.size())); + if (this.inFlightRequests.canSendMore(node.id())) { + this.metadataFetchNodeIndex = metadataNodeIndex(i + 1, nodes.size()); return node; + } } - return null; + + return null; // we failed to find a good destination + } + + /** + * Get the index in the node list of the node to use for the metadata request + */ + private int metadataNodeIndex(int offset, int size) { + return Utils.abs(offset + this.metadataFetchNodeIndex) % size; } /** @@ -209,7 +284,7 @@ public void initiateClose() { /** * Process the set of topic-partitions with data ready to send. If we have a connection to the appropriate node, add * it to the returned set. For any partitions we have no connection to either make one, fetch the appropriate - * metdata to be able to do so + * metadata to be able to do so */ private List processReadyPartitions(Cluster cluster, List ready, long now) { List sendable = new ArrayList(ready.size()); @@ -218,15 +293,11 @@ private List processReadyPartitions(Cluster cluster, List this.reconnectBackoffMs)) { - // we don't have a connection to this node right now, make one - initiateConnect(node, now); - } else if (state.state == ConnectionState.CONNECTED && inFlightRequests.canSendMore(node.id())) { - sendable.add(tp); - } + } else if (nodeStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) { + sendable.add(tp); + } else if (nodeStates.canConnect(node.id(), now)) { + // we don't have a connection to this node right now, make one + initiateConnect(node, now); } } return sendable; @@ -237,13 +308,11 @@ private List processReadyPartitions(Cluster cluster, List disconnects) { + private void handleDisconnects(List disconnects, long now) { + // clear out the in-flight requests for the disconnected broker for (int node : disconnects) { for (InFlightRequest request : this.inFlightRequests.clearAll(node)) { if (request.batches != null) { - for (RecordBatch batch : request.batches.values()) - batch.done(-1L, new NetworkException("The server disconnected unexpectedly without sending a response.")); - this.accumulator.deallocate(request.batches.values()); + for (RecordBatch batch : request.batches.values()) { + if (canRetry(batch, Errors.NETWORK_EXCEPTION)) { + this.accumulator.reenqueue(batch, now); + } else { + batch.done(-1L, new NetworkException("The server disconnected unexpectedly without sending a response.")); + this.accumulator.deallocate(batch); + } + } } - NodeState state = this.nodeState.get(request.request.destination()); - if (state != null) - state.state = ConnectionState.DISCONNECTED; + nodeStates.disconnected(request.request.destination()); } } + // we got a disconnect so we should probably refresh our metadata and see if that broker is dead + if (disconnects.size() > 0) + this.metadata.forceUpdate(); } /** @@ -272,7 +348,7 @@ private void handleDisconnects(List disconnects) { */ private void handleConnects(List connects) { for (Integer id : connects) - this.nodeState.get(id).state = ConnectionState.CONNECTED; + this.nodeStates.connected(id); } /** @@ -286,9 +362,10 @@ public void handleSends(List sends) { if (!request.expectResponse) { requests.pollFirst(); if (request.request.header().apiKey() == ApiKeys.PRODUCE.id) { - for (RecordBatch batch : request.batches.values()) + for (RecordBatch batch : request.batches.values()) { batch.done(-1L, Errors.NONE.exception()); - this.accumulator.deallocate(request.batches.values()); + this.accumulator.deallocate(batch); + } } } } @@ -306,7 +383,7 @@ private void handleResponses(List receives, long now) { Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload()); correlate(req.request.header(), header); if (req.request.header().apiKey() == ApiKeys.PRODUCE.id) - handleProduceResponse(req, body); + handleProduceResponse(req, body, now); else if (req.request.header().apiKey() == ApiKeys.METADATA.id) handleMetadataResponse(body, now); else @@ -327,7 +404,7 @@ private void handleMetadataResponse(Struct body, long now) { /** * Handle a produce response */ - private void handleProduceResponse(InFlightRequest request, Struct response) { + private void handleProduceResponse(InFlightRequest request, Struct response, long now) { for (Object topicResponse : (Object[]) response.get("responses")) { Struct topicRespStruct = (Struct) topicResponse; String topic = (String) topicRespStruct.get("topic"); @@ -335,12 +412,31 @@ private void handleProduceResponse(InFlightRequest request, Struct response) { Struct partRespStruct = (Struct) partResponse; int partition = (Integer) partRespStruct.get("partition"); short errorCode = (Short) partRespStruct.get("error_code"); + + // if we got an error we may need to refresh our metadata + Errors error = Errors.forCode(errorCode); + if (error.exception() instanceof InvalidMetadataException) + metadata.forceUpdate(); + long offset = (Long) partRespStruct.get("base_offset"); RecordBatch batch = request.batches.get(new TopicPartition(topic, partition)); - batch.done(offset, Errors.forCode(errorCode).exception()); + if (canRetry(batch, error)) { + // retry + this.accumulator.reenqueue(batch, now); + } else { + // tell the user the result of their request + batch.done(offset, error.exception()); + this.accumulator.deallocate(batch); + } } } - this.accumulator.deallocate(request.batches.values()); + } + + /** + * We can retry a send if the error is transient and the number of attempts taken is fewer than the maximum allowed + */ + private boolean canRetry(RecordBatch batch, Errors error) { + return batch.attempts < this.retries && error.exception() instanceof RetriableException; } /** @@ -459,6 +555,53 @@ public String toString() { } } + private static class NodeStates { + private final long reconnectBackoffMs; + private final Map nodeState; + + public NodeStates(long reconnectBackoffMs) { + this.reconnectBackoffMs = reconnectBackoffMs; + this.nodeState = new HashMap(); + } + + public boolean canConnect(int node, long now) { + NodeState state = nodeState.get(node); + if (state == null) + return true; + else + return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs; + } + + public void connecting(int node, long now) { + nodeState.put(node, new NodeState(ConnectionState.CONNECTING, now)); + } + + public boolean isConnected(int node) { + NodeState state = nodeState.get(node); + return state != null && state.state == ConnectionState.CONNECTED; + } + + public boolean isConnecting(int node) { + NodeState state = nodeState.get(node); + return state != null && state.state == ConnectionState.CONNECTING; + } + + public void connected(int node) { + nodeState(node).state = ConnectionState.CONNECTED; + } + + public void disconnected(int node) { + nodeState(node).state = ConnectionState.DISCONNECTED; + } + + private NodeState nodeState(int node) { + NodeState state = this.nodeState.get(node); + if (state == null) + throw new IllegalStateException("No entry found for node " + node); + return state; + } + } + /** * An request that hasn't been fully processed yet */ diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index c17a8f8162db9..5caaaae1d2ad1 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common; @@ -23,17 +19,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.kafka.common.utils.Utils; - /** * A representation of a subset of the nodes, topics, and partitions in the Kafka cluster. */ public final class Cluster { - private final AtomicInteger counter = new AtomicInteger(0); private final List nodes; private final Map partitionsByTopicPartition; private final Map> partitionsByTopic; @@ -126,15 +117,4 @@ public List partitionsFor(String topic) { return this.partitionsByTopic.get(topic); } - /** - * Round-robin over the nodes in this cluster - */ - public Node nextNode() { - int size = nodes.size(); - if (size == 0) - throw new IllegalStateException("No known nodes."); - int idx = Utils.abs(counter.getAndIncrement()) % size; - return this.nodes.get(idx); - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java index 673f61d6271c5..eaccf276dbfb3 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java @@ -1,22 +1,22 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; -public class CorruptRecordException extends ApiException { +/** + * This exception indicates a record has failed it's internal CRC check, this generally indicates network or disk + * corruption. + */ +public class CorruptRecordException extends RetriableException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java new file mode 100644 index 0000000000000..8841badb2d783 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidMetadataException.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.errors; + +/** + * An exception that may indicate the client's metadata is out of date + */ +public abstract class InvalidMetadataException extends RetriableException { + + private static final long serialVersionUID = 1L; + + public InvalidMetadataException() { + super(); + } + + public InvalidMetadataException(String message) { + super(message); + } + + public InvalidMetadataException(String message, Throwable cause) { + super(message, cause); + } + + public InvalidMetadataException(Throwable cause) { + super(cause); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java index 0bde6b5a351fc..9d7ebd47a8439 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java @@ -1,35 +1,27 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; -public class LeaderNotAvailableException extends RetryableException { +/** + * There is no currently available leader for the given partition (either because a leadership election is in progress + * or because all replicas are down). + */ +public class LeaderNotAvailableException extends InvalidMetadataException { private static final long serialVersionUID = 1L; - public LeaderNotAvailableException(String message, Throwable cause) { - super(message, cause); - } - public LeaderNotAvailableException(String message) { super(message); } - public LeaderNotAvailableException(Throwable cause) { - super(cause); - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java index 3a041593d76cf..f0baa983f081e 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java @@ -1,22 +1,22 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; -public class NetworkException extends ApiException { +/** + * A misc. network-related IOException occurred when making a request. This could be because the client's metadata is + * out of date and it is making a request to a node that is now dead. + */ +public class NetworkException extends InvalidMetadataException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java index 5adc72ccf2d0c..ad9c77c41c492 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java @@ -1,22 +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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; -public class NotLeaderForPartitionException extends RetryableException { +/** + * This server is not the leader for the given partition + */ +public class NotLeaderForPartitionException extends InvalidMetadataException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java index a3159bb1034e7..0be2f500685b0 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java @@ -1,21 +1,20 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; +/** + * The client has tried to save its offset with associated metadata larger than the maximum size allowed by the server. + */ public class OffsetMetadataTooLarge extends ApiException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java index d01698a3efca7..fc7c6e3471b05 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java @@ -1,22 +1,22 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; -public class OffsetOutOfRangeException extends ApiException { +/** + * This offset is either larger or smaller than the range of offsets the server has for the given partition. + * + */ +public class OffsetOutOfRangeException extends RetriableException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java index ce95ca04aa842..737b7f07b16a0 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java @@ -1,21 +1,20 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; +/** + * This record is larger than the maximum allowable size + */ public class RecordTooLargeException extends ApiException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java b/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java new file mode 100644 index 0000000000000..6c639a972d7e4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/RetriableException.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * A retryable exception is a transient exception that if retried may succeed. + */ +public abstract class RetriableException extends ApiException { + + private static final long serialVersionUID = 1L; + + public RetriableException(String message, Throwable cause) { + super(message, cause); + } + + public RetriableException(String message) { + super(message); + } + + public RetriableException(Throwable cause) { + super(cause); + } + + public RetriableException() { + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java b/clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java deleted file mode 100644 index c7f2f222f712a..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.errors; - -/** - * A retryable exception is an exception that is safe to retry. To be retryable an exception should be - *

    - *
  1. Transient, there is no point retrying a error due to a non-existant topic or message too large - *
  2. Idempotent, the exception is known to not change any state on the server - *
- * A client may choose to retry any request they like, but exceptions extending this class are always safe and sane to - * retry. - */ -public abstract class RetryableException extends ApiException { - - private static final long serialVersionUID = 1L; - - public RetryableException(String message, Throwable cause) { - super(message, cause); - } - - public RetryableException(String message) { - super(message); - } - - public RetryableException(Throwable cause) { - super(cause); - } - - public RetryableException() { - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java index dffd64d19c35a..c7f569ca87369 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java @@ -1,22 +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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; -public class TimeoutException extends ApiException { +/** + * Indicates that a request timed out. + */ +public class TimeoutException extends RetriableException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java index a0690fe2870bf..963ef081db51d 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java @@ -1,21 +1,22 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; +/** + * An error occurred on the server for which the client doesn't have a corresponding error code. This is generally an + * unexpected error. + * + */ public class UnknownServerException extends ApiException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java index 73d1953cbe045..ec423bd01298c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java @@ -1,22 +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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.errors; -public class UnknownTopicOrPartitionException extends ApiException { +/** + * This topic/partition doesn't exist + */ +public class UnknownTopicOrPartitionException extends RetriableException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 8ed4c73146b2e..f1e474cd53011 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.common.network; @@ -32,7 +28,6 @@ import org.apache.kafka.common.KafkaException; - /** * A selector interface for doing non-blocking multi-connection network I/O. *

@@ -302,8 +297,11 @@ else if (ms < 0L) private void close(SelectionKey key) throws IOException { SocketChannel channel = channel(key); Transmissions trans = transmissions(key); - if (trans != null) + if (trans != null) { this.disconnected.add(trans.id); + trans.clearReceive(); + trans.clearSend(); + } key.attach(null); key.cancel(); channel.socket().close(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java index 1bbe83c1bfd75..a3bf07e4ae2da 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java @@ -1,18 +1,14 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer; @@ -25,7 +21,6 @@ import java.util.Iterator; import java.util.List; - import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.RecordBatch; import org.apache.kafka.common.TopicPartition; @@ -140,8 +135,8 @@ public void run() { for (RecordBatch batch : batches) { for (LogEntry entry : batch.records) read++; + accum.deallocate(batch); } - accum.deallocate(batches); } for (Thread t : threads) diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java index 41c028bffbda1..19a01258f7418 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java @@ -1,29 +1,25 @@ /** - * 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. + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. */ package org.apache.kafka.clients.producer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.nio.ByteBuffer; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; - -import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; @@ -46,6 +42,16 @@ public class SenderTest { + private static final String CLIENT_ID = ""; + private static final int MAX_REQUEST_SIZE = 1024 * 1024; + private static final long RECONNECT_BACKOFF_MS = 0L; + private static final short ACKS_ALL = -1; + private static final int MAX_RETRIES = 0; + private static final int REQUEST_TIMEOUT_MS = 10000; + private static final int SEND_BUFFER_SIZE = 64 * 1024; + private static final int RECEIVE_BUFFER_SIZE = 64 * 1024; + + private TopicPartition tp = new TopicPartition("test", 0); private MockTime time = new MockTime(); private MockSelector selector = new MockSelector(time); private int batchSize = 16 * 1024; @@ -53,7 +59,18 @@ public class SenderTest { private Cluster cluster = TestUtils.singletonCluster("test", 1); private Metrics metrics = new Metrics(time); private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, false, metrics, time); - private Sender sender = new Sender(selector, metadata, this.accumulator, "", 1024 * 1024, 0L, (short) -1, 10000, time); + private Sender sender = new Sender(selector, + metadata, + this.accumulator, + CLIENT_ID, + MAX_REQUEST_SIZE, + RECONNECT_BACKOFF_MS, + ACKS_ALL, + MAX_RETRIES, + REQUEST_TIMEOUT_MS, + SEND_BUFFER_SIZE, + RECEIVE_BUFFER_SIZE, + time); @Before public void setup() { @@ -62,7 +79,6 @@ public void setup() { @Test public void testSimple() throws Exception { - TopicPartition tp = new TopicPartition("test", 0); Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); sender.run(time.milliseconds()); assertEquals("We should have connected", 1, selector.connected().size()); @@ -83,6 +99,93 @@ public void testSimple() throws Exception { assertEquals(offset, future.get().offset()); } + @Test + public void testRetries() throws Exception { + // create a sender with retries = 1 + int maxRetries = 1; + Sender sender = new Sender(selector, + metadata, + this.accumulator, + CLIENT_ID, + MAX_REQUEST_SIZE, + RECONNECT_BACKOFF_MS, + ACKS_ALL, + maxRetries, + REQUEST_TIMEOUT_MS, + SEND_BUFFER_SIZE, + RECEIVE_BUFFER_SIZE, + time); + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); + RequestSend request1 = completeSend(sender); + selector.clear(); + selector.completeReceive(produceResponse(request1.header().correlationId(), + cluster.leaderFor(tp).id(), + tp.topic(), + tp.partition(), + -1, + Errors.REQUEST_TIMED_OUT.code())); + sender.run(time.milliseconds()); + selector.clear(); + sender.run(time.milliseconds()); + RequestSend request2 = completeSend(sender); + selector.completeReceive(produceResponse(request2.header().correlationId(), + cluster.leaderFor(tp).id(), + tp.topic(), + tp.partition(), + 42, + Errors.NONE.code())); + sender.run(time.milliseconds()); + assertTrue("Request should retry and complete", future.isDone()); + assertEquals(42, future.get().offset()); + } + + @Test + public void testMetadataRefreshOnNoLeaderException() throws Exception { + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); + RequestSend request = completeSend(); + selector.clear(); + selector.completeReceive(produceResponse(request.header().correlationId(), + cluster.leaderFor(tp).id(), + tp.topic(), + tp.partition(), + -1, + Errors.NOT_LEADER_FOR_PARTITION.code())); + sender.run(time.milliseconds()); + completedWithError(future, Errors.NOT_LEADER_FOR_PARTITION); + assertTrue("Error triggers a metadata update.", metadata.needsUpdate(time.milliseconds())); + } + + @Test + public void testMetadataRefreshOnDisconnect() throws Exception { + Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null); + completeSend(); + selector.clear(); + selector.disconnect(cluster.leaderFor(tp).id()); + sender.run(time.milliseconds()); + completedWithError(future, Errors.NETWORK_EXCEPTION); + assertTrue("The disconnection triggers a metadata update.", metadata.needsUpdate(time.milliseconds())); + } + + private void completedWithError(Future future, Errors error) throws Exception { + assertTrue("Request should be completed", future.isDone()); + try { + future.get(); + fail("Should have thrown an exception."); + } catch (ExecutionException e) { + assertEquals(error.exception().getClass(), e.getCause().getClass()); + } + } + + private RequestSend completeSend() { + return completeSend(sender); + } + + private RequestSend completeSend(Sender sender) { + while (selector.completedSends().size() == 0) + sender.run(time.milliseconds()); + return (RequestSend) selector.completedSends().get(0); + } + private NetworkReceive produceResponse(int correlation, int source, String topic, int part, long offset, int error) { Struct struct = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); Struct response = struct.instance("responses"); From 22f76d8b362417858e61f0f9a2fc80d620790f2f Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Tue, 18 Feb 2014 19:31:30 -0800 Subject: [PATCH 031/567] Fix compile issue from last commit. Oops. :-( --- core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala b/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala index 0d6d7024ea00b..d23ef9a31aa0c 100644 --- a/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/newproducer/MirrorMaker.scala @@ -84,7 +84,7 @@ object MirrorMaker extends Logging { new Blacklist(options.valueOf(blacklistOpt)) val producerConfig = options.valueOf(producerConfigOpt) val producerProps = Utils.loadProps(producerConfig) - producerProps.setProperty(ProducerConfig.BLOCK_ON_BUFFER_FULL, "true") + producerProps.setProperty(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") val consumerConfig = options.valueOf(consumerConfigOpt) val numStreams = options.valueOf(numStreamsOpt) producerChannel = new ProducerDataChannel() From 16c69a844fabfe512f8c090fedde2014e392800b Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Wed, 19 Feb 2014 10:53:11 -0500 Subject: [PATCH 032/567] KAFKA-1274 gradle.properties needs the variables used in the build.gradle patch by Joe Stein; Reviewed by Jun Rao --- README.md | 2 +- gradle.properties | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index ec80083b089dc..d0cdde9ec07fb 100644 --- a/README.md +++ b/README.md @@ -55,7 +55,7 @@ This is for 'core', 'perf', 'contrib:hadoop-consumer', 'contrib:hadoop-producer' ### Publishing the jar for all version of Scala and for all projects to maven ### ./gradlew uploadArchivesAll -Please note for this to work you need to create/update `~/.gradle/gradle.properties` and assign the following variables +Please note for this to work you should create/update `~/.gradle/gradle.properties` and assign the following variables mavenUrl= mavenUsername= diff --git a/gradle.properties b/gradle.properties index 447ee442105e2..4827769a3f8e3 100644 --- a/gradle.properties +++ b/gradle.properties @@ -18,3 +18,6 @@ version=0.8.1 scalaVersion=2.8.0 task=build +mavenUrl= +mavenUsername= +mavenPassword= From a8297cc6ca90f2b528091a46e3cf254e84e74d47 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Thu, 20 Feb 2014 00:11:31 -0500 Subject: [PATCH 033/567] KAFKA-1254 remove vestigial sbt patch by Joe Stein; reviewed by Jun Rao --- LICENSE | 31 ---- README-sbt.md | 60 ------- bin/kafka-run-class.sh | 7 - clients/build.sbt | 11 -- contrib/hadoop-consumer/build.sbt | 1 - contrib/hadoop-producer/build.sbt | 1 - core/build.sbt | 32 ---- examples/build.sbt | 3 - lib/sbt-launch.jar | Bin 1103618 -> 0 bytes perf/build.sbt | 1 - project/Build.scala | 152 ------------------ project/build.properties | 17 -- project/build/KafkaProject.scala | 251 ------------------------------ project/plugins.sbt | 9 -- sbt | 16 -- sbt.bat | 17 -- 16 files changed, 609 deletions(-) delete mode 100644 README-sbt.md delete mode 100644 clients/build.sbt delete mode 100644 contrib/hadoop-consumer/build.sbt delete mode 100644 contrib/hadoop-producer/build.sbt delete mode 100644 core/build.sbt delete mode 100644 examples/build.sbt delete mode 100644 lib/sbt-launch.jar delete mode 100644 perf/build.sbt delete mode 100644 project/Build.scala delete mode 100644 project/build.properties delete mode 100644 project/build/KafkaProject.scala delete mode 100644 project/plugins.sbt delete mode 100755 sbt delete mode 100644 sbt.bat diff --git a/LICENSE b/LICENSE index cb1800b0c39af..d645695673349 100644 --- a/LICENSE +++ b/LICENSE @@ -200,34 +200,3 @@ 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. - ------------------------------------------------------------------------ - -SBT LICENSE - -Copyright (c) 2008, 2009, 2010 Mark Harrah, Jason Zaugg -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: -1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. -2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. -3. The name of the author may not be used to endorse or promote products - derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR -IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES -OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. -IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, -INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT -NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF -THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - ------------------------------------------------------------------------ \ No newline at end of file diff --git a/README-sbt.md b/README-sbt.md deleted file mode 100644 index 10b8d2523605e..0000000000000 --- a/README-sbt.md +++ /dev/null @@ -1,60 +0,0 @@ -# Apache Kafka # - -See our [web site](http://kafka.apache.org) for details on the project. - -## Building it ## -1. ./sbt update -2. ./sbt package -3. ./sbt assembly-package-dependency - -To build for a particular version of Scala (either 2.8.0, 2.8.2, 2.9.1, 2.9.2 or 2.10.1), change step 2 above to: -2. ./sbt "++2.8.0 package" - -To build for all supported versions of Scala, change step 2 above to: -2. ./sbt +package - -## Running it ## -Follow instuctions in http://kafka.apache.org/documentation.html#quickstart - -## Running unit tests ## - ./sbt test - -## Building a binary release zip or gzipped tar ball ## - ./sbt release-zip - ./sbt release-tar -The release file can be found inside ./target/RELEASE/. - -## Other Build Tips ## -Here are some useful sbt commands, to be executed at the sbt command prompt (./sbt). Prefixing with "++ " runs the -command for a specific Scala version, prefixing with "+" will perform the action for all versions of Scala, and no prefix -runs the command for the default (2.8.0) version of Scala. - - -tasks : Lists all the sbt commands and their descriptions -clean : Deletes all generated files (the target directory). -compile : Compile all the sub projects, but not create the jars -test : Run all unit tests in all sub projects -release-zip : Create all the jars, run unit tests and create a deployable release zip -release-tar : Create all the jars, run unit tests and create a deployable release gzipped tar tall -package: Creates jars for src, test, docs etc -projects : List all the sub projects -project sub_project_name : Switch to a particular sub-project. For example, to switch to the core kafka code, use "project core-kafka" - -The following commands can be run only on a particular sub project - -test-only package.test.TestName : Runs only the specified test in the current sub project -run : Provides options to run any of the classes that have a main method. For example, you can switch to project java-examples, and run the examples there by executing "project java-examples" followed by "run" - -For more details please see the [SBT documentation](https://github.com/harrah/xsbt/wiki) - -## Contribution ## - -Kafka is a new project, and we are interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). - -To contribute follow the instructions here: - * http://kafka.apache.org/contributing.html - -We also welcome patches for the website and documentation which can be found here: - * https://svn.apache.org/repos/asf/kafka/site - - - - diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index e0c31ee9b4997..4953af0124f84 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -32,13 +32,6 @@ if [ -z "$SCALA_VERSION" ]; then SCALA_VERSION=2.8.0 fi -# TODO: remove when removing sbt -# assume all dependencies have been packaged into one jar with sbt-assembly's task "assembly-package-dependency" -for file in $base_dir/core/target/scala-${SCALA_VERSION}/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - # run ./gradlew copyDependantLibs to get all dependant jars in a local dir for file in $base_dir/core/build/dependant-libs-${SCALA_VERSION}/*.jar; do diff --git a/clients/build.sbt b/clients/build.sbt deleted file mode 100644 index ca3c8ee3d7e56..0000000000000 --- a/clients/build.sbt +++ /dev/null @@ -1,11 +0,0 @@ -import sbt._ -import Keys._ -import AssemblyKeys._ - -name := "clients" - -libraryDependencies ++= Seq( - "com.novocode" % "junit-interface" % "0.9" % "test" -) - -assemblySettings diff --git a/contrib/hadoop-consumer/build.sbt b/contrib/hadoop-consumer/build.sbt deleted file mode 100644 index 02e95eb8ca2c7..0000000000000 --- a/contrib/hadoop-consumer/build.sbt +++ /dev/null @@ -1 +0,0 @@ -crossPaths := false diff --git a/contrib/hadoop-producer/build.sbt b/contrib/hadoop-producer/build.sbt deleted file mode 100644 index 02e95eb8ca2c7..0000000000000 --- a/contrib/hadoop-producer/build.sbt +++ /dev/null @@ -1 +0,0 @@ -crossPaths := false diff --git a/core/build.sbt b/core/build.sbt deleted file mode 100644 index 3eff64ef3f15a..0000000000000 --- a/core/build.sbt +++ /dev/null @@ -1,32 +0,0 @@ -import sbt._ -import Keys._ -import AssemblyKeys._ - -name := "kafka" - -resolvers ++= Seq( - "SonaType ScalaTest repo" at "https://oss.sonatype.org/content/groups/public/org/scalatest/" -) - -libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _ ) - -libraryDependencies ++= Seq( - "org.apache.zookeeper" % "zookeeper" % "3.3.4", - "com.101tec" % "zkclient" % "0.3", - "org.xerial.snappy" % "snappy-java" % "1.0.5", - "com.yammer.metrics" % "metrics-core" % "2.2.0", - "com.yammer.metrics" % "metrics-annotation" % "2.2.0", - "org.easymock" % "easymock" % "3.0" % "test", - "junit" % "junit" % "4.1" % "test" -) - -libraryDependencies <<= (scalaVersion, libraryDependencies) { (sv, deps) => - deps :+ (sv match { - case "2.8.0" => "org.scalatest" % "scalatest" % "1.2" % "test" - case v if v.startsWith("2.10") => "org.scalatest" %% "scalatest" % "1.9.1" % "test" - case _ => "org.scalatest" %% "scalatest" % "1.8" % "test" - }) -} - -assemblySettings - diff --git a/examples/build.sbt b/examples/build.sbt deleted file mode 100644 index d12d70142f6ec..0000000000000 --- a/examples/build.sbt +++ /dev/null @@ -1,3 +0,0 @@ -name := "kafka-java-examples" - -crossPaths := false diff --git a/lib/sbt-launch.jar b/lib/sbt-launch.jar deleted file mode 100644 index 06ad8d880592ad776fdfc38c573359261314e6c3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1103618 zcmbrlWo#r35+&GXcDsyiW@cu)&CJZq%nW5_W@cMN43JQAtxORf-@6S;D zJ>~!7gRuVIwR1G1H?TJ_GB=^OaP^>!A?a3fY56T zgRg!wZHo+wL=FhpB@&kM*6OdN3E4DZmcaZ#@#}+%O#Nw*9Lx8tkish=4{tbBm63Cj zv#`MHWBT?MJEI2zzbgU_Mo1G}o3F3XhXNsoMnmmhLXfX7(pM3{3ULO4+QU`G!leQ} zTSY{nCa{rGAQj<@M!1n&AU8z1p#-y4S%Al@cuYmNAA@RukoaScAy9#Gn03-nZ~$kR zg(o0?ml2Y6lDL2{!~g;A$2`)A55i3GIvEi>s}7e?6`n5DxJig9Z`KOCQ%O7X$6t#t|+7}5sFT#C4! zQz*@6#sx#*dQ6Qc6GzNhGnbc+xeT0M11J~_y9*H$3{AMaHaynj$-`$q1^le(Oj z>$BlaF4{1)T79>~UrQ46o%T2{!wR(Q*Ica`C*(1l+&F~7#?j+JepHiKht`;WZ*I$F zX<2A2w{6$pAHuLH2bY=IpGFko9dp@kD&xXrx*PbFUJ`8Spg`s?KwSKTCH<+0z8vN^ z6>!|d*qq=H9zZHIk7ml%hbwi?$fBxx{avXX3%<&a9&hg6lu!DqtU%UzKgS?s+K-fyeuMagi1d@g}*x!K}uUQc#oXya*K`W((Nzvw=I_D2Cp;QVNaJ(>g$g%H0&YI*U? z$4`(Jz7#C^YVSN8C6)uB9a9iBegfBkm_j%=uO5O{7im@1 zpUgw%R-yv}Aipy##p&}7!7qAH&Pz%l*&^uJV))pAlx*P|Y|$K2Czd6;AY%}P$Z+Z~_E&6}_e@6emV?SFR&K*Yu?bEJ)%$hw>ti@)H%qF8qBGVQx zv-xK}Pmzrxw)lKn(C`izS*Aq&g{fI~x(v2}2$Bc{fCgFw9JP!H)Miyi@<0KC>ME5U zRpBQ@?lbtd`}E``Sx0&m(0e>)Bq-%PZuas+in6KpP6cPd$!f+gk1Id2Ua2^R%$pUfYp#+Nf zL2n6n(z>6>aHMtO$i)fjG~uKP>Qv!O5=l~q=_Dyq1%`O&#FK(X2K7IT>OzM&A|o}t zLxv7%x_b@jBP-Rlr}i|_-*_l`ZND3PrwxtpQV7Nkk@HaeDMi*grm4Qu=c9Yj2XXb3 z9|tRb{t+p~V@x&LuMaS}kwDLV(ues@Gvq--XLQ4fu2$QcFvNslGAVBI?Y3;xl38$_-y0?I*Y_u4d{9FK4`J)~S{~-^t50qxG6t)%`qwd)n z#PSA!`F#hQ2CEpYOBI6ztqD~;t%5RK}tgvPI7rTVggAev$=W2=Y#-Vr}R8 zOWH%t-s=4q8Ck7gwKQBQ3{2DyAFvzzU6y7AjVgN9ZM6ml`f4SgLulNESY=?N`l|0q za$t1%v2AS{g;?Ql^K`Ju*cFGZ8uCH|2qV(`DwmMxT5P8frE;bO+@#6DEYD2IZB6M~ zf=~x)ADykUs@8CDvFcT8rV(@h^b?UIy*L*H`qEXJwXbp)Sal==<1#|BmtiMsq@Kdn z^t6u#sqt2MtVj~oo%D6P8URpxHwm~Sf{8kO@Rn)4PCMZ19iyu`#ED9%*_8v+MZcXF zwuOlL^dI*&taatFR3;GKc5z?(q*%Dd|h7TZ9z)&8S}@@P6Dpsms8gd2#<1y2y%27FtAMw|1xCz$vaw zqFZecH{y8ljCvbhC=_|{mt+lkCOKxJRD^^wQ^o^A6pa>fI;e*WFJ2QkxD$-ZZ26-f zHU}-kpIa6>mYvOol_yZ~_b&bdl-1*pwqdQeVL4WZWLHXXh`E9iPRXyz8k$)~auo>+ zkyWvXXQ5HxWQ~b|Xk_=ASs}*~6;3)EKG7Q$FFtjR8elC=g$)xHa4qs4OL>1Xc zS2_<2JEbNXJtf$#cj@wu{wY5y*3B)sDyXw{p)r3WiKY-OZMvxkEJRt~za`hM8jE#> zul6+}rHureMaif)K^xmnw7qR11M zG(Cfcwg4g|M5~a?CTE>3|EQYI!7HIH#K}&9Dj?g$RX0G9rq@(90*-#8!`@7$f_U0w zREO(C*p@M;&`#*mo!G3LK&0L>ydAP3th4Sj*$1-(6H$iFb0Vv{wKyR6XE~{@jBoeX zqE|15gaiQdkSlPc5~hWQ`^RQeltdH9iU-+lJY~v>53@hIg-p$;t(Fx#V?P3fnoOqI zgL^C5Hc>R9xVDf%$-DNsi6*#W*uizk?e=|<)OknNfv3;fSCGJDr&i2kvWeFeLkyHJ z(;;Zcgkll>&98x$GVRqRdwp+DMN_K8MClIr7nDPDHiijWks1=&pVfIheB61f)eKZF z0F_M>hc%QrBFX}E(RmLgq;M?a@;RbQmx>Ohhy)8*QZv-klPHRL<^(!yi*#BxCc7B# zfp$uB2P4xDSL|}JSVrPRTcrj#bQ>y{+0BlNHD4}H23bE`+`z!~=I9tK$cow@L*G79 zZb#resrIih7XMry!P>s`_*H%};WfP?xDQ*Z?Ne4(4WX!Zz{+X1re23F)$|*wej)H! zJy>10E=jt_Xnywt&tLvaxU9DRy?|5N!jvyVE5&^Av;9(li63; z!MYghGlcvBJ86JO*6a*7j1PQyFCWRFg})^aWvF@E;}~#bm*t2cVp0?l-NG1vDYB?% zJuEs2&ktw8j_H@mIQ1Qt+u8WRHoI`cR-IsPHe%0Xv5YT*X+cGUqqWzD;%*0jc5>2M zNKrh8Rz3%FKVpx=%es~_KC0*jA6=wjWt9?dDh6*QvERa4Db9=(n9FizJ>@umKU5n- zD{BDrK#^khqRqG{SI}_vlY8Rzt#8*o*fAM?Ue*vzm{7am2SfpQaew5YA;{?<(X$Ap zEQ-`D)n9>}R47Lyq^?XrrMiQ>H+xus^%lm>4w$hepT4f5ORgMP7d^Nz{no@!SB5<@heL z=Y4wzdV42Wpky*`suJMlu^gkM#G7T|0AX_@xz{_Rcs&x1Z;a|ii+Jz05&h~Dhm{tP zlp)c{ALB&x0OR0&&2S9u#)dZ>*V|N95Ncp*bRnt+CgNRgRer`G3ch!|`Lzs(u$=Y1 zXr^3Zy8LAOm@6RpcY%BP8OX1L7YB;BZBJj-0=hnvavkNL1tZ)R7*G^(PzrR23dn}7 z%7YG=O-t{^jCBo6mYk@ZZ7rp_JsXFpz}zAt`Rv}q1x(9{D0N%fTSv9IzE7u9eRd7A zkOanHWWyB;T{oPo3}Js`oKmx_hYd*0v)E@%!^_)2O%I$6Y4fnWnmtu0b0sns!P#x2 z9x!JgGG}^-@zC-`Qzx3IJD&&VerEQ!Z#Iw67QT@++-bJcEFT0iULbBHd1uHaUELB{ z&T4O{5fohHzegO#KT=>YZhkRTA?k{-UVJL!*^-hu7IS;S!o7Cgk=4E+ElcMo{j%n~ zBfyF&SjprMxA_R2t;K&Sy7@C$SJ(4lKvST?=JFiYRCUkoz_0DYS935GS$K%#&1YAg z`aCDthNd1XV@}v(!@sN`0zpUCh54_Kgbjzv*O>|F=#yvU4=~r&ScPvvoFccmB6#E>_cW!dU`* z4kxhQnl~npPV}!Z=?BtYQT-w$16g0umsk-@Ku8EAO7I_P5UXrj?wFYTWgXZ?LZa}J~*j}6n71V1MkRFRYzS|(V08&?zog_jU|U?=a*%ehl!1{ejo3`z|`d?(D5J(&Fr`HR9&nS+-A#OMmuU9dmW^+#Q2&?GO9_g#y1ssE4L`)R%XmryC+VQWOr=`do_;9;b=v& zIj^or-{t!i$;PRD=ducV1;hOi^Poya89RRCU7O%EcTxU{K8M9Qy{Y@yx@Gq!%}IfA zP_aX&_hX}@tY+!m;z?M?)n;W7vB>(kQ*o*7kb=?)jLB3~IprY(Qh8hRXxp_!xF0d% zr{B|yU)$8Md)@iUX-?MQx-6KNX_$mVmgATbCIAgism*#-0nBoo>WDXEMPThm0OTee zcU0`&vOW5N@IX6!-zd0hRAdWnW!Y~b?YI5~;pITwG^Q75_m9-%b$KRvu{GT{#A*(? zQH&~CnV)K3p7k^oHy+02pV821YCL#Y$f!$)azbzr6S267@lH@PDLgQ&e^?yUpcPV-ISE1hZt#w2 zHk7zw_%UBkk6T%(Jn%@Z=Z?^&_NEpfptHOCCz_U0gx&=+K#&RzGgRQ0?Xo&m$3m2F zNCg9--pEp0MP4Bl#nQVqMRGW;>_;10t766GrDpKrL5NABy_zz(EzsFef5J$ddtjC8 z#OOW#B8F4P50RHN|eJ5nbz2uXVk2BEb z;0fHjjTc{2GSyhy^`FcaF3AM87He&aYBN`Fe~Mr$&k^U z^+fLuu93O6c3b^(3?wByWa>p*(`}gS14y0-fp0IoA$VCusr;3mZ3}V)v&f#!Vb0Nc zW7}7#2)Z>`=Z5OH^n{R+`hT7VSF(x`yyhxocG~)sNj_GrUIl!3$6Y@d*WO#(XArlE zYRh+C(b8cb-664yWe^|UMv0{_Hg^7u#gnouG7ku_<3^(}Wf#hV>U@$yKUWvu_2#G!m$j+fizgC@Unno08Ryn=C4pQ+_4oX#pd zrT5r1EL$&FjE#RKddM9qHTvg^+^s5lrNc5WTC_J6TTk#{aWZ6Q5LQhc&*yaLKgClv zw{-Um^4m8Qr2keth5vK$ls9m8HgU8SH?jUZ;w6!Dv@o-`B5yny=RHi}ny!Zr;lW<3oi-x*kRcvx)qw!m{fCCIB!9%=T7Lv+HVIV#r(OK3P7nc;55I0;43EanGLauk$ z9H7Km)ZdE>S>Pzxg&$@)h?`{qDj?MCXduLFob7QUbZ2z|Q(3f=8rT{0c4=`)SQ^t+ zu?<8TjdmIp81^^^#)rjtM;hzBpflZ9UF#i)?TTc>TQ$-mO0C5PQG(0))^7)kMg-T7 z0Fue8fcO@jg@){MzOnCEDe-8Cc4t{;nbbV7RUPxA<_u;xujV5PuIw4d;z+6@DB&51 zZlY#Yh~#pSNi9dVeILGLD9`pS2@r#7%gAOmbidWVzSO*DCWoM+-?dde-B0+u+_gNW zuw+*(_(#0C_|$Wv!van+nv=zy&^rp?a8h+gn3+Y#v0W#BStR@DCA~BD->lsN<22-bs^YE`b7x+DO+<6|H#tG<4!()#s#_QomxuVgU`M7Fci|llGtX%W#s)q{G(P!DVh1NRvHV($|$D>ZC#ZT55 zTUBYbJIBBTbf>|V6*gZL4GJh>nKRuoxdovU$8j^902-;-jitjh&R3LmZ6&!vlDS08 zr6Y@SNq_w*on)*?@#KAEIq~X0l%LR}D$nVvHVww;1X^dNgK(R;YeNYJwXMzQiqvZw z`HrMvv%(Vjm_?!0o$ia7vuzSDV@~i*==D2ow{WLWQJM}lZ8NB5F>BNJnb|mU3-r)Lw4`XZd1#aZ5vIo#+iO zlFxs%ZD};|S{``2vU}DXh1QzX@l*H5czMaciVR;n=`IoEdpTgZ=P+1)0ASY9zsiu7 zr&pER0{UGzWxLT`;|k(Xo(Iil9ujTsD+F_NIAt~*f?J6}TbNwxTXrfKDa>m6z zVYV)r0Tg1dNb2d6`)E8Gl3WX#(VrSOs?aZQEO!Y10d^a9={0O+s*}A{jPogSN+HlF z&GQdsWO9rcv&O3Mq}+vFt8)g5K~@)<@Q6;*?$3=APZ|xeuzjE2$wy`zl61ePa_*1Y zTJk`{bMfC+=yk6>j@UzcBt+iv3&)Bc?B5VqsY$A zaLD!SfPO?s-D7VRZ{Og30KXs0X_xd|13sc-PwzjiTi zlTQQDycv@1!DT(+Xz1F|V|%;sN{1Yh{WJk{A_@D#f@|E>s6<6nXN|f=#eXl#S~7q_DIq=IMeHXk23Xp25>9=D~tT zl#6AvT)`QrTzdHrur?IKOdCY@m0&8ZeL#q3Y$)_SPar*aEwmgoJgm3qPHk^(Zz^~U z*AAL#&8VkCX72^48}tU{H4Km(f{(gaveKatj1O?PO9q8LVa~`$cTEYDgX1B-=7dVw zp#ygO?W5eKcO0Nn_V9qEIMyG;pt%^&0lgRyil9QEVmRtC-(fe-fAW56?ic`Lp>dEn zj(cG@4ubIUQJ%HPL7(GLSf6WqyFj1dcrJu#@lf3t!@0cY!=0}Edxb#1!{H!3m!TZ} zEeq@p(mVJNzI4~zz~2yCv)71Q%h$Q^#zRl6^}Cs%+sMyQKzu9Xkshm}9d<0vYm{h~ zUFwnFERMUaqDt4i(ItBj(Ho!(*J#lt`)63$q&0Xz%;ySGW)o*N-=aBd%qc*(q(?S& zjnj=A))b1*orB=c6@)LvwH%hmO@SzayKZ5!yGWtRbF-+GaGwZh=3v0oGfbY8U{hAr5XSEpen`JF#+j22R&wP=vCUtZ#b%z@(_!=RaZU_z>sc~HV zSG}AZS+wx41o?w1TQcbfRlBvqtEIC7&tIjs6V<$;9IJ71TPf1QhP=`_DWu;?i09f( zmy5{vVRFiTFxP2ZVT-MshHVu~N0NJ<-t+Nxu9rogZ|Y&hkVmy|x&Qu|wB2#amRYE3 zWLLzG9Lws)nwHX>=}y^qTD3JNoOWV^Ww+A~ytE}Y$XHd@Zx<-aG+D6qMa{_32+LWp zykpCpUpUYZ_dXpBYCDvP_g!_OO&n>+HOx2JQgD*#co<~c2LltOv1W-{&SCJ3@;N^x z;Z`i#*t!9tI$@v}U_^NAuta&NqHD(zHx_*JNkETMNPkfJL?^q~`$U($I?f1R6>D?1 zdL5NE*6T1HbZ<|mqK1{0cBICTm2^%Rzx=|d;#Nuu5Y2vv!)B$F%r``1)5As?6K}zj zwUrR|B&rMPyiI-5oCS&f=&|n&B6Bj#WzG*S-O}(PY*SkNpey9h3cGipA8Tu>-NT;t z-y0jbk2hUN1$}Ci~N68SiA0yU z)0B%DZjiv9ZaJAc$9ChAK3pdUV`y>t)=7hLiJqpghD4fBao z-`zn*cm^p?w$d48XOEl(3qh6$D7nY!iK??JY2E8?iD1SylX%{%7wKWZJ-1IMMMB-1 z7jYXp*VPAYr5y~ev&ON!dUWt)E)C-Izb~Wr%!f4%?{`{SJhEn&pSY|&KHYFp#jZUZ z`^T%s^EP{bT}cc=y{5UeY;7A}%I;2KzBSj{RT9^AS&OTYh88hx>0n-=pgfedvTKx+ zQ@`zflx(l)INiA?8j9h?@=q+}qC%9;aroi55oTMG+;t7#>=+HTiXGi_#v83qmNtmR zvXOa0l8h~1!D0JYsHMg*I5_M)Cm-o;kHppMW~;Wk+t-T4Xc-aZU!}^CPTv7eMZ5gi z)xfI(`#tOpe-8NSpQB>RiH+{>yp}vACK$C%V^R*$^!x8&Bn^P+(!iY+v4>CN;r%SE zI+3w)9mv>$IrpA9*2ltiJXS;AJ^o_$%GNUWe#!Hbtx<9^(-KvE>rT1SmJ0c?$r|6F zr9^{DNes2IKxc*cyHrj<(^XAXu4Z0u7qIrg=SiWvq}wk2&lVqDChUX_>w-pPY&lK( z?@BS<1-$U|JPAvdCIdL z1wW>Q1Ir;ZvjdY#RO4zr6JnAE#b^U2%1IWJprlP=rAQgeS7@$8hZmFpkMWf}V;(QI zDv5>-w2QN7Lzol;!f$Y4YUxQ7yj zVg}(;2mEBT8R#)y37nlw_u%t`RlBwO{zdeD=8`@1b%b)mAt{!JKYdc43CEJ@>6<>c zAB-UKPV@94lo{&4ivQiPT<1k`U8Y>mqzK)#NYckwX;=;j%0#I2N6!gcdII(BJven% z=26dVh5ZK7=3Sf5HV|l_Jz$EsuZgnyz=~ISo}%{|A+JBNS@K7XIpgD(2H-1_kmj_` z<6KyIRIST?6d!ycAO$2TFALMx1+{w?i4M^cZ-h~M*8x6|+33d`bRV+Z7{gu=SnE&G ztVulq8wz!r+^AxvFtqsN9Dgl`mRjS1x95`J~GybP)-GJ=r)c&4yHlu zhJ$CC@ZR__)8t9m!O&WemV8{d zJSaEA@0X%DPYe$Ot@uxS3LBvUcnG^7-eBMN!10Fp_ldJ4p~_I9Eb#F1J-w6Ib`|Cw z!m9g?oZ0b8WA}~jQFHQ8+!K5wN;%xPuOp=|AzP6~>vB)IBhU!HkPOA@(wrRD0nSI{ zR|_QvJtia=#w1eSpWz}oyWUs5Ru%xuBd}xrQ|j*h+PU2s!ego3h?D|PY6u%v%5x5X zx%Mh31=Fe)yx>$9Tm5#{;E4d&9`Fk;I6;BKAFhx*kEm;c4;6=uEI1-A6g#S5l+H!= zV@S;*0Ym*CAaGnU1yo52hZW+xL~;(W7$vnP&zqMD))Hp& z<-5;RbW4ZQ2H6v*hHWmCK^#Luwr3zoch0b>uF$re=h&-mpbbAlS2)F1pu{+PX9_@P z45)HOr1|8M-9hedh`2wkv`2fU1^7cmaYjTDE#SJtv)#kN`~nts*Kt0tb_l&OVs}h% z`l+A6yGO8kX4ZWDYxY7XUx;%)U_y&|v1bb~Rz^&;wL*$Z?0K|=GIbh4<7=-$a#8H3 z7(<~LlS4t$#Wk7C*yD_}@a(g*4JxN^?Fr2d93P>%tcJ5zqC&Mevd#jLrD4}1Pm>Eg zo65Rd%eEAhubjDgmydGt)Ob}}oL))^zK&raD_5CG06@!6ov|L=ckaHPiL{YZRm5Iy zuBZi$DBBpxhKYV+-P;0b9w$npd23azOHMi(MnTQRm~t+dfqh6x>Sqy?+- zE=udkgo(s-_A^Bh0Bpz>BeZ?m_&pgE=jNBcH@i^eoeZ|FUFe%GLoScTs zuX_SCi8dV0xIEFZ0NUTuFm=_N=~%152t6Qn2fL-;n!Ur2zkGRZq;ppNKpTdwSK#F=17_yDW+&U z;k%s`$PkE~m-fsuT}WgJ)h&3rXHuDqwE*c7jXZX_)r=@xQ zZ9znxe^#|a=B;RVV+c&PnYk9c3z2DS_~ny^*2xy4`d<=!I>(x|&C35&s7*S}m~u~e zn3|2@aay&v@uXdnvh`@gP3ju5dxcM(la7niD~LWJtF+#ENwnb*x;WFFTF=kf}ShweV_<~uU1geX& z6N$4M&~8SlwnJn)Q*R&H4m>z>+ZJ-$vu}obozU?JKkZnwe-90}L%jdS?X60udy&QN z7_56tz1{b_eO7W}|Gmq8g?!0Y#(asGB8T$~TDAHfZAyCg9q?v4;gh*?&CPV$%`~m$DOK&8*9wh* z(wu6Yt|*=*1ShIhS-$oF78a;YmzB&#v5jZ*RoVcBv(J$_L5*7tIO2V7S2bEtjK$3P zO*G5qVKyhCpoJ!oIZecNX9mXY{)?r~<7V(-^H}*OCmfk#oWvAabqk80yK zo0T!!M=Py7CS;^N&CF(+0{QKlt~=lZ7gK?Hc2XO%upFl$^=WVTaa(KNFh^ZKv~=Po zDFkoUY1La)$^uIT3cc~l)|1=0DLkN z{~v%R4yOd=K)!tw`MX^BpVw-c|HA}A*3Qw!z*^D7+Q8Yu)kMYd@1>KSwX2Ea|HMmP zMy~%Sl8;pWQpDa{`}({B1Vh) z8xnt}q_;X-(to#9`a0Vauw>+oJZ~OITA9Atm1zE zUE90sr^v3+YjB?%-?6rdoqv5DnHxS8ODlQx4+603NE3vA5!n2D{$~Ve{{sU5@xPB& z)RIFH_`$2xtfP(5rU6dBpa__&R)Z18 zq>JobsHc1^$fKE7o<*HYkT9@!RVKb2YG>(ImN6eEQOWW76<3ol;j;fxwAWJ7y-mJW zy%=1BIeaPCcmkn24Q4~(5-%W?bijZPWH@YSdaL|^@_oBWnUYANQ~Ox| z57dglE^>L`lPdP3S}XD3rVU5cyI_8FY1I&7)_Z>erN-G`dA}KywWSH*b&qCAu?S_q zZ{H#cMN{04cS_=c>1_Yknr~BQzHZ&$IOzSa*+};vV!`NNJMaI?Zuz(U^6x}UR?$(y zR{7zZ0$v{!9CtXNxQ0x<78f+UT%)?CDJfMf0|wg&5m_&hQcF5z=(@0HZq`{>$Q%wUa4)a|u(#B+7z@pP`|_w|bHZGMp6^_5HU`+Ks_-D7&}&LZ~ydrh{-;7?J{^p)bomCL!eeH8F|sN-R3C z30fME7BC>lLn_>Z6zIFgcOr(JqiT;h63kId0K^F5ARM%SkmxMZ$DyZ6>otL>A*|Mi z6+_cTJ4Mh|W(ZY7Gv^@dL&5uzI19iN-&nM@y2(nJ85VQQFhP)zr=AT;pFuwV^^N=@ zV7K?)X(_&;U#ls7C6YHLrquS-!mDg~sco-6KsRro+S8efUlFQCri@G+G;}6Qy~_+WjnldDgaN8`_#UV4ZX3ESC^q)bZd9JY^KM=G=h^iXRgY*PrZ4n zKZZ7O&FLFV;n(_y^~d9Vh}l;mw(e!ZA`aXMb%=>pDQV-7AP&5jv4R}^2_`5l_++Cg zwrP&$0wuBCiW$kh@*f;ec4KvzTSpQXX%q1UJvxhy zDe4?e37JbaN7VXD+GMU1rc0pq6J$;b4d_xQ6Co~rO;2rT4ZslAY`yVJw^5r)nvg>k zIY$WP3Ryw74>Jop8~xj(+c7qPlZdwvyJVn$}L?I|A_l!^OtB^)@HOz zpYr6mm4|BR2ChU3vmV~MAL@eiv zbJ7#oB3MkFTZ=@YH2AZB@b-R~cbKhrkk4}(??+I>D<#o0Joz=J*#{i_zEsSVT5C|v zYMRRAp4Ra*qQ*uk#^>y&qoZx&Ok|YW?p<}?Svu#^R*U-Sa`#Jpf09oQ>$6n$*+w6j zN#)C?bVoYzHE+){6wZ1mM+|G!4%B4~=l1Q3-@govDjOPZ6Yy`}U}64SF=P3!1Sa$k zjS1Vi*;?Bf82@kjFI!PZs$U<~SGROJHu=$@#FPMS5{>S-{&@IW^0wf(KT?01J@c1L zv&DUGJB4@v-7CmP*|D{+fTTYB^vHlmj+5*3+r!5V%+3!Hz&g>bLlD6$S;{vd<$fw; zWTus*xB)I95xU$??rMHr8TCTp4=I))gBdL?o}&e>ZL^ISZs$xNM=MXRQs;R#J1Xk|>C4U7$(4gPm% zlWgP`jayv{;tAO?#4@UV&K zrW68Qi@7AC_0lNLN+oYB#0y{?7XE{g|C^^ajbKx=c?HlH5)>e(*%`}zy zPEOf#*ARZIO$ZH$Zis&^3#<1awOHpVj<>RG7TL3q6LT{DW!zR5o0bm48ms z7IgGD6nZn%F)h~Yxu0_k51R}7LYvjs1CTsI`J2+A500Of+0w~+$<)E~nVVCJ`F}++ zJ%eOQl~PB=)sH>X5XA2)vGx$xq?ZAfaJ==_UQexOr_Ey&F$1S>lEtIujny453@WcR zfidzS4T-483|ZcxEVFAkTCDq3`M*2&eYP0n`3A`*tLfY%)S9cln-1(Xsm{@3Jhu;u ze|)VKKu-5q?gTC7%{VpdWm{gg&!l9-i6Gx>G7ZjX<~9bU_c7=jsx!h}FA+lqwirsf zQ-_4MP*t&>4Lt~jf`D-cZ%&|_seZ`%SlT5bpQ=f+QC8QJN%RW`g;&Ir4RsU5&FQR2 zB-W@2FH2(`IF*BqT;b^iKr z0X*Pv`3Mm&*o|P^XCJKO|2Ad`arKDBm3fc${fvXz^7ZULW5<^|8;`BG@F^ROiKer~L_& zh0r58e-X~q%2(+ymRVBI(qZJ=SCgt0GP;4%z7QM_=N#y8Iyb@={=6f`Ezhwp5VheW zyf;*P5vo39xE58s<50x2^K;iJG`bjt9QaJt^^<1JD+fEQJdW$Lgwz?6=S$%QcqwRw z^vMHuTP?|Q5_2DubUr27_+a;|lJw@#$U z4OiZ5W2Of`(3DT|L<011 zPSBk7;|H0!(mr_ z79>z&+cF0uhgiur^lle2862FFk#sx=+RDavSpESjkYp-+pX)*zJ921A#gspXbi6(@ zoIV|A(@*$%-Jl#1S{5AWqO?&OKUx?JcI$(w(W^CEO?N%9=T~edb&U{BX9IAD&zVbR z(!el_YI(tQnLDC5lk%synMd{IxYug8$qI;v;eRk2@3T0vP45IngTC5i4Nzhq#Wy*C z1hUa(93vyHw*nK{A~+^%N!RtV05ie+k$HO?3-K7ru<+RRAKxn<83&@RZ9+@4sB`Gy z&X~|tZ~(AR;m(~$cd`Y%+n8#*@5V(`@lpE)7*?30W!EeIhYb(T53s`tCl(d@#}G2toV3V#+sS{avZ~<4p}|`Uyg)GT3{Hr@++E;YxN<$agGp85U>xRw z=sKA~DVZ?aMw+W~Je)qeK5BCm4yG3~x{;k$`-5NmY|Zj{dhzC?L)EH`WTRAKbk=hk zj>iW`$XY#~kbhNkfyqSmgDX;(z>d6u6mt`UGIJF$3qNV8v z15cjO#vP&)P3CWL2?krXO?3+``rP!W#exDouAQpdBszaCxln7wGS4=9P*^TP+&DB@ zS5ayplA|Qd9OH_hlv4OB*qf{+=yIbM^!$4gBM$kx0L2cTqW51@h_Ze1UaNc|D3Qp| z#vYKlWke#@Z3TkuG+~~u37Mjk08)W`w)h*HJq3lwj^wu0-w!)JC41&qIgC;&zcy3K6Tnl zg==%{-6*;w75Xh#2qSmI_7hp-GvZhmWolI=qv0SuUZwCvqu8Qmq>}X;x%)pi zYj|P2l@D+3a5JCkJtRg2{UQ3$8TZ9ZHEV6$dvXR!YJ`(dyIyu@rj>B0MEwJ`wKP06B_1 z36Xb2K#$^J4O*QZ01EX6TjX68AV;}pA@VK@=usPFj&N56%uw!mh;-(KVW8bWiM%TS z{1gYJBAy$FJ`2K9(QYD$bX5lR&~7q_bkzp2Bixk$c9eTIBHmS_pApV$(;w5%dVGzw z)G!<3J4fJbD6QeWHsF^(doRE}0TiUOslDo;Bgkn9-;e4lp|yo}*1TKObZkN)tigcX zFg!nZWWYQ9f9B(&*Y~XKDmTM`4$MB0lu6*DA4G9c9M?h8XkxN zu$Lf+^_R@c`bi4hKSv^ZBz)unwI!Ap$QwsAKqsFS-SGhRBfaJa-6kYBV3=vy7u*!t zu?77OSP{@C2X2BtBDp8^8iD(Y=<@?hVcuhUuV8qjcD{puito&T>k?n11D{~Lg?HdU zzeIY0KN&$VfCk_(B=)GlIdHOd1GJ8QM1U&@kQVwGiF?p|p7D_YSc7!k=W1?}x@LgT zF^Ona9|@x!;QzgLVD(%Nx^)5ZsJ4f2=i|bjj*%nP$2RogToAMannz$q0Mk6PT?Q(LtIP3N4TxvRTe zuH3*2=;v^7%X53)wxx%28}N5~P$8J8#sLKH1YBEV_VwMP^H0#QdM~7FU+`hHUQ1s^ zkY7eUWcL2O)Zn_L*N8wjnD@Zm-_Wg!-^}X1P7kvG++%ttz;y|)L4e-@_Nw17 zZizrgFnUEgrli9-o zy`Xu-^P+o+!F{E6FhOUK-GROMte^D2U!TOcET4MuH89*;Fwy>jHRQFxUNrFP1>y3Y zjN^%(1!4YAB^izrhi^cjcO)625O9#dH;@d{sfP^lsMMlgq)RN%+%QZ(r9s2KWfPCx+sUj|AV!6imvqSwg)S&*iI@=c5JI++qP}nuGqG1 z+pgHQ?SFpfJEuoq^!JT%y8GhY`|7>eYpk{AdY(P!TyjDq`OkxC7?XH}H4?}M&u~(a z2S8Ghd#o1vTs|p1@o6bNUTJK>z9FpU`h>h1$nnQF-{o@?N*a9>W1-KmRUAhC?kFXB zFqxJj+&S+SL2CS`h4JP46~9&KeHZh*;1DJ|BpZ^N;GTyxwp3Z;8xdwX30J2MG35GE zD>!<#pzPeR)&cKd4~q)Jo>-nh-LXB>%(Z{2{|=P?UF!)*Qyu6}QcyX3` z59BQln}0T!%8p@Oa z9bGaf6xB?7F-ZG)^Tv!wDyAV?Tka}}jCdM#Ym^o`i&6P5DOGWJWUdeU6-MeUUf!0K zq|a@$G~njg?B~I3$Fo%w7Ip6mIs| zRL5}j;=mhI4E&2w+H;CdQJBon7llt>(yknGPqA&vE(K(tqc`Ro_%0RXt?PEC8=Jbu zuj?(d-R@(_U7vSUDbHb@MdE7-=F#WQw0+NfQZS&=X7jmV07J}|c082)h-a?p+|*aW zD6luB-`dahDlu3v{%dq8wA+m|DkK)E$3Cz>Bm-l;KBc3Zz)UM4*&^Haz)J`%Nj=$+ z_*Ud&WN%2v!#essx0pCtTPSeU&{Q^2&t6rbyjXBmk0P4meU$`weW4M(Ii7puHSU=f zYWT-4VEk32HuF&3E4=u;C{rtHv0F%xtl%G$Rm_wn?St51;QZsT87Nty7ku|~?}0ya zZ6^={R$I#>`C`i_&L3K_mh{RUaXoOfgXaoIu?fQ$OL(zu2C(>4YezV+xX~674FJY$ zfuLfr7%lyplpS&^^H4UXt$4#vF9Dwzvq1qo$*f4wRCg?cc`(K;z@%u-$}LzL1{}Df zu{m1EKf&#m7fmPPD0_QQRS$227wG|xJ0<-e^ghJ;amO33;2`JXA{={Tzk=TQV_KgqNCMr~s#5geH3|F}amMfEi$+2R*W6YknTih$JG z8sd`s|23?@U>G`K1^49f8F2yykz>gd4k61GH6k+3+|jzIgbOS;$&)ABRacYZOUQFK zXT!Po5Ln*itn^Z!!H&`|C#rN(W^2H~ek)*}iVy-;OFL5$eAnI@K*8kyRDF1Ft(R|Y z8UpS;J=w&7RE}FzJ+tafgdJ*{iiG(vx!e_O-gmo|TB$7_(ho_HP|suEF4Y_E-w4%qX-(I z3L44O^Ffr#lsj96Y%5AU#>2RG=A8B?+m4~_8}e@^Mi#qTj>0BVs66ef1R35kW{~R~ zSsh%Y!e5MCqnc-7nxX3mo^0dPq})H?ds^Pet&|bnIhrjYwUvh&H;izY_dJ{?Po2V4 zx7?<&6_Xf`7iD|+w=TL!(e4p^V!N>ShnD7DDXB9z_#f02i^Cm-m^Zp7*03Tcr3z%g zv2A`D$P*tOrjV@FA?bG4PJR}Q^D#mvSB0Z#;YakPp$D9-KpdC8(6WQFa09lneDEK& zf~$I-@Cpp7PKV8i?pgLV8v{2 zmkx(ZVkM)9uZqt*xAd+lD0w??mkdwx`r>O-siArP1TkXWuTrS2h(yOhTF48qdQ}4v zWd-3+4!LFY4%M@pd;0w0*|rtun(ZKJLu;R40FlUy$qCnsz36qt2QQ`9z!svb!M(v# zZ*=hVe%$}5>!GjYELY#1cDlQbe?||!LwgmtRr$ot&4p7OEBwo{fQqRrgJ~shK)}PP zuzU9HS1Vv>WT`D@x|8o6foVfp7w#j7qzWXScM7x~kL3Ngka2R>LO?PzG?8D5o#SBM z7yQD^UdETlH>v`tI3e}~!u_%WEV$dj$^A85N(Oi@dnXfl>r~k(Qtp?kOAj2k>V#Hw z5)zACh*@T5|8DNptEm;CGwFkRF*?EUzY>!pVE=M*)8g6dN#$`Zvt{;JVt&=KIMyir z^Nm0I0)fK3otk;x;2lty0Hx1HKY-V9f(~k^^4zc}AGo^G40d7tQ&FNfo)ZbL{8%+Z zmj)q|G$G8BDjqip&w6-D($Q_T6(n(Y3&kUGmoWzmnwb#dt1L8}e;XVX^s|@o-S3ze zJ9Yoqhrv`H6L=n$T!wp)OAId+p)JeQk6c?aJAFUnGmu&WHAJL-3{jexPe^R%XDXS8 z)-C_1HiOiEo(uEkY;JY&SFPbp0_NplxhO}Q)c4~HgH zQPEsqzf-$h$Xv;ip0EM~Dk=q@bIzpRoG_aH94)bc#kWzQ8eX3~c7YCBmMfwv($`36 z4Jgv0TX`~OFp%8e=WO6^@e|nW)aU(9Woc;Eu&@062182@-YPB3MbkVtAL$lq#OoOu z($yz-N7AL$sah_p8RQQSjE9Rm;f^nH1>x)95LxVn#*2|B07-h@Bhm<&QaDP|7Rq-N-K?2mPbhim{E4 z0cL1>6&=}w$i_?^_q(Tf+)Qd>dj>+aA@>={7qT>vXK13g$US5YU*MM11GAbMc~;Qcb-eyTAvTh^2ErP3m*PMS@{5v7)UZdSPHyHPeh)FT z2rK|clTBNzZpy5z28)ATfg|yvQlYRtb^Ofxwz^A1$!Ak&ukJY=NEgp?f~1GDF~07E z`2B;_V_`5EI7l3BY9hwvVJj!KQbCTPrGUb>bHiV`IBP+y8c7R|I#^IC+#(1g1F?!G z#cuqE=^5%%LtbeD86QDAkDW=#SX&DTDS&ZX9c;S!Oa=5YR`?KI$un%2d0st0Z=QW- zyQGrI1HiP3ghkbNq?WQGRnO2KcMqN^SG!_NVl>pVcmx`}|5_6qi;@x}?5>({o3NkM zr2cJ?S2_B9#>1b~-m9mE`SB`n7=Bubc_eY%x@@jmY`l45Lq?xIFuY|#Ie9A%mcO;Q6;Ro5$Yli zQx6>pwGYiyXZKq-#X0z75;nvDJmTC4chc>*xJ~h_N3o;2VDPd7a(cvrAa?zOq2AmC zIXrzq`jHSzYP1004%p-s_DCeYx#q0@ZZWr$skj51rRL1ef~CUCo7gn6r3V`l0}|(i z_hVd#!P_1jM>mMItxuE_9-b+`C=W;+kuKzsrP6$mBEVa;c5ZCDeJJNQ2OTb$h_unR z^p#aF+oolj^s9_+sFF8+;3cY=Rqum6D1W(KZ>=L8QMLt}`G9$YJiQmnIR@76O*J1* zWICT^{`U*N5>(4c2uEeFB`OxYG~}vP-h88F)oz!Gx>#i4_!K$Dk=T^QkN1)-bt!&T z^jj8)G}#TmV;F~S1>o6Ky$zmg>m)h+Jd`BL6URFNN&$@h`)DvGr43-PDPRHjp4W%4 z=|shx+#~t(^0~J%{aW3`={OreqXu^2f@WbcvC`=1DhPS4l!Vmh>}%z4+6v<$%ZoJ=t2IFgGchIkE84 zHuF|tgu?zMfKJ0+E5f@;Lht~?`SnEv)J}33cOx`5)5I0tN~z(6PPWQX@kemyn^Nk4 z{5g-azB~1FWyxCk50u{5vru};zNq;#>3Hc561^GPVF-UU_c&uj$+fXt|IZow@WaCW z3uP{BmN{Pcg6Shp@OlaJs}3pC-UeuK?Nk{~w`NZIrb9YXu9C+bFL0PV3!1&gvr6mCZ~iKD$zC!y#S*Hf~H@hnQz?NV*X6@A4Ppn9l9? z)up?fnJRTKEhtAn(y|~PU(*C9YV!&o)HRIZ4Ic01J!@5g^EqEkZ^d!mcz8>|v7=g} z#}3$GpPH2aQ&e#y{GOS5$Q9i`+2|2s*XpJo>a+uDe!wR??~^pge(HGR2M7=gxm8|y z02Dw5J(A+$$IgY>*%69=6GsEcxhFQtNP|lQM^sdyFz@J{55TEs;n% z7EzY2VW1eHl8pdaJuBh=Tef*N&h!Xbkh6&dZUkpm+9c>CDl9SsixV$Xs2=D0!OJX_ zyE~p}VUC#3GaXt{=3~0}OOfVmzCf17g(A*95nlA`bY`Bbo0WguikoCAC~t9A#HK?+ zakgJLI*ZPEg+YJ;*@^COK326DUL3O!o=GI{G29kN$ z|Ik}~&RlUVj2Ds~r@huHO3weu0@kS9YDiy6|K5-t)Iur?7*JkXk=hw~`5rrDn;;3> z4B%+h5m3WYrm@J6sy!XDWmD$v9t1qR8MEXj>QR6a<$_5=z*+171AwJ{-a&t`eJJ67 zv}u*RI>l_iK49IQGE^5HI{CUeVX3?d0A3qKm+v15{7;r}`FEXud}I=KDF4Xs_6WSz z@GJuZ3Umr=hs|>29J=z;*oZcG>QR2uW^2xkg53gHTepN?Z$gg6_`;+2=L)po^M^l2 z2-HUK^FshZub5Zc(yN0bu;{~ap(=acQO8t%zx_>or&d(TLl_1$+59J;typ zlM6EyEY1co)YmG@?;|sWb=waYlZz&ZzXiwbuYLBY0YOtd#rQ`1&LapND6Pv?2qCN1 zp5^PWmW|^P7tWznc+PKfCNsbmc6Fd}JMhfbe`#;u2aaS{ql zO1yCUBOt{I?a*9Im}L+bc?tOzLUpL%t|2zI6j06&67u*_3`1$?g)MG6I3Meh4fw}! zws&L3<$=d=XIDdyHjE0WTUF+?;Ix`&qDiAMmvLv)HIOGm(J8FoMiH{Nv+o4)`&56? zTmM>)B7O0fo0dfM3#99D!F^mBvtbFjnIK2@jB9x0mgd5GtNaQv!Hs6)LJv7w zTalTWiPc5kXFrG@i(D|_%dldEG0{sIQ|=X`0&c8OLN`n%A2ACR>GFKfk&@MmbSt1g z++H`hJSKK3DqGs?IpQPX#Dl=@cS#A*3i4u7ZRyCZ3IM~j^q(7pa|XXu4jI<_B9+}| z`J<1FV;wEg6_77a3#%0k+||^-A=`{?E$mXlw4Pfh?-F9>v3)4sNz^?@zvxK^UhihnEV6CZ#}k@V80W+Y_K9)iG{UCbgaMJ#!-PL(j^@dkw~Ke^sGV5vi{P{p1w#HEYijyzlRD|-*WVk+!hRy69w9vsvvfc>18RoR@vpw{N( zlCoFat&dWU{dN6%=zT0@6?eIJ-4X_}wR5+2Q6u6bJR5(=32M8gD|X4hLsrvX}2HZp;2Vn!4yu1`5&yT3Ca30*d05gE{7K({AAjz zJ7H`cSBj4Uw4RDE;C8OlsaLhw1&_E{zb2#;7QR-{ zWy*%Gf8k;anjVJ48_i}3MBd6XJqCJrfT_s@wsq^Px1-{|&W?2FO5W3+6J0~3{Jpd( zgI&(&&>^-rGjVJsF5V$Ao>`ZP%RG)_B06RFbE(DFq5D8!bp_{d=DhMwLKcz?t&Ckj zaY+>UuVHOdD_bTjPXu;g<@WC&vE)= z#;K$J=KnNqFHRQ$(LzKp0h$N(dpigrT7r%o@?(N)R7J)*KrAF(SM zTzT%a^#&Qz;2hJL8H#=~eRXGV#7)v1Ryh5b0fNF~i7SY-L>r0k%Vme$-m`YS!C zKwVR-h{&;^2FqNJ98FoM4apI+nJ>3^=1-F?p79Tsk1QHrY36OMB3n9Pso;MGg~Gb|C^LbnO$$yzUd#dMdOsNKh<3^+=RCE zyRcd_A(sq3+k&z|n(rKTP`iHd93J&{zknO>9WioljjX@(JHvE`;ttaq6L@2`r|FEc z+?6j=zneQlbSG`}`wVICF4e=oM4fANVFs*9am8#+K);}E#@HP6zf=Moa7elWs&}7u zJe_aPo!+3aQO1@7N3$wvWtXa@mv z?s);Y3;~DgC(H7A$~PKi5_iK}u7W3D`lpxdphmLTJ&%*Jjj9FG!wl*5)Xp|0*8^gz zFzivV72<0vJ6B8W66yPw;SPzRqXV60)M)Dv^w+~3siL%)k)lJC zM!oI%tG`|?e64gMU!awq9D)y&#$vBd@E&U2SUfX7KW5c8f8)6#du>zS`?=VKxZy<~ z;Pp6uB4!Nm2DiQ=tM>2)0lr|S_H}x|KS?wPy8Ss{c(-{znYr-U&iTqc0>9$9d%gyL z9enB0-hI`v)^>S9yon!A&!=e|H3Bdbt(}Ur;6|*IIbq|=9VP^{8v1P>_E+hf`^4)W z+%5+w1hT6OPPc8@;Ev2E$D=Soe}XCI_UXpLhd{$dqXf~3qXdZD`|kyU6oqZl;Z~E} zt5~(mWk>YS3?|*oOZ|-vvn*nJu-l)9ka;-bYHRk zVM|V_g}BT~ima1>0TIH1gp-Rt#hH3aN&on*XiTg5yCLBLwj=_vrF8eQ9{ZC02S~=C za9FXemUQ=^B~jN~=W>E}(jvfT7raG~2P9Kqx=>~?3m z#qpr3pX?8Pw)_`D|cSD0jN$B_ZvDp34MiztdHJ%7lV1)1vh=>O*z_ zlKRm5#sf_aoo5H!q4BT*@8_bmkxHQlmKkgTQizmlnLq3PnCi*ZhG$ahpF`EAZ2%T) zD)p(`RVoZP2U9DwG6u6yj%tbrTF>2TvpOkg467qxF$|_m@WP|cDeZvXvRhK1?I3B< zl5aM5p&S!jo+FG`Jg_;$>9r-uvC|?8%{dhwI%Z1=Bibj=mv5Z{Ga4SZ^Rm}j>PP7$fwr4BnJ9DY%CIPlH_TN7F zx)~3oHKK)qb>c8c6l^1CsSU(+DT@Z7)C)5A)g7S%-2@rCkQ4J9q0L>xP4L2fWC+Ed?Os^ zUF~&4xU!fdfV3ii64kG`LblkPxr=~wWX=`?MfE37g}auSkJ4jGOM^~^G+F&%C;UT> zH#OKI2+KNaqV(TBDeB(nl1;7(YQ}{>`>7H!lt6Pyp&(88kFBm7)0yNOFwK!yS6~$C z>>P{9-GFK{2p{@@C4vejROyb}79`a)Fd}Ki$2u|1U(LNDs@CGX{?x4p6i^JUSctHk zBA)yuKhv@l|Gm`;P^>B=W-5v($EsPrI1@~UB4~#n#o>wRFCKY_XY<^XQwZUsmHZhD zr{?GPY|S>?my&-4Z>gP^#E0@B*>BCGJ9;0PPt*T4;bd*%#6g`Nr5wuFr=a)=s@VzI za67zn%EmcZGs>l}a6oknbBl9`d5k%;Z6{yJrCOZ`!3H6h>)GN^b4OPkZ$9Ct2*CSg z5|WFF=Nnd72Gcn>-WJ5g- zlbe=E*OP^tj@@%ld|U;^4a6RLAV=K8DOA1Sb>>{Zq;gJ5;1G7wy#USH)Yr_OutD~WvrEU{!CMcVhfKkvxj^t!T+Z)0)ZuzxW z748iDQI#g{5}GFOR!$)2Z-L^aLl9%e;`93Lc?+9dUkma*{n( z@WoSew{(D|0ZWyDZH0US^~-hC8SWD1sz>DY0lO)&%90b^@qpwV>_gnqb>ZaNyu7w@ z?&iTOXyb?XIVWUb23T4}kh~fMsy&P1XzOfs2uDn5FE4o#5p3$iMaOn^io zx^;Y1OEpvP-W&w281K81H_+9{!F4Pb2+#1FjbSJ~rmd=0zRkX`NZ0Cd&gbD=61Rvi;hyTBN<6eW{q0K!?YFQm ztS?3H#h&A&`)f`7FI5*mU%D#!y-m{ryvQNf*W?P zE|L@eMu`?p^>W-%8{v8H}5x~m3d3Xq8`Qp9<-n{cAxuf zm|s5p*qns%P4IIx2WwKRfVtCyp# z+8wPQ$eb)NHp<7iluj50vb$p_x@^*(+%~LWt_bz)?gBm_(X*1AR|l8kSfOw$pTbJm zS|<~Hm%$oXTy7BfKIoQP;IMyhQp2uO!)g+nr0$p=Kr3G1dP4tx7<%uPRLhJ3Wy%rC zIk!cF1g8;kE~|6sH;rECdjy{w$Ufe^uqZsXwiw>)Tccd&=;Q(%d*z+l+jEtxjUbd~ zFsH>FhZI6K5{D!R_tW~8=)&XqYA5~mHE>Jb&j!X1IrgQYIhX0%a6elRD0kzXEO7U% z19Yw&?kz%+eSlam{@UHjMvN<5pq*q()A(@GquWJF+voH35u}Gr zSt+u-rEE7h+zoYFk*Y*hjxHa+dOOyi3@VEvMTu?+3>TdkMhqi~E{|`}H<%`4K4;KY zyAKss#8yp!l}5aUSWkACIZ_;B-VPlT_sLdbn0St(h*(m>HFAvu3JHybil}&xeGX=L zJ|EiVqRmRls#(7hyw8pj8Rp_-HW|k2&6Z4 zbSCB1bc7Kmhy4{xIZUb8u#^9rQ0vmEC}I<1TWZHBcA3A8G1NUGR6K%)k2Ggv21wL^mw!`M%6X`RV%hM2};vF zu8>@@rg_%S!(G`C-@j-VQmYR$)?$j5OE*Mb!Slpss)7sQS7 zj_(!<7(?()TqDFTEv$myOh1Jt6qU0iQxwMb%M0_q@^g1dLF`#w*Tiz>@=0CXAv*V> z;N^wp7E72TL3zgXrL^gQ)Xd_-7rj~d415@XE%WuRuv|>oFBLO@v)>MYV}^7X19pXl zeKareP^*C_dJ5`f_BU~dbTI_6NkbM>v)wwAyv8wO)D$&F(QIwdri|mDZF~)0W+0|L zonU}7XeXUmQq;%<_zKtR2}FhrDdX@Wl*aK(zZ*C?e|eAt4oHG5UQ${|0E!`YCAouT zOedNc8F?M~T(^v5t#?*jf}JKkZ@$er^wf+?T35X==SM;{o6<9!Q8sNExWBt8+V6{wD_J-h*hIXW53 ztA}|F>zvQy;Euvecw)_*G1$b{dH_nQ#!^B*NibpM$U|KAEsNizpWBkTW> zsU*gn$ z{Drf^JnikCdia`pnm$7R0kebepZW6cNP)@gCqE8ryNpp3~XO`)|i34Vi%yTsC)OqhhjMJZ!jD^ zIbIuCWWhAPOJ2)RZLr)rdI&GwARjvaggw@=3K8>LL&WCWo^~?gdhT{@_i-1tMoT}ILg3TY&U+xF2Sr1QV4Wyx{lkb zZowLD?IJcq^x20Hjl&hzoLjC!X8uK<7P6FJ73W_;^ahAnH7bu^3;h|9E2mfJn;uDW z=*x<;pntAk`vP(KiuCF}WI(H38-j zTiOyU#7%DFF#>^Xag`q+z=tSDm7vEZqWZE*{_lGwJp{%UnHkZ3eH; zeY%MPoTeg-RKtyr#sw~0;bZ~@qQ~RQf2%4<=gBF(03r;_EW!4hgv!+OX5s7&e~+bX z(VJ{$=a^PW<)p%s0lad#`Wbc@!UL~LZV`++M8NQD!SHH*(@|m)o5aM-sw<*%({*yA zMk$YsBH!{yMIMW+awFNtc*9R*eSFJ}} zM_CS6D|b)b-5+qepv`;FY|+?uR6{qZpfq}7{7QXwpaOm2fl81`pc?#IeNvFp=!O|+ zbg{o-0w6^RG7xegU1|NS4M8(dY?XUjeD8oaW&T{2_&PwI_y6&A0nHHCx+zJs2VNN6%P&r z!({fv?-!B|4pGD8p&2**nGc(L!%Q=x+AoB81{iAz5)1A;>BZ^Kz280$-<)AqXf9jz zGz&`Zym>tEKf+K>Op39le#_D`k8d)YbM9{v{ri*xIIYv@Sh#P~);>&|Rb3iu>Tk5C>n!7>ae-RD3tgLBOCmU@cK%zkg1WV@9@K6$ zTr$r?jaMPbt&V|y&dVLpY5l6yZ{O(td*iSdxkSKZ?WBssxxd6V&r8k+)81;6rYo>= z?<7<0GUa*4#1f{uPsqnC47%Nf)6xgAXc5)}YJNw0jl6iPz$G?`t9oVXwcrmdD4Uhr z)ay8L>0;_9yJNO(yCDb=7R>C`R%qYcj$U5A6lFHZ*AG9=cN@7@GF2>jdn)%YOc1<>1Zm*k=A`UtF=~nG%PH4mT4n z16&3>I}3;NDNLca%|`Jx@lOZkvx(YkbNTai{6M>DxOowQ8-Odqt}2~l=|XI&4$vb$ zc^eQG$zV)Y7b4xP27-3+iXa_Cv=NIf-FyV+(`@>Rpi68pHk&)%s0XppGjR<9!QY=E zg3Tac`NxHy21Q0^c!H%)KrD;`{VT4@6vhVKc~2gq=QNi9CTR+Jz2 z2^=@giSU(Uon@J+A{3;~v?z*MomDu_U_3iC-aVXOx*2d9?vBgk92J5(PRQJYJFqZ+ z+zkN>1L-TO#-d?G3987|xhG6JMj8(jO$UByb>h6?rjL zLc9;5W~vF&>C*1VD%&%c(*YKDCqDn+u5&XDd);?iWcrUb+HC);)+lRl<7{U5KVwg! zf`-EUZa*@@eB}!G0EU(_YzDX2J=^(beWx@$fp)rQJ&kemkM1 zFkSQ5W=bq2YYHuYm8Y}vZiz*a2?v$wkQ-#U)@7}UT=%d)y79D9!{Q35&9%zsh zA{bf(ZE_Er`x2-Egt^JbXw#tQ=(l;XxPM#G(aggpTKztd&2z@jB4njHtJFhI{$FQn zW{{u}Us!dCB}5ls`aKsZ_t#Voa>&x>KDPeh*+Y_;ca;$15#tin8cG;&Ap*A2;RV_I z=XnKaH2F>P0_@r}4=T|rlzgOe!z7InDhZ2<=@@THr+D;5U7iN;LEISi+26Z5pT_uY zl}0l-axW;^15OBB#0JA#2Vcn|m$2I@dirQ?@!-UaP;pjS4I8gUEO_a1KNOR&#SrVe z{|kwKH2bPTevf#p{}G8<{#WUyVsG}nI^$m;Y$e0@kIugWQcO7Ce_QAp#J$#GS_zDR zATQ_0u1Sq7l2hX99E*1+;9K91N5X-OqBa+DS(X=m`T6et^@DEEZzy`GYDmyPX6M53 zR{^mC_S?XIn@X9NMq3%d4(W4r(CRVQhP7&}F=dj;M+$7;Jx=%{BkH=U-8Ta|qC)c> zvi@bvF<3K)@~Cxq>JnA@H4MTo_J9i$mDI(}!=4i7Be*&_*<#$uF;|{#qKs8ZYJ5Vg^kW_Z!w2M_ieSc_3Ryt>=le09nGvw9HeZl|1nPZzq}HX)EtmS5Qe{q0#FNTdv|Tw z`t&SCngT`O4w;IB*pWtaEu&S2%K4}&T$I}DfzjdJCeVL`)An$dibG|Da2@)=3(eMR z6RnjOiYJ(R6nng~a*JFw@OXVb@%})iGPEO#c%ia3wxf(#Q(IpN%EXxT+xZR0NoP3_ z<7Q|}7~_q@wEG7!B#KRcKt>pcc{c@dI0}P)mv|V4l#NC|%>WHGT7qrR4pS(aGG(^~ zs@D$I0B4DQiaul>N9ldyhLV)sTOceym_Y>h2t#}@EbWLj$WrVq49VF;Nj0l_AkV&_ zY3;P@imI%~VRbCqC`Ih1iICYIrtzXxDSxSuG9A&G15D!vD>q*-rw|UL7NwKpuVS1z zyWYGr$j%rl>L0bRpeQsTufZ?~n4^fs;yUUeqK%?wub0xgd&@b7SSI>NOX1`OoAyaq zku_vit6@P*8JAT{{gkWIlsdw32RzQELLp{r$H&~&aW+c%_)&D1!o@L{vone)`^sdw zwyP`68|)Atl~=#pZVK=(x zb{B!z8Ai=V_{G@t;LqFh041DS4`+9^7TJ5WDHzL z8Q=!IkiSAum3z)m2oWhIO@9I!O@Qb{)cX1#C7(RKq-~~SvUR$FMDdY(kWtA|87YjF zCQ4h%ceB1P{h%p+mAIDd#)c2KBMi`sy#|yBsK)K_0ql!&B;xYpj`^Mor~--j&V`9V zdG)`QD6+G-{6FNx-1(P-HW8Ta%~}Ha#HK7-q7~vYW^F;lV39HA>B8c%is&lzupnVs zNm`=_U|nqa`@Kf(GZz00m4JhQz);U-jTU>Rp0Q_5pR&%&1np#0q7>tdPnd|UH<3Hz zG8XH&1P*5U3qeR3L68$z{T1;zC4l^^L9ncaltKejg7p^#3-Es z9IjWSHb3M+RL>CN1#n*Ty}&A#eyq(#J05SoA4%G~`|)8?8-fb-7o?)kU*0v;7`t*D ziBXI%c5h%g{++1TSvC4u1XX1eT3+OcwjWGj$r&d~+>`<|`s`KusU>|O0-UD;cgS8CK<^QpJ;Fu;{bBl z>((=^IKO3>btjtBQE9g!Z>da+a_|T{9h%Yn@L@Qop`_{GWZ+<(GBi0f^o{>() z6Y}H0;&%F1WRr~P^Y0@jrO>9?bBN&4A!PR-()oxcGhxJOw$SLTR9yrf6qVEZc(bzn zWCD;O7)84z4U8RqFjgEGT|{)eT75vRv?#;#SH)P$&Iq**Gw^)93X~x#E!0RQ7s6u5q{^0xFiJ)7W|^A@MruI(WC9%v8ER-=86Wh+K2|7b7|nE(;yw z;AClE)0>d1Kw@(cDa{Ezd)k&s^|=o1*$nKv_UT)NYMIlEhe6iW#-`2(cd|%l-7P!8 zTY&6dt8qL;Nfw+E3bRUgA7>X5Wsflkq&Cg9OH*tCsc|#VAF$xYn;~fo-QT+;n$wu) z9X@xAZWUVaI>=MD^ZRxL+o3iwm?AWF&UB|!%;rZX;G@d8FBo$V5h_DDqbY-6Nt_X6 zh5FcY5DkO5bnG{*W0k9dm#Pu8wrDTUzt1HcGTCSCA_Zc};vq&aDVLz;)4(?+-A=3; zt;#rv%;pS+r^Lpak;trASxi@tqA(%)Os3L?e~gMAjqWonf5`@8$MgiFkAf%n?rvMM zkZI=FA>;G|9DG8~^s;I-o7uW=lCF0^qE-SIiG4{!x7nX_vnEAX>qAK*Mk`Y)$bjz@!X>)C@`d2(rwR?b z8`krrLIqyyTCXir2d?@4;k`&IhNk{u^OTEwj@*KL4ky^l<6o(Z*Vn20q|3Q%V!LE= zJ^Jy6w*CeEPl~MsZVP>VPjY1cpHl4K|084-LoW?Nk08_?|0n1-Uj)Duz#pJ6Q^*${ z*dR|RYY$w{i6IhUa`2!&wg6!J5yHQm4){qz!S`n!_)mk(l%BvVMlXYk4vXK z$FQ&>iSjCPVZMKK!`V=kCpEkXIA)LiTP0K(?h;38Y1kXR50Pwz`X*1)wiM@Y2D8?| z71^RK->-B=lvS7zR3{$aF2mqA8?K=uI|sFHe#_WoBit-aB47eb-(`GsSE;j2G3Hz_ z(NFqqje5xIVW$~?7o66)duwxcw8V_6t!r;NoM#c^;|-Vz_|5U**Fz5eyTYVBEf+7W zhi;QQ;~n^)m_nf$?_Ym|YV?1ckZJ!7>>3(bej75|(+d9UTgdu*6W@RD-BXethwGI^ z;8rms=YRLhRB!pEQG25b39av=5#J_Xxjr*sq9#V==^KMIubwIY`YRWJ;A;`l|`%hd9y=!UOzkl`d_w(;h4pjdpE=JBq){eA7 z|9t+px!S*)SfY}K4YCNz=g_7hu8D)4se3M4kUqp#-C7%pDGa%8oP8}U5~92yrISI- zB%#R?RyBy*RrfZowg<_KXoF26h0-Z)9yeoqB`rO$_S6cu!y)I^i}BHUHm#2r&^GdV z_>MWulB4FL+@hV(a9cDi?vm5`jNGhTCO-_Gn1g1TBaWOW&M+uX0x9Znf#)1Ze8Qn& z#%)poJK``(v=mYFftlo9f}c!m>=2VJ96JcfK5WMD5jc%#eK0DB&e*b$A*?2UZ;uL_ zK@KUpb)SByah(eW8skcP;@e}DRijn2(Z+h$^qF^<9tk4?b@PRFGYY7C*L>5WOY50R zM%A{RB{lRdi-+v!DHRt`Kh9f ztYy&1>umKhi!kq}qE7q^@7gTWg6zE6o2DxE7_PAYp(l&1O*>l!%JxkDq zbGqieS%z1-l>n4?I%$=P(cYb_Ne}DV^-l|$vwW^lu2{uk)LWn`- zR&(3;<;5R{%Cw@IM(lEB3nFEYpXEqY^!I!7i`wpNs;CY#(kQe>QX;is5pj~aPU8xmHnG_D~#qti|U)KHE_GRT~j|sN~ z=C?$ASY)IKM_jAu)FoyOxZeXxJYGQaPWVTtM{V1DStHf6@+BU$2Z(mCQo`<{roPwR zugemlhQhS?yeUX6;<}KHnWbQr!%a&wf5e#g#kMIh)-fe}XjJa1uLpNcgrDA*VFZqR z3+W8G#{X3N`9^yU5~LyF&d{~XGWNK6Sivk>$0&{wH+3CBN*V%`U45Ou7}itJt(?&( z*UoQ(G-18GYVo1pdHcogWP6Y)<=NlRxofTie$i;MItaSsGw`%cMpE%|q$Qi>kqj)z z-1IGLbG0H{pTCP->@9OcBGd8qbCJ6lCQ}5vV~~UhckeO@d*l;E*ezy+_8bmIWWx>_ zCeqNB#?yc#X53O!-#Yq)-~b%DH;&K!PIl*GK@>ntVVQ+C{)?jC7pYF^2^_`3Th#{= zm;&`goo`B$!(e(W1nIG2CANDO2J3fH0IHpA(q0RUap-$#Mm?1DVnqzIda)C9xhi_g zzM75b8X>FF*n@5-f&y`N%bh9$Ri7$77quauZ8bYEoK)-bQK^&`60--- z)gIH)WbCUg_`VAU=i`J2?z=K&%^HDv7!wO;-WE^=KAhFc?Kf{tJBR>7pUEk_4;#06 zHs6x8AL&U7-#f|czK+U9aA1TNZV%;+%Zk7jsrBq`Nk?nJM+gjV7V%&3`7;37S}YeE zt%LdR>B*3veEEd4ALm{*c{-B>PenBjrT7dw${drX5T8unE!$XEGlWWn*8zSBE=z=s z#TXj~B0Cb8)(>&_wV3mNU`ry-!J~^FT)6@mBy7LaM)YQnq%lq{{R~Vn%Q{&n2IFcTuew;fSS7te=OVOpWk?Yb5A|+HS}%#wv|f#TQ~pu zZU6N2|41~|EY$zlfesR)lmRJq=h`%s1v5)Y;hd0a3{MA3f0>+?KAI*0cI&4|QN>Hn zwVr(?KC;^g%p%KKaN>WM^V=l8xLX-|Ljo;#L*Rb;*m3*m$0ys<=cDX(UpRXLLD0D) zyj_rfNDR{EHd@EqO+Fy1aU0dP50H59PNBbu^?m$B3z*5+n`XOA{J2{aA{iP?4Zj)m z*$EPa#g162s1UxOV6XKJ2%(M@6ig3}M@*Zf`y!u{tPGija~=?|x{qW3;RBK`=6T#} z`wJEtnI?3@5SNw_sTm;72CKT#=K|5BI%=R5OjAs+t6K9JAJVp@TEDnCga+$v9PK5p z`sP+zIAp&HCQHs1Lro3!Mn_D0q!iW}{LfF|(mVL7k~?S%to4n;a62*Z8B5s~965y~ z90hdlDj8rYcQx~r&!nYTzG6CzWpVl4`Z4$l6Dtlu2hk2nGARbhl|t_n%fjXKdzs2TB5&}4Vc6i zuW*fvpY-%V+SXqo`qUO!_Z4E-8D2FGed)S9j%#XzzCC-o>d*dm0%v4%N1pNGXC@=O z7ypbcy?<}Fr_k+B3snW!t(A>6u2S(*HiBw_WJ3VJ)oQv9XxgbTpULu@RDSyTS)>g` zq^%gL7{d*@gxBH5osNmMGO~Ct_!g0Te}~a4?Hhy*4B?VdP?5U5KCohmjDU9YoOP*T zfpy6@Cp-wV1`4GYI+I`<#go9x9`exvV6z2zlj%BR@!2Z=C!Lnjh3Yl5y;V=}me}zw z!-_;5m%{+pQLfH$P{RUk)JN(~^9Du1c_j8NtP^|_U`f^y41-z#Cub%YIy!MuwL8{5 z-(Q7_hJnNT6~Rj8OMYFwiQ2Wt zLP`cD;)q!R$UMT5mkp~?#{g4r4eKE#Of#%7UKRY+4PG&Xb+``2PU5m77ED)jVw+`Y zcTQ6Ddc~R)>c1dNm`3gGM7VI^TDBKmi@g5{0OCu{aBUJUH8_J!Ug&~yB z<2)tx7Wm`FmmlJod3%F^W+2F*VH`XbTwc%`KiPpXtO3|5aopPZDuVhmovw$!T81l% zwsu+AbBMV;}R86Q7Dtr8_zgf>8$heeR}#yCP?~Wr(3)>Y+PLHTpZfI!sRNn%J*Ul{ck@>d1 z_3|EDtLn0pzpX)yUIAj?XIpsB*$uRskTyQYC)K%_81SKA?AWmIBLxe4k{DoU!L6 zE73ef)*L*ERp#I4X?*$>B;4?T9vpH%|9Ev@m14-J?M|{e}AQTU+|qg%Yzh{-*-V ze{;7oRkdG5@6le~TC>IvNe;$+QOqkAGNoFSVcZCQgD0tAB*~1AwsmtnI`m`zc=Y>d zYQsjD-#0SVj{UqUl_5NzbGSP%@5W=cuODG*H4r#d1)g1Q4`z~;SN(oH?&-ZTRbX>i z>n);Glkqm)Y7DcJ^)}13?B+tpZ8q6Jf^tK)2Hs77Q-|%~JRf@lV$K^X5B`wRpl_rb z?}V+}oliNF=8he;Suf^lMo%yhL>Wa%A*&-hj16Mp3MDPHrr(N-l((Pir(LUaAUs>v zm|5X{U+n;Owsfb{gCa|dXS)>;nci$2XtXGfHE#`HvDzO%NzC$jvshWn0M|jfZ2gMz zR+wn6EnM1IMO#0G)FES*e{JI>>fHFMa&I8){E2{(Qrgx@uY#(Ul8lu}aDX2@l3Q_2 z>B$JQpTC&8veucT(<{PYR`bB)KB!KwRZpq4uX<&qR{h-#_n`154CxKVqJqOEdB}?V zB9D6`@1@@DB3Cy!fyN}G-$0psEzUfY8ZFk(Esu=X+4z_GbfscbJ$_5GG9}P!-kADw zRy?yJOYF3hnyiG84YRV}UmsU5D!?q0kxMMU@Wud?qY4abDA&arqO1Zr8@|A78FAl| zCK^@KIwai*7xn31z0*)hkm68=jufJ1>`CZCt_`9CN+f)DwdGa8u&j}`9lV59=R@w% zx>WfH3@1MlZ=H4FY?D~^qaO&|1PS1rIe%~ZcqdX6s&lhC2cA4PQ0vu<<+bm0#o^FUJ{xg5(QKAPD z=7hKWOt?SZ{xaV8AINmfu&>rAmu4jJXG?~jhN6oJc4d6PAL_2gdq@CK~ z-b0-+*Ppj)*TZL-Z;oNFjbFra`jWvx*_?l8l>5=ltY&V&ZWi6BTC`N^h_vm-czv=p zm(6^N{W$QTw?Txt?!8EgdrpB-ss2f~7DD zoq&%cW`HJ^V8R5&0d`jRX89$!lpc}MZL_ z+PqYt3`vw?k2jbrC88z>hvoaZ^p~{--VPNX3N{mE54^UT12g}zer~>TYabkL&*P%2 z5|vMhIx+0e{xTU(@}*id~7g!Agh>a|Y~InlHMM*N|VM$HapxW>1Grgo~O?6Gf2+5!#oP4Ee$)GyrO?6B4ENqo56(tRo-YST) zexYu7S3^BV(aFz5CCJj(K+jPnifW8Y2BmKAYoX{^s!b`GqvWgD>TCGH&{4e9*Hp8> z*U?w8)R$JWp~Fm*LIa;HkjGYxMgi%%MsslRox&UR@4+QT%*Oh#4;aw6{xO06D3Xql z9>YS4;_t>PDSDeT{Y}U;AaG)KAnB4}XcM5q&)+P{Ag}z{=B4DIHu$5aD)@S=e;))z z{%3}x>}=rZ{JSvtPl$!>x>y;I=m~*^6>F0tQ_E0)1w*ek6 z@u`IjWeV;T_q4~WoS$;y{^2`O8wkJsSOesLBbAYc&JW&!c1rIy_G~@yz zmD^n-vFM>-fk0MH)wskV6IgxsRc6f!Qz8GXN2{g!0*GUEl)W)G+_xm*uE#QA4{p)& zDD_1a^keb-bQb{NA-*(i8e)ty|1d_l5*&?1A1k9SRUiwXAwy>O-o^RDLuajr_TIf- zvbVowPyHTkIsPRx$@B`g{-HQ2YGQ9~hqCL!(nsxuEDMoMOUg6!V~d3F zH2xfCTu!p`60>4MB&bR;G8g>3J?K#3@>O(!z`$3b{y6TFI~{#dn-$iRkOtrT=uZX? z=-7}&nD_n-x?)MF5Tx?YnaAfr^(4b``*&-IR7yoHTI#FC9(V$g;C9o6vU$zCyq$ir z0i+6{pbqU<5OxiFyFxCs~$zqM_tU$NERn(kBiD@Yj22bUqF>!a?zqk<7B1z) zZQG#OxL+UgDup1pR!TjIxU}d|Mo3o{^+8z?ZH70o_N0HSk5jWmx)F()U2?nJ6kP5K z-r`mzj-gD3Z38=4c#}Q?Pg1jHiL_W@kEH#&b`iDMvL`R0CRoL_OtGAO#c)Jm2rrMx zoeKgQ@djgJVp;J@9`j`IjFskM3sOkV;9#Y;(X?2=x?ue`lpZh8YWkZfh41z|^+V9a zIn1NN{>)szWXOhqUqcr7w>lN?f9qradG&vH?+a9pl>T6=CTR<&Yn8ks4PT-02SAHK zi?(3G_$%5!9p#THeV2+)qa=KUec&F4fBSL{@wA7;lXViTMm_#HKDBP|@n^d0C5|SNO+z(L<%K3W|BGj5s%17r3Z2g54%94I|W1)r=7H< zTw?PIvDw~tJg?EGfNXeyM(ejh@tVf_#nb*w)TQA$+2Vude=H{k#(n z16y*f&FVSh_p<}G7v4)4Jg>`Wbp$DX9V}DZ0o~|*3j^XZBK3BQ8!0y7{ZeyG#u~Uq zoVB^VxdscHZ~J8yd+DF$U6D~Ml@vg0;kwH**v;GWLv29eN862pdr%C7WuZyM-E6zc z1_pSxhkuo;n=k7w!m5@Y^b`43U0_CvnAJI6msp}f9ND2o*?F{FvZ_UH?N(+}M<0)D zW=Ep7_*bIvPGxGrH%}=q!jwjCv=v@6oWj*Z1LVL+%W)GJ@R)E5Ch`>yzG}D?dnIX4 z_ng--vrCJY_{8I-`e)DM3)C=`0>K(QNsXs($kx)r@%e6Ab>p5Ib-W_U+CyK|{GuT6 z!Vt&iC?Cx26WQ^c>!-`UuyB7U`>>CbHYI&z=Cg%gc@QrVx*tYn?pZGTD|`^amW6<< z6k}3ulIDX1jPpnPc1YW#59KXIySQBW66F+%4%9tr0iQSq^9{=LIaN_8o|F(B(sl?) z$scLjAkq#4Lk;15a0@3Ts1p^*SwQNc+l0>CDWd2I-np~;9!b-g&wo-vQPZ_wYF>>) z`?tpNchZ4>VY>ff9RCL8DiS`a0~9eq>*xvS5+49`ot=aV1ayxI$33D*A`C2F_xt^M zrkm|S90b&l#LqFdv#tAZ*LhkeY@hYJv>4QN+A?l=9^2{`>$nB|e11Xbqn{b`C)Y5N zQ5b3W;{&}|r&)jcEL?6TqVl~zemGx>mJAF^i0MIkXyG0wlLs{C#l&W*TbMAUH(8B&Id|qNq$45(!hymv=O3b}UsJZz6a^Q_;cj<2nPFJEzZQf_7w0M0=hIm>664b|^{R9tUae}}FEN^{mw6El za9ia}2@)EvUywT>@5%5Hh^N|;U#6fM;gmIan9*@_=kUuLrSEuWNI~6dQ0_21mcTuC z6Zfg9BF_*C32j+UCxBEH_)~(;?nr!;nunLm=A-Z}1zlS9K|-97_t@r-*sNV=av{zx z#8K9iL=MeJm5KaSvPk1~{t}Q%x*Q(B^RTg&_{oV=#p$L~6sSg7$i*)Fy?J_2w2rI0 z7hG?AKTDVt6#QLGu7Iqx2cgzDN~;qlD7GJ}^eGTVQYmg%Oo7+fV5#5qx-) zS6>$(F&D&C&qRA|UhHH0LXcxFSs3Jx1U+F-hIbj)a*rYcV4U0rpvJz?^-36@Lqg0; z8r#mx;HPS1Dce~`);Q0SsZ0eIBdnmqT&&xJ9W$^@Jc*P4`)K(MVQ8H9pHV4Hr7HpoDrMi@J2}30 z6ux}9c!1qNE8}42r}xshP!ukR!Yf6d7&{FdgozG%>mk@69dc4xLxNKSe;QWi-MdYV zlqlC+&@0NLxt5bsPfiH-#5lmg!2RfLZj6ndoB*xXS8^}qTDYa7Cgm$o>6uU_3iP2^ z=5s@i9{Uwj>zP|GJ%KyCCk_zFD`Q_Z?txg;W(m#Mx|KDBYE|hoEXzhrmz(wtX&#g+ zg6m^rA()W<5WIP`<;^)^|G>q&=MB19P_o*x9y(x*s@{07GL>?~;clzrTxO%9;9A%D zwg2Z&Z6<2LNCDrR$S{<>W#{3zvX8}ne=>L+OPKm-UV(1eZRk||Uegm!Cd$NTk**r)rO7lJsj&-l)$F{u$0bwVJ`zJ}x z=6>)S8|<@-{2?Edi$BMo4;frdkzP-J{q_0#T=w@J0!Nd7oV?OMUjF^K)oiqvOfft1 z;7JPYR94KbGOL2$^f#6**ISv@*fLMaY0V$5L3;Gg##Hb4k3D}rxjjZ|^;tk&h+dkz zg0@eXw%Tp&4@q}A%}9Rap(1~Jp2+rt(11<8XaMiA$^d2hbC9*gWkF;;iHgY=dB{wV zWC4!QbMR55)uisgu9^TO4M2772I+gidCg~eOr359nB-W@7{C_2LZ0Fl>0;T6JzNB;9 zPsXx^XRhE(-XcWbcgAO*g5AqcJ)d4nN~iKsCwM<>g{s;+tQEPFjlL zQY+O%KC)I8tmKHV8ytoWDEzo`?tc5`lV)`K=;p`V4(K`NF&DY{5fnKD9M4bLzE3-Q zI(w66Vdy%q^CA)PV<5B8ip-hg<~DghC+Sq5Rm4%Ks8{4k#$ zJH1iK%_r+bw@B){!1wKbuhnvgO8On`&dJ@-bzShg05PzA@IMLM%dTI+ecRu?ZA_-hC9hB_30^ICLM$ zDtZ%mCFNbs`_qsk0rwFK31H3~_ALwNrwS=1KGs@p&}pbRwMk)c*f+(xPju31Z9Z); zfunJwukZzehg?xK{6*c|znF&V8x6|(04n^eu_&mn#$<9|)@Gkq|0a&|N@`d3lxAGAZ2SJ6*JG=bz88F(mi zXk%qg7$^u3A$w?lH6n@iZlJtO90UjvL5?!P`!lL9q%V}(!ALW|#sjIUZ-NOIFd*o& zRtay@M9XUCX@}Fc2V9~7K2 z+>E0$3pB1Lt|u@6`I)HMJ~}6T^Swwm9AH2iw>Svn6S4quJMst6dyp>>m4$Jeh0=t- zD-_HOGUo4t0-_hP;U=;~umGLfSfUMpNNqKP*FabkYklEw==++O#YormXC)z}aY{3M z!yt?OgUlGAla>s&n*`RpG`z5mkA^A&?`#t4!8LisaF&MZ%GW()6tOJqCW7&!1))@l z)dq*lmfbq@Z*R)8q?jmatnaD}rr3^@&{UsvzVMw(bk*EtB{7Rocoj44-sMvJpL}BV zx004W`oV63I8aykK_KIn_1hZcZmf(FMzQAinI&mk!AfAdCBIm>8Cg6VU@2p~x1fEr zq@-MEF3J=nJ4`f@isyiHv8Yao!fA(VX^#uErwl!a}PC!k{7%=2d?l!Eh$DV$Dh_%{M}b++Q=IaGP*T=x3~=$m^6yt_d_}eUa0*@97rJ_hiI}PdZ#Pj*7GB>*9SXI z($cJCi#xkf!I1$AseU?Q7KC_MhL>3{f*Shi!HPCBUF#|l#i1QrtFf@TnE1TW-RG;d zR`Ft5ZPQGG(6Z=Y^EQ(c%(Va{W zudRjqw^loryqwTB5a4VW8CC>l2Cr0OYQsRa2m1#Hzv(I`Hv6%@xKKV!?!lCYzyI}q zci~E$r>vH$PoIlSSsDf|S;L8n%VJCciU?Pe>vh}-uIDPF7f0NP=bFP%%a|Wq4)(Ru_ih0$8Oitr_jQPQ}r=*Rr zGrLG0JfU5<%VOQq;>C2C01k8_vLCiXS~8tE?L@IV4%;vnT>M+dK)n zvhm`%RPokA)V=>^8C#5njuOdZB%uVm^SOm-r2cx`CIYMr_A4D|vZQer*W}b3e zyTi$NarelKCf@6gU%!aPF-R?P0csHnENXvC{VT|?QOFM!(fEe25>Cx55Z58{U4x)K&XxRJ~F8p2b>c4>tj?NaQ zuW;cnm=LTeD-X$nnQ_?=nhRg-2k}PA>Jd znM^um7R9eLzqBD(R}hgY;Pr;x9E=B%E5Hvn;K;q3$j^hZ1b;}=+4H#qM z>G5>vxZLB)jR6cUb0n(Or>~N&OhYg%d)Mfl|-Dl;zI86ft-u$ z?J+Gl43)h;@DTg;gT0~EwaMBk;bTO`28>2LM8@|6cJuevN&Tq-}|xaWnDKb*ccH8pcq z9>|&YSe?e*I(VGM&jba1AyfL}$%!a0sKW*jondAWO)>cun2WwF_`fyIZ>H>v3Fn@s zU1Iq{>i+>NydG00c?%>>Vu50Wo3s@LBB;f?RFc_h1U)tv7+~MBk8V@;2yv>faU6nd z@uLdtpgZi#VL*-6wh<(m0#3KNwIgHtB)4e>tX@2OtfSJ+bv-_i%htftha_8f(0$TT zZbwym<-|>zHi}K9`hC^`5|h1;&zy^E5Y~hEI~_CckH=x*)ne71_Wr8~^fhvfX^r-x z)x9wq{G3~5wux-{<6W{t&Y-MX`@5@Zwau48#TpZhvLya?UL%ud6ALR3r^wKXPA!^b zHQ+1?*R|}6%wQ`*&sDQBMPMf?idS^b$W|8(bk9CbO`MI>NOiIgUAY65#-q8Te3J2g z`<=yJ)QdnU$;?A%v3isu8Ef6I3ms5F8Aq7s&V8;ebS=KFMoq93Fo!zg%|ORPsezsR_$cLGO!05ba%=dPzyrhZUDT;7gkhXwxDt(- z1AcB{???DySE%-qMv?0&)pZgj6zAt$QP5X%HpMCW4||hh!EDs}p+D&;Z%><5vf6Mh zIM#*SI?f4~_Dp@b&|%TRMp4Y+`|QlcSOl7{Svu{nM4pf&HH zs1ucP|Cg$Lur>DLp@kU77Aq_YhO<&A9{a_g?Nb^BjXT5F`yBY&_9@T*?oR*ZZpPG) zoG|9GepOI-nRs+H0^mmUC4$g)5;4(5sP(t>ttDp{K>Bcy`Lf@qmvn2_ILZ>cq?Yn& zWUo%iY?Sudu0Jfrbrd#kGgBNy$neeh?z8)tJmh53WR73kNj`q10}7N|gdK$VPPTYW z@Z7FAP26YZdHLO+9^t&P?k$6mfz^l6hdWnzqldUL!b%i-5dfYTynB@+G-)H>c4Czq z;%3;sV(oahO~u~t6jIRFgr_=RZxi$u(FAiTa8+D!no zg9I9iN3D8{SAfFn^a9pCQm&$R@Fc^aO~5gLoMP}128d})9ePxq2SHb(9J2E$)W0(d0Ml_tWdvXZc*CPo-!VJGM0cy4<3`cmg>DGE zRqv93;h`xc?jb|W5)$tkaFFbZpTy)3e#GKuD$MRRfXVLu0HcRV*Fn-TZm88o=dRGz zbY2}`x2+4a7Zbk`;aW(pgMUK6T`FrOrCiq^-&1L%vAqgN!&&5+XO0hC!u0q$SeJQ` z&sk7Fb)C;55HGVFm6cJ?#-l86dTqVLQmC^m^>gui3QLv~`+Ri;J}R!J(JV8H8%VtM zr4F4ep5VIU=h2LgsdkA>)7~LOG*Qwd;DldkY`Y?$6Zy6^cnA>aB@umS(WGsi7|ytA zu1Qk7Vl;(~dZUB&GAHg)(npKVr?Z+3S9(|1)|>=cJi#d|&Osb*U*9SkH50f zdyDUv*xoOyFK1W>Th16$CSqPZ+AewrYWH)`$-uS4n<+BY;X-m8DDm$mt$C_9rRU`Q zz)wW8FI@LG8ARVdojJ$Rk`#*<5g2JX-N+%_WbBB`l8v3U5`s*c1prSzuHwmPv~}Kf z)s#67->R>3R(6U{!RC2|3T5A$Ty=fqSMS#NO55!G4u#GJZL>_)U!@?GeE3hQGo7X% zh(wxOeT;>p!T2++SS2#P#A)8^J_VA^bAkuf#2`$Tl8l~;9(ne2^3cnk62K;;9X->+M3jQ5qK#H zFQ*uo43jo2V>r1X(Dl1Cy{_cQ<5Az6rW_w}zHd?=8vc=dlgwQ^ot}ILASdfYX)uuk z)k}fwIXX`Z8)Xf7YWDl`dC68dLc*CVdcxG12@RQ2;C=g{6VKn-Oi@+ZAN#a%8=4|)m9IEyn zU#!Y*oDs^TAN_B>WVOT>8OE|i?Pdl*P?rf~C-3$-WQyI6uwbLdg_?XtA7ek+CZ#JA zs5lt8pi8#Z5|5PHZ)qJ)us=!!N@~(W^kB)7?j8nRB<(hsAf(e_dv=YghgN4D&kxjR zXbM?5bW%!Wvo_TGPBODujG=S?G>Z&@zN|IDP5iVx!ZmfU6oH1&O0jSqFBQ>cNLihv z{^Lj}NJu=gIrS1nf$X$?Z4keL>r(D8vaP^#dmD!5tcoDr!(*=@SuM)pC$smtQ`-@L zg2+{~#pL`9(N}v|KEsnwI5n}?A@H3bltW0)xeZL-En|j_Bs>DJS9jxrRVdS!2yp!S zgX2zL>|aDld>C}?}47~Rh@*2$D%l%Qv)7|&$C;q9UGEGZshvGc#ja3$>g2v z(c@=&oLJXE5_cWV6WW}L8jTSE;(_$p1p>g+Dgnm(F_NL0#7<^Et2=G1><}%BGvD%C zf(@fweWnj#_N|(U`5OG_KbvMjNBOiLi`;XKp0c1i{D-at@_(JOJ6k0rP_kA z_7wf7RrsPC?1G*nW_VdtG3v~fduFZFdm8G@QRdr&6HCt|%u~ef=mD0^r^3El%K&Qx zM_1Si+6nRw?SM~C8iL-egJ1NNw2e;-K~QxX1)w9dn%4a@v~?T(!=vitm&}bGYghxy zjzIgc%YuXr5etlUspUJFLlO-oZ(dD`n0Z@Sx-=%8N!z$2fNMD69S5; zhHFbzmJ1i6E11&v%Bl?wN;mAP*}G96ReKUknUrr@O4XKWRrN@AMJmHBy$TN9EN*7* zgs|o~_Gnf1iXRY!9ZC)lrOT?~eXoYS+l7YREIzo<#ZoAI$9rTEE>b+=!|}o(VZ`{j zS6#V3BaQkZNMi0c-2KAS;@yBvZQl9u>AT}o6duL{lDi&C zL9f+@!k}REAbt&ubt1DnQRFe^6v;ZaUxc1lP$s`mN@K6p3T{Q}}yn zm`KZvxlEis5sr_dgdU1cx$iW;nNDdu_1;BTlGc3Irl=;xt}cMJ3rIo5aaa>EhUzpF z^^H_}f}P)PTKL>l1MD;!xEFxEfa8o7taW)p7`S-5{39D!)^q`Q2~fXK?|)QG3D%Hd zEp;LAY8MFyH8Y62;U=Bv{_JkJzGwfsDaJlvw1yt%)VztAAlls6dk`UZyTl*pXY|++?a9-h%N)enr>>G;sjUm>zw2!L9fRcm&NKcKqW$3ZN`w3h z0+@^oAa)C(fi~@)UWIIsE)WYH7+~no%b}6F`Yl(%N7AF9G0Lqudj8KLfOzR%3s@rE z6pwad`8RCWP}h(_K~vZ!o$oC3g4+y{}e$y^8SS|LI9 zW=KWuHULUwV}f4^K{=+sL_89G?5v}4)UueH^SboLlxkl!IQEGV$o}@qv<(VM#UkIJw)QvHV#`o`b z0z1Fy<=IZtiatOyr3-l7cZ_8TJZ{hLRlT9Cp*4`}&J8LKN(pKX3IQaMlauo&o#&%R zHd{w@h~F{qh!9rZO@7wZzJ64Uh7e@jEM)*apBqkMlQrVrh;(xRl~ZCIaM)N)ZP2@9 zfw$;!HJ_`&aPWf~;tKtUr-SZ6N33zU@v+_cV0K$(cgQYi5wmwGE2_6DV#-K7CLDDm zb*4c%w^fU*!G*mpvSeQ;k#-JUTL6??D#~1gQ$HNbo$#r|Iadd1EEUI2?Zu8y=rmV1 zq;=OD*(z>)&= zLcpcG!dSaQDplqocS`A)J&C>?r6qmEMJW-`ZtsmfC>W6`tjSqEu0#Swgo6Got5ma1 zv}$b)?aj@g?_$1(WOjh?C3a?ay%hiTO-Oh4BmnPMXyL8golV?WlvRxDFj^OK9?BJN z73H`BY2KO8K2+N9zHu@2CuF@1n!KX7>)+n~GVsXUN9g`ipxn1m5yzyW^c6U4ZX^jB zF*oD>hD$>Tw#UmV>d3o-f4MNKC(65j10a06{`}$reIJw;t?^mxnlv1}mpPK#rj25B z%G9V?qJvlFFUmA?m$hp3Yu?}l^E=NW_P@&;q-Y2}I}-^lUV%P=K?u!N zqOZrQ*Fq#41YagJ4mdKCr%;0LXxOVKXeKDAK%_LgJc}^mE6XLwlaju-DvM(r7pP zs9TVgyoSv1g0KWEJE5EJ$aUgb{8@l*EEL;WWQNT&+YqFzTcN`#eS|?9Z?UC4nbBzOM~CIH`_Vtkk#znMX4_?WQm5 zuC)wZvTzv%rHyG}3{LubK7@u9`DO9RlC~dmth*$7uI_Sq zfKFL^CvIhNO()*iJAMxJlWNPeqtX=bW>*ckJ}XWe`&f^bu?wLPn6aMCPJDo&Ai0*S zosa~M9@i&-05Zf*$(uiHfYvD&k6RxsR*}(0-z_l*PFZeYQn2zqk`hm;h*OD(9T?sr z>H%?%&Izr5!H=AYN29}@4y|sq0%Fy{PibiLovU4sP5HD zqJC>7f45}$|BkPsCZ+~1*8ipJ*QfT;X;}qVAe`9IW>=9aCuMpo4x3;xERG$8fhM?6 zD#C(5mTSHIr|cKo~(XBDqt9Q}<))4CH^NftZ0bg&C!3j?_lP zX0Fsn$3RT#ofCl2Y%6#d0)#<52L;?ZD0fR*dfzLzms%Cto|qr4u#{(dAc%z6MmX3w z*`(@Rh$N4oC~MF1knPd)*qBMojCgKh6awec)T#*!6OVJPrhOE@p4e~a=x$h*TdE*0 z4`WMlUdd>Ve?3HlQQAi3Ux{N*JFL{=UzH^6B=A$ujI;}@78IbfR&QC|C0yN3jGGy` zqDrf9>egC@x|^sMQ={P)PhFOy6QHPgKFg=VPtnL=lB2l|kbK!sh;y@5`qh!s-61g7 z%`1(t&Q2e=9*5Rugah<(!d?_v<+$H`NyP*p1df3E)K;cBoj9*;;zOZc>?Rd%am;^^ zpGi8ex@JOtwF;#N_{aye8ex`96ZR+nYz{*EWIV(o^Ieb{fz?C(8ep;Q+;C`{F8*Qf zT48q9y-ks>PP|QaAU{Z)VxU=fmWKoH3oC_=b+m0C$C&DJHMdjJPsDP1Va40bcIi+} zYc4C*W?QUY``%9tt_ylSHI~UsQHl{;lwxL5BR`~8*-t)+?@kb2+^kjFhd?$!>GGvb zHnjl?qrQp)D5IM^EQ(FOih30v>w!Ph{UBbTOHC;>QZ4M^4SnwbIjx$_O)u~O%uF}5`NcoKrtgM%=6AE(^t4(C!Tg$g+yhB5YVHA&j0bPFZ%0hg&4 z>Ou&06|4KG`TI0SDS^UFYM3Z9dR#uQxUY>xuI?=VL(T4iX?kl}K2kj4>10Wl54Fw3 z8^|EyxI41miyEAb2Gh^ZY!~`;LZmBXPO$;Z^4#;7a*Av4i|8CD2Y|~nNn?P)A;+mY zK5oBRmS<~8rJ@u`E!QmXcqs%e*E94p9Q^LKj|=ikqDR02ME5-}$kEc2_}S8vn8n2~ zo05HLah2RKh(c>aRtqtrwcg=v_BKBO*ndX2HAM)EZJ_}2r@h*fh-FeNIUP1x?h9za zh<{HnpD9e%ufVfz;(k#N{gF|&zN=%>I3#rStjJe()=M7gsV(~%r3NRfpXjEplk6v< zv1bDMm2xN5;d1?bYYCL^A*$$@@X-fV-=B(pkJl_ItiY`t0ORZXn#9AA}EhpHNM|NrrF);G`g@SV^p=+w>1j31vWU60?q2KzESn zH6wkGa}gZsMcRRRQH|`fziS^WA{avQHjkkHsaTXaKxPyC0kH=dw-&uQw zJ2667Zk@|K2m#+V5YjGpa1#%*Q?pK#C-R_1*CPS83G?WPMh`(q8LPARJ0qwxu&k&t z`JD|Qdytu#p{FYW{PpP$3)|RxEP8#T75vytYLu+U|E-9%qp$o8JX!!`TUV3?L_=3p zDvHfVWsJ0%LdJz8+j11d;x!!nQMKgCv6y00y`2S5;mkQDa(v2Ei;5C0>4-`;)%7LC zyu!ZOe4rCRCbGtPc3Qd7q6CA!4~)x}%ZCBi-KB#z4>JqHr?#5Qjn7Y-8Esu%EU=Gf zyBCHzkl~);P~~EKNC)|USbNKu%Az%E7k8J9ySo+c?(Xgmg}WB+PT^3vI~4Bj?!Iw{ z!mYTR(|yxPch0%r*ST5A{j;^^k)xr%NVw!=Q^)8Q`wT>M zl%eYZ&km$vF%DJa!4CPGDiBSLx%EuN4hu)#JRC$(l8$8yOeWk_v?Uz!d3XxDP&xyf zfY=tDzJ?q2jX79(VtHR@QDCwtD+{2+UWjKmEiIAWTqcfYCgOybGY;gg6-*jz&5%;E zYqy>qnyrj&59o8PqRyK4qAvaN+bf4m?k`c}vLy5jUFc)qN=&S1zN-cnVa{U`t76Yh zaWCY$puvJtD0?TBfH|WVU;HejI5{xiDAK7PzEIBU6vNL-Yzmtm;Y+CcVfH;SnWa7B zz(UOAsEK^SC7PG!XOCXPXirm3@ycCw1>@Bvi?V=jMdRCwNF;$hOzK%3LE+Eh%gGaK zR;}}8a$Eozmf_AAk6RT~VGDb7u0meZ*e7>5*zn)$-d3DbCf1~hjCC2sXRS(EDDB04 z9zi+6h*JBXMYP2kT}HO#4JOdwLh3T-9-dhCY?{h8x%O8rLzc&{XKtRVLktXibuV1( zj=Ti*OA*#8nkp<)7AN_B%OJ=$)-I`N#fM_>s{>f_aq~Rmb6a5Y0P0Jog2SdzjXJSb znNq>MP_cVuDVP)iQR;iJ@jq-$mMAcE^DZI0n1aS>Dhk*)H0EL=dk6~QJZx5QZsxQ2 zaL=q(2#_+YRm?6g`1*sY@kUCpkR(;}4`N+oi3xaW=Nl0i&#t>=Dp~ifJ=Ib9#h481bO6vv% z@_UBLpZqeNiv3K=f_USkt8LKgLV#L9?whh+NiKQ99Z?*M6TxA6?Lth$y5(^!F6GFc znSzHKb87|v)TaGbjaj552V`aL`;@4*a0TS=_!h@ATh&Uu8mH<}d`E`tSiUKpIw=TiC8=fmesD7lI^GzOcMI?Q9BbfCDCUojCTCA1Bc#;y*$mfR45Iy)P$Z|KHYNNdE6n z*Z+R1cJWvmsahtn%_QZ~3ug8PQVXl$Val-^Wx{iK))VwXk;O4! z2qNKw1&%!@!!F2EXVk^RzGc?s!MRx$6%px3bOSyQ@Z-Pob$n)bWMw|*th z({Jiv1u^Rl#A+kX#W#t8WTaQPtlO!4+{|(LqjeU)jMHabL+nGSY53pbx1(B{1IFD0pPP7c!pB z%KQc+*bV%G~wko7sEuZQh9w9gRx9XYrMRmg}qy$=pK!x;O5eZbK(4y1&cIo%jy zU*hHrhO2j%7*vjZK*|6c7CCDV!?Pcu(2GX{GCtiHKfYnt2kO&2EYdNeFK@~y91Gjm z8VcIi28znpj#B@TWp!p(8Hxab2d8szR~^cM*)zG17a@-MIib%G2cII)+9G{;T`_zB z!NBZ>G587FcY4>BZ8GqaLlmiVW|x<3Qe+OHac|5O8`2(O9fxmfR}kkt7=#qulqco~ z5%#+SPmCq-R*p0Glu`)m>-z163M#df z%i@{Sr;D&{;TX~|7U8cG%WfQ8yjY0!uzm=$&fh=aDgn}N2>dXul;4>g&_9N0*K0F;9j$h-~X~k)z*bFL_^#r(Nj}*e!$@If}d7~<}?18~D4b#IiNv4Hp?kd674sxRzgeFsT z;*LnQ<~VIQwTG}w`UE1 zB-IKpDeK7=mvRFj+#28Iz-QdMq(4NOOt?`X+#167xH}<)dTJ50g)b66nm{}xn!9mx2osPP zcE@txHxLO{(2U7<(Bc&vOXeAt$u|z1;yp;x5U2**V-nguY>wtX5ET&JW2!{E_tYxY zPvpo9wvKY~o3?Fz0VBzN&)Z>CgWzVx`#du!w~lkSWeMUSL6FTZnbSvfZw{tlipim2 zij1R3dw8{~IgY)wc1q_sy_MM3ewqAjJrW&jcHI+#W&u#v%no6b>C2jkJl+%v4%^R$ zs?tFDbJk>eaQ-3JLh<*^e(i)nvrU&CL2hzH4GeVGUmg+U*k4)w28Qm&y~kwENs_X` z`#)v+SuZhQ=KBL%Ls=C~(P~7F7;-}3AaCDLQyiQW}?B%zSJc?t+aS zs!MtFR74LR4JnvZCE#us4bYg+&_yp|M;T%$|7l(_bwf5S4G<1<1Aw zyZpTA?bv9XV*1xCK%NLUPl0z-^_9%jaITM?L{36P2iCoB!S&KwSK;Er@gkvDp`~=Y z(|yox3k;}0OzyD)F@%DOnjK7C8Jn9}&Z^&hVNbR8D_y5wylqOy?<&KGG>XU&GsbA~ zVXVUah>d3}o9Qz*>_zWP-#x2ugJONR9QeNJZ{9slOzwOV$8+aNRUN1zBRw*6tAO$+ zo~H}UkPYw3_Otxto0Hr5T7$04;NB-DRi(A7Gt=)_JMNzWT#WVgPoJ3rV;L^a^?DB* z-^)JJ)BQKIsHZcW-t8bKJ*3fgYWegP_@KN?4jKdmY~`KPP<7X3Pu*l6$L5O{`bZAU zPn$qSN-@__4mql{HjivHOu8~9Rb2D*uo=KBNxyCfKXcs* zhQN*g+=e&bd0yc+*o!M$-{K6t6c7iP|X-o_m{1%cp1SY`| zkdlueE?)kP@-LkBdyRBE?Xrlz9#y4|pnilodh>O)!zDitXQoFYS_vh&FCM}+R}-7$ zfDwhwFHtZj{=vPpFlyq30{liUsOIyY$psOqpxTwshr;M!@)5Jt9hf(3;`S?F*(oJ4 zKow4za*p$Yq*gWQCUH7?Eav3Yes44u<->1cxDCU`#ts2ufWzm2icY-BKABUGXh0#t z+rh1C3O(gTv);4_$C^-`ctt{3HQ7uxqc`&Um z^&nQBkjP2Kc@g-HK;!&jWiaEnl}q|#;VL=R${6{WOhR_{g@H*+nXR}JE4x_?m6xGJ z7|ls#W`B`$-YN}4e34gEIU^DT`sS9QcDyK}lvrYVU&w;GtLLT&Y1TP*pqPl`Fg*CKk3?G$hfgxkUN>7)>{{`?SnBq&_k-y!|ZqH>=ii3+l) zr?&$2G}skgB&;3yL{Kt^g+-)jmzm<&edrnIep2y2?SK7T6MHN1sI4Of@35>7$AX;* zT6`$Zrrle0d)#G>`Z`ZEYovJuYZ=o-K@A)AlIgSJYc#Cg_9Ut)l@WGumv-R7l|jRAP1gXlGyM*%S2GCw*PGCvxr5hB(06r; z!d`;VjV>8+#*bd8K5E3Nxn7W9NS+3XRA5x&Iy}zU74;;OT}%)e`$M)rx?qgIFu{E= zrGLjwE3MdYUKe+r6@Cprr>(TMHeWS0K6TfYXaZsRCujumQt5&e6XsXN+LC3Iq*oSy zZKCYj@N!UdzDCrp?%%0H#P-;%A#$8~S-KYxv{SvBtM$_jLS;@6xB|u}kXhs*mUJE%4FLaNkEN%fq;t z^ojeP`i?2(K{_F9$C!n@3 zDo)#Ey#ZmzwmEZlls}AcD+;ZuKRn2qL}PhZPV7R(zx9Ckzmp^j3|%M`#;+g(lhUOE z4Qk*uDH1_c-VmSmEFAo^?kZg9JA+jy*k497f)iaio7J)4Z`mub8<0k3+r}tPocHEA zUKL`BgtZ4FwFh)`FP*G{{^s2cta+L;)Fw|JJ|^sk$i0LOQ$uPP--Cr6IbSR|V}ins z`1ruM>*9T9W!%39qfiW3{JGE=&z#r=xd;q)MCSwb=uH;$gFzZly`kVk3<$Plfy^x= z&@hVpd6Gd)8=3pvgDZzj6(L1KnPO5662zeBS{6>VZTQ@)Gz@brS1yT@t3Q-}J1XwlRV-Vw^rU`U{7_*FCiw}D+E6G>ysNcQg8CPe2;d#8 z1EX=1QtF#!nF3RDuuo)z+PrBMfZh(CW&gaY50ceAGTLdP0oSMtz2_9#Ez1SxpwG;*+5uGD2dUWg(Z$H39rV+O_9A#R=>W=kU8g1q zp76=B)p~adjoN}B930pFSSc2kPIP}JB|bB9xK}dRcBoS8Ac@2e$fXHa_$o#6zsVxm zm1uYxXg}@JS<^Y_&ib#PMZ>xkK|EgAcJ2Ba^%aAY;a{zs_bc=Ymu8{V+sbLUO~S0? zi_yGVwlXtG6x@CfK-G-pIPx2EtIbuHD70z>s(0+sSI#9Z!>+(KG0PF`5IfdKCsj3M zVt1(r>emh0Ov*=pmpaK=^v$e3QfN&#=k;`jO`-@pF42`=b`4eBN*?Cb+MOfoi0^KW z1s& zVI}&sK(NSTQXUTV$D7EWWCI?0Ku%(%C&Jr^jLYp zv5ggn7gogbHy$-wqBFjsN@QKtav!5ZMzi>|#%2cV%es``3yQ%W&@ zMl_G1ZmUF{T!B}L@ODE#Qu;)VppilWP5lP=Py}ioxg)B(ZupQ6a|Avi&w~AsH{eHo zM9f5ggF+)(QQk~ZEsR(!_DHkCqU%9qXM<(yP-V+A=to=}ahB@fI$`N-hYzVj}_oNCb_jPMxaFT-n}V3zDckwE*25JH8Z*LlGU>j0_cWiwk9aXS)>(upNnv18izcxLh?D zObWY+KyhC2Ut2bI!EF1j9z=djCB+1y)wH|nM*Wq>)CS=DPS6Lt%}RkQeKMP{-#bAk z-vih>Qf2cWv5Zo4R32S9i_g~qS|XS;;W~Yg z-%W2tnm2cB7!r$p3TNSzE{k1*C z--T9~+=NYlza`w3aEp1iX-G!HkX}edA*e8pAwNDVM&pp2;vqGzkeyFO!K5&5CO@7a zMsuB=W9%u_y0Y&UUqzGdN3 z&!bBn-JQ2-C26nK_bT-j0sKvA`ox#m#Y!nOOiJwu!g*$&9C4t*=nkfM2Cp^@GG*`j zX_aPCqJR1zM`O0(#T~ouvK2h7bNT952*sRxbn2&y&0r1m4&SUB%m7*lS_?(p~9tX!qOQ> z)WO*W;WdOT6)nHC>TLi~>=T9$!`_uQAwRO;?SmV6cC%SK$T-NRURRGb&M<5c zV;1EzzW(EgK0Oq4{_d%xy_EBnNi99VW<_8G;vbJQ&+>j4K)5*~2Q`BKZVH^M^T_(~ zEU2)C7a$5tINn2xO+8wtP!`IQAGc|ss4veDItj%ii@x5}XlCmWL6W_x&m1Tyv$r_} zG=-r1=t3!3<4jL{u824g|4i=`a^q960s3SL&q~^buaA3b)d03K{!M_p*n?U* zl0eKKve~MArq7Y|L7|{KUvf`A@_-ZdC6*u7L7}KSwCwsU(%Xgffnd?v{<kZUiiCjt00efeoQ|AHL5k26 zEtF4IB$teYX?E{=&W}HhJolY~bSp&m+dNCk%>@M8=N~JNCQ&!i58FjF4V97V`3{Ue z5ODfag1Re~bW;}Eowmpx2f2K^!?b3E_pI~G|EP9=r1Pc#1Au^-e=$Y>b-}>@dC=@X zO}+mygBYu_@zuA1!e8F3YumMRE%2;VJI7g1x-DErN*fe^`dz1RV~b)c>ThbdmeN0& z-t$pqn74;=lo@PnRM{&nrC1;n=`yCXePzKc(ECV~tWv+WM8ZR-5p)uoXEVf!+8ORG zSb}otDd&b}^RgK9BJukCP*YyY=V+;qadey;j}hS8j#`fXT(+tuWlF8IiIh#~iS>7- z=xuE}aV{oliZDIWJU##9s36yK**y%j5Kh$$$)}r&4`oq-4v+n|gN=)KcqfW_ zc}yFLLtq<&)HL_CAtBgbIJr;?HA~jP&#Wm^`~$50jYPE z4GoLE$a{?}yA78WgD|oC(=MSRO&%R-Wdy17pWQ@L2mB-3|a%a;%mX3S17?GqoTZEXl)l z!A{WVNR$L}<_SWjp^WfbI87wB`MlAf_c|vtCJ27oa)sf>%$$aufR>;)(y|-2LUpAv zwx+`)ChVWi)ehB%9ZqfZ?1|r8BW4EqP;wMtt6r*jSz+e;YO5T_>S6V#|A8P<=ZFXS5jfFCE3&$U0)x#W)H-L>Px32F zbj{4r$T+?Z8pV#E+SBksqPhjhK;0Ijw;t58T=`J3tg+$Uq_Lsn^OEItefjW6LB6-( z?;l{+y)x;0{DmT#_BeV27(CCYzJVBZK7UpG*$vp*d*&e4w(KEtbk2owuH9JRUDfR| zaC9z(wXEGxalCI+hOG`NWu=>;>)HcZvph#B34cfGbJTB8=GH|Tt`D|w+~$x&Ntyg2 zk0NK0LJOvFiXq2OYSly2lmf~~>rq5jQaHtu`_o6-QaI(1*Q5<1CbcS~aY_R%q%RT% z8IoGn(ORT|R7tJ+XdY5P1?f$SNd6?NpJLJEIuvu1k?zub6p={Md<>CYNvl+mA4y!q zk-$jG^59*Pr!X3m43I*)lRi?D{I-JJAbD_x;9VE~1YZN)!jdlsTwR3TlLvE3WSrLBWVQvkx1}4Z#-y7rV+5#R$a=#TdoBE>ajh zn241tzV8Kr(X4I?sgQXDiHaF}$_PORC#FS|CC=oA3`%p=m>xN12owQ|wRIJV5&t_h zGP0+V$qhJf%B~S_it~;qlsN(=E0;95Db6C>>ZB2TaJ74S%rP^}#-15|Ke1Uj4gD+z zy#;feFk~<;l;wso?hz~36jG{Q<)D{)dK?Qc16`gl5!43`1}j&XFl;a^0_D0f<2XVI zbD2q9U>_Q9c90jsg?m}Q4)3I73xUFb3a?tDOhxm$8q@;zBD2=yE&`MR@8XS$?QCD~ z*sdIu<(e_5f{9}R0l>JB08luERJxAtdpJ#)vNX@`V{xlP%(SfSfQVF^9s)?jM{8|*5xI$9!&J?R%_JkEmwe0o#n=)QMfE=>Nf&qHm=oFPmR ze2QSfnm(|L0aeOcW>J^WM+#NSrZsMa6MTq};;R*8i=&r|EPGqv%Z1K4{!Cc7J->vbL|*R?gmHL zEc(r7MeUe@Cv)W}j|mgLZrW&AJ|#*sZMi{SGpB;O@q+9=>1R%DIY1J9#W!AJ2)L#(*+bwbMu^9^%QQ{DvuPC5&m#%>e(=bUKQ-C=7#WRgwVPM5fIaZOy zQwIR=)IzT?;$v11@icBiBS0B#zR4Hdml5Lu2-Z1j$Zo4Ls!Qsl-6V@{#JR@L{e$nT z2B>uEq3>KJjpm`OJjQ&v38cpuw(X@ds}F;I|GM8;b}|gK6}I~+j(wdbPUf=R>xT-Z zd=+h6Ne_(aL={XMqI_2YWVNjg8o@_9F|POrK8*vzpX=zS2Q1B;1b$Bd!8-K;SPf_X zA>rorvFa^3(7&TawDks?qjB`2c+H(S1*oHaLK23$VmycnH{-+2>XVh-fh`T5Ac9Fl z!K8|S_y~apPlTStq5aM@MPN>YrzplKQ2u*2g|qoB$Mp!)jktE_!0%her!J~L=C|D^ zbkWGI*P5H}WbiLkzWM;e`l5j0fH9!$dj@^?p7{r);Ljn$Tg`u1r>XRRr@=pa-^^|m zSv78%pgxBcQ9=67vNcY{Mm@aASOS7u1OPcQfSMPhGPleENcp-PDnq+%dC^Vy3V!MCka zyrC`b8LM<9SM?>>l87ZAx?w7*|Sjs2#`Rg}~g z_jO;gtad;~O%|EWA}Z|>c@l4LX0q~}>6D!4sJxHpwL;yQhu%<*3x3P~SE=dSNvyJF zk!vES#2Zn?T&5UXjZ0<%XPH>JE{%Rd8EW4cKdBb(ZCC-~o|ICKo%3;1tN~9psy?Di z(r~@hCFT67B>5ITLP{t`v{Ffz7FhAFYbbGNjq-=pv6l=VH7j+7%5h zYc)lba~sQA#}{>RUz6XA8FFW`mV3BPla61cJ+C^dRu|=;1}RxH++A2Cg4IGZw6gu# zsvabMQsV)e)X8l!-(sn4Q>Kc`2Xxf3tKqvNB(ff|%BNf)#N^`}fryAXGcDPdoE<$` zXl4-=4q2xP-tD337)wjs-jD3s&Jbqu*X_0B@3gA#+XTC$YzAhirjxJoe$#p`lY~Ab zjW|>0cuO=Etj|F>QbhcESr}u?f*@Hcw zWnA~mI9D?;InHjI!-2hu#pFIHVBZ77eDnv%hU3z1*Snc@S(6d zWwSLva1w&HgC`y3A8;r5CfElkqoFPiU8e8+B!bA9bz0A0rrH^R7^@f!=f%0ae+u9gu9ZZwsz0O$hrU8*=g2;6@T~f zwCV7ipn?Z?Wm}f}(+TG4LSx$6pY85ycb@o{4y&sI^Tvuw3$oR;GJSlyg}*NDcCQ93 z`B)ClC6$56a^w)vZ05DEG^}_p zbxj;!myA&y(c%HOK}g1AY{<$^l~{k1WNt#^PN=GMFozV4Hz7@#sOK}-V(91_2AOHF zslEytoS%+!!Bi>@WM+|Q5?#p|NRhkiWQ9osy(el*=W~c{LQ`vtuADTU)0_!fOT}2bo2QsViMW?(H9UY}r zzm-5NB$>0y;PSBJs{s--Phym#v6#~*F31R%ZYo9Q(m4ln`^tacUq8r{aG!PbMOQsa zaO2)`K4az$i+G^(44gT3b*(Af9+gb3oi{I=6M55FxL41_93=6Ywy zsmf2s|3J^Oo1#OKRSeDrx?6hJG|%YG=JV=$KBVTWzs9 zK?-^)R|>}TyF`j>TdK^qYYq}Ft0&gRwwZeAH;1(w`WO?PZ6~OupQ@jb(`!n)jr@Cb z;v|uGSC}_6?xrfqoX`K>PWc zBmXdi`Aeb3!9aXN3mqXi1-EP&UB&Sd^yCjbtUp|*Yob_~GMq7k?2&mZa(%=n#EE}`ew5~(v|6Z) zuiXsQ*qI4CyP&P@w2Qb}=B*yDKa$^@Lgv(FUAc>qWRmBVigCQkJ8$|y1gx>jLL?qt zBqZ581Ccc4o*J|0pgxO8{;24&xzl4u8zrjat!}YdtE6W@6nm>n@~sBBYpeslfAxD$ zJ8{b6jfck?hU6=4VBHN2a)}FPvJF5&;TXPvMnFH)v_^$NMAZ0Flq2&T_3S4#3nN10OQd!eT#Zc4C}NauFSbpIHBRfx zW!p?c{WhaAtM$x~BJ1d73;+d)X}zU=V5b&PZEh0?Us3HtRsD6uo z#JYO)n64LIx=7#_|M<;mGL)Xa6JmJTw8=YYp8b*2oS+>=j|UopdZBp&{`}B%@`9c$ zbHn7Rs&=%rmxq=sXSmpb<812XzY?vR?YeePcPPid+VN?fc!xG{KX)}hrpqOwwt%faI>(U z>0WfX3Cdlge+IU($Lsv_^2PyhUb7BKSn)}aPY~IPV?wd{*Rp|tCjTfP!N#fPBiJ@p z?+t5(L+P*<<3O9?kXxJ^iSE?m$A~e*IF0R(aCIciw#Y{^9bMWra9duZ@6}{0B&VXt zr-#ASH$uFfjJuQe+(i}ogCBIjTvM7EhHZM0`qakOv{(jQ2>EF|=?7@3^5@T!5sVgG~N1=Pu%Rbcc7v56raS(zRcv$aHHu zC1C#j8*`cte;y2Lb~tYb4C{BF3;Dh`@P$DHPj3;)Vb8*6T!_9 zK4YmGC49q(_dP_yA-uHA59BKBc$d%<{3=wj{2lBnyr);;nUJto_=Y$xEZh-{FXpbS zDlEs7hcE27cYhV6gBhgbR?icRDVX1r;@dvInKZ)<24NtQFKF&Cr!muCxW~O~V?lpd zXwa9Pe(SJkyk0Qh?jF}>k>pCm$Ch4DRz6K=PboJ+8z?6Ai860%wyJ_N&kBQi1iEsb z#ELZb0l2hK&Ju8&vJZ?)(F&b3lg4fJh$1yj!c(@Tb6NG3k|Sl6yKk+?qIpQz`np_Y zm>kznjS)s;DpoDr|Lta z`#{x}m5e345r{f8LGd)yfW0TeW+>7L(=t%>b%;5%>4zFpu|7Oun8@}t%!j>IFmEQw zXjeI#W^cuSG4Y-wc8G$>J@31ar7J$nt_))>aXQ-ebQc$#^9@A*PTVEWUq>M~Abw{5 zrxY508gG5NlHh=c+amWj2F>ir_QX(LgkZfZ*TTm6(Xhp$P~X8>(d3_jG5bd3aa3`} zojYcKhV{e7sL;-^#>S}5lab{JWBMWs6WI5pjfTdQ0x_9EN>S3vPDAiw&^6F(va=(? z&K8yJJJA_tEe3PNS9?YT&sUdVx6E>NFN@HE2L{8D?PL~7@L>{k8-Xh@i|}NaISeVzDz@Tux_AQKUrdYX%?$Xj2moc`6E;gZ(?tW z5G!%Zuw$2b@+wfdIWoE?kda4`_Z=!9(!IHNe~~@~wzVy3re&8fVoqGks+eahth&xA zU`#muH+YmRQg0KWmEq%vXji4y57}_kcQ2qjYg99-st3c>Ft)O~j_`!W%Z9gQ8lHfc?8zSCEqC zzCVE*m_1G*&uZXvEy1?LD!j|a9#&PYY3#uYiWj>;b#mJI0of|$`ko&g_R_e|q^S&V z=Qy7Pw%4W*pUn-83*oid4fXf%_>2)|vUuGHN$N3EUKn;_tr7=2HPn`TPjyv`^ zd&OVlGl|dg<@#R>oDX#3IvmU+5_kWWX7%NRY3sW_HY{T& zJSEA{izm}$|C?0N!VUY@i38~mYVwi%GpD&J2O0Y* zlJmb#q0Qvs(fA(*m7`O7QlSS4_r)>CnZJL+Ws!{-yPyPT>FjP2C`pkt$|EYUcWl3W z$PzGDFJF$@hw6H?!M`>4BA?)2G;WI`nQNNRejqs6bw^niYw@Av8!66A&Q-dbXe;TJgmVaD%2 z-WB)F?LWk#iT1*=_k@AJQFl!!y`1?p{LwAlGE1pk5DuAXSwW)cum+FW!Kyb5HX~fN z3Cc19T4wiSgRaUOJGrJFUH?o+cTZrwAlv;pp{0z-IuP(l$3cHm$Sxp5PCAo@54M)5 zS`fQHto)IAStG*vve&^^kzDz3qA+c@O%j)VFo{c*`r;5j*-2i%jMnchPUNGGCwnyk zVD;*489nZj7r4ecSljur=UN@#|MUxcCman7{CXi14e*_;7&M^3@srN;$M_0$RRDU=4`k}Ozqmcc_lMUhnty|n zZd7kDKYXwA@1{7V3bPRGEb%tA3eQb`%(X4F%dY0#wezoMC~MZ{X0XpB#ri<~b(#$NJFhS;1VzB97y^)N(!VDR6|CF?Cg*|KQ#?RU2%a5H zJ6UPY8auuJ`$M$HkNxNP9^oJBV7`F~q+YwTPdTCHSM0|OUWyswOgyoO7#4(JF*1C` zLoL*NSc`?Je0j%CLeRsGqiLVJVCvny zs#mtt{9VMm-2#_=EUFt_{@-Oj7A@113(|&&XxM-ZnsaZm^9h&-&ZDaH*s5C6WleN{ z{!l*HWzsS4twN~f6|^%dR~#xWTV%+#n#7cTJnbE+DhJZ{2iGx6Sprs>DbyXJlkd!l zJZ5GeAoczUH?q_gvp3h|!TTc{O9YN+IfoOx(I}SYViBCBJz8-p;6s4cSGhFZfQWZv zx=q=VfYe8%m#gV(UX`Sn{TB6lcsmnLt|9bYG{t_;dgzh}i*zy_qk5}Qwz4jjrU7ms zP5brmTx`V<1(}pa_jRt5Dxr(HeQ_(ejVqO!F@XY`feR-WzQ)rHpfxB5zP@8u?xDz6 zL>hsUXE($wNWCn~Hgn%PRQ^q$5Xqej6*n)ABjt|S-O`V%U|+gQAht(ggzvk;OtXTc zi>3$#w~bJ`3M@3~bm36@L#pz69CWOhz3weejR>KV z^aHyr71#AKMQ$^Few0}GiVLe7?f7TnC*Qu$-HLUST2r2Olw&I^k}fJgm&EaC^r*Zt zS8SNRLYQZOHJH;T<`~H=&*H4Pl4c2;(aA&FGQ)9X>3sa?0RaD__eBt>4X;B80>Y;D z@1hG*{})#F{|nmtSA*xN|7@#8@e|)ve+Lr;uHoS~ZOcokF|&|lMr8A)vQ|;ZcF^d9 z1xy7C_7a)ngOVsSmW*|dWqCPR%*W5Pcz~pFMJdNAM>Qy&G*;JWT*pY7Z;brN; zrUY2Y3cKg($Sq|!3nbF2zNQx8C>Fct9?U-cNimC;SkI#8V!}&Fyi8nL7)~+^PPVx; z->Pa<|7=Dssdd#lW$cCoEB-yvRl=K`Bt((#cy6~t-lo~d#f!#e{(VdDM+DuJG)e@w zmy&3v<;+6N(NLV?%kGj%{1&M7RnN#2nP7gGH9 zNWmBFN_CO>i*|LRuCJ7U={R)F)Xt?df|H}{U?${hY2@0A_dr{ek5`g)RkTL%SnIWi zd?U%78`l3lA4qD@r06m|mX++1^!NSY13J*s^TK#6_&Wc}IKz0Y)`6T7&NC~z1VOC@ zD`If;S87O3`mk}`X*cd3Uf8iHl*3q!jC#RVk#f)9RO9K|fQ{9CeZoOK$-dw0GLi>= zd25Pu2Wb+(cUD&E|k^BsPu9wKexM~(7!Cw-B!f5!+? zum8w;9R#dBh2L3W(K#`eo7fwqN+IepT;@HbZ>GqJv_Z&E`M%%gZ>@>5U zmS#|kn)9AwldP!ITNAfVS57lC0>#j&2E$L=EW$}fLeh!O7117d}_P9g4#p z3P~c27BTO=!1ZG=Xz(CL?PIb*=HuYKK<#7mJ!L*y?IUFocu+7Il=M#Om$n$R*qO4} zc_4$;`m33eA`=G(qdxMjq{NI8%)09zyj=;Xn%(#Lf&;ad$!s+VDA0s-e+3hR_U-@` z4I5`zGH@S8EqNZ*#;3o}zQBNHPt5KsxguQh~8VgNW@f|-dC=vS5t8fJD{U#;M4 z&k6k7UCH@>yet1lh|f=HOaYt)!L1$f&3Ttd2bMXTn?$WBF-quultzA&M4b}HJiFo@ z7g``NUL;*SP)H>4&xNJn8^sJ$ydsDrSO=sXW}SX-T+qOt6`E>j``l6PECbQ(pqQ3q zR@VVw2EC(=LvNi8vNT`=FOLpOr-~gu^+tkC6;6|=+zKhIuwzVwiQSukHrv@bAI;yP zlT|>sJym?f!3`2`Aw4-?_E@^lU)&dUU{jsj+MT5d?zKNv0~b<|bM)8s-by1R4j}Kd zb4+r~t6R?7D= z{F!Z$R~N!s6P@sJGo#V+ir1}2v9K<3qEq^RMnT2UTEGCN;pej{h*c%ua6L@Dm zIGY?6vaLWqOSKymC=6Nn)t%H3B(0~+9D4p$l$37-TM?uS_A^o45swVPiCxnOTgheU z|6%Mcqas_EWzojnoyOhW-Q8*2-QAr=3U_xc+}#^@hsNEZacCU+vDZ51-Fx4Cd+qaM zjyXovpIMTTnO{am;1`*nwCpDTj_Ga8cFr}*wOpqCB{4+Rz1N{u`HOsc|E0!8-EJ-G zWx%m{+9uVZU7>F>3@4Apba6xdn~XF)Tfm>+cQ+4+j1bQoaDdh`Ex25*ZF8Fb(z>z= z9TWmmF2Di)M14@FYo|}m7!F|hNw&xu>d^pdgZZ8PGL5eiwQVb!YjRev3Gbt3qKBYd=|yI9Ysw}fXdNxD>5N$dQFiLAaJBH zy4Y49R1{K`CeZLEOOo1 zSA48>>PRR9=IqOyAU%7D4=J8el`K6;xT6a{vqiS@Y9J^;u$&6pwCH)oBfH~4_Y zEApOIn0rKBBSh-PaQ^G*FC#v7zEBvSPkd$lN9nNtIgb2aQ*Pm9b&7uyx_a(s}nQ(%=aj5%X7!cI=WR%=vR(S>ld0%#*TWD9x z3>$;!AzlaOxXi-53#RZ33JlivK=4hV1B4dqD2C~YPmkCYt(T_lYi~F{*Hdmv)71jv6WJ_ zTDn4W0;i#4h5oid6xHBd-$%WV1Ov)ZzmFBSJ0I4s9h zPoM6m26!<|c5TH`24YB{6he#kd-ts9q(N&*xtSTA{Y$oVw5nzv(9Gq#5MiT=n>FJ7 z=|BFQl5=yL=zPwtvt>#Tv)@>qy_X}M!6d!nv4MbGVkcY7qBwZtY;p*bCAgd^dnmVC zjV9AMb!seiLm7J>p1wl{ynI&e!lDAY{6*cXk=n(#%Sg4 z#c1N-Y{vM1gQ}dBiJ862f62f4{Wp}R+Uv9k(GnS8fFv|*x<9ip3wt56r=v>=iL(jN zb&6^=Y}%D{l6AdO`YvMWv!jG3UW57G5SDwSLc(*HCbB;6PqurQUwyp2@8k8tW2#`w z@793o32O<9e=jiPK!ET=wYNZ)WC$LF*hrkEEf5SwJm5(X(;vcyRFgB(6h{$7mH3vJ z`;9pa1miB<^n_kP<8caX!BM`}+!nI`#ySfAX7AnxvinKV-p?XWr*H4iJ>~8t1D0!9 zE&lrjdxm<%SJRTU*>z~8fSJgvik>nY^MR&^wp~0t+*HOk<;ROz@@)=vQk!5xF8Ps1 z{S~`2l_5>9h#KAUZ`g@(cr!4g^mM%auzIR|hX|Le8;yt>ZRo z%l$8iSd3HnX(Z3?7%W--RCFI33DIp<>)*y~rHlaQHH>HGb>ms`jS^PC;b8Vg1@*)# zjefI}_KzC)_lxZ~Vl49hUZ)t0#`e~+i0tl2asU3P`w33I+^MWoN8Im>(G@06wbF0j zb_^B^c|-ceV=0&w_Hm2ET`;un=C(j27Bq;mi5znG5|^uUNQzQo6Xx$tihrnt?-Cg0 zQea_|;fdf5ob*FS*kD`2ND)(*H9JW8wezJ$2Nj9YHv>y*RVm9*UZ}Mf>t$)0-G0ge zq*U02L)c23O2z+(dBofG2#St8Q2RZzl38AD=>Obvp;T&>X$p{3gx84x2DyLWG(kxHgbKLu>I9Casbd3l6T%c2e_%aeS)aeKe`+IJLks#`(G_D_TOFV-|^k8 zZln5D4clMAJ_|tq{mYtCdARm~vvgFGO8L*xp%OrpZe()BrY)k8(<(3vO*{LB@kucI zCU`o%plSawi}fWBZ@W<%det<(oYjBj@%;7tZfpC~)#B>-2WT54ZKR~~iSj`a6Z(c) zUHQ)UPq0P8Ku33kSci?on*GiR^puTk1xiUX(+#Dg?=y=rVT+>Ijvyx$f;kL2qG&@g z7)nK=P)qRU?^U2*m9+(bRoO^1))vQ9r4bm4j>Kc3!DOm5Mo6MrVlLQ&n=F4)@qdDC zRXZoD?jMAraVLLZ7chq|0W`vQm+x-W_~)0>Z!(l2zTsMrhx0hq^b2*R@7XbUW_qnQ6I>lJ?I2!5e9mMKS-?doqd3A^yBLdiU zuD3Fysdis;eS2rqaJQ_}JPgjzJCe5#0qHfjI;->R`-@s$=$>PaQ+~T`&A#S}fv4&L z$HJzQS0BgeQ^9=f0}y`OvL~;sXx5z3e-fog?*7dfp`bts=i&O+DdW|j5WmBTZR7PxKm%w-Ii)pbpjjr;ASk;!N5zw1j8OWI z$oSC^5~tvneH2*Cgo;5FBYCivGPl}>-kYw1ho@78t& zYD*d}wHLt}n|ed_XCc@RWQxF|%qFzm<#Zf9F3bBKE7d%nb;Q`&B$A!<-j!lwOV;^h z0lfs|SPk6`JhmTXHfbUDUyFaQpl?Tn)Jqe5MI%+;HQ z*OThLM7RRtq=rXfJ>!fOywn%iD}Nx9@%GNeUROMT&P|basR)-2)12TwgJBsEHDo~J z6bzmm1Y3jjA)8kZB%2ljML2RyBLcrKn1rdC{LR3ROfN^g<2JK}}=8goDh!esT z%k`c&7bUG*=p0r?uEDH0swdKr++Zsbt6dk4L1zRu;Tb*?PiY5LS$FzzMd8<=zJi=1 zhgtI;eQ5&E^yZ1my63k_^g%!NK0V)j#M8aO}y{ov`TH=uZX zh}5}Wgj`Jzeo+Jf1m=xLc>kbp9#LeE=vYj}eN=r6?~t-l0_v9n<1G`*a&EbY%>V}Z zc>B#?{3TZ$KGG+zc9+J6A-bwnfTyF(FJLRz4!Ri}mw{CrRGK#a-5+1V#NP2ZZ5BUp z*^%G+MqX17D7O@iJ_!E>T)h|Zv1*^7RrrrWWdC!&$p8OUksS4P6*OVw3kWjE_*Z!$ z<(tI(I_O9$DsWUXOcdA(-}ut8OEUQ8Rr!nQlF{1df2|0m@^@#!N%w|Aaq(Jo_;^pd zdK}Jd8~gw)k~}OaPwNd z0ms$&jdsc1V?$-8G?9lZN0w-jGpMWjRxQ33e%XAy0-=1bN1;sTx*2J8&zXh>zcDzfW~zc{Sc%Ik$-v+GodDHX-D;d-4j}DD^Q*lw}zZQI@hs zV^N}7#oKkFS=e0qt;8Zi$}EQ8(M}mRy^b7SI-@cUuTd*O#*1G2zC=Nl`+|%I{Wbvf zCqsounYrafjTwegaF41r?r)_wrgZ$JILP;sf$kUUvQe|~25V)01WeEp)uE#{YHSki z7)W(_wTOd2lCKE>C$GbVdVn@HTbAv9Pa`VQjAa6kR9kMAl6mkTIQKlUi_ZcOd9Lpb$ zs%nz)qvfXaBT0LpVPfB(Q)N1rPFG2NjR|0RH>j41-XJs{vnXV4Wy=>rGpjtV1aNIa z0cb4R@kTamW4E3vB_A0nT_G<%oM%^?`2o5-t1@c8 z&1xYZ^Wi!QpuC9zf+R;iY)|l{U&60;L3T%wS+CQt>E%!^2k>?=2Vpq# zLzMfGhnqAdj37wGn*c1nXQ+RLXMFSf5B5(LR>gnxXH)tgSIdg-X3ox5 zre^<3hV_4)M*N?LV8Cym0;9sn9~5L-iB&?#VnTzQS>T#pPoQET1!6GC#!fpNtM~&- zWA+=ig-=LtFu8o*Lo~uSJIa-BpRo&r6pAjvXDirq`qAOzUJ>y2c8k&LCTG=Fy&DUF zLGWOgvu?u@hPIQ*jtkm#MF+`_Ac;mK=SewHDb$H@k+~n74Rj};K4Wr}9wAcUk-ILERIh)j;po@$#BwEUgf0w(}|E8z(HDHH$Aa2=|pM6@S5w|un&L3-*zyO zopA)w4}fv$t{Q{e&_*kvb4XA91;cu^2E==A$*9p(6xYH8{+e7*UNj_A$6kxsj-)eN z3O$UYRL6OVE*gp>iQJXfOZZLt>XKtvvOMg>-9n|EOZQY`A%G0AyV76JL2w>Cg7v1u z66|7oqmA&#qVGDywODWrIorvuZ?=mFzg~0PBIGgJ!a1aA$y+!A*l^8f6{X!|WhATM ztwhcumbRH_5y4%)(vK6zyM!w`I^G@&Iv9mIgMmY$wX1kV;2!VS>&rwREI}E?SIa|g zGdAHYiP^4L3cne`N6hl93+wZca&<$UhdZ6@EBwaHvuRRTGOB~3|I>>5&G%1Wg{bPO zuNeakzRTc*SLumr<3UW$8x)gV9-r4$CJw<*NtGEv6YRHED5*!4nk`a0Ao2yLw@lLx zs5Qu(2z=R3@y|g2PD_S%5QDrW)WQ7>lVEBMr`-?$7efebER|>r1XmJ??r3n`@~N>kR5iSn_y>X$@TGOG zE9OMUl{%BA3VGlb0nU5-T~a0GGbIM*VE<$?O<1T1%hvTgp1UGRHDf#KAnB)O!+ zlWpNx;Zf-$RpI1PB>F~jgSbaxRgLxX9I1nJP>ws0r%7B~#g>VpVd$FzYBOr&rI;+^ zJys{7azVjz1LCAbW#sHBKhfXcM5HkgPD}a-O zdY37;V-e=Fh12HO9tdpi{ExF`DwS?qKs&P#w_)+qteqYGS-6h&!2-Wkrr&7nW)ur( zwh&Oq*38v&gDv{A?i0@@Q(Q`ME^MgwEuHkslYN~AbYH3ML@iA1o4?J&TER9wnz4IG z=+yc#QBPSO@xb|_Z_pj}sLy$2k5*W$YJd8= z{J7m#s)j(+n+kx&NM4p+)v7ZEzOKKuM&U{5tfSEUq}xK@Wh?jWfFbA*3`Da#A5~o( zR_H!VYRPNu3RC!lPBE&W+9JHqaNB*$zILbyQHWRD%NVDD9Pg8j@zfKznT7+XM<13F zB}t_MjYkoN=MN_xWXPM+B2s{Qu3lmE6zD%}i+m5Uko&!FLI-S_A zCiI-iFf67D=BYm1-kT2)&OAr}pXCoB`W?W#Nj0v0n%8bpyP;ukXuziP2G=lk-P7&z_oS>NktSn>R0 zSpA3g_J0d26*Kow6G{jB{|>K|&-NBg1KU3q#uTP9BoM1fsjRh540I0&J@4FDa;O5{ zyrRxcmSt+y5Rrv&vit$JMmXyPK)R4aU}f2FGI%ZG@!Q_48%MQa$>2Non-Mtf&du@o z^ZYoa^M%y2U6#G>4P?+4apOs0w}@mdO@pyC25;x z!U04w10t9d8eCzW04M-uKsggnydkzpTc2apE+QCylwta=3!u};m8BMq6UUx;!yMa! z$Qgb4Th*Sx0LcLAKtm*@6pL@s9u%MY4QY>h{dX-4)W>hdnlh{>>8c4v)D!Uo(UG7% zs>*#qk#>`?LPGGff)vQ%2ow+vg^OF?nb`wmEXrMV^>lo{!l$d4_4|`1- zO${(wiRIb3Of3c!D&n$==#i%~x%aNdYYj2ca;sx(bIoz{SomYa1`hFWLfahU5_7JI z%8+qfRgb8eTO0GL4}&+^<}?}6_HzXbzfTl>&8^*@2w63L&Hlp9e}Xjbk$L%cbYK#r z(Xz_((}_~>X!@5Nsq~WKtD1>GEW}zBV}LgT=Iy9BYVbWFrezMB+(0fWmrpjwq(692 z<4_aBY`tOniwdx=T*tb(-F6 zJ?M17#&(>)bBeWEfEk5Tnnm;lv1~qe;DD@O1-)%ofi70%f{W5Uq7h*fIVCa;ELV1v zcHKnzm~o4G^<53_X>L9#s0y;Q7Z`ci?l*Pb!dVru%9+B-nbDLdk}iV@mOQg-u^j17 zspbQs^AZL5p-Z`7VoOM~^)+fvW>o%mgn5?yeSW?Vyq?i~B(^||HW7fO=FiwakzazE z1Qx*YapnQ~MY2&pBw8`;7Cyv0FzRU-tdxh;YItN;)FAPJDakc70&dCf8mAt~@s;-p zw_#Wz0H$4R2$x`%c4@6G3C3ZoKZw$3=>ufyj#!G!;x4HG3;J0ESYDxD2eaGZZ$YWO zTeAk?bPZ8$`z0P&V0aoE5f`@%hsr}kTD5#L7-Hvm~JN8z!XtaLBc4D-h$EYr$i3@Qb@UWv<9AG(blZh@-5 zUb&9fensS#?R9Ak-OXVtUfdP?JM5n`B$quQfU%R#_41JtyRBUI&^q0->Y z)&J4X`bP*ei^$@N zxs|jObTM5+jC2+oTnMJFie;TN+;?m#wAG#!SP|zC^!Z8Gwds2xqaKCgE>sgEpDRoO zHLG53`f1$Rz3!Op@T95rOt#OA$ISb5VNp@Q$NM|dm*81YPu{@#FN`GZb&3*(}=uwWP<$-}uAi39CnexDqZj1L8ZN0^{`w|mNp zfN6Yf_e6*k^VqfxjK;>oGVO|+(CwNSDvNgVr6LXe#WL^98DoUg%0#kHs+)AfTr-|N z0L60Tq%47%#FE-hWhfNn%Mlp84V7k6{j7>H&Go` zyD2uRxWW$g;&0D;z(rqVpLw^V!$r3*if25uWiMrFj_Mgkae~wJi6cwjRs$r7YSBTp zD>-qxKjjoN^F7OCk|z@?lnwAt?&S1VTWxp?Fv@xK*Oh7jIw6q3)rz<_E!GE8Hq3Z> zR|k|i04tA*4*MEb8lRa;_WUoad@4!?V+B=UEvG;L99s=&KGlvSFQEdPh|SZe)3U7> zFyf%b>M>if4{xhThg0gSDE&K#8B||1>ZXLn85Y}+{+=$v2MJZTtoG-lypebKrpNse zTLPt!TACY|Z0WTy$+&{?y{FVS`&uow7QZSnf+n=I(@%>(Du$OKqqtMrZR0PLu2eBhwm=V(vCF&pmhPa{qTWnFVIzdhz z7ikLd@#wv~+QN~{0tO3kwI@$6CQFa!g)`{u^(#kFmlSnJThu~ir4gr_(d1Xi&9}b>V)C|^godBURSNx21F`>L!TwoW{f%7z-^!|4)5jNG1LK2Y zbxMu{j7%2P_WL#T_bGt}p{V3*3VB&ug3vF_$goN0WLgcY2(QrJwSPs){G@47hZE4Y zV+N`mwo7Mf=WEWZ_gR!zt5<6`Thl&>(|#b&i(xk?982s?caTx6 zi)t^8#Gia`nM5D;<}1n%j2jD-0F;}G@MW<#wLveGfC(`H4i;PUmGRjz$1I3dOH*jA z_+UtY1f(6d`O5G$3Ls9~a7Y`Fg~P$@I~i23IkMACgw>2GXNd6?9j-=XHYmc<+|s0N zC`cMwlPSYbZWt4w4nQ7oheN3_(VVIKUXWW5(KoVV5zgeh5=3oTZ%88$AIjIndOq6E zRR)g_PiWf44!4hD_7opIurhTm4uHaWV0LBZ8?4LQHNugcqumGZ>Afb$p`0V#hw;@K z_PD{oQQXtVDZPOK_?X`BC$Z!X=dk4Vahm$Zg;~_e4r>7judgLP&7lcRYNPY&&P$)N zqb*R9oDf?lJP|!6o^X1o_q{fV_dTAHqtm?hkUYN)6W*xd1Q@2Ws4HDouFu%6m z1V+QjUEps}?(3C2c=+lMi=U%@0TKBg=<1`~9;oZ1JcJ1py?IIzYAo$Aidw_gN~f!d z2=V=HjvDtBS&~%XM=4(+AX$n@42@fqf+pp`r~#L~kP#W4-6z={STFwjiKoS9oVR6R z@O~F9rZ0ZH5!fdrc42BW0xe@F?69b4Qd_F>E5%28l|9H=Ge(RmlZjEU=n>0$yM6*GppC4wYI6d4t4}ecLAHzkKqD^f8zh=WPRizE;WV$7Ws5l@Y0m!6CTeGBJs-d{X#~5 zjv`f}5UQ2)mw}glagpw82tqwW1G7>t+QzyTPyO_`EcT#d(`jxg3|f?GclA0QXYuY& zf$XMPomI7$3uVk#7hlI+oW%w9DXLV?Sz)uDPmvsI7w8ChwMKd|7b>giiQCe#_gR)H zcW;esg(BRqOMy^bTv+$Y<^bYREp1VVI3U;8*mLHr^AkWd(V`cMdZk%)e?`Ts13K6% zo(AqQ=c**Yqe*lLh?@bHV&qm7F&Zau+GVvrmZ3*W`rNFOP=8Y%Rg=;DiVjJZ#Tq5q z6|x+RkWiEP>VzxWeWuiidL4?<(+Ms|Hb%3zXLDPwKi~$*K_-CfiB$Mx*+j8Y0SUQh zLebo(M2i4vH;75D3v}XCj(d{X7-Xe=0o8AA?1*?6z}m5(pL)6H6pv(ojxh|k>^ZlKA1ej0pjc^)fe^YB2#gpzns6MKfA{E5nsfC zm2qG0F6ifuU!;s8CsoQ(iuCM;Xn~a4*->8zbqLjGV7?iGvk0m`b#Bv<5sa7Vc_Yl~ z!6}1q-eT=;uzaCQ#?Bfcdv04|uWMnrDxF$OMz@K~KG8*-db&x)%G53I;w{9PBR|h( zO=(#o$sFvzkQ0loj?23R0~c_3w&and3Dp#lW$1&gj^Xu)*HJQ_ zr!t2@B0@sy2l%A}7|{2KC;R>Bh_uZ-Kc(f9*c9sGPn=*`@fG^Wm88N@Wwo2yUM<^4 z$~@9BaoCuLbz ze}0($SeRs4aZJhzWQ&#j_{<}^F8*?{Dtk<^M&e#u{}sBI!p z!l>blw)lnY@~7IP@ivs0CbDRQOc` zF-goVoxE70@Y%CireBl@t^~}fBP<0S%A_W_X)moQ>Qqz&ne9)jrQdzp%Ls}aGv`m3 zv5_t=7ony=6T=0!Dl6v62*s%gBZdOCXj#}-Y1T$KV?NMad6_Dv%#A#mwt&U`>X5E%5dN%O}yqh!86yh)sWgK-r< znC&)aJzX|Z(2K~(koMi1BsVYpDR*E?VV>KUk1z!`uvQa*?~YiYA5C|5B!YSyPa7?o zz(k+{$5Dk|L-L4LFP$F3VZ?HYUSinFlp^9gUN>1vc{Oa^0YEfrt>H1~tO>mFOBd?A z=^;%wXa8;JU(KH5bbS!Fq%eTT^XD!h#*FMe_JQo;mGfUi9IyOF)xggI4l>q1Rd|g5 z!`}2?s*5gvpGxO^9tohSq5qN3GA$WHm$CegzNk{yBrPL^UQaJe4FM}pgANOULOI5k zZD7RhgeKqbKxVHxrqu0BWT?x+<_W;H7`jm1_32>-uCmj?7s2{ zcVcf%Z~=;=GYj~%zqr*O@s1CVh|Ga^v$B+g%rVUeveDNKpTv2V+m3MSwj&rpxW ze}UE+V3uU{Yi-xLuQUPx#!V&_E1Rl!QRy+aReNA8Z7S)}TpvlqaemNp9cgH2_9} zXj~hPRi`sZ49}EcDuQ-{+SncaDvoEq2mKJvHQv7!OtL5X@SDt-M8YtfEA*X2p-5v8 zhm4EM8h3fov8>AB1TP*?98Uv8EvPvO#9PR^o7(i2kR~7HvzWI%iNrmdJ&Z24!Fge# zJr>oJQEYUZpmVVD6OE&ObS$fISkbY(=y*9@3L*+#c4{)L<$QTD1=Lr>^X_V@_3D!a zkuM`Qdzp|Q8|E}&Mf{FrWB={NrdYbkI|VF_Pk`M{cQ{{UiHJ{5HGX%N!oh73xTmK1 z&D}JI2fHwwZpn*lG?>KPI$n#+ix1qEFyM9PaUTIlg ziILg1%f<=kx!II5Uy-1svZ|yqKbCCtj;Lc@Fr}a~{Ijqa-49_C-A-Hgh{#o}!=p)J zZH;^K{F{pNpp?YnBqDXi_+;V_1S|}Y9MMoc=Jy6!Em&=ad5i+~jm^x2h_M6dQ#}Jb5jat;gditK5ISM1Y zxB@74^ObCN$bzxa?Z9jtII3D3^AW3GCz;LEC%rh-6|$$QE6=Pt-<*iP{uEV~Ul8S8 zaFR}|&(>!K?QZ5uuA}^7MRIx%O`O5JZ3rv|!T4aNyH?Ea@_>)(5&jLqDnZE|%ok|R zRhOvGPM;j6&**;7(~vsbM#T6@5;oDyP8yTl>c!<<(4H)P((;70;$64aj;KC_^OR*Q zU_LpNe)%EcM#L4H0$fSSv!>NODGnmJKl@eqEsV*sR9h+nyN)GbhvX=(aQE#`0Yl@K z-@ymTxAk|X_Ov<6C$OTn9Fok>mJWR5+jKvLMLbdTJ|g0NTbO9l#4+xOO02$CwH%z& z69GoI{2SM$xwYCp6{w{0FVX=TXDnaDJf_$ZR9LDS;?teZxW0>%>7?xt`<%qqtaT?|QnP|~oq zQBPl8STzp_i{M8h-q4Z{$qtJ>%5zSMrgYidpIySE*w&h=t+S1%n8&nxO{>?A>oi>; zad;`L81nAkMh^+wslT|HH!7$u&*WoTl%Ke0exE51VXT8OFot6wiLQlSSvYAF&LL4! zz;%OL%z>Bs4L)8r)H=J?>Ln``U3i2VghiLuX}htTAJfTgxnC2>gj8WH4wH$eK~}J| zeRe2y!!6a=5TpBHWWBwV8Fph)c!5RNU+Y@sFUspb!to|NvZMH-c#T!yPvB#>$j;V4 z0cYDMTQ=qJv3>B)<-WcZbumDvkl&Cd9&IG;KPX5RW<}R0`uCdL(mg@PpHJp?7{Wj0 z7MlN0C%DO49Jzr92hNLs(*Eq)@5zL>M8) z;x(goQVflqO@-+1kGZSl`{55>{-7#Iek(u!0&)PbW#C)V!H{=RJf9C!qTD04B9W>#h*3P}@W@-!&>%*$U} zF(l@jYbM0FfU;-uN6UWw8dg$i^PM)Lp#ZHaS&PLE_a|Z`V(DUuago&N*zhBjm#4<{ zq5Hi^4jLp7RI^}GBHNS+!(uwS=4Y|5gQ(JlTim;uvGfF3+dlaw*GcS%r9CqLq#rEly=j zq_T0w5=lz(nU9r>x*WND0bT1ds0C;Y`nZ>s*41lS4~lAlFB{(NxoOSlyJ}=ep0bbT zFVm43SVA~;Fx-2X40>QtR2}+pe*@+(^MmMv3MSHGR9{d=oq|{I_tNMbao$TeeNG_9>>B(Xu z3EV1JT!v84XNPA=e7#EZ4@jqt3wKdU_|V*TATc(IYBkf_FYpVd@b$Tk`g~CRFZWQh zy1rjnulEaWM#|{u;9C{GC%-afl4f=g(p86srr+UpNN`hN0AFye_h~ZwWt1J=+)^f# zv1W3*H1K{&w~2k7Lb4{0R&~mOTL3k7YLOg1hnHAmmW!R}6mqd8EC-qdHd+b6^BL=f z#*9GBtr>cmet0}X+|VCZ=NlGz9B5}OT5OPMWwlrAoo zqF-zqWC}C*j}6Pm+*7W%j>OmWqY?0H$^~eefkbD~!}SUW&n(eVzdw1VY2VH8g3i@z zQ>)oDW>V?RD(6tcHM+@Jo6h1$32BPG5!=4%v%*);F9S9J4+k*5X&!+E6A9L8Jux?DS75Q2Omv=PV9TqjepdXwPHrbj?n)F)- zOEz;k(qLq<(M`>2fD_MvJr4ZH>Ad(ClIM5N=3VM=Mu|3^)1P>v5IYH+ex3GQnS*w* ze~?%d%dpmJpVj~+a9gd?KY$jA$KG(I3V88OOL{GvJ%C-*fm$}4PbDXS;9NWj?i}Gg z={w$}=yskVcIE^4fI8;bmi#f5cY81Iz6c4;qk-&Ubvb&>VRaD@*xmxc zD@Yhp$NY__GCJH~9!P9b_+?;_fXi%wnEe9pf$zsxP=CoYtWs5r4yx9Tmf_1U# zhnQf`bhflXg$v|62{tUiSn{Rtt2`u0*KRPg8O@)c9Nxm{9eXOnVm+m8ktJcwBfVvM zOK24nls7r|Dc2%3raP?Y2lr>k?M*a({MaxWbG&bZ77zu;*FKk9Fp|>*c%0CotT2E% zO|FIFoV!ho5jJdzeVMsJCock7i3c+^~ zyLso^%JIJ_3ix}oPO6`adN`jng@3R@c;O^)CFA)`Hws* zOV(Hj85}q&I3~JgVs8+UWIqbD1{oBpl$4N>yN5gLn5lUVDVgVvpFwyTl_zbXrBzD1 zT*^xfjo`~zKySdn3O{d_4JT%1_1jVRrT>ph|4oOZ_m9V!A756YukIM-6%-(^Ex#y& zJL(LYp*Tdfk?i54Xoz>w?(w5sMYU1vfuRsawlVCPpyY)=efyV3Ib%Nw8%wq+*U&8S zr&zWu)u25L$r(pk$r*^a-swqLrKGT3Qj4+;&WkPtZaKAB|v}O?4 z5LI6={76(ivJW;Q3ma<_OA~VwR#EbW%xVof5=IgHD-I%acDxbscs|ShSX;P1PF#V; z@II3n-`JY0)VOD4v*JL?4g4NF49?76b~Kiw#IUTAX;<_ei@vzru+2&0XNS0D=m}?G zw$V$~y?*xZs>N{{vf9D4!D&+4eniM#+i!jwa9*8=JL5gD) z^-#fF>R;2gSN~5e^8dXmySst)QCQnIYp+X)sI8F6hGgTh(Zn=^j-|- zU_y_`NX50n#kOWcki<6yRNrC$&T`VwAZ;n)s*&HlDH3N+E*@WOzN>0b8!6zkeVDt9 z`_&zDa$Tp>o9L)I{Z2*1jZ+P0LF=-{IA$jG#_KZr2MK6^Qe$Yx;FWq3m?hye!W-hPsYaEZe$aR(-!>{zRI5Nu>wy}%TT(@G97@c9TqV)0a#^`m9?xbeE z7Tw5J2IjxhzG&qE8JN20^`5V4HA2sf7pvXsnV)wvDc&3FWT5CVVjGuDG+qzGRu(oQws`V=u)T zZbDZC{}TdaH)-?0%etM_(&LSW zZCJV9N7urb1L39ajd2}V3B@QQJ?^YW^x=b2-tIFmv}iSR!INX9zXx48W@Vp!Kc>Bx z2PR1h!ghB6NYeF5z2-?>XLw{DYIlsb{UP{Q*Fj?g?2kq)=aN*vzl%q$-XL(`kC=lj zE3tgD{ntxgpmYbzEv#oFqHAHKj2=W0cCSL?gS+^9@1oC4pBSw_JqiM(JkoO5}7Zi%V;@+bb|>+hRYJNZ{a2a&h&dn*ZH>n=3T zu{G7lr_Q+Ku~-Gqc+|#*cA^|3(|qIN=hp4PKhdZPn=~NcB#hjHtj(iFV)=`Hye(;DstdS~-9-P6JUQ~vmO zhjfl^#n@XauZasTT~qV1v+? zISbcj9;e_I(}BI9n@v5y6w078=9W}FS0jTkW8dCh)IR3ntQdNWXIeKOiz4sjATv4{ z;jr|_vapOLrb=q2R8M8dZ1;Y5yZmMbQ@9?LT47L5>XB=z@OhKm$HhSoJw@5|g%>iV zVDHoDIdDhF%~SlhN)x{X68SJSV|T;+PDjYgU&|vaPz7AepBWYX9~b2R-RiKjnWKZV z>;Fr&6?5>gw{Ju-21sp-F}*%IGUn_>Fp?gqJIo12orX{zlHH#8R}#=dP;c=2X5nFb z5FhX^G81$doPAG?VJna;n9-g1zyVNzSs53(2`Z(j>CiegUQ)p>Jq~vlanWaDB?mDB z!RVschuunWf}=F^r5SCVV+29fEvN(cX!98z96){Bz1(#f7bh<{trw;*y-cdB!5pU! zwY4QG#fB{Ug(a?fn5aI=vN2GnKR%pXn3$_#vNl9p-=wc%Vs%&l0i2pfg>K0nLFcc@honVI($}PqcW>gSUO5K!7MAyYVf=0KtvmOL}#s^ zCT$sM%3Fvwt}2DIZ}(Eq*I_8kN4U~5*T7*YNYwJhou@j_Sz2Hd zO-V0xPK5$DnZH7qm?0TACazp;++gK*cUGoME*=54%y12A1n#8yLi8wWV$!}!%>h&G zVO(ildRpyiLv8c9l@on>rjo;-q}mLn%qxZZWmG|1scIANZtcB6r4?RP!|Bfu`RJWa zK7GAGhSlaHK`Vs~(Uc60@srbax$4ep0errA%D3u_K)q&f6OG7ouTpfAiVio^sj|}w z*M0{D;IL6-o&(8}qm`?Q-@@WfUkw>&EI-U}p31p^J z2KiY>ZKMGse)!|x@i9XahKrL9ER(}JOO+JHw2T6TS1L%ldo;!~L@hvFtJUFnml z%Q36BEiCKQckRI*GgOI#%mNoi8$(Nk86yh``D_#Vu9z1{M3-gzgG8C(euLdrT}T%z zabIA=bzpT?Zbd|04p2ZJe_7F+U%ROY4z6RmWvNw(CquKzJWZZ_$)v_bUbe_FNO@51 zc*o*n?5p|I54&}K!%DqPI><8t0!1BKRaJ{I(~|t{%kQ0T5%9`nJJ06vA8i*^i~WN* zf6d!}pw+le7Yl?kM^$-1$8%y!BVnQ(8aCi{^Q6*a}z7iGw9CbDTwR zF1IZPEk~Hrtl9cYym z341HktPn3-tEZJv^*wfPSN!g_zF-eD7=lN>nZ?!(VZod5KM}n57odu3^W<6<28lD$ z@7SUWmEF@lWCo7XJ!F4KAnE(j^BGPBG#SX*!n(4IYNQxnRr z(KNbxHzhUi^EQ>_4(9ftP zF(4yl}D#U>``uUi1RRRKV1x9o>jO@OiJSmoF@kvC}nOCiCcLKcJk58o`TR0 zrMPz4#@0i+^hSdfzCqW5N>hH*y)r#Cz`RwXtO(VJ(a9-8NzhJWa)7;b=@0dQ@v2Qi z=~5ld5xK;4u8n_$dk&0`<6%a)axX;mfcDCZM}l3YKQIvyU^qY%5jY*xS|L3kRao2~ zP?+rXr!5F)QZyQL;JKyl6oMRWWS-!1Z>W>^?5hn^5Y!lEoj zvHSmU_D(^vwcFP2SGH}}EZeS{W!tuG&a!RWwr$(Ctt_9p*4}@_A1C(OC*oXW#JJ1I zF?wdcz4fQH&PJ4r{vbjWO2wdYG}EVp#)0yxyl3q#KKux%iPNiahpMe`$NEveC-~GF z_A_}Q_pLGnu0Wm4L#pTn;=) zRqifaEcq!~OvO{a0LoLSNb8%wK+03JV9ZmpkPc{(T)b0CE;)2azPO`Jo*7|RsTduk zn(kvvp6Pc}&WNt9WD$eUrY3&8E0e^>RU41Itz4p*IEAr5Zuo~^?OZ#Zc4$@EGFfTk z4k1~!q1?G#zXw)KSu=Q5c_WIT%$dw9ux0NTl@3{B$;t*o-uhNI~1_*Ybqzv zm>8~KR!)WtUGL6-d?MGZk*nDpG&9BAkai8lTS7)$k5a@P;Gm$HIDEAd+#2Q)ukM$K zzWllQRI;}xpU98AnE5&4SdAYWln{_7vXZT-ez|)584T6}Ck#9uw}U!&_^r+`p`MXp z6t16Zig``#sZ`o;foF1OIt$1Q1hP`{+B>=v4PLzn?B~!jv$mwEqbw04sTWqVGB)Et zod@2AqK%Jg{M-t7xM~vYF88r7qehNn$o4}XZ$4Q=zd;CbiIECuG~j`A=wF;3+Nxg? zZ-E2fmgbn{y`4~M^>xVCNqIZxDA}%C(SHHS3+=B|EOW*@+God${(HIU zilet#x`Iu#jufNnUQEQoR?9#hR5mm8JhXUm@m11C)6QBN^)IH{H11k$e0+n>4`)NK z3@C*T8ccX{I`TCwj*Os2E#mzieZK?(A?vPVC?(GxeKL+suGOl`^^N(R?ihPpEgJOj z<=d^1;%dH%;e)FyPrnnS=M=R|8Pew}M-wM6F)ix+j=;dJWZbi5qsyipM625ARXFl2 zfNiFIL0~1bdbVZ~rQ=MO!zlDOv7%5#pvGP6VH{P`m z?Iy<@mtVcq#2w<^9WpLZH~YGrJjl6HfM=E-f8it%3ihMqvsI$zSlZP=p$AhjJ?4Si z=Y?^BQtbV{MI|7yTF>lwSvu1z+O54uRLYt}8FCcah!u-k34vsM?_RJuU5eSiJ=~N@ z3`kbrb|slOD{*q7!{quz4lH26usVbv)WIr+y2RKqWTYcHnD+12VX@?@iEPQ_j%kaf zqm=EkQ<%e6$N~rU9ixPvhs(k_-NXT_k(NQ_+=7e3aYs`J#szD5+NA2?i(~~D`29A_ltX28Y%&pha%QO~a= zh{Kds4LVFYkp0Af6+6{^1o$`bsI1ekbURu21*qys5QY(}&EwTVQ%Bjq^R2Cq2mY?b zUzljEoal=oQ)C-IsmtsQDE7Vd^0%%WHAi2XJSSU4d)qbm0^4;cSAha z9eHXaZ=E3hO5TC=|8Slnpl&focz_~eI}H`&litRbZZ}&H8hg6H^)8WCmK={|whF|K za-O#-PX@}q%O%#R>g*jPhoyl9M~X5{wPRhuOmNRtkc^k{KrrbcPvoIXUR1sn%UezXSUzNF(MQ_T zA{_txEKlT;d8ktRX|syr=rWF|xJQp7zkyUTaz)jveluBA(KiFA@2O&y1l;j^&EA1) zYeM0QdDy(*jp|P+EIr|khKZ(dGCM{~;P7vWre0Za14b9!A)R>WK8U8C+i*kY-2WgD zeI}Pf$d;t`#}t>1h*%3Sn-)gDRn7%X)TkZf690b6X*#40X#*!j5#>2>?yLi7F=;|2MDSO^=wr7B!&3AQcj5`WUSqjF zX(*)ajQiLPux(Avb^~DSsa5x~uUTiUTBZK@6W@o-nUHLi8c`?jqMV@j(JPw(&9@TsF8u^n5v;kI7|hP%NlPOEH2 zJrdXKF9BsS1p#=iO_NsdrqQ#uT32aT_Uo}e&=~A(SBZEqO6uE()IO-o(KQPO)+l=U zba)Opp&JRBhC$w!+_+6{M@WMkBieN^SmbxUEzcm&OZ=qY>LxQAj`kyfaawCE|#!Sz$qj>qrYX=AT0HisbT?m0Z1 zIk}8Ezc6QKv1SvmX7u#2e1oO#P!An)`EsI5BTwC;PAx_rVGli0?s-Ptzx&+3$J}T3 zYRPkGaSi($g=-Q=|VM7wBgxAnR!}vN$^u=2QlMFbzLD9W4grb$J3BZC*8C0 z0N~p0iS*Z6=0lKldo+48w05kocP?AKwZz=6_Peri0olqS+~eaTGedT?Mn1TvL%Ig$UYG3&1A7K^tWHNdf-ilY$*V{{C zAs}}knj8&EcYwk*;bJH-Z&-k_$-}fLa#!jMW7C_#l&z7q- z#SE9mMs(6TBoAi_e?rpXY#KE29jJlX9P7XVa+)jqD^3$RT69Kl#K2-x2|h4!Uwm48 zINv|JoL0=`%`~~QPqxi2osK^g=zga8H07pu{&va;y-6G%UR(`}`leiP+LTpQv{<55 zLDUAEPH&W*Zbz$%^w90=Wf-UjM>u`X&_-+`^>H=FLt#dgWa<|#9w7(AKmYc^4SF4Z zhX3V36p8=;NSXiL@%>M7Ty~<>zmh6Lb}g(@TVQNxsdfZJCT$~WL}<*Tq53gpGz}gP|9Ev1w^nrz z_l(EwcZP`Zi%G>Ll59s3o9efWCV}(R+9B&;*5n%S+^iO7LA;RM*vR|C7KTEbmSwjI zfPZ0{aQ*G{|;rp{?}-<|0JyczpC;79;7ELX(=trBYw;JlfzM_I`(P`6v2th zgU+jB1~CmvL*P@*>}iS1Sux01X5{gGP=8{CNtzjp?tUr6co|N93F`HHxz66?@OruW zeSLp}^Z~CJZ^hSB9F|8_MX5kZA*Yb>q|^};p#%m90{sRK0gIvxGZgR-@&#iOMH>Q* zG!X7<6QTp5GG!pr*9;E%pgI9N1IE{v9R4j`;U2(=HXCQG&kWmL?)*K+!Q{MYKjGQ` zN>iF;U@lRt$MjT8mwk$o^uq{lc_Y&)_g{m4U++$J`@V(a48-C(N>_sF_hU5LfWR--@z2q9QvT7|YS%>zEJve1e-u4NK#gj?v?N9X)KyG;5>oYU2^ z()!pg6Ef3ve3zZ)*w>9Q;x7XfzWL7Zp=ToM>WY;#urnfBBfqvW2{R1GlW3=L5K^X4 z?N}-v)C&jODOzMa2c?meaH{&fGr{1YFxF!8*kDlWb(}YV$N#!>q_XRX7gBS{?=-w5 zz3lfbhx;IdjsrBVi5Z`jKaZjW z(?m=h61x_MIVFDqS01eBK6e*DB$6Sj8B|oTJYPKR`+L-E;Y<#adrpW$8i|DEt1xLF z>4xoqFCjUeCQINe0{$9xJC`ut(K`TO@;eLmIu-7E)GX{Km#9TgjCkHP0-x(bf+0ZZ zX=VQHvFp$ModnlG)6{m8`l0#|Y06lGv~{p`lVNngh>o-1v2kFuE>Lw!)ni3`~PMd%y==r}zwNMk!6ZgMG?eM=A zHUEF04gR~RWlWq6j18O({!iG*|MZjVqo6|%@~?1UIn`##Wp*fftpW2cL^7mMy4`y^sBj7mt=7Y4mHll8iZ&4qSTkP~hD z#g2m*&N$A-wR=Km!!_peL-Rhwh??5K{a|^T4xDB|QO3k>>0`bt+3%97jEAPhG6^Rl zOMIep-Cxwd7BFh=hGTc)du%*4+fEybqkEC&u&N?%FvY-`VWX-}FBbTjD{3+ZsIA1@ z_0&=<)ruI5w@oPEJ;R_(-u7x4uRfU{dzId^J9g!3c;`+b`}su`iq0^>u=7RAgSnw& zJhSYf6D>nzCy7ABt7^Ilzs0#HQYz32B8PMf%6Y2nzJZ?w*n8fC&c5h*kpjR9W(=C- zJa4e(b8TPyXY`*vUbpS1Szaq2Cl5ot1bpE7MBmBLUX2bMIChnQoQMwlkRe+Q!oi%csq<$+5r_f}-CkBhP1Cd~UuLk$XFjRmP;>|s`PZX#Dbx+ej zPCz>EcOEmup-2@XyE%aB1=?G6aCN?TU(j1<@b#V(j=wLJg(rv`l&^UIpPu6V(6!S2 zJaOECR#s=2Ysj0$Ig&eun|XUwg4H`nzN&o%Z^6OXdkeVUJ~39@(Q9nqa4lAz5FA#X z@J*IaXg4c&NmswbmEMTCn=ie?AN(B$rt7j6J9@0~pQr4yaqhTpHFshi_6_hm?y zBp!|goJi~eI~n#2;;tmP2P;EK)g5j4&%=WXwx~tcOE`@4xw%5t3&AaEP4-RGmyv+a zO_$pF`Sa({!j4v6TyanrJAv4@LeU_}?&bNm`HM%rpg5#EwzS4*&rqboVi*78lFLk) zt%|;D%Jr0)Wy$=~McieQeeaZosO895qp7Om4i;j zm`tfBSy zBgTF2B$u+K(f5^dzX^_s3ToyOdq%{zjMPWHu3YGllYf`FE>2~ul8mL?*DyjT)eSQz zAo!T9qP`BJ)E{e!X;isIxr0){ZdMrE?=ToljW63y*rq6SS;-f>6Z$!yk6IL=+ z$qkW&yL-3MeCa)AD5z8sS_LC99Z-8*E;EE{*Ou2**6LIX?3+>4?m(VLhrginc8sqg zAGr#4NO^3{@f^VsvjT{Vl5~W2@3NAJ^#~_W+cE~i%k<2!<`eEMdJ&{VQ*%5pK5WLG zz<^$xzM8B!9LJ%tg#p5l6y`Y~$~C4(u6afF&;qM;bckrd-513$ShHLY3}eY3SW zeRupMG#)Jc`Km)^Onmzq+S-=IZb1?_t7BJ;ujV0W*|Dy*Wo2J*FUtjEAO2hb;V^AF zWABkKgu=};plt5sjT+Jr-<&{h8h?7&Oh)-ZMWB!J7bgZR|@q%JwvKL|*!BbAR2{ zC1k3Mq2qWcSMDn zsU1W)1%Kpfp>WKKFY4iU+JpssDOspI4?{K^11%LTcThTlf?_{=*kV4i)qWo^Uphm7lcdt8dKhc33gM=?6YUYeY$1%G>ML@ zqf9-17kgml&DWQs2Cn~e(i@a*qcIgF(b75bVqi0L0*db~L4O+lQY3t+ju?u|-MZCz zTMid4ZmMyOICQNN<_0w?%iNO+^o^*(JfHMIfhbZWb|h7Yg2!Y;%?`5&!jn~w&~_jC zMMp-zWe9}U{xBQO2rJJuO^;!=>U3ikZmEeX*H%n}(QCiXi7O2qZS9$0Acz;9q}pw} zzdl-8Ni&1FuW1XIQoa&i?sG~9Z_0hN7-LL&Wf{Rp@Hxmy>inl09uP?v6Npxar(CMT zJ3iFf5x%liU&G7h7IXvMra}~=B+>|za2+oDTukY=8h46r1|HPLydbyhe9U%JVWR6* zKHjK9r->K@wH!w+E$<&nxaSnsXcNj6Wlm_k6y8RY6qd5B`RHMlnFyg%!a`Y9Wpz=R zx~PL2VlTmMNu0lCFq1*>7{#%iCCsH%4KwtlXT0_3V4dz$hz8zyn&{tYQ#rZegK2{~ zIdbRTpLn^gO|uOfq*7c>Ydfm33WJA7KS;I`57fz1kxw^R=Qf(k-k8OMCa3e?S1jyI zoGLL>lBJs(XZp$5!SJn44(vPI+R4k&oUeuo>k4g{TkT-!+i7;!beAJtz#=Qg8M|IVOfjWBJE(5jv7ha^UIwL)`ZlR z3^4$fqu$1oJ**v#Swkq0%{SBVw>_k;ose=)8gAuriTU~^7j*ADo!MP^-1p|1$*oVmfrkc%JO-9i%mWoz=-79ZFwz<`I zM@{sYh3K$WdZs6_K7-QVU7tF2zPaXFqIz5?U3;h0Xril6l|)4s<3UtL%trSF%-*f@}|LKP0?El3a z{I}!PMwS-=NC7eH^pZokFhB0hn-ss>JWP@z?0b6~MsB8>CkvoK6+VD)S2%>6ye31= z-I%!T!DHw4j}9u{5gw{GWD)cjA#I#BODb@BPxzEU-$NWQtvvobCA0)s<95G%)2+2~ z6{sW%%K5p&rA4Q_?wF{Xtk~scy+KRc#=Pp?rCr#mC)%U(Yhl|L=gT^VQP74THk-W+9F{l64zk}jQVDX&0|9tXJI?a-OOubcMMoa>!e#oW&?e~^C|6ceUD)tPe*PMD{=)~3wT->jMG86OX)ykpW+Gv4k^ z`BtYUr@FSLsHePh(r4KpQKr1J)A@}~W;5QLOuxI*e+H-Y7#=aEKhc2dvT4O$`_$X&T2^BXv#dcW7o9BH>Ky=V{o|hto|R(lxGFB6Upf|7cvXM?y>= z7^VABKGqsLwA0kY4JS9WNusMw9$;%|Q%7DJ*Xy8j8#{E2ZkjlR(%>fy18LwU4G%WB3!}$Q9N0AgI%)6|MKWpN#}6Dd zc!(l@ncOGQXng)gryn~I(dZ(LOf|WWpwUepwrT*B(&(lR<21M{ zq1%lg*l2W-MEZ>%=xB8PihN}@V6-Bx<3p)f=K{t+Zp-RZ0j(jg%kImC{7mn&gLGE| z=7oGB)+e)v^#=xa1r`AIX(ip9M~YY)eb+gNKnWxj?0oG3X`PD};<9X9(#7 zg^?yZD!<|Z)UkH?kXzZ;#yf7{p}%(@zkY8@Mf#!wC!&<`x- zZU;kS1O$^cb<@;Gd8G*4mZs!#N9->L^@#z*cw>61t`EkJG}e$uiUVa$T$kRb0osDt z7StCF`Yp6q3wlFXm)OS(dP9OpTo>E-8#L$F9h?6vl()bh46q(DAib{^k^@m66cmWr zUm6%4SO@Y58BabYOJ+|KconHFx$heEhUg9wm~S1p`HC2r zUPGS@PHWtjWKMeT78IXkPIM2^e-x@qc8>&@fMhT0Pr3jd5CqUKe-!P&z_;cK>Zqd_wy_ebk^kBzK(tKZJzs?(RhX66hS!HY9iKz_rNk(S6*Y zJ)(VH$e4!yTrXK*T~~C#2q^9$eVL%&B6}z>EO*{8plht4HNWgJfpSIlJHP_26ku2y za2oW%HwE-b{3D^X{r^mYq5=m(!jdy2^sUjPP5TP=eIR0{ffZlD0;?gmVF3}PB0EwN z5zN{O_BBGSLBbL;u)>@ybU9h%bE3uUkL)sA-bpc zX@T~T-RYn!*MEV+BLmFPX-wph0AlDgX53I$#Pu zA~0v@6@s!qQ1|z=(DsLtN!rAbg1Z!uNC4XrnB#JaL*~9Q#3OgCpmb_O>b{-CBS;VK zzUjo|f2#&vM7pR&qDvKtu15$d@l94ldf!iVNZB`nxCCkb+&hyv=J=Ju5ZpiglS>5k zaX2DPq`NVEe7!5=A4N(cMm(`@r?9|7k3ixR#pbBWh%&+?59RDxUQl_`Uw57&G=(~*= zTS9OJQGJqLd5Eh=;BQ#p-%z0wnFx-^mrv{OFyfi~9Ve(;9^Z1}7R7UK;7tI&h_*m6 z4Ae0(m5%ftGH~icaYS39dS|4a+R%0p=d1i^H>m4lU_@K2o$}C^a(C{J4w4t37qvc> zotjwqVr_t5VTi9s4XN5Yo|upDj>tb2fD-`-TDsc(v=;G>C?V$~5&4NqdUj$Vx(7p` zJdm>dh~O^{t9|o|0EFz8huKM#_a1hl-t+ns0DMJ0#fx(RqY;2~<2x9e+1jRppSZs7 z$z0_jyH9eFPgy43C;?y=hw%GeM0fdk4*opx0KYPEfXKCIC$71Nh68%?6A(eF4Q__vZYjGw6%)Md|amCG6Xu%uf1EY=hC?I3Ke z)f+l$rSCR9n${C@p(qifk}&>J9;$J1`y&8zsi&$!*bsW?iQJF@V9}^^=crx~c%nCHSK3KkXEMphM1LNb+!C z1Unr`UfQ1O%)Hb*yQnl35ox*p9EujP+vc1wnD+smz?X*XlpIy;%B+O{{z)(6`?(lp z9?-5_bGGfLw(*(NtjV2NS)0Ugp ze@DrN%(&Krr!tMvWA@Ob!(_`|`;8rpmKMb2i4AKK!oaTNa>!F+?ql|J<>?3c034yc zz=~0c9`565KuMd#(B}&Cns~XYy`%-Dz3!=6dyx;(9BXA`Tn>C)iaTm3s2V>5NSETO z^T#(5@6UOF3uXXQBO4n_kNdtd(hE7{5Drz?j*?VdW}0D0su0DZvEiY?Vtor>wwT}t zspP}_5dtN_ea;qSXD3r!ZqFB7a(~d}9h!tqm54==ShX3n4SbmbB9)rx$q;gic+2l+ zWMk5E?go%R&!;4EXX2x{upY-#IVpLkieThg#4Hq~l5yZ=B^U}enoP!}snKb}EQ^T9 zH`8ZsV4y7SfBwjNfydiF&!fmQ99f4FrM(pCu+KsKHlV=_9G&oMjRxVB)Yca?~|&m?Myh6a~Nc}O)ZCGl|ZxqbJk zdhM#cM6xq@qBLO?ALASBD9M(_KN2z(6++j>PhLu}<5Q}gF+~#yyky0lgsOX2ltQrt z6vAJ`yOG-KSqsc{fD#d|ozD@O#9-p`^cmt<$LPu`Ex&MiL)CUa_kekYdt5OPm2 zFDy8>Q$MS&vaXcD)Ok1_gcNUVC@qlf+>{Vj%l}=XI{tGGTdp~fsvv>trW*=7EY|x3 ze?J+MbAYs6iQcSx!3~r4Fi?t>*v(gP+YL8Gpx5XhFk!(EMcF_ zLQ43W!8nTc?5RDuC1m|RH_?AiSLM}p-QND$QY*1`p5TWnQ}Aagd&9~VJYuq>rtmQC z+moOY->%6rM^WeW-ZNkk(^#4jdYyUF6m^!Cx>{RZ(*gHDe3(KJ{EduxV#3sLmU8%! zN>cW38?ryJ4nq}JK=gz}`VY9yB#&-$LA`JAg`RjTDhI>x<}-ddu-mF@ZAUep_Bz4G z=q|Gy&wJ_+3AZ03Bk4AKK{nJfEHpPt()qBHnQp7=iOFSJq31H!%h?U+J4$Mc%FUnZ z{@dR!m3FUW;XHnV=Lc9YW`!ZMdMu4qO=J;X#V7?jXyWP5g?dJ<1C#mc`Lti?m9Ln8jOD2r-gY(X7E)t$I-6kg_#OnI8I8nG zn;99gNS`lZ0K5f#aF9>gjA{jG2e$E=lMwnh*B(D-C9m^?B@>d zGHp@C+-1LTNH#ZA+SJ-gdzOz{Eq#2o?2|36>><=F(Ac3}(0o#lRCG301?tmarI%eQ zHm!KRVBalf*SeaPB7y1Posu=KEA8w*mV@~Y&?Ykqp2FQ~<8Mo!YIEOPZf(r96;8`n zI#Y+;#8^O7`T)kzo6gnn)KFAKr7_wR3Xo+-1V(kDYrthEa2+_c7d$Q%#Uffn8~(vP z$Y{($TgFx3+xQ!|w#(7Jy%%tFPfL}@Ib2&>u;@>mke2o`aH`U*7ZIXN{n*Vb+v=g* zxmt%xAR$>KAGYO;d^idc261j~cyOv;z67=@8muo+NJbHuEqgWGFY4usLpg z{}PxfMFsn&CR@)_;AOmJP7%AsD~QT<<>;cRvwXvgec?q+R9&J#+$zx-ug0e6I(;SM z7++7aB9uB_#vy?T|xnks9)%}`ZB~H;Xm~o0WGe(3hZjf^$_qQs;g?d{zh~Qa%*DkBuvU$<$u3C`Aw&W_)9NXZ5{77$ z%PfBb$oPIEH%gRGF8{e9Z%#==I31wP7`GH!95!q^bBqw2Uy@XVFC?2nZ8-_mBH@eC zX2*tXPCTRa1WNsV5`v})r9kflF*fXF$(tdf7>NM)3kK%c{QdogLUj5LfL>jJhaD~H z9>2jb=kXpwe$M3v+(9qnNTJ#5C5qS>jutJr%-}G6@*F+QRPjE+ z)Qs)JR$A%uFv~FL)8Qwld|_QJH=87m>SK6ueVmE1Xhor@E3FhoZX%NTwXf-ZYi{S< zePNnZFz(+kI1#j;2JgulSXfRKeOBazQ8Vn$izgvR>f%d5h*ewyuClr(^39b!3_w_P z&KHpbF^2TjGXi<{S8@L%RI~9W^_NJZ-cf-crEv~vr6k|iJbPt5%zjv6lXw(Exu*r(f!JMQz(QR-6l#Hs8&W#=`Z=HcEfCJ<;#o% z&yLX-%oyxgx}}S1IGo2R_38A}>ff~EPoSp3%)h`U87+`q&aaRI{nwx21bqa8y4F!B zU`~~D{@50vBWxsZ%2}WgVeY0XePocUb+hqiXiSAhXyR899CrTV9A?JEw4ogqXkE}@ zAM96+7N{cI}el*0Pyg zh}a@ll!CrqOiw5kkV&h&&{mn$)*)JIHE^o#;56bBlTkF<79+| zP@qEgHy3$u4_W(L7Mji(psq2ja0W)djmE_{Y+-tWTT=wg8z)D)MEKs%bax|Bg z8xonV99{_*>^rtlg|&Po@B6jj6_MtHRk@FApG=NeDxS^=1aY|Usz4`9p+Ez7LYBbi zvQQD_P0T+~p~-sC?kgmLB?7md5*Xu*O%Q=6sg6sCsA1SYv zJGrJ)u}`?`^wHkAoLy|W;NEiH>XAp=FNxl{l?$Ti9pb$wGps-NYV|Lq;sAm^PcIm4 z>s^Fl67$#*77Hv+jh+J}a@VJ)&PKs3BTS>Ql zOm|%}tzjJo+vaN@_^%aHXmvxk8~?|b0S{L(n64fiGqX>Zpv63DA^HcU@M4oVh+qC_ zvi?SWaDxg3MuNfTd;ab65Rv^5r9vPnvY~{4d?;Y?W0UW)`kK)Ah(5EypZSig>MK+q zotbX?mhBu%)@X|wQ@9#a+Lo%oMSEI4_1F&=HoqZt88F3t0IsFmRWS2a4B}-Y#*CP+ zDzV+&w(eSxN%BPQRd5Lf0&icCJ;{Jf9Guy~XQlyU7TETg9m-h~vu`lX7zBe(13}5b zMa?)KQo!XE5#S$!b{#~HJ3YvjTch9g&)Il$Fs%oI#I|i1gM85-k(m~#>X!r zOQz=##wa@*@IG)I!sKelEPJ0x21S$lQ5zH}=O@{*6_f)z+k;LG$a3 zg|*@RG)HBBeOp(5*MDRe@&VSiCM-FW+owYHb=@rr%`Mi@0=|ga*jmlfb{J51k1As7 zHTM*i@#>kQOa@FV(-9qs0jt=Kjv;#lHB&xyMghYaLz8;F04%O{h^!50$Tg}2197YD?&wV{I^+vFKr4U@QK*kk`09CCR@Z8E%r;kHQSSu zC6?@csVhw=mN?sgKWS7Exk^LQZNeW8RwBP^+MoTSsi|62p#%q0m4`Lr>^Nqh4WwcO z$6lJ#(#q2kPl0)TI53p0uzvDFj7Jf%+o=5)(1N5M7y;p4OKlpm2f0)^p0G8Nx(+EC zGYuZpFi(DvS&2e-w}D8D3tBOpc0F$I=ycTC*5<6KS>aKn8EZMw=rn|aP@GH=pG4nc zveI3~Un^S{vX+O$---GbvD(76@dm|dU0D0eJ>@5(IT0eS;Fg2&4JgJmXR@wt;9(7X z+YkX(kq356uQ!0@G@OKJb}JYgZRnf#Hz2J`9Eet8Cn!dfn>xwbe>Z)nT}NM`{VKl3 zPFV;foimCm341@-iMfLPwzn{&>FCCDU1HPL*`V%+ufra!KC*E%@d+F(+emnCR$8O6 z!iZL4<_*Q!lq~o)i(k$gK%6?qIDEzRp0(Ab$`+y>vVB7X+b>0Xx5Ec;g5fFX{7O^(R|>nLZF zufl?_Y-bo^dOXh7Hys23EvlEDLVy*P?j=O#EEq3hdRl%R>IbaE%2gBOYVz|vWA=Ie zh8p=}!+=QlhtZyBq*zKHTpQYHk0}iybuio>Y-EmlAG;0uM#TM}n&8nrE_blwZ}6KL zzkNAt$gRGRe!ca3dlYUUThqvWbXVxDk(E9?YwE3W+fc9d?z+fs5U({m!~>H26_XeEcn?eXedUJ#9admmTt1LyNo8nPZC`q zdy&^3?N*Rm(brwh)89cuH#C8G?V#8~=w1C2oVOuU|7nTQ9?T*lV1Wg@M=1O=tnzz? z1*nHzCYV5aasSt%G$7Uj=+m?!U{@yEeyu5maZz+c#*EpZBQBi*K4#yK$?!@iE?;DF z*>!vGb+D*HJE{x=&$+-gZbq7XmgP^tp6eJ1(RNo8f_B))lrYrU(2o+J9x!hTl`>h(KD{I2FP8&s`*g>+KP>ec8~XUw*F zSkGDWmh~C)@|D^eZ8&sbm1$DmVF)lC!lA5{U>i*Z+kag~vz88PFpRzv2K+Y{0Z4u7 zfj5@F@9e0XgZ7*}72DVoV&Xscb`ZCWrr}#VA1buB=a*e~5uEI&Tbue4SFU{Se5tj9 zWpObqn$NUt$MvUsca2$_8#G`GHZOzu)U$#&xp7+E;Gay289=KoNHZfhwZKQ~9VyHn z4Hzpy2{c!Wt)aC0Q~IN=IW;%N&2?;>s?hAimix4%gAp|Fyk{=WhzW1+f+=PvWpI2CWi3wHMyTnnitw7}_E%l)jHWvxX4Bxn(5>6qf@zzO{GXHH#ogpLqu4a zEqsasckNh;9-5mN4XOmPT9_HPTHV9`b0xVnRGCNJiS1c$rH|N&F4}mh54;IKYz<*~ z@I2RN`zQKJ)IDo^O1?BbeS7A;kvlzbdw8Ti`%1VaQfK_eOs&ys?U`oC)|hs!F_yPa zQv=5fk>*M55*9AY+FXD3Lm#i^@us2yajO}R36CO^+iktDB27w6;e{cjM=Y*gj41aD z4{Dl|i6|VhURX$uF#lKloRat%)|seZ_Y~m~6XY{FC8u+)v`9iJUC$X-1xz-{I!xTK zE-1AZ)b$bdcaq>K{8eF^L&xEtCyX+uu=#cmqBckFw)pPW)d!HIkK!N0*197$l*l)+2fCWj-|fLN~)_P z;cTLmMuhK>V<(osjzdwUv%I%o<{FsEhfHXiGW$T&p}UmAwxd`)F{>U?_U^D_ z>YwQBhum9XxAC0>`b-~4l52AE6TaXuU;AQ#jY3974R|(jChV@{4)otJ$fj7(+)gqkQf)xcx9DOUH9x%E9 zxW3m%fV68Z3Qzc}_Frre3hN6R7bBe|K9*|7i!BDsHWEWtG9t5sG^ zC`PKL_Lb`&*s|QIc()7o+l4upy4y(vw*;%Vx@u|Nu(Y(KGIuqj#R%awzTJh!4B7C& zwK}jmpStYrJ(nrzrcLAYIJe}YuP-NO? z&IYxA(rjHg=`-#IX!+3PM9c&dZhVvb@>vg_7JmwDwfTCt)<49;Vz^~4?lUQDgyO$a zKUIY0ne$6W1?|MYf2}mP>loc&(d^rM?e_~gdT8$4QlTolLP5qh2alR6*3m6xH7PGn zl*FAy$?LUIGfh#Qx0AwKY{Qof=%lGj1*09CIAtQ_qWU5BF-x%hFdrFqWrWn!C+&j5 z0H)hkHuaf%@m9Scv`24)@VtN7ze{DsqeP){_Oo!_<)V;Gt$wxqt~mkK+@oYU{4`CI z%($GJ+0|q^n?G8JwdKO2m0r2hzWg=Qyrl(=&|>tMp60gb#LMUQi^2CFEI-Wd6B7Ny z*nWx~@WVHvrHfGmK0oA?2QvdOKX}=}K*G}ANGk?iOZP7#R4A=I&+Un)rV%6~Q&iah zhqZT#uEg86y{m$XZQHhO+qP}nwr$(CZQDs@#i*q6<$v}*?Y8gX?0X*W(^{>q*D>cB zW6a+B4;w-uw=X>O5fO87iaxwAaMKS{wmhk4)_$60w4geBo1cm*=5EMn(AOQZHuJu^ z&6QuT1~3YH{(D(c&WZzNX2<(gqRa@-NmV5U%e8&Je#jx>;K=Iu5(^q@&v5+}mh}GC z0QL)^<&}&*;44}icBrXgv;;l2s(wG5vXjoIwXz##dKQ?A*+{Ehs%`5h^3BJq>blst`J(y&Yd!GOTUq;m;uyBPeu(K7i-!C`fiC2 zWW2#lc<~R6gnkmEMp`UjdeD52#u$%QBXfF;62Y5c2{8k$0#y6E{*B2>c5_3TgZF|9 z0M`tl#6jkE68`AKJ0leNipba}cnX`ttsmbi)Zvo6jB6CNs1Rq6o{#rq6cytjXgGz$ zc4ip8@<$^bR+Od?iH(BP9l&a49V||*gRv#+;)b4^`sMGYK|uj=IWIP2 z{q=!Sy{q0%R;aOPZ3xr_y>{Wnf=37^66v9+(V;t~g*3IZiTN+-1*EwRqQt~j#EWQf z3(RVUZ%umP5zI<W}J~+xO2nHc=CnOUoH0n7~qkiJ~FD7B!X%wD()yH7Z1*p~V z-Fk_~04Nc?c9qAVokMhY`Nyyz!ynzGzlOXF!9NIpG0|pIOCdLUGX%n*id2h78<)Ih zhuK68v$rZoo7plX)F3Bqcrt|KP?a`DjWD~`q|g3F8@VQ6$m*ezTrV?({Uey%G*lR9 z7U99IH+|MK4B;KA-XAq0s7rW0vNHFfI@HdlzbiG`Gml}nKw=QZyFkM6Jj6)TMB0)) z=Z_2ZIL7^{P{AjwKMzE}gJ8kCod^IDQ5J|*k4)73^)MnKPa!KR>wBnno=wl_&aCfF zB2xQePrN3?C-ofBy83IzfF~MrO$FhIbAcp@Cf?Jc`g6o2mRe*rjqpk;;i+W&T@hf8 z>uAC$G1ts&*SV6dh2Eo?x)J4QBszN= z6KifXu;){^OuwM%0~?+dAy-<|2-nyOS+ubW_}0heFIZMQp0^TZ?U`ir9pw;D=cI^t zqszY)7QZSjyGL>oh+$#v(7Gc5C*|#9*RvpRy78X_2?g0bS?gpPTW2DyynZN5wPy-*kkd&I^ZKvT z_7XxIG$ASAi7YF)XPU=(iV|946{a2;6!Bc93NMo(xIolJXO}$s%r5EjSuXA%8UhR1 z4}CjS#Y8Q^obRY5UpU55zW5M;a*+^jpZsbw zasl#j0kU;8gA+YBQ1KJ0wBYWE*hJ|=>y+|?^9+9(Am#NpI20mB#51S51SeV0c4 z99J2}zx?%7^99K`vStnrJ%{t2?s!zr?tkBIHaXJlo7{iQ@#mo9EbGP?O$)@7IB!f{ z%#O1#JdYmt3tDHGZeHmFQkD7zt&f0f=#4}|x)GwnFQZ-sv1g)QWbs0Mrf}=#o0B<= zJG@3Z_LmT$3(L^$OigS%;%|?5>L_Fn?;q-cO%g26KYM`jLOAs}EJ?5i#I96EsWM)W zvO@do3c9pP9a1wsD%?$0HA_UVu?ueem3fci#f*4Ush~S)E|5-fp#f4D;4!Cp-Rdd9tWr-z`BSU(kgQTzWCBCgUQQ~XtEu%PeR70XqY+q;@L@g-yEzf;JCDRAVqlt(J-&i%76MmZo{7F31M$Mt9k{Ypo+7(^(-Ryc zeJc+`<#NX5mQK@yo>T%?Jn^iw4Ky>Z^ zTKFx4%ohUW3Allb7eA;I>0459f2w#Qvw!9#LhIJC{hmq9r_OYMB2N{uyN|qi=h=Rtl4cKV~zHb3ykiaIa*m_dig%i!df`Mdo14lV5Q0#SJUrhIK~%mp-q_j+aL#b{eI>3 zzD@>GmmE>ygX4^3XTwZ%N;H&N;+Mx~T9F-x^7zBY5=Dq4~Vv{VJ4D zPpTvrFVlm+w2bvD{oQ|O8BH|feE?m_58bYDp+}RL2s6JS+-mZs7tVDT?#gwH3393a znS+<(>+D}n{0lr{`ZTH^KeQbGliHW@f2Z~pwEku^e5Z^$|0^4_M@h#HQ30Mu>P=n0 zAjFs4a;qjXMqKy`hU@z1^ zgHD(pC4k%C?GO614Y~R-R>T;DWdcIB7DI=DZ zBg!lF(lSZ#3YXuBFqNr7qy*AC=kn2O7e2IUbUElaLTxZ*1!~-Dj+mejRmi8dLTs6# zZvhvoXUc3HQ_g8E?K=kV53?2-Yv>GRj&E#;t4>CgbX8>2kC12~Q8%7B9iVtV>5qu- zXWJ=}Z+pd2ef(?)yKUH0T*Xj)go(j#1U>G%xiSvYgU%C<>K&gIliO>eE2O5LibRcS z)A@>3&t{A}Ry>0)Zu5|XMCVTINN-X@w}*kTwNGFG{PUfR?|bFk@&DsEi} zDKrVx8E%#?j~Z4x6rTrAdw**LPZUi~PAq1M;%0nw$zUu7fVYKRMj4YR$?hapu&=5E3G<*USUDpRYQFz`yTLGBEWlkP6oRX0 zW(J@ffPwiczA6Fp(SG#z%f(BCL3pM=nNcI!K6)kmtB|?ljxH^}9k6=eL4yCYZ2f;u zdi|HA|KGB8Ap>h`6GtcFe|`R|noCr*{y|y&m9<@x4Q;&ujRq$~mM$wPNY<5>DoE98 zrr9Kfugx(nz369FG9^vP_upP7)AL5ab*9k+1c2#8p%O`<_5$|tzNwmc%RZzQi^T~l zlS0oNW_rCo&nL#ud0!vT(EYgHH2Z-+r2J|65#kZ*@%NQ}s)2gb?sxiQ0og*n3ij7% zE)z^jppkiTL^v?^PxvC>j%PbNEz%8!$s+j9LgniS)4U6dYg0N@1P2jj+`tO<^UCfX z8PkVjI@>c2PXyo#Ump~+h2!smE)X4n#5y`^4~lpb8$cc*B1?;>k9t5zU?=4tHiN)G zKrp?^3x2Oh+k+V(e#MHX3uzB>hmsgG5FTL8ioZgC=h%#v2XkSc;HLRTNI>Wv`24af zrkS0`w!rEsrY2Vqk|--oiY+Nq0){RyaUEUwe!D(}w+nYKO^Kv@kaSdyx`f&=UuNpww(z~s$1|^EL8EAWD%2^BB z&XDVTff1T2zO`Xl@H?`4A2+YJm?S5lb%h#MUL?W)PZeMd93!r}cR|yM*|5~?QF&Am zJMud#L8ZPDWEu%EwwnBRxCOd17VNzuhRC~;anF<2%3_afQO<$Eb#(UU9%g$%%w_A5 zX-%ds3a*g8yxi3+e30F+GD~RbtDtMPD-k5gL?c%RgwSVs?&>I$(^963Us@alSItl8usEY3aT6hDY_J?Evy!*Szp!#BbYPx8t8w?3F{>=gSb|NB zNb()9Qk-z0ft*AI`_qN;{xXUS)i*Cf*b zY2u-T{6Y2^MqAOBs8*Y%DhpysbeRQKHz${9D%6^Q$bl9r}VxMd#y zy+7rj9G8ej$LU;pgQI_tNe7(g?I7%S!uDRq|CEPsh==@~T|oEDCF~WiVG(&{CPfAO zfn>0OY_NOq&nm@aot~uKOszna&+&qZsmvY}m6QyhiwBLj330c)H`SsN$|SnkFnrvC z+9b3Hb97HDQ2c_tD5H3*YFbgR$Rtp~jEzDu9D!Wn>o>Jc5DGr{dHJyG%AoxH{+UdY zKgF&*VphI*=AzgWRWxFMj{!Zd>#0@(i?k!=9hJLpv)HHDeI$8)I;piylJK=!Yj zTxJ%L)#UpYpZ$+FK5sAzEr|Y)TeItZG1OT3T5x3rdhlV+G1v0?l7$K$|VKtW=t5 zn_F(=JZHV=GNjQB!at9EUUr@4I9{@y=DwXyV|!ize1DABHY5myKO)>DhZYEP;q0V` z?cru3++>Gwgt=&U-+*b6ZsNl_LS585HiWuxca>o)LfLS4onb86V?&rBO|;4S*aOPY zxQ+-{k|DG0dV>}$*XH&EdIO#z6zDwHp$TR*pcp%!R7w<7QyTVx@pB9N>d@{tv2 z-%!}l80ai?=TLN^Z5h;84Aypq*eP^P4Bkn2WAJY{v@m+D!D}&A5&RSXFly)(I`2sU z)(*5542+L0NFyMw=Tz1A@#UFW7JY7sS_K5901O z1-Bz*2@^C~RaUD(;6~MlfMxqYYo%4-VeAx+tZkTZz#c3?3@-ktL4UkW7%n87r##G} zlG~<|p;Tl!9k(E&usH%XW<^i<8|rbO)TQiKb_(VC^7`Q^{9v;Pd>4NTWv<{yye@Lz zTcVkU3TtlQH@7H2NFm>-TECcI46&B2!Hy1C5i=ZcUBR=0hX0dYy^+=OTR(ZwAY3I! zWtw?LKR1QMX%q6SCr(~C>axIgW^H|EsThp;u$mXjtZ9Q*#pu$Th-nqQU6xd> zu^{$BgSuC^1Uo{QEQ`gCPw6`wW~AMm>K$O4M~7O7(>J~;*sYu~ z$OR+SVAYFC%=34)m5(2M*=j>Y)vWpD`~%sRk+F>{W*B=y&FSQHW*AeWR#cN=izKno zU-SlwI&HS*M6@eUaiyV9f-9zMSPXL@RFzyK%2GXQZPZxM0egDagdbaojT-(v1->Fv zjNd#CUFkCu5aep`(3aBIQdXKqO&cf#KM7Gh$k60{$(AT`rN-@FZc(^#uPveYRBbi3 zz0uYgOpQbqp_eQ>&%cUx>v10{@6^k1A#7KGcGG{upj%w64DISuSB7k&(Cx5W;^XJ1 z&o47}*zUgBG@Uu>LuQnRZ3W>tc&f?UCe#cY)kp>?cd6(7sF;Bs!el`v>mcl#XlO6f zSTm{)0Qt0Z-$3;vpN-7%<-t}8GFUvoIM*$FipTh=6Q%hM*1;o=E*ny)N z?HF0_J6{@bSa?C`RH&h(^y`elh8_bXy0C#GcDW)xY`6 zqXYA%IFKV|b#P0|`~504Tc)bHxq(2L{+hA4c%%FYU8%Jm0av^g}qH-f$M zEniE75ViXz?yu(mV7W6fJUjxp5n9k3}G!I6U3AP zcCe!+GM(RMg*RGCpY;i`d?R4uv%ckdNPtR)92GI0nVpXzV8W-IT{yU5MS^;+0MA;k z>W~{~qfy!c)2Wdu0`APP<~msF2)eKNDeIK7(+p|%bn4qqzOSSPuuPSWT$;qBl6+!OjpweIDAlxb z6s%#9$PygTZ3k9tv6eCviA5ONRRXJ=2uMkdkQtJ$4$J~juu~xGatB$kheA;hqaA5L zo;o3ipEcmp?NfG(m-0p0q+TFZpp?5j;F2tmUKG1}v2>^> zNwRGjSl39BZG;);@1!T)sPb+o{-uK5Fc8@X`kg@_Pmac6L@RlChC?Aek>oh{l&Fg|)v4)q$oq zp8MD#rQ{T4+-7hgUHf91TW6;d32PLoO^${EtNh!a{LwI3TUd-gKs(+4_NR9_T5J!x zOxdDA^2992Xj$}JO_Sa&yr-NDWmA&5EuD2MXBdOp>3T_fbnZ2$@IVT^W;{hQO1fDy zl*6L=%n!DCRKf8eV&r^va^hV=WpI*o^5jBQa-s}JYGhHR$SHRX^eCi3_C&n6FGv)2 zl4M9%gF<;?X;K?L#^3(8UvU2fP)WH#Iw^ez7FePFr_r1Df7h^yxLO$fW2Yzn?=QmN z`D6cT-$s83qHhho!JXvwLu4w;Mit3uSlq+=Bz$O=Xwt?=jyE`rp_>Dna~i;HD7$ZXQvnr zV{=bWqjxZ~H5JpV!Q?uVJ1?ZK$neZqE50G>!0C8X7mzj3b^M{SN*)Frc#k<&Rnge$QA&^aS5CWx~EM?IC%qR0^hkr7OJPeZRjP?U!0;kvLo zL+$3Ci*aT965QA7+*~1cVx}Vaf}q*_phXOg6SUK7Bz+G1In6&>MP-1Mx%@kK zH(hU2aLFuughd>k6oK42T@YL$k1UhS&wsYPGla9H%OA6)%s+fn%fE2+19Z8vVG^kF zg)m1T-X6l2y8t8=VL)m52*c1lR`t9b32G_o?@!)Ff457t<-h#*Sb}k_@F;p&<#F$9 z3;eFZ)|{1X1Y#v|;U3lIB6ppcu`>+Km2Gu#_yT z3F8D^Dr|9EriklfrTm526OGtsfe@V>fr7|LNhn?wZYY^v9hn|(I8JI0xipw9zE6D* zkv$|06vdwFAItHpLcTiq8C9rNeuv(x^dQ}KNxjC-o{AjpHhOstc+WMxk>kKBkcIhLwG2g`{1T=z`>2E6|ox z+AvjUMCHxBW}D90Uzw)g@#mb&l83>b!=mHf%aY;Ngo48S= zjbClzBEG{B^9gn4U~COF)RlZjM@TQZg;u7-A~) zkS+TIgsDqN6YDPCiaGfc`jabJEnL#)hDiAn3)!(NeO)vb@=k!DH|m(RFrjz(ONbrU z(gy~^4t`7vs4UM2+?OL8I$dJMf&Fy8?ocCcFr@s=TmSLJL7bms4`e@Zey z68(+rdq+F*m zTazS86BpY(Q{@%JeU#|Y^qwR6t~j0fHtqzjDn(6%7#}+v6Z_vt!TDvR<5+J zR_?UvS$)iU*)dKSD*`S1{S#LBlKW?;Y5QeX>6Ygn-QVNY4VHFlz|09y8rp-Yy9|JW zp|fz;7@}kOkQ}1I)Lj8&hU!_p_Jp=Ee+Ue1VeU?d5U@gKQvnnLkqZ>|xdUW?3<79?babP~I=PPziGXH+&>+1z;W!C_wm^6@ z!i)e6Kc)1m`bU8BCWLygXVPrm8foy-&fBrpAg@thEBk4H+|Jj-=q_G4ZQ<7|js!Gb z;>U4O!;g4zdVv0bz5_&X_WDyeaYtBy+AiItL*?|-VsG`sVs8&0IAQlKallLty@2En zpg4Q!kofv@Y1ba%wM&ihiG23e>->bQlqP8McC8r< zzTa>;!_pB|nFnuCW3lGkvyRA3rL>I{1uAfjBqJSeFvnMK;l_q`^!U~WD^Ju{Y?fm% z6dr(+L59gJW91MT%amek2xzb`>Y6xX(qzq4@~gS2NUowXd2;ffB2|h92DnCWzt^v>D)+N^W!Gd=ecu zXFDlvKKI0$Qy{0!>tV_gVIWU4gQ*fHZtXf`o~lH`XH7AAHi;YJ;1Upj5$9sfNOTn~ zBcMcFfmefJ^x^KFHP|2?)j^4GIa>A!X^dG@b8PDDnn~dLn3``9Y%NI|#sNK7KEQs0 z=p;1_-dL0ovBBNeHhA5fNW?3OZPJ>h5JEAWsXyOgSk6N-+F`v203F=YG`JV%ph!!a zA`~q*X%a;EL$Q?uI%}yJ;gi-@ef0cMEd0S@9pM9SlbB4rD=!&^q`0=!QZQ=Ierd1X z!Gg(PS;)Lss;;bTte{Rcs-G3EiWx!Fw_v~-#wxrSBj3*`T;O!B(fkY_q(1vsbP4?Z zW}kf&cjK=$iVvvJDD4Y`kg(7=muGgt&=7&%`7?H`WBs55k#WGY`f%xmr6nUt!|=wY z>Ck$sL;76u)*wYj#g0{7wEP4ELEWq+3}~hdhcIAnAAH#0zR>#$CPYnT@gO zC^S$ZMJ8F(jqo)E^e=$FAjqY02MO{3oY6mHdF@0#G;mH7=J40glU z;BI%QohQF4A5deOM2$OBXfGhE5iCv`fVRrzqe86+5rPh~kzmLKsk)}XN2Q@2G+0cV zG-pLak-4u;#1o9#ZaZ!2wtl;9weO%d9Cu*3A>JOCgE4XdGb=wHwnzH9XL)@KdL@%~ zA|`Ad8nqIyWdS(D44>ncw3F8(#bZ?7a7XJt@spNt4S^;s0AlWHlCgwA;?+ndb*xz< z6?e?4{t{{+D~-Qz8lNbVH0Ow2i6h^(LY)yI@|&?&Q#}*9)aQdCjP2~JTQOWYzJg7r zvmd8(7ekC8m(`2ayw&pSAr*^S7441roL$RGnK)u9c$bXz1!nsvigUXHu1o4~z;lJc zD<{gRX-gVfw1yTMLz9bfXbq@z2`EuYX)9F5BwVhgCM(x`;oNB(yTn7vu2@OenN<=t zx-F;n$CFIs3BWTq77o+rLzW>s>7ld;fQ&kkjp4{X%k!PXk&djklR!}|`be%s9L+cd4G!t6G05~0yO zjh~!88fxOrP7j9`w>@CU>c`}aA$n0#T?=|U&ASTmXny?kdgSYV@tfG^@pS$XrGB zUlZ9NnUXJo6ZNnx++40`t&lYS$+3Oz*q2$0d{%cdWzU#`hc#BHGd{VkbP#oveEJ^b zx|d4#$aRy=Zo24Kf?KP;A3k138uS8C9c>8S;_+g9DVFEL#bp6~v!sMvhG0Hn&UEA? zt-@eem>K3~fhsF^RP;(^xRi%V%pBg3AS&PZOuA%5lY8cI#oD9VI|g;g?$kP^os#DP zw)03Vn-iw(a&mvy&ea`RIx{NN^k-~=N4X?cZtUaF89cia(7wv;jw51_1e43057P|c zU8Y)o({{4~bnGr(nuwHOiHSH++dkv<6R!2;D#mpgbUtE#F&5(d?bR7C$yI^m0j7I| z-p+VD0Xo0VmCcO8u#%E5Ni{6s4Ej|`kY@6D=J@V}vbk&tvYY{4$-=yib?$JX)>-Eh z5MPue`%)qrrFNQvcccvL#w!pyJKK>wHA}d5jN~0r^?ls8#DiEJZjah`ce=72P4P`h zo;O3W3~yV6^luGN10g5yc`i zz70MtXFq=5lQ?zAPWNW#ZEDqzuEk+bl17b(C(`yXVt2)uM&=?cCB;2rSRnZK} zI??CKr2yHT$*Z2k1~k0@jAtmdoc2HDtX#J9roMO`DrB$Stif}{Zx`}uL}EGv+#QJ? zjX8B!D(osZ`x$$q-#o!J`yBUk?w0`j%ns;|x0%1o*y*gyV5c3oqS#9ri{bYrRJmF#)Xy1Dgvi(uDyJzjhN zmA+`6!;L2b`QwKa#((Nj|Cc%f6{l}LJF&2dy@{>ye{>LhIiIyoY!OY-!6~~xs!RBvbq`Y`M zJ)OZc=OJe{`}g(jjmjT7)mS?|A+etPpf#8a@wa3g1{&(iJBYa1_UT?hOkz3OKhAgAa0kN$BSwJ! zazLYT&Adh27lAFCTbMnA(?44p5C1Vi8=7rgv?3`0r?C!x&T3OcrTB7O1%-rOEPbam?WV(3v4u z!#GYY1_j#Y+Qv!yK6jgij+)_GJY@}#=dyZ5EPyntxLyxHv+!;6>5D@%5j<86>KlZBTtlgsxeWEaZ3CWnbe7vL*3XMsUs64ytSm{th%7^KFh zB`r;{9!df}&%g`!=I?jQ{u^z6%0+mLdCn&o2%CHwu*iQ6{`0Hh_Lb0}t?Y>*=7v({ zT#2wu!a9LYJ~{y}I;NPXQv3E}^z0sF54|8L!Z zm%EY~#y30mMj~km)liGQWM{HHg_*8Gv+6i`RI_=z9g0p6H+;Dml8uNIUV`8htz7PJM)d zF*oThJF0@QX3R}@1jeXKE(YTYu}CzY1>zBfZ`U#u%F1XkB<++EVX%t3x&4fF+O7&&|76S9}2 z-6=48i~w|RBt4MIJuC2*pm_h!YkD9aVeNir{RfeNZ!|EZFQxP+Hk^b%NJaM74dU-c zE6tR`qGkuY;<;j5vgUW zx`>t8YJT&cI9dnUedlj~0F1KU7gUN8t3r>+~ z5E0b_rvX+~6mGyEo7DmfeXMMTnW@|>%VSyWK4*LucI%xK@ip0Rk z6p|%-jhuKv+i{`Bln!-~laB+7mXBt5wq7&xBrX>NLb4GhWIlu=ZZ#u|Fd4II*g=5^ zP7>(JGh~`D&ALQ9gGA0~w|Mj@kMfeUnT%R(HD{k@1{Wc!;}yGQonuh*PIGyy^7?3) z5eWmGkf`~S`S6laum!pFxfJCa=lp)a_GKSDb-} z6MJ>+Fy`)fJ$*0Te3=;nHs%Yn=kwF&v)h+)dSSQq?{4j`Z0yu;SP{VpFJkq!yJ*D9 zP9cOIHV)QmtRP=Zq{Kl)f?^L{k|-$ICLy?Qt4Rz!=>RXh4Cc=sB4XG2Kfph0wRbl+ zGqI{sfTQ@(iA5J%?bQbf2ZRH2JPoy&84)pM?s1JFM@ZJ$jedz&$3nzGkd4uwPFs~kUIvXC&6^qAITAG?|v=I&PO@nu6}c-oT+26Xaa$~>mO zQJ9iYXOK_<$jx5fVFo6_KJIise*#qzd;>#>?&JxXYvtB}lE{!TA{aXa@1DJ1CJWl# zs8jQ;Qsi@SW5O8w49*WTCUo#6*T{LaKMwPBtw_zX3|`5=+d9>=3ZNVhasIykEWM}B zxbQx^*3(3!gu03mqalwBY_KS~j^{OCOtnvbtwC7wTMfrxyIIu&^F!Z}E<5I7+FKo^ z^Wa7;ZMbVO9#CP3-~ zqKbVGbuF-bkfT1Jqj`s0J?8b6$ra*e_N*<`jOuHpb0+Tx5#>Gen}WHlz7}LokRz!@ zW0Vf?^Ric*%3>#Gx#eC zv((LXcKZ$8g*RzsLuMMtMJXV*?hF$(;-;*=do0_bOrYIq)iMIZ8(;O_=X@CHamIm= zUEoS@NPYAj=kS0O9tw-1TX#^JS&ng)uSjj_N)9n<@ODV}Fm-+_Q7}rl*@4lDn6PT0 zns()K)wo8QN@ePtrX^XJ>cxU}$rln!qhB-*6Ot9HLs={F>{H?|tX+Z6!Yc1)#u^kIG2>|T&Nw$1Dr=d`v^nlqiU zg_mt6f_@cZ&s?=dy5+PrtyRh{t!1(<>zQ71;GK4t+3he<+XRC53ui2_wFU4OB4;xL6e^ zPt_IW#B$?gEF?s*U+w)08t+}ik7cC=Dnkre=V05a;^6(&yI`hRuyRo+N1)nZewB)#9m15S~ zd|E2ekt!>z*BFG%ww}l_Aeo%76)Y(pps*2bX$GAt@iI4{irImZu^dCSz*74R z&RGE=xGCWhQRo${(h@dOr)m)D8AkAPa(i%U3&u@Tk~FY_0G}jkXspzpRg{8!(WHk6 z>0UX`Vw&XAk>#MWnBf}Uybwxm;?oJt;c)G0ItId2p^8`%bo?o`RGGw9+vW5;x!BcV zH~pQi@j^x2SvRXQ`lk?X*s$k?Q4XjWsr~T>Wa6 z+3H_s>Nl2Cwcs)?(kctLc&M35B_)X)gvZ<0QWh4#h$Nh(jO>K4QX{1khz!SvLyd>C{V?X^BB6Bx(`2+w&iCcPgvAdZUxI0vW)2oK@4y#FjGJpv9K9Y`Vo z3Y7P3c!6CFpTa&DsR=-W=IjiieJn`9OALSrkn56<&F>ch%0qY!ghE-Tghver*QN$g z{B(L^M59cAZ}0yG0CAz5i(R9m`0Gw9~Rg7b1Q8I$Gg!tA5e-r zAy_c(G|3nzXkeq$G`PIknQ9Zz>raOr?)+!f#g?fPk<27%jm)q>oNZ!h>G+@4t|mHk z$QVw9NV^idshKjKUObyMbUmBiFhH1jG$fJ%swQAp$ovYXp(wZ6ym|}m9LCv(#k-&v zr?=L88i3&=N_?P2vyK@K(g~w*i5J#E_T8z$=*@?_jR^KI_pezhs>Z;RbNd+{$j>8k zE}cV-TsS(#3rdY4L}K(Ts0aRxw1#jYyI=;LWjycM)>O}b>CowVl6puyHl4z4g~dt3?3T`0&*2cv_2;qVhZb2rz;{0rP4CX zsdTTQB}~zYkrg|XA*F*VQL8l_Lop>P$CY|QfDaL>S}>zO(Br5n%VG~PlB^L%rl}=P zs0_HuX`U==ws1@*tE<=K)?tmDt%P+8ETKK0bIa+&HQ{60%pcT(StQMwoRi+W7PWQ=jAT5#f}FCjYZQt@c~$DG&3ou%0eytgEGiF5(7T zk@%_{H3zGKcS{zThglVA$R(+q)-n=P=Z1SsI>MuH`CJo8W;93y1MjnG?3{qdpNRr- z@`h-+p%h}Om*2*Kb+&MK4#(FCM^{{g=C$OGZ8FM8TOEZgvCNK#5hSCqx?n^ub9#Cm@bzc=i5WI zE^d)JerGi*5UgaGy;{y8ZiKx`oLi_xmS@OE6u&+>@0cvvy4-7#CDIl3@Ce0GQpE!= z>jFo3?KGm1X)*1eKgVbrc@(X}IfQfErSj-@`#O-z$18zmz492C!&OZq~& zU)M}@83yU}(0;s;f8V{g0e%0wwma+owgz(S_=$NsjP7J{n!0z{0qc$(3rI`BE^)x2 zxQ;s4b^1I)(xScEv60hqILOS4UR))@ z9mDUmx{p5-AH1PpuQ{+GUiMUXXW2G^Fd`qc@#`pccYLv1?lYcFbytN^?cM*HXQ;k^iHx$`DRU zIYVy0$lV{Om_59})4qeKJ(dx(clj=WtyDLC^D?Nk*N(>W^t-SpU2Cr%&51G6REVgy zm`5ghj*?P4hC&^fP4;M0Q+Z!R+lV4>r0#G{RgTXsuDm{LcxmFD`xnis=q3TI^0F6L zeS47$I%`Xp!4E1|vyS@MLo@lux%NfG9l=iFu|7Q7_Zyo7Tc@yk2?Ow3RdXBk!3#(( zhNK`=kvX$g53C9g@EOqcbw2=xooO7#GOV>fu-QP6>#`9MQ7?ql`~Z(0`bN{}{dC6W z6?KU9%Pi^F)Oy;X8rssRr0*JdpBp$98?3{uud$x7oW!tuG+qP}n_Np%1c9*(r+qP|^ z3thUk&woeUdmqj|XFbe_6)_)Xbv%{t3Xb6PZ)c+adgKcK@qw=tWx2Ojo1!>)MK(Tp&E^8u_F&11igvW_5V5Yq- z#uL{+wcL&1tdY=4+*H$JiRtn@Fjq$)uRi3I@&vT#RMK4kGX$u1`(#UfhVCS5v)SlH z+=V6J96Pjwku9C!AXq7JKWHP>xzEnV6{D_LJyU9|hSU@%O6Gi9<_zV@G-p-B^}?w2 z7LsSh!OJ+#n5v^JhyrmSsTEY=z&TevEK}^n1y@Awi3xX?iFX9WKXoi?|5=mbG9S(b z+|l`os7LU6WiBtvXAMqklOge~>8@2_?ASkJGoY`<<#0XQmPMcFLZ+2H+<4|ks<}&# z&^1UaH^+0L?e$kdc3;=fe)p_(5}gUow4{&jBICT{`aIYO{n-{6$HMWTctbnHZU^+x zp{?4f0B5!v9VJdF4Ucs)c#X4Rn6pteb-`cd^$+SN&D6;j+6?lumS06z_Q|ZyQD8hq zH0=e9D6$WhYG9Yv^eRFF`ZAwCXk_UNCOWJ4+c?~y{hg|`8=EG?dR=+#5aIC5pM4SX zK>LWY`;uKTu0)C&lie;o*~bhQ-mtMUt{?REr)axz+Bc@w%tLuMruMjd2QX&4S(A`>{ zo#Ti3@dFd@KMC{y-r@S6#OS|;`TuJn*5wOjs48~v;r2&ndt!0|3Mf~%C^!@sm-^e% zrY!~_5-iwjBMFWvW!^tYirA^%tApys7YyBAIjrWov;>u)#@lDIV$)UgM^jEFK(eEx@r5Co83%GVqK3TyVSJyN*1Imdc{3Tt-79xGg# z4X4OJ1~z`BQ5ayv%Ci%Yf;~O&*bH!C&92;o!k%8Jl^%G-&i%Cq42NOeDmh>chhfty zI)K6ED>cw#`BD$4v3#ij)L6Z=0~{=0QUMM&FR_3d>lY2UorPN(xSns(kNsIJwOG7Z zTsTH6cWhaoM*e2)TuiR8czBk{tCD14bB`;xoj-=d)+Uyu!{(Jl9#A=^!}{7u{#7`p z!TMT3E--i4Lj9?d(xZITR-s!q{-IRrJ_?hrAPf}siIRnWu@X?U2VH`{QL5Ie5d>#UTwR4+^(WiCdH%j{<)%4 zBIUW_o?q=#DCN2G9!PCib**&#s=`*|D5PSuU|gW$o=AT5BzN4awgb>4U|+5RT2CF%txyJ_E=G0VUdY!6v)0&KWNpO#`N?%ZWC z3g&SHIiMPpa~e!-p9^F1WCW#cU(|_dfDAT`*_PC>8*E=2gVs3J+9?Ajb(avJM9&#v zMZuD7+^3C^JzTOdXpOPxT#o+6gnJfjW}F0*8YRH^9Kd^s?iRxk(#Feo2JQ)Chz{Y=TN>lA zMnqxKTfE>6Y{A;W#xePd28O{xu;7j}h8FxRm;|fT>RIwk|E2>qVE#5-9e1P{xWT$M z@(e7<*&_l-VCI^x4&BlYJiszAc8=XL4V=IRSiDdStiXOh;NUIGfIje~p(2(ptQV{Z z7Jw<*5Oc&a8D=>WL(j4FWgN^R!hmUW=9YPY2DXOzJ?aM^?10{(au()q^VZxwalj5E z?`Unvo)7Tl3}fIPBF4LPBeShZYZ}q;tpxx9lb=+uFQ3VjsTIo=&<9&W&#AE~ADBU} zIYQsApFQP>G2jQgf%Q3h3p!vAD8tM(^UU32fURNa?7L+j=zt9{dr3D!7!bhhHhqcR zg8+OX6Eaf5+JB=!eF4A#V?YsRC;e)mMC&fG6W)LvtOL`#WS|M=FbnP#U}&Eg@PPH3 zNN_BE%8)VIoYG^Ud|v=`bpwun(g<@(5KX4dm`Y>urw&wY9?m=M;*9XezBG7Zr?7AT zZ8^^QAd5X@WF;oP`RdqhD~|9!JpcoX-{b}GW2Ai#cdp+NVSoqrhUq- z(~`=W5o{Jt__xZZf&ZR85^B{>`>0b&192>;huFtz#|_mucV>)w-X&}2#bj7IBJvEQ zM;PL9TnkZ;hPYW+FNE=`*oFKNHJ#y^SN*kgr!DKLQjom}nOWk$XOEnJM1JX;~ zlh%vv<<^VsUvT5XzHsy6M7V$~jp;)i5Z6mR$Z5xo>RCAA>=aLva7xqf6ihqFxUsBS zIs(N%7X?VsK4QXNI5V%v8?CFtGH(`5Sb1jBm$6LlMrKq!)ah3AR=RrOUik7N)FRI; z9?||zk5M>vhNoXR(#79^lFhQ{yK1j8b~|yAfVs8Tqc_zYj9Y4zON-JvA)E9?wsAy$ zc4rpI?HA!pS>lAtn=3)kICG|osyNaXU_W#Irtr4m>bSK9QxNvnCYIH~bDY)Z5`@4K z0l-VE_mwekmdrwBlwf`pCdkOaR$?tPva(c}afa;CKAzokSIwEh=vPimd9n?_1Ik>z zShmY1c6O=SeX$K@k63uOW|-fhW>${!#=4GNTGi4R>c6^vcDTMGH(%VFbqb}`?76i2 z;t?zy;dmC&?+li?y`tw9O|)Qb_kG_)uU|FM<9xkLFFXbLr3Z!O$shEyZb8(8HX;Mvx*d;lW>GU3anq`-*!hZcqn+@@_k2knUg z=8c4g7v+>HEtj~UoxG=v9re*C5){^*vMEGTa)+CLOt$-CI4WhFaQG9@MpdSCJ5WivuD>`E=le>cLm;Q2IHK{g~Kg`%|~BmpuoPAD-x>?!yZ6JbH;KN zBO0_Yi2Q2|_^+35pSA+0=%e?1AS=|OxB46LqWB%H9eaQ*l)VDs{v8LHb>XLW{SN^$ z&?Pn=HS~uBRXkjDxiO{lJ25^te*@BUBcH{W^YeG3>qLOvI@7k4e1BD{<`9auxIhn~DI&D;fwIaKcxfm$2_Bm?KEX^ORplB1*+RWq_}oLj(H(fPl5p*a1_3lZ1t8ie z6}jHhGb~~R>Qs-l^WN1U$-y*-q1E>=yQJ*Rrtp9#sj_fS(2*jIjS6;aphOy01`p?})??t~(R$GvUyPgeEhyd_Vp- z8e2hao=q7bSm_HJ-OrRmmhJd}x7ml}f;T14;x)MY$p5;rHOu^yJb;ngncF1z_0~SU z8VXzMB5pAh1dcT`A<2%ROdMswo^?tKd}?7|6nR%pr5af3 z+bj3V#Js599y{yP;XR22knm-4#}WEm5#rQ)os-@{L`yP0E!EiYt;SND_|qd28c!MbX#YL`=2QBp8}K4ZceIyX+zz%}f~VcCd|A z64Q(U!&yugK@m}4sOWG8iPXc%EPe$3hgFm#dB+*16ILCpvaoBmHh0rj(d?|^bTP?= z(CXy@b5437+sVUb6Sk!X=~O+V#!#W2qMvuNG*Yv)b&HEPl>|F(rjve}N-Kt0%%7>( z;dM#D;qEi`(;#hUp3R=n^qCP(RHs(wYes_8f+|ziR@%{bKxiH_+in+J%w#Uq7!|N| zc?-SOo_*@YTU9oJC0FKJhF&CBnyJtuWpooygYdwYOBm0GYkcvk>Y%)(Bfxek$jj8S zWX3`qk3+cS%Cjo;FD5E3rA(XU=ErDQ#vh5lyr@J=IxgHU^i$5Dx4stZ`=;d#bVvMZ zJ64C!4DsYz)|fD|dCC3+YK9Nzp2x3-_$LrNNzL6DKczxZQ7d!Gw78g}x(Te>U!YT{ zYoE0HIn+;ANG7a4QJ>42ppADZF09hG5mgsnCn+;GfBS@HOB5e(N{EkF+sm}5jVCu! zARS^Hcc+|Duao_eYr~u3c66dLSuF)geV(wxoU!A!C$9TKNts`v?25O)@`w24S@Bay zzWX?UsulvI^f$wcz-jvh5xE*9w|8D<@hT(__X`uB-8P#z!&%VJTzj|U<-F2Id|R54 zueea{SC1uz_~nnM)Z%114saujN* z?2*yW-G*w-TFTSbP7B-C3zgQe#>Q}w>SiOND=%;GwKr(Lc^dOUYgj3r@$a?T_~di% z*(zqbTb%f?`eVGafae(5Tmu&@K6@2iHccaSHPoYd9 zEjEwU^5$Dk<6_OQs<63k@sLar{XskC2c07Uhp9^qT{GcT1&umeN+O<{G|8<@=I+sC3=r zd*d>Rw1hO&Kc2;W6As?B2GFgAmF#a`H?t2I$$T-bn(Rt~dyooZi@xQ!Pc_!Cp$9kf zuXq8mE8cpt$Iqf7L_qX46t%pV`>%-kK74Kkv(jZ^0W~WsHgQ=vr25-FUBEAH{+z#C zMzCG8R}W8C-tH*uxA*vZ(wfG2xe;MSr7R9-z~HlXrQTT_yqDg`9*#98bH|3bQeaAb zj;FI&+`QFN0z=4UPvxrug|;;4VZvU?zCuA>Bm&b(1v8`zganr#wOmd#^0e9f_~DF( z4io+Ecz0SFr)QFnnzwaK9QXi$Cou#sA1^-&xNKx+{ zwsQlmsY+6cxJb_|2ct0t_7vCh-~NnWcsnrA)Pzqyv)Ogm{ptAK9lCB1GW(ZU&RSia zHmf~&A2$Ft(E=$L^+$gU-0?F}yOi+_>@5Z!7TfolGW!Xa)l-na^EYPJp(j(3PYkiA zVNVo5`{#9-wwemVaFH|*J&B(OOI+N|om!#mNL3aWn}WPr*FbOerqZst`=Ji-3&~&+ zWA{eo-UfTX$a(5D$os2GP){$=607O6BnFk?Z)Q)daRi|_difuQ2e#z|yL;>M$eue# za$Uii^;y_}M)RS@;vp$N?gGGH*GPH7xZ1(v{2G2m5%yAa)|v;AWp?2QL8obr5$}Wq zk;R~E1#MZ@mb-X~!Zf$5HM9BmCbf67nC@udQkOA#rD^70CLHLAFT=yksk+lwo1wAo zi*oZxO$rOONtRG1r&5cO5~V+n;;V?1uAcQ%FUxbbqOHcJ3G)>!5)Zatb7&;ZgXCn! zEiJ`uiI+~ehhCQ#*Qm92b~LpWn%+rsh3AryQV`VDKUgP7E)z5swS{@6L+2ux2Qt$S zJ{pDJ$XCd=FkBNCCKGw?$$uMAn!BoeZ+ZpbVA$u!Tg=%uOl4jB)h{qTu43)5YY-xD zW$SQLqER9MR!D9|P#n42eF`qJkyzHW4M*gVwn`g(w&R_H9HwFLOZ>4KpeJF%N;a{j zH4(}u%^R~Ks~lX?7v@xAsM?I8CVI@TE#+&}s(@4-Ig3{kr1{BvyOYd$SfTXlFS8Go3l~$uVKK6n?Wp)%ziW2(hppEW^~P-JJ82OEt~Mt zYkN<_gn2Fkkx-g>|W!AxM2$M zq%iSH2N(c)jw75P8JJ5M6Kl~qO}}>uD^1uM?Ry7GjD%zJ2NZ=Pve5;F7ZlzmAi@PR6YT{$dtQV*w?W_=i|gF z!~Q8-t#-6vdFS^xA2$(2S5l&d6p*SMuO&K$E}EPSfQnv3Sj& zE9NQnd!OjTnVL~!d9|VJGaKu7xMf`YL{3ljS@iZz4Gyl(&bnM1>k7z6%iQ)- z>s`IZ!n2u6e#Nnb@i0!xF^oIyHamH^;Kz0ifDjEo0C{Ow2z>6{F@&NNsKxoThu!|8 zch_AVzu#39ILG(duHKst;;lqw=(&zzb#}-7=Bdd&L4+{OLADM8l`pX^T9+Doeb)=b7Wt?bmsE0buZxM6SE4wugW5u!VlFs)=U+jgHfq1KF=8XVxz+#zl-Zyc9Y~gcd-WP zlAnZq@q-`ugWQGlm)Sj6!4z*NsY+E5p$a$t2Mfv5Ty~f{-y6Evr2)6mVLF_YE!Ko-$F0|>Nnl@|i7Z%xV61?xvYqTHSIX}-S*PutNq&(vF zH#*%FMo1PxL=hiYV?_AD{Nt6uS{drK1~03} z=a%K61yO)GK(962|3PRs>xFXf?a`H2Qt{No(FsRarxQfJ+&*N7??)O&8t;#+NEw2_ ziO323K$>eZ^q>F|UPxvrg);G_co>=jj>RCLKa7yUi$hs2jdY%qFF`p~bl8*oJuJ-MOT8fxfT6pUc)c&*iW!{+G8C zn6_kUqwP$q!b=}If;5QL@k;SBY)q|Q9qD_w(uJ5<8++)+P1f24g%H89SL%wL;ziij zyoBH!ZdIeT)bP?3mKDcFEqyHdb`aTWdb;c20zM6}L!*&GRjtK~v+JQ77tE{!VtPmp z{;;3I0${}%LUJI%P>)L(N7%?*Jtupd-7VMhLF+iM^UPM z@)&g!B@H|uOuhV#_@j`I2GbM<891T}%;!eGJ)xGigY^n$fB)Uwn4tGNy|P1Xgo%XS z7H*&*_n)V)K*A4F+vaBvTf{yzH zhZFJWg<6`{jXpVR7<@OJvcu7BSfJd53ny*x%BUs;r<3CCGQR!QQX@27f#s(I^_Q_A zZ!QZGj6}P9X8Vd*aDXuyclnY?Cw(q06xFm<lmB0xGs|3x?w`YB?`&9jKu)=90a2 zJWFQE<*M1KhL$lKyoY0yD&UH;qMniOItn0OS)x!X0A7>O@@Nwz0s~LtTvt$sa22?; zcvMsb7v<xV_$V&g*8h@f;rY4mN> zqHKq8?DD-}Z)4cMC_aY74Hn$iQ37TL6K>nmfen5_zjUO-8AOQPHfh3tp%c9bWy1+X z!0h%Ilf2XZ4(i%BU#G`MrPMFHzaUoWWKg{en;rBZhEZx&M6;j7;C{kb;}kvIH_s3o z4`&>$OE-R$VVvEmj!W0_6KY&(A9IMIl#*tU^$3$V*(ZI9jt14&9L<^9php_z7uYLcm*inkeiEsSgNG~#VHalk{OC$**0HQ_oe8fjc0y zDb1IRSNk*iQ^x^7J3~9kipsp8EJ8KO+quBb7(yWy;qXLTFnDN#F@fm5fSwK4N398S|ZTq5-CXL&U7kXla6d zPkFpS=h=U6ii(oN3_3C?v0|^vQU05Pwz6~>L9^NZ420d%>+VdD-$+rWH;FPUe63JI zEwpDQ2fJ;(^|l6%;q}-ft6yUSk*#jmI{9iDVuy=V^~7zVX8xw6TfeU1)8Nr3NWufY zV+#=e{IF&*#jz*Z$b4Du$}ZJ2YP|`eF)@aY@!{DVKPI+0wF~CjGS@I4<>`a=THDzB z3=@Ci+tzJ}??6%9q`*g0Q1t&m-;UZikhy|bHyS+=5(0B(M1fWc6Z+NXI2k;58r+r+ zQf5eBMY0~I`9s5)!4nNGah`y`H>ef@qECYpvs)|<(X)2;R&fp|JhIO-J^iPcF**7I zqx7IHEm~BqY<4t>C`c@_#cqNzUM!l%+_WM+rcnKAWT zEvUGeqcgHPP|If<>X-?O8$|M%7o+rti1QS4LDCmCDiyVa>QL@7;3&67Wz;$7C?qpX17JXfw>|WR0FYqe^2(3V$Q70l&&s3W^(!M~=sI`a_nF%-#aC<^2`i>a!UW3wh7?4> z#JarQ$+cS%s3=3clyQG7Q(da=A@R#bcsG=cOgB}l6)z#Ft`;j$DAbUS_!iino!`lU zFPzdf8Y+#2Ga+!eajZKGDV5?(OYB3oQMuV37ESX!>rWR~$(^O8A%+L%=o}kAFj2Or zYNP#!?Eoq=lxqZMZez|K1sl8%e+n%Rsa;i6m)Tp#5S|)?(>zD)0VGwaCF&CeZw_Pc zbc}ia05u;gmiHU_bgSDMG{GrF3-H-FU1wlQGW4%)*fxN^^%YRH&kkvyw`_76HR`9% zGk?~hl>}!v@kZfVtk~F43pdtS4A=s)4eNs9SRvHN2JnF8c}w76b^e}Qd=$rk?>N?U z?}i~CSaQ^zM=jJ%pv|qk<&fPiDp}Qx5l$z7XBY3Cglk*mmW^LlzD4EN3nSTTTqLir z?BfsxRm3)#G*t$EltRSVBM-`#Xu$)QQrJus*-Nly7(*n>9a$Za0aDg2ob!0gDAf*{ z$G*Xk&P&kC4*^mLw}XS`5O`oPXQ{g;+c%L)iyHu#F1Izyr1Rzl$wQr2Ysz+cn)ihR;?G(5?!jOK6q|?CD#R+swjA;=oE6Z+0N(Ev$1^I za1INWy4TKhy{5&it0k*MHxi$YS*s4W1^!8%E2_(^9*WdEI!7K4rp0@RB}Kac%iY8? zgbON zu2cFXhFN_eyw1lU>t8(T6HK*u6nHr7+>>iO8qCB~P8fn3rM=}XyM zFX9l*BzkkOpJ2cB7IQn}esuz-HTmY{PSu6Lp6Q5tYMiCr-D!~FhFEuUTT9%EKBf!I z*EW(#6iZ&o7h{rc(h^94S3V&*q)Ls096wpwvIrF={YJ5D#%WpBmTJ_)UB6?(&KW1- zm{8YvY*nOdHJM2^rqAN4ajUN%nHv@}VwE#P_C9S2m8B8mGamwud)mn|0d!SW{ot9y%LLTsKusf<8{?STL zELT#GEN$k5S@&_uN7ArfiK%sE3@-;tQGmOv>+8x0vWZL2g_FDCo~8^hpDg3lLpJ8$ z`T+Ag?`iQcB6-7xfv-f3JkIDWv15ON3GB%A3O7SW^AkVG2xV&Mdmw z)-`^ql`_)u$(lAa$^2lf&ekm(+jz6nj0$^hxM8AlA#zSB1?{FZ<){VDN6*J5L;DaB zQME9eJn<7HAsIy;{>qK!h&!iy)PKbeAf{(K{`U^o{6T6udw`_Pbc)CUqf&uaM?3`m0kVm#$@O;%MvM+^gn2q|<<(lpsECEd4!RT3|d!8sIBBmJ&w|@Rh}) zCeRo^N*b@br#b?CZLAmMFRDtIp~4u$NR~N@8fUvFD^csN;-xfDav#%3NjtO|FT0mB zi}THDW(ex2{tyba3{X7NI6Cma*_4_n=e&kDD=BTT1q7KjQ~aiD4o*%X>FL4()}?M1 zE_}ra@i4jVW{Wtsn#~@R(yfrr1{a`WFFGX9Ih~d7TR4xEk!=~gtlMmGyAi2Ibo^^z z`8-2AF2dNOt!yQ#7KtCVa8~DbBU7D^Bc0KOnmlOV`2*T&gBDco45TXsXx4xWbC^oR zu$dWFUM+Ih)r^ysW~f%5D(oT|$+3$WSv!U4_9rvEzG`f63mU_v((nMsIOvOh2BZLW zQfQ$pz}JhqtlrEDB^QG!Sj~fSkj-&8o&$&QMd1*UCGU5aGlw)Oc0Xl z35KF^uHa-EBZ%}qsX!9`7JOV&R0-sBGU>2Mx3lUrkk~7lDfNZnPiK5tWqN?vxcK~; zq^|u1Yy1~M!#jQQO#sRiOU5rl?WXE((9|UZ3E9;8}_};tH_@vj8u-zYFh{;wj*#948;qW)Ptb zB_3xS2xC_;X{^sH1r{{4HK4nt^%i%@NF|H6E0#~Z^OuCwdP{cP8LUSn)?`T1cF|yf zkYgd7w+Jk@0HVjC7aPO5#!=1caZ^g{-$i$w&6Xf5AD!u11g&mLPWCaY(Gs&Nxo9wh zhBdiz3=K>rcZr9}4{%6?=Cle(gIr?WM+k#b8Lmx@Az3y0Sxpib_FQ8l{#?r5xgpQ6 zd>_J@p09o5gybbOAyM$#Kj~)b^|bZBwjqP21E}ok*Oaxv?Pi|)eMPetxPzb(1u8zH z7HziKMStMqD8hJx`T!vm;NKQk<^7(0fPdEw`e681L(qhfGlneCj~`~=P1*lF@a=y> zx&QlxDLDR%3;ORjRia^|i@JjLWuTdEhNXys(NRcwKUz;|!x*pn3rOgKofno6-7X8Z z10N$TPBO)RM~#!e)uG-f>z8OJ8iYW`R$iGM+T!yw?Q%>Ub)#>E^% z6A&S`h4AHavgz{qy!p*?=Xn?pFyKTmWcj=npNt9L^jwXR*z{bCap+*8+H-;uNOTnH zk0W~*?N=asm+hY+>m=PvL)K6IW~mrRy)=btkaea)Fl_jALJ5CgRk%-p<`rd#)@m$!ys+cUK@ zAt}7t5&j6irw@n^xxT1|0s{~ab+AT&XC~jkJ$nG{2dyA;uZTgYC%kOfr9~4H>t8%%9pYS^Nx0M)JeoQLi^h$wv+^>m+E~{Oj(LF$id{QYMJ;uo z+=+{$$wnh0_h9IGl;O2ZwX5DteNTtG`DlI(MXzjR<0qm*2`?buY>A2yV^rdsh?8-U zSewQRk%rFfMR$wJAt{+0kB#hpqY+fgkv|Juxz-D6ugo^fOxf`bZ#;(?4@l6L9mmAm zH;Uv~C-c8Ed{IDguW(&~>43sbbxkyhDR-g3#o_Ft?bED-c)caz2OA+{hRA_DR7gvQ zng-c}HcKCSmntOGg3PY8b+2S&VSkC&Fe8%0tWb4KU>F!pari5wFq}z`UR<`@ShRL| zNVEhtcJOSj%y)!*7tYT-Jvmki&h`5)I_D?H;qzIEsuCB_L7gWMj}%U*7b@)&+eQp| zGICc8OR4OaH0LJQXnuE+=i7d@qb7z9dA3MIz&& zDu5ezCY_ve;AAEmg&5~eSw1-GI~1R3Sx493u8z}HG9gR!iq0_aMomI3*Sdhhnryjh zE~YwIq^fwP8WzEk*VurUQeS8AKm~7J%3Pq7##B7WwxF#jNG=DixA$%muBHiAoGxIy zkF7o_{|EADs3^nEC4*{l^0%omtJHK+yHJBy=2|tpay7yNcIKK5rIU(`rAdWZ$B3#< zn|Awtol{h~LWEWMU`Nwq52adV*BtsUnQLu_scaw#;{-!RYS(?dt z2f?7M&l|RSnV6XSx$(cj#!1m@$`fphNI6loGoz>87_ay+sH1a zndN$lggFXlXQ8y#k@?Nm4E@8jg>V_5`j?MPajkcFTPg3QX40+;?R#f!ed&p^`TIYO z#Vc+y-5F0k>8TFICkmSNn85-+AO%DeMiN9w^2t2GIUw{H`WN!9rp`0^GxO7hxdztr zZJ^cqE~!-Se+3sL8|mG$+GqA#=TdWtkpkzDRTUv%8RF6}E~Lqz(5*%yU-o;u79w{+ z8WRY&;*@@uF05g|y>DP-#B!F4y9WIX{uMrRGSf*9naM zQPG^KPb`;IXT%v(CMRu+<&v^b?*~t<%*ZxfLmk&lrCog^d=Gt|g+D(#sEgqL(5?eK z>5Kc+6n3r5ZgVVbUs&D9H9n*Jq8e>R@i-CubrL}Jg~jHN72y=e;+4--*iG^bY$X^3 zjmaGp@x(?@(a#&!{0q5~ct|n{rR+*^B0ScVoJYJsbqZgKb6KhofA(y7wMZozFSk>L z=n;RGnK}I#o3^{g8gd0pF;h_c1HU;lokK^puhBK4bE6}Bd_UH1hp-qxA$VcEox;7`>lBE_-gME#UT3Kfsrhk1%S^5#xG1oSMo*3zsFw@42 zh7){pa^46`y{G3I`|bZaFn<&jS(R)(3Yh*aEV4@3dW6pSOvoI~JwUb(*ffFVA_SV@n#>@CzepQr4cS=L?0OR647}YaA zfmbvUkid`|Z{)VqM$884>QFdXZfKqA?k7Kt(4M%`T(2g=Q$FK5Tp+7u;*0@U+1?)` z&bHTY5s5RWhGaG5hs|JOnwdO_@K^IUG`;T+0_o23Lc;W+wz07hzU=3?6^B@QJQ{o}^dXI#%bCp;&c8z)aA3=TU$n8RnV-*CK)z6eu)oK$^9yi=xeLR!Z zLKEEx63khqpcroH)uMxBUV@Y&hL)-Mzd;M&ZSv|Ntc9@$1TAVzaDq@%T+7! zX&$zb=TbZF?_*OH!b*R{%o9_~HFZll@B)_w>Tt5J-zR@l`B3N<0$l3?mZ~*yYlaco zc5>2!v=Y_M*5+w;vLfUTlw5v`-?s}I7zu^bPi`k3IqI$nprs2DbRcocC)|KSF^o+?Hnp#14_`TbHy|BZT zqtM3p9Ik}aXFID}1zWU^M85Wf5tgc<;gH@e(zTH1Z#|w-&gvxW9TMl6HKwAsTD=R{ zxqLf`SIxA`uR&*oWs(4MVP}~!Z=JU34UhY61uuyiwQEP2PEKnzwH=e2XRoWWtmPj1 zOhnW+UZG+mQw|}(KwN|nfOLY#AWfG1ADaIrl5>k#f z;(<2`Yo9UBtX`~u@H!xycc|6feSOwEvouS55y7;KyU~gk5P27}jlIJ$=`6jKIzIWaMsnTu??o%fm4^oZ!gK_Q>FR((1} z^1z%CThD*RYN#>AmiRZ08}A$2@ZZPk|6)t+YUJ$t?~U5O=fZzNmFJJc3O_hFI0(4D zJGi|&I6ghNI5;{!I6JtyIJgpAUe#c|=tuTor|ZXVV&P<`tIZUnI5>OOWalI@PrmTU zd$OuHxPRVcr)tmqX7OTS{9rL|{!dj!YbPmVC8+rE87dh#%5epGI{IahjN#Eg9Rg|0@hY=St%$wwj2!n(|3m1mY6<;sfbrJFsA} z(gXEDfW!$w{~}1%gLp{`I*}7jm6mkn1bLBY=7LB%2Zk6`?fKdF`0y5|A zwzBd)i_!nbdG6a>^@oGA1%r{Jk%^@lgO$4%gNcK)8H2OgKV5a|-}9R5|Ak$|sMski zC}Q$?;Ls#9V=^kD8hBo{ORcHuXA>bs&xIBsf%G|SsY{IMO0CeX^~{SBN*ceLe|uZx z>Kv0Z%$cpZ_{{iSpK($6`G5ZX3)f5R42_OvifW3+t9mQhml%A+FUmweA%_M$KrLJ! zY8GKE8x|}oK#X}NfgoDSW|6aO&d9qwN%^SpB>i`I88rujbt^8Qy}?7zs5|3rAT zmX{%_1=?3vZPuDCxfLYn8tN%w6e}Vd{k@b71QulGFVuxf) z(`PG=#S~Ge>~~P7?4?WV<0_Zq#+9oE5mcyBxne-+pj~T}YjY0o`B2AbIYjo7aaJk+h^s^$=VHg7^4=9skZ62DjxcA}d@Y4?3&^Y$<=|8_A>iX`*)^x29P(Z;ak%sa^clN)OV4Y>jQMr&iUk z3r8osxXM^hEunsRei@6`PFid4>Rg1YV`5capJ{BWL&ih$EP>6@|GVI0?g~0up1(fi z{M&opxv+#Xqhb97uz_?j<}5CoausXFMcb2U4BR2U)*|6=*<)#hT0~3VkggKzDIp`< zo-RnS^`0o~wGd|Qwtu01A|fp8{QF{q`NALV-Z8yAFv1|z{2X}JU)wQF?&B)Hl#F$Ghs#Wjp`W$rd zvWkvYM~kk@myhC)5=$y~>=tg6Q`b&<57OVBahyMm!@bx>85ZrAozIV!_i5K-r%_Ho znl8j9_H-+vPHt|)V0_OKxHtGl$T~ObrWkgp>ht$19&%MC)!S)Pf10V^zFD#0h&tBj zEUmYtIx5-F7PMG+*sZAGG+jCQGuvwhYEUFO=_R%4=+RoBh-b~S;7)J$dUc)i*yH)H zIcJMFSTJXuFRu&@daMk`%HbxD^Y+;~#n*BqPX6s^67?I-le_rLH6^D~_~1#T>M{4v z(BHthxc(!NMKoOibLVP%(g&GE3mGC!=&T!8WO@mIjx4~LQY|HIl_ z2SxU6U4jLLyG!A2iMvy{ySuw2E`__hJB7QuTjDN-yHmJR6b!%b^%p%I)9+1B#AL*c z%s=zqefB>0tiASHp>?ZUN4B&SlVpNk&6dfHM*f zKKW~)j~N>S+aq8+z)0N3T|MgV?*Sn{>uW1gfpjji-|CTaeak#v()qotJLAaoMuLLj zouj_8tOY?WP7mvEHWF1F1-PdaZwFaZ{ti%`v#yazS-h6v)NBoVn+_Vm7Jj3@nKy%%VV=}yvUwnFx&<(mfUqaP_E6>(`m&w8Elt^3+D4yY@~s3CDC;C z6`S3Jg~DNtVy}iyE)$z4mvbNUYV}pYFi%a)kI_SII)r+)@?;L;yz=aPhwV}Zo(_6Q&jWbspq_XZn9+bf2n=J`V?*fMi?=mhvzvnmSd!wD;X_vX& zyu0Vs$UjvxnXX$sMnTg{7P9OL(pS6S$pU4PIsDVQG=;o$DlIgsDTK@~wDTJZj{Hfo z=}W(*M{1Jfr^%<(lgy;gf2mk)RZlK4;+>)`@N*}nJ?|YTI>V+Q1oGbHL=xS*GN;y*VHRZgtxv6s0i`%|#CJeT? zCIcDZxWnms(<4LRl}DW5a1b5o4AjaVM*DbgPE}f5n?#+$fN(%|@c1H3LK4h!h9yEd zefNeB>H8jbCe2ZE#}#gO6exU=GT<>1%&odAL(NrxLhjGTMvNN6y()#-pbmXG+Tj5Vouh!x>(Lzgc)C!l zJy>f%4>&T#T;`rk8MQ+ioHjXL)U_pxD_sFnYThj{rwF{KQ?ea!ahgR7T+(Ru%8qWK z{IRVAG3)92{eQ(o6&j;@f}|>MiwN+YP+8)nEOJ3rLd5gdq7|-!$vec>fq*>>+JnTw z{XDVq|K*%W>|;*`7i1 z8D@BesF5gN;BTCY&t+leiG-Eaz>WVM*-7}8sa;k8tIMAsRWM5~n?IHg0xK&m2k`ql z=*&CRXL|mLpV~_d=HZIa&^VmM(yAx2`O4i@lJ7x4Ctwzjyl(j8ma~@6|IA1svpAB>y8Hm16<@%`&?a|j zj)u2ku%5s#SQi<%qp?Wjv6({3-}Z12K9_Y=RiXpMyDSFfwUedKe0~DVA=!BpBDCnNzR_845f!2h;id~eBza5Pzo83b6l~E5;$=g_8ibJjA3HXOvmKO zqf*I^51i_02TbdvqXp%k^g{9jV>kkl9V1xP*->+gq6-YP81qw?oG)=NkCYRoQ*x;? z%|>ae16Ij@bj8Icncz))`=~Wd&^m5qriXoH?_noga|LR%_%=3K_i9TXvE#`j0K?obIi&^DAE_%RsPn z0Ws<#u@;z!wsp9nOJN7g%Xy1a**5L$veVxM0OKzmX);s;xej!@DEEXTf%t!Dbbf2M zv7x!)iS4XC5`)50n4|avTD9QKdRXY}7fj6{r_WWa`2=clGp6zQ*gTGC7@!VGG!`}1 zPF3XdOChL~@Dh^Qq?(sFUv<5ER>d(dgdr*7e zYxLlnq3HZx+l1)129LZs3NR5@sT;2micOVz!POX14%ViRF4Q1sa@-ilKlGT~OSfjV ziqQ+eXX@z!&*G+c+x^p)^%1jB2xDN3{qBo}gio~jis=^HsnrkvtgD3N%(BjXxl2no zL3G*(fCgnE9+rrxBd@MPtv0Q7ZRZ(@IT=!zReJt<)Niq9WHVba z0NGzaYCAD=IUMtXsSxwG%n(dPvyZp}YKD&m#^HOuV3=0O$#m*+z6jqh4s+pWXelgrt z&5haSSe&s~8nd>gzR~5314)Az$p@I}R{9PWp4nP~9$v{k?HgO13czhXtGc&IN)@Nz~_;V=fy)-f=nT0>?dx;M-l{O#uBpCjFNLW+VH;+HQ06#ogp z`S+e`E>2cvZf1<)R<>sHMs{X$MnK?yJ5~Ho=tkZRio*>G=^wym%JTmS*@#0KwN7TM z_6}{RZVv?pE|T)cqlrV+?7YqYJW!3tCky%_oaLviZtVn179?qJ3?^F`8BzZ|(@HrL zl`P23SE<9-_`9)_c!jUIRuTq7t676iwVuPxWYSs!4ZaNMQu8@C1NJ2nP0Pc{n zA!MQ~0^q+87(`{vp4?A-rT_E#_vjVtzeQmF$IUC7{p-iSl#R%l|8=fSQI>I75JGt` zsXyOK?@=Z^RMIaX7wSyL!-!XeM^jc){EpvNog=q0ej`4?6P`EVHyR?NgsLhQ4lfAD z?3Q0vfQ`h<`saePaXb2GWJck~7keY-zTq!v;JQda3L*Z!C6r)lQOFT^R@{LRRZPMX zvTRZM0j=zCU=0&|dJGVZHF#Cr*+l2Dcr-_TxWmOd1(@~Sgje{{^i~^o(~kKK#h~W^`Bv zQRxNh_`2NTnN=*_(juK+=i)LAq~5!BIiu^M-S5HqkW!N0Ec6TJ@j zI$+fTNzfF9OF8+s#EpB&!tVnl@J0}v#ToWq)7lh8YqUkTiA1?I_yk3^!wfY7mD1mj z4G+Wh77hD4Ak!Acb77Luvxnp}Nb6&{|F1dnzb=!1 zEs|y)D?_}cIo8ewcZymQ9RW_t@37qXdGWyz7_sCnU}0QHZb;d|A}!0)Xj*8AzlVx9 zXe8Qc(nJOmz>lma5=J=!g2CndmGhw`kT0E5I-82|<$5yossXDFCU-u0ev>_%4qIKG ztas)|{4c9MalgU+MMBVUkyCD%a4jadBtY#fgbe$9xW^M7!O;@qC#jAB(Gp`GP0=Hx z9#zrsX1e72#<-PcRSEU#pb4`sVGx~Jmo%vIQ%(h>W7?$x(#QQE-9Kh|*8$aSFC|6{5lwY!)wy7XUo@n>@Wx{_=@mE%+JC7 ztq@ELU?cs#_)XibINqQg8F0-be;)l?(dYDjL7ca&ehq{klWPNTOA|wj_z~i~eiDQd zZbKSYO*lqokBl#H2%^w{AO{%3%&*_li@*fT>hiuwBT#Z1a=$HuQT3lAY?1k&B2cnW zy9$wYNquR=RW<9=ef}CGh}H)VrNXiUGieM|gu;itLhr-H-3dj8F(e5J!fj(Q1e?+G zZ%2T`ku?$;8tFTRm`wnS+7*OAVa@;6GwQ!z&-FF6?-zukIa5GDQvV`C*~l2~EJUxy z^*7zY9a$(NC_G$SV<1V;6oPM&A+o(95I!iYUZ+2A&kM4Zr43WTV#f#w74#)Y2D1y% zuWILF(ig9V=9!gGzn|Bmp!1p!^2%KOX~z|AE6N753)!!1=kH3z&dd!PR7YTMA3DYx z1%uKxn62vdcU$#q65X1eL63^S^&4_XcSFg3>Kc{083=6Zl{iE=q@7hbfdOZC77vUp z4DVr+vob z6YD?rxY@q#yYq+*WCWgz-P1ku+W#fgpnHbiS|;-BWi(0)WORPQgsNqEMh_A+=?*m5 zVS)VUAAd>g7p&Lo_p2{5Fv%f3FpeTU*d2~k^cN7LChtaB$%heJDS$!0VG*MS<&)wE z|DqcpN7G1;wxAj4$5Dd7cB37@Si~@*eHTJSBAE-e^7u(AJ5n6Uggzts1)_<}bmc;S zs#MIyuM^&=W0P>62V8H%2Jv~_+skI?5FTPcd-M=i?4CoxW=PJ(%Iyca&^F-5xmUI8 z;P_6YPz=>if|z)#xEc882w{#_qg=MJOJtNr^|+a(l0R%|r4xmJ_~;j{;e##tvX*0m zU8-fAJ-_-+sRe%ecB^zD{AgR>mcAGtmxc(SY0$MvW3K|TFXuKj)YP{eVL*&ErRO}oh2%w!X|s4R@BSvDPhSy@rI z=|LKRf_OlV$Y#|wuQ~uJe4&<`ABLb??Cz@33?;0srl;ZV+DA0U!L0NGBU&x)t|Tji zNYbtMg#NOjG=|pd||!c{+0-K)>)L41N(z~9huuL!e3Jl&K#a~Eu46#JaNKfnHCP$C_5nbg~I z&^AdeH+~vA!ft==FQ_YI@EFo3$21a{vKw8H?!oQ|$-p@qquogu8{82we7_4oASy5s zz8w519$h>dr-2JZ6!smfYA(-qLE0awuJI(Bg0t!uqDE^qmNP8e#=iEzL%avaDr7iB zrIiO|?a^P_+OTpc>o%As&t)ebIFDx$hLT~bBK0E{NOIZANG%LK@61unvdD-ROIj73 z3Dy1Zq@E?5A28CFFlp5syd_9U)Ur*Sk%%OHpc>2Y7cax*@p*gPrJth4wa-hYq!awI zi$Zr;N?NK%lmIR$4qK9n@N{Ct`~x&9I!0CCEgO9E7a%VsV2xlE&HgRWw&7Z_SW{Nr zF&jUY=u-q!BCo~Rhq^9dKcOCkULHPz3k%8u1GO2C%cgvK$(w8rofy6JAtmH@Ca?6z zn-se(*K4y3kPzD0B9<0@D!@8trEm>~8%Q+~BNigA)2J2WQx)?PmsLED`_awJvlFgF%$%>{`kh6W*%ueo zzL!3_&jb76H<8r9vdVU;rc$S+tkG(kJrKVx@y?h^_|d>#o<_iLfy^i^PZTi~Uu^Ri zL%q2~(@>Va78gC*-!SF9YY1c{gHUcXmFXJqu57Byx|}sQE1ljoz`HkIRZ=Ifx(O;S zW?ig6e7a{&SrZ>_H&5OW>?WVv8l#Xl2Gi0@9AE_LksH!%mQS1}(q7wb?z)9=p7ayf z|IFTZ>qN8Pv!|34lzuu-FgcOeA-J}7K{t?A*I!@JgDVONMGdR3JjR)lM@A11$M3CgtPe zxTv)Q<#Br>h77&42qwqtVE3D5Ysr#up##myNg_;zzy0zKw%R~=UcY~or^O#dj|uxiN3-~1VhSlTHsCGQ zqIm<4uC-`;dk%q{!^2)eX@Y;=_1O;!ag0dKn&Bxq;S;is+@FUTNxFSmzeEmRsnp{A zl;KjB%n`)DdDf8<4)KzCailxK7Ek+aRT}DU8$C%-SMK?>sXM4_z0A{_ z{o9}BBZmt@U4Glhyw^F(OWG+C;lU=VFsOs|0!O}mR z*Y!(-a+~k!PUQTLDQw$>a8^eZs^ePB0C%Xby@yXgx{J_DUp7D4%^M-n&t}Y6-ty#yu%gSED~lfcL1Vt0%}Mn>}ES+YWD7eU%XH3 zLO)6u!0e9wDU@_$aE$nB2hp1dmrJH{oGsUP>FpXL6eK*b2%H6nr{ZrmM-8&JfmNRG z{xa(r_ovlfLmu$RoolX<`5=?#l=N}E93%Pg118Wu^YQRlphL=&XQjC5udMl$(AXoe;H;z2uBZ{I(ExN)Vz zLx%s*>$yQ#(>8RLi{R%JHj_M~ZC{pF)Q0F1>g|0|#{il9U8Q@Xp<6>a$ofNd@aEQV z9L!*g#uPWJINHJ+UE#YU6Gp!3>h7(Z(=E;$NmQf>oOi!G1Aa{2D;M+#-!EC01~Hi_ zU{{;+fE0}$(n!uA*0Nvz%}8Ad*~Thb>`8+BbZ2`|PVhkv0Y~Fi)1VF#|L;+5=c8Lw zR%i)u^>>*lh-g!D@3h1^^Zep6`Thz8+$IRLE|-2z5a*l6ufT z9HE3rZdc6>WTuHR|>vS3LzfA(B?X@NX|e(MuOSK zT?H3U`2hDD8mP0+TezvxBz7yPE%_DO(P^!UrnK-wSXFf}T)uVGK$}{qGZt|t)G1v* z40d-ba){6W0d%+wB-@8AeXWBgfy0}e^F2w=6KAbu3iFMbMb6qlGpdmt7%etC^Z`b! z=Mq;>7}GbxJo>^UVx)Q}iQ{+0;nc2ueR*4jjmn%*<*k1=!sw8hP>smMZYSt!p?whf z_2oBaUrxdJkM9Heqs&kT&;AYQM+>w@?j=j`4 zuJ>jaY)S7W;weO7KA4K5*D-?AL*W!bcByQ){@$+Riro&s^pl$HNhqp9eD~Xv=rMT; z#9T|C)U-&nI6WqDSa>segHxmkDMp0}xeai^rj`WI zIyKH1ON9#kNl@b$i|Jk7y$~h*IZ%1#F6D)Ofnok7_Ui&(;_Ouc%yw(vFt;Gehp&u|_~}_ z9w^*1xlVtE_>GnqKsB`8pTxn*XTpn=+e{yHY%6!&MgN6yrDI2I-SryH>e6go?i!9F zQ=qGF63GMR5HX`^MFOj*%X5`*v~tC)6Ym-3 z_~%i%(sk#8+rX_kdo{A}k^EIyUIoT`YX^nRvTQCzz3W}l(RQJ<_>eR@^o|vt*sK8w zfR0L{-Pe^-c57;4+S=Snfhc?VxBULmsc28<9@uy+`mXp2cw_X{RH}0ynsoFRzYsQ$ z81XHC+-pjFM2G&HYYw&n-UHgJfz@k5?(siBL~kKyK{5s+w6HisI&Xxh0jg^hZ#do^ zc;yv0KAP0H(px{mxnD-P4W%fA;iY>nCI<#PJ0CicbAD22w|)X!-()Z!ipC9QA89AU z5tC?*jXH+x=#FEmasB^r_x<*3yzfiyWuuu$qCt*R;pnszcZs8tb1d6^-Id?FzBlBr z$7z7|mD9gL#OQAY!gxcgT^pn${;{+bTwEZ`LNy|QbBPr_5ys2OU~9{rUYw#Hq_?CB z*$g?(fRp46ZSh0#e=gT@k5{YRgwMe2x{dn@9JF?0bl0}hf~0!Ak1U}0!ac*6$aj!h z9lB}FRV5CRx6B98+lytVW#pBYT&;y}m!H05G;X3Vvgs!T{Wc+OYovy=5HnhrPv4wW zdRB*9_CWz!RL&EY8BShY;ydQDZkHeZUV|F8=8zj5bDvjgn`)Xcr+U2flbGUQ%Z)!` z{3d&6Z(+=kpcXN2z%4e%rgr)-MY22O)c1 zKi^23~VTKzZaz>B%5yt&m*S zFjkk`47ES`u1`l@f4lz>HJp%_>}!KECaQmX92H%9Ml*zCUx^H9r2%4xXF*9?b^RzL zjX$({H2TI@#?BaV{K1eoGAO?j;RTv?CP|a2#X^hmL4AWU7v?elPH8Seax_3b8FZSC zd{>WywMSky7*v2-+NV7(p`Q%}mM!mSM)?L~(SN83rIT<=q29-@3A+6C(?4o9N zoZeV-gB=XXeEsE~F>8jdfGP2zl_7k#B|3}W;0|p<2Cp#+uoN)AU9%P7DqzSDViiD^ z!k*X5aUrjVOgMJeLAvhR^ND!iaKikUV|)V>8Tf&NDlj6CS|^k3k0*w0j}}+>uFNOF zhs}spJ0LSu?NPoGrV*~No3eN3k@8HdPv0YFLbT($C*i^@+BMdOzx6EmPfMZ`5`G@# z(*jK7bFlBbYb=Z`b8LFGe1o=PMk`y|J*><<^Wtr9{~8%j3|uw?>FMUJChN`>A$oWjnybR=(o= zIjOn%!e0aP+xa)~U&wzRF)^!F{o7|-!}UK>Pg(v?8z2A6Xl0J7oeG{A20tFSr;(JT zbom;#K^}AKwfWx=t z?-R(LQ;(V`{tyrFSL91xKs5yi%4c%7RnE&Qnk7(Z`FVB8rbpB zUQx5C>KtNMb!cgWh9wognwrffj@#IjC;LHEue|(zG1$e+ve&siUJGAajFk;Rx5aiM zYY8RCriKpj)8U}5%;(BP9BhU2>@WKA5XlkbiBz&z$1=H zJ5Q9;uKFWhcXFYdya4IoCtfuSq8~SSyloEAQ@38(mx^ zJg*ncZg1C1QDUeKgEb4p{#cY|YUPDN9Mnx)8*jhT%%y!r|CK>Y*Q(`ES?sD^zwxuZ zoOV5?!_2{{7Hp#0c3o}R?P+Wc^xc|lrDR+Ei6Zy)saAV1W6+!B8z|2!GOFT|7ss0L zH_Ptt)^taa(DO^=Up@ms*QdUspEZK1L#dV0v1*OBvUgiiw@`AVq?~vfsO`VrJeb3~ z9^Xc7rs6_~4PbMz12qwDyAW{~W)I%UAj+{L`Ra9Mo3e8jm4~ z6BmCk z)frJjM^eH19XP*ieYmy zfW;{5{p0!0zmUpc*Ghj_8BPR4n~J41m!8jFR={jcOX-j1lXV$uHoCTI+sgi$<;|QB zZ@q_nErEOG0^eMR+?Cz#e6-Q{wAJnR{{Duz!*Gd5jaP=F$qpRp%LJnfA*W=QwMv%@ zG9ry>3x%L$2`|Vuq6}h|&1wteM(1R5Fvf+awBPTL>PeHUDzw}q#KPGtPD%6e9#N!Q zLiSA4QP!F%rmJv05=-4kT86}?6szrzr@t+yRBOd_c-#_x}>ok(4|^tOiai{sQaHXm$~C)Tr( zX=gZ;TB@)ML2vNv=zVxp zje-8E2NLkHXFbSHFq$RLBwew8rfRefGe;O^eT$&^JJmSFDl5SbQpA~O^!Vei%{y`d zCp{C}G@}^tUcM_@H8(d)+=XQHlfGBkadp~ns?{nYvB@w0A?j4`Cw0kFmEU~D9JlyY zQ&6zR>0@NFcCW60c*h%j#M1~Q6Cs1QduRtcOBdFfD_;H0K?;xGu$<1u%_DC+#+ zy*~E{`HL)u5C1ul*xhw2d(B;*Qw`FGa>>76=eLK71AK$dVi&ke|I+cBnyi!pyBgL+ z1Y-Xh6}R_XlCXG)8dN4uR7<8ULh5pi9m2NrQMi>0E)4tqvk-Mkn?qBLV&2 zIfOc!89A9)GOGO7*ZNIJilGDzS_Z7q0P_?jpYpe~Vo0YuGiZ(BcYPu8hjI#LjBcX@9J+OzyezE4V zbi&}DZ2J~|VEztf{*Iv^=NZp*z%#J=U!L||E0n7L44I(*$DRJ)Nhmn~4|xklDL2pm zeuMgNenX5Bzs#T_2LDV^1ZYj$?k^5LL_$-b0##UKlp=~XBps4{?I~+dA|-_ntP8fE z95yy2w4>9tL4Uf^^L>6U0hS z+W{l$&Lwk&qCjZUdrzGa+WT{aP@QgzvNl#f&YawPm+sd-!<w+?Eo0c3q_1*Z24_0l(SfQU`nj3RDH)bF zrQgoXC|gMfv;>ZfkrYJokT20(B6dnA{WOEC13TwsSrE4d=6ho{5QLcj#zWlCT)#H53??K3KC({H2-4GbfN^N^vUg*!tCJ>O(i-OT7*cT;%n zxa4Qjw4haf&F=4t=1nxZu;#Lss*d308plB>E{QyldtFQ&%drfDaME_nd~PUc0HO6T z{>daElJrRkAHAB-^kjzOlm};c9k`ON2_?H6i8u%<$H+@CE2D?o6g1+}!|^6?pI3aJ z_&vp}QZ4*mCQOgMl}>)UMO=&PbdN0KGHQ1gsd^!QBvkW=be;m|#U@6F_icvSgdl|G z-rq@_fzY?#JLqprHa7>zIU76jFhcee`JVza0v1Zn^%;?u|No-G`rlEJlL1%4;IGBC zux6245Wu1j44=!#Zn89LLJbzOF>11W!p~bsauf$~2KrIHg4wTOG=;0vzbefUprUCo zZ2k2*-WcqDI)2(m@8d?YO}0(8`@Oc(Sbybjv>6Z)18Aws7)HVmP1Lb^9M-%R3KnoU z*9`HE=?BswBbd4GUezq5<<_VX;#SdN()vspE*I9AuJqt4tSQ^<#ZD_jiS(2UduAc< zu=HT(_zTJNN*gP1sAg&o+#M*gV)&GwE3LpMm+FvW^hB3$H_!&*4GUn4?D9+1fY2|+ z>JJLrDGg>VK6#-0J%(+%xEqljQp6u-v8m}1>R?ypLKEqn=CQGcv-7wtF7fPMO>W~( z7zER2LM5-C!2$o4S(SnnV}YS4>W0Kml9%XQYQVH@=tpKEA9IR!9Aaxkne;FHo@3_C zSi#T0fd5Ax&i|)c#`ZshGbn`OV>2&mVprYbi_iB@%KQr2DcBREhEeN}gmzeO1JN_n zoKntyEXtr_@P$($zf(IZ!>8tYt-P!~9bHU&e{}hM;TmZ9Oqa|UM7%7H7JMcE4^zGB z5L{}MX4s&jYqz?F>3=1R&S%1W`zK+T7dOf~&*e2a1s~cFg5Z(?R6tztokGv!fEZFgekur`b?M~nFSbkQ$s%snR{r4HRi1Zrgg^k(aIb+!Z$9F zfYf@SFmZXUh@Ux*1k{V$hB4In3Q5hkKbeIgZK&n*{~S0wo$jC?(fww`m@Ht<-tCI7 zjy47*>?3|~eg@_JKV}U7zd1_&Z#S5Kjcot-e)F#gH*#%xJBBDjniF~f(Shl`S85Sdh z(yq9l)@Q6{+WC7wygy(KzapC*m~A8|#XD+%y1AD3G~-0#OvnNt%%vC-CKGhkQAFgw zdsDNLYp4SDDk9AQWo z)Ntd-7Fko3ChwrOy^*Q!(Gg2JL+k~=hRXroQbV5AaZN32BGPmZ`Ia{P~t`d$%$>gJDW0aDVh0)K_xe28!_@RWP&cJiF&aZAR~ zb`e%*;+*9pr?@n5p-4(BpZa#C*<4$>$l5gXU6%D#A%3k40u7=TOjGQALp< zwBf{5zTf8vH7l~=iVDKGd`kNSok$e4LJ7K_{ug;~Iz2Ru{j*?n{l`~EhW`)i^Ir?H z+Q;1xedPA~_vM&-Dsr6|yBZjD^noxHbFeuvP8~@)S{R!Vd?B$+EcsXYtM0D*_$?`Y z4@RLr4o5U%$oU^v-A}K|FXlC6+47+2t3N@j-J2P!pP!6XyCPmPXzFjH8%$Y4{bURWxag zO3_!0Q$?`~r6Mwnu~NzJV%Ca9@uVS`2q=nYu(H-53pwD*b5+x9}a3*A&JUqSfPjtGrg$; z5GD*4HmuM`@i%zLV#Q7D!D{i*M3tJ}Gy#0$hvOQarLY7i_OP|yXrn&H_BgfP7@`DC zZ%P60iNo6sD|9l%GP0>{09dU=(=iemtp@UmL{gd6hJ)l{d@W_O@+^RBS}|cmGM$WP z!$D$kZo{HXs@H@vk=7ZV%wJO)gVfiF`7c`EsAYmot?B@X2^6h>3o02AGplp}=IH!; z1DkHD_^5ILASRh24&Z54E}0r>RxX+`tJ8q(65Fu@ze4ed>-z&CKzxlF z5C`imzheWwjq;q@=MK?Bb^SGf2=-IS%>uj^I{!XpDmGMo^dA9zQ4sp1 z*DL`VDA!{0nu9%}eSgq#c7mIAxe78E-3A~Mq6pnD{JPS6UFU*iUJ!?ei?k;IGS;{{wo zc%h6^#7p#vTGhX_TbCT0rk!vSZ7og`(-5`qk9he$=`O71g)h;&?$E2<@5GgpruCM=VLNDuDI5U2ZlG1?vU*D*d){2t?)MeSNb`CN zBHu-s;|;_N`2+A^ZHcdAX`$|}RCkZk%8jYFD94yx0=! zpf1L`6uVzqzUSeaMk~LgfbOA=Ni~0L73|GH-v>$l9P&n~raR7PYHf zIx2igd8P-c0!(PFua(it3@0vV5zeYDMrNl9H)pH19Z2 zAU%S90qYI_H8*EUXqXn<=Rh!(>7L8A5=+3wP7UJg&pB}e z+muckG;+Q%0vHluMt5p_M~y10W#&L5gB-yD!?w;jA}bit!T{hXf##__5@-9bS7By% z!0*SU<<~+bciWU+u&^k2QCYFwQ8>hDF-VNqY`RwDLKpnP>H51 zVDE|DP@!iP{XXPOWl;GMnuv5uIK`8ELnb00$yl z*3b8~5=r7^-3}%JgWs1HZk)s5{qwg#rf`!pJs;%`T4P_`BBqE_>C<$g@WgmenwXRx z5Ze(vnUSh1U?;AmK;utD2=@9w!mF|Y*{YJ2p`5do9oHVC-hl)!Y}E~81Ozs&jQ`3$ zi=mBpI9trF<(ES{hSDqP%~zi9p>J7|U|0?HxT7wOfEbW>eF=39A|`Cb4Q`n%9fO#& z?pYLsH(lWSEs**MKA5e8=F}yozvNYjTHE>(H1AZ%!rDVu8DnS4XrCQWPv%(vAT7M1 zlu>@4qs?PLmk33B=1;EM+2JA(NPeF(A? ziu3u$J|#p_3Iy}zm`AWH;H|^keA2A1nV`DiB^X-Q%}mPpgCJ1Z8YE>Knz_=|m;!Ih)au$J_JqqFUxLS5ZJ8oh;6$w8DeN-n0)vB?bk*5Z&@+!SQD ziQ|gMDEp+=a#?9f6rTF>&>~`xSwMh&`6cU7UYet+4n6W)0 zoF9__nT`EXc*07hDEpaoThLxZy@_4C@{>eW=N_*S&&=ysr$k&^Avs2i?2Me1G62>; z-MD(YNn_~pr)cPS+nQULP>3C^bb2RbX-NlfC9WY+-7`50m{{h@pC|N!BI3>U*rj=d ztnuZK*v9_1u+C-uWY3&|-+1`_aVBUp~*!?le1n6R}J{3T_KbD80EB`VV{ zO{0x_^u@aT&2o5xy|V(xS9QoD7xxDO(l?zaIpH7e9}h zn-IM?A5qTL+-WWWh%;%rW^J*Yzm(crkhe7>y@fnen;f8PQ&;IqZHi5Z_~^-cPC0uu3RcSGqu4RtsA84)bmkP()~(4rZ3t6S?uN}10z z;z?SQ2w~==XDS})f7{K4tiui$NieNr>+68+F}ZeYIMHlMQ~Ale!m{SD>HU;P)=5J*n{03Ud4ih*=TTnv6%XY&u=?mCyA|Ze_G|8A z2hRStV26L!n;&m_3G7+~vF`lR*V0nDu!(4NmYmHh*PgYNz)@1aZa+PfwQ?uj($R%c z&{Y&*5Gba{TW6kJv#(pwG{G6wzQn#1WuZo>#@_9u{xERb)SI((!esro$zg8EFmYqy z5odjH?-nQe?6sLxuq(Vj0aOVwO}SHBXc6hPnZDU$I7?s`3wlGm1bM7!i_c(BT{iE| zhNu4s*^Uv02sFL4H&Fe7sAZDw*EAH-39grUy|h%&2U7|ibaX_l5l){`tzSybNn`+w zJqakISel)SFfys2KQ9`Cin1QY61l?3I1yPt9HLj;4%TdAc>RLKz6%Solzmc0aB33R z1b0zV{*`V{gz!`6iG3ad*X6DNOXH^<_Ck#@aqYF(U)Havh3XRQt|6BNX|I8D82W5^ zVMW?P89lL&J~p>FP!25_XbEyzE;8S+!z^CI?G5cHi=i#GUM+r$vzuEv=s_a9 zo?fXe0Rz1%~&p@LAwbvj{1uDJ}$(BLwxhO>1CcVzgibt?ME<$H7qJ+9NG?*K9R#C>-p7{E>MhsbRgU5Vr`D4D_rhIQ~{G^Ya}t;Y){rY%rP}s$Cj)J(vlL*%wiN)&hx9 z2#fNX@`8lqP_Y5w#m9qsul z^6kKQTk0_%b+8{4ukqwPlz-8t)alNW7E_ERFGa(KQ_a)u*;u7@^D&Z>lUb&V-NA>O zpy{z_R%-aYPpC5~6b%axmmATyHF@CK1L zL8^u3047|tSJ8XwwwH3k&rV&>e!UL%*fU3y%^gmSP9g=8_(SnjkyQ5#1KqD`PBW2X z@^7RaXv&-onTu$0`6~KF=ij$`hB(dSMmtn)U405wZMbCg2d-)!LWGUzbp~9K#OAcvPBO4^Ur*|&Yj4FW zfxnF5+G(+D0Ce;ZA;uJ506;-eY(u+8l@)`PtoGcR(AO4;ugL)<5}V!a=Rv!e_GB~+ z&SdaOcuus;w*zQ}y$6P(@^TrQHq04h2^yhU!mb`$1ZKR_qrJ-nB2y`+f&o&Bvqf;q zQq)YHQ+azRs6P2`%EV19RhVGeejJ#GJ(O^B++Z$d59Zd(WGM6sQzp6rCzEVpjL<(D zzkM5POY#3C-Y?QluXPV-VB5Nl6!MI(U}zJ-)gl)EEg6+FDa`#mgmb>m57Gc!A;n36n?BnVIpJnVIdFnH@u}|M$Hst+d*A?~bO` z)9Oz>EmhA{J-@2QP1>6*4Gd`~rGuq(hsby`v->us2tb%^BXnR`UI|&b@93n8IEJWo z`$UlBl~SpdYU<;{mvU(E1Not~vdOvOBlP4q(rxS67%;h&`in66+Ft&)#6?8%HzD%1 z_`$HqKqUgT`v}F2s`L|bOa#QKtno`=N~9aZ`tF(^{d1p&ekskE1v zAzNbMQeN0nxWSD66@pyM0w?>YW#gO;jih0Imu)PX*6n0_7ZE>PSr^G}d3r_v;>j z?wa*x!9ysg7_E@>XNiXaOj4LKZS)jXW+RFli)+K*jJDUq-Y(s#@=x48b9mCYr$YsN ztIi_oFJcAvhzGq!YbT8zmkMYw$g?N}VIBr=>ERG&665HP2*LZiDiPdb3PY`8%M92K zIOr6+3b2K#yJF&2f$1@&hZo6&;5b?>LxLj8yjlnCS^A(1{^*c^ylmm(GbhbCzy(Y0 z%$!8Y-ylx;HfX97l*K~`1Lu@G+>I{RmX@%)aeTkpxS8;anev&4=YVFoyMYO}#aFHX zXV4sf72)Edw2O=sj}K3EjZVqs5OdGz9(#MYvjsXxH%(rlL8vud(#x=t4-o|0BUY$h zsnR@O>qTI?b3$j?cU(;ra{_<6^4X0W@HdYnVJ;hWd?V;KX2RY!O%DeR?Q~-rmVGpM zy$`6)5tm0{U9X|e1iP_ARB&t5QB$3eA=H}j<;0BG7i1SMpIG^zhmU={ydnjby^<`| z*FBroKv211)=c)xO$FbV%%XM0^V9!5rPTt36zPM4I zXi5$XFiFOwW_i$0Rx)L@%uOuKmU@`KAEQaOZp%p9@G4G~WUk-7C0oor)}+7ULy$^f zfta$r#?=j(q)oiiFxvx0bNhJC(Ur~%2xOsFa1x^1dd(uz%# zR5`0^<4u&B-qBvV&5d^$Z-YVEipF^x4djR)7MQHy+s)M-rRmp!T)VwNG;P~mgtLwlvir#*X-1Kl;T;upN z|EOKepBrsi=q^?r^rxs5LU?DjN6JWiFvkM|8~evLt>rS@eY*hIyOTqpQM*&;t#)G( zDctp~SCnkYtqi!; zEi-927CqZ<;XQ34HrqP4djcu*ThC`3!GQ8ToBP+c_F8?vK&I^V&2EJT9^89T^)>`) z@#MCur^%a~a-fJ}`(}K;|NgN$#Ox3epSW{G@f102d6B8fl`L(3-7lF}VzKD~@pzj0 zCbf4dR$9vq^|88I*FVF#UUa-x1Zl;LTs-)F4CI&`cE1QdCn(F3?w7D51k@xo7N3ui z)fTY)+Q(Qcp!O}#C?fSw?(!Zdt!_=!W9`_)9*)Wq%`;|YiF0mD=FuNInqGAinQxRs z2JpfkS%le&%Gkc!l<^dfux{(^xyiNeW>l)hSMfO>UtzJP=if&}$V?N3HR<3)ujqNs z){dnIO{S?%_KjH`w{SQYq!lenWLVAFD2Pd_1ABg)bPu7HdKPlbDD95M+O{r>st-u1 z2Z8J6$fMYZ&S6r1eZ#kBenY4ZwwXa1>^JydHo4*LbCl$dBgY`F9;c>lNRaFfgiKWVk@UJTD36`ilK+ zKS-Ds`KZV00_ODM9)`BjFzkY(wMdEMGciK@*U%EaDmr9pc|oZwj%NzR_c?HL zpVv3Ndx`ZO=JU2MUKF;0F=!Igd4TT7FfdfCX(D-)l_R-PDity{)-a%%zwa2CLw% zs!Cf(6?&@>ibRTS7YzhF69a>)U_=Y!6zkvdEFodFR8W132!h=O!Gid&fH0et2)1|L zYFry!a1(KTN*l=7j%OpxTCUPEq@$L%Oy;wzb7VoZCr~3X)g)Yw^e#)9ylySVuoCs? z7N#vZGPn5i=K9Q-pS?CV##ilsQCl~ptXymzjP=<2S>1rWJD5N%IjiQt{0ds#*A)+q zKC%WzbL~q!(74r^@c@mZxw&4Mje>P;w;9)GrK!rgeVDz8z_!Xkwg|+oCENzelgu{; z07G~bLid}HRkoquxcvE>d#9_cDKLAE2jw-a{YxHyAuk7TaI_rixm!Y-7)m3II}*(G z*Xx!_w8=akj8_IDuES}^{+d@8#OuILsf6Or*%~u5Y(lIsSHL)0A4=C+ZS%ksDxUPi z5loJm#Z8?zKO5+fR;@Sf4MfUX>SKRNxgvAgH$q z(+X7`vc4;~^5v;tcLv%G40-*-9rn_nuNEh}f9dE`X%10!%?};^>A-%2=)ewCPXk=l z5?(XlL0;7b?)o%;;Wv@~)NMlkXeii4a|L)0$pzavQ0UjP?n=9|F4+avtPotYdttun zgRWMsu1Q~Eefw752lxYXR+p~@9$DXpK0w=T0KaRU=ZL4ko)?sradD=dqR6{ZnQP^= zK>aZjU)u3QZB3-@L6r}{*_6H+-?KJdir%c&ZuFe$W9_~RbAGv#V%7Ar5Hgf{O8&3Pa%3j+* zFf}kpSB|b+ySx5ifB*Zz-kk?ueIIve?6IV)jPVgg%le)f@jZ>88M)jGD(L}&3_#N9pmE@t^4?z)a>t$PS2o0m|J zd+E7+1^86Zup$=^;g@_f?%#T%dvLr`#&FbEB+*CoDRJVQxr=iSncaw%h6-%kFk+3i z^gg-JB*|>=QR#HrK0n%Io0V5N5~?R+PfYOlbOR=BNKtG0#CjJXea%fsq++%Re)`br z;og0p=uNuXXOV6v`KO(uS;A%DeQQTvi+D|&n6Zh4p}6gr!TJ936b~Jkz3>7wvHA95 z=mH;7C4z|7g$QPVm-iuxv^}iJ`D07%*J^GDG@<;lpljn_gQtIHLUg~ruO=}C=|Tmc zTT&S>_p7)cbqrYZryAEzg@Y8bImEQ-T)4rd@Qqu1&lTpVtZ=4XSNxp?!%+qfT{zWlHk4~{nV_>^;LI^n*eh>?a9(u!Sk?8Gtv#k;-B*y&%@X0U zcF>=c=c|j0bFpZi>u#HCOqZgY*f4PfZEIH@psUGA0fW?8taEsIBhqdCaYv7pVW2Wk z(AJGcC!|33=2W$zJ>ozDUp1-WtIL7IYlkB$kDOI^a-Vn2uI3|a8MbE1q3Pb9^~!5< z@KMV8D*exC!cpG37d24y-7VQrS0`w^-HzuKIS|oHz|Hgw!cWHdQxLgEbD#YCJ9`eT z{kLfl`vx4=9e!&bI_3y+oUmMTSU^$);a7}2>VUk{Qy50!^Xlb@$r z1BAy(2qLxPm7jXq&E{TM2YyqlcT{+hVS6NeV-B>*4K~+%WDH{y!0u#C(CdB|rt*jF3x1&I6KF&h&du z1dU`ya;{bh!#5L^Kh&iu%Cqru;+yC6i3@8?!XRN+;th07>9Sbu zeNZsDEP8hl;2$Z%#Y>96us!r`PqncO0m_=kYNts*eOS#t!6+|X;%f{~2b-z9hrM~X zM(M-4T}L=D-|lUBHrBa%3@ljTBVUKU%Jc);6aL(46NPNUx4m(Y31DDkbSH0XEIUHx zAKq~S79Pa*8V5RH+MB?2?A2px$sOQ*8`JSok5Bs1bZ0STu>+yGpt_UC&4#3%jn}^4d>j_0`qdAj^Gy;PoD*>JW|2Ca;~)H{ zS$a6#XY|3Fz7gpzWA}?#lGy$F&OPee(EcVgV_u})If!K0S7mj|mJ(*gi4TxFt!sbj zDI08e9^_Q#UKSkHa9Plz=zo<9u^rs~#)tj~GIdCxFOT*G)A%?rstH@@XW8{7D7!Il zYsj!#aM(ZM78CBanS6FXCos_OYu9rM@LC^i@PUZyBba$YV$5$GyG8f&e5Rg3n^Kk(~gsyviTm}#fG45}9Fv^j@}&c*3Y=O8L~Wp_5g!#j0s>c5g)ajIImk4rr>P^!TGR*wy6GlB(t zw@oDj2^LZPPBQv-YT&8^qY6UeH6glEdX)7}z+hUMGg1w}ntl|WQAY<$CT~W zD8M!1tC}o(8DL%_WO`0|-wx=dz_9<$fSmSN6QMqHL$nS4*HlY7;h}bX=S}UA&8T_A zjeEom-5yc5gSu~Djb@OT3X72sIMV#ic}NN{=AYTW^=t}r@lxX4Gvr68ustMA=pl#G0R>ux+)*TklF7L!ia#Jmw+LTkQ6VyS32a}LW zo@!Mnt#iELwvPiUh$2f53-_;S1{H170ILJ6rP=p`P|Jl0RGqY+JYu47Vxno-1fa-d z(gJYYY#i5eu){L%fYJ15Qr|6o?hG&o$AkTDC@{@;=G zg*|ok+ZZN?1!enhmSIABBY-#Fbfohevg%ygox|Jrwq{MhYu75427|CZ5d({-QOUw z#Iv)M$8=rTa848O^2 zk(g$nbCBST`m|n3yb=E(-UQOf`2L;}70t;ya__ zI|G$EBmPD2{=1NfCnp;XDJJ_@Cv^)abxSRE%Pln*ArLDWkIX80%Pw_`D0M^sfp8;> znZnxZQpyn43FS*J$j=P5YKd`xg?^xZOyMgmMrr(5#)=Chi$V zV@J@+ICl?zV*&SyxO)xx!|e1zI_wBi`RIRmQc&cOybMLkgvUw3u~3Uj8oj{>j(*#> zwPKpJf^q6{L?F=7YH6Ug;AmP}jT998d)ERWEWr&>Ney@lC9w`i2tX9b-5%`q6Zjj_ z(AP75_~^DV!7AdNASai_(O%V zrG=WoNR`TZS)-j!NHZ7PACNW#=9VBfjj5hu9pA{p{C-q&1g`JFoGJ0Uj&?O%4m zD+WntREcLm=V)3sLN`3#F#zkCo$V#v@+@1`uxvs{- z-!EWw7W0#2tE=N0V*($LLmruh@{yNA)m&#dp2J6~8o+lQ&hRoxa$v=i-)5{GHI=B@ z!3{ikZxDf@KXIXk(xN^!4tz_rCp{)alD=Y2#Up6A2rO$7A8uqrsp55}ON(Qe?Kse9b zYY(%ot}rlXk(~M5_@j(Wu!s3&3FqLrokc#T5n;Q0^lPovpPpZam$K~fIM|dI<2xlY z>}NUlF;@e$O=#iyBYI3)>}#~3?1cMT3U)q~JPDaU<}6w8r1L zt|52gnZja`4jI=G|E9alggDHI+bjI&88Mq#eh_RD2YP-CW1i(6dle4b&rsUWaKB-6 z2W{RJRNy@=8a@KoPawhnF2u>0B~W}6(B&W#Pt*&&ve1Wy+Cm)~G1fUt=5J)zh? zaL1TA2?Lc5+`%-%+El~qRKvKwa=gW7v^4}akGTTCkh^1Yb4^-2`5u>EwF-zWaNqXu zpAT4^tbX|h=6^L+OOJeGtt(rZJxycJSYywe^}ama4MTX(FNDh?`=#@fR{S44g;7Sv zGzc(|t9@~U+OBjI&f%h%^|5h-Jg#&|$LJCGCV3i)09cP?Kk@7enYm!Ye_^bzO1p=5Gb z3!*iF0b?w8PeNrJuU&(L_>}%&2bTKi+XjGl%kEZ^D;EssDK_oXeBA5AGtPjqy<0vg zrr^6?-GM@MgP>z35bL`KIB>g+!^EP`;#}ym@it9!IR@05qh3x15>@;J%03L0-tP+k zmPkGfC!7jUEDA86h2gl{Bki=ps0>paAFTkOqUo8&92jL>t%JFvZE8O?_u@QS#K%BL z%J&*qKwSzB7Nnm1X+gqGfU0sE!cq-P6OgiUd+M@qxukF$@q(lG<`d^4^zV)qH>@@0 zjI`&4CLG=`9TMhoQgYyhlKYz@H;AmhF;cpgYCQku3KW??kCxO$e)rAZQVATruf$>! zOex9qyZbNfXh-Lh6rlU^CEoGBgX{ly1)Kh7xxSjAqlKZ7wW+Y3t+Szp?f=O7YZEhxGpwAM#$&83 zk|LF|u`deeto3U&g`7;gZ6=hHSn`r;JD}h;_twbR)BjnNqtBCGGJfG_vsOm zeu!GrMMP%ZA+jhwX3pGwgiIx|drq&>)BBYN+QniZS_XZ=i2XrSikhtm*~y{sK2OQ( z9-{En5pa$5AKC#nZ)Ly||0# zr5P)%b;-{(UazN(yT?o?N4t-Yqv@V68@)A#-Z(>9OqZY=S!_9MFZgJb z9(Vv$2RuA{IJ^v$8p4``a(`kt6Nn5TjjcKx5Co4wirO8^#9$~SDsn0s8-JY^B#D7L zq|!JRrADUOxZ4~~Wh-L;)dDU@y0yp<3Yly|T4Sj;tcLuvL}&0TZuf8{=*Ph&8XeF* z+yzl!FArq?EF}yb!}b>Bg)5ahPgEElL5^rT0eps<$E8^@FbD+r1TuFSIPs~6aqGwo zID!%uFER}ejT_f_vC)!s*oHz5v48Ff-*tFu%40dR=43M*N3)L3cu2J;uKv+{B@+ z*Hqp8@xw761vxe+#aa?8L$EwKOa6?UuLC?U))qgd#qs%*3%lk#=k#Z+-4@TYxVMB} z`4}Y04P77kw_Dc`eGb8MbUY1l9w3WTOnk`{R9uB`evEv}txkslD&iAf=Y_-EGy<(# zDTV033|jG^MGNXS>F|vIz@01BWK3#JGX{=f(j|P0v!+kJOI4Rh-^bycMxh`u2cz%+ z=r`yX*S*(Q3Wz721T%uRViVZu&m~*e}U<#!2_Nj;;H(z^g?zZBHWA|fpR$oZCP**MT&{cY#6jW1V97rBdD#^se*|yn+ zd~e6-&y~yd=Xb83=g`Oe?P55q8uu5zloqUfSsy;(Z!kX-YV5nd zzP7qW!kW(fu=&)^#&^$(1tjV)1<)9Yw;tk|)lMMnVYTz2Hgodn@Hb&o5uN7$i|R|nC{}v-m;5J^ z3Di$H2$3*y`-tjAhdlwo#dMORpM)<{J*3O^Vg(|l`}9VVFO~UMoX#)fTQvOK{eOaw zfKL@{E5A7vu#_05L^O+!;8i`NB&-$C+Y{cYFfXLx6BY_CIe|HT(X@EOz|~6iiZocV z^5@cU`LJPPe&4J_Cn89~J(>r$mwvoBrgvS1x7;qh*X-DH9LXG^tp~V#*9==vv4Mlu zs&|C{*e>AL6`=Ab1+)8EwDfr4342e1B@=oRclXi%*BYkF;0&@1ra zpcB=#$=xl9Dr#AnvVo2Y%IoAa6L!{`L}~v@J;2Cr%+|LSvEa6lxxG8<`6{3`+fQ0r z2wDgg13^3+H++z_vim>Fn_&5?6ZmO(vp(DZet8`KC)2E9WBotKGX+COCsW7&!8@BO z$bMqA=$TbE`4&x8Kf0(1c^heLq3kdOzsZXZmbz~iN^ho{jhiXoGlmL?$%n)E{T7h& zmW3DcS{d#%-0Ez(yL){=%#8v?v`467Sut?%GPM$hYgFb&7QtIyV`HXQ@qh}<+=zw( z#7QolwsA3@cW=?Y@qsK=8|8H|6yGV1Wf$%aX!snSSZeS_U{cOhuxYq&Tmn4nivC!!y!V=awm_HVdj)1@-MJ4>TYoycP@~kC<^F*$~ zL}8d7A&#ANjqOHWxsw;<;?1z@2(n^qDlXPY?<$zE>t?$*2q4bJ z(=)`L$l^PL@0mkv9kTCs%r3Lh5?if+p`2`TjWB z7)@zGdw^fKy(^3P@%Z8b`_T)ptGNP2G)}uppu&ab-feNIH%1~?WJ;d$;GT`!=|SQ z9AU-}s}F=zRNaOGD}YaOft6YZ%>cXWB?SxmzxgAXS8TEABYUsN-BE7ccGLD#i|X2M7!V(aazeVFS|{J~WZ@?UNx#1%tgMj1Zkr(W$J=_R(Mu*Wa`!AFL5 z9CD2gsUbo*-Ws9`FUHCX&mqDYI-8C}jpRV;8bE|WST)-&CwKoNG_#nrq%T3!l;n>m zz)#2%UCTRtBHrk4H4}QrgHAe?F)#we*v?BiUHUg1r_R!dch8r5uYHU+ zFu;m(8~ps0e@y_2L9o!4>U%4CGTMl~)CY7OVG5chA2H>kXUs;yQUZ(eSpdHJSw6n9 zUmuKsApfK~1vkcH*Zuzs+gmWavgUrKmFa-_?<_Xk|Fw1a{{T%bH*cN6n;e3qmY;kW z0$ASYPOXy3HcU=W?#T%FH%HY?aD-q&w(9C+kqw04V6R8TeoKDA zk-6#$!Ict{5JK!pQ@`Ajq)hSl^*erbH8(eNGylAv8uxpU`60gS-*8gfChc)hb4WUL zp!+56AyHo@?!i#|Chk#E`zG&UQGd`5+Mx!EBp%0eN$jD7c}Y4;Z)orFqdp~Z$?Scg z@}VNfa_Q_*g&C72OVXuq$r%+5z@Z{1aw+Z+qar69Cvj;SB@JkznoCzEbBP-Tiu4SK zg~_63kT*-{5Hr)&$?mye(cG|vrAT)Q?Ioes#Je{Q@Q`nj-^in0#ktoE z$b~&gdjHxp3xf;OCbLa?&K>|gw|UD&_4=fHs<QL%5*osW$1~PrE0T=MqO0~j$i}2Z%JJj$Wa(&7e zys~}s7`(!LtQfq~eGM4A;(bmSIt4pd7#q1eVo)`z*FT|Z)UTnT>{PBPpzPGHk>R)U zcdFpGig!MT>en&wTX{QvcRO?rT;T5jW$a=W8ROI`g+Zo3Lkh4WfEK&jRWJg(k10Ga zF)!JW5X=nt3bN8gF-T4raKx%a(Zl*SNCpyUB^qdCYH1EDz_G+m$I-&6cI%V+B-ej| z08HSkf$L;aK(9nBUSOqxAtc|S(NvX#HNNS$tq3t_t-_3wd zeJ3(Jx|I%nIQapFyx=&%9S$ZFouS5%!n8dEI1GRYhyoCUQ~?A4O4iK&YaWm?j+`-9 z)-C|ViZf-zm9dKta>DVQwr2*D15#LYrtG1>NdR@`O7qsZT{EmI865C8&?-O{z}dEA zoxQ_tREaS4XRa+jbC(HZ3E~DI;9xR8W3S;%nYV@-5`ftOoUI0kR+^-0tU5aZ+Yt^7 zT=Ek}t%FcU5mHz>RHxXO9Xm$M@w+ToIwVrqKxzh4sFW~aY^2GMMIm#IG-S$%(HHyh z)NvG(Q04)gr9<%5faHsPF&E~=pGth3~d}j(`Qz&6~Kix@ij+-SfhT7<{EcLh(~pB-O;HA~bT3+{M9wiTGE_rNUdf=WB!q z=AntsWpE$^0~2YI4XVS7bO>m(73v@wRPm?nQNVqG1Z*VJYZm+|0s{Z?aPeEF;!uGy zt}KiR11Q$PGaAA{-Kp<>iZ?)qm~i$z4-Bh81ez(+Ll2{f@Vh}>8it)Yp_}}07ad(P zJtc=acg;a^x@MJ)sy!BlMOHejO&UVdY~>q`oAB`Auo4wsy^WeZxDEUKW2M>Q_tQON z3`nDnaLO=w6#z?>>PCqXEC$9j8gi-o*C*G!XQja=lDxg2b((`#?!dD_bDAyskMh0u zeDOhs?0)c`F2;}V|I{2Zo;$*0Ztwq1Tw~pYhQ~OZIDjYz1XRwyZ|z9|n;TOVAK-2-=J zZ{ou3>R3CYxM{kF6Fl_K6>fxY%ENy|l&HSZ;+>;`ml5WH(zPYUEFmBfznZ6>$=$5Jrjez;&1%-H)(aE(Uj`LR8-5S8jX!+D&D{W z7Mif8*nImALQM>~f%`Rk(z;+_inM!AQcpV6ne{N-J5hD6rWo3r5MkLgam+>Ix?iQ- z=^UAwk$himC$;{$a0`PAJbdRBT4L{w{jAyMQ*t*Xpuw(eFWi=O2f#Wwo;j7k|X8Em9M_0R!$b8z4uA5AqsZU;D^6JD`$-jWk=QMDy|NEE}DPm93IqK{>GkH{NsseUhFuXTEn}5)TS1 z>n!FqyGN!w>sQROwKvuXo7~6U>M}j1AMVJUOaIMFU`C$&wLKYERMu!N`=K8nc1a|Q z91_^hD1s|?5zs($93h5$Wm*aF9wojAXFj3*(=nLZczXS@s*LBTCRQx7-XAKOgz(U8 z&(_N`0xQ6@0s5dXyQXakWuaQW&iWZD! zl_oiT!;^@#NDL^@IaRfIqGN_6uAI7~U@X+ydhVCgM(84`mCuKSJ7{ZR+w8Q<&gQRU z^oP&Mq7OvG;*7a8v+?Q_j=AgeL$d}EK1(X=#mR*K;ZBy*BbI{OtUs6}-`Yo?=&|Zc*fB z2Pr17O!@g;y6HU0d495lJI0R>3qcN1*{pHbF))o-(X1uD+&-_Y*Z2~4J*np`OHJ-l z#5mK`OHr3hT}a%YfAfPB$(Ju*jV}6S9IJMR++*kJQyDQ!M5dm(25DF#xaV7cR#;!l zud(}WH^xAf+<)IlxTCuRxOsTjtpxcFvgD|xRn7g|9e7Lfd42_b95lD^@g;m49y*`~ zd#@Ia%RUqrTo5wvWqJveWv1xB-h8xD-hhR(X=lx<|vvCdyC zK(5N7%vU}YR$In{!oPe-UAhse^0*AU@6(I%@w^PvxcvSfs!a1Y%R^#pjjDj^??x#M zlQ5cg1*Km(-+_Rn0p7xj+5Njjwz$FLbex;SmvVj?w$ZSy0@GH2K!fdrPf30pal~}wE z8*#}0WXBhnG0@~GVlK`M91dyY-H(twdc(7(Db4h6y%;-eL(#3jUr0!vGIT|!>bUH> z57ac6!(lgb4M5%x^cR1nFQ|O1#aXoj(ODs`@uj+TJMfC#(n$#x*3{LNlcN+i(d?1@ z5Vq;oK$rxPr5U+YaZ+?;6;C3HkOabb_mz%nS`8E}C{SPQfMzU@j;VD-jXph$Yg}aZ zEhlya8|bFrqU$U}G*zTIXuO#z1+E5gUoqi8MeKm)os=U?faYX*8M$#M33+|@o|?pPcaws@{BN{as{A?p4Fdxjm(jFfc#c)sC<1kZn-(D| z4eb1S`V@X;=g?2%ff?>-U`KA|k4c9(@r#MXMIM$!c$UW)Y)l7M&!|-krN; zXr=q#A%+09&m_fre^C}K5gNr$*ONzqiwl3Cmu*dgf7f3(dXC+{aAZY)SvyplgkNnE z3!;&tjCWK&<&*54Q7KZ35i5IrLH`j+QN(P0&MgK1@CAx~-gMLn5m`=j?(kO!;jIGt z0~wY8#vJ^pF0doc*Q(+W%}{LR2WN(^N`}RqV}cJkv*jqPRJY9Rj7Eo1gTf|$zJki1 z{auV>g2wNMqL`qYlf?`V%TGrOJyM> zLS7LQocQhn`S&?cv0ZIlPjhXl!1ff)G{cTWHX55s&AdJ9@SB4cmD8?r51D!4$%)Yd zN5bpksRKDCn#1-uf>I$`F}{plRp!%5@*L?nd%AOEm`Q)SRI77!P7Kx(hvHbURcSNO zCBwE_wj7oHK~vB&R`EQ^VOB!Rd$!FJ28Xyq^ojkJa>AD{cUn-qn-JDCvR#Tr2{VJ7 zzxE<9pGuU3(iFaq^OWEYSnJ^(t^TriPE!DB$ZraCuF6)WKOcT8A2BkUkfd>j$}e-L zhFm(Mc!4Ld-Z+JA2!Ak>_V4p3Bm3u|b_6nR=y$kvebtjTB=N{fO^HiyH;FC!+o?ZW z=(X8ry?X!3N-#lf>B3aX`5G`!GC2h8%!KhI_nnH@)Hf#oOV$LwrH|m+bCoA4K|7LZ z`5gnf!4LB$TlO|lae)bEM`ZSL=G$Z`(zkn@f;csq2odj!f@WR7XKbV~xLUfA@s(d% znMg|B7T8j#TR(T|f{*;TIy&B9I-k|^IiZzT!{Lm`G{4-)%FYS2xe>P$Oit}m%Sg%y3VJ{b#_Zo zYPbEwpcHyrv&Qa6C7>d=S^eYCzD=wO$H|cD;_mV${(U;9xhfX%ePRrP7&?Ch6JNy3 zbdI8xJxP)K={T6|xD>`}gk1e~OZ6>|n-Y=yVq>@EBg`V;1h+E2Y9OY$uOw7Ayei~- z2FUO5bu$`{9+uFZkP&zp9+vnmUzy3l8mCmLA?Q^a1@TBD*L%KCg>vcy-NYRb(D+1@ zCAa`}Qh0$@_@02V>rzVOe7>RLI@B_0Pu05C%0T$b0yj&rkcTtBW$u|uLu@X_kJNuL z62U3cJe%ZDJkz5nl1w>}w_}7jEOTX7YEY$0s>k{qYtuvRPxw#A;ysnYseuS3GTN^{ zmLlekEDtURcRI|&*-S8Zi|0yh^P;@vXcNWK*b^Uo!Fn^*&2O!2-7y4KOwVC1 zEt*@Yx;${NRkr`txu|XT6unzml(rGm^>L_Pk|_mB!xlAj8fG^q&J+>IrPd<27O| zXzxsC?1tA@PS)mGpPQ|$NN&PTkl;h`<~kAKsr9m?KRehnu=V0%|8|R*uJW%cQak+F zU5PhUbrs-e0~#ME{5{^WYhC`%`t@JY+R(R775xs?b8>@<74hDU@G7I+-w#aT@PXfW z;cd38lO4@dIvQHSi6sh`f_i0px)eE@5k9hU=YcGM-j`TptlxMH1yF7H6nz*>h}A)Z z_83u4^ifU(X*UvHkw$D$aVbE==_@V84$;;jTZ~uY1y}Ik!yjECf{foGsQCs6yxG4g zTa}W34^Q~}>25*ZjiJ?+IawSvw{6boZ6nfJSY~O7LJ$&dy*BaEDEvYCOJ?AthQ+IN zHA&H5AicS|-pO4^l!IX%a9(toX_Lj44BnAQ1yK4!pc$4wWI+?};}yhZ14{bRL~`IS z#5OZuZ)&l2q3fT-_BxTEdml^!@p>US`?*@fOs&E~ujWJ0th zw<$E>S`vnMN<%tIW4uaJIxI}D@{+n@JYRl!8>~$<#d_v$-6p#U!%RLhZe>Y