From 2fe7bd03d531ccfdfe55ea33082bf8bfbb762d2f Mon Sep 17 00:00:00 2001 From: shizhili Date: Wed, 20 Sep 2023 17:59:10 +0800 Subject: [PATCH] backport support kv storage --- patch009-backport-Support-KV-Storage.patch | 3943 ++++++++++++++++++++ rocketmq-all-5.1.3-source-release.zip | Bin 8962036 -> 8962028 bytes rocketmq.spec | 6 +- 3 files changed, 3948 insertions(+), 1 deletion(-) create mode 100644 patch009-backport-Support-KV-Storage.patch diff --git a/patch009-backport-Support-KV-Storage.patch b/patch009-backport-Support-KV-Storage.patch new file mode 100644 index 0000000..eb9ee6a --- /dev/null +++ b/patch009-backport-Support-KV-Storage.patch @@ -0,0 +1,3943 @@ +From 3a6ef0400c8f3dc420b8781c619e66d47d1c4336 Mon Sep 17 00:00:00 2001 +From: fujian-zfj <2573259572@qq.com> +Date: Sat, 5 Aug 2023 00:32:11 +0800 +Subject: [PATCH 1/4] [ISSUE #7064] [RIP-66-1] Support KV(RocksDB) Storage for + Metadata (#7092) + +* typo int readme[ecosystem] + +* rocksdb metadata + +* add unit test + +* fix testOffsetPersistInMemory + +* fix unit test + +* fix unit test + +* remove unused import + +* move RocksDBOffsetSerialize to broker moudle + +* Fix bazel build scripts + +Signed-off-by: Li Zhanhui + +* Flag QueryMsgByKeyIT as flaky as it fails at frequency: 5 out of 32 + +Signed-off-by: Li Zhanhui + +* change public to private of some inner method + +--------- + +Signed-off-by: Li Zhanhui +Co-authored-by: Li Zhanhui +--- + WORKSPACE | 1 + + broker/BUILD.bazel | 3 + + .../rocketmq/broker/BrokerController.java | 41 +- + .../broker/offset/ConsumerOffsetManager.java | 20 +- + .../offset/RocksDBConsumerOffsetManager.java | 102 +++ + .../RocksDBLmqConsumerOffsetManager.java | 103 +++ + .../offset/RocksDBOffsetSerializeWrapper.java | 34 + + .../schedule/ScheduleMessageService.java | 5 +- + .../RocksDBLmqSubscriptionGroupManager.java | 46 ++ + .../RocksDBSubscriptionGroupManager.java | 112 ++++ + .../SubscriptionGroupManager.java | 64 +- + .../topic/RocksDBLmqTopicConfigManager.java | 57 ++ + .../topic/RocksDBTopicConfigManager.java | 95 +++ + .../broker/topic/TopicConfigManager.java | 110 ++-- + .../src/main/resources/rmq.broker.logback.xml | 37 ++ + .../RocksDBConsumerOffsetManagerTest.java | 113 ++++ + .../processor/AdminBrokerProcessorTest.java | 126 +++- + .../ForbiddenTest.java | 3 +- + .../SubscriptionGroupManagerTest.java | 25 + + .../topic/RocksdbTopicConfigManagerTest.java | 375 +++++++++++ + client/BUILD.bazel | 1 + + common/BUILD.bazel | 1 + + common/pom.xml | 4 + + .../apache/rocketmq/common/ConfigManager.java | 22 +- + .../common/config/AbstractRocksDBStorage.java | 613 ++++++++++++++++++ + .../common/config/ConfigRocksDBStorage.java | 250 +++++++ + .../common/config/RocksDBConfigManager.java | 108 +++ + .../rocketmq/common/constant/LoggerName.java | 1 + + .../rocketmq/example/quickstart/Consumer.java | 3 +- + pom.xml | 6 + + remoting/BUILD.bazel | 1 + + .../org/apache/rocketmq/store/StoreType.java | 32 + + .../store/config/MessageStoreConfig.java | 41 ++ + test/BUILD.bazel | 3 + + tieredstore/BUILD.bazel | 1 + + 35 files changed, 2473 insertions(+), 86 deletions(-) + create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java + create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBLmqConsumerOffsetManager.java + create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapper.java + create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBLmqSubscriptionGroupManager.java + create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java + create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBLmqTopicConfigManager.java + create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java + create mode 100644 broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java + rename broker/src/test/java/org/apache/rocketmq/broker/{substription => subscription}/ForbiddenTest.java (95%) + create mode 100644 broker/src/test/java/org/apache/rocketmq/broker/topic/RocksdbTopicConfigManagerTest.java + create mode 100644 common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java + create mode 100644 common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java + create mode 100644 common/src/main/java/org/apache/rocketmq/common/config/RocksDBConfigManager.java + create mode 100644 store/src/main/java/org/apache/rocketmq/store/StoreType.java + +diff --git a/WORKSPACE b/WORKSPACE +index e3a8f37dc..a8a0aafe9 100644 +--- a/WORKSPACE ++++ b/WORKSPACE +@@ -105,6 +105,7 @@ maven_install( + "com.fasterxml.jackson.core:jackson-databind:2.13.4.2", + "com.adobe.testing:s3mock-junit4:2.11.0", + "io.github.aliyunmq:rocketmq-grpc-netty-codec-haproxy:1.0.0", ++ "io.github.aliyunmq:rocketmq-rocksdb:1.0.3", + ], + fetch_sources = True, + repositories = [ +diff --git a/broker/BUILD.bazel b/broker/BUILD.bazel +index d0d3a2f96..6adcdc7b9 100644 +--- a/broker/BUILD.bazel ++++ b/broker/BUILD.bazel +@@ -53,6 +53,8 @@ java_library( + "@maven//:io_github_aliyunmq_rocketmq_logback_classic", + "@maven//:org_slf4j_jul_to_slf4j", + "@maven//:io_github_aliyunmq_rocketmq_shaded_slf4j_api_bridge", ++ "@maven//:io_github_aliyunmq_rocketmq_rocksdb", ++ "@maven//:net_java_dev_jna_jna", + ], + ) + +@@ -81,6 +83,7 @@ java_library( + "@maven//:org_apache_commons_commons_lang3", + "@maven//:io_github_aliyunmq_rocketmq_slf4j_api", + "@maven//:org_powermock_powermock_core", ++ "@maven//:io_opentelemetry_opentelemetry_api", + ], + ) + +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +index 972457194..30b1d2299 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +@@ -45,6 +45,8 @@ import org.apache.rocketmq.broker.offset.BroadcastOffsetManager; + import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; + import org.apache.rocketmq.broker.offset.ConsumerOrderInfoManager; + import org.apache.rocketmq.broker.offset.LmqConsumerOffsetManager; ++import org.apache.rocketmq.broker.offset.RocksDBConsumerOffsetManager; ++import org.apache.rocketmq.broker.offset.RocksDBLmqConsumerOffsetManager; + import org.apache.rocketmq.broker.out.BrokerOuterAPI; + import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin; + import org.apache.rocketmq.broker.processor.AckMessageProcessor; +@@ -66,8 +68,12 @@ import org.apache.rocketmq.broker.processor.SendMessageProcessor; + import org.apache.rocketmq.broker.schedule.ScheduleMessageService; + import org.apache.rocketmq.broker.slave.SlaveSynchronize; + import org.apache.rocketmq.broker.subscription.LmqSubscriptionGroupManager; ++import org.apache.rocketmq.broker.subscription.RocksDBLmqSubscriptionGroupManager; ++import org.apache.rocketmq.broker.subscription.RocksDBSubscriptionGroupManager; + import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; + import org.apache.rocketmq.broker.topic.LmqTopicConfigManager; ++import org.apache.rocketmq.broker.topic.RocksDBLmqTopicConfigManager; ++import org.apache.rocketmq.broker.topic.RocksDBTopicConfigManager; + import org.apache.rocketmq.broker.topic.TopicConfigManager; + import org.apache.rocketmq.broker.topic.TopicQueueMappingCleanService; + import org.apache.rocketmq.broker.topic.TopicQueueMappingManager; +@@ -120,6 +126,7 @@ import org.apache.rocketmq.store.DefaultMessageStore; + import org.apache.rocketmq.store.MessageArrivingListener; + import org.apache.rocketmq.store.MessageStore; + import org.apache.rocketmq.store.PutMessageResult; ++import org.apache.rocketmq.store.StoreType; + import org.apache.rocketmq.store.config.BrokerRole; + import org.apache.rocketmq.store.config.MessageStoreConfig; + import org.apache.rocketmq.store.dledger.DLedgerCommitLog; +@@ -301,9 +308,16 @@ public class BrokerController { + this.messageStoreConfig = messageStoreConfig; + this.setStoreHost(new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), getListenPort())); + this.brokerStatsManager = messageStoreConfig.isEnableLmq() ? new LmqBrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()) : new BrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()); +- this.consumerOffsetManager = messageStoreConfig.isEnableLmq() ? new LmqConsumerOffsetManager(this) : new ConsumerOffsetManager(this); + this.broadcastOffsetManager = new BroadcastOffsetManager(this); +- this.topicConfigManager = messageStoreConfig.isEnableLmq() ? new LmqTopicConfigManager(this) : new TopicConfigManager(this); ++ if (isEnableRocksDBStore()) { ++ this.topicConfigManager = messageStoreConfig.isEnableLmq() ? new RocksDBLmqTopicConfigManager(this) : new RocksDBTopicConfigManager(this); ++ this.subscriptionGroupManager = messageStoreConfig.isEnableLmq() ? new RocksDBLmqSubscriptionGroupManager(this) : new RocksDBSubscriptionGroupManager(this); ++ this.consumerOffsetManager = messageStoreConfig.isEnableLmq() ? new RocksDBLmqConsumerOffsetManager(this) : new RocksDBConsumerOffsetManager(this); ++ } else { ++ this.topicConfigManager = messageStoreConfig.isEnableLmq() ? new LmqTopicConfigManager(this) : new TopicConfigManager(this); ++ this.subscriptionGroupManager = messageStoreConfig.isEnableLmq() ? new LmqSubscriptionGroupManager(this) : new SubscriptionGroupManager(this); ++ this.consumerOffsetManager = messageStoreConfig.isEnableLmq() ? new LmqConsumerOffsetManager(this) : new ConsumerOffsetManager(this); ++ } + this.topicQueueMappingManager = new TopicQueueMappingManager(this); + this.pullMessageProcessor = new PullMessageProcessor(this); + this.peekMessageProcessor = new PeekMessageProcessor(this); +@@ -324,7 +338,6 @@ public class BrokerController { + this.popInflightMessageCounter = new PopInflightMessageCounter(this); + this.clientHousekeepingService = new ClientHousekeepingService(this); + this.broker2Client = new Broker2Client(this); +- this.subscriptionGroupManager = messageStoreConfig.isEnableLmq() ? new LmqSubscriptionGroupManager(this) : new SubscriptionGroupManager(this); + this.scheduleMessageService = new ScheduleMessageService(this); + this.coldDataPullRequestHoldService = new ColdDataPullRequestHoldService(this); + this.coldDataCgCtrService = new ColdDataCgCtrService(this); +@@ -1383,8 +1396,6 @@ public class BrokerController { + this.adminBrokerExecutor.shutdown(); + } + +- this.consumerOffsetManager.persist(); +- + if (this.brokerFastFailure != null) { + this.brokerFastFailure.shutdown(); + } +@@ -1449,8 +1460,20 @@ public class BrokerController { + shutdownScheduledExecutorService(this.syncBrokerMemberGroupExecutorService); + shutdownScheduledExecutorService(this.brokerHeartbeatExecutorService); + +- this.topicConfigManager.persist(); +- this.subscriptionGroupManager.persist(); ++ if (this.topicConfigManager != null) { ++ this.topicConfigManager.persist(); ++ this.topicConfigManager.stop(); ++ } ++ ++ if (this.subscriptionGroupManager != null) { ++ this.subscriptionGroupManager.persist(); ++ this.subscriptionGroupManager.stop(); ++ } ++ ++ if (this.consumerOffsetManager != null) { ++ this.consumerOffsetManager.persist(); ++ this.consumerOffsetManager.stop(); ++ } + + for (BrokerAttachedPlugin brokerAttachedPlugin : brokerAttachedPlugins) { + if (brokerAttachedPlugin != null) { +@@ -2375,4 +2398,8 @@ public class BrokerController { + public void setColdDataCgCtrService(ColdDataCgCtrService coldDataCgCtrService) { + this.coldDataCgCtrService = coldDataCgCtrService; + } ++ ++ public boolean isEnableRocksDBStore() { ++ return StoreType.DEFAULT_ROCKSDB.getStoreType().equalsIgnoreCase(this.messageStoreConfig.getStoreType()); ++ } + } +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java +index 8bf4e9a59..21f20dde3 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOffsetManager.java +@@ -16,7 +16,6 @@ + */ + package org.apache.rocketmq.broker.offset; + +-import com.google.common.base.Strings; + import java.util.HashMap; + import java.util.HashSet; + import java.util.Iterator; +@@ -26,6 +25,9 @@ import java.util.Set; + import java.util.concurrent.ConcurrentHashMap; + import java.util.concurrent.ConcurrentMap; + import java.util.concurrent.atomic.AtomicLong; ++ ++import com.google.common.base.Strings; ++ + import org.apache.rocketmq.broker.BrokerController; + import org.apache.rocketmq.broker.BrokerPathConfigHelper; + import org.apache.rocketmq.common.ConfigManager; +@@ -37,12 +39,12 @@ import org.apache.rocketmq.remoting.protocol.DataVersion; + import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + + public class ConsumerOffsetManager extends ConfigManager { +- private static final Logger LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); ++ protected static final Logger LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); + public static final String TOPIC_GROUP_SEPARATOR = "@"; + + private DataVersion dataVersion = new DataVersion(); + +- private ConcurrentMap> offsetTable = ++ protected ConcurrentMap> offsetTable = + new ConcurrentHashMap<>(512); + + private final ConcurrentMap> resetOffsetTable = +@@ -62,6 +64,10 @@ public class ConsumerOffsetManager extends ConfigManager { + this.brokerController = brokerController; + } + ++ protected void removeConsumerOffset(String topicAtGroup) { ++ ++ } ++ + public void cleanOffset(String group) { + Iterator>> it = this.offsetTable.entrySet().iterator(); + while (it.hasNext()) { +@@ -71,6 +77,7 @@ public class ConsumerOffsetManager extends ConfigManager { + String[] arrays = topicAtGroup.split(TOPIC_GROUP_SEPARATOR); + if (arrays.length == 2 && group.equals(arrays[1])) { + it.remove(); ++ removeConsumerOffset(topicAtGroup); + LOG.warn("Clean group's offset, {}, {}", topicAtGroup, next.getValue()); + } + } +@@ -86,6 +93,7 @@ public class ConsumerOffsetManager extends ConfigManager { + String[] arrays = topicAtGroup.split(TOPIC_GROUP_SEPARATOR); + if (arrays.length == 2 && topic.equals(arrays[0])) { + it.remove(); ++ removeConsumerOffset(topicAtGroup); + LOG.warn("Clean topic's offset, {}, {}", topicAtGroup, next.getValue()); + } + } +@@ -105,6 +113,7 @@ public class ConsumerOffsetManager extends ConfigManager { + if (null == brokerController.getConsumerManager().findSubscriptionData(group, topic) + && this.offsetBehindMuchThanData(topic, next.getValue())) { + it.remove(); ++ removeConsumerOffset(topicAtGroup); + LOG.warn("remove topic offset, {}", topicAtGroup); + } + } +@@ -313,8 +322,10 @@ public class ConsumerOffsetManager extends ConfigManager { + for (String group : filterGroups.split(",")) { + Iterator it = topicGroups.iterator(); + while (it.hasNext()) { +- if (group.equals(it.next().split(TOPIC_GROUP_SEPARATOR)[1])) { ++ String topicAtGroup = it.next(); ++ if (group.equals(topicAtGroup.split(TOPIC_GROUP_SEPARATOR)[1])) { + it.remove(); ++ removeConsumerOffset(topicAtGroup); + } + } + } +@@ -371,6 +382,7 @@ public class ConsumerOffsetManager extends ConfigManager { + String[] arrays = topicAtGroup.split(TOPIC_GROUP_SEPARATOR); + if (arrays.length == 2 && group.equals(arrays[1])) { + it.remove(); ++ removeConsumerOffset(topicAtGroup); + LOG.warn("clean group offset {}", topicAtGroup); + } + } +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java +new file mode 100644 +index 000000000..5695a3356 +--- /dev/null ++++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java +@@ -0,0 +1,102 @@ ++/* ++ * 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.rocketmq.broker.offset; ++ ++import java.io.File; ++import java.util.Iterator; ++import java.util.Map.Entry; ++import java.util.concurrent.ConcurrentMap; ++ ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.config.RocksDBConfigManager; ++import org.apache.rocketmq.common.utils.DataConverter; ++import org.rocksdb.WriteBatch; ++ ++import com.alibaba.fastjson.JSON; ++import com.alibaba.fastjson.serializer.SerializerFeature; ++ ++public class RocksDBConsumerOffsetManager extends ConsumerOffsetManager { ++ ++ public RocksDBConsumerOffsetManager(BrokerController brokerController) { ++ super(brokerController); ++ this.rocksDBConfigManager = new RocksDBConfigManager(this.brokerController.getMessageStoreConfig().getMemTableFlushInterval()); ++ } ++ ++ @Override ++ public boolean load() { ++ return this.rocksDBConfigManager.load(configFilePath(), this::decode0); ++ } ++ ++ @Override ++ public boolean stop() { ++ return this.rocksDBConfigManager.stop(); ++ } ++ ++ @Override ++ protected void removeConsumerOffset(String topicAtGroup) { ++ try { ++ byte[] keyBytes = topicAtGroup.getBytes(DataConverter.charset); ++ this.rocksDBConfigManager.delete(keyBytes); ++ } catch (Exception e) { ++ LOG.error("kv remove consumerOffset Failed, {}", topicAtGroup); ++ } ++ } ++ ++ @Override ++ protected void decode0(final byte[] key, final byte[] body) { ++ String topicAtGroup = new String(key, DataConverter.charset); ++ RocksDBOffsetSerializeWrapper wrapper = JSON.parseObject(body, RocksDBOffsetSerializeWrapper.class); ++ ++ this.offsetTable.put(topicAtGroup, wrapper.getOffsetTable()); ++ LOG.info("load exist local offset, {}, {}", topicAtGroup, wrapper.getOffsetTable()); ++ } ++ ++ @Override ++ public String configFilePath() { ++ return this.brokerController.getMessageStoreConfig().getStorePathRootDir() + File.separator + "config" + File.separator + "consumerOffsets" + File.separator; ++ } ++ ++ @Override ++ public synchronized void persist() { ++ WriteBatch writeBatch = new WriteBatch(); ++ try { ++ Iterator>> iterator = this.offsetTable.entrySet().iterator(); ++ while (iterator.hasNext()) { ++ Entry> entry = iterator.next(); ++ putWriteBatch(writeBatch, entry.getKey(), entry.getValue()); ++ ++ if (writeBatch.getDataSize() >= 4 * 1024) { ++ this.rocksDBConfigManager.batchPutWithWal(writeBatch); ++ } ++ } ++ this.rocksDBConfigManager.batchPutWithWal(writeBatch); ++ this.rocksDBConfigManager.flushWAL(); ++ } catch (Exception e) { ++ LOG.error("consumer offset persist Failed", e); ++ } finally { ++ writeBatch.close(); ++ } ++ } ++ ++ private void putWriteBatch(final WriteBatch writeBatch, final String topicGroupName, final ConcurrentMap offsetMap) throws Exception { ++ byte[] keyBytes = topicGroupName.getBytes(DataConverter.charset); ++ RocksDBOffsetSerializeWrapper wrapper = new RocksDBOffsetSerializeWrapper(); ++ wrapper.setOffsetTable(offsetMap); ++ byte[] valueBytes = JSON.toJSONBytes(wrapper, SerializerFeature.BrowserCompatible); ++ writeBatch.put(keyBytes, valueBytes); ++ } ++} +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBLmqConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBLmqConsumerOffsetManager.java +new file mode 100644 +index 000000000..d0faa6614 +--- /dev/null ++++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBLmqConsumerOffsetManager.java +@@ -0,0 +1,103 @@ ++/* ++ * 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.rocketmq.broker.offset; ++ ++import java.util.HashMap; ++import java.util.Map; ++import java.util.concurrent.ConcurrentHashMap; ++ ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.MixAll; ++import org.apache.rocketmq.remoting.protocol.RemotingSerializable; ++ ++public class RocksDBLmqConsumerOffsetManager extends RocksDBConsumerOffsetManager { ++ private ConcurrentHashMap lmqOffsetTable = new ConcurrentHashMap<>(512); ++ ++ public RocksDBLmqConsumerOffsetManager(BrokerController brokerController) { ++ super(brokerController); ++ } ++ ++ @Override ++ public long queryOffset(final String group, final String topic, final int queueId) { ++ if (!MixAll.isLmq(group)) { ++ return super.queryOffset(group, topic, queueId); ++ } ++ // topic@group ++ String key = topic + TOPIC_GROUP_SEPARATOR + group; ++ Long offset = lmqOffsetTable.get(key); ++ if (offset != null) { ++ return offset; ++ } ++ return -1; ++ } ++ ++ @Override ++ public Map queryOffset(final String group, final String topic) { ++ if (!MixAll.isLmq(group)) { ++ return super.queryOffset(group, topic); ++ } ++ Map map = new HashMap<>(); ++ // topic@group ++ String key = topic + TOPIC_GROUP_SEPARATOR + group; ++ Long offset = lmqOffsetTable.get(key); ++ if (offset != null) { ++ map.put(0, offset); ++ } ++ return map; ++ } ++ ++ @Override ++ public void commitOffset(final String clientHost, final String group, final String topic, final int queueId, ++ final long offset) { ++ if (!MixAll.isLmq(group)) { ++ super.commitOffset(clientHost, group, topic, queueId, offset); ++ return; ++ } ++ // topic@group ++ String key = topic + TOPIC_GROUP_SEPARATOR + group; ++ lmqOffsetTable.put(key, offset); ++ } ++ ++ @Override ++ public String encode() { ++ return this.encode(false); ++ } ++ ++ @Override ++ public void decode(String jsonString) { ++ if (jsonString != null) { ++ RocksDBLmqConsumerOffsetManager obj = RemotingSerializable.fromJson(jsonString, RocksDBLmqConsumerOffsetManager.class); ++ if (obj != null) { ++ super.setOffsetTable(obj.getOffsetTable()); ++ this.lmqOffsetTable = obj.lmqOffsetTable; ++ } ++ } ++ } ++ ++ @Override ++ public String encode(final boolean prettyFormat) { ++ return RemotingSerializable.toJson(this, prettyFormat); ++ } ++ ++ public ConcurrentHashMap getLmqOffsetTable() { ++ return lmqOffsetTable; ++ } ++ ++ public void setLmqOffsetTable(ConcurrentHashMap lmqOffsetTable) { ++ this.lmqOffsetTable = lmqOffsetTable; ++ } ++} +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapper.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapper.java +new file mode 100644 +index 000000000..7a90fd62f +--- /dev/null ++++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBOffsetSerializeWrapper.java +@@ -0,0 +1,34 @@ ++/* ++ * 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.rocketmq.broker.offset; ++ ++import java.util.concurrent.ConcurrentHashMap; ++import java.util.concurrent.ConcurrentMap; ++ ++import org.apache.rocketmq.remoting.protocol.RemotingSerializable; ++ ++public class RocksDBOffsetSerializeWrapper extends RemotingSerializable { ++ private ConcurrentMap offsetTable = new ConcurrentHashMap(16); ++ ++ public ConcurrentMap getOffsetTable() { ++ return offsetTable; ++ } ++ ++ public void setOffsetTable(ConcurrentMap offsetTable) { ++ this.offsetTable = offsetTable; ++ } ++} +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java +index 26f09dcd0..aed0ee19f 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java +@@ -92,7 +92,7 @@ public class ScheduleMessageService extends ConfigManager { + this.brokerController = brokerController; + this.enableAsyncDeliver = brokerController.getMessageStoreConfig().isEnableScheduleAsyncDeliver(); + scheduledPersistService = new ScheduledThreadPoolExecutor(1, +- new ThreadFactoryImpl("ScheduleMessageServicePersistThread", true, brokerController.getBrokerConfig())); ++ new ThreadFactoryImpl("ScheduleMessageServicePersistThread", true, brokerController.getBrokerConfig())); + } + + public static int queueId2DelayLevel(final int queueId) { +@@ -169,7 +169,7 @@ public class ScheduleMessageService extends ConfigManager { + ThreadUtils.shutdown(scheduledPersistService); + } + +- public void stop() { ++ public boolean stop() { + if (this.started.compareAndSet(true, false) && null != this.deliverExecutorService) { + this.deliverExecutorService.shutdown(); + try { +@@ -193,6 +193,7 @@ public class ScheduleMessageService extends ConfigManager { + + this.persist(); + } ++ return true; + } + + public boolean isStarted() { +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBLmqSubscriptionGroupManager.java b/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBLmqSubscriptionGroupManager.java +new file mode 100644 +index 000000000..8c05d0bd9 +--- /dev/null ++++ b/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBLmqSubscriptionGroupManager.java +@@ -0,0 +1,46 @@ ++/* ++ * Licensed to the Apache Software Foundation (ASF) under one or more ++ * contributor license agreements. See the NOTICE file distributed with ++ * this work for additional information regarding copyright ownership. ++ * The ASF licenses this file to You under the Apache License, Version 2.0 ++ * (the "License"); you may not use this file except in compliance with ++ * the License. You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, software ++ * distributed under the License is distributed on an "AS IS" BASIS, ++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ++ * See the License for the specific language governing permissions and ++ * limitations under the License. ++ */ ++package org.apache.rocketmq.broker.subscription; ++ ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.MixAll; ++import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; ++ ++public class RocksDBLmqSubscriptionGroupManager extends RocksDBSubscriptionGroupManager { ++ ++ public RocksDBLmqSubscriptionGroupManager(BrokerController brokerController) { ++ super(brokerController); ++ } ++ ++ @Override ++ public SubscriptionGroupConfig findSubscriptionGroupConfig(final String group) { ++ if (MixAll.isLmq(group)) { ++ SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); ++ subscriptionGroupConfig.setGroupName(group); ++ return subscriptionGroupConfig; ++ } ++ return super.findSubscriptionGroupConfig(group); ++ } ++ ++ @Override ++ public void updateSubscriptionGroupConfig(final SubscriptionGroupConfig config) { ++ if (config == null || MixAll.isLmq(config.getGroupName())) { ++ return; ++ } ++ super.updateSubscriptionGroupConfig(config); ++ } ++} +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java b/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java +new file mode 100644 +index 000000000..6503970af +--- /dev/null ++++ b/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java +@@ -0,0 +1,112 @@ ++/* ++ * Licensed to the Apache Software Foundation (ASF) under one or more ++ * contributor license agreements. See the NOTICE file distributed with ++ * this work for additional information regarding copyright ownership. ++ * The ASF licenses this file to You under the Apache License, Version 2.0 ++ * (the "License"); you may not use this file except in compliance with ++ * the License. You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, software ++ * distributed under the License is distributed on an "AS IS" BASIS, ++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ++ * See the License for the specific language governing permissions and ++ * limitations under the License. ++ */ ++package org.apache.rocketmq.broker.subscription; ++ ++import java.io.File; ++ ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.config.RocksDBConfigManager; ++import org.apache.rocketmq.common.utils.DataConverter; ++import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; ++ ++import com.alibaba.fastjson.JSON; ++import com.alibaba.fastjson.serializer.SerializerFeature; ++ ++public class RocksDBSubscriptionGroupManager extends SubscriptionGroupManager { ++ ++ public RocksDBSubscriptionGroupManager(BrokerController brokerController) { ++ super(brokerController, false); ++ this.rocksDBConfigManager = new RocksDBConfigManager(this.brokerController.getMessageStoreConfig().getMemTableFlushInterval()); ++ } ++ ++ @Override ++ public boolean load() { ++ if (!this.rocksDBConfigManager.load(configFilePath(), this::decode0)) { ++ return false; ++ } ++ this.init(); ++ return true; ++ } ++ ++ @Override ++ public boolean stop() { ++ return this.rocksDBConfigManager.stop(); ++ } ++ ++ @Override ++ protected SubscriptionGroupConfig putSubscriptionGroupConfig(SubscriptionGroupConfig subscriptionGroupConfig) { ++ String groupName = subscriptionGroupConfig.getGroupName(); ++ SubscriptionGroupConfig oldConfig = this.subscriptionGroupTable.put(groupName, subscriptionGroupConfig); ++ ++ try { ++ byte[] keyBytes = groupName.getBytes(DataConverter.charset); ++ byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, SerializerFeature.BrowserCompatible); ++ this.rocksDBConfigManager.put(keyBytes, keyBytes.length, valueBytes); ++ } catch (Exception e) { ++ log.error("kv put sub Failed, {}", subscriptionGroupConfig.toString()); ++ } ++ return oldConfig; ++ } ++ ++ @Override ++ protected SubscriptionGroupConfig putSubscriptionGroupConfigIfAbsent(SubscriptionGroupConfig subscriptionGroupConfig) { ++ String groupName = subscriptionGroupConfig.getGroupName(); ++ SubscriptionGroupConfig oldConfig = this.subscriptionGroupTable.putIfAbsent(groupName, subscriptionGroupConfig); ++ if (oldConfig == null) { ++ try { ++ byte[] keyBytes = groupName.getBytes(DataConverter.charset); ++ byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, SerializerFeature.BrowserCompatible); ++ this.rocksDBConfigManager.put(keyBytes, keyBytes.length, valueBytes); ++ } catch (Exception e) { ++ log.error("kv put sub Failed, {}", subscriptionGroupConfig.toString()); ++ } ++ } ++ return oldConfig; ++ } ++ ++ @Override ++ protected SubscriptionGroupConfig removeSubscriptionGroupConfig(String groupName) { ++ SubscriptionGroupConfig subscriptionGroupConfig = this.subscriptionGroupTable.remove(groupName); ++ try { ++ this.rocksDBConfigManager.delete(groupName.getBytes(DataConverter.charset)); ++ } catch (Exception e) { ++ log.error("kv delete sub Failed, {}", subscriptionGroupConfig.toString()); ++ } ++ return subscriptionGroupConfig; ++ } ++ ++ @Override ++ protected void decode0(byte[] key, byte[] body) { ++ String groupName = new String(key, DataConverter.charset); ++ SubscriptionGroupConfig subscriptionGroupConfig = JSON.parseObject(body, SubscriptionGroupConfig.class); ++ ++ this.subscriptionGroupTable.put(groupName, subscriptionGroupConfig); ++ log.info("load exist local sub, {}", subscriptionGroupConfig.toString()); ++ } ++ ++ @Override ++ public synchronized void persist() { ++ if (brokerController.getMessageStoreConfig().isRealTimePersistRocksDBConfig()) { ++ this.rocksDBConfigManager.flushWAL(); ++ } ++ } ++ ++ @Override ++ public String configFilePath() { ++ return this.brokerController.getMessageStoreConfig().getStorePathRootDir() + File.separator + "config" + File.separator + "subscriptionGroups" + File.separator; ++ } ++} +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java b/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java +index 0ae11313f..74e39c0fe 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java +@@ -40,81 +40,103 @@ import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; + + public class SubscriptionGroupManager extends ConfigManager { +- private static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); ++ protected static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); + +- private ConcurrentMap subscriptionGroupTable = ++ protected ConcurrentMap subscriptionGroupTable = + new ConcurrentHashMap<>(1024); + + private ConcurrentMap> forbiddenTable = + new ConcurrentHashMap<>(4); + + private final DataVersion dataVersion = new DataVersion(); +- private transient BrokerController brokerController; ++ protected transient BrokerController brokerController; + + public SubscriptionGroupManager() { + this.init(); + } + + public SubscriptionGroupManager(BrokerController brokerController) { ++ this(brokerController, true); ++ } ++ ++ public SubscriptionGroupManager(BrokerController brokerController, boolean init) { + this.brokerController = brokerController; +- this.init(); ++ if (init) { ++ init(); ++ } + } + +- private void init() { ++ protected void init() { + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.TOOLS_CONSUMER_GROUP); +- this.subscriptionGroupTable.put(MixAll.TOOLS_CONSUMER_GROUP, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.FILTERSRV_CONSUMER_GROUP); +- this.subscriptionGroupTable.put(MixAll.FILTERSRV_CONSUMER_GROUP, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.SELF_TEST_CONSUMER_GROUP); +- this.subscriptionGroupTable.put(MixAll.SELF_TEST_CONSUMER_GROUP, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.ONS_HTTP_PROXY_GROUP); + subscriptionGroupConfig.setConsumeBroadcastEnable(true); +- this.subscriptionGroupTable.put(MixAll.ONS_HTTP_PROXY_GROUP, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.CID_ONSAPI_PULL_GROUP); + subscriptionGroupConfig.setConsumeBroadcastEnable(true); +- this.subscriptionGroupTable.put(MixAll.CID_ONSAPI_PULL_GROUP, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.CID_ONSAPI_PERMISSION_GROUP); + subscriptionGroupConfig.setConsumeBroadcastEnable(true); +- this.subscriptionGroupTable.put(MixAll.CID_ONSAPI_PERMISSION_GROUP, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.CID_ONSAPI_OWNER_GROUP); + subscriptionGroupConfig.setConsumeBroadcastEnable(true); +- this.subscriptionGroupTable.put(MixAll.CID_ONSAPI_OWNER_GROUP, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + + { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(MixAll.CID_SYS_RMQ_TRANS); + subscriptionGroupConfig.setConsumeBroadcastEnable(true); +- this.subscriptionGroupTable.put(MixAll.CID_SYS_RMQ_TRANS, subscriptionGroupConfig); ++ putSubscriptionGroupConfig(subscriptionGroupConfig); + } + } + ++ protected SubscriptionGroupConfig putSubscriptionGroupConfig(SubscriptionGroupConfig subscriptionGroupConfig) { ++ return this.subscriptionGroupTable.put(subscriptionGroupConfig.getGroupName(), subscriptionGroupConfig); ++ } ++ ++ protected SubscriptionGroupConfig putSubscriptionGroupConfigIfAbsent(SubscriptionGroupConfig subscriptionGroupConfig) { ++ return this.subscriptionGroupTable.putIfAbsent(subscriptionGroupConfig.getGroupName(), subscriptionGroupConfig); ++ } ++ ++ protected SubscriptionGroupConfig getSubscriptionGroupConfig(String groupName) { ++ return this.subscriptionGroupTable.get(groupName); ++ } ++ ++ protected SubscriptionGroupConfig removeSubscriptionGroupConfig(String groupName) { ++ return this.subscriptionGroupTable.remove(groupName); ++ } ++ + public void updateSubscriptionGroupConfig(final SubscriptionGroupConfig config) { + Map newAttributes = request(config); + Map currentAttributes = current(config.getGroupName()); +@@ -127,7 +149,7 @@ public class SubscriptionGroupManager extends ConfigManager { + + config.setAttributes(finalAttributes); + +- SubscriptionGroupConfig old = this.subscriptionGroupTable.put(config.getGroupName(), config); ++ SubscriptionGroupConfig old = putSubscriptionGroupConfig(config); + if (old != null) { + log.info("update subscription group config, old: {} new: {}", old, config); + } else { +@@ -218,7 +240,7 @@ public class SubscriptionGroupManager extends ConfigManager { + } + + public void disableConsume(final String groupName) { +- SubscriptionGroupConfig old = this.subscriptionGroupTable.get(groupName); ++ SubscriptionGroupConfig old = getSubscriptionGroupConfig(groupName); + if (old != null) { + old.setConsumeEnable(false); + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; +@@ -227,7 +249,7 @@ public class SubscriptionGroupManager extends ConfigManager { + } + + public SubscriptionGroupConfig findSubscriptionGroupConfig(final String group) { +- SubscriptionGroupConfig subscriptionGroupConfig = this.subscriptionGroupTable.get(group); ++ SubscriptionGroupConfig subscriptionGroupConfig = getSubscriptionGroupConfig(group); + if (null == subscriptionGroupConfig) { + if (brokerController.getBrokerConfig().isAutoCreateSubscriptionGroup() || MixAll.isSysConsumerGroup(group)) { + if (group.length() > Validators.CHARACTER_MAX_LENGTH || TopicValidator.isTopicOrGroupIllegal(group)) { +@@ -235,7 +257,7 @@ public class SubscriptionGroupManager extends ConfigManager { + } + subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(group); +- SubscriptionGroupConfig preConfig = this.subscriptionGroupTable.putIfAbsent(group, subscriptionGroupConfig); ++ SubscriptionGroupConfig preConfig = putSubscriptionGroupConfigIfAbsent(subscriptionGroupConfig); + if (null == preConfig) { + log.info("auto create a subscription group, {}", subscriptionGroupConfig.toString()); + } +@@ -305,7 +327,7 @@ public class SubscriptionGroupManager extends ConfigManager { + } + + public void deleteSubscriptionGroupConfig(final String groupName) { +- SubscriptionGroupConfig old = this.subscriptionGroupTable.remove(groupName); ++ SubscriptionGroupConfig old = removeSubscriptionGroupConfig(groupName); + this.forbiddenTable.remove(groupName); + if (old != null) { + log.info("delete subscription group OK, subscription group:{}", old); +@@ -317,8 +339,12 @@ public class SubscriptionGroupManager extends ConfigManager { + } + } + ++ + public void setSubscriptionGroupTable(ConcurrentMap subscriptionGroupTable) { +- this.subscriptionGroupTable = subscriptionGroupTable; ++ this.subscriptionGroupTable.clear(); ++ for (String key : subscriptionGroupTable.keySet()) { ++ putSubscriptionGroupConfig(subscriptionGroupTable.get(key)); ++ } + } + + public boolean containsSubscriptionGroup(String group) { +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBLmqTopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBLmqTopicConfigManager.java +new file mode 100644 +index 000000000..d049a8dbc +--- /dev/null ++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBLmqTopicConfigManager.java +@@ -0,0 +1,57 @@ ++/* ++ * 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.rocketmq.broker.topic; ++ ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.MixAll; ++import org.apache.rocketmq.common.TopicConfig; ++import org.apache.rocketmq.common.constant.PermName; ++ ++public class RocksDBLmqTopicConfigManager extends RocksDBTopicConfigManager { ++ ++ public RocksDBLmqTopicConfigManager(BrokerController brokerController) { ++ super(brokerController); ++ } ++ ++ @Override ++ public TopicConfig selectTopicConfig(final String topic) { ++ if (MixAll.isLmq(topic)) { ++ return simpleLmqTopicConfig(topic); ++ } ++ return super.selectTopicConfig(topic); ++ } ++ ++ @Override ++ public void updateTopicConfig(final TopicConfig topicConfig) { ++ if (topicConfig == null || MixAll.isLmq(topicConfig.getTopicName())) { ++ return; ++ } ++ super.updateTopicConfig(topicConfig); ++ } ++ ++ @Override ++ public boolean containsTopic(String topic) { ++ if (MixAll.isLmq(topic)) { ++ return true; ++ } ++ return super.containsTopic(topic); ++ } ++ ++ private TopicConfig simpleLmqTopicConfig(String topic) { ++ return new TopicConfig(topic, 1, 1, PermName.PERM_READ | PermName.PERM_WRITE); ++ } ++} +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java +new file mode 100644 +index 000000000..7da0d7c8a +--- /dev/null ++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java +@@ -0,0 +1,95 @@ ++/* ++ * 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.rocketmq.broker.topic; ++ ++import java.io.File; ++ ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.TopicConfig; ++import org.apache.rocketmq.common.config.RocksDBConfigManager; ++import org.apache.rocketmq.common.utils.DataConverter; ++ ++import com.alibaba.fastjson.JSON; ++import com.alibaba.fastjson.serializer.SerializerFeature; ++ ++public class RocksDBTopicConfigManager extends TopicConfigManager { ++ ++ public RocksDBTopicConfigManager(BrokerController brokerController) { ++ super(brokerController, false); ++ this.rocksDBConfigManager = new RocksDBConfigManager(this.brokerController.getMessageStoreConfig().getMemTableFlushInterval()); ++ } ++ ++ @Override ++ public boolean load() { ++ if (!this.rocksDBConfigManager.load(configFilePath(), this::decode0)) { ++ return false; ++ } ++ this.init(); ++ return true; ++ } ++ ++ @Override ++ public boolean stop() { ++ return this.rocksDBConfigManager.stop(); ++ } ++ ++ @Override ++ protected void decode0(byte[] key, byte[] body) { ++ String topicName = new String(key, DataConverter.charset); ++ TopicConfig topicConfig = JSON.parseObject(body, TopicConfig.class); ++ ++ this.topicConfigTable.put(topicName, topicConfig); ++ log.info("load exist local topic, {}", topicConfig.toString()); ++ } ++ ++ @Override ++ public String configFilePath() { ++ return this.brokerController.getMessageStoreConfig().getStorePathRootDir() + File.separator + "config" + File.separator + "topics" + File.separator; ++ } ++ ++ @Override ++ protected TopicConfig putTopicConfig(TopicConfig topicConfig) { ++ String topicName = topicConfig.getTopicName(); ++ TopicConfig oldTopicConfig = this.topicConfigTable.put(topicName, topicConfig); ++ try { ++ byte[] keyBytes = topicName.getBytes(DataConverter.charset); ++ byte[] valueBytes = JSON.toJSONBytes(topicConfig, SerializerFeature.BrowserCompatible); ++ this.rocksDBConfigManager.put(keyBytes, keyBytes.length, valueBytes); ++ } catch (Exception e) { ++ log.error("kv put topic Failed, {}", topicConfig.toString(), e); ++ } ++ return oldTopicConfig; ++ } ++ ++ @Override ++ protected TopicConfig removeTopicConfig(String topicName) { ++ TopicConfig topicConfig = this.topicConfigTable.remove(topicName); ++ try { ++ this.rocksDBConfigManager.delete(topicName.getBytes(DataConverter.charset)); ++ } catch (Exception e) { ++ log.error("kv remove topic Failed, {}", topicConfig.toString()); ++ } ++ return topicConfig; ++ } ++ ++ @Override ++ public synchronized void persist() { ++ if (brokerController.getMessageStoreConfig().isRealTimePersistRocksDBConfig()) { ++ this.rocksDBConfigManager.flushWAL(); ++ } ++ } ++} +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java +index e90530512..1c3b9711f 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java +@@ -16,7 +16,6 @@ + */ + package org.apache.rocketmq.broker.topic; + +-import com.google.common.collect.ImmutableMap; + import java.util.HashMap; + import java.util.Iterator; + import java.util.Map; +@@ -27,6 +26,9 @@ import java.util.concurrent.ConcurrentMap; + import java.util.concurrent.TimeUnit; + import java.util.concurrent.locks.Lock; + import java.util.concurrent.locks.ReentrantLock; ++ ++import com.google.common.collect.ImmutableMap; ++ + import org.apache.commons.lang3.StringUtils; + import org.apache.rocketmq.broker.BrokerController; + import org.apache.rocketmq.broker.BrokerPathConfigHelper; +@@ -50,27 +52,38 @@ import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper; + import static com.google.common.base.Preconditions.checkNotNull; + + public class TopicConfigManager extends ConfigManager { +- private static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); ++ protected static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); + private static final long LOCK_TIMEOUT_MILLIS = 3000; + private static final int SCHEDULE_TOPIC_QUEUE_NUM = 18; + + private transient final Lock topicConfigTableLock = new ReentrantLock(); +- private ConcurrentMap topicConfigTable = new ConcurrentHashMap<>(1024); ++ protected ConcurrentMap topicConfigTable = new ConcurrentHashMap<>(1024); + private DataVersion dataVersion = new DataVersion(); +- private transient BrokerController brokerController; ++ protected transient BrokerController brokerController; + + public TopicConfigManager() { ++ + } + + public TopicConfigManager(BrokerController brokerController) { ++ this(brokerController, true); ++ } ++ ++ public TopicConfigManager(BrokerController brokerController, boolean init) { + this.brokerController = brokerController; ++ if (init) { ++ init(); ++ } ++ } ++ ++ protected void init() { + { + String topic = TopicValidator.RMQ_SYS_SELF_TEST_TOPIC; + TopicConfig topicConfig = new TopicConfig(topic); + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + if (this.brokerController.getBrokerConfig().isAutoCreateTopicEnable()) { +@@ -83,7 +96,7 @@ public class TopicConfigManager extends ConfigManager { + .getDefaultTopicQueueNums()); + int perm = PermName.PERM_INHERIT | PermName.PERM_READ | PermName.PERM_WRITE; + topicConfig.setPerm(perm); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + } + { +@@ -92,7 +105,7 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1024); + topicConfig.setWriteQueueNums(1024); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + String topic = this.brokerController.getBrokerConfig().getBrokerClusterName(); +@@ -103,7 +116,7 @@ public class TopicConfigManager extends ConfigManager { + perm |= PermName.PERM_READ | PermName.PERM_WRITE; + } + topicConfig.setPerm(perm); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + +@@ -117,7 +130,7 @@ public class TopicConfigManager extends ConfigManager { + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); + topicConfig.setPerm(perm); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + String topic = TopicValidator.RMQ_SYS_OFFSET_MOVED_EVENT; +@@ -125,7 +138,7 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + String topic = TopicValidator.RMQ_SYS_SCHEDULE_TOPIC; +@@ -133,7 +146,7 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(SCHEDULE_TOPIC_QUEUE_NUM); + topicConfig.setWriteQueueNums(SCHEDULE_TOPIC_QUEUE_NUM); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + if (this.brokerController.getBrokerConfig().isTraceTopicEnable()) { +@@ -142,7 +155,7 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + } + { +@@ -151,7 +164,7 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + // PopAckConstants.REVIVE_TOPIC +@@ -160,7 +173,7 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(this.brokerController.getBrokerConfig().getReviveQueueNum()); + topicConfig.setWriteQueueNums(this.brokerController.getBrokerConfig().getReviveQueueNum()); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + // sync broker member group topic +@@ -170,7 +183,7 @@ public class TopicConfigManager extends ConfigManager { + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); + topicConfig.setPerm(PermName.PERM_INHERIT); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + { + // TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC +@@ -179,7 +192,7 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + + { +@@ -189,12 +202,24 @@ public class TopicConfigManager extends ConfigManager { + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + } + } + ++ protected TopicConfig putTopicConfig(TopicConfig topicConfig) { ++ return this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ } ++ ++ protected TopicConfig getTopicConfig(String topicName) { ++ return this.topicConfigTable.get(topicName); ++ } ++ ++ protected TopicConfig removeTopicConfig(String topicName) { ++ return this.topicConfigTable.remove(topicName); ++ } ++ + public TopicConfig selectTopicConfig(final String topic) { +- return this.topicConfigTable.get(topic); ++ return getTopicConfig(topic); + } + + public TopicConfig createTopicInSendMessageMethod(final String topic, final String defaultTopic, +@@ -205,12 +230,12 @@ public class TopicConfigManager extends ConfigManager { + try { + if (this.topicConfigTableLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) { + try { +- topicConfig = this.topicConfigTable.get(topic); ++ topicConfig = getTopicConfig(topic); + if (topicConfig != null) { + return topicConfig; + } + +- TopicConfig defaultTopicConfig = this.topicConfigTable.get(defaultTopic); ++ TopicConfig defaultTopicConfig = getTopicConfig(defaultTopic); + if (defaultTopicConfig != null) { + if (defaultTopic.equals(TopicValidator.AUTO_CREATE_TOPIC_KEY_TOPIC)) { + if (!this.brokerController.getBrokerConfig().isAutoCreateTopicEnable()) { +@@ -247,7 +272,7 @@ public class TopicConfigManager extends ConfigManager { + log.info("Create new topic by default topic:[{}] config:[{}] producer:[{}]", + defaultTopic, topicConfig, remoteAddress); + +- this.topicConfigTable.put(topic, topicConfig); ++ putTopicConfig(topicConfig); + + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; + dataVersion.nextVersion(stateMachineVersion); +@@ -287,12 +312,12 @@ public class TopicConfigManager extends ConfigManager { + try { + if (this.topicConfigTableLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) { + try { +- TopicConfig existedTopicConfig = this.topicConfigTable.get(topicConfig.getTopicName()); ++ TopicConfig existedTopicConfig = getTopicConfig(topicConfig.getTopicName()); + if (existedTopicConfig != null) { + return existedTopicConfig; + } + log.info("Create new topic [{}] config:[{}]", topicConfig.getTopicName(), topicConfig); +- this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ putTopicConfig(topicConfig); + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; + dataVersion.nextVersion(stateMachineVersion); + createNew = true; +@@ -305,13 +330,9 @@ public class TopicConfigManager extends ConfigManager { + log.error("createTopicIfAbsent ", e); + } + if (createNew && register) { +- if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) { +- this.brokerController.registerSingleTopicAll(topicConfig); +- } else { +- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion); +- } ++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion); + } +- return this.topicConfigTable.get(topicConfig.getTopicName()); ++ return getTopicConfig(topicConfig.getTopicName()); + } + + public TopicConfig createTopicInSendMessageBackMethod( +@@ -328,7 +349,7 @@ public class TopicConfigManager extends ConfigManager { + final int perm, + final boolean isOrder, + final int topicSysFlag) { +- TopicConfig topicConfig = this.topicConfigTable.get(topic); ++ TopicConfig topicConfig = getTopicConfig(topic); + if (topicConfig != null) { + if (isOrder != topicConfig.isOrder()) { + topicConfig.setOrder(isOrder); +@@ -342,7 +363,7 @@ public class TopicConfigManager extends ConfigManager { + try { + if (this.topicConfigTableLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) { + try { +- topicConfig = this.topicConfigTable.get(topic); ++ topicConfig = getTopicConfig(topic); + if (topicConfig != null) { + return topicConfig; + } +@@ -355,7 +376,7 @@ public class TopicConfigManager extends ConfigManager { + topicConfig.setOrder(isOrder); + + log.info("create new topic {}", topicConfig); +- this.topicConfigTable.put(topic, topicConfig); ++ putTopicConfig(topicConfig); + createNew = true; + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; + dataVersion.nextVersion(stateMachineVersion); +@@ -376,7 +397,7 @@ public class TopicConfigManager extends ConfigManager { + } + + public TopicConfig createTopicOfTranCheckMaxTime(final int clientDefaultTopicQueueNums, final int perm) { +- TopicConfig topicConfig = this.topicConfigTable.get(TopicValidator.RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC); ++ TopicConfig topicConfig = getTopicConfig(TopicValidator.RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC); + if (topicConfig != null) + return topicConfig; + +@@ -385,7 +406,7 @@ public class TopicConfigManager extends ConfigManager { + try { + if (this.topicConfigTableLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) { + try { +- topicConfig = this.topicConfigTable.get(TopicValidator.RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC); ++ topicConfig = getTopicConfig(TopicValidator.RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC); + if (topicConfig != null) + return topicConfig; + +@@ -396,7 +417,7 @@ public class TopicConfigManager extends ConfigManager { + topicConfig.setTopicSysFlag(0); + + log.info("create new topic {}", topicConfig); +- this.topicConfigTable.put(TopicValidator.RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC, topicConfig); ++ putTopicConfig(topicConfig); + createNew = true; + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; + dataVersion.nextVersion(stateMachineVersion); +@@ -418,7 +439,7 @@ public class TopicConfigManager extends ConfigManager { + + public void updateTopicUnitFlag(final String topic, final boolean unit) { + +- TopicConfig topicConfig = this.topicConfigTable.get(topic); ++ TopicConfig topicConfig = getTopicConfig(topic); + if (topicConfig != null) { + int oldTopicSysFlag = topicConfig.getTopicSysFlag(); + if (unit) { +@@ -430,7 +451,7 @@ public class TopicConfigManager extends ConfigManager { + log.info("update topic sys flag. oldTopicSysFlag={}, newTopicSysFlag={}", oldTopicSysFlag, + topicConfig.getTopicSysFlag()); + +- this.topicConfigTable.put(topic, topicConfig); ++ putTopicConfig(topicConfig); + + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; + dataVersion.nextVersion(stateMachineVersion); +@@ -441,7 +462,7 @@ public class TopicConfigManager extends ConfigManager { + } + + public void updateTopicUnitSubFlag(final String topic, final boolean hasUnitSub) { +- TopicConfig topicConfig = this.topicConfigTable.get(topic); ++ TopicConfig topicConfig = getTopicConfig(topic); + if (topicConfig != null) { + int oldTopicSysFlag = topicConfig.getTopicSysFlag(); + if (hasUnitSub) { +@@ -453,7 +474,7 @@ public class TopicConfigManager extends ConfigManager { + log.info("update topic sys flag. oldTopicSysFlag={}, newTopicSysFlag={}", oldTopicSysFlag, + topicConfig.getTopicSysFlag()); + +- this.topicConfigTable.put(topic, topicConfig); ++ putTopicConfig(topicConfig); + + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; + dataVersion.nextVersion(stateMachineVersion); +@@ -469,6 +490,7 @@ public class TopicConfigManager extends ConfigManager { + Map newAttributes = request(topicConfig); + Map currentAttributes = current(topicConfig.getTopicName()); + ++ + Map finalAttributes = AttributeUtil.alterCurrentAttributes( + this.topicConfigTable.get(topicConfig.getTopicName()) == null, + TopicAttributes.ALL, +@@ -477,7 +499,7 @@ public class TopicConfigManager extends ConfigManager { + + topicConfig.setAttributes(finalAttributes); + +- TopicConfig old = this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); ++ TopicConfig old = putTopicConfig(topicConfig); + if (old != null) { + log.info("update topic config, old:[{}] new:[{}]", old, topicConfig); + } else { +@@ -496,7 +518,7 @@ public class TopicConfigManager extends ConfigManager { + boolean isChange = false; + Set orderTopics = orderKVTableFromNs.getTable().keySet(); + for (String topic : orderTopics) { +- TopicConfig topicConfig = this.topicConfigTable.get(topic); ++ TopicConfig topicConfig = getTopicConfig(topic); + if (topicConfig != null && !topicConfig.isOrder()) { + topicConfig.setOrder(true); + isChange = true; +@@ -534,7 +556,7 @@ public class TopicConfigManager extends ConfigManager { + } + + public boolean isOrderTopic(final String topic) { +- TopicConfig topicConfig = this.topicConfigTable.get(topic); ++ TopicConfig topicConfig = getTopicConfig(topic); + if (topicConfig == null) { + return false; + } else { +@@ -543,7 +565,7 @@ public class TopicConfigManager extends ConfigManager { + } + + public void deleteTopicConfig(final String topic) { +- TopicConfig old = this.topicConfigTable.remove(topic); ++ TopicConfig old = removeTopicConfig(topic); + if (old != null) { + log.info("delete topic config OK, topic: {}", old); + long stateMachineVersion = brokerController.getMessageStore() != null ? brokerController.getMessageStore().getStateMachineVersion() : 0; +@@ -619,7 +641,7 @@ public class TopicConfigManager extends ConfigManager { + } + + private Map current(String topic) { +- TopicConfig topicConfig = this.topicConfigTable.get(topic); ++ TopicConfig topicConfig = getTopicConfig(topic); + if (topicConfig == null) { + return new HashMap<>(); + } else { +diff --git a/broker/src/main/resources/rmq.broker.logback.xml b/broker/src/main/resources/rmq.broker.logback.xml +index 78b1aea41..7d49f6664 100644 +--- a/broker/src/main/resources/rmq.broker.logback.xml ++++ b/broker/src/main/resources/rmq.broker.logback.xml +@@ -145,6 +145,39 @@ + + + ++ ++ ++ brokerContainerLogDir ++ ${file.separator} ++ ++ ++ ++ ++ ${user.home}${file.separator}logs${file.separator}rocketmqlogs${brokerLogDir:-${file.separator}}${brokerContainerLogDir}${file.separator}rocksdb.log ++ ++ true ++ ++ ++ ${user.home}${file.separator}logs${file.separator}rocketmqlogs${brokerLogDir:-${file.separator}}${brokerContainerLogDir}${file.separator}otherdays${file.separator}rocksdb.%i.log.gz ++ ++ 1 ++ 10 ++ ++ ++ 128MB ++ ++ ++ %d{yyy-MM-dd HH:mm:ss,GMT+8} %p %t - %m%n ++ UTF-8 ++ ++ ++ ++ ++ ++ ++ ++ + + + brokerContainerLogDir +@@ -579,6 +612,10 @@ + + + ++ ++ ++ ++ + + + +diff --git a/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java +new file mode 100644 +index 000000000..58b690c9a +--- /dev/null ++++ b/broker/src/test/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManagerTest.java +@@ -0,0 +1,113 @@ ++/* ++ * 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.rocketmq.broker.offset; ++ ++import java.util.concurrent.ConcurrentHashMap; ++import java.util.concurrent.ConcurrentMap; ++ ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.MixAll; ++import org.apache.rocketmq.store.config.MessageStoreConfig; ++import org.junit.After; ++import org.junit.Assert; ++import org.junit.Before; ++import org.junit.Test; ++import org.mockito.Mockito; ++ ++import static org.assertj.core.api.Assertions.assertThat; ++ ++public class RocksDBConsumerOffsetManagerTest { ++ ++ private static final String KEY = "FooBar@FooBarGroup"; ++ ++ private BrokerController brokerController; ++ ++ private ConsumerOffsetManager consumerOffsetManager; ++ ++ @Before ++ public void init() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ brokerController = Mockito.mock(BrokerController.class); ++ MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); ++ Mockito.when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); ++ ++ consumerOffsetManager = new RocksDBConsumerOffsetManager(brokerController); ++ consumerOffsetManager.load(); ++ ++ ConcurrentHashMap> offsetTable = new ConcurrentHashMap<>(512); ++ offsetTable.put(KEY,new ConcurrentHashMap() {{ ++ put(1,2L); ++ put(2,3L); ++ }}); ++ consumerOffsetManager.setOffsetTable(offsetTable); ++ } ++ ++ @After ++ public void destroy() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ if (consumerOffsetManager != null) { ++ consumerOffsetManager.stop(); ++ } ++ } ++ ++ @Test ++ public void cleanOffsetByTopic_NotExist() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ consumerOffsetManager.cleanOffsetByTopic("InvalidTopic"); ++ assertThat(consumerOffsetManager.getOffsetTable().containsKey(KEY)).isTrue(); ++ } ++ ++ @Test ++ public void cleanOffsetByTopic_Exist() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ consumerOffsetManager.cleanOffsetByTopic("FooBar"); ++ assertThat(!consumerOffsetManager.getOffsetTable().containsKey(KEY)).isTrue(); ++ } ++ ++ @Test ++ public void testOffsetPersistInMemory() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ ConcurrentMap> offsetTable = consumerOffsetManager.getOffsetTable(); ++ ConcurrentMap table = new ConcurrentHashMap<>(); ++ table.put(0, 1L); ++ table.put(1, 3L); ++ String group = "G1"; ++ offsetTable.put(group, table); ++ ++ consumerOffsetManager.persist(); ++ consumerOffsetManager.stop(); ++ consumerOffsetManager.load(); ++ ++ ConcurrentMap offsetTableLoaded = consumerOffsetManager.getOffsetTable().get(group); ++ Assert.assertEquals(table, offsetTableLoaded); ++ } ++ ++ private boolean notToBeExecuted() { ++ return MixAll.isMac(); ++ } ++} +diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java +index a470c0cf2..d33a217f7 100644 +--- a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java ++++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java +@@ -36,6 +36,8 @@ import org.apache.rocketmq.broker.client.ConsumerGroupInfo; + import org.apache.rocketmq.broker.client.ConsumerManager; + import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; + import org.apache.rocketmq.broker.schedule.ScheduleMessageService; ++import org.apache.rocketmq.broker.subscription.RocksDBSubscriptionGroupManager; ++import org.apache.rocketmq.broker.topic.RocksDBTopicConfigManager; + import org.apache.rocketmq.broker.topic.TopicConfigManager; + import org.apache.rocketmq.common.BoundaryType; + import org.apache.rocketmq.common.BrokerConfig; +@@ -76,6 +78,7 @@ import org.apache.rocketmq.store.config.BrokerRole; + import org.apache.rocketmq.store.config.MessageStoreConfig; + import org.apache.rocketmq.store.logfile.DefaultMappedFile; + import org.apache.rocketmq.store.stats.BrokerStats; ++import org.junit.After; + import org.junit.Before; + import org.junit.Test; + import org.junit.runner.RunWith; +@@ -114,7 +117,7 @@ public class AdminBrokerProcessorTest { + private SendMessageProcessor sendMessageProcessor; + + @Mock +- private ConcurrentMap inFlyWritingCouterMap; ++ private ConcurrentMap inFlyWritingCounterMap; + + private Set systemTopicSet; + private String topic; +@@ -162,6 +165,40 @@ public class AdminBrokerProcessorTest { + brokerController.getMessageStoreConfig().setTimerWheelEnable(false); + } + ++ @After ++ public void destroy() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ if (brokerController.getSubscriptionGroupManager() != null) { ++ brokerController.getSubscriptionGroupManager().stop(); ++ } ++ if (brokerController.getTopicConfigManager() != null) { ++ brokerController.getTopicConfigManager().stop(); ++ } ++ if (brokerController.getConsumerOffsetManager() != null) { ++ brokerController.getConsumerOffsetManager().stop(); ++ } ++ } ++ ++ private void initRocksdbTopicManager() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ RocksDBTopicConfigManager rocksDBTopicConfigManager = new RocksDBTopicConfigManager(brokerController); ++ brokerController.setTopicConfigManager(rocksDBTopicConfigManager); ++ rocksDBTopicConfigManager.load(); ++ } ++ ++ private void initRocksdbSubscriptionManager() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ RocksDBSubscriptionGroupManager rocksDBSubscriptionGroupManager = new RocksDBSubscriptionGroupManager(brokerController); ++ brokerController.setSubscriptionGroupManager(rocksDBSubscriptionGroupManager); ++ rocksDBSubscriptionGroupManager.load(); ++ } ++ + @Test + public void testProcessRequest_success() throws RemotingCommandException, UnknownHostException { + RemotingCommand request = createUpdateBrokerConfigCommand(); +@@ -177,6 +214,15 @@ public class AdminBrokerProcessorTest { + assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR); + } + ++ @Test ++ public void testUpdateAndCreateTopicInRocksdb() throws Exception { ++ if (notToBeExecuted()) { ++ return; ++ } ++ initRocksdbTopicManager(); ++ testUpdateAndCreateTopic(); ++ } ++ + @Test + public void testUpdateAndCreateTopic() throws Exception { + //test system topic +@@ -197,7 +243,15 @@ public class AdminBrokerProcessorTest { + request = buildCreateTopicRequest(topic); + response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); ++ } + ++ @Test ++ public void testUpdateAndCreateTopicOnSlaveInRocksdb() throws Exception { ++ if (notToBeExecuted()) { ++ return; ++ } ++ initRocksdbTopicManager(); ++ testUpdateAndCreateTopicOnSlave(); + } + + @Test +@@ -217,6 +271,15 @@ public class AdminBrokerProcessorTest { + "please execute it from master broker."); + } + ++ @Test ++ public void testDeleteTopicInRocksdb() throws Exception { ++ if (notToBeExecuted()) { ++ return; ++ } ++ initRocksdbTopicManager(); ++ testDeleteTopic(); ++ } ++ + @Test + public void testDeleteTopic() throws Exception { + //test system topic +@@ -233,6 +296,15 @@ public class AdminBrokerProcessorTest { + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + ++ @Test ++ public void testDeleteTopicOnSlaveInRocksdb() throws Exception { ++ if (notToBeExecuted()) { ++ return; ++ } ++ initRocksdbTopicManager(); ++ testDeleteTopicOnSlave(); ++ } ++ + @Test + public void testDeleteTopicOnSlave() throws Exception { + // setup +@@ -249,6 +321,15 @@ public class AdminBrokerProcessorTest { + "please execute it from master broker."); + } + ++ @Test ++ public void testGetAllTopicConfigInRocksdb() throws Exception { ++ if (notToBeExecuted()) { ++ return; ++ } ++ initRocksdbTopicManager(); ++ testGetAllTopicConfig(); ++ } ++ + @Test + public void testGetAllTopicConfig() throws Exception { + GetAllTopicConfigResponseHeader getAllTopicConfigResponseHeader = new GetAllTopicConfigResponseHeader(); +@@ -400,6 +481,12 @@ public class AdminBrokerProcessorTest { + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + ++ @Test ++ public void testUpdateAndCreateSubscriptionGroupInRocksdb() throws Exception { ++ initRocksdbSubscriptionManager(); ++ testUpdateAndCreateSubscriptionGroup(); ++ } ++ + @Test + public void testUpdateAndCreateSubscriptionGroup() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_SUBSCRIPTIONGROUP, null); +@@ -415,6 +502,12 @@ public class AdminBrokerProcessorTest { + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + ++ @Test ++ public void testUpdateAndCreateSubscriptionGroupOnSlaveInRocksdb() throws Exception { ++ initRocksdbSubscriptionManager(); ++ testUpdateAndCreateSubscriptionGroupOnSlave(); ++ } ++ + @Test + public void testUpdateAndCreateSubscriptionGroupOnSlave() throws RemotingCommandException { + // Setup +@@ -439,6 +532,12 @@ public class AdminBrokerProcessorTest { + "please execute it from master broker."); + } + ++ @Test ++ public void testGetAllSubscriptionGroupInRocksdb() throws Exception { ++ initRocksdbSubscriptionManager(); ++ testGetAllSubscriptionGroup(); ++ } ++ + @Test + public void testGetAllSubscriptionGroup() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_SUBSCRIPTIONGROUP_CONFIG, null); +@@ -446,6 +545,12 @@ public class AdminBrokerProcessorTest { + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + ++ @Test ++ public void testDeleteSubscriptionGroupInRocksdb() throws Exception { ++ initRocksdbSubscriptionManager(); ++ testDeleteSubscriptionGroup(); ++ } ++ + @Test + public void testDeleteSubscriptionGroup() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_SUBSCRIPTIONGROUP, null); +@@ -455,6 +560,12 @@ public class AdminBrokerProcessorTest { + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + ++ @Test ++ public void testDeleteSubscriptionGroupOnSlaveInRocksdb() throws Exception { ++ initRocksdbSubscriptionManager(); ++ testDeleteSubscriptionGroupOnSlave(); ++ } ++ + @Test + public void testDeleteSubscriptionGroupOnSlave() throws RemotingCommandException { + // Setup +@@ -547,6 +658,15 @@ public class AdminBrokerProcessorTest { + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + ++ @Test ++ public void testGetTopicConfigInRocksdb() throws Exception { ++ if (notToBeExecuted()) { ++ return; ++ } ++ initRocksdbTopicManager(); ++ testGetTopicConfig(); ++ } ++ + @Test + public void testGetTopicConfig() throws Exception { + String topic = "foobar"; +@@ -630,4 +750,8 @@ public class AdminBrokerProcessorTest { + request.makeCustomHeaderToNet(); + return request; + } ++ ++ private boolean notToBeExecuted() { ++ return MixAll.isMac(); ++ } + } +diff --git a/broker/src/test/java/org/apache/rocketmq/broker/substription/ForbiddenTest.java b/broker/src/test/java/org/apache/rocketmq/broker/subscription/ForbiddenTest.java +similarity index 95% +rename from broker/src/test/java/org/apache/rocketmq/broker/substription/ForbiddenTest.java +rename to broker/src/test/java/org/apache/rocketmq/broker/subscription/ForbiddenTest.java +index 2ac5ee320..bdaee3b3c 100644 +--- a/broker/src/test/java/org/apache/rocketmq/broker/substription/ForbiddenTest.java ++++ b/broker/src/test/java/org/apache/rocketmq/broker/subscription/ForbiddenTest.java +@@ -15,12 +15,11 @@ + * limitations under the License. + */ + +-package org.apache.rocketmq.broker.substription; ++package org.apache.rocketmq.broker.subscription; + + import static org.junit.Assert.assertEquals; + + import org.apache.rocketmq.broker.BrokerController; +-import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; + import org.apache.rocketmq.common.BrokerConfig; + import org.apache.rocketmq.remoting.netty.NettyClientConfig; + import org.apache.rocketmq.remoting.netty.NettyServerConfig; +diff --git a/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java +index 6337c69ea..3c829437c 100644 +--- a/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java ++++ b/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java +@@ -20,9 +20,12 @@ package org.apache.rocketmq.broker.subscription; + import com.google.common.collect.ImmutableMap; + import java.util.Map; + import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.MixAll; + import org.apache.rocketmq.common.SubscriptionGroupAttributes; + import org.apache.rocketmq.common.attribute.BooleanAttribute; + import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; ++import org.apache.rocketmq.store.config.MessageStoreConfig; ++import org.junit.After; + import org.junit.Before; + import org.junit.Test; + import org.junit.runner.RunWith; +@@ -54,6 +57,28 @@ public class SubscriptionGroupManagerTest { + doNothing().when(subscriptionGroupManager).persist(); + } + ++ @After ++ public void destroy() { ++ if (MixAll.isMac()) { ++ return; ++ } ++ if (subscriptionGroupManager != null) { ++ subscriptionGroupManager.stop(); ++ } ++ } ++ ++ @Test ++ public void testUpdateAndCreateSubscriptionGroupInRocksdb() { ++ if (MixAll.isMac()) { ++ return; ++ } ++ when(brokerControllerMock.getMessageStoreConfig()).thenReturn(new MessageStoreConfig()); ++ subscriptionGroupManager = spy(new RocksDBSubscriptionGroupManager(brokerControllerMock)); ++ subscriptionGroupManager.load(); ++ group += System.currentTimeMillis(); ++ updateSubscriptionGroupConfig(); ++ } ++ + @Test + public void updateSubscriptionGroupConfig() { + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); +diff --git a/broker/src/test/java/org/apache/rocketmq/broker/topic/RocksdbTopicConfigManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/topic/RocksdbTopicConfigManagerTest.java +new file mode 100644 +index 000000000..ed71a3313 +--- /dev/null ++++ b/broker/src/test/java/org/apache/rocketmq/broker/topic/RocksdbTopicConfigManagerTest.java +@@ -0,0 +1,375 @@ ++/* ++ * 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.rocketmq.broker.topic; ++ ++import java.util.HashMap; ++import java.util.List; ++import java.util.Map; ++import java.util.Optional; ++import org.apache.rocketmq.broker.BrokerController; ++import org.apache.rocketmq.common.BrokerConfig; ++import org.apache.rocketmq.common.MixAll; ++import org.apache.rocketmq.common.TopicAttributes; ++import org.apache.rocketmq.common.TopicConfig; ++import org.apache.rocketmq.common.attribute.Attribute; ++import org.apache.rocketmq.common.attribute.BooleanAttribute; ++import org.apache.rocketmq.common.attribute.CQType; ++import org.apache.rocketmq.common.attribute.EnumAttribute; ++import org.apache.rocketmq.common.attribute.LongRangeAttribute; ++import org.apache.rocketmq.common.utils.QueueTypeUtils; ++import org.apache.rocketmq.store.DefaultMessageStore; ++import org.apache.rocketmq.store.config.MessageStoreConfig; ++import org.junit.After; ++import org.junit.Assert; ++import org.junit.Before; ++import org.junit.Test; ++import org.junit.runner.RunWith; ++import org.mockito.Mock; ++import org.mockito.junit.MockitoJUnitRunner; ++ ++import static com.google.common.collect.Sets.newHashSet; ++import static java.util.Arrays.asList; ++import static org.mockito.Mockito.when; ++ ++@RunWith(MockitoJUnitRunner.class) ++public class RocksdbTopicConfigManagerTest { ++ private RocksDBTopicConfigManager topicConfigManager; ++ @Mock ++ private BrokerController brokerController; ++ ++ @Mock ++ private DefaultMessageStore defaultMessageStore; ++ ++ @Before ++ public void init() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ BrokerConfig brokerConfig = new BrokerConfig(); ++ when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); ++ MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); ++ when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); ++ when(brokerController.getMessageStore()).thenReturn(defaultMessageStore); ++ when(defaultMessageStore.getStateMachineVersion()).thenReturn(0L); ++ topicConfigManager = new RocksDBTopicConfigManager(brokerController); ++ topicConfigManager.load(); ++ } ++ ++ @After ++ public void destroy() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ if (topicConfigManager != null) { ++ topicConfigManager.stop(); ++ } ++ } ++ ++ @Test ++ public void testAddUnsupportedKeyOnCreating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String unsupportedKey = "key4"; ++ String topicName = "testAddUnsupportedKeyOnCreating-" + System.currentTimeMillis(); ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+enum.key", "enum-2"); ++ attributes.put("+" + unsupportedKey, "value1"); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topicName); ++ topicConfig.setAttributes(attributes); ++ ++ RuntimeException runtimeException = Assert.assertThrows(RuntimeException.class, () -> topicConfigManager.updateTopicConfig(topicConfig)); ++ Assert.assertEquals("unsupported key: " + unsupportedKey, runtimeException.getMessage()); ++ } ++ ++ @Test ++ public void testAddWrongFormatKeyOnCreating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String topicName = "testAddWrongFormatKeyOnCreating-" + System.currentTimeMillis(); ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("++enum.key", "value1"); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topicName); ++ topicConfig.setAttributes(attributes); ++ ++ RuntimeException runtimeException = Assert.assertThrows(RuntimeException.class, () -> topicConfigManager.updateTopicConfig(topicConfig)); ++ Assert.assertEquals("kv string format wrong.", runtimeException.getMessage()); ++ } ++ ++ @Test ++ public void testDeleteKeyOnCreating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String topicName = "testDeleteKeyOnCreating-" + System.currentTimeMillis(); ++ ++ String key = "enum.key"; ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("-" + key, ""); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topicName); ++ topicConfig.setAttributes(attributes); ++ ++ RuntimeException runtimeException = Assert.assertThrows(RuntimeException.class, () -> topicConfigManager.updateTopicConfig(topicConfig)); ++ Assert.assertEquals("only add attribute is supported while creating topic. key: " + key, runtimeException.getMessage()); ++ } ++ ++ @Test ++ public void testAddWrongValueOnCreating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String topicName = "testAddWrongValueOnCreating-" + System.currentTimeMillis(); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+" + TopicAttributes.QUEUE_TYPE_ATTRIBUTE.getName(), "wrong-value"); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topicName); ++ topicConfig.setAttributes(attributes); ++ ++ RuntimeException runtimeException = Assert.assertThrows(RuntimeException.class, () -> topicConfigManager.updateTopicConfig(topicConfig)); ++ Assert.assertEquals("value is not in set: [SimpleCQ, BatchCQ]", runtimeException.getMessage()); ++ } ++ ++ @Test ++ public void testNormalAddKeyOnCreating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String topic = "testNormalAddKeyOnCreating-" + System.currentTimeMillis(); ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+enum.key", "enum-2"); ++ attributes.put("+long.range.key", "16"); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topic); ++ topicConfig.setAttributes(attributes); ++ topicConfigManager.updateTopicConfig(topicConfig); ++ ++ TopicConfig existingTopicConfig = topicConfigManager.getTopicConfigTable().get(topic); ++ Assert.assertEquals("enum-2", existingTopicConfig.getAttributes().get("enum.key")); ++ Assert.assertEquals("16", existingTopicConfig.getAttributes().get("long.range.key")); ++ // assert file ++ } ++ ++ @Test ++ public void testAddDuplicatedKeyOnUpdating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String duplicatedKey = "long.range.key"; ++ String topicName = "testAddDuplicatedKeyOnUpdating-" + System.currentTimeMillis(); ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+enum.key", "enum-3"); ++ attributes.put("+bool.key", "true"); ++ attributes.put("+long.range.key", "12"); ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topicName); ++ topicConfig.setAttributes(attributes); ++ topicConfigManager.updateTopicConfig(topicConfig); ++ ++ ++ ++ attributes = new HashMap<>(); ++ attributes.put("+" + duplicatedKey, "11"); ++ attributes.put("-" + duplicatedKey, ""); ++ TopicConfig duplicateTopicConfig = new TopicConfig(); ++ duplicateTopicConfig.setTopicName(topicName); ++ duplicateTopicConfig.setAttributes(attributes); ++ ++ RuntimeException runtimeException = Assert.assertThrows(RuntimeException.class, () -> topicConfigManager.updateTopicConfig(duplicateTopicConfig)); ++ Assert.assertEquals("alter duplication key. key: " + duplicatedKey, runtimeException.getMessage()); ++ } ++ ++ @Test ++ public void testDeleteNonexistentKeyOnUpdating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String key = "nonexisting.key"; ++ String topicName = "testDeleteNonexistentKeyOnUpdating-" + System.currentTimeMillis(); ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+enum.key", "enum-2"); ++ attributes.put("+bool.key", "true"); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topicName); ++ topicConfig.setAttributes(attributes); ++ ++ topicConfigManager.updateTopicConfig(topicConfig); ++ ++ attributes = new HashMap<>(); ++ attributes.clear(); ++ attributes.put("-" + key, ""); ++ topicConfig.setAttributes(attributes); ++ RuntimeException runtimeException = Assert.assertThrows(RuntimeException.class, () -> topicConfigManager.updateTopicConfig(topicConfig)); ++ Assert.assertEquals("attempt to delete a nonexistent key: " + key, runtimeException.getMessage()); ++ } ++ ++ @Test ++ public void testAlterTopicWithoutChangingAttributes() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String topic = "testAlterTopicWithoutChangingAttributes-" + System.currentTimeMillis(); ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+enum.key", "enum-2"); ++ attributes.put("+bool.key", "true"); ++ ++ TopicConfig topicConfigInit = new TopicConfig(); ++ topicConfigInit.setTopicName(topic); ++ topicConfigInit.setAttributes(attributes); ++ ++ topicConfigManager.updateTopicConfig(topicConfigInit); ++ Assert.assertEquals("enum-2", topicConfigManager.getTopicConfigTable().get(topic).getAttributes().get("enum.key")); ++ Assert.assertEquals("true", topicConfigManager.getTopicConfigTable().get(topic).getAttributes().get("bool.key")); ++ ++ TopicConfig topicConfigAlter = new TopicConfig(); ++ topicConfigAlter.setTopicName(topic); ++ topicConfigAlter.setReadQueueNums(10); ++ topicConfigAlter.setWriteQueueNums(10); ++ topicConfigManager.updateTopicConfig(topicConfigAlter); ++ Assert.assertEquals("enum-2", topicConfigManager.getTopicConfigTable().get(topic).getAttributes().get("enum.key")); ++ Assert.assertEquals("true", topicConfigManager.getTopicConfigTable().get(topic).getAttributes().get("bool.key")); ++ } ++ ++ @Test ++ public void testNormalUpdateUnchangeableKeyOnUpdating() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String topic = "testNormalUpdateUnchangeableKeyOnUpdating-" + System.currentTimeMillis(); ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", true, false), ++ new LongRangeAttribute("long.range.key", false, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+long.range.key", "14"); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topic); ++ topicConfig.setAttributes(attributes); ++ ++ topicConfigManager.updateTopicConfig(topicConfig); ++ ++ attributes.put("+long.range.key", "16"); ++ topicConfig.setAttributes(attributes); ++ RuntimeException runtimeException = Assert.assertThrows(RuntimeException.class, () -> topicConfigManager.updateTopicConfig(topicConfig)); ++ Assert.assertEquals("attempt to update an unchangeable attribute. key: long.range.key", runtimeException.getMessage()); ++ } ++ ++ @Test ++ public void testNormalQueryKeyOnGetting() { ++ if (notToBeExecuted()) { ++ return; ++ } ++ String topic = "testNormalQueryKeyOnGetting-" + System.currentTimeMillis(); ++ String unchangeable = "bool.key"; ++ ++ supportAttributes(asList( ++ new EnumAttribute("enum.key", true, newHashSet("enum-1", "enum-2", "enum-3"), "enum-1"), ++ new BooleanAttribute("bool.key", false, false), ++ new LongRangeAttribute("long.range.key", true, 10, 20, 15) ++ )); ++ ++ Map attributes = new HashMap<>(); ++ attributes.put("+" + unchangeable, "true"); ++ ++ TopicConfig topicConfig = new TopicConfig(); ++ topicConfig.setTopicName(topic); ++ topicConfig.setAttributes(attributes); ++ ++ topicConfigManager.updateTopicConfig(topicConfig); ++ ++ TopicConfig topicConfigUpdated = topicConfigManager.getTopicConfigTable().get(topic); ++ Assert.assertEquals(CQType.SimpleCQ, QueueTypeUtils.getCQType(Optional.of(topicConfigUpdated))); ++ ++ Assert.assertEquals("true", topicConfigUpdated.getAttributes().get(unchangeable)); ++ } ++ ++ private void supportAttributes(List supportAttributes) { ++ Map supportedAttributes = new HashMap<>(); ++ ++ for (Attribute supportAttribute : supportAttributes) { ++ supportedAttributes.put(supportAttribute.getName(), supportAttribute); ++ } ++ ++ TopicAttributes.ALL.putAll(supportedAttributes); ++ } ++ ++ private boolean notToBeExecuted() { ++ return MixAll.isMac(); ++ } ++} +diff --git a/client/BUILD.bazel b/client/BUILD.bazel +index e491cfcef..46e29452b 100644 +--- a/client/BUILD.bazel ++++ b/client/BUILD.bazel +@@ -33,6 +33,7 @@ java_library( + "@maven//:commons_collections_commons_collections", + "@maven//:io_github_aliyunmq_rocketmq_slf4j_api", + "@maven//:io_github_aliyunmq_rocketmq_logback_classic", ++ "@maven//:com_google_guava_guava", + ], + ) + +diff --git a/common/BUILD.bazel b/common/BUILD.bazel +index 831c85e3d..a95a19ccd 100644 +--- a/common/BUILD.bazel ++++ b/common/BUILD.bazel +@@ -39,6 +39,7 @@ java_library( + "@maven//:org_lz4_lz4_java", + "@maven//:io_github_aliyunmq_rocketmq_slf4j_api", + "@maven//:io_github_aliyunmq_rocketmq_logback_classic", ++ "@maven//:io_github_aliyunmq_rocketmq_rocksdb", + ], + ) + +diff --git a/common/pom.xml b/common/pom.xml +index 9796d1b2d..31eb0f087 100644 +--- a/common/pom.xml ++++ b/common/pom.xml +@@ -104,5 +104,9 @@ + io.github.aliyunmq + rocketmq-logback-classic + ++ ++ io.github.aliyunmq ++ rocketmq-rocksdb ++ + + +diff --git a/common/src/main/java/org/apache/rocketmq/common/ConfigManager.java b/common/src/main/java/org/apache/rocketmq/common/ConfigManager.java +index f712e1694..6c3bed47c 100644 +--- a/common/src/main/java/org/apache/rocketmq/common/ConfigManager.java ++++ b/common/src/main/java/org/apache/rocketmq/common/ConfigManager.java +@@ -18,6 +18,8 @@ package org.apache.rocketmq.common; + + import java.io.IOException; + import java.util.Map; ++ ++import org.apache.rocketmq.common.config.RocksDBConfigManager; + import org.apache.rocketmq.common.constant.LoggerName; + import org.apache.rocketmq.logging.org.slf4j.Logger; + import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +@@ -25,7 +27,7 @@ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; + public abstract class ConfigManager { + private static final Logger log = LoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME); + +- public abstract String encode(); ++ protected RocksDBConfigManager rocksDBConfigManager; + + public boolean load() { + String fileName = null; +@@ -46,8 +48,6 @@ public abstract class ConfigManager { + } + } + +- public abstract String configFilePath(); +- + private boolean loadBak() { + String fileName = null; + try { +@@ -66,8 +66,6 @@ public abstract class ConfigManager { + return true; + } + +- public abstract void decode(final String jsonString); +- + public synchronized void persist(String topicName, T t) { + // stub for future + this.persist(); +@@ -90,5 +88,19 @@ public abstract class ConfigManager { + } + } + ++ protected void decode0(final byte[] key, final byte[] body) { ++ ++ } ++ ++ public boolean stop() { ++ return true; ++ } ++ ++ public abstract String configFilePath(); ++ ++ public abstract String encode(); ++ + public abstract String encode(final boolean prettyFormat); ++ ++ public abstract void decode(final String jsonString); + } +diff --git a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java +new file mode 100644 +index 000000000..e3673baad +--- /dev/null ++++ b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java +@@ -0,0 +1,613 @@ ++/* ++ * 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.rocketmq.common.config; ++ ++import java.nio.ByteBuffer; ++import java.nio.charset.Charset; ++import java.util.ArrayList; ++import java.util.List; ++import java.util.Map; ++import java.util.concurrent.ArrayBlockingQueue; ++import java.util.concurrent.ScheduledExecutorService; ++import java.util.concurrent.ScheduledThreadPoolExecutor; ++import java.util.concurrent.Semaphore; ++import java.util.concurrent.ThreadPoolExecutor; ++import java.util.concurrent.TimeUnit; ++ ++import com.google.common.collect.Lists; ++import com.google.common.collect.Maps; ++ ++import org.apache.rocketmq.common.ThreadFactoryImpl; ++import org.apache.rocketmq.common.constant.LoggerName; ++import org.apache.rocketmq.logging.org.slf4j.Logger; ++import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; ++import org.rocksdb.ColumnFamilyDescriptor; ++import org.rocksdb.ColumnFamilyHandle; ++import org.rocksdb.ColumnFamilyOptions; ++import org.rocksdb.CompactRangeOptions; ++import org.rocksdb.CompactionOptions; ++import org.rocksdb.DBOptions; ++import org.rocksdb.FlushOptions; ++import org.rocksdb.LiveFileMetaData; ++import org.rocksdb.Priority; ++import org.rocksdb.ReadOptions; ++import org.rocksdb.RocksDB; ++import org.rocksdb.RocksDBException; ++import org.rocksdb.RocksIterator; ++import org.rocksdb.Statistics; ++import org.rocksdb.Status; ++import org.rocksdb.WriteBatch; ++import org.rocksdb.WriteOptions; ++ ++import static org.rocksdb.RocksDB.NOT_FOUND; ++ ++public abstract class AbstractRocksDBStorage { ++ protected static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKSDB_LOGGER_NAME); ++ ++ private static final Charset CHARSET_UTF8 = Charset.forName("UTF-8"); ++ private static final String SPACE = " | "; ++ ++ protected String dbPath; ++ protected boolean readOnly; ++ protected RocksDB db; ++ protected DBOptions options; ++ ++ protected WriteOptions writeOptions; ++ protected WriteOptions ableWalWriteOptions; ++ ++ protected ReadOptions readOptions; ++ protected ReadOptions totalOrderReadOptions; ++ ++ protected CompactionOptions compactionOptions; ++ protected CompactRangeOptions compactRangeOptions; ++ ++ protected ColumnFamilyHandle defaultCFHandle; ++ protected final List cfOptions = new ArrayList(); ++ ++ protected volatile boolean loaded; ++ private volatile boolean closed; ++ ++ private final Semaphore reloadPermit = new Semaphore(1); ++ private final ScheduledExecutorService reloadScheduler = new ScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("RocksDBStorageReloadService_")); ++ private final ThreadPoolExecutor manualCompactionThread = new ThreadPoolExecutor( ++ 1, 1, 1000 * 60, TimeUnit.MILLISECONDS, ++ new ArrayBlockingQueue(1), ++ new ThreadFactoryImpl("RocksDBManualCompactionService_"), ++ new ThreadPoolExecutor.DiscardOldestPolicy()); ++ ++ static { ++ RocksDB.loadLibrary(); ++ } ++ ++ public boolean hold() { ++ if (!this.loaded || this.db == null || this.closed) { ++ LOGGER.error("hold rocksdb Failed. {}", this.dbPath); ++ return false; ++ } else { ++ return true; ++ } ++ } ++ ++ public void release() { ++ } ++ ++ protected void put(ColumnFamilyHandle cfHandle, WriteOptions writeOptions, ++ final byte[] keyBytes, final int keyLen, ++ final byte[] valueBytes, final int valueLen) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ this.db.put(cfHandle, writeOptions, keyBytes, 0, keyLen, valueBytes, 0, valueLen); ++ } catch (RocksDBException e) { ++ scheduleReloadRocksdb(e); ++ LOGGER.error("put Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected void put(ColumnFamilyHandle cfHandle, WriteOptions writeOptions, ++ final ByteBuffer keyBB, final ByteBuffer valueBB) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ this.db.put(cfHandle, writeOptions, keyBB, valueBB); ++ } catch (RocksDBException e) { ++ scheduleReloadRocksdb(e); ++ LOGGER.error("put Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected void batchPut(WriteOptions writeOptions, final WriteBatch batch) throws RocksDBException { ++ try { ++ this.db.write(writeOptions, batch); ++ } catch (RocksDBException e) { ++ scheduleReloadRocksdb(e); ++ LOGGER.error("batchPut Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ batch.clear(); ++ } ++ } ++ ++ protected byte[] get(ColumnFamilyHandle cfHandle, ReadOptions readOptions, byte[] keyBytes) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ return this.db.get(cfHandle, readOptions, keyBytes); ++ } catch (RocksDBException e) { ++ LOGGER.error("get Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected boolean get(ColumnFamilyHandle cfHandle, ReadOptions readOptions, ++ final ByteBuffer keyBB, final ByteBuffer valueBB) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ return this.db.get(cfHandle, readOptions, keyBB, valueBB) != NOT_FOUND; ++ } catch (RocksDBException e) { ++ LOGGER.error("get Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected List multiGet(final ReadOptions readOptions, ++ final List columnFamilyHandleList, ++ final List keys) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ return this.db.multiGetAsList(readOptions, columnFamilyHandleList, keys); ++ } catch (RocksDBException e) { ++ LOGGER.error("multiGet Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected void delete(ColumnFamilyHandle cfHandle, WriteOptions writeOptions, byte[] keyBytes) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ this.db.delete(cfHandle, writeOptions, keyBytes); ++ } catch (RocksDBException e) { ++ LOGGER.error("delete Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected void delete(ColumnFamilyHandle cfHandle, WriteOptions writeOptions, ByteBuffer keyBB) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ this.db.delete(cfHandle, writeOptions, keyBB); ++ } catch (RocksDBException e) { ++ LOGGER.error("delete Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected WrappedRocksIterator newIterator(ColumnFamilyHandle cfHandle, ReadOptions readOptions) { ++ return new WrappedRocksIterator(this.db.newIterator(cfHandle, readOptions)); ++ } ++ ++ protected void rangeDelete(ColumnFamilyHandle cfHandle, WriteOptions writeOptions, ++ final byte[] startKey, final byte[] endKey) throws RocksDBException { ++ if (!hold()) { ++ throw new IllegalStateException("rocksDB:" + this + " is not ready"); ++ } ++ try { ++ this.db.deleteRange(cfHandle, writeOptions, startKey, endKey); ++ } catch (RocksDBException e) { ++ scheduleReloadRocksdb(e); ++ LOGGER.error("rangeDelete Failed. {}, {}", this.dbPath, getStatusError(e)); ++ throw e; ++ } finally { ++ release(); ++ } ++ } ++ ++ protected void manualCompactionDefaultCfMaxLevel(final CompactionOptions compactionOptions) throws Exception { ++ final ColumnFamilyHandle defaultCFHandle = this.defaultCFHandle; ++ final byte[] defaultCFName = defaultCFHandle.getName(); ++ List fileMetaDataList = this.db.getLiveFilesMetaData(); ++ if (fileMetaDataList == null || fileMetaDataList.isEmpty()) { ++ return; ++ } ++ ++ List defaultLiveFileDataList = Lists.newArrayList(); ++ List inputFileNames = Lists.newArrayList(); ++ int maxLevel = 0; ++ for (LiveFileMetaData fileMetaData : fileMetaDataList) { ++ if (compareTo(fileMetaData.columnFamilyName(), defaultCFName) != 0) { ++ continue; ++ } ++ defaultLiveFileDataList.add(fileMetaData); ++ if (fileMetaData.level() > maxLevel) { ++ maxLevel = fileMetaData.level(); ++ } ++ } ++ if (maxLevel == 0) { ++ LOGGER.info("manualCompactionDefaultCfFiles skip level 0."); ++ return; ++ } ++ ++ for (LiveFileMetaData fileMetaData : defaultLiveFileDataList) { ++ if (fileMetaData.level() != maxLevel || fileMetaData.beingCompacted()) { ++ continue; ++ } ++ inputFileNames.add(fileMetaData.path() + fileMetaData.fileName()); ++ } ++ if (!inputFileNames.isEmpty()) { ++ List outputLists = this.db.compactFiles(compactionOptions, defaultCFHandle, ++ inputFileNames, maxLevel, -1, null); ++ LOGGER.info("manualCompactionDefaultCfFiles OK. src: {}, dst: {}", inputFileNames, outputLists); ++ } else { ++ LOGGER.info("manualCompactionDefaultCfFiles Empty."); ++ } ++ } ++ ++ protected void manualCompactionDefaultCfRange(CompactRangeOptions compactRangeOptions) { ++ if (!hold()) { ++ return; ++ } ++ long s1 = System.currentTimeMillis(); ++ boolean result = true; ++ try { ++ LOGGER.info("manualCompaction Start. {}", this.dbPath); ++ this.db.compactRange(this.defaultCFHandle, null, null, compactRangeOptions); ++ } catch (RocksDBException e) { ++ result = false; ++ scheduleReloadRocksdb(e); ++ LOGGER.error("manualCompaction Failed. {}, {}", this.dbPath, getStatusError(e)); ++ } finally { ++ release(); ++ LOGGER.info("manualCompaction End. {}, rt: {}(ms), result: {}", this.dbPath, System.currentTimeMillis() - s1, result); ++ } ++ } ++ ++ protected void manualCompaction(long minPhyOffset, final CompactRangeOptions compactRangeOptions) { ++ this.manualCompactionThread.submit(new Runnable() { ++ @Override ++ public void run() { ++ manualCompactionDefaultCfRange(compactRangeOptions); ++ } ++ }); ++ } ++ ++ protected void open(final List cfDescriptors, ++ final List cfHandles) throws RocksDBException { ++ if (this.readOnly) { ++ this.db = RocksDB.openReadOnly(this.options, this.dbPath, cfDescriptors, cfHandles); ++ } else { ++ this.db = RocksDB.open(this.options, this.dbPath, cfDescriptors, cfHandles); ++ } ++ this.db.getEnv().setBackgroundThreads(8, Priority.HIGH); ++ this.db.getEnv().setBackgroundThreads(8, Priority.LOW); ++ ++ if (this.db == null) { ++ throw new RocksDBException("open rocksdb null"); ++ } ++ } ++ ++ protected abstract boolean postLoad(); ++ ++ public synchronized boolean start() { ++ if (this.loaded) { ++ return true; ++ } ++ if (postLoad()) { ++ this.loaded = true; ++ LOGGER.info("start OK. {}", this.dbPath); ++ this.closed = false; ++ return true; ++ } else { ++ return false; ++ } ++ } ++ ++ protected abstract void preShutdown(); ++ ++ public synchronized boolean shutdown() { ++ try { ++ if (!this.loaded) { ++ return true; ++ } ++ ++ final FlushOptions flushOptions = new FlushOptions(); ++ flushOptions.setWaitForFlush(true); ++ try { ++ flush(flushOptions); ++ } finally { ++ flushOptions.close(); ++ } ++ this.db.cancelAllBackgroundWork(true); ++ this.db.pauseBackgroundWork(); ++ //The close order is matter. ++ //1. close column family handles ++ preShutdown(); ++ ++ this.defaultCFHandle.close(); ++ //2. close column family options. ++ for (final ColumnFamilyOptions opt : this.cfOptions) { ++ opt.close(); ++ } ++ //3. close options ++ if (this.writeOptions != null) { ++ this.writeOptions.close(); ++ } ++ if (this.ableWalWriteOptions != null) { ++ this.ableWalWriteOptions.close(); ++ } ++ if (this.readOptions != null) { ++ this.readOptions.close(); ++ } ++ if (this.totalOrderReadOptions != null) { ++ this.totalOrderReadOptions.close(); ++ } ++ if (this.options != null) { ++ this.options.close(); ++ } ++ //4. close db. ++ if (db != null) { ++ this.db.syncWal(); ++ this.db.closeE(); ++ } ++ //5. help gc. ++ this.cfOptions.clear(); ++ this.db = null; ++ this.readOptions = null; ++ this.totalOrderReadOptions = null; ++ this.writeOptions = null; ++ this.ableWalWriteOptions = null; ++ this.options = null; ++ ++ this.loaded = false; ++ LOGGER.info("shutdown OK. {}", this.dbPath); ++ } catch (Exception e) { ++ LOGGER.error("shutdown Failed. {}", this.dbPath, e); ++ return false; ++ } ++ return true; ++ } ++ ++ public void flush(final FlushOptions flushOptions) { ++ if (!this.loaded || this.readOnly || closed) { ++ return; ++ } ++ ++ try { ++ if (db != null) { ++ this.db.flush(flushOptions); ++ } ++ } catch (RocksDBException e) { ++ scheduleReloadRocksdb(e); ++ LOGGER.error("flush Failed. {}, {}", this.dbPath, getStatusError(e)); ++ } ++ } ++ ++ public Statistics getStatistics() { ++ return this.options.statistics(); ++ } ++ ++ public ColumnFamilyHandle getDefaultCFHandle() { ++ return defaultCFHandle; ++ } ++ ++ public List getCompactionStatus() { ++ if (!hold()) { ++ return null; ++ } ++ try { ++ return this.db.getLiveFilesMetaData(); ++ } finally { ++ release(); ++ } ++ } ++ ++ private void scheduleReloadRocksdb(RocksDBException rocksDBException) { ++ if (rocksDBException == null || rocksDBException.getStatus() == null) { ++ return; ++ } ++ Status status = rocksDBException.getStatus(); ++ Status.Code code = status.getCode(); ++ // Status.Code.Incomplete == code ++ if (Status.Code.Aborted == code || Status.Code.Corruption == code || Status.Code.Undefined == code) { ++ LOGGER.error("scheduleReloadRocksdb. {}, {}", this.dbPath, getStatusError(rocksDBException)); ++ scheduleReloadRocksdb0(); ++ } ++ } ++ ++ private void scheduleReloadRocksdb0() { ++ if (!this.reloadPermit.tryAcquire()) { ++ return; ++ } ++ this.closed = true; ++ this.reloadScheduler.schedule(new Runnable() { ++ @Override ++ public void run() { ++ boolean result = true; ++ try { ++ reloadRocksdb(); ++ } catch (Exception e) { ++ result = false; ++ } finally { ++ reloadPermit.release(); ++ } ++ // try to reload rocksdb next time ++ if (!result) { ++ LOGGER.info("reload rocksdb Retry. {}", dbPath); ++ scheduleReloadRocksdb0(); ++ } ++ } ++ }, 10, TimeUnit.SECONDS); ++ } ++ ++ private void reloadRocksdb() throws Exception { ++ LOGGER.info("reload rocksdb Start. {}", this.dbPath); ++ if (!shutdown() || !start()) { ++ LOGGER.error("reload rocksdb Failed. {}", dbPath); ++ throw new Exception("reload rocksdb Error"); ++ } ++ LOGGER.info("reload rocksdb OK. {}", this.dbPath); ++ } ++ ++ public void flushWAL() throws RocksDBException { ++ this.db.flushWal(true); ++ } ++ ++ protected class WrappedRocksIterator { ++ private final RocksIterator iterator; ++ ++ public WrappedRocksIterator(final RocksIterator iterator) { ++ this.iterator = iterator; ++ } ++ ++ public byte[] key() { ++ return iterator.key(); ++ } ++ ++ public byte[] value() { ++ return iterator.value(); ++ } ++ ++ public void next() { ++ iterator.next(); ++ } ++ ++ public void prev() { ++ iterator.prev(); ++ } ++ ++ public void seek(byte[] target) { ++ iterator.seek(target); ++ } ++ ++ public void seekForPrev(byte[] target) { ++ iterator.seekForPrev(target); ++ } ++ ++ public void seekToFirst() { ++ iterator.seekToFirst(); ++ } ++ ++ public boolean isValid() { ++ return iterator.isValid(); ++ } ++ ++ public void close() { ++ iterator.close(); ++ } ++ } ++ ++ private String getStatusError(RocksDBException e) { ++ if (e == null || e.getStatus() == null) { ++ return "null"; ++ } ++ Status status = e.getStatus(); ++ StringBuilder sb = new StringBuilder(64); ++ sb.append("code: "); ++ if (status.getCode() != null) { ++ sb.append(status.getCode().name()); ++ } else { ++ sb.append("null"); ++ } ++ sb.append(", ").append("subCode: "); ++ if (status.getSubCode() != null) { ++ sb.append(status.getSubCode().name()); ++ } else { ++ sb.append("null"); ++ } ++ sb.append(", ").append("state: ").append(status.getState()); ++ return sb.toString(); ++ } ++ ++ public void statRocksdb(Logger logger) { ++ try { ++ ++ List liveFileMetaDataList = this.getCompactionStatus(); ++ if (liveFileMetaDataList == null || liveFileMetaDataList.isEmpty()) { ++ return; ++ } ++ Map map = Maps.newHashMap(); ++ for (LiveFileMetaData metaData : liveFileMetaDataList) { ++ StringBuilder sb = map.get(metaData.level()); ++ if (sb == null) { ++ sb = new StringBuilder(256); ++ map.put(metaData.level(), sb); ++ } ++ sb.append(new String(metaData.columnFamilyName(), CHARSET_UTF8)).append(SPACE). ++ append(metaData.fileName()).append(SPACE). ++ append("s: ").append(metaData.size()).append(SPACE). ++ append("a: ").append(metaData.numEntries()).append(SPACE). ++ append("r: ").append(metaData.numReadsSampled()).append(SPACE). ++ append("d: ").append(metaData.numDeletions()).append(SPACE). ++ append(metaData.beingCompacted()).append("\n"); ++ } ++ for (Map.Entry entry : map.entrySet()) { ++ logger.info("level: {}\n{}", entry.getKey(), entry.getValue().toString()); ++ } ++ ++ String blockCacheMemUsage = this.db.getProperty("rocksdb.block-cache-usage"); ++ String indexesAndFilterBlockMemUsage = this.db.getProperty("rocksdb.estimate-table-readers-mem"); ++ String memTableMemUsage = this.db.getProperty("rocksdb.cur-size-all-mem-tables"); ++ String blocksPinnedByIteratorMemUsage = this.db.getProperty("rocksdb.block-cache-pinned-usage"); ++ logger.info("MemUsage. blockCache: {}, indexesAndFilterBlock: {}, memtable: {}, blocksPinnedByIterator: {}", ++ blockCacheMemUsage, indexesAndFilterBlockMemUsage, memTableMemUsage, blocksPinnedByIteratorMemUsage); ++ } catch (Exception ignored) { ++ } ++ } ++ ++ public int compareTo(byte[] v1, byte[] v2) { ++ int len1 = v1.length; ++ int len2 = v2.length; ++ int lim = Math.min(len1, len2); ++ ++ int k = 0; ++ while (k < lim) { ++ byte c1 = v1[k]; ++ byte c2 = v2[k]; ++ if (c1 != c2) { ++ return c1 - c2; ++ } ++ k++; ++ } ++ return len1 - len2; ++ } ++} +\ No newline at end of file +diff --git a/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java +new file mode 100644 +index 000000000..9d05ed282 +--- /dev/null ++++ b/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java +@@ -0,0 +1,250 @@ ++/* ++ * 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.rocketmq.common.config; ++ ++import java.io.File; ++import java.nio.ByteBuffer; ++import java.util.ArrayList; ++import java.util.List; ++ ++import org.apache.commons.lang3.StringUtils; ++import org.apache.rocketmq.common.UtilAll; ++import org.rocksdb.BlockBasedTableConfig; ++import org.rocksdb.BloomFilter; ++import org.rocksdb.ColumnFamilyDescriptor; ++import org.rocksdb.ColumnFamilyHandle; ++import org.rocksdb.ColumnFamilyOptions; ++import org.rocksdb.CompactRangeOptions; ++import org.rocksdb.CompactRangeOptions.BottommostLevelCompaction; ++import org.rocksdb.CompactionOptions; ++import org.rocksdb.CompactionStyle; ++import org.rocksdb.CompressionType; ++import org.rocksdb.DBOptions; ++import org.rocksdb.DataBlockIndexType; ++import org.rocksdb.IndexType; ++import org.rocksdb.InfoLogLevel; ++import org.rocksdb.LRUCache; ++import org.rocksdb.RateLimiter; ++import org.rocksdb.ReadOptions; ++import org.rocksdb.RocksDB; ++import org.rocksdb.RocksDBException; ++import org.rocksdb.RocksIterator; ++import org.rocksdb.SkipListMemTableConfig; ++import org.rocksdb.Statistics; ++import org.rocksdb.StatsLevel; ++import org.rocksdb.StringAppendOperator; ++import org.rocksdb.WALRecoveryMode; ++import org.rocksdb.WriteBatch; ++import org.rocksdb.WriteOptions; ++import org.rocksdb.util.SizeUnit; ++ ++public class ConfigRocksDBStorage extends AbstractRocksDBStorage { ++ ++ public ConfigRocksDBStorage(final String dbPath) { ++ super(); ++ this.dbPath = dbPath; ++ this.readOnly = false; ++ } ++ ++ private void initOptions() { ++ this.options = createConfigDBOptions(); ++ ++ this.writeOptions = new WriteOptions(); ++ this.writeOptions.setSync(false); ++ this.writeOptions.setDisableWAL(true); ++ this.writeOptions.setNoSlowdown(true); ++ ++ this.ableWalWriteOptions = new WriteOptions(); ++ this.ableWalWriteOptions.setSync(false); ++ this.ableWalWriteOptions.setDisableWAL(false); ++ this.ableWalWriteOptions.setNoSlowdown(true); ++ ++ this.readOptions = new ReadOptions(); ++ this.readOptions.setPrefixSameAsStart(true); ++ this.readOptions.setTotalOrderSeek(false); ++ this.readOptions.setTailing(false); ++ ++ this.totalOrderReadOptions = new ReadOptions(); ++ this.totalOrderReadOptions.setPrefixSameAsStart(false); ++ this.totalOrderReadOptions.setTotalOrderSeek(false); ++ this.totalOrderReadOptions.setTailing(false); ++ ++ this.compactRangeOptions = new CompactRangeOptions(); ++ this.compactRangeOptions.setBottommostLevelCompaction(BottommostLevelCompaction.kForce); ++ this.compactRangeOptions.setAllowWriteStall(true); ++ this.compactRangeOptions.setExclusiveManualCompaction(false); ++ this.compactRangeOptions.setChangeLevel(true); ++ this.compactRangeOptions.setTargetLevel(-1); ++ this.compactRangeOptions.setMaxSubcompactions(4); ++ ++ this.compactionOptions = new CompactionOptions(); ++ this.compactionOptions.setCompression(CompressionType.LZ4_COMPRESSION); ++ this.compactionOptions.setMaxSubcompactions(4); ++ this.compactionOptions.setOutputFileSizeLimit(4 * 1024 * 1024 * 1024L); ++ } ++ ++ @Override ++ protected boolean postLoad() { ++ try { ++ UtilAll.ensureDirOK(this.dbPath); ++ ++ initOptions(); ++ ++ final List cfDescriptors = new ArrayList(); ++ ++ ColumnFamilyOptions defaultOptions = createConfigOptions(); ++ this.cfOptions.add(defaultOptions); ++ cfDescriptors.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, defaultOptions)); ++ ++ final List cfHandles = new ArrayList(); ++ open(cfDescriptors, cfHandles); ++ ++ this.defaultCFHandle = cfHandles.get(0); ++ } catch (final Exception e) { ++ AbstractRocksDBStorage.LOGGER.error("postLoad Failed. {}", this.dbPath, e); ++ return false; ++ } ++ return true; ++ } ++ ++ @Override ++ protected void preShutdown() { ++ ++ } ++ ++ private ColumnFamilyOptions createConfigOptions() { ++ BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig(). ++ setFormatVersion(5). ++ setIndexType(IndexType.kBinarySearch). ++ setDataBlockIndexType(DataBlockIndexType.kDataBlockBinarySearch). ++ setBlockSize(32 * SizeUnit.KB). ++ setFilterPolicy(new BloomFilter(16, false)). ++ // Indicating if we'd put index/filter blocks to the block cache. ++ setCacheIndexAndFilterBlocks(false). ++ setCacheIndexAndFilterBlocksWithHighPriority(true). ++ setPinL0FilterAndIndexBlocksInCache(false). ++ setPinTopLevelIndexAndFilter(true). ++ setBlockCache(new LRUCache(4 * SizeUnit.MB, 8, false)). ++ setWholeKeyFiltering(true); ++ ++ ColumnFamilyOptions options = new ColumnFamilyOptions(); ++ return options.setMaxWriteBufferNumber(2). ++ // MemTable size, memtable(cache) -> immutable memtable(cache) -> sst(disk) ++ setWriteBufferSize(8 * SizeUnit.MB). ++ setMinWriteBufferNumberToMerge(1). ++ setTableFormatConfig(blockBasedTableConfig). ++ setMemTableConfig(new SkipListMemTableConfig()). ++ setCompressionType(CompressionType.NO_COMPRESSION). ++ setNumLevels(7). ++ setCompactionStyle(CompactionStyle.LEVEL). ++ setLevel0FileNumCompactionTrigger(4). ++ setLevel0SlowdownWritesTrigger(8). ++ setLevel0StopWritesTrigger(12). ++ // The target file size for compaction. ++ setTargetFileSizeBase(64 * SizeUnit.MB). ++ setTargetFileSizeMultiplier(2). ++ // The upper-bound of the total size of L1 files in bytes ++ setMaxBytesForLevelBase(256 * SizeUnit.MB). ++ setMaxBytesForLevelMultiplier(2). ++ setMergeOperator(new StringAppendOperator()). ++ setInplaceUpdateSupport(true); ++ } ++ ++ private DBOptions createConfigDBOptions() { ++ //Turn based on https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide ++ // and http://gitlab.alibaba-inc.com/aloha/aloha/blob/branch_2_5_0/jstorm-core/src/main/java/com/alibaba/jstorm/cache/rocksdb/RocksDbOptionsFactory.java ++ DBOptions options = new DBOptions(); ++ Statistics statistics = new Statistics(); ++ statistics.setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS); ++ return options. ++ setDbLogDir(getDBLogDir()). ++ setInfoLogLevel(InfoLogLevel.INFO_LEVEL). ++ setWalRecoveryMode(WALRecoveryMode.SkipAnyCorruptedRecords). ++ setManualWalFlush(true). ++ setMaxTotalWalSize(500 * SizeUnit.MB). ++ setWalSizeLimitMB(0). ++ setWalTtlSeconds(0). ++ setCreateIfMissing(true). ++ setCreateMissingColumnFamilies(true). ++ setMaxOpenFiles(-1). ++ setMaxLogFileSize(1 * SizeUnit.GB). ++ setKeepLogFileNum(5). ++ setMaxManifestFileSize(1 * SizeUnit.GB). ++ setAllowConcurrentMemtableWrite(false). ++ setStatistics(statistics). ++ setStatsDumpPeriodSec(600). ++ setAtomicFlush(true). ++ setMaxBackgroundJobs(32). ++ setMaxSubcompactions(4). ++ setParanoidChecks(true). ++ setDelayedWriteRate(16 * SizeUnit.MB). ++ setRateLimiter(new RateLimiter(100 * SizeUnit.MB)). ++ setUseDirectIoForFlushAndCompaction(true). ++ setUseDirectReads(true); ++ } ++ ++ private static String getDBLogDir() { ++ String rootPath = System.getProperty("user.home"); ++ if (StringUtils.isEmpty(rootPath)) { ++ return ""; ++ } ++ rootPath = rootPath + File.separator + "logs"; ++ UtilAll.ensureDirOK(rootPath); ++ return rootPath + File.separator + "rocketmqlogs" + File.separator; ++ } ++ ++ public void put(final byte[] keyBytes, final int keyLen, final byte[] valueBytes) throws Exception { ++ put(this.defaultCFHandle, this.ableWalWriteOptions, keyBytes, keyLen, valueBytes, valueBytes.length); ++ } ++ ++ public void put(final ByteBuffer keyBB, final ByteBuffer valueBB) throws Exception { ++ put(this.defaultCFHandle, this.ableWalWriteOptions, keyBB, valueBB); ++ } ++ ++ public byte[] get(final byte[] keyBytes) throws Exception { ++ return get(this.defaultCFHandle, this.totalOrderReadOptions, keyBytes); ++ } ++ ++ public void delete(final byte[] keyBytes) throws Exception { ++ delete(this.defaultCFHandle, this.ableWalWriteOptions, keyBytes); ++ } ++ ++ public List multiGet(final List cfhList, final List keys) throws ++ RocksDBException { ++ return multiGet(this.totalOrderReadOptions, cfhList, keys); ++ } ++ ++ public void batchPut(final WriteBatch batch) throws RocksDBException { ++ batchPut(this.writeOptions, batch); ++ } ++ ++ public void batchPutWithWal(final WriteBatch batch) throws RocksDBException { ++ batchPut(this.ableWalWriteOptions, batch); ++ } ++ ++ public RocksIterator iterator() { ++ return this.db.newIterator(this.defaultCFHandle, this.totalOrderReadOptions); ++ } ++ ++ public void rangeDelete(final byte[] startKey, final byte[] endKey) throws RocksDBException { ++ rangeDelete(this.defaultCFHandle, this.writeOptions, startKey, endKey); ++ } ++ ++ public RocksIterator iterator(ReadOptions readOptions) { ++ return this.db.newIterator(this.defaultCFHandle, readOptions); ++ } ++} +diff --git a/common/src/main/java/org/apache/rocketmq/common/config/RocksDBConfigManager.java b/common/src/main/java/org/apache/rocketmq/common/config/RocksDBConfigManager.java +new file mode 100644 +index 000000000..f958bbdf0 +--- /dev/null ++++ b/common/src/main/java/org/apache/rocketmq/common/config/RocksDBConfigManager.java +@@ -0,0 +1,108 @@ ++/* ++ * 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.rocketmq.common.config; ++ ++import java.util.function.BiConsumer; ++ ++import org.apache.rocketmq.common.constant.LoggerName; ++import org.apache.rocketmq.logging.org.slf4j.Logger; ++import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; ++import org.rocksdb.FlushOptions; ++import org.rocksdb.RocksIterator; ++import org.rocksdb.WriteBatch; ++ ++public class RocksDBConfigManager { ++ protected static final Logger BROKER_LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); ++ ++ protected volatile boolean isStop = false; ++ protected ConfigRocksDBStorage configRocksDBStorage = null; ++ private FlushOptions flushOptions = null; ++ private volatile long lastFlushMemTableMicroSecond = 0; ++ private final long memTableFlushInterval; ++ ++ public RocksDBConfigManager(long memTableFlushInterval) { ++ this.memTableFlushInterval = memTableFlushInterval; ++ } ++ ++ public boolean load(String configFilePath, BiConsumer biConsumer) { ++ this.isStop = false; ++ this.configRocksDBStorage = new ConfigRocksDBStorage(configFilePath); ++ if (!this.configRocksDBStorage.start()) { ++ return false; ++ } ++ RocksIterator iterator = this.configRocksDBStorage.iterator(); ++ try { ++ iterator.seekToFirst(); ++ while (iterator.isValid()) { ++ biConsumer.accept(iterator.key(), iterator.value()); ++ iterator.next(); ++ } ++ } finally { ++ iterator.close(); ++ } ++ ++ this.flushOptions = new FlushOptions(); ++ this.flushOptions.setWaitForFlush(false); ++ this.flushOptions.setAllowWriteStall(false); ++ return true; ++ } ++ ++ public void start() { ++ } ++ ++ public boolean stop() { ++ this.isStop = true; ++ if (this.configRocksDBStorage != null) { ++ return this.configRocksDBStorage.shutdown(); ++ } ++ if (this.flushOptions != null) { ++ this.flushOptions.close(); ++ } ++ return true; ++ } ++ ++ public void flushWAL() { ++ try { ++ if (this.isStop) { ++ return; ++ } ++ if (this.configRocksDBStorage != null) { ++ this.configRocksDBStorage.flushWAL(); ++ ++ long now = System.currentTimeMillis(); ++ if (now > this.lastFlushMemTableMicroSecond + this.memTableFlushInterval) { ++ this.configRocksDBStorage.flush(this.flushOptions); ++ this.lastFlushMemTableMicroSecond = now; ++ } ++ } ++ } catch (Exception e) { ++ BROKER_LOG.error("kv flush WAL Failed.", e); ++ } ++ } ++ ++ public void put(final byte[] keyBytes, final int keyLen, final byte[] valueBytes) throws Exception { ++ this.configRocksDBStorage.put(keyBytes, keyLen, valueBytes); ++ } ++ ++ public void delete(final byte[] keyBytes) throws Exception { ++ this.configRocksDBStorage.delete(keyBytes); ++ } ++ ++ public void batchPutWithWal(final WriteBatch batch) throws Exception { ++ this.configRocksDBStorage.batchPutWithWal(batch); ++ } ++} +diff --git a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java +index c1176ea15..cb04b00b3 100644 +--- a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java ++++ b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java +@@ -51,4 +51,5 @@ public class LoggerName { + public static final String PROXY_LOGGER_NAME = "RocketmqProxy"; + public static final String PROXY_WATER_MARK_LOGGER_NAME = "RocketmqProxyWatermark"; + public static final String ROCKETMQ_COLDCTR_LOGGER_NAME = "RocketmqColdCtr"; ++ public static final String ROCKSDB_LOGGER_NAME = "RocketmqRocksDB"; + } +diff --git a/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java b/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java +index b104016fb..41c9eedd9 100644 +--- a/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java ++++ b/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java +@@ -49,8 +49,7 @@ public class Consumer { + * } + * + */ +- // Uncomment the following line while debugging, namesrvAddr should be set to your local address +-// consumer.setNamesrvAddr(DEFAULT_NAMESRVADDR); ++ consumer.setNamesrvAddr(DEFAULT_NAMESRVADDR); + + /* + * Specify where to start in case the specific consumer group is a brand-new one. +diff --git a/pom.xml b/pom.xml +index 4d5dd1dec..3a08d75f2 100644 +--- a/pom.xml ++++ b/pom.xml +@@ -137,6 +137,7 @@ + 1.26.0-alpha + 2.0.6 + 2.20.29 ++ 1.0.3 + 2.13.4.2 + + +@@ -711,6 +712,11 @@ + slf4j-api + ${slf4j-api.version} + ++ ++ io.github.aliyunmq ++ rocketmq-rocksdb ++ ${rocksdb.version} ++ + + io.github.aliyunmq + rocketmq-shaded-slf4j-api-bridge +diff --git a/remoting/BUILD.bazel b/remoting/BUILD.bazel +index e3e1bce3b..db8b24301 100644 +--- a/remoting/BUILD.bazel ++++ b/remoting/BUILD.bazel +@@ -38,6 +38,7 @@ java_library( + "@maven//:org_apache_commons_commons_lang3", + "@maven//:io_github_aliyunmq_rocketmq_slf4j_api", + "@maven//:io_github_aliyunmq_rocketmq_logback_classic", ++ "@maven//:commons_collections_commons_collections", + ], + ) + +diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreType.java b/store/src/main/java/org/apache/rocketmq/store/StoreType.java +new file mode 100644 +index 000000000..4f9c4d0e4 +--- /dev/null ++++ b/store/src/main/java/org/apache/rocketmq/store/StoreType.java +@@ -0,0 +1,32 @@ ++/* ++ * 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.rocketmq.store; ++ ++public enum StoreType { ++ DEFAULT("default"), ++ DEFAULT_ROCKSDB("defaultRocksDB"); ++ ++ private String storeType; ++ ++ StoreType(String storeType) { ++ this.storeType = storeType; ++ } ++ ++ public String getStoreType() { ++ return storeType; ++ } ++} +diff --git a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java +index 4f204d742..efb728ac0 100644 +--- a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java ++++ b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java +@@ -20,6 +20,7 @@ import java.io.File; + + import org.apache.rocketmq.common.annotation.ImportantField; + import org.apache.rocketmq.store.ConsumeQueue; ++import org.apache.rocketmq.store.StoreType; + import org.apache.rocketmq.store.queue.BatchConsumeQueue; + + public class MessageStoreConfig { +@@ -102,6 +103,9 @@ public class MessageStoreConfig { + private int timerMetricSmallThreshold = 1000000; + private int timerProgressLogIntervalMs = 10 * 1000; + ++ // default, defaultRocksDB ++ @ImportantField ++ private String storeType = StoreType.DEFAULT.getStoreType(); + // ConsumeQueue file size,default is 30W + private int mappedFileSizeConsumeQueue = 300000 * ConsumeQueue.CQ_STORE_UNIT_SIZE; + // enable consume queue ext +@@ -392,6 +396,11 @@ public class MessageStoreConfig { + + private int batchDispatchRequestThreadPoolNums = 16; + ++ // rocksdb mode ++ private boolean realTimePersistRocksDBConfig = true; ++ private long memTableFlushInterval = 60 * 60 * 1000L; ++ private boolean enableRocksDBLog = false; ++ + public boolean isDebugLockEnable() { + return debugLockEnable; + } +@@ -488,6 +497,14 @@ public class MessageStoreConfig { + this.mappedFileSizeCommitLog = mappedFileSizeCommitLog; + } + ++ public String getStoreType() { ++ return storeType; ++ } ++ ++ public void setStoreType(String storeType) { ++ this.storeType = storeType; ++ } ++ + public int getMappedFileSizeConsumeQueue() { + + int factor = (int) Math.ceil(this.mappedFileSizeConsumeQueue / (ConsumeQueue.CQ_STORE_UNIT_SIZE * 1.0)); +@@ -1710,4 +1727,28 @@ public class MessageStoreConfig { + public void setBatchDispatchRequestThreadPoolNums(int batchDispatchRequestThreadPoolNums) { + this.batchDispatchRequestThreadPoolNums = batchDispatchRequestThreadPoolNums; + } ++ ++ public boolean isRealTimePersistRocksDBConfig() { ++ return realTimePersistRocksDBConfig; ++ } ++ ++ public void setRealTimePersistRocksDBConfig(boolean realTimePersistRocksDBConfig) { ++ this.realTimePersistRocksDBConfig = realTimePersistRocksDBConfig; ++ } ++ ++ public long getMemTableFlushInterval() { ++ return memTableFlushInterval; ++ } ++ ++ public void setMemTableFlushInterval(long memTableFlushInterval) { ++ this.memTableFlushInterval = memTableFlushInterval; ++ } ++ ++ public boolean isEnableRocksDBLog() { ++ return enableRocksDBLog; ++ } ++ ++ public void setEnableRocksDBLog(boolean enableRocksDBLog) { ++ this.enableRocksDBLog = enableRocksDBLog; ++ } + } +diff --git a/test/BUILD.bazel b/test/BUILD.bazel +index 058532df7..5df71200c 100644 +--- a/test/BUILD.bazel ++++ b/test/BUILD.bazel +@@ -128,6 +128,9 @@ GenTestRules( + "src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT", + "src/test/java/org/apache/rocketmq/test/client/producer/querymsg/QueryMsgByIdExceptionIT", + ], ++ flaky_tests = [ ++ "src/test/java/org/apache/rocketmq/test/client/producer/querymsg/QueryMsgByKeyIT", ++ ], + test_files = glob(["src/test/java/**/*IT.java"]), + deps = [ + ":tests", +diff --git a/tieredstore/BUILD.bazel b/tieredstore/BUILD.bazel +index bc7d8f938..5b3885a4e 100644 +--- a/tieredstore/BUILD.bazel ++++ b/tieredstore/BUILD.bazel +@@ -66,6 +66,7 @@ java_library( + "@maven//:com_google_guava_guava", + "@maven//:io_github_aliyunmq_rocketmq_slf4j_api", + "@maven//:io_github_aliyunmq_rocketmq_shaded_slf4j_api_bridge", ++ "@maven//:net_java_dev_jna_jna", + ], + ) + +-- +2.32.0.windows.2 + + +From 6bc2c8474a0ce1e2833c82dffea7b1d8f718fcd7 Mon Sep 17 00:00:00 2001 +From: rongtong +Date: Wed, 9 Aug 2023 16:11:37 +0800 +Subject: [PATCH 2/4] [ISSUE #7135] Temporarily ignoring plainAccessValidator + test (#7135) + +--- + .../rocketmq/acl/plain/PlainAccessControlFlowTest.java | 5 +++++ + .../apache/rocketmq/acl/plain/PlainAccessValidatorTest.java | 3 +++ + .../rocketmq/acl/plain/PlainPermissionManagerTest.java | 3 +++ + 3 files changed, 11 insertions(+) + +diff --git a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java +index 519345714..e7fd0932f 100644 +--- a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java ++++ b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java +@@ -31,6 +31,7 @@ import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader; + import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader; + import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeaderV2; + import org.junit.Assert; ++import org.junit.Ignore; + import org.junit.Test; + + import java.io.File; +@@ -43,6 +44,7 @@ import java.util.Collections; + import java.util.LinkedList; + import java.util.List; + ++ + /** + *

In this class, we'll test the following scenarios, each containing several consecutive operations on ACL, + *

like updating and deleting ACL, changing config files and checking validations. +@@ -50,6 +52,9 @@ import java.util.List; + *

Case 2: Only conf/acl/plain_acl.yml exists; + *

Case 3: Both conf/plain_acl.yml and conf/acl/plain_acl.yml exists. + */ ++ ++// Ignore this test case as it is currently unable to pass on ubuntu workflow ++@Ignore + public class PlainAccessControlFlowTest { + public static final String DEFAULT_TOPIC = "topic-acl"; + +diff --git a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java +index ef0cffbdc..a3a925758 100644 +--- a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java ++++ b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java +@@ -56,8 +56,11 @@ import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; + import org.junit.After; + import org.junit.Assert; + import org.junit.Before; ++import org.junit.Ignore; + import org.junit.Test; + ++// Ignore this test case as it is currently unable to pass on ubuntu workflow ++@Ignore + public class PlainAccessValidatorTest { + + private PlainAccessValidator plainAccessValidator; +diff --git a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java +index 941d8c779..aa7539f3a 100644 +--- a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java ++++ b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java +@@ -29,6 +29,7 @@ import org.assertj.core.api.Assertions; + import org.assertj.core.util.Lists; + import org.junit.Assert; + import org.junit.Before; ++import org.junit.Ignore; + import org.junit.Test; + + import java.io.File; +@@ -41,6 +42,8 @@ import java.util.List; + import java.util.Map; + import java.util.Set; + ++// Ignore this test case as it is currently unable to pass on ubuntu workflow ++@Ignore + public class PlainPermissionManagerTest { + + PlainPermissionManager plainPermissionManager; +-- +2.32.0.windows.2 + + +From 04683ec05808d63f742f8702a9bd3a2fb846c154 Mon Sep 17 00:00:00 2001 +From: lk +Date: Wed, 9 Aug 2023 19:08:33 +0800 +Subject: [PATCH 3/4] [ISSUE 7117] check message is in memory or not when init + consumer offset for pop (#7118) + +--- + .../broker/processor/AckMessageProcessor.java | 1 - + .../broker/processor/PopMessageProcessor.java | 40 ++++++++++++------- + .../apache/rocketmq/common/BrokerConfig.java | 9 +++++ + .../service/route/TopicRouteService.java | 2 +- + 4 files changed, 36 insertions(+), 16 deletions(-) + +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java +index 2140aa881..687811409 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java +@@ -308,7 +308,6 @@ public class AckMessageProcessor implements NettyRequestProcessor { + && putMessageResult.getPutMessageStatus() != PutMessageStatus.SLAVE_NOT_AVAILABLE) { + POP_LOGGER.error("put ack msg error:" + putMessageResult); + } +- System.out.printf("put ack to store %s", ackMsg); + PopMetricsManager.incPopReviveAckPutCount(ackMsg, putMessageResult.getPutMessageStatus()); + brokerController.getPopInflightMessageCounter().decrementInFlightMessageNum(topic, consumeGroup, popTime, qId, ackCount); + } +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java +index 53e172561..441f7de08 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java +@@ -639,20 +639,7 @@ public class PopMessageProcessor implements NettyRequestProcessor { + + long offset = this.brokerController.getConsumerOffsetManager().queryOffset(group, topic, queueId); + if (offset < 0) { +- if (ConsumeInitMode.MIN == initMode) { +- offset = this.brokerController.getMessageStore().getMinOffsetInQueue(topic, queueId); +- } else { +- // pop last one,then commit offset. +- offset = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId) - 1; +- // max & no consumer offset +- if (offset < 0) { +- offset = 0; +- } +- if (init) { +- this.brokerController.getConsumerOffsetManager().commitOffset( +- "getPopOffset", group, topic, queueId, offset); +- } +- } ++ offset = this.getInitOffset(topic, group, queueId, initMode, init); + } + + if (checkResetOffset) { +@@ -670,6 +657,31 @@ public class PopMessageProcessor implements NettyRequestProcessor { + } + } + ++ private long getInitOffset(String topic, String group, int queueId, int initMode, boolean init) { ++ long offset; ++ if (ConsumeInitMode.MIN == initMode) { ++ return this.brokerController.getMessageStore().getMinOffsetInQueue(topic, queueId); ++ } else { ++ if (this.brokerController.getBrokerConfig().isInitPopOffsetByCheckMsgInMem() && ++ this.brokerController.getMessageStore().getMinOffsetInQueue(topic, queueId) <= 0 && ++ this.brokerController.getMessageStore().checkInMemByConsumeOffset(topic, queueId, 0, 1)) { ++ offset = 0; ++ } else { ++ // pop last one,then commit offset. ++ offset = this.brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId) - 1; ++ // max & no consumer offset ++ if (offset < 0) { ++ offset = 0; ++ } ++ } ++ if (init) { ++ this.brokerController.getConsumerOffsetManager().commitOffset( ++ "getPopOffset", group, topic, queueId, offset); ++ } ++ } ++ return offset; ++ } ++ + public final MessageExtBrokerInner buildCkMsg(final PopCheckPoint ck, final int reviveQid) { + MessageExtBrokerInner msgInner = new MessageExtBrokerInner(); + +diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +index 02c692e2b..a815636b1 100644 +--- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java ++++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +@@ -222,6 +222,7 @@ public class BrokerConfig extends BrokerIdentity { + private int popCkOffsetMaxQueueSize = 20000; + private boolean enablePopBatchAck = false; + private boolean enableNotifyAfterPopOrderLockRelease = true; ++ private boolean initPopOffsetByCheckMsgInMem = true; + + private boolean realTimeNotifyConsumerChange = true; + +@@ -1264,6 +1265,14 @@ public class BrokerConfig extends BrokerIdentity { + this.enableNotifyAfterPopOrderLockRelease = enableNotifyAfterPopOrderLockRelease; + } + ++ public boolean isInitPopOffsetByCheckMsgInMem() { ++ return initPopOffsetByCheckMsgInMem; ++ } ++ ++ public void setInitPopOffsetByCheckMsgInMem(boolean initPopOffsetByCheckMsgInMem) { ++ this.initPopOffsetByCheckMsgInMem = initPopOffsetByCheckMsgInMem; ++ } ++ + public boolean isRealTimeNotifyConsumerChange() { + return realTimeNotifyConsumerChange; + } +diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java +index b6b14faa4..e012a5465 100644 +--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java ++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java +@@ -133,7 +133,7 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown { + protected MessageQueueView buildMessageQueueView(String topic, TopicRouteData topicRouteData) { + if (isTopicRouteValid(topicRouteData)) { + MessageQueueView tmp = new MessageQueueView(topic, topicRouteData); +- log.info("load topic route from namesrv. topic: {}, queue: {}", topic, tmp); ++ log.debug("load topic route from namesrv. topic: {}, queue: {}", topic, tmp); + return tmp; + } + return MessageQueueView.WRAPPED_EMPTY_QUEUE; +-- +2.32.0.windows.2 + + +From bcba5a8e628e35086c699852388990ba8a4bdcf8 Mon Sep 17 00:00:00 2001 +From: rongtong +Date: Thu, 10 Aug 2023 10:19:34 +0800 +Subject: [PATCH 4/4] [ISSUE #7146] Log output error needs to be corrected + (#7147) + +--- + .../org/apache/rocketmq/broker/out/BrokerOuterAPI.java | 8 ++++---- + .../org/apache/rocketmq/example/quickstart/Consumer.java | 3 ++- + .../org/apache/rocketmq/example/quickstart/Producer.java | 2 +- + 3 files changed, 7 insertions(+), 6 deletions(-) + +diff --git a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java +index 1793a83c0..ae81e8b11 100644 +--- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java ++++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java +@@ -654,9 +654,9 @@ public class BrokerOuterAPI { + try { + RemotingCommand response = BrokerOuterAPI.this.remotingClient.invokeSync(namesrvAddr, request, timeoutMills); + assert response != null; +- LOGGER.info("Register single topic %s to broker %s with response code %s", topic, brokerName, response.getCode()); ++ LOGGER.info("Register single topic {} to broker {} with response code {}", topic, brokerName, response.getCode()); + } catch (Exception e) { +- LOGGER.warn(String.format("Register single topic %s to broker %s exception", topic, brokerName), e); ++ LOGGER.warn("Register single topic {} to broker {} exception", topic, brokerName, e); + } finally { + countDownLatch.countDown(); + } +@@ -722,10 +722,10 @@ public class BrokerOuterAPI { + default: + break; + } +- LOGGER.warn("Query data version from name server {} OK, changed {}, broker {},name server {}", namesrvAddr, changed, topicConfigWrapper.getDataVersion(), nameServerDataVersion == null ? "" : nameServerDataVersion); ++ LOGGER.warn("Query data version from name server {} OK, changed {}, broker {}, name server {}", namesrvAddr, changed, topicConfigWrapper.getDataVersion(), nameServerDataVersion == null ? "" : nameServerDataVersion); + } catch (Exception e) { + changedList.add(Boolean.TRUE); +- LOGGER.error("Query data version from name server {} Exception, {}", namesrvAddr, e); ++ LOGGER.error("Query data version from name server {} exception", namesrvAddr, e); + } finally { + countDownLatch.countDown(); + } +diff --git a/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java b/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java +index 41c9eedd9..3a101bf66 100644 +--- a/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java ++++ b/example/src/main/java/org/apache/rocketmq/example/quickstart/Consumer.java +@@ -49,7 +49,8 @@ public class Consumer { + * } + * + */ +- consumer.setNamesrvAddr(DEFAULT_NAMESRVADDR); ++ // Uncomment the following line while debugging, namesrvAddr should be set to your local address ++ // consumer.setNamesrvAddr(DEFAULT_NAMESRVADDR); + + /* + * Specify where to start in case the specific consumer group is a brand-new one. +diff --git a/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java b/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java +index 2c67e463e..aac295030 100644 +--- a/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java ++++ b/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java +@@ -54,7 +54,7 @@ public class Producer { + * + */ + // Uncomment the following line while debugging, namesrvAddr should be set to your local address +- producer.setNamesrvAddr(DEFAULT_NAMESRVADDR); ++ // producer.setNamesrvAddr(DEFAULT_NAMESRVADDR); + + /* + * Launch the instance. +-- +2.32.0.windows.2 + diff --git a/rocketmq-all-5.1.3-source-release.zip b/rocketmq-all-5.1.3-source-release.zip index 4ba03d04594471b8be6af24d57a3a0770946f8d6..d2d0527ff4744acd6d30620c68e10716bbd85124 100644 GIT binary patch delta 177671 zcmaf6cUaR&^M4aMq>@lnu!~)=ps1jzXUE>?DTEMeLcj(npaOPLl(mauubhYx6~%UE z$8z?D*gN*K|CaE2@9%Lx;d~zcaK4k-X*)aHX5al-WB#1}k@93tZSulN;+B*5#Jtw1 zkv9J^+o8=>eRX#ZA3MSBKOFP_(0}It$%*qCBl;Csc**Od84JDX-P)Xi=JXitn$!7i z8&X-BW<^47@~tq+svdpd2hu%0*o~xd)17jL^sI+bvL9>yL2=A^*;Bxx@p{c-fQ1`+ z#c>%ZN$(xf%h+J<`?PvFtA^P){VG4z%+P^m=492oH0PYleRK?s!qI)Fu#|K94|mP6 z7%+}SpEaO2&}aFet}L`lpJYgv$ZCPfypL%PSmGX|1C3uv+@~~~9FO5POtX}qxeTyy z)__agoc*K5GGA!R=`8TVXh1_Z<_8czH4vb>2I*MT!n(0rxTM}}iBU?I`X=4;rzO6B zOLHd2-U%948LkP5{hsFNWWu8hy%edi#YNnlU^Oct<;D2LF=ugPFeCr}zV3*Tp|4-r zwsKnNKr{Pvt69|lk#`y=N8~AME)yrRU)D{W&r-|jlLAo7Qkq~l%Mn$y$*F%?Kz`VV z0Pm@C#G#+WZlv5Y-Ib)>lVBr=C|F}ZD>Zb#A#TsFiMwwbz+&u~$AA$zkW3!P22@>poUwF@0s==0)B zY->5^4*^ft9uBl+30ymkx_+B;K7g_0%ZC8e?^bghcFDoxzHTI2lrAONx?sR&RcIDf z^k$ec8luXybZn^EvTaBnms6K`Xl;P;6S67=QHl3BzAu ztq0dxkxBZa1Zknu(^ahF7yj|Qu_Hsq@R`d|vUChFR-BhY7CXv`bCH$9FsK3N9w6fD z##z8JLMobZSl2M5HRovAK4|#Sij&9EFzj&W6o3}B9XQ`CShiETaO$yMW7y)$kpgg8 zAC5O5Qpb-|2pGIQg!90TZSrCer#qnCP{{$2*YLZFqhrM(Z6i5KmUMn3XCdIPua@J? zYKmy%Ie{#VYw?^_Y=egB<2ct?Fo{p*uwmcWLst0&x{>-RoJLF&#?QJG&T!@nIW%3_ ztLjI!$)#ZA;D-o7s_b$!3*(UAsoFjnptRtK?EV9p0&I}gWaQRQpSQbozPH~p7G|W$PHn7NsePtXD0H40V8OC-z zL+dLX0|39f!AW72O8pS2sY3;489VS9?mXcHS+i^pc*D5^%-H!WrzxyALiBqoo4AVZiTJr+b`2~!|Ze1ftD=z(PeHqP{@^Als|^X^xIsHH49$y zkQ)d{o4w$gvDp%F{K{R*(kT7Pbz<#B9+{i1VUd?unB{^7yYtMf0DGgQW>Z-ajZxn? zP?y#=6OkdZblaSpJ%C+hb0ZRWg;{M@_7b17Ke-v%WSgZPLT6nSceC4KeHGz=9iwn;l?#6{(Cg z+sGnsj51qh!GcHV%nHm+>M16g>Zz!ot}zkCrH>|C$C}x*B+iaCqk}u+F~StH9CH@D zGtDeZ$b$Q4o3(Ud!PB;yH3RC~zr$=j0P73QG65&nd(A=s_Sh0LxfR>wgQI3me3*Dg zYIDukv&iST=3ju2H>}JLgXDaht$7|xBgD>p1Q5Nnskze3q-9==uUZUP&vhqhEzO6q zMDMmVzX2FK?O~3LqV+UKp5)WTd>%{sW*770JQlpGuelQ&waJYE<||p`83WDh0!tZ& znV$iDYvD);jvr$_j!kvQ$58Vew#B6~bCnxwRzaruG8VZw(>w#1WN^Ou96uKO*(>we zz>%^(npXlMSyGEl4Ot?-oh)1dI5xmS2z+;Tu!RJKtR3Sl?uc00E3+)NII`e?l@=U; z9ktye#g4`PP;7A@3_?#2Tf7C9d~w2JFBq&1e_6cYv$TsVEG$6WJ6maC4`M~-4~xzK z+*xRO7Wk!;i)92Dt>?D1+{`l4ua)H$pl){$%iADayLPd>13DS6yQL6F({hkyF>vm8 zLo8>4Cg+Z@Gy}9(kFhKP*l|IY(*f90W+|{R8AIkTsTu{(1~(>hjb$gYQe_Do@SMu> z1)zLJV<`dbRmNDV+02rtCt8BkNR(uG6ZD&}>6Yz5i`_FUw}bfdZMx-h){dKIShfc~ z{CuJ1P5|DXXNd;siEAz6#Vqa5MV8*|EQs_uVhJM2>m!yET-mTQ@vh}2mU6oa%Oyba z8xJgJvU4CZ_O&I5tc~7S27!(@_p9Xx(0dnJSOtNs;D@c1pM>QvsIJumz~9h@R(vos zkM*!3EDdQ}t7f1p`E{{639ReZ%_^Uzajl!xV8C^!->g;u>)sz?bqa8OaDTM^dc$VL#6@GZisky0#wr~&7_75;4}?*Svswt^s>LL$OCa_=oo%%e z5LvvyYBdn1f38(7%ipV9D|0qb8dLoG>#d?#CtkYVY6(lG*9I#GHpCP0>p-a?WtY_} z5CW48Rt~J;Hsl?)0(L7nW`%<8s*_edLF9RT*$Sj>8?IP&1Wx#(+^QMSVv~nfb%EhV zys&ZxJuUg8)n7nH*=H-1+IWAn`pBDaK9tep@2Vbbd@U?8|H{d6@Ny)T7x+zy&eobW z7@22l9mtR~e6qEE4Yc`LVEqnw2wej1MeQ*VTbIr)Pzs3I#w-HI^f(<(o>tKySkfU`dYoULZ zirW$oPbyE1OT|VU6WfHD)(=2(Hg$xx zBj|{mg01bq5PLMzdJXH@Rh`>|%&MTf=OHtdAf|Tx zWc>y#Y507%22uR&ck7$LmUp-|AS0Y*X0yfgbh_?C%3-*bO(tupFIF~wps$Q^wJBzi z>({mU0d(x%)Mf~<-`tiq_W^i{hs|C9_Vcs}06nXyi%keJt*40qtJc}fXQi=Qy>;3 zFSJ?9uKE)9Wi}w^yuHju1AJ!mYMXhW#rJD%9<^bGd4ADm1MtWEt2WtOlYKt*8|_gU z7Q!Z^^14kIV*Z`}w^C;V+^O=04LyCrtRHTDY~u!ataxS<3j&brlg*sotdo50X1kds zy}P^ZW;YhBQ`-&(^S!gtHI+3jRmXu8@i z5-1~Ytz9?(J8ZNY2*A26cDsOWJ8ZYR4D?^S(C#F_eph7I0C-uGeRdMiWWNJ;$Lg@+ zxL&vW3Wzi>w?pyh_&vK)z{%o=c9#Ko@)Nt~0G#yPE(Ms}@~zz)fNlN3j_y4&y`$|X zyISn%L;}CtWwXeizuP?odaX3$q1BxwR=ie#ze{}HZ7{4KapZXd1zNlCHUnu6*X1<^ z(tK#ZYXul==f>Lzh&*k^YhiCX&bOT6Ovd{2WF))w6agvd$^&M<(Uo@;$he~?&k+c7 zsW0ys0B;_|I|slsNAMB=m>on9$m8d^%OUG zFr5b?SI!Kc7Zi?yc+Yj*Cg0-YqCsBjzPw_ee8qd%2CIax- z3%tj`Ua^;W=Ky%nRUV2)<8JUS0@?w0cy|D}-#uOt0P`R5PJ_(`Dioetsu37X88^P@o8<*Mh82AoVC%a;OEZkWJd&zdr1B0mApC{N+j zGigkxEX(Bk0dUJ~{(Zo9y&QfgU^6x0qfP2LOZi~Qpz|_5>QBv9@pk~WAFbiv2jCm) z`E)s$36}>q@xg9U)Moy9Kx5!`zBv#^wv)dfh~Bb@e}e@Ry0`9XXupS#@`M+Aq3w^B z@SlQq4;<#BslvQt{1Sj2e3E|@$mDUB|H1UZlVLG*5FK@q-<9=)yo-DpXhwLIPvaJ& zXX8S){4W1r7qX2wKDYR$7P1w|Vpn2yiyy*p^v}X^Q{pQSNNZj=HsUzfbjFx6(wvc_ zdKV^x<3=jq^8bA`W6G4a&Zh3Sk45p)Fu{Mg?FXj=NTuV5_TPwOX807lA(3c#jK z)-b_+z)12K!84Y|KWmY>YJrG!4-w!akVZ6mX~FF$nhWfBIi*AY!H|%7Gp5*>-kKtB zLj?O6eUSs9RJp$fCu8$;akD8YPzaoBI6X<(`?pn2it=w^fuhuO?Wk(UiSF}ON!qmr z`OK8axpyfAdBC|AIc28fJF5Tvo*W4!xh_%dFA4r6Xi5)23V zo$VCCEzrQn48aeOSy*NZ=BYMRF~Dxp5}xR$EdH)RY4|b z@%I~o@j!I13c-3nPccLTm%fZhc7(AQZ~#N3DFAVD2ea_AkwLhumlh> zuPgirhFiO4Lg3j2&4jT)sjXWH`vY*MyYMAY-?R=wHvr!2C0q&C<}^Kot^j*cZ((;} z$NK|>V8LYWK;dWL*^37Y!9t_g5Mc>WpT!vA5&+&FDm*J?4LLhOs0LhfCkQ1VC^ktK zqFw*gOrZ#1i(}2w3kB)&b!5JB5uvpi>tK7XTW&g{i|m!cx#;_hUk^2}WcbQES7R zsr4yg8KA@~6Gj1W<^|zr0M5QFYy;GF_PP+w<<{I5)(6?(g~!4RfcCEELLCUD z9o`DjnxODAj0HEp3KIZ>R57%s0Ao*)xu_}dmA>>;CBPmb5ak1GOGi;Z&}3+B5p@ct z$@#7#bg*hs0}%*TT^fq4L5stii|D0zhV&|T5lXg(v=T(OIg2Wx1^0s1eQ3i$wYeqJFAtV zEdU%GE_w!(J0?=($rV(w6Z4~6%e!X(`A-I)%-?C(oaB{HX=`MRe2ErGQ>z#|^~;Sh zT2Eoo=tB!CI*Ll(I)p4AXK>3t*>*%{&Fm$6{_y{;V=p(S0nUn%R~L+NJe582yHD$| zjw9Xv$g~R&O8WM>;&EWg<$2Cn`=zZ*WPBkcV|`j9i2XV*_@HJ{%AdXtji2QOUJr+b%;L*mY)7JTaR^{aK$ zn<15wX0dmN&SjM-hNJF}T3CF)g~OO7ljfx-?f0yo6x(_Gqp@yzhAXGif^I(k zu=mc{>u(;t{(FM=+zDqUtR2+xKw~e#u*=Opq_B)1@4pc3A z3l8!Wh0l(Yc<#M(&%euBjd?@><=xDDA4dX@K zy|TlzzYcJ+X=M}EcmTJ%<;l|g^Nss)Yu~*n?f1^oy+?3*B`XW4_>kGuym(>_vn7Qdvbk1zo084-nVsk z(s=tWN&P6W$~d5jOWf9|?#0Nw4#B;EBB@0!?F`hu>p_gDEWjo!KVo=?`zrv~RZ8tSM^R()0` z?i+RFQm#Wut1oSK_dPO$+}u-fzfM_~P2cN&H6ucuXeJv#H|a!Z&3HwEC>adz&Blws zdPLcHQEw2L8Yhdq0a!O#bQ_HwQ$@6_2NU4?O&8q)V9zo z$g({mu=?j+EE)>fUcO&M@2NBVeJd5wSs4@X`y3U4>DSAnq8Tjd>goHkxw8J*yrRXT2-So9+HK2XiMHcsYsbypw5A zTwA=B_1|N)#W5hs>Cs4xZWeen6Yq3lNx$hX2Fn|{zT$d7mSerd6M?w~_Y>y&YHic?w5p5Ni9}QD<_xYd5s0hS*okff6~%PXv?`5IlvXuqgMRO5 zLJ_Z5h`}hmc%?WBbQ$}#VlK<#{L8<&yV~?d#!H;R)$XH-Edrwl|-rm8qYgQNJEO;-LlDseC zTG&C2#9o;Tn~`%3rgkJAUqEj);!Lika3ep7wZlyP_##eZzL2S3#oHJi7<|8puK-7A zDzF~_d^z3T-V0!#XlPHD9U0HR*u?%j2pD(U*@JRyRn`Qz1nYR&Ph%?lux#Htz}_8LDJI0;9)y;;5%%a5sv*WcpVe?wtUX%SEswWH z5ixMQ{c%ttpxs=1J!>YiE608<+oEQnJ-V`;yw18Shn@W_IwbI=N-3qVz>W{9+~0Ft;9q}%+Hyv~5`p}r$dt3m$l3(S3^am|j7VOU1qxJn=SM1|WAJr)6PAL~@ z%^H)T-KX{>-o8^C5Zy;}Fs{A1VSgWV`iGD0xgcr&yV4%5w{L%Ee+U>o^|SpuaAkF> ztz-i$XBRsO+A8cUkf58ij$#Se(=z;!NE!g+dN@l|Y>R81C9T-LTV=(ZRXylbvc1Hq zhIlZ-LlVc*Jntci0-igty#y)9rL!ajU~lXyi2>lo-6XWWB-6DDeI?Pr{oDK|`4ik^ zvj~)^SsKZKlEHw+wGhc5;4>Y=B{CM9Y`Z9NGVIq#ZUH7YMN1MvXKbyLpd%z}5}-(~ z<0a@|PDv7C$IxoB1ns|;rAul7PX5k-*vn=}%D_xBX|@EcG)d=3XsKGpK!fH>(CNg5 zizMi1!Ry76LO^8K3Mgcw)shI-6o#d1B_Y5HRvRR*IV^+Ln|Mai!n_XlFAiE&a=| zQClo+%O$m~rP1uSMb_vig#M(lA<|B&2Hn&*RO(8;@}+W?T8u!t5;L#@fLo}f9$a$V zR>~tAMbd#}u%pz8p;^t!5=Ut`%Zf2K&`J{Nu#z~n8R-%`Rq%6}6rC~X(npF;qU7|i@>`}GP)_X4NSz^4J^M{83TeS5 ze-1-V-D$XVNDY^CQ)p6nB~aRjd>JVPS#9l6QWP3m`a`gHkn|mE07APQ=@p`Qd?Afwkz2i#jt1IaP>CWZ|FyIy zm|)7jNYmItJcd`64ktm9aKhRl)rMun-`*h{r2c1|9OA$jKCqsHmZfpBo`cIU7W_fy z@L&uJes-ZPloSyma*Rs?RFkFo(1=9Q)@sG z3toM%*0Luo*zZj3`xPwMx1i44fh_p;mpbikGT^NGS6vMCue$DFV{q1;fV%Xr9{sCN z{~8$X1TDEE zZq{nA=7!Z?Exg!%N1k+T0k)aykbb-YB(G}=4{|oMg)1xKm&_K1FPSadPGLS7ylq-d z8^D6kjc7HNwP@DuvH^zMWdj508x2*h-N|oVhyAVX+Sb=u(`HQ?=STkr877Sz+&Q{h zT{rIzX4Peg&+>3;&ydY(;4s9{z+q^7PW5ly2e$*i9W=Pz#KWwGWYNR=Rk183r7b)0 zNH=l&T1*3m%IINvt^UE{lV;u8vkNv^gS)k-e;w$br(tlnjtBO$gj~LKJmJBDCufd2 z-zN8amwKdh#Z;@G1Ldy)I%{^(D8uZc(Gj*r)EVJkKunL)(M#GAU*e@?n#tNpybL=@ zXH^IGliTUe2Dj5)tlBZ33_jnxw7<@R)2?^uy#o+=Q|NsUU_U7K-tJ)X{`JuHQ=M}x z?$ZjrbgUyU%e>pNOuL`+&I5t^&Nc7GVisHS%zHvhR;kMzeLxEC=;Y%C1P`t2L!Suw z)p-oV8u`3&W5FW^_;A=&M&mF(w9LB=DcC}H@`jZ7n_f2y-9i%?<7-Hw{HZpi{!pLJ zY%`HVeZGLvEiu?9P{1<3AmeWF<8O>=#003U~!_$&pg zj9KmT6@Vvf^63UN{Kqz*+Fsn;(L7%rQe;Z1DXw5gqct=n8(8IjH=GWxI z8J|cNS$o#UqK3S47d2P@IiJz&59#RWF_6?QnkpcDUAloBfDYUJK{w`jbbG_<`seT& zl1}6f)SxdG@6;=as zre!B*mO+}od}o#TGhbO~%l?9tlreUBRT%;NXu|y8~i%n!|9IdwLA(6tmT52o6JP^8w6Ik`C&0+(ZUNw!<*M!h9YLRc&^zFa=^F8}$JAfP zzPF?Eiz+3r7#P4K%ls5}WUyZVJN+~y`URk3))R*Za6#%gYjglmSl2NDBS24I860pN z^z?Cx0Cc+GusYz8=~Gw@UQ(&9Mh8q}O|FOuKs7l>#RnV$61a^ESkKb9IWC|ANN_eK zfW8&Qfo61As=r@&I}- z>(?kjI^+lZ&a(GDKOh{i7qu|}71my}Enp@XbG&y4oCdwld2hg05c2*y6wni}?e=HD zL%@m4=>Ts4)|?CY3c!h%0!9Mx)oTHC4~#LS_I3cePL_N(U^g3PNsk8s;Cj^a2LY}? z$XYJ~jsYKe`8r?&%g4Gm0rXidhP{-(1L!Q90nhmsun2^OcIJWe0ro|kz)gUW*@8eI zOebMreW0fZN#I>zUw@~-DFFLvoj`ilknz7K^#TbiOm_W1R7~%*TOhj1U(_-%5C~Jx zBX9{z`nX5nT7aD585qbSKlBV-07RVLIgl2ZVq~c@R;-rJ;3U zC>vz!B#tc+spw+mD5m{1C$g#zy>R#|BmbMsQED|jggKa;^9i)b4WccYJkMM8 z7^ab{e<@2r-=(akRiCWtg(}GJ=@mGORq%-40#kr{-svCs3gpwbM+c&sL4$$I=Y;=0QrJ)f?79 zO#rrt7c>PVrfVF6(AlIP&OwPFyPfJ5L~ofeKH}FRNCChlt%HUEaNV{+&jGl$XHbz@ z&Uv06L%*;!UoXu0kmg8?yT2YigKFCSZPhslOo~Y7AoN;{-Y4ig8@Y`!fzFlMk+hf~ ziyGd_7?Km_MkqgxS$8mo@cq~61c-jGQ$zGNPPWSX207MnGevl!nimsKQyBR*PPQl= z6O%3aP(8h5(>JI^4d>jAvo4C$l@+IDzaTJp{?#u?Y&sop(`6d1_BKUM-v{su@+KW$ zP(2)MXAibtejgMx0XS#pF+q#L^ysc8=qhV8av~;Z9pgKO$+1E2Kv z`2hZ&%nL$SMaoDJDovKTENB>@{U|@ECFt}GHwVRlA#df5paB5;@1meaAf&h48?=#? z{odXnRN->q!JtA`e?})VI&@m!X|6f9d$5TF$v6@;nQcaLG^mW_$w-enE}P!`&t1=* zs?lBP=YMrqibLxboD4FtNI+lTb|VzW%k+u|nR5#HcE{5}JkxJqa0XG|Zc<^-54>$hz2N3JPr%40|KBVDwxOi4WmYkCP~v%lbBw%q$cso)d%xUhl8K? z31%!;Z57%8whHml2ZQAIxjwi((1rW>;Cxo-it)i{SNLUWa4IV_nUsmF)-)@47clX& z*}-6SX#SkwaM1BBNpLfe5O-UJ8hpGexDDv|Zu!BDz!(y+8TqBnmSFmrHRA+Oy;4}apeku4p0FSvAyaxoc zdar{6*cPY14&DLs_`V;4$AWHg`b#i9d-Ch-!|1f`yht}vDhv^l$sF)(nh~dZPgfXH z3qR~ug=4(LOv%(DmD~`C@d-AQ`yTQMGzQQYx7(2V6Q(sJmnTfCXX=O%>1-Cl&Su*O20-F8eytX>eaA@Ld6}!n2rF_PP1Omz%l0&wz+$mE+Xu8O_@%kwt z<3Vg{of$$S8RH%$GeQbk8fh~_6rhK^o*z;Myyo1}kk?>|=KQ)4^mgEZEg|Q?z__v? z1mrD)e-F6?mU-GA2zh8aNguYDMyGzqLcpBm^05#U<}FWz%mO-FeL93*7+`cJKZgu0 zI3IEtXrcT@2z~d9p|S622)k@#hX@m~ zyOC^%(1FZn^2{L=Y)EZ&40Q!Af6XOyDF9Dx6pG&S&1e-0($OxhLt}uQc6SXeU==XV zJ9HHgwMp;L^p{)TQOO}Lg zX89PgG?Z={|JtC@m5d3_xfU`ZnY1cY!r+j+RiQ~N!4`R;UD<)nNYGd-g%WhG;h;m+ zm`^CqzYO{wr2g&Dx_AN3IQ&=9_?Pj2P!)~V*qHFEw|DgM2tCeHLm%N=v5M zk(rxAYq4?{ZVvrn`aEapK1wINxN4E3h0Ke5-#^Wo4A?)-@n?>~IH!JlC|XS}`#ls@ zBfVA>+8B(g{9~aHOuMrDAmxcJDjH?8Cr=O3v@`pAC>W#;oC@8=1}DRmvQTHxuQp!_ zJq_G4^L8k@D%QIqbUWyUX3s-Ug06h4G88>*-t|LhFlcha_t5VknirYN{s!WdSj#T3 z{l@6I`-QRzYzO#FBm)7}LM%g9U`9EW6u0ht33>_50Bc(Ai zc4d@Yi;-<&wYeZxMqd^ERb*nDAOlI~u>={aTBe*Ry9S1Wfiq=bed+y7nHVH>jzrdw zZ*s~Rf13t?h_y1X?DT1^>>+T6S6gHUz-mFsZe&Z1K{f}_us9%_4!|pp%Kic!KK+EO z9@u6q_)B&TfR~iZMgbz<@5^+o2a?H8WMD+}dMbMWqC@MKG9Cyq{XfV~gNWJWyKEWg zH|MP7c0l%L0(mV!MCvFP1GYVD%a^k*cel2@Eja1cu(ce>c(b(}p1*1@M^`3XyyR%( zI^J6z1Qd0%yBrl>ob#K!v+4Q8p*PeO-w%-MS;58+lsmhy;8hWFR2}DRwEQs}3kfYi zDK%V=m(K>lWn7{>5(KkX{EEtw$C1wwaEk?#bgIjM5GgUrN%j#K1YS&80Ek+%n= zJu~Ig0r<=exf?5}!EBz~8w7$O%j8Z%lYv0+ooav4W_c>hRO2mjbTBQrK)x5q>sBa# z5BgurJ#tW_-JsYfF91Xu9FRMJWTwv<`6bXvT3(VXKqOdRE`JJW=iZag0LpZKB!3Bd zTI=WX1*}TSp36}Yyz{T+plXMq?7jRh7z#?i$iccx#8>%pmc1g0B9Da{j+iMj0NCG3 zaR*4=*H(dQtwstI>ws44Ix6T1L&kAG*HVB%dtGhC7Ql$SuEG<5*ELX}S6K{=6_tQ( zLsLaFK;%^mMFyJ?{4;8kp|vu4k^_DrrZanmG~;Kvrz%{XBdw%e=8$1UUWL-SXG}hR zr;}nf=x)hAidVqgy*cJ^|hAn5>WhAG(mP zSjXx&Yl`9sh#Y}giirSxZ5 z-xNInkyNhoA@JC-7Ro6A+saO&Cdx+8&bysY`78kk;-7^=Z0k?m8b%f zx4-fY8+eR^!@=Xe#&+y$IMiU={HDCP;Elbi+Q_asX7`PBT<~!;LqUkLEz1IVEmwlI z-xP&%5>Tg2xUvrb%QQ+<(d|dHvLC=6qgB2Je&;_{iSnS=6O?HFGQPy<&PCi{ZjQKS=G+jBKff{KVD@0WiWe0L_h7!!s zug*};W_eGXsXPpxIq$Pr2^L!4E>^Yx^0!{D^aK7rW34iiT>&NawxZy2X{(ZM-7u;x z`d!%rfGu|`7Xn5$7c18QaBzvT8%RAjA6A0ktU97R1RTZlgc22kwmGA039x^=prps7 z8EGb8R-$)~svRBKHA18IJZ&QaUfz`bHsJK7GbAB z_9^fT>jg+Fx`agmVQ%<@H2`2!^4o@b{QRJ4`d7u zLS~c(hoOz0ZHh3OlQPzw5FQ3rP@Ad4J^=RGM29^CdTJgYhF+kXIxZ{@U~^K!HUstT zOAVU}um@*^O$6!gm04j|*~GIdJXfD0EDv+4;UpOy6$UV>Fvj+5xw1VeT@z-a_ITB( zFtqBsiJhSTH$IeynHV38IK~UGl~p)(lQXPI$)YgR6UV%ys^oo9SO)7xGjpLXe_I-M z5$LjMUKpyAYMCDf=GedIhoOrB4L5|@0MF>UB`gsPg$H+qtz)@PEDGBRJfLH7SU1yC zn4wdCPGO8{lkX3+BiE*6h)iy1wJr%0f>tOvj!X6)4$EddfCL^111IIpj)ols6R`_t z!ioX?W#_}tF@%nn!iEFAExs1k7=TaT3Pb1KWOu__0xd6nib9OvvoN%kzxOTdtkO?m zjoHahQ9soMu!q#bOvMi9WQa|8ZNqmn6?$!@wUr7j^d{S>EWkCQ?7AvCcVS}HmqsdZ z&gDR3RRyp^K{Hin0RGfcg`!WPyUK?h7$d5sLW_>7-l{8rk%9eH z=w{FwKUE%xcf&`h762meMytF5*f&u10PyW8Q#k?bV5O=K0B41(=s|5}ymgFJ?FHcS z7*!q6q+Pt~EC4?ltI7mKQYWg=X7h|mDzvV5Hcd4F&>o$kLMiL#>8g0pWb$kky4gG? zTZN{{jTWlVCV3iBfm+*!S4*Lk87o!ja2Icl${C2$VZ90k+(DaFV}LljwyDrHv=;@c z3@{)}*{!+@)-V!^RRw@{`x2EB5LtIvRTp4iJf=DXz-v#aS^;qRX_XcTFFOxA$?m^Y z=+G4Zx=IMvMq52lbq7r*KUSedR;OJQL~d+`_?$ty6B{Z-E1zZ5191d}mgcOZ$fxg5h$iA5v7OLE%9_QEP^VKL#x>9Tkp(UBkfe-vE(uA>ji6 z_?$faC>UJUMuel#*d#ifKJdZVrbS%%4%X~X;=4>r6@j%;;Q^Gv}xY@Mu3qW?; z8R4`j05cLL&I$hrgiM|veg#|~R&Ea011G+}JscePU9cm3257P6u5dKMD-7YNO616r za1oesC6 z=hyHPK*~I>8qIOUmg<>6Ra7AUyU9Dh!m^A9x9=Bb~1Q4s8Mb@(ODe~xP0KM z-U`@R-9SwjahdMfrit3Xy4U+A>Tp0Ks<}EAfFHF|-v;1I9%>NUiCn3!Whm*O)_|XQ zdZ|&Ocd9GYnOAppK1-xZXO0F=FZD3?+XiJHT=*6$|33^mWH9}2z3GgZyv2i zV`tw$bqv5h5~7|1nxx7>zVta28od~e!WvLefaHm=7d4R#o^VEv~c5)6>tTR#12iP=vJ>(cxEmv;_#W(|2 zs{aNJEArLTLEreaUVV`58^<@OQ$f39wy3)R(yMo<(eeJm!YW3X-neg%`ZU`jDOOJf zq+1+N2LN!+VKqIe&ycS3hx#Z>`qdxmyFi%olj?5({Nk*-9ca=0qWT=b7G6<10&LG4 z>SqAktCt2g9cP@)%1KNlPygAq8<#euY6ad zVR1A!0_|l6T1EU0h}5x*xC6kp!if4Ho_CQ(AP@3$j_?PH+2s;}0?6R{5orEb)F|Q+ z;ACslh@*gZTC0erpzj;nMRWz&+)fc_Wot#3i1VPy$=xF2SR&Q2{LIvd23$k?-VqAW zrPua}K>Lbc`$wRtRyHUCjqDpoM8pH8nvX%MCv+&47zPJLj0Hcr7#e}XfNNMpD8Qbj zj!*+ICn_R^1y?I0KQ;m_1bx;;@YtUi91|n{WWj_^lpPG*2@xoVj8Bf}$aaW-ERn>L zGmOcKK-;qWXGJ^&*gNM%%m(1WIT2{eJUKVwB*6Ax8iA6p+E-Q+t1Ews0?1x?G+u?|zpv90sA}ZKUVHk2MVgQ)3 zY`Pa=4Tv0i7_kua+VB?<13|PYeG_p2OuXm5k3iGR%+C?%bWtCU23=E}X{G58#JOju z*#MecD$ulH*H{f#oi)>(*mxb|p+UiOxu@oNT^4)sNX=p(ylH5LHdJ6013PRSNc)X?>HhBisjYzAx(OVxlQoPVck zXpsVjM%QT?wEYp7rP&DBwx6Y028fi*)3jhihv6g9qyZvX%QTh%yke!MF<6#6ze$6R zW!~GWISN!_*r7p(OxN$!#F*YQ3YBL%lLwE&{Yg1}@{qieXS$MeYxtlWm+ywGe<;?} z1`^dT(ToD%j8cshfHxh{bOhj6e`wH|Ca05{dcc$WlxbE2m-u*I6U5qHc|n7+blc0C z)oi>o?nfG{3wzwwh>7sJhTUQ^;#616s=`URt^wEcewNJ&Wf(G63NRMSqD|Ce`lFG$ zu_!V9?3jC(&z_SpqR`?m?MS62(}D=2Gwn&X3o4jcRV%BC43XEH>PWJ0`LicujKpZc ztp5=cA4Y$WPSesO$0XYNFYFsw(o;!ak;Kn6771E%9T`zm#Df%IeD~ z<6}*G_6u#AP|3iXL!2iXP!MnJ6Ae`m6FhjYHRuGQ#|OkP zStK9Oh_Q;yW#>;uOVQdG-LRz96)|{`4VllzGjl}9j|7MKd~c*!nvrsGq_l?mKgO%0 z6zAW{MO4xGm#d?+)Q>o_Ez2S;8j-|8x7bI5D*%HeP-{_6kzT-jJL^U&fsPwDibOlY z3tC26fi?I|Z6oKfp3$dWBz>8gaTtBaNFUaGKAa?E^xe<(!uLnl% z0ocVuBIkm8+GBzvLGs24iPVC9<}_U-T3Pc>j6|18dQ6P`3L^8M)JSuXjwv!C3xJ{; z%#551zz63>js&zDEsR9P8n`PW%fQ*_yIUja(Mu-Vsn{6_u5ZmPj6|0;|1d;?QbRF& zB9*`fxA#T50%7b*BfkUeT1O-8*`bAWJrfDGxt^YhTnZGj@M7d>V93u8BX#V^Mv|T) z$=#mOxWszpyf2Y!K$_CSDylB%Vp>7ecEDe2X;e#)Flk((5&`Y!O{01PgY{?~1wz2R z)=?XPvX{1t>RpF5`l~@v;84%f!BOnA^7)eNuDg&Kn_>?FDz@Mf^od+dNujfWxU@xp1$BevpV)}f+kZ5TQVP2%O|=*Ai$X(i(ZQ&6p!`06L_G$*x!cL8bdUs=-HMvQ@~62Sg&u%5yBCGdikm-) zLMw-#o<}_gjJ&U`Vw>^1%#TQ`&Hs)<*E9Bgi`oP_K%#YYJvUaG{5H|`03#RLM(;r# zv|}{w`wV||dq#tyv7~48VlW1V4~ez}q?1QPzXen0$h-;-aDMtkiY?nXBOz7u&rx|bJg zvi&YGU~G(bjd_V^G>BOPqGTQS7<7aou6@h|Ao~Ze7;9iMflmw_shEKytY-|m^cvDT z<|1$&-+nP4St3GP|6|M#5GM1?V>3Zl?8uL$Q!^HU51HD*j z`WD5`Vo6sN#iB)FpMA0G0qM7;vA0+UGL#>Ql>;LTy%76HBeqGm53y)Bebc8{aIm!~ zATAx~@2Pp*6nmC*ic?%EnDhBHj+@IOKW!X02pF=jd)x&OC7b)iwFTZ92000R=^qhxo8)FUh~vL zI}l*^_SDj;AQQWed1+$+xQDm)2CzY^?%HUU#)j_NY(S%LFD=@8c-2q)mzBv17`|WW z()6NWdLvugU293Qzh;^)&DDRH_8RDx?M7-FfmA1ajMks!{mdBcI-sp7!P?*0id zGFJA^KCKnhaC4PTp_-DRtm|!F=lXI4uf&7LIcv%8cx=5Rg=+3F8o0_XFX4xUGe`}o$@+I0G>^B1?ewqdU^AjnT zG+nI)gR$-Kc#6nxa2(BxTWB3q$D^#A4BT-DeJ&SfE|Y zTC4T%S{}=Vq5m%JS+=EWLJf+w;BwK)Vr>Q>BrDNAW(gU_ltS>3BU&yXqWD7#4kHli z#g2v-C$xO_C*=G$t&^d_8SQnp@oFhrx>9e*5GAxDzH!>NLyv z+0gT`_8!})!SlJcHw!lMV#s@?rDwo@?Sj?cCce=YvEK|-1{4u!?>Wa}GZHuarUi3E zDix@1Lv5JHk~E~7=}@Y^*itu=71Qw5Mt2nuxzE#~{mg3Bx3kv)Chywo(9NU&{=`|g zfzhMkiL>qu+jzBamb!vQsy{{6*KK8qHEEzbWs2D6Cb})mFRGMp3^PJ5DQ~6&;b~TL zT_udbiwNh7R$^GwN{2#JoVyO4ZtBrqH_i;jT_ZIILwQFXs*1I_v+f))+)N)GJ<$KF zYZ~Za*Og07_5|`8GJ5GQnifZDNb93J1V|3+ulo+bbi{oP!15uwR{$&#BNjfm~H(XB9g#k|br%|_J9ieQh4umyB<18H* z+zl^g=+L;nb+)c65a;6}9SDnty^D1%!Tjp|3f&nnk-xBBhZe41ZPslEI@z{O_Xhw^ zEr7jXdZ7-bMm=`x=usS|&wSYf1L>B1y7fS@ii5~y4IK~b3IOd7M|EOA#OXLB^8BQ3 zDIk)2M)xN$L+lk@IKZyBruz(*HlN?sEn#OzRoQq>=}=mP{DH1j4avCc16?MgxvJ+J zT9fhzx(UoDgXW>m0dPUzbnxYpmstt!q;gTbt?5mBOKI`7-K*g8gZ*}uP zrKQ(jbfDmD?pIxBwqbI|A|BjjnqwJX2G&7h#PLH+oU@??Zj7pKL`u|IHNDptR4*QE zRDP=$zZ~@GO-s!_ z+T+E<1G^N*#J2%H`7t5h1IXBEVtgRj=UqK5zAxyEf6s{@0HWOR-1v0R|L-k{&jn4+ zSQYO9CiW-t<7a>-mD}P|fjF;s#iI(w%l5_30StN_jc*EQYtO{<0E4qGKqAd<#2*2B zS6|-5F9ycR{~8aL_8C{^J;(#P ziuHf68_`7EP!GCCVMD!vU2-8w&GoZcrX5@8zkqn%ptYXf6#G?hW8@h+hjw03W>$N$ zeU%-nNh404Mz18@r)F8!5WSDI*KcF_N$a3r18m%`iyj?c$?dL3_1X{g)_-BSCnpB# z*^JvT%}>7$sH^h`{d$(YS0nVUfIXK0J*p`?GE|QqH|e3$(+XL?Dwi~f)Nf#0yc($= z1XhAQb^15Jt2&Lc9y-#@+Y!% z{U*SHbEbY5;P>WiJ*wQ*Vu2ovC|4KgAF(GrjGpy;g?=)tx}=q~e=s@J&{g^{pywm` z(DBylBY8X4sLYrO%GUi?u%9*AD=n|>qP;FWKB0if}on;-@>9IX?;8p1K_ zg!y1|Pc2Gd*Ln=i#0hzz!K;o5p!kh}S2rOA=z4MEghwoUM(5hvA|aWzd#9EOpqjzQ zmI)z%7ggJY=B(?I6`l!uSp`OPOh9q>RM&**fOJ?7i2Y}u1auN$h+jf1AaZ(Gf{c}( zbRCnhkLAzOKLOQj8x@k^2}mzhCe#6>k3}Sa95_moa0=vny%Q4Bz!u@9)P%Jxdo$7! z=u{sJw|DosGQNs5jnpu|4tL_#{y$n&!a zDPaG4!p#H)h*eK25~hPlPmhG4(;y9=Cd>p)?t7K+jZMLf&Xd~Bw#v=g=2|5F+q)}yu>gN3o0au zU>=$&P5b~x%Gu6|AX2rhl^6(+m$@VYXYJye$N@9Rfz1+!U=!qM`&W1Lw1#?f;_99V z4rThfCoX37^VmJn0<2HahbnuKY*k{X8YTzX-4nr)a&O)UzeYh)9G3_tbuw+@Log7JN=yW&DF04O zMCjn$%&9qv;P{AsZX!MBz>p4^ zpLiI6-4`a(y8#S##NxyafWNvc6XyU$?OT(mWoTq|$xozzW9i>G`Zu2bO`v}h4PEk+ zHn=nBhSwL8P*q*SKIJGXBo)nz%U4pgx=^nu-qB!?PKGgO~Ms7R6$-^wf=$?J_y zjs0Dbl*;m%buUQ*Jj=M^c&=q~UA!V`~h!YFgO~dy)iF)<#d0E&(a6UnZ?%RZ{XY z=?YjSTKjjBJMgW0Uz3tq(u?AfH?fqPa+2o*KRj-k>5xoC4;~@*e!WG%Pg;HGRk#_wMw4N%1d6jC+}ft6nP{&0)cOMCbs}% zzJqtNEzpB|pX4v>5JI{PN(QSg?*}EL;@p3XNX`bMlUkM>0gkxeib@7cNK2!Wy8*v= zoS00j6#N>_jIL|ED4Ca*TpM5JFy`Haey@x3@}IqIBtyNrZ_!`=Ti>E6|Jt|epGom1 z-P4l61Ee8o$zb=nR(kSEkjecyH<=c^`c)AmBPSUgRO+}e`5tiY=F5|Z1LvN#Hdz3M zqfHx<107helawUisl_tFy@D7iypr4#T&l8oocsc`$bFui1w41jo8&ZmmXYQbDc6Ae z+uNn+fuApQNNEOWpRSX#7=RNSrW^-+`?XI=XIFHMo@5LI^?yqlZt`qjX`SixY~SIu zCPW^S(vzfq&vGPnd#8XSwBEf_;(!Qyhoqp>_QFvq<{<7gQl)^UffuTjXTUQa$D}j_ z;M$2P=*nKR2`M9hFV0Fy0q;=yO-i9OH#H2aP*3lC$=Vy*{VgAF?%O>AO&g3El5F&=}pLr**u^_*G(z& z0Qvn}Q|6d1QJ`o=$xkRqnZhd0_V<+6fSz4Z$~BoH^(0tFGCdo|zqg@4Y|nJ|C%XS65fpbnf2foqzl`{eoH; zzEzzvDln(vl2z?-rNN+2Q~5ep*rnA_S+aTUoTf7WU+I!|Ri%WWPn)R{%(Z>e3ailF z{L?~I{+t_>ruK^2f#PZS>laoZN~tnwFIE0rIy9|`%3vz3Vw%dTbKOd5A60ZTt)BKm zt$AWbn%d9A95ki9QE3H_j7>YN^5=yU)6`BL_s&iW+>$>vZG}n_lpRv}b3I6V zq4wvwY1rT^TP&!*AWdZw^m#$rD3xzv>|beW_xBboPs39dR?(flCheKZpGR#-TdY!R z5!=<`8Dt_NL)3iB*n+{z-eJ5`rnm)KrHaPqR@mFRDPTBDRE|OS`V()xb+> zYI{B>uBFYlcDB#};#xsYXu2C5zm;a)M#A%3X(Ox)HvD$l80&(OKJ0EHpx}O*?tjcr zu;XcERrO+bgVy}--sabF5kKriSYLw*{x4lk?!z>dwaW&N(h8}}Q98X(Yp&wz^`BJ# zto@c2rb0KfNpGw2=T(K$H>v!2fMdGNkN3!f`m#~lq0~|k^Dr$83M!`C!@R!ZJR$J4 zn@W>XLYLlIWjH>yNIKrcwE6&dg`}&T^z9XzenQ30MiJ?1@2z=ON>_XBJH2Z9KPn;C zzFE4;$&q)>($x-LUvHUSPsNb%HtDe{e}?wyeig0CNRh@C^C4#Nxac2-QvBHTCDx&@ zAD7-vWkx@6dU~G9sz8bP=_+mi`T6O%;9%wYC5zND?7TSLL8WaOxIA5L4{6+*bVrq$ zU*5)awUb@fccdRu(YnnJk*-yle;1rjU!h{jgsbUl3*^=BrsL@* zs}?lsQM%f}fH5!9)#i*>-lp$XY08&>NmuFedwxyFT?VTh|FO-$S(a7Z6n4r`35F9+ z8ERY7k!~4TDup@dmC;J&&n5gb+Nk_Ft7wL!%AeLzjK$W4DO5fqOlAC1 zuS$lu%Ab2gXV|KEWO(C@jw9S+<6jj^?in)@RkCb9E@POAr>;&?tH~LYGe)SWE|Qb+Mx_#h=Vd%r`E$UZ8Rt|C z`MxY;g-Xx9ZFR;hl`Ok%&QLpNzj9l~OO)o>%@hWa@btMB| zCb0_n*0(ZL7C!X1RYQKz{fu0dq8ERbQCQ{AKflRXtKvV$PZ@Z*)XI?j?-|RiVtZ3B zgO^HKCOH^(s5rHg-q2o!-ci_~wsrP6&`?aJ9Vt@Mu+++qEoVy_2C9^$V|l|$l}dLH zHx#iRi9u9F!y;?dFDe?wt5`k0rs0IjpL^6d3{u(Dee|>8FKg8cni=v`;MQ#o)~?>- z(9U2|aq64|1D-~*YC&stGrU#li=%rQI;e!gU;PZuD!TU!HmH3{i$)oWs(b}!MjM{1 z9DY7M!+=*WqXOf_e(&oQVy7#F;NIm)V#1xpP1DhDd2ZBPr5CL0a;Dmol?8vIoL z9I@Y^cDiBV5yLMkwC8EV1(k2+;{}7t`LSge4VzS)-S4KMl}q-O3NcpY+)yk0k|Vrs zY$~hy=_lCt&QKOgKR3A9<1O2t3jG8xgN&Zg>bb#RlWdmGcy92KN%t{nw?dRQduwei ze65tFwGj2*8j8Y+idj0%Qb$IFg5d?%PSO^2N9!~$PD~n!PsNTA^S0@f6{Z#G0lCtv3OpKS2I#m0X*p1rHBr_9QAmBtm-x0K<{y_xnzi&FA& z_^z=!%}ie~elWzy3PPm}^kae!vO+|DGK6VXkf4`EmDxTW=sy}6CbGi>6=JJzGAr^z!2vDuY4a3hg zw;IuqYGd@5=9*nARi~52Y?10B=#o7()f@aZM)4E1xWHd?xuzv|9VC60e+j^Bglif! z@=25hbAxKOJZp@8FX&@y^wT)h<3xe2G01+&@DJkWGN-7(f)MybxRY3S!_14ri}-zaO3Wy8l}EWw;UFF8YAJi(k3w&D29fTpepFg zasiK_|43RBg57!XNlG!)X$4dIgBU6x;SXX+({?3G+6rblC=^peeTk&)twUzCCwi{? zDwa0g;)h#{@q?@3g??zgjtSal7d$KiQ`c~BlG7h~VN&;vjOYfxc^d;P=}9`@8>LN- zGHDz{JJ{G~2QGGk5yg!19=eqeV%txOm^ZB6I7Z%vd}GMk>tqQ;6t*b=E95;DoPHY*Z;Uo4gPNt;^XsSjXpLKg{DCNBnbs z$rP=y**)3|TKhAXy^$!{<8Lf!@rrw3mKS_w$oF>wazke<3~2!hk6fu_ql2vhMpw%iq5gA#IknFsN<>8-Tqr7h|+a0~+Rt&x56w0wa#f&8^{-&=gU?8^J z`yv=o9-bECpGKRjSQ|dYu{KIXGU?BjWVdEvBk&}}M&42!uPUgxT%7kyltH{3j(WdU zhsaQ42t24gPHYRvVIxamIl9&mq=ocZV)tyb1ak=;f=>Qk0?Xi9M?k8VRH)ykNA+Lo zGif+{(DTltYZn|Ib!)%`m7#)U*i2SE`v?1+Q)X$zD6ED2p_1Xc9=p0m4TZEQb%y*= z>}x5_>yoteFO6CbNV%9Y*qxj#&D}_)?44#K9~U%lB+-p&QZ3FU{BjxQl*V$T6vUQQ zRB?7$R%1Eb@(Fim!ICaods*KLx~gN0f! zL6miuPNRw8ByA||(g%cKs<-s_me##FwY4K!GM(KlHoA|(6!xvy-`d08?nX!0{fokZ zEyJ*+bEHUKPwZifz8%Z^Z9`_5W>Z^1OLPj3nOct-D+Zg<8>KDViBb=U2v_9cJV{YW zk2xNRr$aCRq+LAKRQU+BYiN?F7SAq3D4LFyc?z|YdP(rfkw$UWKpMCHg`aIfB!>L# zej?`qp-XHkL4`^@{s^955wrg`P~b9F*%Xr@G(ng&6x=HtyFmV6MRDI&!nDPPG3i80 zO`L9xHPK@XoUY6xl3L}2Dy&r|L06c($Hq(M3j7vuB051Z!!4wcD@?AcuyjLJ%q?xA zh&V%?Dup~Dx*8uj6nl;hPPIe$#b;E*;uoGOf#*dO@{qwaf7&@iK=NxAvV%Dz6a_q1 z9kXniEkUAd@R4J&&)6RsYhbZn=LpD)8pbG#Blfj1c|o}|iZ;$^jzrniGd886^Jhgl z=U!0M_NZJjjSgHcVh~NMrAV{678+V_k-%Xs+QY5`w$4zsHunvA`06qNiO^`A;oThG z)KM%rMWL48mrIbKu|{wB*d@^g3YzjPyDmUCAC1C-2d)#~g0aS;aI-$ojr5JLgZjp8 zljtit0rl;Ah(&l_$|QE|1pizY;lcYkob$z|xHaYyjo8{FB38~CIm(a*7e&MwrcJ>? zK>mj=D%nn|hw#B05;(?JQ^s0pyZXpQJdkkDvv5Jjs?5vEX>?Gu0Y=~WmlAkl17n0G zGN`!c8ZdodBuJ}VaQH^OE86#y{jy2TCdxcf55_}?VJX_Zu zP4zq*qi%c6xWWm?>Pc|_X2xI(PTjd){L7<%3e3a0#|J*B5v@{XpJ{KeaOR~maHbX5sSvl< zP(*2&T$X0MCD^F+RsUjtYY$U+;@*(^1w&B!IZ!zp9~Y3D|Dn~v2?~u(6VR^9QY7_n2S@nL&ejiZ zCUCzl)ZnDpp)(e7%vp&(f_+?VJ!BC=CEnE;HP62yfeky2Zm`T@oH(kZQGEZ~3<-zS z*0y%CfUP@RPUPiajb2U?7URi(ayHyj zT{L1|@Jy6C%a$@|*sOmpN-;>3OBpnF9&JW$Ci5vU4W4aYh-rgG4uj5N17=zV{3IVTlw8FK_D!SGsc-kiVcCa)!4ORc_slWi37+B&IteRX#|N-4SZ4 zu2izcx#+L>so;#=Qk7v4(F5zLb!A&O7_ptN6wrWk+pkywo((hgsN~__|jcr=}Xpm zHj5PCh!kTH*_4Bftos`pd0UF~?`>NgQj034{7d`W%EtoZs!7yl^fp$uIG$!QJ!^~7 z;?^^nA$>4YyC@N<3a)*5Q$%(}^+lvX0}%;_<9!vgONV}l+;1Wxc%Lo8Xb+3}@#Pp= z=A1p%RtwkrVdW0*B*Gmm=2C9CDX8k`Kv7y8g8M6$FFW=}OlrAAAnjGW=W6=yKEyMxTYevk-sBXb&|kQ zj9JF#?qH}d83eOP7<5F!P1Y20{a-_i;{#E%>rr`%PzYbBuwcp{%u5;4==vG0fLz5(V0!MzNuzX~&dTqErv+v%B|0F^`h3<(xCD z4Ji}=--jzow0;=E?jMwJ=x}3cS)pS&FJ%Aw@mMqDjM@+CahzWW^-weKF zOkG{;ow!&RR~iNk#ZiggV3ci{LTv-)`;bcu;pGGLbRGfPBx-~ zpIQk>OlLb+m@=l2mn>eWfDJQ|o8DVg0MkczNV$CvO^$U6EK=P~V7YCh&Tt?88T62BvQUmp59G{Uat_7Yx(0 zjEO8~jHehr?8HV5hxH|xv}?|7+}=p(Z0BI@gygZPT!I0a~xLn^j`&OkzBjdkTG6S^kb_PMPD@-_Ki1=wS){UQ^anw(?b6V*#1^G z3d*ibP()}?KbKjM1YVQ*BIEQj zW4s`CG8X5Y~q2K?Ri$2~wP>`dd46%2ZS^ z=mJ@g<7Qvnniw8nk6WnI&>8V}n5HB+K1~vuteJlBiV1N0PdVGA$jP6MSr}f6h`1y~ z(jhZYn(|(hx&qFB@u})ryrk>SVcm|aVK1LE_p*UD*?a?uDz0xG1=PZxLEYeL9eY2V z2`PeM$sd^aKre>;0wrfE=Fy#JqO^8lL3*sUy@&Ol!TYx8o8TPu%_V=vb%p$p+ zXYT-cIf~r=$w7U0up!DeCPQi`dw=K*9O?QC&Rk7^6}GXsfP4eq#YR5G)reZNP+GsV zAU&069}I5U5(R8_X$oP|_Asp%THy3IpBR&`=gvk8d_pBiZg=|t2%E#T5Z7)Fa&8eK zC;yJtJwZ$LbJ3Psl?5bYE{|Ml<_|+2T7NZE57j2&5yAu#DZS3;{3U1DIsLSy_7Mi1Cy zdqKzoEYNe}l3?U``@+@(huaI#l<>|>RT5;h#MyRg9*2Ba`ekk4%uxV=jh z=SQ9|K`lMEiBbo6{?a~BrhoaMZRO8vqvYr z)m;B}4)XT=lRqQ7!^CzDfwG|QdW!9i=41V}D$bB(xDxN+Av+sD^6B_U!X_a`}{<3>cFnuzG3 zT&hD6YtC;IHdDDUU^Qlj>sldD)`x4NCA2G>u#n9f3Yvn4;F6GK=}Ra304Uq+&6sV) z_7d22i)7iO0Nb=?TTtJ>aiV4Rfa_o4nHietEa}3eeJ%dN5wkaR&T(*+omr`s$e=Dz zbQ_;1k?(%#2VM8#Vy1M3i`uBSW_F33jT8`E!*Ya^JNU@ky#jVY&N~tLCtXCu?N@To&YkF<2jiKvGwe;kWxrax6#Iz2 z2hkS8F0|$2B*t}x=erbR0H>phKz}q{)To2iyLrn>vx$(bakwp{2Mjy+%4f+yvI7Rnz~otkKEFILe5ktTlRC8b}faYwCDS=bkh!q zZIC?_{)$cx_ETheoNDAnbgwt(N0WPt5JK|vo$!UraZ1V@;lnv;;sua;wBNG#xO#ef0~f-IdNP1R<)zzdg59b`m1s+^fpJ)AvZRuo@kp^@WJ?B!13($5;N zOh>T0%XhG7Gp6M^{P??77ZG2$e2y!i?&e%HOh1mp6a8*Hm*3YrQ#?&VW~@AqX8cx9 zK%$?x=%M96M?cx_3a!<)ZzR!jYN(?VJh{N#PV*D*i)hO$1s5VaV?kM7zsO47OrkGo zu}dJNk8*UghL>ML_&csXn2X>KPYqZf8O!)NCmPWuB|))k#(4PmGPjCy)BKAHUYe!H zraOkijjO!Q2|2-#?!B3UZqVWiA2pG5$`zEh94@FYD3$315i6}D!VBtN#p2&gV_bhY zwbs$cIwQMlsH*E&hD@>qX1z5y)SCOZqkLpmcDk6K9jwy2xYLh=!MlWd<%f;Vc|al^7NL`+g!w%FR_G0Zlkr8a9xDeU&3L2AIqaUjmtDL|%xzLJ>p z0#Xp~!1q>}S}0#d&6cf?QGK?fn4$tCJ>lU> zqmVIAP^r$HArmZ~!=;!oO`Bl$w_Y$(o9PUf9Z0hQCBJ&T&NCqM*H4o7!;e6`o#{o+zg-8(-E(LDUa+VDFQ(oy8W~eC-y+ zQZ&rod&lVqee?r?fW55EQ1;Q9L3JoS?6t+YiL`&dwGC$h!&05(t(9F}34T4*$wPLe zj5_P=))FM?18!r~d8hcwdb~sLgvBU1w@jxp)=Q+3?-8ESP7dSMqR~#y@c4^jMc_&{ zR_llNtXhAOrx0s^jI8{dL>j!%$qBX$RUGwg`4RIyX~pHwbt-H<%HNjfq|-`o(lmkT zZ?Ka>)kR_6Cm#8<61o2qM*hv;l_o@gR&+?QXR$QXK4WR_EtJ4}&N}&6&uJ=sL3`&e zk#Nht@RQ>-acsU^BK`IicSk>NSDd;T{}r{|R&adbn;o+Dec+^pm~Uv+Uz;WR(jTB> zPJWXVl`Rc5d8~+_bK4|oRzG$Mvaa#f-_eG+LlO>0gl^E`GoKK$O0>1nI)lv?=dRLm zA$CDwwsS249NRg-c$-Xd)0_^2AG68yvu$13oEV36G$QkdRE;8Oald8T#5OjB@(JSqFJ`nXb0*i#dtjo|lkjif!#tW9n~ad-_&!bcGd#GR2D;bWU!< zpUzq+>VSF*v2uP z16S|zeJVn#J0bEc-K?e{9~tS_LCh=0tjDIGY(jU)FAA)5MzLWuiHFOYA4*+wzq8Iw z`XwHoZqZ~O=v&m$XeHDYJ=J)h8Npa60>5dvr%2^2dly}f79CakJXaNFX@&{etVN?L zJTr^Yj~>=r6n%Bb=wI`!bUCG5aIIc)wt^O4!`#rMqwgdfY%hz$W@&g*Eq_5!)5H;8 z7S3$XCRlhbMV~3Z`n6G?i6>YKOA!BBIGC&Bt|($3cjRi8l5l~YTrkijwoU(x9YTvP zF7giHa)>#NZDwIu>yas*grXR`<$)H~Yak#oo|$4pLKB`p8jJbaSpTs?*S>ArQM^@ZS)1fivB zT~RFll#OB$9AI)$g_ggIqL%(!1f<17mjIk&^A#;Ja#S!PeYZ=r#5{4q?VVzZO6&DX zQEBh?GSkP%D(yuvRQ7hipbRTb55tRdlZ=JHN$Po= zA^3G3bF{afv3@Cm8GYo46AUiN`vxjlf~~7O${wB;H9EmRwyvJASGlVfUkcUEIxpeQ z;$JL9q%P2*G}?dSiUi3ht!O&7l}0Ul?n;n(KCXJ}6Hdbe6c#!>lBBs+hUe_Q2KU`V zgIwh;$hIdEd~#Xd+SAVTk+Nv$i{}#LL@`$n*)=sf)it6Va%J8~xI(2}3q#Z=MF51C z$5LH?Cvx_%wXAD^%!fEss$k;;kztuhKUnmmni(I0n)Vad5Atid;`(f;BC{5uh)ma$ zDoCyGTGkpV*F@pHE#3_O!%|z6)%7scGc!QY)2bb=5oEMf%P+SAa{Y^mTq#+8ZNl+1 z!Q0ZJH9CCgTT$m_B2d8!ynh#lF`J%1JGK-Wk%RmLD0fEU$7q~$O)(&{2?uPT_)e7qB8<^o|#I<6;t zslt6h_2Of6)j>p6EX?B8B3w*1P^C4CmmMLpUS>!5^DZC3l7ftCm{C?IK^i~awYbdR zgw&{x$f|e&Sy`PA^ho-INw0Jfq=_|n2SBIHdUh48(%0l}qGVAuG1&($wS)CFMa?uW za-OaDOrfWs3s;2PWt(fHa84~$*gr);oNM#2!tpO2VZke3%#}O;a&?3GZhRq$7G%l| zXXba2uEo&p+lf(FTe}4z%w6ePQZ^jHX!e3ubr5-kW~X3qgCbcI+odR@qGi5)@*`OhfPe{1@7L%)>?X8jgZF-xw}O`x;Isr zJGUuHuj~+{H=ASUy7MUpj+K^{KO?tex4_MMfzGiS7igLxlK;1WOn!qsg0yPMmaPnN zJ|Q6YzhM_wr-9--@7WxYlcz+3#334A3;gelAhlWsd{;<|7bIJt@3YPdh-K+d4~Z># zIFc=Yw8V^FUlO?LyOR-VlXTQKb3QMBDD=~THNEYf1lR|x$}4(jn%$osjvt%Xv;ZT zS{l$6xkqmW?qSy{)M?L;O zfpI`)DKI>Btqq?ewVu|Y^h*cizH3CTI$W)+)yu|nR6*_?k&CgHaNnNbEyD#JdFhBd z&ba7eB3B!J?!-TB+JsN*gbt`tNn8goXaRO*V*!a7 zrj3Si35v*ClYm+pHxZFK5Y{lxPBHP>0u|J%1Ir=u1xFPz$tP zaPkJ9x#5Q&rpf8WksP+>v?`l#+EEGelQGB8-#7-rT{Fd%NJ4sbMZ~)wM@ZA_Ty2o_ zISN8c|AOehsWQ|Rw*RU1u|lsi^za}V>I|{nxQA(gaAK%HB=6Ez{$FF=8r@M{;xG&!@pV6AUPZ{*s{lIzn#%dY6~ zx<}crhUB6N9oSiMLgCVC^MuqCz%D1Tx@Mo`9|ZMWbh6|3g^UYhSq_SVHydOZ_W9t7;<5h#OKF92m!){PzFgJ*}9)po<@wdPQ zztQ5s_Q5=G2=N($NZUgKGG+)LztJQt`?w$|x*iWQT*&7(Q`#;=F>RR>66E~QO!)}X zdl(|O3na*Hjm`(259MZ1M$YF1BxazSBdp${$mqA>s3n&pZaI7LkjGSeoqRu$>Z$Aq zbfo68sLl3qS>2g;;# ztI3qhZv>=n8J#_xOjo$*T{>#%^HxNP!gpsy8Q7I`f7JUtSucm@_@dhGq5iCpS!{qzq#CFvY0D)5A<7Sd66I>set;i^}e`Qw1jY(e5?A{f|=XM)ec@{YQ;zMmS5=9wtoU zhSFZ9O9T@nLD*z2pla+e8B6ddN%`5%-gsBma|*_4=gLB|$x{?DbZ`nLi>?N@reuom z+tGw()KpCNCW)+u{`EjGN)ycx7n!qaTg{S z0%QJAXx{z@He^>IbuwCZ_z#_4W*Hejd?qVZ4_2xas4z?C0M0r5W1*(EM?Yp+d)(#9 zl%EGL$icLq%+hF3jCRKXO1lgY(>mtN64#vokL7tm{aI*x|3Nas0rt+~g}?>uFwNkh z3L7f5SoL|q%64c`ez#cBnQv{Lgd{p2wQ9wq`&lH~! zp_x|eDR6pz=I=7iopUk8BNB9j6ML~ov%1>jx&V>O(^=YXuya2e9xzK`_;H5Rn=K&C z^Y{lr%Wq#!=;UMc@$&?-?@0_s*RFhcP6L(p3&4GGW}VnW zhD`qxk=~001ul#DM41%S!9Qr+mI%_P7j=cL75HCKG_K2*GhDo@N;^#}7Gq?E=L>qY zx6zQv2Y9&9wqN33h}>Q;Abam(R?q(8w&G@en5MxNK{{ngru_czi6v;}^X*L96JnMs zI*7ETD0SaW(wt?ygtP+MXunYRYu@7EZPhYGgLV#a&A%MMleVY0QRJdI%Mn?2*c?Hh zFdi;vD*EwSD-cc0|c+f1q11j zmd^8h&&T{$NRc7yxPz%+Yu91Hf0_RB%H!qP_$-0ST|?Rve}b;GZ;Q zSz!|*Gdu+(cT=YLo&|NlPw>V#Q@^*Fn-=0WbB9sdjLn$VC7@M$wI5&3jc5hm?p zY1pXdRy{I%y}*mk_7%Xyo3VXe4wJxDo6x(y$}$*IlUU?*wMXXp5id7;*4!H2hJL zV8P@+oE@x>+7#@@v@g30>RazoxT(`z%x~%*wBWzr2*JsbUqJ1dM#j_7sO>pNg(MzM#Bv6UiS{scKW}qrOg_*A>yBo|ThFt)9KihT@X-t=y|D?icv43Z;VuUe znS$?SFr?RRH(zUH4@3G*6_6(T+}y2^zW*RnJ6k|}4ytA3e+ZFbIRY~6kfJjFX41gf zg0%W!-r&&|c=};ez9k)vybEH)z-+ybHh zQAte#p6jXguj0Eqyo}+aEI~)n*!UQlId_$yXXi1+jI`zhMT>fPvp}XC=RGAIB3g4C zO8) z>1Wd&Y|f(=T(b~QAK=NJFimazdELvICO_d3ef)n+a~n;Ntuy=gPbD|h=>)qSvixnU zhncMo`&I8GES>KU_x)A&06#^*(|^&R6BfG5l}vfh@AQjaeq0eEB#I&MDz8aWcdWaD zkE_IaU+YHH;TmS&sv2jUz;G?IcvV&!`k?@!cwYEVM=D<5?{z|M+5Ys?Xzfow^_Km9 zK~PPpmd}0k?(Am=tV&J&Q{U?tj*IFDHDyS}xPNw?*A&)5nCxktv}%8?_i9jbSNw0%4;^4i!Ddb}rj3y*MWf0nAHBwt79 z_0}(tAHRi;OK72>@s-|my)z`<<{8l2J)>`u%<0CAudbWV5vI ztd+3M8)Fvtzo$6dGyfjuR&gvB+k@9OtTn6iz0K|;IB|j$+feqe)9>3_1>~A zU^WS}tvcJBujgZNr~;9XF%*wkBukI!gRMWYPmi%Y|I9O|?)`*sbh5)gwz-SVlIUxC z2dL)AFPgIDQQOGBxY!l81@qq+pfbOBipq~znB-@C{guS?pP|^ajH}Q=zke|s>Fsv{ zn?6Ud`$`V_!;Z9ZT1)*C^MNtN*Ko!i+-_pDAA8O>J_%g@0>S$0Ip_-I?_xMeC!KUm zvdc@1_FuMf&;c4f#?L~!dqoyuGXx`)?*Z&)pzj_j`1u7^r}T>?jj68@ zJhI;mV!(@MW2shZ9nwRaci6ShexsOfH~olh#i=*gR-6==?y!UXDdsKzyeNkuZ!w1h zCrM9tc>h*WHCt@mt7>`WZ)9 zmp;jzmGc=jl>d(-;zbMkl)WC9B`^HkogqYdS*OhxOubCP;Eh0k_u?``sj>I?icSi7 z$(1>RTM4xi9sPE>0RHkYuoHDDya5)6N~w4%wJoHS>ze^YhEGRj}J)-R0w?x@9ctqo918yR$R} z9rP+=Yx2|Jn!Uvvm0_N(NgNFAv&B7yfQo`NdW5b5yzl1j2_HxCMuUzJyr?2Uo)+Rx zpe3I#J>9iX)(%U6Kf*&9k*Y4e+&y5PouUXA`m*9OxN|Kt`~WwNYxHvukREX8YLAu; zjAldy*ln+9Fz?x;YcHFn5e_Etv%ms*L9zpuIkKUUcDaK|JQYp5EaBhry}Bh$nY1F5 zcjTHE=3$olp9MjlBQGQ+qhET-ut=RuZq|F%iB5Q5^aDpKTEc-8baq2&)0U!oJNWKu z!e0)O+~}vDuiB|Va4aXi*3Wijq58aC;`25-3vpmhvE}B;DP@7FhIi9^fbw53`S2x4&YrAb4g3UdrwnUi!-QV z8hasfc7z1k>17i8KSI7SWOJs7RD`7)-Bn&Q+~$qk?8y?Yjt~E8=q_c&8KSfVtU2KB z2+w^Kky5xYYB{dp%32RBt}reyN7Ppll70EdMut!JMda3B0&>&W)KXq@Ka|$oEGj4m zfBTu_cT!e8!Pik>8$(KFKgCzkzCPinTIdU*BP(0>z4Bye7B5jA)L=a#K@=2}=jXRbdZaAs>vobQ z!viwavcxu$e!%gLS&&?m|0D}7=k`4%q{uIBDk}3D-SzqjukAB^C@l$}%kyGUmQRB* z%h^&yA7OHYKH;1r?@lSksu%D0i>ioo`RX1dyCY8UaIJ^DM=W|mA~4{k7NUWiHDgdiIq zg<#-}4Hcv@u^z6_GgQ%}P7Y-X!bHRwzQ%g^L0FiA+uB(XE{Tysnu0Ks_&cE3Plaiw z){-b&S-~W(oAX#cfp=IUO}|-3@Zr9STvG5p3nT0H^T5kd%GaZ64q$;2fq`-pZ?-V* zVgz>qImkN_%NL8khZJd=W(f}x{L2IteqjRgoICWYXcG69$yR#C;&@MZSC<4TWhIQDKD`9#sY)jC=npM+RI52~4_%X~ApIwl*QCDGh6-vfsvH55VdXoI4WEoY zS=oE@)aRc4Q;-&%LdPGh&6`PHt^M%U2vdNF3#3MIg#0!u3LW6OSU~i3Op!A81YATj zdeuQQ_N^4P*u%t&9v-qybGjSTYOR3eU%?j9>XGK{buptS90{?$?{A7mNL3$?3u z^%Q=oQ4f)?2LxnyW0M1Ht!I+&zrSb5--jhyroHiSkUx9Q-ke%mA8TRKNr8*`giWIK z{Y*M?vEVvMb99~#aHWC5bG8jJpPKk1KImK|wBX|6DSyw6?l&B2h}qP}`xgRM9FiL; zVrxtzROI)cg+N7(c~e4_)VMJc4){AjtRAt{;t6`HA1c*ma~foAvtw08!IKVo1>B^UV@Sr_zzjAA8X(#A0-U%Wn?6zw%{cBY;6nd zYg$);HVLLG|Ld@Pk(QX>PU31oQoN@f3~Q+pqVr$ImUR^;tt+UA)Cbu?uSQHSll_9@2l;q(b z!}FkVO~V+><4O||aRkFaPZwAk!wc3m0?*i;WL$D{(G~|N)rJQb<<_zd=Jv6bfaJC@ zi5K#4Q6fy!yR9Htxy|!`{vJVGTh!v(QGn5d&akm9FBaKyvn|?ED_%e%+9?L=N$pU< zzg+}rKK{ev5}GS+OBMuf)3KCw+w&sOgB_D5dCE^IM0OQ%jL43r!f>a($=#xnimmN{ z`ThQ@$d&o=r?Zlw`M-%MsgzIO+IB>wR6h~%gv}j!@hQVw47oHwL`p!1P71`(36YM2 zMZ^>CcH+$lo$z;$MJHYyCLq&Jpf~!)DoU0Ui&~nF6p;I|Ch_<$J^eg)nZj4mX#$DC zjQb;TJb=jRMRBNXi$Or%#PQ)ZNz28fbf-yF;0_bxd887uks%ew3dpJTp3dNwpva?2 z0wTWn&~pNx7g5??+Y}$3Zi=7>qtOGlwN>b;+nMQ^E+F`6i6avlSHGVHUAy48I)p; z^I9UB`E;>>R7m2HPG#*q&2%Ltm%h;(By6rq`SLvmr~kSOj>e>AZ_pq*Nkmcm}c*8L4cKR z2S85Bao-yh8qk^?uoZGU_5w#SCR7WozEa;`M_pq~ya z9AVFI3bnh_O|JTOvJL4RMJjuU)_HkDM1~?M7S3N&|37&a1Utfqo{A=>xu?SXm-m$k z^?h+|BBPJOg;;J^*xd(RIRBAK3UTsesWP3W#Z!SR_#JCay3Rxh{dmejjs!r>ev0oQ zr5_fo!!uC}A+C5$9xZv+&m?~tuL#?auzjh7r={@KRSJcpjLUzm7R5D`7>nxiPkvQ*SE=%WM)Eal}7MF;TQXi1~-0Q}NC zzQ{ROxKSPh=2RK{-)>2Y(*v+gDF0mr)X)g_rQhYn_*sM;EP0v9>L|I zfjHB}Hz65OFbFpt{v4?AZNWh1Gdt)yz*G-z{%-PzR+YW*LB2r>E`1Pk6CLH8JH!s+ z=Fsq?y9;z3Vv3e6xzobRRT8v;z#)o8x6u&P{HrTVHbnN>zF>l)dM1d48@0TA;OG#Z z8`XH;P(((13dpkh_)$Nth0e6%Vum62z(?eK;NM|f6>az~YviQ`-Ei!-TKbDw>fp>U zg(LEZqpJ0R48g*TFo`$nX}@n&b1(V$s#q{1yUO&iGu=5OF(9s#fLpyxRbhB*%sX); zw+J^_!!$!n3(^ZCd3QnI7d;kzv8*7~CSZXJujdOO9@86ixJUhFR&nFk}!mg7}j*9C3uaM70*L zV+L}6RuNSZ=K$L?_-9G_u(5N6ZyDHTb*Qd{I~o*U=>!8R{i&8vgxnrpZjkX#v2q@0 zL{*jXNy%O~rOS^CDg+&kJW#2WCo1Cl_DqgAz&{d%ta@g|j6Rx?)P*?gnbsS>ru%CZ z-sqKyTtX8uHwVbi0H zDG6NjGd34{$11pcj0?RY;r7~l`~G;w({~*5KG!7t#&P^ZqfzsnacJk$ zih9sx5?4Xu36s!>6+U*;!9gd{4-3QqOfTbiZxja;q(+!M@yGN1)8wYgs)UjHR)`JWd#Tp=T7A|(KBDi zr(&Uf3e4iYp9AnwooR{;YH!9NT_f|5jvYu`Y>Zt6k?le&LjLCfJ#3uDm6987O+$lA zR0tBv6F!}{=xoMidwyzhL87ly2_G?5Zrt6Pj;WhXlQPeD6!(K261!-6KVk;je%!U3 zL>Yn}P)=eG9gcC+mzOXp?Lv#dm~0+Dr*Gl+wKf}-<;lR4IZdlV)E^3O^c+*(a$ZRA zg*HK|eafk3r6BoMQ!PAGrkvv#Do+`GEktTpw`S3>dZr?~vop~fN6Lgs!FHj|VQ7v5 zoS%bW^3PHbZK(m_pa6$03bPFSg#rW(g8^U~VTqlWfZcJ?7wm6QKo0FcOHsk|W?^3Q z5+Wo<5$G_RgZ2I~h1u?(V#O?wFcdAvXQQ%V#Vab6g}18cfHwfS^=;;$M%Vv1SQrec zyc$W_eSIYze4k@-wVf;j>$q3)fL?RCGNPx<#jJ;FRex-)0{5%YdpHEd4 znZht-9*<~RJlV^bfS;;~j4zxBspc+IM(1q0%tx?FIDs|c`2v#`_Ri-%Bcqa_diop%tk!CK~|K@ zL-E#CwS?4T^LT1@w!^k;)!IxF0$-4`L=H`}BNk#2!6M$-wYCeKxY)@BCNESt|I*;v zme0Y48kO{BW}TEMa7gW72tKJu)Z$Mxt8uIpT<|Ba4KlreF@N`#GT-cD^$>PSp)76@ zf}ij)Zz>6n74WH)MG9p%8520d%#>ZM5XUV>@$d|@Si7<|9x7U_$mRlLs*E!;)EzE6 z&AUUgdBiq_GlHj>`P{!0djjm_f$gJ3W^wwd+B#_7lV6Y|b@59u%lO4+=GiGcEq{E8 z#4L8t`t=g5s4{EJ%#O>owa{d#gkh`pDNB**vd7G9TFOh$?!nvMdt{b`2Surj%M+KO z6>T3&nKjG!9D*$Uk1=y!amLrWp870DX4ZQ%6Ivn)A5yy@v6NlUxQ~oxToBkSH-!TF z*a@ny;5(r-il74>`S=T*D@^q*Axi=}%MnnL2`WLYl|0nR!d@#;^F*_B`>t3|IJ#1? zr{%N?k=YS~0{_d3V?Iln)T_B*+VfR>=qrqED&s)AZbzuM+EmwK6}1&}R^xDCDG4H= zZf2BxoJVPCke>MqtXpRa{a;%h6V{*w_XG}3?ZRQfHT;VuH!7umEh3r}QU$SV6+2wn zOuA{XAic7dudGp8Z9Yo3qzO_|pPbJlg^=kCamo~s`KzNm;Bmg92FGv1dM&yRM}*;1 z1g^GQtPbjA@ewAy(w(yo{c;ztaji48k`=*sJ?8QF4R4&G_aJc%Oe9n zFs3^Y69VXXaSee@eQe|An0Uk#u-VN z=JGnwZ#Ri&vB;H{Ye3pPK3ymVLLH1`?D77cPb{VUr?;MdzB@w0xfqbrUs8)>se}F=tHO9 zGAqK*P=4Np%~@^VJS1_XZ2m-p^2A3Zsr`fT_*p^R3R=wI%_AR$cj(!K#Xu|TvG=Ur zO_`)N{EuOGxjts*|8c~fv&qc={|vT|RBL87r`@+H2{17eWqLoN!IEE=*97z36kmIW9QxDiHU?SdvF%6 zK6d`)kB%%aVgjvQEDU`AF*8q&@#7lwJ%%60?{RBhC#HhhD+fqF&PQuB`rWhAES_=_ z`%Nkq;b;fVa<%%e=4bcKsu zKz*#+?i7N{?wCRBS^Z%9B1P~ecw>M2;1o`tY#y1x;suI+sa*l;nfVmO+i|e8s6ep< zcZG@jzc!~Xaayr!^SE@ZqP_ZT))ZC-e|IC|FVP}8&UyJXcJ(D~V8B`IPlM0!Q*l&N zeJaITCbB}~hew);TzEz?xH*S^(AX9iq|rNZTWCScSQiLCD`v-Tka@x7wz2XB?esD% zO+!nKutBfp*NW_KVKDU^jv<21@d+p$9t%xWs5s4W@k}KB=)%c!Sh#uBCCUQN^RXe7 zuzW8?2@7fqWuTGK^7FiIs3W{`9yR{dTtGG~=lvil>v#d3(7Uf7&Ap)L^9q>MJ6({{ z@F;(ItQJBqau<=7wil6`HBR7YtXFMgtor#PJ|t>5UEqT+@f}-|(#U%)mpVZ1CEmYK zhQ}^phDB$H3LHQ>z!t`BYzWc&- z{TGLxiFtw+^1#Pqv2=HKZU3C^vKzycD-FJTuY|7GF*qZx@Hil|W3QmuC(Y8lD~fW5 zGmCTAva}HpaFwSe&2)mvKA%aeSidp0vLB982a>d=^=V+*F8Pn7b*xVVFX)8%Cahzc z8%a+CQ?`4mq3{w3yI3Ng1W~W>1Y6>Ih`ed)2tVK8*0P(CnkOVkhNlt%eF{D|(eYUu zSh6&WKJv|>o9LTko0%XP8r&!_1x^aHi&j5SeM>F3030e?whCPiG z8POrr8%1G&ZB`uQHIH)vzu)+a1LW-A9-x-t!6JgsgSP1E0I;t0E$fOsy-#bj31BUsJ|q_b|=Dnj+^5t)KF`rZc4H>k9s@&=)JEQ@-TZ zFFpG33^PiN77_eKYsq8H^#0b(XXvu-jYLEX;R6(|rEqA}Oh6`&!isczA^C{dVodcH zsO1_*Tw&>uxFD!GR#BTDU!Xy!VnkKtt&trs5%KOIAYrde;+x!b^m%EVAec5GP6r=f zNz7p7E%qAIHi#DyZ&>zP(ZkF05w3ygZltBscH*E6Jds+bH=5aD+*cY z1D4>iS(?8nt^%wL=btxyG^ZIdVw9ku?lNTykb)Zqqd%ern~ehU^`oNap0i02(9shF z67$2C{5~o0SD!Gya#I-675abXCBXywVVZk0nII8@zVOL2nK|JfMJ^F@8JQ((ZFOH! zcM0s*m@_bYANY1G&O!FdFdZR0_7w{>d@7e>%CuZi)%1Lvx2*2z z#g(R)@jI&X9UHf&s~MLFE50iRBG`7b?(%LDa%gcY4o_^` z@VcS9Dc!fRTJHpU+blcSZKL>N@7bV&4;;aVT{T(az>7@G)nwtQa-XP0yo*2$g7ZJD z^^?Ej_r(^K{zZarkopQgOldPfuGv!vk<8;PZ8xZ6$Fme1XNS@;rv)i0aDmQWqlUdm<`Zyss@V=ZAT}!G?Fr@oq3G&q$Rke1`5-<1BncvaQSOxE% z%Q;aMU2N685u~eKvc&Nmf~a^L z1iAB1h0?O2qD>8mbjm6PFXH3%uqw^w2k&>=;KdL*X7wZIU%Uk)SYA1vS>h#oI%M0= zMJRbxg*DdAwPf@T?lz^jMM(U zGHtI}-NJR&pP?&uo z;WDV@*cLA0np5fBzr7=5H&f8Oqu|XgLzodOFmx3F!OTw)8<7@O&?k$b z`a9i3Lb-M%D7~3EWz8)>1FWOTk4d+j8YAPmW?G8rF>oj(@vQn z$s=hqHUPs*@}89PSWptV9w6cDm+@CP2+6sET}e`@ELTnKe1ZFZCB87+D5X$kR~l87 z+R2bXmX?xwooNTy-`c~TGFi@0woI1JA|>^$%dlhzMMS(RMzf3?hXg@Eg)A2sUY0wX z-o^N{EUW3m5+v$Pe4zE4HA&?#j|s;_P77DdWr@dp$qc{ph}=0XBDkjaHQv`s)yyFY zI=ERLol)exgl!q3=F`Rt zJEot0SY8Qj8COh5`vOr*3{#gwYdP~vF=+(!ugo=a?t3aPlTspnim@v)n zf*^Zj8UIQ6 zhRY&n5?tZK9Ep^*zSV1DMU03P5of6SG$8;^)yxvFE3pd8sfC!fmINDBE6ef{fbcc! zV#vN00)nO#f=#^=ydi(IBu8e?<`@CFSvyO-FG8oAuk=mOLU0s*?XB7f$h6l9&JZ&g zztPV95?o+)ltdLX{%jOx*)dKI2UvIR)9N7CudAHXLy@{!;?*RI%9y&y?HnlKj@Qi+ zFPPDawOcege9lM#F+^vHH{eOZxoA|-DMO+lre2o#&UjpXqHu8k2Xwuq%of<7Ix+#s$K zKFcgMgMY52mohUV;Kknw4$!AGUo56?Hk(WB;L2FV8?ib275;QyCMvUoK4tKX(f$1x zhtl^Q_@`{QT7dWZb$0&2+U`(};3UtTAB86^^iku?o9ss|-m0s-Hoc)UmZ1DBvhq+AP0_rN~7x|>C5=^wNN zE-kUfE-E-@7~3*Syk1SmL5{V=+`8?S>vM)K|0byXWc`FzsI>NJC2Ze1i~rgevoG(W zh(y4n+Pu}Kcze|vwN$?(K`5}(WB53cxRo);4f-Tk@iC>B68MzNs6d)Tot_L8EpV>B8&CK|h9 zqCaCt6MIiFcE5e*KJK~qKJx#vmTQGK`|O!Br_Vj58+pt(K*G8MGQR|EP0Ymmn%y&d z>KedRAw_E`k~X4~-=nLF=^+i`#`B%O1yV1v89DbfwAZknU}+yt{XT z=f#1FG^|%OQnaYX+vq}7{eoAEBeEKrE9AYx!pxnXO9MoAR;QW8VwsFnYo(2i$vPB z64J4#WclmD`=%->BJnFD9g5xP!a>>o(V~hp)8p-^U|6O*-Hw($GUw-uRtnBDCNq?# z4>#Dfe*kPM)mnw9JKNdfNimS-wo|2{^e)CQ{I`e&u z=dvK&x&!7ScKS>tYfYxJCXXUdap_{NO?~nZtdBd7R0a0rnPgb@ zw@rdohtf5WACu(7hwGeurXtPj6ZwCBURpL3RI4;#IeASX+_*bh{oIN#*$@HS#$n_6rQYLV51l zNMDL1SutamwF!^S&|M*7Jr!k#*hmK=;2FA&Wy7jf8=cB=|+82WoOEIe2!5sf~VOEC>^94b&RY7Y%g$Lq(7= zF0*Nofc2jr2mN-BRMJpt5oy*(?Y>9{K6P958pcqN7T52BqYe78jKSJhk%BJ6wfi!WdG@A+5Oa;MHmX4(0ny{rsrnH5ddE+`!`Yln@%LuZ$Mb(DqhBdNa@b5uF z(4BHB$J)^o?<@y;*9eT-1vi)Y^>grd&uY|dT4rx*Z;~eQBFDz&G= zE0{GpPm#sPqhYsxl$wK@&|;RM3mh>EdLLM%Tdrf>`*xO0%L7PCj7S?x8)i#)^5&df z>4tK3iWSHNns{!24~36M$!QnIoX;h5ps#PDzzw3ZIfiQ9D+kiTBq3GD8}@4E2=NQxkwhAtUwCyWmcl->4rwZIMY1fW__+w`O==bQNh+m_L&dd7q~*+ zqOx95O=Xs#1&~g_w|F6qxtoc|qo%bmq)%RyDGw>>}XmQO)#RSmrizET-7;-d9AKBTxNyAuhv^eLTUJTj+cn4R|LUTE~mLacG z=e2)LJb}y2{65A^t83(fmV)3%3naoet1LY%KxsbtNCr{ZxZebBPQHdJVp*nofiCx{ zlh3G9A94*fOb}Wu2SFg-o)xx)zt0S&lgkYbcq<@v@UX0EZl@>6k(DhwR{%Hlss`sd zKE{Qvc-Z>rz}UPMYJ20Rf_wTls)A0+g_e3xk!F-cr@ihEIc4JIx9xpZ>OfoUqKkQS zHfc3%S**ofUoDp{?7EOI0SSJrp_=5D%B1 z#X8dBQt|;@?oPKzsgHGcdhu(}cYduB_)tpe=#QwNoy;_wyXG6>5+3vO-ycAqWY^)pYUu$T4xizBomHv)|u^-kwzcI2 znS5tqz!q5EushA&jBhi;_r;c?Ma^WGmdC7|KGbqMy4bKgvGz2gs?0i@Iw=pPR_m*Q zyv@s0_q{7O*=>c6!_gYZ^b1fFJqq5@)iE)|R!I*EQB1s5Ck!FZJTr|*NvEo!UDB!@cyAw+hN*k77V22?cz%7vd=Y@mkt#& zY3`4ib#(CsUNy|J2qvtIcBSS=3=L-b4j68mrKAM}+u{dim&{#-nVnV8m}k$4GX=Jx zb-03O&KFqP)H+nmv*wPoMB3I?Gv=T9_fG#YTkbGhr0Q-xQ$ELn{<;Y=yL#|$WGZov zNYq0V(i~l{(D)|^w>AVfgN!3_+w^5s~RQjG<|9cUzl(G?1KQz%}Y%=$1T>LE9PwPT%ui6_CQ08Y<7~ z0~~4HF&V)UDm{m**JXHq@TleOH);ae&5APVbOLm<-l|d`+Er#i2u+%1=o11@0(ahy zR-c9+kVzl<_M~Pfo1l6mAPLSI$QP$%pUNe%{}dokJXDAi!T9nN5h1t0aHsdFkASG1 z9;dBw25E1yU?6$cjPjt}XJnpv+wYSzsG)1iinJEU_yQTvVwQCUV8KUk&PT{uX&(<7 zcdH6PBwafzS4y1B^%o?YQcaa2syKWwyxsMkITG`#ZwSGxPF++f4Fz;nQT_OTuppxdUyRf`w1yPDCntA@oavy3b_AFgUEOD}`xs!h7x3hX(9`rnX6>RD z%=0*xZbG16H1ia0xkXU?ium+zAZ^lJNef5?I`xw*Fs=$`@g|61j_4WXWc}5lY}x>K zYIg|(=&(KtF7Hxi!y?l-e>Rt)!z)^4DMtlAOC3B-@=aEx+66Mcz=zsyk^){E%YB)m z!UtRh^RaX#ZO&EM9JnrdP7{Juoh|9tkYYUx;wXQ^06Thj&9I1VcpZ?{IU+4X%FyZS zvM6{-ZCWa%{VC$u0DIlKn)AE35Cp-?6o}^yLvI&x1Jd1JD$WREL z-2|@3Iu)ni+HmJ6(ww;ow>R9Z+7h5^32ck=xdHmgaDP0fVKLoBEcDCjDm32=*YpoG z;*N(^oHGqC9N?>c*N3V9{R0N>I3{3-Y~-RH{cvB7=UjE#oK{G3mSB-=we0VC8w7@M_(yYL==K_Lnv3O zckdK^#YYsnJ_G^1S0IQel+_Wix9?%A?@O_iYZ~K4rbluFWD7Pw0zsy?1`_(%U}W_= zhU|>T_E?p1Ssf_779@uy%t=CRPnFX^9yN$5Ln~*?Z~<$6_5_;CmsfGstRY*T0+Lfv zg?Le3ix^)Tab6DcOf}>gaP_NdaCe`{5ro&@Yq7Cb*lZOv;egx8KT)?EhG`(-f5}SA zI==V|I(m1~KoBVLp(?8<`{=rontuZ~D?*QRqrAU0eVeFnR|PKnGd<3p`ut;90#Ew~ zxLSP`9O60Ub?+B);ALN=kVaa^T&_Jw+HO&bH0cYJ!qZu@p!mz|+VmKG6!+g41&&BZ z8EW3zP-q=qfN?uEsS16kCdOb_ zzhhnjaxO`MynJO?SGRo)>68(Q-l8Ck6XiX%bI|39soo3lo-_rF$c`7y^p$5v`8eB- zHz;u13_ANph9r5u*c4y!-eW%d^sS+xlz9iz-&mSTN3P4#KuxznSmVWJ? zy)VX{p4w!&+Ftxp$@Ap`L!Q|!i)^9j?N*BqOJo^#UY3CL@-9Vsq(qjw=;pL;B_ZvD z?YN@cvr1;^2iyZf+Wx5UW;8Xj%hC@@O_|t=DPwR=uTFFG1hy+ z9%<)b|1Co*#=6q_hsdqf^OIv8fSdk5mCAuuJ%cA7E|k@sPgm4(geT3HG`Qvep#R$M zD0>tR$j!i}T8mZx6_7JKL1l#-dYmV@I%lchyg3Vvoq;>?hZg57J;aUhnKP`KcvlOt zi*}%4E?N3h2Vc2>%IBdTXHQ`cv0hZmRc3+p_HqUIgC`m=qI~FgT(k5Yma`jhAO5Am zA!?|v8~X+1ckKLCUGma5)?N2TDUVe@Vdt+3x#FIsUhK#Qxq5&l^1UA7NaF%xeRS(K z2LB|$fh8!_HLEW*^puqb{dK7AKzl_H?vQ0ik3F;WA0jJyWd+!zQi^L z4G7!Gu7ZR=OL^)#RbF^c@J5;J@K7QA4zWwD_YI;`TOw=Dt$HH-P&G}H&m zGs`JyyMBZ38fScEneYeddcIIzzJdZ_7Q{XMbvX9Y8(&a`)YRZ6mXo{_N* zx)T($vSw<#f65Jpz#T6Oie%Nzv1KS?x1lLVgaEQAMS(nBhD!BYNS1Ef zGdYz9&FXC56#QWO3*KQ8kS7!yKnJE9 z%xL~0+SRZ0G$<>UU`1J4@r&6_mzUQ-*dCs)(ev;dRG5@t$<(q9j>sO{9IKyo_%EQd z6|<%nk;Q(5ntNx7;9(!T)9FK~P`a})BD@>9f?8I}(ua4CRe}wFvml1rRL)Y*MR7JK zR)#d)o>xT{KQAA<=TYtIS#j2(>!l|QHS(bai3m21 z!=8lwT0gKgpH+i3Km8yW6X{|#*~`r};o9=D7Ytq+iYJz;XH6^e5>xzq+o0C(q+($G zJF&hxjck6kvr4{N*v2`MbEqNC^>HZdIi!W;g&JDO3jy)EptAVWZ#(4_m=8OBUlR`8 zk0qCIAS(ik8?~rqXv4#5fvWI^iYrCVZ}I6yyRPOeRtW>H${iI~Rd=+VJIu#nP+9H1 z3c=HZwX^hR@z&G^%f2THq}<1bcH8EK5cH>zZ8$r8>0^UmZ0jKHRV`#<9of&ak%t7t z7K=5NCA@ByJW?t;m#n&g{BA?5>t^Y%y=YJmg4zh;)kUT35)v0kDYkM7$n?(vu~|(A zYs1FiZHSWRRE@Kv@%6LRA&Fsm^}#g4U17qv4LgDJrql1_Jcw~a8UTkkkXa*exP!k& zoP*BwJfaT^5RiVfpkbDJG?=F>hZ@4+b_nSSX}S7w74?l%WOr&K=qM4QKmwY?RnyJX z#oE;7HhI-Yh7L5Ay%}zY^`U2t!9E$$K*0{F6D2p1b0#IHO~4+B$e#+S)l{}0-pMem zDOe_iY9RO~6R2zOOcmM;RCntMTz6X9Og4BS&3%m5Mr$+@!ip6BI797rc|_a)4*S(+ zb)d5?vO;xXW4_O$b8{H*r$w6HGtP-tG?zW$*5>Hs=5$fgy0wt~F*EZ6UyHg6X>FR- zLRLuL_iWQc2-+0&1iU#l86#WEtZ?0KGk#z6Ksch@cPvuv-ep$Q_X|ktBbwJTt1WG6 zDHXAA&bNebwzD*@NE-`=wSwT2{z|gbt+Lc3GVHY05H3M%PjEu$Ktoz*dF#?I5Re9O z0uqt?i9uNX6A0%HQqs@DAG)OZLOLJM^$PY1ZKP_9Eupqyhz$zCjy8sg_#bVMY<8NE zwx`0jFe;+0L2r6nK>Ww35H(Z$dPS*EQJnL~DfFZ8N0C*$X}BD|`?bqzqT8m%$3o_? zK)sPtI^BK}-DscHR{M#Kg~^pByW3{6u&ecVHhxybB|{`t_u+Gi+?y8XPRSjx4qVF8 zKGbzOewiI5SK8dBQ)bhRei*{FUqhH+b4TRfX^tXIIXlpSPA!V_phx!%ok#Z4IQ`tK z@jN9>=#7Cmk-IF;g;w^H^_;n@b_RFu5(W1(X^<0b=`5QoXPe)$#3h@;xZI5xo>ICP zNOHS?1e;?l2zv^7FwmVUca^;d>*~`LutRGVUFW-I=|9<(i9k<^vkYA@h+p_Vt_*F8 zFsS(z2;2NEO8SUy(oCKn%<6_(ki1Qi*4P*4Px-IqG>AhWXYf2gzpMm0-rdle_WT&< zp4$T@G<+8$a<}8v4s#EK{nL&b{M+P!!uR%QTzT^BYcO|mPncWkCk2vsGTxhfdl?wp z_5!5aQ3cZC30@}&`UNFxbq3w>GvFqlRBU?~??4^?hYr(9<2eC~DNr>!P*iWjaQIbk zK)yPsfxP`1-FUq|2FucK8uA)?u}HfsMLkKtq%SAlnOuKIrLwZ4z6hP%!m9(q*+2Bn zYFyMHxf;0iLqX@zyG?`+LulItuMzainx#u+?hirt`wC~yxp+q^_cwHF zUB!ot`@W2Gqlie9rSnq-eDY=7hq_&7vR3!Q^8=#E1vw3}r<^E5t@t4dDqm_L1NLUQ z(x=hVSG+G|Y&2?K!3%|I#hxse+yQcqBb8Nsry!pC4Z<$-0fyRgdjOOjl@NQ{m!IWH zOC1Ng(1hL+%zWd1#MscJ1l^d5L8xFuTo{EN#!T+TkA@sX#)9;Q6>fvuz!071xLj() z0X)sYNV=-VAWwQ7EA!0Ybf1Cxz9!Vgm~hFFtkUG>3;VT&#P;*+!oRs|Qp9w^8#~%3 zhIybqUS^0T4Q@_N1YhkCJkf1!m%)Z8NXx+hdsi<8j8pno3|+eG5P(OA zX~5|JF=cNIuWAqK^4#y}5Y)q`bu{2j%?A3><3)0y;p|UN0%=eK4GtYY9^N{b~oZDvuOIueZwr^554esf5^iIt#%QC`B1n(nsH$vJa%NZpNN}C;sqDsO>lXRa|9USC?r2AEyL3GY?acnI?D&$I@OaQ%tI1{q}7t zAT44w5Rkg%4n+Zmk2LruVI;6M6ZKepa2+Y*M4X}8X~g9LpJ$X}BcNXCqT;S@1j z?rGquw9hmlt*_sV)f?Zirm&!iZub;V=>B0r3+s(jJe3|cUGV+KtnNMw|7%v4IzuGw zL)XU2O2ttno0*CrC>>2l`z$q=`ucIma1ntwkqm}(`+g6CTMDg(B4#57O19Jml2guk6`{7#X^>_v9_?vR;dzxSI0 zMW13OCD`JM5LA7lq<#1TIkvj!r_W44 zx;G~4!AL?Q+HrH{N@Vwcmcb~jf z2;Aow3a$Md5X`lq^0_F|qVW#hjXsAm4tHwN|`8y;&_avoBK?z&ICtl^3R;Y6PY33nird zLX5WgbrmTl{}pJ}Vwr6wn9VLp6wMF9OFnqJ>mnc zBmSoZ?iF?$2s&Pc^j>24D3`kgkgnKhplS?SDi=9?8C373@Nw&20y2UMm&%TZP4-&` zg3)~x0xXd5&2*W;fu{syL4N^Bq=@AP3+60`v^*}g5*GYE808bX!r+!3D*$;Gr$ElH zkV|Q{z-uL>A=nlm2ZC3doRjjiTDEr|1JxpJp5C*~^Ivyx?1W zpUlrjFQEOV!=Gd>&4|}8Ic7}JsK8=b|4sATIR+K|@S35>QgBXud26HJe*>fI;-W80 z<#3^yYYbz@zBPbMou#m}qvBYmtp(&|j>3ZALurb-i^{L6)$yLzUGRl)S9b%6!^-+ z=teXJ!}o|y%7fys4=^HkYnFb)a_6lu{PB-Uax@@+YF=RI(%ruUWHjzt6SXMkJ463* zL`Y8`w@A;yIrkFekj~DTxDAj1?C<~qVqNNoR2GJBhp;x@m=tcA^L>^(mAn=2WnGt@ zzXxveH5I3R+T=$h&V7$OHovJh1}AcTh>lk~nD8)m@-c?7Z+F1M6K*Lcqn9pCn@i|B zXzH^Q+P=nK43TGqbjr}gKT+pvjWN`%;2(f%iM<#CSK@hmh)yME@Q#2?!VU}p8M4c; zdiin}q(0bpA*ARi@an+sEcKw9oDuc@S3v$Nc2s^3Se{!ThzaQf5LbkB&}$6=B1`%Y zqKozd5{q3K79H5iywA{ycW@n~AKBm7QtCeWwHCptP}`o^lCck?1noDpp?|#xVZ+XT zIDVazfbgsaA9P9$^8SxGARg*daK#3&8t22{UnGPBSo&N0UJ1z8-l{f^Iq2hmyMhdP z8tSXS5yWw$Tg^@CCK)aX4un-HM{^IOKhlQ3_(N^^pHK%+TcikjVLtkkW*8Fhr9Bdm z|5Z>35bE-yu85=+%{TaD+1ju{`VpZ|Wd+{suwi68TE}4gxM~8qxQLDi|D#O|bYnjj zpkzAzvtcOpJOX!JudgUWG);|{-&lm{>a`=Vym1pn3d`+i*iqTRaYg^~C)1TDXN9BXHa`W ze?=jLj?+MX{?)K#>YHs)>M}_K&p9VQaU-M4a zd9(~fQ;fwAwlS+zh$lV%pRDujSq}C&e4{vD7l#yA@?X0Eg0)i0AxI}m`c2d3#mbp} zgDtJsGfVFFL2$;8hFaX^A`s(n9l9vw6Bi9#bID7Pj@YJ1IRKD-Nfwwily-=^Hdq&n z+GhVxfpjVyR8c2omiB*OtjfQM zvFi0D1?TB#DoKrhH$*)y+%Z(yY1dS^8@=9nBZ9p#L1h{}?G%-0*=R0yM7(>dw zK^9luK^9N_so;F?8uqF5y9<^tUntURcMW;3d=FBmH;R;t{$*4O4(~j{r!B3!S=&1S z-uhkvcdU`guj!RgOz|c9 zB=JNb<*~tCcS8(X zm-s300fz>ara%|7zA|vT4cv&^$lI4y6z7D2*6QZj=%>*9OhP>A@J9)_OGs0r*~PV= z0T&&tP!)Dh2%_++38g5-M^5JX23EH};paapSXBO*rN92-+Mi%KSxGghrnRCSe*yAk zHH)Huu-NYUw_##p{u_{5HB}Z*D*KP2ibnne$kEy=i)sd6pI_@ajI3WzC2*wO&kgQ= z{~QE`4OId+%K6t&MKAsfNdKl9mdb4n^@TSG<-LHm;o&OIlLpzR_~c$OwA^pI7z(&w z8#Rd|g%3&aroipRHTiz8U`M(37E`+?gz21$Nnfb#@=hA6{MW`_?F|T;U}_@T23M1i z_gQ}Qt~*p(RfGouybtpe6+A5};X`X&{#)UlJ{lbK+EZv;LOE-2lR<{OKkBbZ2y65^ zhEv{yd(;381nw!Or7j(2pzS z6O^>bK^0x=<5v1aKCH{dHMrM*7Nt6x)CB^Y0*sZx@rg<~%qb*N=HEr-L4*GzxZ=7bUVu&T9=O?X@6htc5CV zm&Wi4Zcx~~h2g~Ml3M9rux>EGio?~?9 zwNGspu=_ruseyOhCwJ4!*C;YGF5si7|Z;<6_|5JH>S(v&(|z z8K!6g7Z`$85K_*N>C2A|HFOrHYofzNv*!&<#f;EZ0zo?%7XS&|#z2!A+KWWXhaBl zoBnSVrNIt~`tF*WeXa5)f15J2HP|$SS~SOk;D_>3C-0!KsQ?#y*pW@RX)x9J3u}Qw z_eA|Hge^Z_0hWh5E4VHn%Erdh`5!{s&0Uce%>|tZ1E3vMt7uY}mJE)<54P*P6);@*O4Wh-PC#&WRVAsOYwycS ziTZiu`amTOChiUtU0Jqtz(Z}#73gfZDTzAQNGxWU9aaT-*;+**K*sgMSG)R&?$os^ zis@Ai#r~sJ4LL1S4Fq3>2^~Xdc{P(7BxP3}YX*WR)j*J5SJ44~Rng_0aoehcxg4h&8rkl#3L`3z z)q={0Vj4kd^aKU+dm}?lt=t%@@OYl|WyXv|{Zg5I{zl-oWCPussIaG$rm{&h=~wu{cGOHoIhwv5HOnoosKz$~u4|5}9FNn!OYozd zUy>bk>wfM}dNqdyPvpaZz1}Ur_wO0tgWF~oqsnl?!BnM(|^Yn4(CpX&nwLLSjEj7Q*00@jCG20gVK8D?0v7eJ0xM@T*cwJj9bw=a zye;sFDW2}*4_!6`J3;!^hFW$r^%2_+vAMRBN&iXO!5xYUJ3-N|k}A%gI(0VS8kGSq zr!#O5+*O>|amZWx9O~3Q zf&nxsYH+abU>aXEFs(aCtMnFW2av=6rszA?W<5agPk%+4H)wDfD(GQoQO^bB(*X)3 zwWmpc-s_v5koFo#2YZ?(>rCt43xadWB3U28#yhpKa}{^_4ASpLD$+@x$>l8HCy_Cl z1-X$3eKzQASYr(6jif%G7a<(z%%s8IG{HrViTp;+w^Ik>jhsFx^uFUHc2*x*H+U=Y zanoS^Br1J^M1$F$>|e>N8~EPgk$vIOnv;uQc67h59IntMgxX%rDiW5YW&LD1a@H^O zgZ*B!ixBSAx4)t4&%_V5Yv&dr)GmyZH;IIB_ktoJXWyRoMw--fVf^jsaU_hqx3UOU zk_JV|84y?UZ=!&Bj)SNuAold;jf3^GQPvj|4SnanDT3je+%1E>=wY;>mUs^U?C`fm zFejRWABrjjAN=gNk-XweaTFdS3zR$1Au%xI7E6cHlNiHHs!lAVws|6L68#V>doZ?c z+}^?Zl}3$i0y(=#GlPDJgKdj;C=|^H${G+m5QVjMmm+vI(4<})!7gkXk2S|@@{Gd( ze=y#l_+30W8{~6-MfJut0)u3RdF0Z6- zEbYdznFLd-qAc-Aux5$ij>em=38os9nJ6_gH^2K;xOAm>s_Nk+tENG18=Geg)n9q~l2 zC`drCo3e@l!oJxjA>P*ezS%c*uBj~kR!92SHF*-D-qI#LQvbS|z5*5A9qddsQw*)V zX9{ZlAp_2lE*B2=q)Wr(D9Wz;;;F%P)$T|Ixz7hX(bcCIaAv0(lx$ChlFuIMu>N$~ z)3Eh4Y9wG`&m>IP>_}%u%E5&5#xFX2`cmLNqKGs(=CY9!(!jFLB7N#I#Exo}7~(`H z(+o!aDR4i&Rah9;aFlFy{A5G2Ej=4;s%hQU_{!DuEUiN|M#~z;SFVPa5XqVuPtAtf ze$RrKqIS=j;agR09H_S0)X=&W@vW+`xktnqQmk9hg7G2d`Hw996oJoCdnLF8}aghGvBc!;3 zCum4nYGH3E=rZZh(Z*ka6lP(Xv421m88P)GIy(O!MLZkfpJ(B74A$K6J$==8Cxg7(D^k~NJ;9v0U^AI`a@i3 z-+j4#EA}*ooqxg<+^LBs{kh~96Tvd2wjvE{GQ^KY^pR;<(c0!i^rK$idI~&0!{kOm z8FHxM51f5E4soNH3`~WbTB$gF_|j=Il-_BhNXOt0U9IC2ztg)b(ujkIF9c1Iu40o0 zM;c5D=%K=K+$$0;$XFmtfYZF00;Q`GR2&w>Qw@`)4O0Q>ouoo=26NO9U)?!DrW%k5 z+`wcFuH0DE5v|u5=b8oF@X;DvrAb3Pb%(7Ow@2V6rK>nM%Gf{Dj#^|PA6Cb8hnWEW zG(!PDSunyGhX;Vup2}rXUo&vmG6gQZ$iFN&ECg+EGRJKCLMIrR4MC;(3Nxy{x_skg zfzu#;`K2PAF-=CK`EsCA>nzEpV{+1Ax-2hlcWKkn?!J)_bv)q8`DPQ%ooVVtOJ~Sr ztn$K6LorX=tRNfzfTr7ZrlEvK&xC~~z7vqqx{L+Sf*^E{;zhkMiT*Qid>1%_Ur)qI#4_g2B>&6mR-m()`s%`QpLmY`!z5iBVS z2O65>Vcn$s76G@jibmUsDY94Qb`bK3km4Sg z#Zm{ycLy&@okOa;t^yg4_?m%>&_zp(W@^YOt|NMD=um(8=3Co_{+`d?SXKVj*AT@gPJ>)WqQ z){~iQ{QG}RX5v0dN=)g+umP0!0X!|W71F?P;p5RGU{5HFgxRfb3Tz(Ck~6%j-c^^|}U!$&-E; z#*l|e`e{$u-)Z#*!;m@nDclpj5$*}NqmZrIDBZ)O+g{w-z6qmS!9+t#qJIqv+CLV^ zaYdzzjxf~r+FwfQId4(p!@UiY)q>5)#-euuGJra6kuK#-rf)F?l-OBuD0%Hk>PDEY z+0&CPYW={~P1v+qKM$D4l_jPO?ah;`7VeB)>JN4EDD#2%&j$a#p1ik)#Jf@aR+D-) z3>T%i3%o@4h6*OP)ldg=`A#-fR$Kc!sJ$T}cJ%dk2LE3XkeoIuiyd{{COcbZ$=C+S zSM4pvJl-aIem=EZWxL70#Ohd!xa-iNxco$OB2?VRS;8j4A`VZ1o!aMx_?uYj_n?@k z!Iaf)^JXi8cfgA9AzGe)Mh^AT?e=Hj2?2bJJ;q!ftT241NnWTY%45n-U?wJ8Fi&^N z&#FvO{|AV>EU<{*EFW4*cP%Ajt_sZiMMY{vgKTizt(aRxr|kl;<}wAC+ijQ|-TP)J z!kFL6!r~C5{{}<(A75=Dhszyk(uSda6w+9BRcysQ^Z_M0d}9G(zF3Alx4^|${*dep zT($?m>l-XUEXwT2W3OCOaE%fZpb{mwSTMz}39j!2&jJm`g*xvV>aKgRkrmF|2jJoD zl8mR@uGWrhv>(7Z2PBZ!19sH;*icvNF6Ev;PdlPT#~qL@gHbm372WO1A5lh5KN>op+CQS_m{dxI zs5?kFoc-uWbfe?_RH*~S{$%JlzWoUV^>Cooq65#39FnERxs5smNDl6)7Lew3hWT5c zYj8dch<_~w^2K2}OW-{RI}gK*kLxJX!luLgsL9U;N5%dO0yECJ3LTg|>(?xszFk^cbceNjh~<2r=5al6}u%Ue@RAh8zcLOy?q2H!9nEn1{7;{}X^c$EjG6 zeI8iz`VG@JGM+e`|H1+>F3&g%(I>%@I93A*oC>45oR*GYoXaWTTryRhKYe;yjup>D zSS0@xifsK#72-%S>rn7oDjpTnz7&vR)_iVfpyLA^VieMeXAF~-Z9;l;jUsLG8@jfd z1%^_MF94*}4h4dRq9@%dkVdNAiESqwCHxg%i?q)db1pZZ1;G`IH0LlJVfFo??=OH% z->(ofKRV2tuK!~2mHD`#&>I~TaQ(;M9lwI7($6Z71C?4d#kXhzz}5^nhb?m>Ddbnv zXH;-bj&Lk=I*;@Lr$u^9FwU7=bKQn}(fHo>#dLco1a>p-meyeHDCIJ$OO0Wg(6+dc zF#Ufh#S`ZgEEdczFODqa9&z(ob(`THx%}DUx3DG*A)l~OaD=B%x{o> z_ehmG(XfAq>2KTRb7HO~hwJCb`*1U}$is+|!^>JLy>Jo!YE9u66@Lkj1nW^^Re#Ev zT3!T4$W`RcYH8o>GE5ufqJdz2?@qORhvUx3%d(*PnyQCE2K$!#DhYG0nfz${6~k0` z3x2Tu4xe%@>eWyP-?ldND!jTdOqIIO?W=||555M1h`Oo{{RJnNg*3dqD)l4B9;g!9 zv#$KPXz6t*8WX2cRJza*4eM42s;?6T1TPCd#`lDynloLpA&uKWUN=nY<$OG4===R} zH(GfE^I~UwYP*4EwqN{m&(SbTXm}Gk`fODo8*j?xC|B=?Lh7|sk;eRP=-ik84(S_w zPZRWrCHU*+7JTvU-m_2|{Rhk#epJDAM!16t{xFQowzoj_$wLM5;-BGuH0zcuX8vxt z{T3Yd;}Zpva|bMSZX0@@A-BOY`ndwRdRu-0<;;}31F6Gnl|Wq(^TOrK9ZWP=x{Xlg zhCHw8={LfQE?<#fnpKyVm=s_ETE#PX|DxYLka?7q80>GpCua{_0e+}C0(Z~eGr5)+ z@{t7Ux2+Aoj~PbWI#O&;ubPf1Lt)8=Vhnx&;H>%*=t7I|lddB>Au!{bS}+{w(ykNv zD%}p9M(9@>uUlF0>pG3V);rC`$_GoJpQ`={87&=Q!LZj>J~HSX6NTk;;Ui?UYEKIg zm08`yN z2%21g=6nB7!{M`O+eWzS)^ttQ7eTj;2(ljWj{OB~<2G3^RdC4| zRv-vHdDGkHa(HL&^$$x%0Oem$xm8wiF0}4niQ^`{yKb_6WEqNX`5$=8zh1I04YQLm z10L}*UZ4lKQ&WL(oMUCfWDgyV)z^3l3!^{Q;7T`5_R-ltCWqQq z`&5;dq`Zh^*Z+Qau)`a$r9Sdqx!g>zC$)1)3a z${6-jGIkpp&G8$RtY6$5OH!1yxSniJvqvRYwoY@{&KzKKhEhwIqo|p^S-(kaj6IZ( z8l~j#rM+35-(c7Zwe37clZ6uwX8ks4KS!iZ7^gs1)W;rZ?6byeM2=>4Qq5ZtuREIk zZI@5bPz`c2>(41{bOP1l3=Je^dU6e2UO0?vat2lARDoMjRD-yj`nZ4~&O}?wng`Pr z7qhx$o29|7kj|fGvDDS9ejw%J1$SM|{x_I^wFW%&wj@bOqaXjvz{) zr=`p|rs!_)DC?>;Pl2GusN25T51TClJe^yebNeW0WIT2<;u$9)XN?d1!|C+k&nn-;lT(0fkS zdArsovMFuWA2Hup8tH%Cf-@s1vKoE{AFthu5h=sZjO~HjR27aCvM;%cF1uVHOAi=Y zROcO*G>y&f6p&wBrx4-~^`Gui)UWb4H!m`u8UGd1fqNC{m;kdH>|yDl07%atP^9iH zQAit(Vf?dd=?lvO=lPQ;h6P3R@J^_$+s}$;3(LvI%FNXRkz(gji?pTLo+ka09AeG< zQs90)A()eBQjl4HhU-WWq&H5hd9$a^!3M~}U_j!25s=&>(~7E&3k?fF<;Xa%IPrRj zq2$pzTs<>%1>EH~QNSq_=!j{4SWDS?aIPMfhqA!S3LR>!H$DENxN!^Dv&K7nd=p~fX#g? zq|G1b3ApEXq&|0W`-`@BA%)*#ks^T#Tbi-U|6fC;lg`~#8P5ISnF9W>idkLH zGS%=ZpgQ-LB1P}yh+*C^;&20-uJeDvbG8;kPj#I`cm$a6l7$3c1T1t%0cA8W6wsAw zP@4Njg>gf}6B*UbYOI~RwawMhuh@J*HVrV?<~yZ)q*KQBH*rnTk8Y1_75GNil=9Ya zQJ1KFC_Lh7ui(zq!!Ik&WAcJC4#hMo*P5^u8=lw(25<3BDO)TvqFSyBE@LRBgZ*on zK|m;r_F8ZP+~0M}g$mHuMl~AH&G9 zeu@+|K7=-YEW2?oz)W2M+64$~nmEo^O#Q<65MaBpoI=M{8bhm}_0W)A7r1%B3hvCw zWIJ-LXI6vU*b?ePm#X97NHlzGJEMXHSI?|(9l`a%GUg*iiaCKg%=7c=x9h`cpH>x8 zwNvAg-`oJnj^g~8@MdU3bA4T-8s8AoZM76>(J1IbPhHIFyCKZ-ThkPGif9D;rq@xx zoZn-OWN*MYOxD~fxG`|E>RWJ)4a5HNR)9PeknxQ)R2Ws&({r4oJx!3KK{&A{3}aLLFOIDV+zM9=E!S#`?ylHb9#1+GTPfh8>gP;_&15y;d;~NHX`eQX zL-Vz#mCX&=0Fx`dZI0>|*-oKLX<=@xbMBg+hDO(?qk=>m^`Lbh+IGbn9I^06ds!rNcrszv-ZZH0CEEhg)ZhU^u&Z(yC@!+$^XQFyDf0D zqiJqebAO6zV^%jK@ym#ml%ij;wu=?W-n8jriUYZ~HS6Oyo!f$JWRe2OZ)!K^>_F~0+Jw3#3v?7Scv(Z=Anj?>UO zx^_hCc#G?lgiG?KrBtG;-SxQKPGCATMRgip|LtT}PwVr9{)bMmr!=mY78!!ca+|&b zYSQ4&kY2-+BC558_(kV5?=H~cwLrDjiL$zw)p!#-?XZA6TP7g-y?LX%Lg2kx*f&N; z>a+>@s}+I#?fFWv-m|Z{3^i*z#gS%jN^zvU5i+NI@ynGhsANw?idk!v6ez$N)-`i& zH>g~@PRTC{(48uFH&lb(_`x=QqnaZG4!awwL5UuK_~3SFOA$HIksr_ws(VPQIOn?r z?w73s$EB@y=NwCq>k02>e6QG2&{O)J(|Ywn+7>@(vcIgCS&e!z!Mv`AY-%0|j&j*f(M-W8En zy}Dc6Yn+=t)rBfWfq38V3h^V4D2&MI@=)69Rp>g?Zj0XaV@i3NVyD|l&Tk3blu{Q8 zA0uN^-HsqO)$5Luw(wbsr}m@)=k|;g`dgp9;4F8)tELYjr*K0&o%bDGvOrSCVl8Qq@pnwBU> zFRn!E6CvGEPmu-NQazxPk4HR4+;& zB3;M0Djid?ZE^^@zDq4tumjENnCeH>lMLW)Ng!R-NyT~7nIui;mV_qtL0?s>jt2Z$ zy;&UDj5hb9dtUMs4O`H47)Z}pq=$#esTUuHa~cjOmcoH`kv2aS>*6zM$cMHH@CK&6 z0@pcN!Ic<`66`WUW4-8I95Yk(GlE^?6nJuGYFQm4+f_Xoc6nzikQQHHJytNwpqX)n z0+)!R;le^txlond)Kb>Z(l1E?&K5_*1Ub~Y~895B<_Llv=FY# zw?}~`Vz+?k_t7pJjmBxt9gQ)h*(ta&^m9qhE8m`lBi$D&0%xnO7!6~;nU!e1&x{iw(UTPC1m z#(J5XRKYlNNoqPyLm=u;pAsYW^~dRz3RmYvTp~{CP}T08Ds`ao>4w$L1_4R8rMZ*Q zgrU!w%8xhu74@Zj*!GaUB2Doe=}sLJWoOJf)`(xvCn`aci7@D#gp?z~}{x>8q@G?fW2bu7rv~12$wMtChMcjKFzP!)bB=Vo1s~ zK>GJoA?k+*mNuLYY0rMD6!#NOm+s_(YCAy5Q_2j3!7FAU?H?BDsv#rGTaQAqGXbfe zpb!9pBV;pWr^go5ngvKx+)yjr%#iePBR%Q(ENKSgUJ6_u7HwKwd7Vqx$^5d%q0bdu zM#e~&|NiFp#T-x$8lys;C~qc;*(pc5kf*1ua!?~Hjo0IxsQ^D0RUJ05(M+^+&$(#l z53m*!c0xas`qHwwvJSIbyJjnp=JO109X1bEJ(5y;+A?pXckXTzn^k|B8nNXKt9kot?{#ExH3D!HBxt~AHw-kz*} zb7WueO~1<&AvIru@qfu;=?rF{z8z&#uoz|2bOVzJoNNo7aL97{66A&QRH+B~ER{n) z@671B6p&lrYan?C(1|}BH2tK$d zASPWEj#-7Y*RH8)^#eh<)sX)DyDG)x^U+8LTEALmUzSIMI~4nsdGLSx0E6F`FyjK_ zQpxEnO=*i^?!j|Ihq(NaU|d*aFgK%5{u0*2vC zcKsGj;Ya5_j7?JW03_U#jk*V<}=`DPMEjy}Xd7(DIE^$~xLrPt#8$UNMr2 zC>S(%6AVeKtSD$yI}LlBHp@(~mZ;5UcblD5xY0b6f?KBHy0Og$P4-(r5>rDVVMtDs zw9>Q=KkJlT5xAdf>v0uzjg*aA8*Z??Zv!<=sLn|YpScyZ`>}=u`(R@UNON~$ z-uvx#=^|c&o!t(mcPgR`E_jk0N~XVuVcmKZag?E2QE48!9)@A#b^w;!rwFEgTIKci z>nKs9B52E`G&|bA(_q7$ovSq1$76(fddaKULXrAleJ|Egsj z1WHPLu0lM>|2rvVmTSAfvPeSQsP}I9F@)X5TVkqBB(DPq)jZrS&E*)*0v0sTT^YgQ zn-Y^miYB^?A~Bd2^X(M0iVHcJ>h_qj_bSTX@ zXc$cn3u*6JBCR@6=45-88k*1Wc>*+)+<%f3ykB+w$~66ItbeY^L0|G*m8L&q!__Nr z8Erj|@XgRe218$bW6-#sk-aHu4H{zdNkbcKei)WdT&a+4IV__es1>2MB{wR9jyuwv zDEeoEs=MDCRN?&~HVW{HH1l7%++ts}JOWi2c`C%0jvSHUwi`z2mEx7qeQBOF z@uef$g#SL)cOP{R$$M8py+x9e%0Oz#XJuM#N(m@$-h~g>;JuNP_^zVZ9QqI zvrSKdr$eE_gY4?Rz*KsYh7W_MFk$=cmI9gfB&{rsI4zyV!K#-JS((Z|BE7+7tQ`f#-kA#3$^X~x=0vA@5&ka%dx*A`Kz}@2u|)qCC|yJ9cyWL z4zOHXD*Ocnh|+P^-BidK=W}Pi&e)8WG^#6gIxmA6%rO4EIiSQh8nGiiJul~Gu_2kE zCHA^oQok-U%H_YOc%|sX|3KE&L!$A9r{)zVwud+Mp7FL|iVq3-{RW=sQWE1%bAOZ7 zpU+h6tvt$&3Vt)YmKYvn0oJ@Iqgbp<1dmGGs5pvJ_oFlRoM`qgs{Rm%mKgq#MUqzw zT)WyC0-)JkNTW2irm6S zm7x%OLnY{X6_`EEESO_gWd&l2_d*=s#v%^tI?C6Y;!>|s`aRM!J6Z5GuN%HQz3yj# zZt7-%CiWlYO<{k??;)(dY$4Pq_p)HB4lrQ&5Z>uRvwsQC{ua#f_)+DoO`LWEJfFle zX0%R?X!0fQ$Ft-!+I~}0d}3i#coXE~2U*0ml1KUL$Yt%%PGU?~dY?7Qk?j636h-JC zFrds(E#~AXIMXG_5Zb#UfcHmeff2W4TVb!vyai1Eks3_yZ3Cp~Z9pD69XE`&jl-oar%-(LL8Zr$~2S9fuC< zu^iEO*ZRQUgtV^i4y?(J*`c;)SkOp!{S{}2cVvfO6v-OlXoe;%CCqjH9a=J;VkF&T zfkga1E`+)U%GS#)d`H^9R|HFQ-PcT>^$y2fE`OR^>8`};wgj<`J1pp<=|+R8#V&mH z8hlNpXkXm@aiO+E86}-Z*dFwH^Efbl4vXn6(}MM-KBG-wSv=PCq|Ap@ld0$SzGwACGg1zY3B{0rQle`;_W-=GAv&&0Etxz9C_Lho$#gDrpgy#4|etM@{x|lcegBe#lo`&J;Doa&uM zwX|X7B$#SdN-sY0!zJ|QEmV62YVecZ$>kw`jy&=XZjHc`I>Nf}_h$9ICRWI6Zf88H z1L;D~Z1v-=U|-lS8zE^`=1CjtrF+qgu=G;a_x_f%$qul=1DEfx0$XX5E%!KrhyH-j z*5_l5m4!XWd(nYm<6UT~ZMHhe$AxO4wvF&4jxbfYiWukC7S&+YJJ~?k^6Ht{?zS;_ zginC;XO4HEQG~ ztDByfZ;c%wpZ8KAA@(wtvY>5Zh4gc^w-D!Yy&SUrZC9}%kF47>$npIw(cC!h}*TB*1m@0kUXmyzzQ zivn=BmpDoxfYet}Ih2UnR)rR&T;ReX5Oa-#aEnB??kvp4Bcz1?fLUZs-ZfiX|FI)Ys{L)>lyeEpP{yD>!($np&LBG|@|T zfvk6f7r>Rb>A^m<{B4qh?iv}+V6{^gob&ivRR5$&U-#Wgf$Fdpg3il{W?e}4u)gEu zxB&a&c{t&)Jm>NCDZ(dP{gBKjPpA8!5|_TILh6ybuc3K-;tQ6|cU4F;YTaF)Kj%LF zy)RfAzg8jj=zYoYrL2!q?kx?f5F0w{kH)AS2JpVTpTTJp{DA$|g|S7wv=gQHj(5`q z+t|Sk%K&@PONYh#x@FNEJ9IZpHeU%W#%E~*MhSg~@V!4Eaa9FGy&g`}7c{2YLD`A_ z-~N7;g8Tn;^~cLXSDPkGiBoep@DbDoWbd3M;={P65vnP>>-e%%Qv<>G zf>7JQj*1k$YZ)B@e^+bLPe}E9D^3PM`E3@Y(dJ>}?P*Uum{Z=%PMQ7tWiW8p;{*ZY zKD3663jxF#*EL!o<>;gG+4`NMeaiz-pBe>H4gz>v5U1r_8?vF&)YqI zt^h}L!K3OGvKQ%E_q-25KXQzc41<9$RjFvGQ@ttz@-CfHD`scu(t3OZ!I()(+MJJM zBjL}5$3KEf`>BewLnT9(ab(VT{d{FCo<|p^H_jO!U>$VxUub}zn4y~NK}#2occ!sh z4PBx`6%4}%XmzFR1j<-EzBH|TY zu3rtRA&-?;Dv-Qt+43W!80ueF1O3lmsstb9#>r+So5qAS3S!kU4Dv@C8{XF)S_35I z)+&&cn%UU0o`+t->QUT&7)#x?1E%W+HH{x#-Hy9(v`^Y|{-1|}``s25r<#ug5k9oH zCNh`0Tb24zr&`(iuiz7Ff#CK51p@oAUDkM@yh#`!D}PcUP84?vC9K7#MchjEf9kLx^{R8 zOY7*4)AF^$c4tMhI`h-JIU7&YhT4u(gw(N&V=;>@zPIn^b0YnJT|!*{ypVP*YF_LS ztn@r=aDR8b?CwRs^Qqg9a1(dE?AFDtwJfW#bqa#QZjbk%$Ae_0=yQ%^Fdsqu;<~Xr5)o3UuXpD<$2;5xU)oq~)qk?v_0dfX-3+CfnD#U{_8fB}; zSea#WBd|=ot%2mHWcyK_#!?&GJpGjd3CW+}N0%B)n|bsv_gW(jZi?m0&M^|gk%IM2 zP?>icaO%<9vt9A=DbR*eduI2el*!qSRCa8(dy&`KQqN|vbcclC*06&UaSOc5gff)U z+|aR{6X37$z`BJG_s89z=tGg4C%9OHbDIO)##IA8Fcw2?^%l|xY(ZoT;Bawx5vdzp zZ;`Fvi(M%kkZ*ldhy#Jjj*@~V1X}BTAi!tJXuyvqVfh;pWzf5&CHhf3?h&3HM>Sey zH`KXhLMxEesGuQv)hb){D^~*^nZBx`K;Aci3sz;y(&4NE(uM z8b_*DHNlmZ`%DtYhYYECX1DKCRJOuqN{akx*}*i;#em~HO%aRAh*pw0w|%yHw3)N? z_y6nbT%)Qwt~hSJLBa(Uw9$$Wh|xr>SfXw0V z=W+L*GqcZVU6{HAO|rW!X8Y$DJYVpZgru&;*mW0b_<^tvpTkjruSA<+J1dWK_A{AM zui03@BjW{JJWkY%h|%Lf3{NzOU=}xB3>=zJj`_N@Ia>4msU^K?JXHBC5HJgi){JAy zR9bR0kor^uF)m%uS)mSTT!15G6D$TgF#(Tl@<>yPHGeiCMl&CfS{(zVF;iCnq!V*} zssyq=6GdaF2ID7!yX4flu9GpVE>tQ(tV!bN(n?S)`o20_=iIRtt6hwRmWo_SOG|~) z)w{C_CxOezH#lQd&;_3Os|#}cMut$DZutDx7CNpImwr@;UvyLj8;kyO*!!+U=%Zoe*7(^R~J-DepX z_v2BORvFoxs|K^R??_eV$>5H^gMzXTOR6IO>codlrKN}b>($f2t*n>0-+$(W!!L1I z3&T5Z`9w$VHmE+VsYd+9(984tGoa<=1_OD4xo1ih;y&fknW*^NpXtc!by{B*GfN=! zQ@3gsv%M+mwipfF{9h z^7_gLr8l`q(jQBAJQ_iKlIET6_!Ij7kEJ^v)x@8Q_4b?2lOW#YGF8k&2$|5#pNMD8 zPn^E2-);QNbS)IkrHMH&34QYJ9nLx1W-QsE=X}_q@q0-@!DA;r@p)B53fWijwkz|A zWo8e)Z#;wFY$d|J-9;TAFM~DfjZ1$*6VS z|G02Ed;M7#-Udkp2_KJtdBdT_6+6TbAmxJ`n9ZZu8_C)vc43&S6DxIyAdx2WR}PeL z2wBn{VxN%i4t@JPsbql>E?@SaMM5#fT>}>5F}1&e1hSorg_}=hU&B8-PJhur`Y|P6 zbjm$`2(&N-TKb0>NFeKxY6(-n{f(;&o1F@$VapJmi+EQHVzXaCt-eVWULV~I$cfaE zEd^sJoT{%ws~=0PC6CWlz}$f6bLp|mW#~@ z`n9@H!WATX`{zeKKu^VFqNf&Y=12l}eZ|&$Cgy2&nIRybxEs4QIM;aZLN)l?3e;f3 zcFt`u&C@k2A=tNDSJ3A9Zt7A53!$vE zR#=FTDLFuHe5WJt|KLJ`D9HvQVmSXVj>NO}BZk`lT1KX$Rl*THAo1FJqGbBAPOwcu z!jfE=_VgnIe#ykqoQQm?OWThl2KgyC`JV?Z*92#71O)-ByC~*0_zym;u^0{FYT@YP z+;JDBH;XB-bX~p%`+Q*+tPySmnJh)djLHWb5vaq6G>piLqyeiTEyNv2so3XOQ@*eZ zVP`nDCs<%CYK>Sw$RwE|y0rZ`WSAQYx`#^0ez#CS)@XHO%ue{FgdkKgMkT+9%RX8Q zWa%ps5)r9{cpE6a0HqH@XKXqsR8N#G=Fkt#TOXqiDLf(ze5n8+u^K%Ql70iSU(kphIXoNLW6b9Ppjtsu>sG~37k1%F1li7p8a)s()> zu1fXeH@kduIMPm()mj7vWgaOa4I@|~^(%`M^KVGFo9oFpEw9TMR1Djw7jTz}RfW9k ztO`ur*EV6wg>T}>T!yF#KEJgkcQ!#wS8T^}1mPL-d|HvJ5+J8O&}MJ=WHVRW|K1=i=G%M@j(O zd1nJ|;#Tr8+eMUK-wO>of*X7y>Q|E?PK+BMSNJw>sCL(wq)@*52 zx)g6i2amrkVJHlzS+-dw;?ZqT=l8wA4KvxDyo75%=+gC16+d=*zU9SC#C9lH_HP66 zMRbc)NK)s)%HbVLo(O^=qNQ-vmo4^Hkxt4wc@KZ61=|ujsrV+9HjSZNL;`{EP-XXs zk||{E(0bWUbe2H`y24T6)_8R%ls)P$GqpQ~#rPEy+sI&#xM>$wD_5oP+$dhyS&3qr zOibATwXdm_G(8vWLRoV|g)(#ctFYY=4mr^5jsQH4sDM|i|lRbrUa(=AnaI`6k0F~WR*ma6P_=GjWCjsE4k+!HD!}{7u^+{3 zn<#_WBIwQ5ZxbMWh~w7>pmEeR0os|*9JL(}@ZkYy95q9Lek|jls6Cmc;UJj9b7UrM zk&5$l(jX>KJp}O7LIEOmmc-CU3Pyc|npH0qj2~N+)n3iQa#izj((5bbhp>fWo<@|T z_WWO<{kG--F@l2S*I<0Q+QQr(X;YEv*Y4FWYD?G=w56eqEjuDc6Nzte@ykUXRSP*n`5l7;Zbd%Om#Mq@iLu)k4Lj-GZUb6T_7u<|Jo&9P}{Q zrZG>d{5pt#SK|>strP1d<<5+$hsCxZ_lU8B6JhEz9^kV+N5O=A?qRSs6KWc9G&csE z^@t}Qv&;?qWffoeKBfiO>f4xy$E=Icaa3~6R~{9IjtfijZ#@>?8rqpDEoy)*^Y3cm rl4o8h?DZ>uR^Os_4&h(fd4~Uq8S6v819|>={sX`C?Qq~b761J|aa=q% delta 178578 zcma%kcUV))^ZrdJ36P!$*Z>OC{|R&P6+{0=#Y?LZy+d&VnbOwh>D60>#>XV z+QojYVDH#_`)%SapD#1P-yb{=&wD2C&d$!x&X%+1`OiYjcl3`qUEmBB^-IrF4yJMyWz3`toGz9SzIcW6jYT#&WH^$oyTm?F;O8#!UMrHp59nY3TmOwN&#inj<cXnw3#kAWBj?*%zNlZkAI@>|bm=f3i_CoQERrXpQ)H+VRkm&?uamRIw1S*r-_ z={LkCHQS{_(c*T?%`1s`*5B&e&hMQrAj4fPe_3JvrgTy6_%4VM=5?8asM5OIXsF8m zZjGSRnmkj~t2=NQQx9F(n|7>HQkSGP3xlp|`t8?VL-5~vUZK6alr+sf(;NRODR{;H zB{4hL`uBd(+$!CXY_BVCXe7<6k#0viNz%n+pB4QtvWSh?rf#2kd}xM)!Nt}P9UHNp z2hp~P^J+l!M*3x>KbB5IdXFMI)D{vPpGU(-)kwaz?gC##vG`>N5wzo~CU~?a9@A8= zxuSIv5+RY`cFz^{^2l{rFCanLDqMn?+Fi7+`gS*xIFX=pVlyO02Ous^>A&%SUn0~6 zs)j|_Mcf)l{fRZk^4Kkw3u(jUG+p-YpdfT@(v|is zAnNnv9>U;RITz`k=U*YNqJ6DJoTBnw0aA%&A4`zJ6yLsE0mYaEHp>}$5VMHOk|FSZ z0m}y=v6Zp5LhYC{4y*@uu*gFVSe+4WaZ?r&v6%vQmJT{Iaq(s;A@y8u);vVNHf>ps z(9lHDo)rqB6LY*hD<7JV8QF<-4eDVGa(T~uTuE9_mWb4qgqrjAW&)E;Es}q243r_> zfnt9nM*~vXOn^K;*o`$3s7iWuXZ;C=RN~x&bqk`ro~)-3weH1Y0Pn9702RRL0FT_^0tO>Q7S7Is!g0mu~tIi3!_-I1kfPgrmz-6;ni8J!wB=3 z>8wI1>^_4v%>u$lXR(GO909qkg;4lyE^9TyQA}7Y3sF#&l%CnMh&8|xlGMp(F$jEb z1#6rYthshAW~K(kti>EieReY|j0@rRyIJK(Gd4cLazolH;v_4J4@o{;WsO9*$5*fh zB5?U*Rv=QO?;BRGE#$8F#(IXh-wiH139(c#pS=qqsprV{kwWgIW^7v?v};yd_G&2H zsU7~fn>O1||&0qi~bq2DRLg5nw*{cw%P6%U9u!aTl zlvrI3E7@~+kXo%}r($to6uTG-7mj8(K%DaO1U3?%rc7kNMXWR}ll>=R)5vUgO{6bP z%VFO|8h7jhb|1u7uPkF%AO<a8{0w%ZEbagy$ITA(-F1< z7YbiF%|;^8)HCcnq(F~LY!0Gc_nYhtSVhw~@Ms0e)$!ARQgOi7q1DlcWW* z;H-qKc+H*^%_P?rG5zrpL?_8;iyQsK|Z? z3x7DnV#LiY_9Ky?+}mOeY$(#gVlmQty0){RNpjVtAj0;wSO+=w`C4ED^ACTET%^A) z?`4rCge2{PEt=JVup!l=G186KjkG92;Ha?{QxMnxm~Ih{NUJAXM8cUbb7_V}BVV)2 zV9RxMCf)d?MHXz211BxMBB}Aoa~6k?^m@q!ivn2Rz85V9A!gY0(n5)(5%u0$MrhV%UO{66x;GR67*LHEgcZ`PD(77K{Jh#S~?*UoT8eRXORAr zQO5{ZUC(49oh?Uj%r=sEJZ&T^H_KF5`ZPC7wJY@O@BJ*13~)`Lq8RHpO$qISYFRch`8|BhE~#AkYrgWt8x*9bNXAYvxji6 zXe$;X9Wugd9FjFXNVmF=j3d{wt==MC@8%q<-N<-bw8ZKSk_4_Pwz3jH1^4Z=k{}_Y z{7Jd%dish>qp0tfP_Cct{uPMbP3uyI9{ug7LH7)-90=dGxd1 zhy-4b!+8*=L{2)s1bnvTpM;KU|cr-&i^WW2QxABL0G)2$c7I#x`# z9#t0#$F8+r3z_Q`TQ5ZPIJv<(+YWN5cUdE$R;6n8>&RB;b%R`kH ztour#bihOF2T03wernA}X4vvi)&$nk;^O0_t(xniGgcFlQM;k`>BXj#Q?df|CdJyRG0@!f0vAnuW=nc<6xBk~s?@ zO$#lj2HZwqhK=J`AifkaYEXx>voPK(#hfva4sQ;f5 zId_-i0DQwFQ<0@HLrRrjI00&6bDu**1F9~KmE7kjV9Nd9!mS(e z>(7;$U%2%*)}|&Y*ko?^oz6y0y17)hgiASoGcPr9c#)3J5$hW_odb#26>cLEWU#R! zr%|B?&uO;<^`H5IQ&9sNYk~!L4HRx;$z5s%mlyBZa03w7SIE6(1xcFL;6C6%66g9{ zdmAX-y&2aI88su@aSI`bq&>GA5?7T#mdf(MQxo?oggKMX`MXi_m z-x_8a`#zwI!?$WpjKfIIbFMQ~YVULIPcDQ{z328vyr}6H?tKJy`pMmmz>h6#LXb{4 zz{VyVDRRHqrlk;S!m4NU7+Kx=+}LIv6yDavCX@qVCvThPaHaOY5u&6ry*7EZIt(f2 zd>fl_pxZFvZEe_)(70by%-x;M3D)*ZIY0X z^Yu@gobIp*7VvG0A@x9kZLuqad97^&#So6~VOtaFC%HpxD-r$XgxF3;%<3LtYlj%B zr^@!EH7v4TyzMNcTa~8R(zRq@slws54oHzF#@Lo3MJ}9ZOV1Yo5_yL0K{kY&PqFnw z8fW}m+Z0bI{o|nR87YKsT(*7c2;s|>wl@(455BeCg(#T)(RLWpI2@a(>Diff^$@o@IL%Iq6nTG!-O<{x zI=j}`eM3l&ZnVRqjCzY*DWXZM?RHlXxYka)=Ll@S$8H?b#)Sv%-XPK|kJ!}_f*oCx0o1Lp*tsM6rQEc;jf~O}5A0ea23+~nt{AC} z;)Pv9q{z9i?A#Fr&%U!;gOE)5XxCH%{k7DFw;I-Fq%H3VQkyaV-- z=!pQZ?Umevw*U&;_vF<_?9{6t?<>*-gMxT;@c>Y#591-5`JTZ%G1BNkVLTUPf}f_~ z(MSTQ-$(HlL9O$Zyfnnrp0T`BNM&rbJT=mc6H<9dHc0l2#{5Y?hF1$YtFa`5w**n7 z*EAkIg#@}-Kn~9vS=O1p5G%3kB3>JWW9m}gXap87=RHOm%w{F841u2%@v!J3Ddt^7 zxZiK!l_T)|&AjbML$2MoOz|1scBDvc z8IO&)RQg5U4%jw5FY%%fC)s(8*Bc2@;@dndNBDl1w*V$cg!Y6Q%{Go^@&fWMjq&fY2@{6E;pL_BRuioT&Sp8QPX*I+6#`1CwL)#uEXDaM8d z@U&?t965vk6e&A&4j;?;o6Y0zL!>_~;2%M}<@6H%2V@kyu$+%9I{OszBao%5bwzv{ zv#Q$Hm^+&mol6rYNgT|-5mlGavq#>fbQj+$w~w7IzVuA$ofOw4k-b>1#Lq$?B7FkH zZlJ{f)_l8xXTKI3=x=|!(o1EkU2LGwf4kT~WxU8sPwe<|kIno!qJdQxu^(;bf3|?| ztL=Qfh1nIWl9Wk~M3W-)G#W1e{<_v~KF1Qm-uwCM5%|R+{tEYd&@)IQBh%AruyW;2%P|#=S4b_zzZID}M0#NU!_O79d7lYayUVq=8Y7 zaRnW1AuN^(Xo3hzoMJB+j#$;Mwjc^oWSp~rK%@oXoB~G6upl5r5*4c9f z{gD=XvQW?-QKZFE!78MVkMaczAoZ*Q0nOv9mPqKLg+1f6QZNnChA9$Efj?pT6$?fp z^}kdiC_+fGw+aR!{_}znkQI|IoE=1sf=EAcw zNYc2gFdC`LvK~Sy5-<<;7t+0Zph@i^LJ?x@j={qE2zQcPC`8~Hk;0D%oU0MiR1p+8 zJ6;%w)Uq-~ScGV@a=36hLehDRkRHPV+$+ZmYa{TvG+_fI2(mMUa}kbA;Y7rBTIL9k zBOY%zTUd&ecq3OxH@1L&Vj`rAu>d}?PCh215}Vun>1${mkCo5!>qp` zbVGdQ-Zdc|i$SjY@}_VFB3*c2*bA1+e0gki3ZQ)R=R)Lc6!Cr~M26@quY}kUg36CV z`tCWP?)XE9BxB4omS{WDzf0^z>k*SA))b{9?)kl*=r06*&{P!8CgTsY+Zgj?FXDV{ zl1(*9Rg(F6;fA|yWK&+Y%>AjGvhLPWS%-azMGo2CYu%0&Zt1<~ zwmMVXBO__FW^?9}l`W<;tiP9eSNF ztQgbe8M|%rhfwD~8rY<>cof|ka{<_6qFQzZwU!3G{qIcO+hmo&lxa>L8 z|FCo2iO+J!$CddyjoY8)>#^yRQ@kefMz(avsB_&_LGDf0EgJZygTv~BsOCilj_~Ux_$Qfn+^FBAJ5yPdAnPd zG=16lTLC#kJwKGzo|pI5%GTra+2{+6P7Pr-y%yJMpGRfE#1|(^Qvxr(-!!u4(%8}` zc8{GlMfnY!dV%FLyH&u;Iu9D1oUrOGPr+3+JW=Rnb<(BaM7xvq7gb!g?DfXB|9}?z zEz-~4>2Y`8`8L+FrW@8NNB4TOxp-#%#1nr#+aca1KT&7Jy6Me5EoDy^mu=s$pvU8Y z9c>OY50G~VY|<=F9=F}uf1)8QCs|uonbUpwot`hZpKN<*a75OE=K+^L4P&NUefZ;p zb$Pi*nfiF2nI~04WL@3%N?tj)`*6n|L9PHsAx<1o7Bij*1aCpxJk zS5I%+XZfbtpSzYFW|~CxcbKGLOkPP=+_PdnS( z^;g?IpLcy!q$7XXp~NpY!j2tRORSuGEqvJQ`Rarq_a%4N_jl@CvS7`;x}HuYGcL4y zG2LTOapJzG2_07To|ts)?EBJ{H8uOX=ikjb=Qc5d)9bk7i{|ySY_|{XG&%Egz(C1g zC$1&rJ{*3^(DCND3l5!TFKC#a)9UeyIvH&PY5H8uxj1|NNXsEq!d~7{+_w-raI` zOvGu&Gl#a0Z*%b^x4g}sZz;PUu1js_xumnZJU6@M()T^q)^hIBF#E~dn!V+DAN(G+ z_WpB}{F~=v-BnNRfdk8WU;BEyw!VGL%tw-8pBq)~xyQ=+v&)Ewe+>5ETHJg7zeAh2aW(QRbtcr!pm z@5%s*+#D*pg}^7mL>CcweWYjt0(a1e9@K-&5uRD1Ik3dCEKv}mw%2qKeGeSyH)ocJ zPT0Uo$Mv})WM(!gS2PV$o2K`SClS$^86Y~oP=w^m`b8qTnFf@JUn<%LcRx+b9c4jb zuB7zUq_)JbNMuDetP&w}iesxVgTxexumsC_t%x2j06JK$7i~h|BO65$#C#98hy+M_ zVYypGH{AgDqrIXd2)yTjXgR{2ctq46X_`}iijY;r)Z?O;NP1y7CyItG#yq(o`i>~_ z;j(BABE9jNh`zB1ns(Q15t9BW%0+b$j_kXl9K=1`9$*}&ABgC56co7Xv1lbUYRD5& zB9i4?el4O0|3MQR`5@W?&z_Sh0x`1o(NrjQM(PqG5syZw@7jwC5MP+JtaJ2oz#SWoFxAae|B5`Zqxrk2eq9VlW$NIEWsUV8K+@)l z&0I9|uO%qOZ!Dn!N|MQmVq`zSJxLso><1L-#dKZ@%ra(#n4S~|@VK$!D~J)58R9Zz zccbe}@kqoiPs|msfz;`_;w^|G{zTkP2&JbNiMxm(d~=8R0GmAY3bQ9ccQ|fDbV6)P zJZ9HN2CmwrVg;}rGp$to647Gn5ph1OiOC&GKGN>sS0D5JR@2>RPi{&|&WX)zmy*{K$v`8A8Fw>YYH502-MwdNNrJ}Y zRqYH1;^LCwOzKRxhDJ42Hf@S=-?vQM1Jua$`1T0B>9H>f?;+t5r}K!>NZa$`4X}0J zoflt48vVvyaRB10tsjZoA=2YNioYO>7Fl1!Kae;upCdu8TllY0XE(}lGIRgNIi4gF zn1E67C01qvi>Vbwu@W;|UB*__0YYQjjRxqihA_5Rfw!d1?;5LYs>GKFY9JTu5w{wW zeAp0IYDl^uBf492Nih_@(_G?-G>=OQi3E!oJ`(IOXhIjs3P|nLRYKQZL6n-;LxP1m zL0`#ndno--Dbd3gA+fQNl~A}&oP@sg4NCM(lF<9*06bPNaYm$zM@SALL1ye2$$P{z z)5c47Bhmrsk`oBrV6sGj6xn5(qz(*VBq>L-7OL$!Q^H4LuK!Yr1Khqdx!TVQJU3F- zQPP$y>zKhecYcJ}tdtbN;ts5o(Dim;yx8@U6eQ7lvPpu3hN2S5eWd#?+9zRiA@%yB z61vC@N>rYZ97I~NPbs*wUYLS z#oA%~}@ zG!Ws)X(8>$5e=$xz=z!FCY3VZ+e?vLn?$XVIxsq4=`BRla6f6PC7@#{9X;hywG?J( zFSK6g`$(~~Gb#P05l9Ux21@BxDX>|_P-#sV1_+h5XAXr*$0I-KuaKTYW`G`1(rn0o zJW5KhY6ZozqNVf{A%J_uOX+z40Iy1tZbhVH^=M764wuFtB!Q!);Yd^L87qCw0!996 z*>mHibe0d)@kp1_%m6e~yG*GZkzSc4rE3{LIy+lRUsbBgZSopIy~c`JIa7Ka(Rbz? z=_$AlL*f=nb%2d|vQSEQA#BAf+owz|2n$N(13|#`HWUEkv~Zd{Rp9 zX{R(FISun}NhM^oIzwc3xZ8h4nhQ!LQCFqofDM=z zSEcky+^VKw&OVTOv&q?;(ggUd_bu!vg#M%AQ0sr6-Di95qdJO&XEciR7@`Ka>6(%X4rRk7^{Y6S&x~$q-WcN?0 zGjs5p^a*^%&}UreY~%u4HWw20x0cy6*%mVT20+z982aoK0ynjhTxGIB2t3U}M$ZTU>B?F%dcFj}J)LCKaRE#< zkywy(4P<)wY(PU9eZL6&>Z=!APhhLD7{l+}RbX)TQ{2FmnogTl1iVYM99QDzI>%GA$v$LUSweurdSa;uXJ zPS(kXPBH~7Ufo$X6iJy+_LSu#N&4viGA$pHe3HvnH-Ye`DY8i9?5=Z;tfo7Z?zKxM zMAVKsB#VOsLd7B3P^7hc9F-+N;e$tIU6Hw`v`m&}2lZHSLv|9T63qBpvWYfOn)gVi zK_(ID&t=KT`2O>)Obhj!@J?1|0ECOX*LW}t!ud;URPF?D*3zN&^v{7=I@ED=J@A;3 zzxJ(=5bNoV5%AgY>5d~tLij7YX25m`57}9B@e=^=D6Dn=EY^UIPy>q7s!(m#Lt*Ny&lXXMX%bQ<^1k6qyJZpZXwZpZZc z$u`R5UF=hb@Dq9=4!l&}GQ&=Q=MN00{Fl0_xTKU+RWHCd($kov{gMzRCkUV)l<}ejR#4mUo}N^sgV|-6ybpg2^ls zb_7DRFdKIS-fLlcZnC+@EHA|7eP?+weP?;E%7w+V8nhUF2)1PQgBIyfxc0*qSKR@e zH9ctn{ToRC1~Jo<2G7__^>HB^&$W>NS*Gk4C(4M(o(1FteJYrs7=+N>ygj;Rs z7&OgnW~IodnbqY;DZRBw)RaR?@-yuJNe93f#B1yGTnb|ww~x;XB--x@_DPXIX`cii zS7ap78+@#gktA!3PY@C}A5HYxgpd?X^|^;gFP!PK5m^%$w$f)DG}pbAJ_Ses&)w+L zPz-DH=K-Hl&7d3hf98Xv<|m)|v_tABdFw+zrvWB2R-b&{xWW=S&b}Qs^x7y2SeNrM%>R!CCPZk6wFSRfr4XZ3+F9jALs zA-vtP(;H~jf5yrqNNAk?y7S<{kmzN1zspFjaf$drw||4BGPJ^o{%Ia6C$6iq*hM*b`0M6zEsM-qtt$wrcpjGrbU4lMz9*57GI zO78#g>lS2i%z89UMuHCgE6gJKRj+*cdcc|#q@LQncuU)A39Ptr5b7CbVh zISakX_+vxW@EOfoE?*Lg7=O5P5mDbqziE5cb$uN$gb4AXe|u(!xpHkz#; z77K+}tRMDL1mVJc!}5@kpx4P^=}3v!ZVYRS1QnP2!#Y|+?u4hq=+Zk-TTnUd032nQ zTW^QeK)Bi8hGinsd92{8NRi{Y!H;XgB8S%xRv{#h-GZ+m20Ys$80jIsTL$k&IHc`^ z8(G3(u$OOe4d@w6QlH>p*w>lMiNPxnb5ETRJQDFC;k4i!7-#>>duXiXlCmoqf`7ai zjK+251|zehjdO$Tt<9##yuav>)@qK?FBb&ckp>HcYe7TzTNpfw1FdnnAow9-=u_*0 z;YTx>HtU1;!3b`uo=ZuvAIL3?Pw2<(TuI5vVEby$9Q`Z_?v4JK^t(+Lg6Oa7;di2% ziZ*@Lj>?jV28Uq2FTyA@|6T zkVJ&qFEnHY6n+~TG8W;O5D_vBDbYa_f}IXo7$1T-aOZ@O{Rjs?C4|181)|jBks+I* znbwX9(ID1y8XrP$z6M4eIw>TZ3*lGUA*U@Me0pZcdL(>J%?s&@a9>{>@(|IatRTb( zf$OXa`G&xai$ew@@QjTiSqR)GnQ^UYQ5#nD#Yz+-9NlMxm-cz_7eRg^0(vvCCT7Kj?EaCn5hcTY+O{eNy2;-vcos!VAb`UN@4oqoK>p@5q+9Qi}Jd*$s)VS!PEv#4WQf z(S_36rK?tRb)yrZNK3Xp9oh@=@SF>wO%Um0mqU9ZaNTR6N~9TP-3o1lNFS*PorEN; zL!XAylT=mJX12ZvO-1t9wqHZ(yDC+scl->Ef+&;14jX{L7p%gbBk)<9uhUZK}vGNtjw2X>f^#}l`Ek- ziOGysnbijRap&oz<{wEHGP@b=hE+S3G1}8cstTJ*GJT72cI(VKq~s}bU%5%fGy$V! z{B{CHWqvyWqcSbb*Pd)}4_gTP$I#|sVk8!g^bT7Ogzp%fc zZAhO1VXFYgj)7tCH~?iVL&I#j5Oz|A`AVRBB&3GXmr#J$3>X(iuQLNMH!W-c!W}m? ztQpd!SI!Jew1S$r%nJ)Zq{l1@Yk*jF!?Lh7P!Uaj7`+q|=yzsC*j8x1|D4!s6LsRf zJ*X2KKl5oeN=EfCM#(i{@PnGn{54_cpqBqRFD;=K{+`l*&P!!}>%3Ivx6VstI^z9^ ze)7|mR9p>fTdSIz!-OT&xq%6HmW1(;`PHQzVJo0;785oe@gCWMurr9Sy*?k-%KVzq z>_BSt`I(Z2M#X(dMQ#}Vq{RSh0W9_IUtz85L)h(W7(EaPDpdR{<2nq+blea^J^L*vYp>WSm@?9_-5Rtze#%t!Nznm_AfOM;8 zZ+UHN2&d{@hH!+GhhM0eX2JEbx_Hu0cj(%{X~uxI#{r zW?}{QoGkaSfYg(x${QfvYv(M?8)Ik7yC8b(S}3PqjRxK^pir(rW+w;M%a=mo#0~Nc zM7^IS^1Ey({d%k11(B}D$kPyb^KSWj7>Sr2`{g^~3o>NF6?q5@Dvz$nH(5eOif+h< zBORmJ9XUP4f+mVD*>SGKOByaDw;m#~;@70NbR=CX$iFrQ23unfF})p4Mck_iTW=r9 z=Zl6`nIDk_kLB4wt-k{%{Q#U75sjt`5S?!55SO5D82S^FVkI|p$X7CNQIt5pkjH}p z$if%$B7ideE9GL2Xjl~;8G4SlDVr?%h_r$!JkS@fdjhim0*7dPW<7i@z$9a;WyKe{ znPn7mha2Z_Y#xtlYt7U=VZK|=0zF9bd zWz}>K=ObaUO^a~E6mMIEx5R|ohp&drFWQHv2qB!@JA5_le56tT@MTc=V*hX}SY`4! zG<+o#UJ@1_YW_4#c@9mn8byWAfImDK72XQ5w^eNTNDC-EBO#oIv8vI7)X|1-g~H|9 z@F)(1gGPp*Lww3JE&R1Dl=hk)PCpt5+@$WD@G@jn{60Ut019u&3%|^R@Z1&Q50Ms~ zS{%L<3b$StPUAYzL$fh_I)ry@38$|m05cufjb>w7Xb)2#GhHlPT0obk#y<^*afReP4POMy@p~3N z3L0ni@k;syI-u)cAHwN94^``HEIqGIl$SB6H6&>@Cd){;O9-Mtz}N82kksy5cwNN1 z+p!gk5cn=v@y7fz->V{HCo+B`uwsCs4biR6Y(xT+SsYPt)p25>M3IOzU5$E*5~$+k zdWw9+9GOiOMaWl6I zEfW=VOBobsIv-cDo#r*h1q}H7XQpBdK$&Ti6&>MVXX-6951{$VOogDDMGMme&*h84 zxtb#nG$R-W3jUP`&|m&056FPORL@c@gNdSKw&Evp2DDzT0tuz(a}{)X7c`{@QP8VK zL7v@iv7(_6!imL-2Z*Odl_-$4zPg(g+YstI+Z2w-1kYxV;xv-*KRB$Q0U7AG@=wJ^ zWY(ZOuQ-Wh)IL`f^mE~WWYG!?2{=)2aS(Q>{hULXyh{avvg3itn^peN4&N1k;A-OvSa zW4j0~(&JZ)A{rwduv`{V1noM%M#M&>g$LD%*k=o+UECu2!{LKWY#M=hv1hXgx@ipp zL4;?-V#sXk6>$%l=zVVy(GE$^x_6404>_K8il74yaGiPGBj_$UfG78ku!6JazlXxX z!5JMkO+DQ8PMsVBH|hntc$e~$a2qM zU4)niIfjpmsE>SUW{&X~A2tpEmGD3%V+^w?_#gLkui+F&z#f*y)JS5Ppxe;+1 zS@X%f7qJ-WC!UoNc1Ya=-bK_zNDN;iuw9vXKO>d^wYU34dLxI=(j}2d6+22J>G3Jh zy3K1w($^&bJhpBm-DC!ES-r?G#7-R>MbfJif%Nkxk?moKC9S<8=Rn~rUXhM!>u1(ENOHppHYNpDgIB&7wBB?w8{ipUyBX0mEa+L{oysk#l%R2z`vqoghT%z zvH*eWKZ%@-z|8Z=a-`}-FC*!Vs6fBtZzESBW*z@Ek{$vC<5I}aNH`2{w~Shc=yAXz zsx<;za-!&GRe&D$wo$JTt?l_yjS=ZUQPgBO!v8%CBj4~%uM>r=8+EE1H3Lb7uGf!x zX$KRhYaUVEvHZG4R1UI~a;kmQaYTBBZ`5@J&gl|mjd;n59#PAoB7=KIc_FJ3^9Dri zWkc!6!BO*(DW#Pn>N8SzfjUZxw7Yjq)GA2*Fed6Sl40&jj2ex=T~neOA@IoIQOyx} z+vuo0h@mEqkD{FqB(>?2qRNpJST{YYH&XW#^P)1y%3YD!8ST0fKnO~kd*LE^JLbfDwAQ8Dw zrmrovo7{j*%%=x_fFJLut)%Z70(iHJ65IEi;HpI2r-hsH7@{J#nbHw~y*!jXk<2x# zjdCrUaNfn1_rcM4_ivW|(bRWML3vN68y?GThOgR8(PBsryjs+;=6Re~cpH&?Y znG<2k9ti9lp``b)0qFxuWiJc3DCrQbd<~;N+ zdh;2Ob{wmumuyvqP3=IBzqKT@3zY)={$-Pl%S6m_=@XSBaZY+|6FuuNQ`w44Oj9D0 z;{|ET8PG1R(v^phNvZWLq=#Kr(swX`bmcWAJuVF3b+?uDJA(l3cuz@R%>Xd# zk@5us|9Gn0iB#vhfgRhX}S z>8p|;b*wW`g*4-(fhxN50klpER?)l@v~D}OssIWtX(lVdD5T9$ZzK84pY{zupBk%J6ekTrz%!3K=-^W~h)9C?LmZ zJCJ->&QV=NQoTocD(v#oV+&P_p+!>`sptWCaQ@`(Qk4yoxP4u&N=3AuxK6bSQny;K z+JdzBhfS(Zh(~tYszRnEFSe?LNX;pn%qC-Ysb+&VCAD@_B2b~jyHy8lAUyYoioO*9 zq{EJ@=w3U3Kb%qxw1jYQnW`ZI&$y(br?~)$^IxiFhz&#UU;%@7SEWFP`!P?^u6OCB zsv+DYCiJsy^wXsuRdhNH0>RVIDmXCD7iwxT`@g8@*GGW*FTShjx+#D=vei~n2nW?r z)7c0pbx$qzRwz87w)!qoqRv^}9)Wk&SJO@b6fwA}eQhCJ>ZRU-s6DB*nqFuNNGjW@ z=?Wi!8~UiPBCwsGn!e-$q?P{a0&56M0@QO6>23YgKCrj!7@&TDsQrG3+5zFH6QZt- zz&+(^dI%ZlcO^o-8-WW{>e@(=r()D+5qNE)dJ00)S*NDEw4lh|scO202H=?^)uRw@ z`*CWTc>?L(6V>!GbO3kEP}3zo06S!<>BJYn_p;S=cNxHbGu8BFX)va3nuFG|`+PM$ zx&%m0FHk!o)p@r>O=BvM<}X(dLr79rs_6?=KzhS!^<*mucio`A!-jCn5_Jhe@^-6Q ziI8Y^shtq%+`a092&^tuyCZPHVYL>irSoyLkBm5_rssn|Ezgvxg?12renZ_EDY9d^ znl9J^k`H&$PVD|rO^;UrY55cNOa%V&Turk+Al>YhdKm&=d#koJe_isj-E`fg&3dIb zDcLi*4Vh@6u_9i3Cja}508mQdCv_xJisYM`PAIBMZx?EEAj+(_(9juuRbhtBLE&Iq zO$;&_I495`vyTj+<^V!HP^#IC_@dNaLqFvXhJsUdG$pXZO|PqYi=>4!8)%Y{1l6vw zhB_|jwe6c}D&cU*v~8&wijXXAtMNzRR~gwpkwYxIZ{uUw^B20OyQ zBF!`;UOg+;U>QKYjhcH%ypnCxh>)DN6{A@POMJ;_=okZfq+*YT?yvy3%>fNvdIRv4 zLmK)fLsgh;xEo!I@i?Z@!e`8e6B-Xh_h)A{%Mb%Byr{tKcJ=tv0Q zZsll{S#(!pffTvnp=KijCp^>CN8qI|HFWE#ss)&=x0(}3Eh9f`uzAk;Z<=gKvOOW1 zu7m;Vy_V7RT7CdG;YQQ#%&IV<&FsLuvWv!Y*Y?6_Iiln$S@e3y#6&qpBWty!vUW5B zO}@QOv<7L3rY_M*2)w#M^lb#5?-orXd)4M7E+Nr1neok{W8pLAy+<@n1grjpncl)^ zGXQ^T9lZj=Mw{6)2ir%}m#?e-WOaRC%<)=uiLOBWv3`%}?T~|c)H~W>AquH_iZ;Gq zG#xstmdJ1hL?PmoQw>+9gTmW~BjHYXxRh@%mqqEAENVI|QS2*=ZH(IE&tXjk+oq%t@5MjwIHoAyQDK`K*lF#0$`xSDc zA4YFP3TXH=njT&R-qrq1bRZ&4-bZ7jqW$M+x;9%}7;muwu#)_@q^g3lh)2 z*v3!~0t)hkF~JC&Dvd$LB1Yg4gUt_z*NSH8y_#B@ZY4>ga$ zmcPQiW6mQ*c4{4y3`u^)@|-a-_1MhYjxkY4msb16(CtHDp1poCG^zo3R<{^zV2=%m zNkOEa^utU~=uj$U1OsD+BR|O<5<|lPDEC%Ki~^DF6CM+dzy~8^#zFX(F;wapY@KL# zObieHgt-zQ^CyG}ok-VU4kgE6DWtbPrVZ>6|F*;!NXIx#h@rc$pwiRVrc{gTFTvHpF2S~xKAv7fvBqVjsdZB;Q@@y2M5Kj1hUNV z!Lhv&lb;FaH6eBl0$)gvU5t>7pAy>?*=%5D#-<@8J?F++BXIcq*oJmc zlQ{*k^q3@2aK(yP`bl~Kk1UF%XFdTOy*8F61t`LGxS!*`dI98q}`e0vB>pM^-sjobR1aZL>NCsKxPMu;d>vyc&^04>j+Fw=smoD|GL65UQ9eIVmX9=HP-y) zK(7*}IFjyHV@E+_p1B&E05_|Q#hC6Kesw#xx;uwWb)_=Dxj!h#jHRy7dWb4wtJ`I! zb%j^Vkb~v1xv(zkir8Re+3obBSbD?|xY4WUu@{kmaPf8QVdPX%(PyMJ7~z*#KEmPt zBQ_6CmrQL(Z-M9ptnusahct`0`tT9`Hh5if&LR#upGQ5hf=x`{0x96e$*Q^U#`Mj4 zD&tsK4BkRwlH2JblF+xZ|q6@o}~gb{ZC68>vk1 zi1=`XWJh#7y>$X)ZBtV)MOy0P&mnzIGCH0h@UDsR=lM{-MLF>o%w!U#5u&Vnf~=ae zVx|#7w47!wGdqUD@y|4NnMR1&%cuP2njKd{%W8~&N5J@ceLteR7Ef<;Er^F18hKI> zkBsaymd77LTw1XzemS&+uqZwPX{iYtTB=WW&#TU;JZ)dc)HLC)ZTS3{y73~e`Hh}_+5`mYE?k} zq%xkqzX9N}@8Z`Y9iY|EcxP8pXw`iLRigxFB)-jUoUjwAWN7mQ+V_Eex7#HkBkTC~ z2@8-Rv{8=)Yb5FF7?AMR0UD~4GJ)=Eff8p$Cm?>-A|_!mG6%aAm#`Lr^AZ#29u=@( z16{(_hES1*^Ad^>wU;bRpoc9$`<_{nFdIo)lL`~)${&!fT$O+w9UZwo!4~N@b2cT+ zfoiuXNvMZ-gKB#My$}FY(yKJ#0TTBb9Zz_M^gWmJ33O*3ki5H^&CQ zlO`f#V>4M|CC1^HScpW)8;ugN0}mce6GtJ|-raJ=rBNbiyWaAiQE1>y~?z(neoKzinY#A9yItm@H;h&Nb` zNu=+$gBB~vOq^XC!a)Uz^i$!WlYK8t%!Yc96^XYH^)go{Iw5J`mK})>NLp}hPa=#M zq|0rCJu`D};xsnU(fCY8bQ!jmOy>iMGoS!-<6z=tBxIEyOQe$lU`_t1#1h2s&YVr8 z9TeCt@IoR7fi;&CCm}_Cyq4&KbQiaZ#99K#J)ko2Cl4wZ`89D0(iJ~gBq29k6Qx}e zk|ObWN$!Xj^tMm3vWJQ&+>_2Awdv@cM9)M6lYDQLbWaB1SG|**YeLvrm1K+bm-ewq z`~9Kgan>hIha83Lljs^URAh5f5dv>zl5QbhQm{KI5@`g{iKJruSR!D$7_?Wv81)Hc1e*PtHEn*ePCSR!c@s11f3If2?$i21q>(sqLh zvT1r_y6rmXuvTFHrc0At(B*M%M3k&`sOIHWqQ9Z*LQ2oa)`FV-TDGH;DHDs+Z%U)L zl+ll_9o9BBFGuz(w3|d>RWtMP5!&Z$*yWx5bv@anRIObOf0Cln(wGOB6tP+-3&=Dh zL3;%$K%cCoyKG>h&|4iOST{1Rj)KD@$7+DWrU^)|E~4GqfvZRewS$bAC~F&e86Il_$3_07mL627Dov;#+cO(VwR~7Ene$HTz}z{krRjIo-~C;UqsM8N zsrrmi|E5ZDz|R^1rA_L4*Ht+-5hlTfnnl z^|QRGtwfB=F4x|JCSl&+)pm!lNf##ap_ZNn2i7q97SsNTb~{oEeMSR7)t@l5$CN=B z)cEdOZ9e=lZ8du33@`-sJ9;$~fV+Ir(kvXn!QZumq0P7N)%}HVthCV4{Z7E~%k*!! zIz-9kHahx>5~%pEpSXEu3G#$Gq=T##>drvJ{OycsGFlSQ;#Yrb;-Fg(pFMEUoiavH&%g4$9w6@kQNDOrK9KjfixZBI5lXS2 z?gvs!I?%pGq+NRGUSiU{b@ba}z$UW3x}y*__GWwLXP}OrpsD(k|LHj`2I}ZFW4t1vf3%ZA3IlTB$pR*u;Oeu@?Z6Ud1|^5CNFA zK}V0}0C>+P9Sx@dPS~O=LM&KshtXw$sXpw~(W~MB$&NibF+y^6Kj!z$`h&Vf2+7bR zx<74Tk9gJWx6jMsLA@jbPM6E!PCS_ zOygU+8XTzazh6MGCmSaiJW0un6kD^q`u?P%Z;C@T8~U>+{-QARlD}y#^YXf`6)AYE zqhDG)Cx+WG|5lzB^$VE|`-{|H; z;h}GJ?P2L8|C??ZoR;_dt~-aUcDQp>`oqg;Oos5ZO*PrO0TF7ltjNzADag`FZTpnD zP_upZDNB%E9aksidQB)js7=Z_^IOr$v008}l|W3lwfm;DCmTDaI6{gm9aHG{Ws`uw^9L#JjHYCbzNEy%ALf5aaYu}L<41}c0!u9PEvs?takQB2YsWVtB2`K= z;>FSPNEt_88 z4M|B>mM@u|k;NyXW_lN>s$VmGA>u`Ey!7;74RFPwZS?fEb^uTC(SL;%Bh$L-*FoWK z{`ysjB|rDp7eV1o0s6X#9yj~z=}ogh57`hs{m2M_Sz&s5C0A8jkvkFk)lhgrq`n_g z#rM(rH;6BOOwiNMID!%lDf)86Aj*;Y4N%y5lzuHj{nt4CHbjvn8G7svu%}b?%OLfF zY5GU-bco5D*3Z*VfHvtcpY{n*9Z{ZMWxmHPng5Toxh&EL5Yb}P&lc-r5JNEq`nCxC zW0jtsT&UV&CcW!SrO8G`iM}IoS*M3Rg|uIXIo-Q;`rb%6{Wcqo3F^Ceho0VDRke6h zxkrx-S&R4TMZJ^9+tum^G+3pCLC z;Q)1w!G_id92#P%jZjZk80N#wxoL#q6p}<*#T(L)gn0f4!%C<}?~w+2ju-UPITH-< z>$J3e>1Wr{3=Y+N%!N*5`T;tVW6u7<90h9!m`eklpbb1qMCh zCuF?=Il9whgW)WqQ^U=MmIxfP&2R}Ru@7UQ@3sIl4Bu^_^8^6bE;Xbh@cN^MamY4! zyNiY>WSf5NHN#ZI^7MeoY(&936^3l2$gvL%-(kLGavqmVxhwIznAPbwPS&~7poiXa zu+p#wR($kJ!!tzZ)OQAYvl%ecp-%?7(+c4CUk$5}8W#UB3`a;jEmBoT7$~$&od>CV z*rk3z2Fbp{R3u2f7N&+G!eP=>cqWMXB28r>$BpfrQ~R5-e@QI=@1E{gQ*TKMu4mbr zZ;3qD)CI6QYg|*UkZw;uAnBjiCe;tNT2Px*`2m!s8ssuFNj#9HK}>9MCa(#hsYRQBOw(zoVqh1 zmF^J$t*2{KqY(I;K9$}%0HkM)Or@V=1@Oypso1fr^z>A^2n{MTJu?+K`O#u>Dm^O! zsOwHmJ%qq7vr`)*`nkdtPb|VyLkTQ?(*_m2Lj4FAG!Y-*Eajg8q%9f1~K% zXy(hpF{?d6ZJ5m`#?T9U!RqYDGh-Z(70G3n#yCMuOi}99M%qhs_s7&Q-#1KhzYaQF z8zUp4>toElZ@2T>n2Au=p4Z1np?8^BX-?pl_cp6;HS3?7Iu9@@`2QHY>VPVfuf1Hl zE}epaT_`pNh#jcdA}C_6Eq0LiUW~P4avxa#VfxURZ(t!oFiVti-L%YEl9 z_xJH1_dN5=nKNf*&g5HC`x|h=m)@`d6~Djs3DgxJrZx<^^_`FM0xkeBRcb-@?eTX1 zdlA6m4GyZ8#{Gz^H$9Rg;M?eAYlxYU><&xcOmc_B)jEx8xOaQ}MYSQ|#6RO#sjD*e z{`kupi{ZUrjIU&+Mz451K3Rjh;`{OI)Zhp2$IrG_|2+B2___}2pZ~H=w$dofLw)i) zJ2m>bPx8R;xEg+=trW~EmfR3t1SD&8tNOs?&1%Bafyo#St32JUc=9xj0&gmnyju+( zRXW*QgJV&JWHI40TuKW>h&mI%SUQUd=Yf~Tg4 zJpb7gQT{)hB4YkmQ-n57lk0!KPlCWP$r`)kSH>hSaaNmb(uCyA-(O;h&`S-= zl$|HngD-CG==Cz$$CpwPrs}GlH@oqHTQ+>$+!Ge z^#{M5NnWPr7;!eafvcM1*v;f;8l%E*catY+_<5H{$r!Q|4=ASNJ8wCV#KfXiz6*y}Asa>!b|PP--lqkC~Fd)44g@hMLKYX}|3r%V*8eVxKT>d~T}8d?bPAq)QI?Us zAMcr$f{e1MF8|RJ(wBJ|W^T0O_X0EX^>cv?qFcC?R%KZDDVE!Rr8HE@NEtja+f-YE z!C_dIF7>~*sObPKYWW{G3LCPL{>KMnZ<_I4!WIxpNm2LEeDL-f*Io0{WW+0>639CnpbUurZ^yK1TH)f~T7OFg8)F}qgkJB^+YCQtN6|&Agb@_Ks@whq%-m8tD5yQ=K*Jr0bu$T+Okgf9eVijvt4mYD^Jf z-l$X!UyL4|iXW~~RpW&5sc$s;@SoCBPiXwy!(#PdC_Aq4 zv&D?mms&s1NzJ$X?nj}&cElFbu`UL))E?WsOSgrW@6ugiUZ=_K-)S-XvQkqtnnu%K zQt^VJvLw5@XQ!Ui_&IV_>T`{s>#k2-sZoh9bF|!M^VZaK4Mie$rQ+s`N|9;@QXgx0 zWX0dKsGt6pYN4Utm*2H&YuAgDsW&xDf8%T_ZY8M{nf_PmV)aNTyql6+EX@Tr^_=YR zoe#`!`8!qpGRdx5S5haci)J3K!eG|T)c^5;`PSM)8jV+|(|?XvDF1(rS5f^Z*Z7YS zJmGfgJhibb@1$BetAF-+o*J!T@!|QJKc9J%8mfVI_>kI8S*SkRa#e#sZ)$c8r}<4204&gEKTD;a+%_3XEXvO>(XiXOq?oyx==O^-zZo8 z92uJSyT^B%f5je3dMrrjma(&S>Emq^_ z&e3UpRaC_%Kc#&F*M(^{zQ}$V>)7EQf(&9oI zuGFhYK3q+EtD(u4+i7?>L4|JeAWduXx#(%yL5+UD|LZi3#jw(E(r~*&MLqFTTK(_W zQ~E@#r+445o|GN(uhY|Uhg|vVu6*nCOpTu>IHtGO`1zh|dRvX38+oVOY5d%^Ncu{3 zhYha5=_}RcIvSiFsu2cRR7m&I__5ZT87tJ(Ib}0`(&#svSI#)2@pGwK z86!2O!|S6mR;i2Fy?#cv2H2%}##*%=$D3!EG#Xx9y9_)XrSgS>E*WptL-@{wjE)+v z(5H8Xg9df_pbTtED(YS%GJ-UQ>p3YIFPzlq**|6AgMt40|^FK4uNg#L#OWpqFM0Qs*nNtH27%Kc{WkxBRXnEZ8{Yy{~?A5&48^ix=-zwV5aAU)(9Z|`9(91|Wj*6{}m zD`_?{9T2)GK!DEdkm)5Os$B~|==-hi1Sj3tKmjtSR!>J5wA>_DcGBGpz1|lO9slqH z7U74LhAR~X_@28Nj*xq3vIwvIHX{hmJju}O+`<{83|K$SaFJyi`ZU8|H>^(FX1&Jh>)ML6XP?UUhU~H2d&!QQ z^k}KoZ3M`WHyI_&1XQMqZAF2X+CoSWs@tc4_?ig{_0+?Qw;4{lmhI$B_H%Rmu#L(N{qM)>CiI2IRn100_q!^igJ z`01ZBN~-X1`7`=KT7M3!{3SzdtRgK8`hu`|gUm2*&CclXm&DNN7^&R7@%%u;F(*eU3B3(}dh zR-T*F+!`X?GUa{C&1#u?oo$90VGoCth_n=JQ{zO0E9CkrIHp=+F0Vv{AGn-U^kDmj zOw{W}8qB!_!8j{}*n?41`BW$j?v5AJta);36*aAKP?~5Ik_UG$1Cj0qH#lHzkTp+u z&R{iXnPJ5t!Nw3`wg(Z-w?V`|S!P5daH_}kAViEUB9^Z(BPz~y8srH#Z51dhJ47|! zV@4I9duXc{{A9=Vq5h`nVFBPer393>SCpx}JyQH7ARyb`&`74}7bcB3ZBAc$ZuZY^ za41%x###qN%|2&Fm4JAEgTvecaDQ- zI3`s8F{AtuzRmv1g)JBEyp>)rXqZ3Am#2ALB6Ij5?r-Zvss zZ}BJOhZ;)3@uJ*FcZ%XbR;@ovRT8QP8N>*UigC4ug&s133{JWmLj;I(P3(lEvlL?A z+ux|yxeQ~F@?IS~XiYQR4j4QkJ{XHJT)=@Urq2t;5(JJW7Vrt-qcP=C*B8e<=VT_W z1oJ|8=Rwl$aX99UBq@ILT@ta+K8gjrO#;y(!x~^j-6;j~iyNAPe{ls>>*7dtk)$!O zNg^eabZKlMpW}2*WjveB)+W&I5?GSqX-v=yZa1;;f|ezDd+YHQYji|OG(}qs1&SL+ zgHNAn0=m0O@>g-@~cPDte8D{S;X(&elvgsG_2+ zt{IH&YD-0ISL;{FQ`kU}Ar@8Dp+!n1)XQZfYz;TWfirF!fT>ZY7;e!_Wbqb(%H@Kk zJ_f?RuyD%JtRUx z;8`2qFVn2@!*7B#v8ut(?9-HX>QpRrc2z9&j3Xk%0Ybl6dO%P$9@Y@JO*PD@%y9vi zTg_0-Oogo-Bkz+;&;`bg;iYW5d!i?For_xUu8w&moDv{GHF#G_%-w4sv(-64+7B$s z!?zmT1_be0sHj)}KN)}Y)iTB3uC1UGc5!3SJ!c5_sd4va+7OFj~Cz1tS0)wnA45LZ7~#FbG)UNXqS zvH~PA%23-}4a25ku(%@%wW?pOkl?dLSs4LEStIHrq-KVTyp?cWsq>~Q} z2O0_Rq}EpcklTQp7JGP4c+~(IjWGfwxRJpTmZW1qklv8vh&HDos(!Agh^ubSvj&IfC;u-Uw-7 zYspUnZe}~34fw9)D|Dnyx#ma}HbsOKmklQbxyK;4<_M6bU2Td&*LciGHtb9>hU$J= z06n4&-61fVn~w+@M)sgF}=cs?IB?6gy03|nKch1Dw*jAvS-Hjku`m~>sRtem9vlo*EFC!vbT z)M?nQMH|E&JuKpiK;#tE>UJAmLBwa%mhl}GakdaMU5oD|C`x?hoIH&$M9+`9d$s5I}RIz8O(gj)H17I9uM{U};GCzKn43fS2ShV{d!H>``H0(|SNpeh}U z3O(kKI?(+Lnn|@fai8IEEy7GCE?iNef9ZnK%^so@*Ua}ATp+HvNsKahpq=5*aMfgM z4HRfUEfiGjitYNT4PS|%JX*zL9zBZ)X$i97p3V(+$Mf=1zjZ4f`^GPUBCe9T0!RZ2 zd~-@MX)X0)1_iz$rI{4(I7t>VDDcf9!S@1R_cAP*M&Ju1&`;-aIOMF5Ogz~{u~HADY7!|yfw6A)~(6Jwu!+R8y&M|96 z1q|qqf)2?{FakF8H;Ak2_1OylgZ`-6hjb<#&n5>ppzFbB)>+3@_d>T*I8|vi7JbF3sQ^iGu?m12V>vU6 z*(_VMpxoQT5I-wBNFJvcW3IVd$qjc?g7=29;|)d3O66b2R+}&$E5oHN=L`YYzPwAr zs58{^Mf*Z=>j0~wGBGLrM>10U(TQU`AUcI-vvtJMRVyh4b@WI~o(@r5iVzR*n4dYLHIjTWRhtqhRm zL68Clc|1{o40@Mo2dNVjg+Dj}Ah+K57!C zZN5j4=1ehI!=Xuv1m^*RqKPNA!LFc zmd!@Lrj-Gl_6C6c6om$1(-AkYNFi>zp^7;p8c#o#V}eB4oRBn_IRlf;E)R$28v20i zOoOO6NjuI&X?i7=wg=2?Z1BqZU~Q}JM@L)O$USphEtaGw{A+=ey7alcMu|1fMgi{@ zDZo$K*x(Ic$#_oi3A0g&%$5vdFc*phb>^U;RSXmS2#0)ZoYlJm|N7hD{?#0e1s=6y zI4Xq$bT49KtJZ7PT;zMxiP>V3Iiq18u>0!yGuGd+I7W~NVa08{)oK0aVcPZy0_0#R zn_yWvC~W_^7n62?^e`LS{~932XCYPT-W&|mE7%kPCs3HN3Lq}HpNNyU9kS4K25C8n zAT(_Ep3hqdP0lZlWx|fIqNzdLq^G2P>!amj7NF(&j}stITjDP`Lzi!rykCG>bTJ5! zJy0^6{CJQG?-NOtL z^|JuM*?mbhBxNx|tg|H<4d`bRAj4sZ8LGRmLWKB2dO!3ORYd$~2~suxRm{x?l9uvS zj~;9pVcTzlG`%NI;Z$o0EwdTH5fS1KCqD89P3s2tZVOWFb%Sy16m2Nwjs*U2otBLT zFUL&F+!t}R%uY`3dHNejVdhM! z+M2CG+6Pw9c0E=;<$P$Bq7SlKjgWQ@5(us<_(I;FS{0Maz}o^vuo%CcKZ4QoxHZ_0 zSBHpFTX?aC4=Ut{H*tfK>2mIajSZ|>tB8Bftws8zFac6;9p9y)!Wh?~bS-W{A_28F zT$snz%=Nt8Qj?n(!4Lywb|__IEAK(eMne92!xR~4ilYKFy&+@j3hA2+cJSHJ)7 zw=33#%Iq-sTRee~eb{}P7k-L-X4 zpOszckGuBwIK&k`oZ!v5$C!zsx^A8VGZN_G+C5u`@3{E+tGH`_M*;SMxG~&H!~Vk6 zv?Bi~+FFvAJOeLC|HsxJ7N_xJEqv3x05_aiIjK0>r&?(NvRM&^``*s`9+GP8yjKih z$-cJ$xt^pN3y_uY-1P(OWnEu!>(h$BzCuBp5%$)2ke5rzB?g^BmuOj3NK25j$L(F< z{K#9E5R9Wa*1tgn0xidc@HMwa`9(%bzVibLEPPC$^UXArWewm^_$V|0Ov zzK)LIbyk6Eau#t9>j}7*2@ZOA5@B?Nb7vJRBKmXK)#7oLzp$&#c7 z>N#xpf3_=Rn&1!wUqb9OHuM+du^i(C98tNJw=1gN_paDqfRL4v+>Lhd?|Fqa^?xGO zaRts7UWQ}7Ln?ExrlBBwxWJb;&e0w!dKg_~J;vY*NOeXE>6Yo}1qBzly{KYn-E#M6 zfy(s~?*DH7OQF@JzmTfa7!gte23+J7NbAeXE~2d-CX0{;GF=8r8>jhly75P#})>R~Fv_%jmT{T3?igKJukL?z-C<14% zb6b!WldmCU+HV2`6?Fv5>wI~XhLou5Xs*OlB81A~Eu+Fku25Yo+?ry>Fqg#%X5Qdb zl-sErm|Jl?*eK#6x7n4Dc?;owzKOW1`2wz9uALX0nJ`g2Dn_Qa{vb%h>O0v%TsH6Y zh~UO8BmfSvfhCo(VI|z=!zG1l8@~`s)*T+L(HVs&r_u6}cMMJzQSkBswz}*)y!t3Q ziP6E~hlWn@^1Pij^uNo!Fuy)tM4odOZFZMKf@Mc}sYUFtVFca4cFSaWV|aWIb#HDf zNOK?KW?Hb`!4<~;!yAL4E5-o(0vO~+sCb`ePKEDu9|__DnKT)m-REn4BrX3hN(Yw` zq}gxnJYjkdzWhvsN`Nw2!&Mhr3J;)g9{scb*uAiv-1RF?jG(=G}2j&Sbd}w{o4&&qU{LP;B_E~Qf-dm@Q!sN3J91R=V*yHCqB}>D!n)~Jr z+U^~mvnXWsgp{|uMbBP~w?j6)MaY@CBBU4`Ji@yauG{>D0;Jvt+&VZpnV-5Ne{gw^ z<*I_6Hp_@0)88v1`I8K?AzOrqQzBeS3)MARDGKZ%VT64E6jg3FHu{JJEiN!zJmej> z$A01?U+Aa(g!QZKE&GRp)PI+~56t?^8xs-i{EP(QPguTjP~i(-3!tGZ=?fAZzz(>u5O@~t zH7DTa?h}chA2IfXrKc5cu+G^*-txW`A{{C@`#_l9!BK6wY2Q$#?R^Bs8Lu&%!r;aO zW;q*W2h-=p8vS)1VjZ zz%CB>0E(5u&kqd(8*RX`JN8hw-*fN7?bA?Q!CZmLtCvF&cxI_!@vuUc zzZObzYkdZfg|4>ZMWeFgC9()<~7OT7Zln<>01X&5g1_h;f4e zIcQ@PC%a^qPfXe_he>GGvhr~Rb zZo5o$u$6fTtyh(^$HG)NB7n`2F|NVd^FkBpGkYXm|4P77y?^qw!}BDEkyKi%7%7M9 zrr5xFcO&kR6^7eh5dV#jti(6g5&7l>2{;PhH$B%fQ$;7ljVvMHFqkf2%@r^D< zaj7G}3XZr>T+pTw-2@2EUVI?20!Dq()=7@E!WD6=MhZBZ1^Bob#eG^@q5UvMBGsNF z6u6;6v&YJDVvk61Q1zuuI>_uL)HP6?f06{cF2c_MMYY?hP|fe`r!4-KAe zsc?<64EOH}3C?$fqciN;sc7rqg}82OM4Wi8knEq}jnbp*1?iBZj;`=%oTC$5yC`YH zY>~*%2XS*Z3AnW>j&6{5kI$;<1j9)mwClKS0%V2(A%lNcu#EFXmS(>S5S$1FfZk8h z9aQr}$mqQSBq<0(+ueQ&pJ_48QJ>{+bkb!X5MVfs5nX{wRb;KByo()nMM4-_#3=4` zv@L=SC;gU0x^y#M%T=8e4h%rGO5PK2wYNEXfOJ=tR$KD|u>7y?3lJ3BkdlM)AZY5xi%RxK{!Mtt0N6QY5fF2sP;PR*+(4 z2EoGKitZ~Z7zt*55Fn3&xglukaq0^b;L3n<)^{)jlXd$lNNF-vtC&$dFGi{jD29+$ zmT*2C2?%n(7^f#lKyie8#$Jj+aKh&Ti;63}`r0)`%~f`mFgCl6eM4F__bMuON(n4> zac>c)hlu;w@DJ`#l)YR@%pNyVxrVDs@if3ToHJgq?okTDUFb+B}IrM>@02MXN*__%V&@V*qt)l;6&2~ek{ZNgTkI!wFRkb z1#A_gUMWuON0&u{(sd+|NC&54S)mGT_2m%uw!Q?GRN6@o{%%hGD(q#3Ez(qiMM^y+ zmN$x*mgp|*vhr8~A6tsJn&vK#YCW+6vRr5)JLuMBL>qa z2r&OpCl9#&BR4WR@0@{xG^~!34cJvuEX0IWLV^;51W0xz-f~FCyG(jwh#>td!pR*r zwdaN-`Vzwn={2Y2Whx`VP7a}Id62oJG$XH%g9=uzbVg2jkFJ7AFN#7tcwR*jmTYK= zRu8X=J)T8cVFB(K0%2EGMdSIWDl*v_L`Vp9e9iYBX>#9jvLJna-p*D&x%bA&eS_*) zkq@Pig3eBX;Oxn}dQ!QdI%ecLO+r;X-pL!fmgBcG3Aech;_A&5a7Akx#ZVl_!cg6* zIf7tlxX~Ie*Hl#4x0;x?>s%2cpB?`ij?#kpOe)TLX@=je78c4yG4+%~1ORT$h z7{uH51Wt%T@c#1>@a-t0yx#h)P?Y!JUlRDshPa@-Z@0u}>*MhJ%Vh}!wX}u74HP5G z$_AM8zN-=(QJvhKy16 zSV!x1&8-A!-V-|~2)nNsb~9Tb!9xzg8RA!5n=2Bn7y&y)Bd)W9NaX-mqK)C^E|hdB z*AgMy^%4kDd4DHBE@0p_c#@7Oj;c|<_PF2URi8cFMJF{ntb;&PmP$^64xbf%~_*u%FjE#Nk^F^UH&?zF** z{8~;B3~pYTtIHsOkwfF&WOY7$5i!KI`cS+ zSp3;4$-R++^g}~uSJ^~`)>B=^GHEYp)kFkXx+o^Rms&VuHO65zR!D*K z@y0%oTGk#H5u%-Kv$`of?nDP?dCy-I&r`-5r^@QKT!I3WfrDwhae@pqv@7Z~W{#jS z>SPNGdN@0(GAz{%VW+YftOH!`gJXxjCZ8+PJ;hH81Zh-)(a)R(oxHEp-`N>560l&i z7K#wO_>rKPc$Mppd5l{kfy^B2jAwq84+}hGxX9%aT!&%KzG_@{BH}8n7I3aTjN;^# z0*K1%U|Any7g;P!w?jYoKv(*$P@2?J5$tX2iKHtw3296BQb_yvLh0VEf)vy0;b|{E zOCV1z+8gO7?hqg!=3<>^_vR4tu_6=k*IT{OV8eEa8R2_26P+DpC!h&Dp$}#my_ey{ zd!w}3a&oG(d^!?zfPs3;4oK52^Cx{#C4W4b#_ZAkM@6+P{t+{Ha8!_@!U6Jus~>hR zzy2Ynbq43yBs)l7FwjPhWAhY;{z$dxj8G%v`y0jM;1qb!!G%kIGHFQ&-s5Z|i!rIN zfdi1B)&&V9V7aqB3?86pA9ERQC9eL8R9EpAvsY4X^isgeYa+w}Vh8f}N)XzzaJ(); zTp&*Zp>qvmBvSfL!A!}k0{pzdGYJefjG8FMHIxtSUDUoU=HvODoBA6##8-V?KlqMF z5cHip`@e8V8TG~e5Im?{Xs=4patb#Fnf}mFH0dS*l9gnX-wPX*gu-3_O446X;$sPQ z4-FrS(mF71siMO64nu+~98xRmAl7TdaNaVgZ4Modp8EK?$i!g&FryKl&CzO7{BH`^ z&v`8-sRa9f!!)Wlg#I3Z8tnWa!8IGn2N{}#z5gOeb4MDRn0<+s^@B&DT2sFY(!^25 z1hbSZcX^c2-=YcJNi%kYS)cTF5Hp(FhKPq)Lw=^Q66Ef2E(t?Ry4b&i@9EpHfnq!;Zx>FCt&= zIK|ff^KnRp+XWKHbr%~L@{fx*j32Macf)uDk0~Mn&pL&N2`VHjaNUv-7hOz(%Vnh9 z@|{CqXgCiQNxz6v65Ny&g?{7SDh9)IQuwvE&Oz!2qXws9_UpMzl2IiYZ@) z@_;(H$BZAQ#UWyLEeUw(J69aQGZbb#nSrEN8wxlqM=7Xd;A?}l%{t0}9^AO82&n*1 z43cn@$<1Hwor0nbk!nSQB;Ox)5c5vbys=FtX ze#MF0*B~sQs*#q?#FAX^B%%6vR*y##8uRB4c+c(JmEjti+amSXnL4;T^jKol*r~otg758ACnU0D8avhZNYFvnsne?9&>k!WE*R9r zI(wn)@_nYFwQYKe5I@MC%E#=sm+_pyz-kJo$n7J-9U*3#QC!9%6HS_ij4S(zkYI55 z$}K~Xi0KH49V$Y+U|gi$1+E72KAv#z8E)n9LR@`)1z7q*0ogMHE2KJ{&%%xuOXvr2 zP4w=vWo;s;5y73&ut zyXoaodkh}6EHppC?IAS*-G1{_g-ufCAa}i=C1C5he7Z&1*O|-A0fmsh`Vcrhm$T3) zb8?9wy)_cYpaDNCdLFwWm~-~e*rOyYmEaua@%35?D+dk31{XUIafg?SI2X7+Oz$bX zFi2WiW}#LWR*ASkn3ct~BFIh#xw2M-xIlGKlx-j&;1Flb0#X^w#@?K zV5;5$vL#Z==lA(Y(qxMWv4hA3iYnZP57WvX8FSkvzyl3>Z}_r+=SPgIGZpa9zY1{I zKCU*PpMYLl?l*-?o?VDqjNdQdA{TK5Xy&;65EJx*cZ(F2Q}h>1);d>^F8+o0?-cHw z`^D&I5dm9Qqn{s|f!VfNteCaVSd8gcofOjNF2Gh0v_w%F_(f5gXJ;7PWbU0Ph})gV ze(L~9%dq<#vs9tw(xptx^Ad<(wqnV0>KJ}^1bK3Ki+FZaqqUoCRt2 zBkZAOY{ZD7%m%LO>`=U{6So09;+~5H_x`3nNWI$Qy%G7gxl3@!CqF)Lf;{6g8b$n?-Fl#d}D%}j+^A7vSvzxJx)Rz$;VOta}a@ZEk zJhOZu!6%IIqPK9DC6koRLCD>TBE$~febIZVan~8{UR4PWCm4>3WEr^2dl7{nn#fQ;D12YPycVrqRCxwGFME+B&)M_Ok!aB+jW%6p7GcOzlX zF#;^LiHn2!Y*E=57d-^+L4Qck5OA~ha9<+BOzEI7|Lq9^vY>;DJ+$7-OVML52FwMM z1*vx|G7s|SQz0TNy$@9?HC=$L-lteA|Ii)Nyx51@w4NUu5eWX zj$DVGf$II!bsH5`{_Tb&P1+y4C!^)eeShFkUjYv18RH@BBp;ROpjXmKEQtG4N&5RI z740MV6sk4vl>oto3HkA$vrIbhql6&!go_WvpXL=omNlG4T#e5XT(eUye&BjWfva%_ zaYudt!$ang%;z?qQM3i$KUsr4gKjj$Qi99)6OA?Ktb%XOS>#*iAi?dsg58!X-+Jc| zw*v>~f)lY&8IImb3PdJZ|Hvt66dWvg8(qk^v zC~ZQ7^s1GcwSQvL`GrzksPTjf7Ze>$mkS8lg+nbXi+LqNT<)dnu>1=l1vr>8i0xMw zoQeFUD8^0(*@yS%8RWQyt1~!!RzSL3L`ZT|0rJO1MH{uagwiTFxG{oAKUW|1;fKu( za-y98!KEVk^teSiSNRy69WNj^mUAs8n}ASE_zR9=4~?-f^QzJMJJ;c;>|6fEOe=Ti zU?_7%;njVwAf%v&1marR)f)!6DdOWIR}puhw}_KRRDUq(wtk}28jja=^^(;!Qz+m_ zTK8vqfW2bP3$YmkMFbZH0oYZ>A{Fz#<)9ym)Bv(7A zJ(~xKM9}>oBv`^BHn4UQ)`IF7x6^&Zz27HN*?{rB!a;Kxq%V%5OqZ~Kxv42!{PhSE zbb^?wuVP;MWRAkt&2gwLga^P0Yen?5_W@@01czFd)r%~&vGlkmwY=pxs6OPg zv5-D^qx9+mS9$1I^ArPhgY-vyC6#X49e9L(RRsrKCat&F)l=r=bkxe>l92t&mpBRf zqdmU`Kwi68Dvet0Y7JAKD9U_!71nX?6RhLvmnBpPb_J(=g=cktiY0z=MUJzEZ);uc z)#h<{hTy#GO7QIsm|=VdkATPn_A>B{TN+?%sPmjpn+~5w3l4gYcG-MS2|l=ubptCD zwN&&4GM{}c0bkjJmXsdc+*%e%^Ijn7)@Ksj{{4tky%&)268SP;ia7CfES2N(5iDI) zJ~QWA0fOzp2{z|*N2GdN^FfeCp1_6p_*aTbF}=b(YGdrcJPRQ~F!hb1D#BkQBtZSet|wgU!JgOLeMpz+KNar0&QbzD_$QW5MZ5J4(r(6&4iwT>hdmcvOUYkcd5ay| zNPJO4%=xD)SZKZEJu@oeJQo3SdnG>GzwjMj$tUGb-c%@8##I6jz2)k!o*ot4QKaeO zA*6YE$ygPhzvt01E|G`otns4=OvhQN`id~`1CIr0i?G&x9Jup7U`=I7aSQLe7J*hD zxwWa^$ncX;-TTP>kq$2GdZ=j8p8gWJ?<3c8u;~+Lq%os*z9J12lcZUc?^;}zhK>YG zFF+SvR9b?=eH^_k`k>CXS3LpZ`o&lcTACEjb@nq>@mmh*D4P+beL-n4d>MpUF87PE zvrHQH6{TC7iuAQ0*V_$G(^~LpIo0lUhU=hMTSCcZUQM=unMk3cNM6wKT~C&H;54YMdA( z$>Ve#H+&jRa-_X54r_`(K4zeQ-^aZOj>7rCGJKaEAbJ<)fUN|H5=<8uY_z?t_c_k;4{Hrq5 zZq{&V8Q9>$OOHz_o-ot_eXy26B&Z18R=c^vCkOtfEmeGZN3`&U$#R@MK)+>3rsi;23jtkimag!~`{EFXK{S$V@{?T37aJEm`Ou38G{V){=4Je_?!0)u%ql zI9Ey9Mt0zX?9y{FOV;148kI6@kiu=>5GW454sydwQLem0pe}NfD;lr&3dR!$Lx#JB zfJ-|bViRLSH?&*W8Ua#!v|CyAS!X+Ugw)$8K+@bz^07CUNsH|eq&Yj?@BqMSH+!gK z!djBvd8OQvJrHbtNCD3BR5S>B%VNn90T$zBviq<3)73c&n}i<~6N-n*DEGx)n7h*n zMed*<>}D&o9f4!K5!~vs9PA@|(1o^l?0wM2s~?CsFWAtVk2>TxbePHV3FHT2&mnzQ zjN;9ezDO{jQ2N|g@sY$pKUAa73n6Vk{KxDMcShm9eUUdzP!%5g@n&;483hwb5U%bo z=rNafh0?EoxH)AVoL_}v@ThXH9X{4m~@4Kzi_`U~DK z)%eJy^=19VvHvGc2o&CJ(c9 zFAI6mc-!4M#3c8#_zz;W$a_%X&z1hbVkYva;!W?vP7T4R>f> zk0t90Gt2PGC7-xKf*$|XYjiD(#Tp&O2zmh3%+;a%a>AVx_*TV*&VtmbuDb&)DaZ35 zyM%|j%LAFwu_D|Rc7?ioK!v4>^_gkqG1K*3MVtfNuk2nzwo6Cc-6|mNT3-!&d2E<(yrO*gKB+^6s$MHQ3y z&K|BD<7J4&0&VngcRUDFl@Eku(BG?Kvii#eNb_nY`R&Xp)llk>=OC(>20^Fs?s^EW zt{`Yq9U&fT1%ei-?mRc~o`qC3exlyTPrNsW+Sn=Z2(RvUlNFS^G&kOV&y3t%an2 zA7{x%LEvI{7g)NAuL+S`(a0NgmgUTPs)m*Geux>ddKs(>39iQSLWLdE+QpSCC*C;XZS z7cZI6t%n1VXqkEs1!=4IIJ8C9Gl|EX=}ju9WA1nyz8)5$IlisQtdRZ?ojm0j7DDyP zZCDf*LjOj>C+$SYD7P5o?~f9Q{v4V~`ZyjPE_q@P!=gfjp#Pj*&<(;6z={!l~2^>J0=kW}70(iY=#8c;S$IO zXAeDWY|b?$++!CHdEZ~AjwFq1bCZ1IV4%AKzAi!nU)r4ONbH|HG0)r0(FMct`wfL< zkymlo7O3cfhD=%+4*GgHYtD$tZU=wF@5Cj7< z4Y=9W5Y`qVyYO{8%4kRzl=l9WN$bF|E{bx!VA44J zun0?wek$**#>TO<_+b$y#WpB!?9`=DsNWDjEW#k?;<2$W9ItTnw0LCM%^~7}ZTgtF zV^?f;UH=dWurbMJ9MijE&D4X#J+T*$TJC|n(OtO{Q~3&3DjH_3Jb^2675eR2mB7&U@Ivu2MW~CTGG=TkCbjEx<4W8{)TKynu8n zu1s$%!t6rnxZWo50yCwh#m|~LA#Lt0tQV&~5(0MStXdybBZouWq2MmIGwH*zr0!{k zOThDwObw*ME2@2YZUpJu7a{m?qZ#4>r|=hZXGYr_-}<7ud+~S&V`=lFNqowj)TN1b zKRndIq)Gj7fOz+#LfxYMFps+&B0o}#N;<*&{uunva1~iRA+W!qD=Ckcc_D$}B0NN# z2b}E7gI%+_jso8!JV#Qf6@J`m053VW^L$?hsSO$5JZxpFSnM+{IyXFEB1#kUJ$xX2 zprUMR24dNkDR9N1^dOU$*@$H4j)M>vgQroXeC6TBAccon4o2MPqEg&olNj(5)m8>s zkEc^aD*X_?eo1Ac&*j!BAwoQ0&k%*;cZMKKt`u_PBl=M6P=!KiLlKuyTBIrhFNP}0 z7MO&P(d9%)aaplw2X98GD8*&Te01ud5hj=aTFZzZhNK~QdPXdzIhik98^$%D1$Bqv z=%Sy)G{Ipuo_6YDEwJ;{L;qpewOz(ztg1Z3lM@tQM#iJB+peqb_K2nKmHCloD zX*A-hHBjQdC~@)rp7Mx)L1Sf_O=A??kH#SP*JdKF82ESN-wvn!$Qfg?*n^`*h#O#W zd}URD=i`H1>&cGcl#XEuYCvcSPyBpJvSLO5z&OnINLvBoGMpP2lzOY{cq3XJujJoWL6Cti}QPSFZ z-*U2>0LiO@C0e(Z4-v$IouU)eOvCQ1A)cvXB}beie5;N%AD_zmGy=~|#VnsE%5i?` z4j=nS?2UTKargp48t0=y^k^EEvSnWZSN%hVxHL=JKmCletBn3|*k~#XGt;?OleGIr z)cFoc@uQCtDQQ|d12t$ofTgVfLo@ggOLL(CP0P4IMI-DWKSR+Xf(*#g zZoB|#9pi~t0S&yOD5GTza$%AHIoH(_$3P{d*+i`PN=D3W;#L87b(smLDqDDNbY^z% zG*9__@%=6aZ!PP<%a|~~68{K1NhXu{BpVrpb~{@>6QnOpJOk1-FcZ`6rMG;fAN(_hY#tlMwsBjq};Tz)6azeG$XN`baTdpEBhpapgn{ zTK-d%fR$nJWbSojfq9dWr*#mC>qFujY$cy2a|*%)|Ad%vNrjlSpLi|MX{k*=nfxq= zoG27$EcUX4(sLCQ;Zu-eu)&)ZzdodQ^0tLrpup@N<7@ur+zx#(5X>?(*^GtLm$d!u}i3{31! zKO?(WdnNd^u|+w^`I#G+W?vsxv~Vu`D5T(L8%OelR6tZK5cx2kA?7O1ZWv}>*_v|2e=HAM7vU71sW&g9raKH>q_FYic?h=lnb0Z^rz_M&S zHxji}(U$rx#d6*wX)QRgR58f-3V49D{(OBkdA;CL>+gi8eF$JaGim<9FC{Z|Rd9?N;(L{~N@r`L6c z?Bz&yY<(fwa>ZEavI4a+;C>a$iUO7HD-^@qOa>XWM}YL3QP&?ntmY0uThujh--r=( zU&(_o(zV}8g|4eP63=ES)-`4)$bMB z{8LPT-$ns^TBA@tcrE7lS91Y^CJ&Yc3UoZ=b89B83SZW8UCGct#xOxsS)*G}B0>*a z)?qDQCTTd_DiKi{0@f??T@!}5HtP|0Kc2A=P7eiP5#Eq;SpmMi9+gZSEhyPFI>Hsg zH*g4g}FaBAi?WG>5ERdo{=hv)r7iXQTEyCjG!#c->6tOKC%%B zs>~3ic^5ILt<@{SSzWB=n-JG}fq0CTTC`tqcJ25-ptEF4%K8c z!YtQGGIL0aaD;y66}XcO*Kw18bJ=1Nmq&@E#TJw%?-4T-J&E49JhufsDeRCSE&FqX zFO<)b(6g^vfBG3q*gXfUsAjH!3!4|=3G1aewm6oTgBi8|Lx9}=8`pY+wkjN-qV<_) z1c>7{T!dV*l{*uuyKi}fyd(L@;o@tu`;t-7`HjOwkVuj z3*)wNAxjW1YZ}(SB2&8nXuAhn|Lp$yYxoP4pjO{Od2cj#g-iPzbmXtq~|vApd4mOVO<5U`rOa zD0z=6k{^cevK{3SIDuFsZ#mx+7vjYF`G3aqt=2;QW1Nlr2jh7c-2hy&V`PaZOo9Iu z&i5QB1!{-$(epW`D1l8qWPB-1YmHvj{g#UISkDeg`hlN0rXZ!=KW4$O<0}AWk`ss0=rJ~=F5U|OUKKd zp~vjl3J`XRk2zbP;?<*G_=DwmiP!=5E#;#hoy~o~Fufmg3I`Z?pfi3aI$Pm1>8DYP zptn-sg4nu_aO1QBXn6)ZuMIZfb`1~vxURw#y~G2Z@lzJc0Dc<7y}|cz2^SIT4Q)ms z-?K9cK994AYvm>3i$BW?LB2YTLGJiUAk-yZKg$P0+?vF1bK!fpl-pJe1QO2itu11i zjX&salv1+XImZo7IJZ2+B~?)3dgMv8V&AZun}>aTx2lD>h**2LhQFA*Ribh}k5nVZ-?!omv?3(tPp&u}o6GnUSppN}5IZ>cCtrS~ zkBn>xkChLH4|^%oJPnU^g*0WrKJ5Y)C!k*;I3hL#o?TGZ=3fYDIa+{3MPsc?!+si# zXZ?k0?gUAem+=>w-Rv(SOOJJh5DeEK7JrdJrZUJs+X<4j-{b)|B?yYuX6=_`k$1_2 zdr_Bo;77BF0tG}oNk_G3bs4i5ctFnL#3mIQlLe|InhESXB1!T1vSK>m{Wp>pKPrI) zEr|8`{!wKA zS!p3fbxg5_q#evDNU(1RZbg*1!ICweo7>!8c6^=?FS_Hg`1Jw;!rFFmla-Y2u7=;F z^zibosfg@=9W{}dn`nnT2_$O=I=a3OKSauEKI<0R+y~zxMMVUnlv^foB0!qZsm`MK z9+C)YeKOV#qHgnQAs^~{8>#k7aZB!C;Qj2jVn8c)2XPH7p~pQ_A|OP}jaY6nsg1KB zjk~KzJM%6|U;7JE$9pFEaiYfeQ0i8JNyP_z$x`?cF#N3(9Jr6YrS!BSWx4PlO!h?z z3H9vaqJCdH`aa?sHWsPGsUDe$PUDu3hPJxQF7TpM7kk)$S#kJvQg9bNr(+!F z3FRLsN}2coGy1))NL5<)B{F&z=zT}H^U%~C7C+=pN^2LtKg4RE(TPbXnO%-bX?TRB z&*Ft-QFXfD<>kkUe5Y1N{bC-Yet#q~+(0;?gnW67EN*>8mLQp#=yYtzK*2(lo^txO zPjSrJ&jrr#B1+4|zR!?8CrKo526)C$pD>a}x` zgz?9>yfTE_`^(|{XHz%udZ)-L;vJ@~iJ!${(%b@*H=KUQmub$|$Au5a_gJRi@rzgt zVtTJ=%Lm>gfeycP#iW&t=&s7H3kH2a$TSRM3-fS=Kc*l~`pkvJMl@c9kC@S@Jpyj! zM?STuXKZew)S}9~{LIc-1-Mk0n5D2QnPSE#%qrz~fnwdFF8=ECa% zlU6EBf9#xqO!>_9pj@slJo?VUbs!ttestaWp{+>{#Fh?-FMH_8D4$kOXn=@vPuWtKMM=#Wi4Ys|N- zy%Kl8I#V27$k#tP3y_?Xv3O$r9(M4j+VjH?^kmcm8>E`4m*Z?<|2;IYw5?JV#Mq*K zhuoFmrwfWa-ZB$ySjg>*Y^3_6Q^&<+)7P(;Z&^reE6F0IA|3Ao#D zy3~Nj_PjbsJ$DCG?+N=1@Sv&kf2gSk)R|bd2ox9P3L={P4`J5w|wL99F^qNdBWB6L|nt>5~?kpneuh_ z#{|JRhuI#jTN3n*;%p$%D^r}=7`*UuZ4Q2Uxe!tcti2U1QQnx*c?@<6A+=H{l>tcb8;97y(`IpakzzT&dQBO{1R}0XvPk6$=K^`dr%Un{MX|{m(nW}P z)s9YM-u;OQ;{WUTWlJ`CjhaF@$nG9jRQ045*ajn{(o7MehiSo?^6h|Q403HbgNPFi zD#GXx6ztz1q2Fo-z1Q3#WTedFOgR0Tyq0eeN!{Ok=|Viz?U3)KP&3Rse; zSTOS5!;VR0?4m}E-Ne{?V(bNtCYD%}*b62xcKy!I!rgoCBj2Cz&DpcFv%9n9?#^DS zjPyUSG@5!=NjFZ1=2n69mw!dtXmUEy*NP;LZZ(0Eo}%$#Cf<@ z1Iu6p{A-L6KBY!_6Qc9L>4SkFa8Zck0mS;#Ah{R%mNMkVRH@ZLRj-H+_iuGI{`35# z#&wJV&1Dn$VP5YJ$M!JX{MJy5iKbmUCmre!&JR#%0t7 zZX`|~Yq&z8>Dq#qA+16I8U8;VB)e!oUjvnNpklk`cte4!N!4el&oRt$u?|>jHHI36l$@hKbdsxalIyguIX+6p`OCvwC zcsSZ-^=8a~)zs=kx$7TTyM`at;Aa`WZ)$Y1sE~oZd^YRx(5~VECZM zP^iUV0nuC~&+z^?R3_WstzWVK@YsCpdRG%P>JtiFjULWddszI8_qe8joExR_1=8;7 zDn0&B&$!`SnsjM$ zU-pBi8`sa3Hnvqu2Uc)g3)t!Wd=2bMFIp(4$quMl7>ZeAxd!p4Mj7CLIPd95Jru|-HDsmpv21UFLp^FTm5WZHeL)sW2v z%WaLcwRURKqEx$WdJ#I|AK^t)uJ(7e*Z=!ANU>zEF@?5fQ{DROAE?}i=#~51s!qUC z?$*VBl+xWFpj+$TCt(Up;%(AsWX)IqN8?T0tyrmBi(OLjPez7y6N!{W{dIxw7v zcI1#13jAA0WB#M7Gu8Yq&}ys1(e*F;dC`#`*@R~x6hQ15PPw!|@m65S(=J^E5r0Z+~EAl&*x z>ruDv=}GzjacM4e2lrtXL~Cu{m%@TfI?U|>oNpsRz&N~vyI7T)+an+Bi3y``qtoje zb{F~cYs1PW4Ugp^^}{Li@FA6mHiD>*I9o zD;CEgYW^{VJEagau?XV=l`k84v=>Rk^rPgQayCRi;{RLIXZ=mRm~SFEbP%b!84Ast z7gJf#mlhPu(9U=8{!*EaB1K;s8W92C(pULsu9WZl!r0sLAk*tW(b_Qz!VPs(H%SmY zBpoZ2eo8A`hYfLT*Iko(Qq6|QL;rjhTZ+E~WPdDiVUg*^`>83B(4cW|O^S!6Bh@g? zMevIh(gBn`RAtWEpNm4;@AIVllG2OQz-W`Zof8d6g8_oT7>%W83sZMNqhx`W2HC8u8{MUqHdPO)a(G;Li(5Vad<+B~FBbyB+;#t0nZSobp>1T|cgydop~4LeMn z+uwtbTic07a7n5?D8j|?xrO)DZ(DS@-H9OWNUF%p7akA3G^-AGC|@P`pt$YlLe2?iZISBBi88BG#{qs z-h8DkU|xi~dueeH@I84ZdOg({j&=p0E6tdPXzPJ5)TqD#V_GI*Olz}S#|lk~5_*!h zr_9xT?2rbjO5>7EdM-A#={H+w{W^qd3{4k_zd>u!aHW#cT09kcT2Prf%UE3dFmavh4Ji8!&?K72m{=?H14cce3I+phI_VuG@xKv4dP4l-brVqwS^0&?xC4sv0X$wd|Y z1d#r(bdaw2C*So*|M3ZQIr<*e`8>TAo%lql!N(nHI|#x1)ZwuTD#OWmhjV_OG;OqL zJ;xA^zfTKf_Jy zdCr;u5eF`(;HAMaia-_>uF|oriw>8WnqK@rrb(>Y{V`B&r8xz4`Yi=B>VfJ*C)dZ! zu>kieVxp(FzomGShweHmJ~M_mrg<81+6Pf_gv6l%g?^4*&gR7prV1hvYSUqr5H`~< zb$OTw@D_}%4U1g1Oco0|8P>N?gvtDUM2hZ)#&_lk0xEyaf_}8NnQ3OiA7C4l6v_Hg z*S)6Jc5osL@N=HD#ippDlyo=>A==YSxEX1{%?y+*jB}rqt{H&8SgtM>iKE(+;GCOO z(%~|HO!ubXNs5mh-oG^kWJG?m<|uD_K8OFA9U=q+DA_yGp8{v7=D_upIvHxzcwd8f zQcB5a7xGG1^o(nn4jgJj*ea`p#f>6^B3&uBv7$S33A3xGZK$OcOa5q!sx+QIrMDFV z>=Eb$OkBwd;!+<7>4bdcxuNnokfxnvvQ59LeRfXvVckf)t89a9E-8o`?dcv}iuNQ! zwVW@NYHXkuJ?P97*x^`)PBjrmwq2u0_pB*`DG!ku6>T&y=eT7G6zwrUkYwd6$`iOc zL!^$mmob5RJyo%Dvie=|cA$RPII7^@w0@ehGnbktbAOGJjKNc*i%`UL#W;LAdd>7t zG^sl!^^5d1)G7}s#Zx6D?V56zd`jou477+fdD773sA<>V)BR}TOr-{Ey=f*2`w~|x zM4d4%WR@DV7?L;(kVzIDBr-9wbXKOx?3JcS#PvVZi;~MmHG5`~lbImduZJv7j;wB| zJjN}Y4P33+I;!yBqdjQPEqrgU^%<($ZdW4d3#9OyZ%k3j&~KP|+Ei2h1g_Af8Gk#y zgngQSzXZ4ET8V4^D%#i3y}1U8Y?9K`bFj#3c3)K?AJpqR2i34tK{PMU$2Ur4K{|O4 z)yYbqn%~mUI6@p39OV9on}Hy0Pl}XFGH(sK#6&_vuKAOOGX_+uJ{OAjJEBQlXvthP z<;Ahq5XbjU=%jt-sZpP|2u`1-;S19v>;vunZ$5VO22jdEENoL8EX8Thd{dKp1cYPg zWto0#T1+V_djJ*JLgzxb2e5lBkGM4p(+5(>LQ^4YOS*HRO% z{!$cZKV`2#fjFPFOV#!XGd%*r@wAstmFun+Z~AnZuFT>!-@QHp(w8=tFpX;LZo{$D zu_8Ur{wUew<)D9*Cq4ZtvXcFd;_j~j2`a+`S&mWV3^8)M1;Ei-SHRwf5YBc*I_1~b znbKIO+^->(+Fv}%o#OpW8x}8Df;gk5gruxW*P1lD^}r_d?M4j5v&u!`9g}ZN&82d= zDBR8b27O|O!qqlZ;(@j{AF90?s(p&R2#sp}YPCvcH~&F5sX^M9SZ5l%#*|U!8Yp>H z56KZHnS<9V`*S^YTMJz8UOK9iYtw7xTLb5ZLL3K0>ZF(5ZGSmcX+7fYHg>*yrC}%2C+jK3c_6BHhQ9&vi2-F~ikF2mpe;T&YRMduj@YJn- zL_5yNS^0(F=u>?+LCv@^lH%^}SZ~_XC(5`F&AOhLB6SUa-{MTUbyTBdYwi0Mx(=MG zgS1GAGLCnQ8~q({-+ZCNt#|5Qii&Mk*5}6JFi$8HK~a_a7p3csObxB*L^PJ%&1ftO z70xBAsihQUPBbac3xgjQ)9k}o7~FGH$I;k;yfCP>L`olTDQHJHdka)vt{_?);N#Op zR*AF?_1hrC%wqEEIGUwR_1nTc`+dOw7cl#c|3=UiF#EmFf&ym0pR5)sjI&=h^@JVJ zyT}?Ty*ewZtl^-X^5E6h3rGZQ+G*O5y1o+xKS^oUm*})HyG*Oug}ZVPp_8WVHk2kKz_NRgLFEeJP<>s9ROt3 z4IKnaiy+!`$uwGo{{UR~`#N0C4{B?Osl0v!q-L%Tf>nj~&4VF1fH+vG-iIWfm31Xb{@lI_N#~zbnYip(Yz0V3bEMp zAw>x_gwwc((v4Bwcyi#3cWeYz&M{2RIv-Zv5Z}hAT^rk12&Xw*_y+7<3G7rB-PT zr)_;QkYWIq~D+z9Rt?V~Fi9)IE@u1lmz z_c4;|SIulYo~<>A&Jei3TA<1IcR z-PkCyD4o7&a(=#-0QqjT2Eoxtoe$t~e(Mw|wOb_fNB7;y(e9LS8C|5yR1IfzN1?wV zkAy5KZ80u7h)({k{GQgJ9KXc}>-!kEdQMkMUpDvlDX7~eSCB`?uO;Nw^mI20nHp_; z3!4w=cD;%g-fFFeD@~!Zu~_zKpr#DG%&W3hLegTff2p7RbCw&gLFfH>keqn<#y)}S zQyW8$EHhPAzdbs}oYiRBEtW%Z{riWUFz5!j|M}4fE@^+9?pJ}kctpeDhRiH{5C3wY zNs*a1fqVV4h6}R)7~1D2C=&fUEcgKL!U-dz*v06?Pthd}hoD$L_dme*@gD-$mr!3t zXxblYbDx{stviw>$B;FDo9Y!e z+ZgWo4?4QtknMaAj6+IFNRwA4<+!OW@ndP$XOV>|XAZWS^t%e2=b-x_Y3Of&`B3hC zr5c-kYE6L{KnouzcZ*9ZA&!+oq@eXfwIX5SsSiP1_I)XRRXQfX{>#GUGBJpH`3P+z zse#0GDjQRpmONH{l&M@EgX)JS5|Ujm#?JuZ70#i@=zDcq2#6)$Ve-r6X}J*misQUO z`?tC3`wV(NzLa(lg6{vZIr2d$^n3gS`d!S!b^S|?h^*Sozo6Q49Fo;oYHx`2q3fMu z3V6t3L3oVO!`H&0R((Qu^6Z9$dMCvrEA@&oe&4S)+?Wt=_5NdW!At&uW;r;~nwJ6Y z=08=H${RCLpV5uyn5pSsKwxV=eF~%fl_wqguW1u+&A(uti!-nyEw(^P)38J8v=hJY z5&H}TJtt6|SJ>JK85mQ>;A7Niv|y58iK9i&bz6s0BGov>AHJ|S{+q^W8mW(wse8S*>+3XIF=)00A$ zC>ocBCave;Qt@2#%4(TucxFry``xHoZ@@BUjRq-W7$LY|cLd~{jS>>`*0eBK`xesS z-)RITDROR10dGDhzXNXUHVua$=c@U4F@Z*(L0^-kze}-E=~MLm%iM z?>sF|)X&+X9ogri&vHf{HJ<2kzr^5EiHlMg_cuZut3K61USC3!(O>=I-M*TyG)Muv zeT#}9qaX7i*zF6lf4DomC?G@MX)Nw^yr{)^6T!I{AVD}{>}%;xU_tn&J7_X?Z?EqV zxGl~S2Mggn`v(Q#5?z5?T2#Zi5OxlYbFW}GKn^1Ew8%1l|FHzEax+c(&k0;Q!bNMi z_U_8qJcGfRsn{W~h#@UrrQT+)~=mP`@Y+#K+b1 z5!(5Ep2k0|C;tDhO;=leAuV9jb&@wYD)>PYcld$Qd@b>G z!pEY0OJMKqT@sz(01oI14tyQMh2UDTC8$hEwT|TyUMh*a{5lETaB_9TZkJcx*aF7A zC8dDd(nIoPHHrO*XdxFAq9nx;XV346_M6g(UzeW5$<$D1#~!QhL>Nn+6eS+xbh z7VSGTUrf6$AZx$WAZ2OyM`{wlSQ44htXr)cT z`GaI329zhE4KPG%?UrZP%E;stnkZ-5Sh~t}QE|3x-z#tnEPw^IrqYT(6=m z!?Owgu>1Eh3CZq*Eg8Fw^>tOjGUK=gDN8M?nMypg8XyC)N)u``%e`vK$Xr!5ssrNo zs}8cdy2+p46_D+>bddNO$|)*!kKfOOoQW+)Wlou<^6hGZWzYi&39Dr(NnOvzmbAY% z;}HVfcTY5`B3ZR9t~CGGSa*BuF@cT7B2APJoW?!U&uc+C>9v8$IE`s8W#Nt3}J=K6%hf8&#!*c8(>cQH#ForC6ACPp! zqSHg1DcZ5Wm(k;kK!9ETht7C5$PC7Zld|@iyVV18D{MR(nejEoi5`3OccR1>=*W7V z`3b4R^^k{Wwf{<1g4XviUCNtS9~pgIvw#$C)VxIh0=8h>apYNOh1zaF;SKQ_Y1J|H z5ti+8u>nX|nsC~*f+6!70`hfZ9hFxjOa1&og4fNqFr8Krm2s88?hBH0W3ZIRuBT?E zag8lncxBEgp?ZH~^Lx`q2XETMqFrC-0~^yr`eWkO1T&(*b~+rgFNzOS>Xw@rI-b?* zZ=4!U371KNUq;oUUiJI?Q`7Cr-rUcVnpr#?PSTkcmRg3BPOR(C%}|`W9|;mfO>1s3 z-dOI~9MaPtOX-C!(9`a+6hGT`p`U=%u|ELL$HQAj2x)!80dPJJyfII3ePsUv?n!+R zhEv22iNgG&z&MfD+TudNz0tN$wp3k_F97}1 z63rT?6IgL%?nOOYS+o=C*n#(<)lsIpxP*ttML8M%I;wxckCU-30j;6SXNek~by4); zh)cY=H8ePrq(QvMr;SBB)5t6z;t$84N9iD|+bAU&a#=u9M(ZHW+FFbct|YbvW@!uk$v@(n{vbmn`@nGYk4d{!nB?ie!mftnfBZI0$0;IJZ`$A9^yztZ znyH^&&O>@m>hD1(Eoy?q>zYj@cw!Boqqx+*vrIhg=L%%H!8DIL!Zc&?NO(s}e|q%m z0B@Su$uxJ})CrJZmJ7%L@^T#LOtp#)a--v4zz?}URI_=0hja2;sZH7xRh@j!A+QVD zO?kZ6Ep*+#tiLm5cQLJXu6031g};>i;zV70nx&YaOZ0r7IA8wnEPOcM(Y@+8NHleG0b zXEek`LN<8}DnbkPnMyP)8Z1Td0YI<-f}<&!c0?1{6agK1*OCm%2TeNSWT$~C0vGyNEHwMCQa85%S3@Sx|15~N`x$4rT zW%}UOa2ex|no9F_07|nf4|4sa#h>O3R0O<=Oz;_CoLVd>D;akkK=Hx{DLAV&X%HB% zm6wpWgVaW(^zz~h{fVklnmZXP<-{rbFv0UU5Dc#-A)!B8%F^-R0bYi?PmDVf4_x>E z>2T|63~;4kgH1*JaxidP5M4#s+iQr)ZFLy}X()o825|8K(6F$sLp{S=r^Cn-P1p{l%+ zQRbJ3R+7(0(ao)h_Mfb3yWA)QVfk~Qfb^!&OYzRsF-27+PnJ4Fn3_&1f}7;QU9&n^ z-RNzKB9!JB4&5duXfSu$G~8mmTk=pql2SB?GX-=U97NU;ii*pyeFVzz%O?_-`+K}E zO>SfQx-xPks6N6skbI@+w`cKgv}6PtQuYKjUzVzk0=O6=g5-gr+9)-Kao^4zg)CR4 zOUUKa0cB|7C#L4O;uEmEKaCE2Vlf_i_Yf~0e)I)R8i%RkP<01R9{%F#(GM(bVt+J@ zr$_a$jWf>DLVq+&PLI~Hpy26|hhz1xM6v?rN8hr*I6va0Zp%3$MQ?JSF`$6c7ptbD z$+sAbIpIZxt86%eflwJf^w(I6pTk#_HqO#VF!|E8OeC~he{V{Ky(TS@Ic~`s5J2zh zt9b=C!ja2_beN&zxr|4$qBsyR4j*}1&lmmL$8=w1uYi2MQZoDDY9fVAFzvYvn*hkd zH4>7u6rEH5QICyL{d*~$HPK?ceYtNU(*BSq-M0~@db@T&km33?H`Bfw2e{MBG}vnz zwylI+0S-3ccn$hsmy`x?20q|>Q{%ib3Ax#@TO|kpaiy5gE!qJp-h!jg(KuJ^(?Lp3 zwrG!V@l^KU4^+nrN2otp#cjXFPSnl2$?#*5bRZpmEcD0#7y2Y!bvcGT`_VLdRl|id zK^S*%KuM~(MD+#kF3}cf^mh)$W`HpnpM0so5ftvRMdA1xR4FTPvvJ|fYS9AK@r~(Y zLU7}P%p)*oKn3zTX)??#8w`UW_N)dRyc;E*gX%#UDi2n@Z3b|gt_xkXZ(BSaczc&x zPQ(7X{@gEziQhN{nJ>C0rPptufQCI$5%hlORFHc8Eg_GmstVyO8$32yYshnf)V_Q( z=-M<;%zh^+&iy-}EKQuQRA=WOH4}sCbVqu!3YI=S!&1aP_Q|UksK1yQXk;OUG@Nl8 zr1VT=gtG=`I4lEPX^_7Af-OfF^m96-jml#uJIQCWu@4)ikg4bHh!7I53zNZhF`ixwe}r3o!edM#`xk(h>i z(!&-5T_~GOb~r>}%@Fz#?>Yo-!bB0|w3q7J;j%5+SydMO~jX%bRnnfh|W(&S~3u1=Tq&+#LF_~W;kYA#2~8)b;JiIg^RUWG+;;<(6B;S=L zsx~VD`7ukQa-!oa)qW?J$ZHiKjpj>>W078xLbjM#=6+>re8(0E#yMKqCu1)(`x_K2 zVyP@x)zv1a8?_p_WUP>ipvf5*qkd~3{d%>KCQ|kqHN5dk;OrVOJFJ&!xu^ACs~}v6 zFV}(~Y?FkPTxaUGUDrX1ux#8Q%dQ{jLD$!rn!59PK=y5y1aN40ym}M-gWd^pMn?s1 z@@|RixWP0>KDOOtsMiNXTaBj{I|h2v+BK$3!#0Af>`#&`dN=yLU4BlR0Qmz!X*OBr z=bQV$CiK2qCxoSwDC%2NtC;gGQf$c)g*9$@b@>hgk8?7&obN1U^7%N1ZnYefckg>q zWH2Ki#HByH85G0+kQC@XPSoXl-Ill*duM+S$bs8Nh#OVhVrme5w*WWczJ?2;#P5}P zc)*$d=RkL=y%nS9o<|Y~_gM;;-ApseYg<9}#}f&`E=Fl;eiuDEZJXj_?N@FCZqiff zJ!)(>DW0?)(tXb*OYU}43$3;T(vMyX>2TV*!=hat;mHcUN5y}@ETG^`l5ch*ZG8tF z1Un#36uL`!57uu0{%~ATSi@;2f>;`~8`7mkG^sQF@ou0GwLfN>dHp7EP2F_3=shMU zyI>C>UwP^vfqP9|ruSYz?v|h@7j&DS-jusn*_2B@(pRz+ylZn{A6QD@u?u0G{9PL_ zs<>0lO}K26_5=4v0OyuJrcjDO$ zL2z%`q~i3>!Rkzlelj`GV?P0h`*4i==#Z)9(IH5K!lV@2Nq9=stZBO@gNzfRv27$A zT07CvCWE}l?~th>R67FN!tFJj7e$2)a-~y8lofcZ>i3Sc^sZ$fg?AcrD!5Ky^L-IJyv;chRJ$pT_@Y3`jgyY5I6{B@kLTr4$;8suXLi^;{k zdJ0;MOwjd@?XlPgjHs@rrabHByD=HV1XX{^6gy>hJ8$7RpbAOSAZ6%Yjw%a>lPq%v zBjDOm8pM^pKBHzf998_qh(X5LdG&D`ybPtCHTjw*V@<^!HdzPn{IgnBvjP)-h5{Wk zbP(Tjy6K>B{2kAs7@MbS5HGrTPU*--3pfwR-b@WrhDv8(FVb|7&WG-tM{e18G9}Mw zUi2s%y~1OZ+JR8pZy#R#GoI;AHYUffu0AWIVhg2==c* zMQ^-dYLOZ1O@`XJRaCn1Z0od(;Mss#f$)|o={Tz=Fo`Aj3*DTq5|ulm5}k*)CoCm20vZ{ zLENuWy5`iN>J;vzKEZGtjk(uB;PYEv9uwl6DENlOc=RIi23U?-gwzU*ccEH;s?!Wy z?D2mBlKZ!Wc!@GKt8c}_Ft<#VcK4Af=YZ!jO-vEM8<(|}rmz-PZ^MXp`7cu%_t!Gb z!rP|l{*Bwv;-&*RYuIoHkoHbgC%~FWnXgel`UWE}mfuDFJSies zHdeuwvi~*sbA3H>nY-KrRhYZPJ+5nYqA7*r${29`xzPVU_*QsH9NPxq zJruY;h!Z1J%XW<`Z%~c1-1Pui?)GJ>EYE@vwue9jm(|LL=VN{1g6x_4Jp%4hu*3!J zf_pEMp6qTgK6mYAk8v_$N=1=l3hjBUJP_;i=VP$csU)O}^XrVa*LLKhEQ_nkWW!!s zN@WGd6=1?IpCEn3ni7^$AuiZn(My%%jI#qb#P$8lvOGUK)}z_q;JNv}|{So2DslhEnqbE5Fu>GY&XKnrPhGiC7Nizx62!`e}1I z^VG7?ppx}pWEI>>OXftr&(xPI);i`HAPd?`$b)C9m-AVjabYGM*0Ve6Q_m{) z0%Y^MFj-a%3V&sdsXjT+>h0t3E%yc19TDAS#;6_RSHN8_!E~jklve8;XY3-((mvc& zDXD!Vd`v`~FL@QWxzYan7GpO)-UB(Oy++P&A|&|jYm-5S^fu8hz->s;@NV}msI3RQ zg`y#Yw49B;mKA*_Am0yW2&{tRcIGB=@*NN}h6==BLhmd|VTGZ7<}G}NHG}6Yw&J8? zip1^@!`G|Y4p!~69cLKt0LX+4PC_2s zjVnzXU9862uoncR%Vf<_IMMvEasCD>Y-NWyp0H}As%mgctm&stU>S*rdx}{T=(Us8 zcw@P}Go%5tBtf*bpq}aN0?2_cBm`rOJ6*bvjHifPl>g@0+Z7ip{2HSDv0s}JSI(dd z@2Z{1#E87Th!t&Pp1=*IwzK2P8gRU7j{nNkC+04q*+s1j3`TBS40 zNWmVevG7@^J|2KP`c8x34B%22qQe@K?_%v6dRjdk-047Z>nMuY7*~Xf)iy1P?s{7N z9rj9u`=+?}jV&Ss&)uH&LWTqP=rW8cZq*zb7jNEnQy)BjKnM5VVd{vi+LRIqGoJ91 z#HF2#^P!U7RxPk8(>C*l)dCOcAbZZld6H#MLBm2>AAk!T(}DXPh^u8N4wDuw37qFC zBd!eXDQVTV;J&SC!=~TSI5$I#k;t=Jno6{!l&Th{Eq2mmf~LP{@P>wpX7~RHFq_+W)(FHyxI-Z*;qE=Fg9l`~*7*vcWtt{y@#m{3NXL~x2XPEpu< zbuW)DgOG}hJ73;pW&a9bne8j3JO{kw5g%Y*_uERu8)q?h14No+O0HU_M$c*g7Sbnmq!eXxrtwv++WLf-%txz2gU0oxl3lA=8|3?0CRkDp1iv?u1Xb$9 z7cl*t@;{S3b~O|5Byy@3U%r{u(MFu`VD7 z2T2yxVk3jJSWG+~OMD*#Zqp$Wk`@zRz(jBU0Fz#L%b?ExOapF_coV2GrbzXy+MYg_ zuxf${)OV!Jm{&wtEgCOpOso&}tAD0J@cerN6Qo-MK(>vQ5L8_O&CI_cATK|cki`wH zT9h7XbRk`AmC}Sprf#sI5v2c4l~V2oVV}nrusWXH7`VqXWwmF|h;t^dCdy0jRN&(& zCWn;qg@9)ps-${T@O(c<=Ep{PGrgcHDbx(OS@X2~*robX_|aq+!$O8##c$2fQ9~C= zQk=Ka_LtcJ^O}RI!Ac=bqMpx^@oiSO_~y_R!&-n~?rNFawe0u`hTg-c`I?0Ra(kW3 z1G8e|e8RaU^7wF*kj7H_s`yfdrpkwHqP`c>0#1CflCiBo@Q)r+YaLkZ&ffU81}=EF zj;inG_>$z|XPPCQ5V(FnNL*MOtMMe*_%@L4_(@}NrPpn&>f5mB$)Rll`TCd!!P{!F z1Dxsp0XUEO1JyPjYkyCGU*FWg%$Y%Jd zGYEdfn|+YIA8(q_1!w%vl?&3bnOQE!jYQPg<5=sgNZxKn%ahEci;x&aHYrt zEG{&)yXx#%V}&@@!$C<+ni4U%wEcCd@jbv|!y9ZG?s*SW-PHUDkU2voB>N+iOTH8%h*6jY#NFf1QR? zgQU3FcPA27^+Ci5&G3VxtlH^zF2uYj5I7<1L>>g=(CWdSbZm@Lj+cpiX?5BbiOb$R zxEOVaGKG_T91W^UTQvyIyR9Ew);=E4ju_xt?a*)~s70*FlaGr9Waw@Q$&EElvs?6s z^uj(JLEttx7yavNT((yN_XJPciLycG64YSlV1)N5tiIE;;fw7-lgt@_T557g2W~ge zq?KhL_&z%-ddS%PmVh}zh~s*EbQoyuL#qd=Cc;*W7B7%xouSwv)|%AosR|m;E~;}J zs^?{%^s6|N`}i>qxh*~~)4Il+nECOsUKfQll%~e36+7q4kB{wTX+8U3^?W?I#wC%q z9=#iEE#OieKR&ihL9}@R(~HN)9CE4UFl!z1Oh9^lWOnfYnL~eSnIwgpN0%w_mW#s$ zP3S@4g(hiUlM(y|SgC{AM9c!rD2%Gsp@QuQ}KpsrWc+b^2|DvgDHy=qh*f z1dm?~E=sR-J&~Emj0Cfzi%h!@KMc$>#oG)UU4^t3H65jxIRtA-94#4V{jawf((X72Jf4oY|kcQ&{`MI7*+lv-lOF%G^5agl99oULHR`nW7 zH9nePHC{EHH~|DdTjwt%R(z~fioqJ+qb7p4Zw21 zw81czFC}FEheMrdzfCP>nB|2HEO84Zgd1=6m&h&nl&W;bxhyvovC%r2=ITx>p37OR zs*~gP^!SdYStqSUXwPv~U)-l&PJvo~Y6w3{o2rs?^K80Jk_SyQHO~&ypzhH;X`dXc zHyxj*(z2$n1*G*ZNpSrb3Y{`tRRG_eX}WcYappK|pM)p>V)dZaTVa5!E+zx?m;u@e zKWem2bZCb1^L&?O&}%Y$AC z0fx%p*`Ln>6z9))2?>Z!@TKeprmPMv0L#b80@9r)WJ_p z#qnAMg5wHOoMtUDHMPS6vSpHFskhjw1*wsTnT#Bm)b%}6L|h(o~hp^sM4jTM*=r4&&wt?*1{9}E5~qI~UdL^At`^2_=E0%kIS zSf#GOK}=6wtQCGwX&DQQGa2@mZs& zjLSNoP?+w1V`^%}SA*%ShA2*%t4-=`6^OW-ilgcpwTr~sG`I$ZQ`SJ?A$O!oIcuz1 z97WzTc3+FC>4!*(!fa#Ls@iL2S2;iQPy|*3NFQ z8n0U-aGHOiXCVq`JR-42h-=kR;=D;8CUYgI!zN`~R_u7qM0c97$?8_~Dx^Wbda+%thG#EwV1xN@1L+#&;&KG2lC3a=Pd-zkH{hEc3W* z2Ilwfd6=Mw*rFP@S#7H^W`n@oNXWx@_D?Kr*nY(NC8SWxWGT*yhqkA`H?_NG--F?- zf&@_77G)jIWaSn>(njc50=BA|1Vg%S1*Fy|d3x;Msy_4agtyo>t6!nLqjd}uwyCDa z412bLVdkPdhTUH!dQ)O~A57nIYySwwPijfXq92tM!3#nhbLvXL+iFAoDDj{&GNK*&P}SOK zO-l~Kwf1Q!aqkYQv7Rfe*-xM!(Ns#a-yiC0@7?YQ$l;a}vj09Bam*o81I=nO)Hppl z(niAnZ8Fr$5QdgB9n;(dU;AO6dSV0NYV)D4hDTdD&E><;BD=H1ooX|*yy2S%LuMZV z|PpyJ0%{|4bNgmzw{VJgZh0X1Y@W6 z(TXPsB|jH}zJ|pZ=lkbzFt1|iXhYZbOCO53bSF?n9cRe2$x|_r_B&}hp|^dB$z=Dw zk?`Pb)O_$sRr5UHZAO5lGuECo^pvTAesv0pgdxC^kPe%Ug`@jvl@*^RZ*m$%EWb%Y zk}qJFKVsF;0`9xsKMkrL`y?(RM@=$#jynCI5Fq}-8IuP0c9}|2<|l#t)L@qDXTW{z zun_dk+J`o@cHrmQc*Yi;einNTW60qgwqV|zRUds=*z9K{k2{^8+=;gTtkxBbxFrzZ z{VWhYDd_A_H+p;w9op_vZ{|7R`u-|$r!QK|Qs8-$&+C33kVls^s?v1%ys8e)?czDq zvga?TW!I|`w^%HH$N!>C&$uB?+x^E~4Cau}A3|sG@%b)zLc=$5nLvS5udK zi$5Iy`d5P(KT5B=0C)GM&@ee;sqt!PpnXnJ7lE1VmWSDY9;f}TUsQ$`i}6D4K6zrh z)i|7YNTC%5aoA-wwc_HYUPcaA0`n-a3J;*dzbXIEnEJn=p8ui)SMagFR?T646dkHO zG9#VVr$3d_H-DHiYy2mqjxVHi!=I*)y17ES<(-t?{TG!w{+22W zANT+I7RvHrVH$oLV=eDmRJd(w89i@<)BU9Cev>w@G-u{cR<>;q9%2w z;&)XQaV@vI3&`qPI>?nu*s#!d!vi9~;ZuRbf|T_`M$WYJ{iM>Ac2AXqqlYZ4pJW{E zPqxsd@xO1HVrMkPx-;fJdiPG8J`-iffNSi!Z|PEe1tLtM=9;5Y7`bt20(57igCFx-ylRw{(3zq%sBnwZKpD%~@`sW^`_D_Hd-K4{v zd7|cW{2tPS@1!*4FOz#+^cT{mZq}qumYW4`aTAHluAAI}TD(vY{`kaCr=M&tAjU(2KfgfQ+AV~jI{Cdc zA2EFi>94J%H2bB=gsoz**w zY^%<1z%n>bI;=}_0UsA{cZZQrzd;T9MM##w9=K*>7phhS!gU5#6ZPnURwy@t0^+K;N~`Ln-sPg!y1n*3`o&Y67ueRvNOFa zY%}h@baDd3^D_a7pbwLhedwOM(wQ@QDWwYQ>io?Y$-xE)XC$b+X1u4jMT5mM+mAhMO$=6B>j|a&mP+aFo9JOJLQ`BRWp1)7 zb@nn<^RWe{dU%Z&*+fIkZEPzgE~9COUile^hrrW=a1d%duMcJGKw-e{Fc!Ng4hltf!rC6}|$eXb7*pT1d^ zA*f^ldD7^T&@CBH!3o_qtWPd)&+>jLuz2H10s;B1l+AeF{DqLtLu@x;r!-%i@dn^2 zUr1*ltecPqHAWBZSK9V|ehnSCik0;@rC~+e50XCSG|nD}_^Fo4#h%~?mgPT5$QwUX z9v%H5EqPQ*kNK;1#}mr30iX{*A*E>L6%7T{j8871Fhu9m z`>Pn7;OL!bV-?c~^^bt~ypjEGLRFh~+d#FY6IEe*Fo1_|FQ}t-Ho)E`Qc%xP*j)q*)UT4nZ%McJ$#G?{gwG~TC-0Iq@3oDy)|11(9A>%{U5UfGGtcEV~U0JCf zW+5dgGaiS}!`zgq98OR6q3XUV#`fPYSf)Yy52l^IDaJmNpOc?s+X@o*psvz`4YRXM zigA#BKnLpS25G+1tc@a^#Pff!<=Lu|kcZ(?!6|_TTVlHAL-Wgs6KJ^t5BUnyfU&W`^T%VZW4P=&ubo6<5@d8? zYGS*pqt@7s6Wh49m^f{02ERGCi)4XgXpx^E6C}1vaW~BH7G|}y#naVxDMoL|&89;O zSa0q}GB@t#HR@ug?v8`1EymUFVk!idC3V7V3-WVj(2#B>&;smqx3py$K$vzR!M&24}&*VvSho$(Dd+!u4yv1OefgKRIaswp` za4xiQe2P0YY-2Ni)fv_XxKr^Gm(#}73j^9hnvo!-F=@zXW?P$f?hH%85XbLEYSJnc z^?^-07|iM}7-q7^$xk(vDpT1HOmZZ32+1q&Fvb%jeiC|Y-% z#_dLny4gG^d8ElUyoQ`#0B+FUpu7(ywFj=wWQogZud0ZP=HCHXTC5ri zTR7058pjZEOEdNIW>aJu*t5O?&a}3Upe;OIgRsdFOl)$BFD3U;_TqI(i%!t6*Bng; zH$q9YA&N)E#{v>KS3*#TQWW3WrrpxuEa!Dbmfdh|U$nUVwo;Vby4bXvAbhK=a~ENc zl^V{CVisYrvTJ*spSirQAmyia2l%7y zMlgbxbhjDLNrv?RuEZ{hWBYgfE~TiwG42T5{=FI>zNc(~aqYUu+>d~pctGNmp^k0G zbU@!Nc*KhCi5ly3FfVga6PdeEm<6p>JK~ZT>jly=hjp2wW?d;Ae;YiP>j9I^n;pwj ziV&X_LGl6`|k+>8Jo{jyf>W255MFk%d2Q@XPl$`TRHhpf;w|#8G^0VTtn-fu(LYDkPj1*02(`A&t))@7EEko9OVDD$Hf|T-hdauyN z4$Qh z)#Kj&mf}qwsp;C6I0PpSqV*Brd-FiY=N_pH@8CQ+#Bnno;b?#8?qS zKXr?>b)&8^rZIGUjLqNS8%^j;k7LxFSxfF1_exU&Av*p}ai(u#)i}skDbK{hw*Q!L zLDa9mQb%0lE7Z{muP-}~=t`Lb;7(rmS2W!4D-N*v7rLp#7+2j_g?Ou%q9{x6T!tGT zy%C>_3$+Z)!*~v|X^~<0ILIlT1VJKkT{v#BSPdeRdorh^Z*rwg$;e1CA z2E`|B^2BbPha+fPli|e;6b!671VI0edB6`^4G%P6cmaHIh|RCi^`45XoB_k@ndu2O zKZh`Sl8Dwcw9oJWO8itQD^3g)8lO0vV*efyPH73li_(TfRl)q3>DNSL;yF~0=`_^j zqGt_-vOPvBOfZ!hrn2Dz^-CU(TTR1kZiQxyR=}dPeV8I+jb9H#hA$@TF||{n`1B+b z<9A8Owfl@bpx4pi#i&}cP4k4DYkV>=tLB);Q$6LS3nGK!&QM96E*^v?d$ZpK3+b&j6K;4 z0sP>5Jz4Tdn=#PE=8?cW*_nq49%b6V89EB$%pdg>hjzlaZ$_DnTj3J`51+^bW`1It zyIm6E-_PWUVf%`PhQ;Oj;0$Vf$)~9C(UQ=hYs+uGCtMg$7zshB%Loq|_O%)d*_xxrVJa|JLEI>JoLa_m zi89Jbf-t`kg=zkH)sdOthw&ir!HYv84?xP$-b+eJHFi9%Dj|VsHs}8@@MXtfs9Aa< zN;JNjNDn+GLy4e`v~=)jNKU%nvCr-r}n->($|8yOI5syWVOZ6F^f$*5yBb0 zwSeG6o^-5Ltu8s3^P=t|nep6GiVZ26;_|)~3-*Z}P)@fDlU8*z0Ez3ZL2&aZLutkN z4*AGriUpAx+=Cu`G@>}woT+GeMzMbi%>2~=4OfghO;y96sy4(XYB>!%f^VjpT4<$d z(5vDQA)T4;msy)TLhxy_5G2#c=_dc#AWH{=?F<$AwWN9-%mODm5TxTlp z!W!};A%)XKAz~=(vk|UzQCChTcsCOSD-@(CrA!`y=z6nEHas#5xHyYO#kgYhai(eY zI3*Lf8qeJw>of;mE%+sonL_*^gDRDCN*wvtek_gA5v0W z9LTa619v570n!|ICPa-uQxEc5Fv2B^Y}(~pcCdX{j4e4DWjdvnEtp5aYM?6GD3BUrkz^sy`axYZwIB>^!oh*rj;N0|IR=%h! z?0*ure-mnC&<3U10r7Eo<_0ij*V945>y9i%p&Qjk1Mj^I+z4EuCOTa1Mr8qhb*9NC zK<0&M5aV9I_qQn3+qRk%9|*ry<388HeE~`9sDnV45_GNA$O5|F(eD6$7j6WXqM@6W zLtr00x0is7&-X!`(Z}B-ZS#+XAc=zejP#*}T}FD_&vR;R0d97L4)?gnNKbp5!&cya z!3n-Rz6!KqtLj{QZoCN&^z5+pq_*2s1+Xr|wju42Jn7YKYHO9Hf!iV7FhZuCwO#p^ zGv6cZMr*3G%T|jvO~=Y3bcYgh(v_c%G=3?=Luq=nJ6+vjnkDdIrvw|d+-Yl>Z!q3w zI%lJAci9>k!eDU2<{hTi8A95`@bn-{d2?wiOS|Rsv4FRfLZ{G;gSL+Uv75A+rQx(I zS{V#!LmZnR;LkzK^=Iv~6(y%{)DS03;&?(y-Ra-UDlNaF7duT$>rJorDk)p+*ykpP z@>;>Qu!*ccyX{t(v*am)crr6ay8g#xow=v&M@8N=rSPC?Q_%SIPZ~0~(*b}f%Sd0r z9_R4`aO)QsaUL}12jvi$?=)}@3+Vbm?3S?Eabjo?puxEUpdW2MN9jhfq_0{$#SY`yr6xHKcrqcC3u=L>)Z@ z(*1}Ult)mKx*S$JXk4n)!ys_pqd|&N*!Lq5bgYFsx5&5}M^O5|Y4%ZDFIsrS@GZxi z&O+!I{G%jkeN=fpPWmbSa6EWKN^c)E^{_I>AnklAPl2CcQqvO>-gHbru3nZ9&!6GV z>>wkL1JV^2IYmj+E&{UTpmH9pMwt_U+`T0sGftR_(EGZn*-pJHktl2tYI@RSwjn1$ zR{F7oTso<~b?`--M!3)^6xsi3tgAL-c%CL#AwAJk zlRA@I=TV+C>$NGvT!DMoH!qWfYiiobOQPl1P}Liwln#X{__|uEw7d>mgb&OUpSrGk z376RG2FTMBSp110Z{8)Zp2U(~)a0hk7)fX9Ad?w(=R+S48s%p&8z0I)G7^FKZM|vw z@2V~l0?LP~B#tt!V0(|sLne+YNqc`(AEa2}=SicCv-;|zB|Iv5R4IG-kP#;Mf>a4V zJ)$7^t4~evkTeNza7(Qd#<<_RO-3 zDn?!JC|mHCztMM4m!}jYz+gu%fAzblupOBiiyM7&*EH^25s=AWY7lI6O+}U1HLNK! zO;xyyjA@Ed;C&PKhhLe}j9H|maieWtjq;+!AF3-GJQj%HbcK2?8%5pv;Pj{0#j&o` z%vH_HxLBzVpz~e>=GZ(aef@#*9=uKM@(=^R1@U7Z+d}ED!jp|3ntV@=^`Kb~v376& zi-eqfsJ?b^PH!J#xKE-wPi%Fl!z1N4SyjHnu>6Kd`ae4PNL3F@lY9|z%=4y@w#%Ez z@G<=~{;Lt$MH6FO^l<(HMm8Sp4*z+f1%|W@_LrG(nk}M&wj5fidPV- z;1uX{TYJNISq`}I4GX#(=FdFus(VMIh%!_Hmyb6Aj^s(Rua7Mt)`sgpXyS2qB>^Bc zDZ87RPw^x=<)M`Ndt_*zG}-bKo+7O)j+lzfpFdTL3zoL{7xnTFr9HPrQ?18igUD;C z`ra*T4hYA(uVwn7e~%3|KsfJT-U;adYW7@J9j6`i9BGHYN0VGL{zqwljIBJE=t-3Pv zWm1b!Xb0-RYx1NWuNLF!^Cucxol=T3JN*vnb2UWNobf8P0L)pQfhVf!9) zN`@!RXp>spzUER2Bj?h%#3VFC&3%+l%Ms>kIc4}cZjI8U4QRvWRD3~mQhIUawoH(a z+*nj@va`u!k~8wSt{^=OJ&_NrxhAF}3YJR-Lc`+%j?iV3OGfv6U6f1PKTb#+8Daym zU#c@+2%6ik>)9`@VreV;bv^r~auY<_0{qf478Kx@YEBd>{>v}@z|v3hwdV@*bb}3I zaPh}ABaXUSQn5Dr2fJ9;>P91W$qVI|7<@V1RGaQIwS?tp$qF^mU{~@G7jA?$c5duL z4`iD0rIf;rm7;CKtwvAR@e5N{;}=Ny1kVi9BEb`6SC>mEd>YoVUZ&=r>II0yDhbK$ ziH>>QC8HRnR{h+VY2D($#p5D~&>A=&+Fx8zahVc+p`|4=3UH5Kl|Wi2ymcp}@J2DFtqR8AGe|q2Giw@*^pQb1iKkU|sk#S?T}@2@1^6 zjLo-nKMDjv<$;6xB$a&!8KSd2s#H1HG<1f?pxF1ymSJL?HLSXcR(~t zp*ZqoW^MP4`K||pZ`5ZJ0_~k?PtrIa8edK|X7>9}QUxT6(k7z!cdw^vH3Q4X=ChNg@9VDC&1`Ptn{lh$3^N^^hD8Qz&^o)`b}Bj@{^GiPR= znK|>$b4Iu8H9!+*X{l7n15m1`@kL>4sd~F6(eD}zq`pKTNLU^$wb1U_>0sFKC^H}^ zv-p9946PgpE%)%Hpm0B~G|!Lyx=D6wa(}|3P_-8sHU$T37WVtss!T45<^V`#G9cgnq`+j1}Hha>FxQnhV_j=2XUD-L$~F1 zD<6z5l)jr28@HirxTLnI84ME>_7YMau61K^VVZh~)EWjNw%$poA0dqihZD6&ox*PH z9RA>(x#$#)B?3h737ycK0;=~T5zu=Dx2EgGkB^iMM-@mPc;}pDm64kGx^qzxt}@+m zJWq|HFA)^VqN6l5L`wT2PhHd$f32$vWgkV!9!th3!Sa#|?Ad57oDCWx*B_Kn@FEHB zmtZ*_X<4GRp{o--?w4P=ffHe@(1_&o_ySk|l3Guq?KY}TJhVfX%)=Y~ynDWwJ)I2;Ys@u~qiGe`4ddtb;Gt4iXo z>D*bI;%!ycm(|3}6$$N$gyVE>w1!kY(Iy*@MdR6nB+YmB4l3Ho2ideVEtM@;BA=j{ zJ-q`L+aWF;Ml3l)AH;7)_i*%8$ZtA{m|evsc#Hkcel5)JzK42q?I1L`rhu z@A`2jLun(yUGp@kYHGIiVAn^QxZbhg3Va0@#O6L{LbAQAY8S2Y7kDT3@+fVXvr?#p z)uXh42auCzv^JgnvC)Rv&ew{}(4*zRO4#B!820RE&~!6SQ@^Esd^C`m!3L!CP3>Wp zH%b-|r!KCG|P2*sBv;U?FY z4;Q2xoA8_MA$B>$L=}6}50tsLl6QoIct@NX<`n%d8(KG@NO~kYDi@7v+wzVH?N;LJ= zlMAig*u6x|Q+IMWH*2CM_P6O2Pn?YlK3iqQt#MllyD9xBDG4^tw<2*{?18K~NtTaX z@J}Bc+ox4_3O-?#*^ik_f3{&qCEtxPVmm7X`Ag zOK0uAt|nhh0e5w!LElGf?4F7~+ItC{gx1>suHeL34ElQMvMwzt=hB5t3&8!qrrt{j zbT&K(?wxg_RG~^s^ETSRrOCJL?yPFF9G?$=fllK-S@VA&p4Cplz|wLXbyGH36X$by zH<%2EPHhoN@ST~(5vbH5aphCM{i{mlyxHRwRv)EM1gF0M_*k_8wr|yXxsIgUwxg3o zzKBk8p^n3zJFG~PVAwaKxnWJcE`3Eg??r}kQ(bNBfMIPU?bG){>!hWR~j=wTCXvz_W{*1QMfYp8OZnV5OZF zWCvDS=kRA&(oDV3?ImcOf6l-qf1(Xwr%swW2l;d!@B^>1{ApT5d3!5zcVr-@?%IIi z&8U~o$5*_Je&l=Epf~9&>p-QNXhr06RY!2%Vww!QX)wE&j?6gLR%z+C){d-tI;>qc zXRVn!9S;24#EBV-vNdnQy&}2|Xi;w+Aw!gJ#s@ctv?3Tox{9!dwyKv&VNd2bAv-*! zmBB1Zk4C;{Q!gE%4dntUeZ+OuN+Wl=8U4+eqms~Q9oNBQeqF{CcXx4&RqkC^_?ecdQFWk%4Y-H z)s8L7cdh`lXPb5$s&%9rQltswso$;Mt~&{-bHE+zWZ2m?!3RGdSRYG@@g$xqXu1dg5k=M?7sP)ezg3p!a z%ZPyLa89rdAIHweh_gOIsPbdY`k$vRGc}C=0u*Ta5Wx**^&gvVw!EMtmmU*{g?WbA zdMOF9=^KHgxbqX0U7o3RzRw{V3Tp>1f(KKc5iBbiZENF}xeJS6by%Xr=00T$biG0? zZ7jU1SPW;2lbx(+J*ti|nNZX+iG+-r5XJXq$1~-?M633wWjdm++T|+eXsfhqmRu=P zOc$F4cQOk3vyi&6#w@uA(9=DYXM;FWDv5Fp3q(hrTusmfuwp$YT!Mp{e})6W9|$Q0 z2R>{(FoQ*-3AR7opxfE=#k3a~uy!y@u!`N+}4 zrEueHHAk?#hYNKGMT~xsDxX}ax`{M$dRY*KsIBqOTC`VlF51h#kt0cYTC6gwY|Vp$ z_J_H2)B#(NtBs2XZEEb%T}#hZ@8Xq*j3?|#8Kgac~olV5Bw8uT@nBILlY z)J66-yxVUFclr|Nx}0mvTD}8F=64*~ebE-B6lm30WT}{(kGfp>y^btbd|&$ylVeRc zxgLZ#YCLuMO<3vsUoM?`-4?7^q3Fi*jxJ4Qhr%(pP8?%cnd>+FCBKa_>@p%}-qyt0 zfSS_305&`xWKr)yOG$wyW`Fr!INV6;63oyB<2%b>BEa2As6`|T35BqL-Dlkb=|G@i zVW$g~t2{n%eV7ZPZHNO?roM*X8YHeCsWxWW}MJ}HDthZkvLCyJywMUa*Z<5Gk?y_L4a zXT&{NOfjmmZj_F^!s?4nzTGW`f@ASqLGDUZ-5gvAY4}7g9Z_O3ZYzOw{`10kXJ#oi zZKE}$n5yG?ngK_6)}4J@Y8p{IRzYpzOwO&yu=iD}jNCq>Bfl-=$i#O|dz;1YLK^>u zFhoIPfKoFQXgs$XH8V6{aGlwu)tdNvjSTs|2UVL2b)?Aox>0yGwn+aQMMG_bXG_cQ z*p*hKrA!k64k;Z`4(Yryt^i?I;C)7n-*j&A` zJ+ovDG$vIUI0Ssiu4J4XaZp=(d28_?0y$n|Kzpc4kJcD?MQ)cgOVBg4;eghJdm zP9k)5@wny#nA~_#sIss@8>A{46!$fAslCOHEDReoHQw>t2n8uuIdZcSQ)R_@J1&PZ zSM7H?SBD#q^xgwi9&AR1CVt?ktkAcgZWyH1*X(^;wVt;HwSK|ED!0K&sG_*po4J1|e@7yH zBR_<`N>7#Rr!+L3vH5FP2R?7yiVo)1(*PrG7D<3`2Xa{Z=-vitb`OWT7@^UYK0tb_ zzs_Yk-)JgJSQW|=^oTB)!Cne_w?jcDQXKDtg2)|m zRHKvWJEM5f&h60DlXidXz+-VkxHLP&(O;=f(){U8AkEPn@r!b(Yd}J>b^*a<0;_gu zIZn4|pn85he{5~FrsgqgjB}_n)s0qihOLHP_ZiY|>G0oEC0xGWic3b?Tn;wu23s(lvyMH8jGm~K?M-Pv9kpnXf|*hs z`=eGCo{BfV4q9%{H6SRSJByv+@N%u_^RJugq;r-qFY1KUHtm7hpK^uTPAohV-Oc#A z(^gmMUX(DfKyXw$v=QUOVyv={kkbEYCi9`VeSV)Nej}habat7pER}66a=5d1J~p++ z?vGKf_T>V>HPprEFD~np1^dA*+Mse4wztgT>k5Y*0Ni)03cIseA2@szn6`!P)^V=w zZK2|Ns9L>8Ahao?F36G_fJ|-_2<-H13<^0%1l(VVZm3qjX@tIO z#|6@XVc$snyhg~TPk_91#(-c8sEsg%-l!Shu>4Q81 - 5.1.3-10 +- backport support kv storage + * Tue Sep 19 2023 ShiZhili - 5.1.3-9 - backport add allow boundary type