From 1fd78338be7bd3bb527e1671324deb9fb78ef73b Mon Sep 17 00:00:00 2001 From: ssssssnake <157384081@qq.com> Date: Tue, 12 Mar 2019 14:22:36 +0800 Subject: [PATCH 001/141] variable "timestamp" should be -1 when the timestamp parameter is set to "now" --- .../rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java index 3f2bc1b6b52..266de75c86f 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommand.java @@ -75,7 +75,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t String group = commandLine.getOptionValue("g").trim(); String topic = commandLine.getOptionValue("t").trim(); String timeStampStr = commandLine.getOptionValue("s").trim(); - long timestamp = timeStampStr.equals("now") ? System.currentTimeMillis() : 0; + long timestamp = timeStampStr.equals("now") ? -1 : 0; try { if (timestamp == 0) { From 9be760a62c9ce532dcf99fda5d269dd6b8b61480 Mon Sep 17 00:00:00 2001 From: xuhongcao Date: Thu, 14 Mar 2019 11:01:31 +0800 Subject: [PATCH 002/141] add pull consumer --- .../API_Reference_ DefaultPullConsumer.md | 143 ++++++++++++++++++ 1 file changed, 143 insertions(+) create mode 100644 docs/cn/client/java/API_Reference_ DefaultPullConsumer.md diff --git a/docs/cn/client/java/API_Reference_ DefaultPullConsumer.md b/docs/cn/client/java/API_Reference_ DefaultPullConsumer.md new file mode 100644 index 00000000000..64a2b7bd80f --- /dev/null +++ b/docs/cn/client/java/API_Reference_ DefaultPullConsumer.md @@ -0,0 +1,143 @@ +## DefaultPullConsumer +--- +### 类简介 + +1. `DefaultMQPullConsumer extends ClientConfig implements MQPullConsumer` + +2. `DefaultMQPullConsumer`主动的从Broker拉取消息,主动权由应用控制,可以实现批量的消费消息。Pull方式取消息的过程需要用户自己写,首先通过打算消费的Topic拿到MessageQueue的集合,遍历MessageQueue集合,然后针对每个MessageQueue批量取消息,也可以自定义与控制offset位置。 + +3. 优势:consumer可以按需消费,不用担心自己处理能力,而broker堆积消息也会相对简单,无需记录每一个要发送消息的状态,只需要维护所有消息的队列和偏移量就可以了。所以对于慢消费,消息量有限且到来的速度不均匀的情况,pull模式比较合适消息延迟与忙等。 + +4. 缺点:由于主动权在消费方,消费方无法及时获取最新的消息。比较适合不及时批处理场景。 + +``` java + + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer; +import org.apache.rocketmq.client.consumer.PullResult; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; + +public class MQPullConsumer { + + private static final Map OFFSE_TABLE = new HashMap(); + + public static void main(String[] args) throws MQClientException { + DefaultMQPullConsumer consumer = new DefaultMQPullConsumer("groupName"); + consumer.setNamesrvAddr("127.0.0.1:9876"); + consumer.start(); + // 从指定topic中拉取所有消息队列 + Set mqs = consumer.fetchSubscribeMessageQueues("order-topic"); + for(MessageQueue mq:mqs){ + try { + // 获取消息的offset,指定从store中获取 + long offset = consumer.fetchConsumeOffset(mq,true); + System.out.println("consumer from the queue:"+mq+":"+offset); + while(true){ + PullResult pullResult = consumer.pullBlockIfNotFound(mq, null, + getMessageQueueOffset(mq), 32); + putMessageQueueOffset(mq,pullResult.getNextBeginOffset()); + switch(pullResult.getPullStatus()){ + case FOUND: + List messageExtList = pullResult.getMsgFoundList(); + for (MessageExt m : messageExtList) { + System.out.println(new String(m.getBody())); + } + break; + case NO_MATCHED_MSG: + break; + case NO_NEW_MSG: + break; + case OFFSET_ILLEGAL: + break; + } + } + } catch (Exception e) { + e.printStackTrace(); + } + } + consumer.shutdown(); + } + + // 保存上次消费的消息下标 + private static void putMessageQueueOffset(MessageQueue mq, + long nextBeginOffset) { + OFFSE_TABLE.put(mq, nextBeginOffset); + } + + // 获取上次消费的消息的下标 + private static Long getMessageQueueOffset(MessageQueue mq) { + Long offset = OFFSE_TABLE.get(mq); + if(offset != null){ + return offset; + } + return 0l; + } + + +} +``` + + + +### 字段摘要 +|类型|字段名称|描述| +|------|-------|-------| +|DefaultMQPullConsumerImpl|defaultMQPullConsumerImpl|DefaultMQPullConsumer的内部核心处理默认实现| +|String|consumerGroup|消费的唯一分组| +|long|brokerSuspendMaxTimeMillis|consumer取连接broker的最大延迟时间,不建议修改| +|long|consumerTimeoutMillisWhenSuspend|pull取连接的最大超时时间,必须大于brokerSuspendMaxTimeMillis,不建议修改| +|long|consumerPullTimeoutMillis|socket连接的最大超时时间,不建议修改| +|String|messageModel|默认cluster模式| +|int|messageQueueListener|消息queue监听器,用来获取topic的queue变化| +|int|offsetStore|RemoteBrokerOffsetStore 远程与本地offset存储器| +|int|registerTopics|注册到该consumer的topic集合| +|int|allocateMessageQueueStrategy|consumer的默认获取queue的负载分配策略算法| + +### 构造方法摘要 + +|方法名称|方法描述| +|-------|------------| +|DefaultMQPullConsumer()|由默认参数值创建一个Pull消费者 | +|DefaultMQPullConsumer(final String consumerGroup, RPCHook rpcHook)|使用指定的分组名,hook创建一个消费者| +|DefaultMQPullConsumer(final String consumerGroup)|使用指定的分组名消费者| +|DefaultMQPullConsumer(RPCHook rpcHook)|使用指定的hook创建一个生产者| + + +### 使用方法摘要 + +|返回值|方法名称|方法描述| +|-------|-------|------------| +|MQAdmin接口method|-------|------------| +|void|createTopic(String key, String newTopic, int queueNum)|在broker上创建指定的topic| +|void|createTopic(String key, String newTopic, int queueNum, int topicSysFlag)|在broker上创建指定的topic| +|long|earliestMsgStoreTime(MessageQueue mq)|查询最早的消息存储时间| +|long|maxOffset(MessageQueue mq)|查询给定消息队列的最大offset| +|long|minOffset(MessageQueue mq)|查询给定消息队列的最小offset| +|QueryResult|queryMessage(String topic, String key, int maxNum, long begin, long end)|按关键字查询消息| +|long|searchOffset(MessageQueue mq, long timestamp)|查找指定时间的消息队列的物理offset| +|MessageExt|viewMessage(String offsetMsgId)|根据给定的msgId查询消息| +|MessageExt|public MessageExt viewMessage(String topic, String msgId)|根据给定的msgId查询消息,并指定topic| +|MQConsumer接口method|-------|------------| +|Set|fetchSubscribeMessageQueues(String topic)|根据topic获取订阅的Queue| +|void|sendMessageBack(final MessageExt msg, final int delayLevel)|如果消息出来失败,可以发送回去延迟消费,delayLevel=DelayConf.DELAY_LEVEL| +|void|sendMessageBack(final MessageExt msg, final int delayLevel, final String brokerName)|如果消息出来失败,可以发送回去延迟消费,delayLevel=DelayConf.DELAY_LEVEL| +|MQPullConsumer接口method|-------|------------| +|long|fetchConsumeOffset(MessageQueue mq, boolean fromStore)|查询给定消息队列的最大offset| +|PullResult |pull(final MessageQueue mq, final String subExpression, final long offset,final int maxNums)|异步拉取制定匹配的消息| +|PullResult| pull(final MessageQueue mq, final String subExpression, final long offset,final int maxNums, final long timeout)|异步拉取制定匹配的消息| +|PullResult|pull(final MessageQueue mq, final MessageSelector selector, final long offset,final int maxNums)|异步拉取制定匹配的消息,通过MessageSelector器来过滤消息,参考org.apache.rocketmq.common.filter.ExpressionType| +|PullResult|pullBlockIfNotFound(final MessageQueue mq, final String subExpression,final long offset, final int maxNums)|异步拉取制定匹配的消息,如果没有消息讲block住,并指定超时时间consumerPullTimeoutMillis| +|void|pullBlockIfNotFound(final MessageQueue mq, final String subExpression, final long offset,final int maxNums, final PullCallback pullCallback)|异步拉取制定匹配的消息,如果没有消息讲block住,并指定超时时间consumerPullTimeoutMillis,通过回调pullCallback来消费| +|void|updateConsumeOffset(final MessageQueue mq, final long offset)|更新指定mq的offset| +|long|fetchMessageQueuesInBalance(String topic)|根据topic获取订阅的Queue(是balance分配后的)| +|void|void sendMessageBack(MessageExt msg, int delayLevel, String brokerName, String consumerGroup)|如果消息出来失败,可以发送回去延迟消费,delayLevel=DelayConf.DELAY_LEVEL,消息可能在同一个consumerGroup消费| +|void|shutdown()|关闭当前消费者实例并释放相关资源| +|void|start()|启动消费者| + From 13f561ba87ecfe062613ac63e23249627d2cf40e Mon Sep 17 00:00:00 2001 From: zhangjidi2016 <1017543663@qq.com> Date: Thu, 3 Jun 2021 11:00:13 +0800 Subject: [PATCH 003/141] [ISSUE #2969] Add a skip accumulation message command in mqadmin. --- .../tools/command/MQAdminStartup.java | 2 + .../offset/SkipAccumulationSubCommand.java | 126 ++++++++++++++++++ .../offset/SkipAccumulationCommandTest.java | 77 +++++++++++ 3 files changed, 205 insertions(+) create mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java create mode 100644 tools/src/test/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationCommandTest.java diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java index f9477445bea..f70cb6f3df1 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java @@ -68,6 +68,7 @@ import org.apache.rocketmq.tools.command.namesrv.WipeWritePermSubCommand; import org.apache.rocketmq.tools.command.offset.CloneGroupOffsetCommand; import org.apache.rocketmq.tools.command.offset.ResetOffsetByTimeCommand; +import org.apache.rocketmq.tools.command.offset.SkipAccumulationSubCommand; import org.apache.rocketmq.tools.command.queue.QueryConsumeQueueCommand; import org.apache.rocketmq.tools.command.stats.StatsAllSubCommand; import org.apache.rocketmq.tools.command.topic.AllocateMQSubCommand; @@ -186,6 +187,7 @@ public static void initCommand() { initCommand(new WipeWritePermSubCommand()); initCommand(new ResetOffsetByTimeCommand()); + initCommand(new SkipAccumulationSubCommand()); initCommand(new UpdateOrderConfCommand()); initCommand(new CleanExpiredCQSubCommand()); diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java new file mode 100644 index 00000000000..95ab7e81a99 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java @@ -0,0 +1,126 @@ +/* + * 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.tools.command.offset; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.admin.RollbackStats; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.ResponseCode; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class SkipAccumulationSubCommand implements SubCommand { + + @Override + public String commandName() { + return "skipAccumulatedMessage"; + } + + @Override + public String commandDesc() { + return "Skip all messages that are accumulated (not consumed) currently"; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("g", "group", true, "set the consumer group"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("t", "topic", true, "set the topic"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("f", "force", true, "set the force rollback by timestamp switch[true|false]"); + opt.setRequired(false); + options.addOption(opt); + return options; + } + + @Override + public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException { + long timestamp = -1; + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + try { + String group = commandLine.getOptionValue("g").trim(); + String topic = commandLine.getOptionValue("t").trim(); + boolean force = true; + if (commandLine.hasOption('f')) { + force = Boolean.valueOf(commandLine.getOptionValue("f").trim()); + } + + defaultMQAdminExt.start(); + Map offsetTable; + try { + offsetTable = defaultMQAdminExt.resetOffsetByTimestamp(topic, group, timestamp, force); + } catch (MQClientException e) { + if (ResponseCode.CONSUMER_NOT_ONLINE == e.getResponseCode()) { + List rollbackStatsList = defaultMQAdminExt.resetOffsetByTimestampOld(group, topic, timestamp, force); + System.out.printf("%-20s %-20s %-20s %-20s %-20s %-20s%n", + "#brokerName", + "#queueId", + "#brokerOffset", + "#consumerOffset", + "#timestampOffset", + "#rollbackOffset" + ); + + for (RollbackStats rollbackStats : rollbackStatsList) { + System.out.printf("%-20s %-20d %-20d %-20d %-20d %-20d%n", + UtilAll.frontStringAtLeast(rollbackStats.getBrokerName(), 32), + rollbackStats.getQueueId(), + rollbackStats.getBrokerOffset(), + rollbackStats.getConsumerOffset(), + rollbackStats.getTimestampOffset(), + rollbackStats.getRollbackOffset() + ); + } + return; + } + throw e; + } + + System.out.printf("%-40s %-40s %-40s%n", + "#brokerName", + "#queueId", + "#offset"); + + Iterator> iterator = offsetTable.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + System.out.printf("%-40s %-40d %-40d%n", + UtilAll.frontStringAtLeast(entry.getKey().getBrokerName(), 32), + entry.getKey().getQueueId(), + entry.getValue()); + } + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } +} diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationCommandTest.java new file mode 100644 index 00000000000..38c23a3e857 --- /dev/null +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationCommandTest.java @@ -0,0 +1,77 @@ +/* + * 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.tools.command.offset; + +import java.lang.reflect.Field; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; +import org.apache.rocketmq.client.ClientConfig; +import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.MQClientManager; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; +import org.apache.rocketmq.tools.command.SubCommandException; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +import static org.mockito.Mockito.mock; + +public class SkipAccumulationCommandTest { + private static DefaultMQAdminExt defaultMQAdminExt; + private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); + private static MQClientAPIImpl mQClientAPIImpl; + + @BeforeClass + public static void init() throws Exception { + mQClientAPIImpl = mock(MQClientAPIImpl.class); + defaultMQAdminExt = new DefaultMQAdminExt(); + defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + + Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); + field.setAccessible(true); + field.set(defaultMQAdminExtImpl, mqClientInstance); + field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); + field.setAccessible(true); + field.set(mqClientInstance, mQClientAPIImpl); + field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); + field.setAccessible(true); + field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + } + + @AfterClass + public static void terminate() { + defaultMQAdminExt.shutdown(); + } + + @Ignore + @Test + public void testExecute() throws SubCommandException { + System.setProperty("rocketmq.namesrv.addr", "127.0.0.1:9876"); + SkipAccumulationSubCommand cmd = new SkipAccumulationSubCommand(); + Options options = ServerUtil.buildCommandlineOptions(new Options()); + String[] subargs = new String[] {"-g group-test", "-t topic-test", "-f false"}; + final CommandLine commandLine = + ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); + cmd.execute(commandLine, options, null); + } +} From e0406b2aa42126cf7ff9067ae7bbd985652dfb82 Mon Sep 17 00:00:00 2001 From: zhangjidi2016 Date: Fri, 23 Jul 2021 14:42:28 +0800 Subject: [PATCH 004/141] [ISSUE #3173]Isolate Broker logs when multiple Broker services are deployed on the same machine --- .../apache/rocketmq/broker/BrokerStartup.java | 1 + distribution/conf/logback_broker.xml | 48 +++++++++---------- 2 files changed, 25 insertions(+), 24 deletions(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java index 960b848461d..96e43d1126f 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java @@ -187,6 +187,7 @@ public static BrokerController createBrokerController(String[] args) { JoranConfigurator configurator = new JoranConfigurator(); configurator.setContext(lc); lc.reset(); + System.setProperty("brokerLogDir", brokerConfig.getBrokerName() + "_" + brokerConfig.getBrokerId()); configurator.doConfigure(brokerConfig.getRocketmqHome() + "/conf/logback_broker.xml"); if (commandLine.hasOption('p')) { diff --git a/distribution/conf/logback_broker.xml b/distribution/conf/logback_broker.xml index 9d1a6b17618..8113ac9c283 100644 --- a/distribution/conf/logback_broker.xml +++ b/distribution/conf/logback_broker.xml @@ -19,10 +19,10 @@ - ${user.home}/logs/rocketmqlogs/broker_default.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/broker_default.log true - ${user.home}/logs/rocketmqlogs/otherdays/broker_default.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/broker_default.%i.log.gz 1 10 @@ -37,10 +37,10 @@ - ${user.home}/logs/rocketmqlogs/broker.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/broker.log true - ${user.home}/logs/rocketmqlogs/otherdays/broker.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/broker.%i.log.gz 1 20 @@ -58,10 +58,10 @@ - ${user.home}/logs/rocketmqlogs/protection.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/protection.log true - ${user.home}/logs/rocketmqlogs/otherdays/protection.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/protection.%i.log.gz 1 10 @@ -79,10 +79,10 @@ - ${user.home}/logs/rocketmqlogs/watermark.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/watermark.log true - ${user.home}/logs/rocketmqlogs/otherdays/watermark.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/watermark.%i.log.gz 1 10 @@ -100,10 +100,10 @@ - ${user.home}/logs/rocketmqlogs/store.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/store.log true - ${user.home}/logs/rocketmqlogs/otherdays/store.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/store.%i.log.gz 1 10 @@ -121,10 +121,10 @@ - ${user.home}/logs/rocketmqlogs/remoting.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/remoting.log true - ${user.home}/logs/rocketmqlogs/otherdays/remoting.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/remoting.%i.log.gz 1 10 @@ -142,10 +142,10 @@ - ${user.home}/logs/rocketmqlogs/storeerror.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/storeerror.log true - ${user.home}/logs/rocketmqlogs/otherdays/storeerror.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/storeerror.%i.log.gz 1 10 @@ -164,10 +164,10 @@ - ${user.home}/logs/rocketmqlogs/transaction.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/transaction.log true - ${user.home}/logs/rocketmqlogs/otherdays/transaction.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/transaction.%i.log.gz 1 10 @@ -185,10 +185,10 @@ - ${user.home}/logs/rocketmqlogs/lock.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/lock.log true - ${user.home}/logs/rocketmqlogs/otherdays/lock.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/lock.%i.log.gz 1 5 @@ -206,10 +206,10 @@ - ${user.home}/logs/rocketmqlogs/filter.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/filter.log true - ${user.home}/logs/rocketmqlogs/otherdays/filter.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/filter.%i.log.gz 1 10 @@ -227,10 +227,10 @@ - ${user.home}/logs/rocketmqlogs/stats.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/stats.log true - ${user.home}/logs/rocketmqlogs/otherdays/stats.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/stats.%i.log.gz 1 5 @@ -245,10 +245,10 @@ - ${user.home}/logs/rocketmqlogs/commercial.log + ${user.home}/logs/rocketmqlogs/${brokerLogDir}/commercial.log true - ${user.home}/logs/rocketmqlogs/otherdays/commercial.%i.log.gz + ${user.home}/logs/rocketmqlogs/otherdays/${brokerLogDir}/commercial.%i.log.gz 1 10 From 47175d88c60738157ae57fa5b2fde2563180ca34 Mon Sep 17 00:00:00 2001 From: zhangjidi2016 Date: Sun, 25 Jul 2021 16:52:23 +0800 Subject: [PATCH 005/141] use dLegerSelfId instead of brokerId in dleger mode --- .../main/java/org/apache/rocketmq/broker/BrokerStartup.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java index 96e43d1126f..84243c907b0 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java @@ -188,6 +188,9 @@ public static BrokerController createBrokerController(String[] args) { configurator.setContext(lc); lc.reset(); System.setProperty("brokerLogDir", brokerConfig.getBrokerName() + "_" + brokerConfig.getBrokerId()); + if (messageStoreConfig.isEnableDLegerCommitLog()) { + System.setProperty("brokerLogDir", brokerConfig.getBrokerName() + "_" + messageStoreConfig.getdLegerSelfId()); + } configurator.doConfigure(brokerConfig.getRocketmqHome() + "/conf/logback_broker.xml"); if (commandLine.hasOption('p')) { From 764cfba296f4894a7021a3bd8be222bc1ab59cdf Mon Sep 17 00:00:00 2001 From: zhangjidi2016 Date: Tue, 3 Aug 2021 10:02:03 +0800 Subject: [PATCH 006/141] retrigger ci From c98d5a80462085dcd0f9e362a93bd811d0fefd95 Mon Sep 17 00:00:00 2001 From: slievrly Date: Wed, 4 Aug 2021 14:27:53 +0800 Subject: [PATCH 007/141] optimize namesrv default address docs --- docs/cn/best_practice.md | 2 +- docs/en/Configuration_Client.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/cn/best_practice.md b/docs/cn/best_practice.md index b24c6eca57b..130fb3baefc 100755 --- a/docs/cn/best_practice.md +++ b/docs/cn/best_practice.md @@ -237,7 +237,7 @@ export NAMESRV_ADDR=192.168.0.1:9876;192.168.0.2:9876 ``` 客户端默认每隔2分钟访问一次这个HTTP服务器,并更新本地的Name Server地址。URL已经在代码中硬编码,可通过修改/etc/hosts文件来改变要访问的服务器,例如在/etc/hosts增加如下配置: ```text -10.232.22.67 jmenv.taobao.net +10.232.22.67 jmenv.tbsite.net ``` 推荐使用HTTP静态服务器寻址方式,好处是客户端部署简单,且Name Server集群可以热升级。 diff --git a/docs/en/Configuration_Client.md b/docs/en/Configuration_Client.md index dedb4240b7c..5f25fa07ffa 100644 --- a/docs/en/Configuration_Client.md +++ b/docs/en/Configuration_Client.md @@ -32,7 +32,7 @@ After client started, it will access the http static server address, as: Date: Mon, 16 Aug 2021 16:54:08 +0800 Subject: [PATCH 008/141] add defaultRequestProcessor test --- .../DefaultRequestProcessorTest.java | 149 +++++++++++++++++- 1 file changed, 148 insertions(+), 1 deletion(-) diff --git a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java index d4a2f66f99f..22cf978fe49 100644 --- a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java +++ b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessorTest.java @@ -235,6 +235,153 @@ public void testProcessRequest_UnregisterBroker() throws RemotingCommandExceptio assertThat((Map) brokerAddrTable.get(routes)).isNotEmpty(); } + @Test + public void testGetRouteInfoByTopic() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC); + RemotingCommand remotingCommandSuccess = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommandSuccess.getCode()).isEqualTo(ResponseCode.SUCCESS); + request.getExtFields().put("topic", "test"); + RemotingCommand remotingCommandNoTopicRouteInfo = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommandNoTopicRouteInfo.getCode()).isEqualTo(ResponseCode.TOPIC_NOT_EXIST); + } + + @Test + public void testGetBrokerClusterInfo() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_BROKER_CLUSTER_INFO); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testWipeWritePermOfBroker() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.WIPE_WRITE_PERM_OF_BROKER); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetAllTopicListFromNameserver() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_ALL_TOPIC_LIST_FROM_NAMESERVER); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testDeleteTopicInNamesrv() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.DELETE_TOPIC_IN_NAMESRV); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetKVListByNamespace() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_KVLIST_BY_NAMESPACE); + request.addExtField("namespace", "default-namespace-1"); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.QUERY_NOT_FOUND); + namesrvController.getKvConfigManager().putKVConfig("default-namespace-1", "key", "value"); + RemotingCommand remotingCommandSuccess = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommandSuccess.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetTopicsByCluster() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_TOPICS_BY_CLUSTER); + request.addExtField("cluster", "default-cluster"); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetSystemTopicListFromNs() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_SYSTEM_TOPIC_LIST_FROM_NS); + request.addExtField("cluster", "default-cluster"); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetUnitTopicList() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_UNIT_TOPIC_LIST); + request.addExtField("cluster", "default-cluster"); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetHasUnitSubTopicList() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_HAS_UNIT_SUB_TOPIC_LIST); + request.addExtField("cluster", "default-cluster"); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetHasUnitSubUnUnitTopicList() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_HAS_UNIT_SUB_UNUNIT_TOPIC_LIST); + request.addExtField("cluster", "default-cluster"); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testUpdateConfig() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.UPDATE_NAMESRV_CONFIG); + request.addExtField("cluster", "default-cluster"); + Map propertiesMap = new HashMap<>(); + propertiesMap.put("key", "value"); + request.setBody(propertiesMap.toString().getBytes()); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetConfig() throws RemotingCommandException { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.channel()).thenReturn(null); + RemotingCommand request = getRemotingCommand(RequestCode.GET_NAMESRV_CONFIG); + request.addExtField("cluster", "default-cluster"); + RemotingCommand remotingCommand = defaultRequestProcessor.processRequest(ctx, request); + assertThat(remotingCommand.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + private RemotingCommand getRemotingCommand(int code) { + RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader(); + header.setBrokerName("broker"); + RemotingCommand request = RemotingCommand.createRequestCommand(code, header); + request.addExtField("brokerName", "broker"); + request.addExtField("brokerAddr", "10.10.1.1"); + request.addExtField("clusterName", "cluster"); + request.addExtField("haServerAddr", "10.10.2.1"); + request.addExtField("brokerId", "2333"); + request.addExtField("topic", "unit-test"); + return request; + } + private static RemotingCommand genSampleRegisterCmd(boolean reg) { RegisterBrokerRequestHeader header = new RegisterBrokerRequestHeader(); header.setBrokerName("broker"); @@ -268,7 +415,7 @@ private void registerRouteInfoManager() { topicConfigConcurrentHashMap.put("unit-test", topicConfig); topicConfigSerializeWrapper.setTopicConfigTable(topicConfigConcurrentHashMap); Channel channel = mock(Channel.class); - RegisterBrokerResult registerBrokerResult = routeInfoManager.registerBroker("default-cluster", "127.0.0.1:10911", "default-broker", 1234, "127.0.0.1:1001", + RegisterBrokerResult registerBrokerResult = routeInfoManager.registerBroker("default-cluster", "127.0.0.1:10911", "default-broker", 0, "127.0.0.1:1001", topicConfigSerializeWrapper, new ArrayList(), channel); } From fc5180e6b11a2c5abe7f1c181b65dac8eb032352 Mon Sep 17 00:00:00 2001 From: zhangjidi Date: Wed, 15 Sep 2021 16:33:56 +0800 Subject: [PATCH 009/141] isolate broker's log can be configured --- .../java/org/apache/rocketmq/broker/BrokerStartup.java | 7 +++++-- .../java/org/apache/rocketmq/common/BrokerConfig.java | 10 ++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java index 84243c907b0..bb2eb9135a7 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java @@ -187,8 +187,11 @@ public static BrokerController createBrokerController(String[] args) { JoranConfigurator configurator = new JoranConfigurator(); configurator.setContext(lc); lc.reset(); - System.setProperty("brokerLogDir", brokerConfig.getBrokerName() + "_" + brokerConfig.getBrokerId()); - if (messageStoreConfig.isEnableDLegerCommitLog()) { + System.setProperty("brokerLogDir", ""); + if (brokerConfig.isIsolateLogEnable()) { + System.setProperty("brokerLogDir", brokerConfig.getBrokerName() + "_" + brokerConfig.getBrokerId()); + } + if (brokerConfig.isIsolateLogEnable() && messageStoreConfig.isEnableDLegerCommitLog()) { System.setProperty("brokerLogDir", brokerConfig.getBrokerName() + "_" + messageStoreConfig.getdLegerSelfId()); } configurator.doConfigure(brokerConfig.getRocketmqHome() + "/conf/logback_broker.xml"); 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 f710cdb4030..a6169d74d11 100644 --- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java @@ -187,6 +187,8 @@ public class BrokerConfig { private boolean autoDeleteUnusedStats = false; + private boolean isolateLogEnable = false; + public static String localHostName() { try { return InetAddress.getLocalHost().getHostName(); @@ -804,4 +806,12 @@ public boolean isAutoDeleteUnusedStats() { public void setAutoDeleteUnusedStats(boolean autoDeleteUnusedStats) { this.autoDeleteUnusedStats = autoDeleteUnusedStats; } + + public boolean isIsolateLogEnable() { + return isolateLogEnable; + } + + public void setIsolateLogEnable(boolean isolateLogEnable) { + this.isolateLogEnable = isolateLogEnable; + } } From 6b8a2b2fce91d6486b477adb2cb762a3e119c9f4 Mon Sep 17 00:00:00 2001 From: alitor <1069919773@qq.com> Date: Tue, 26 Oct 2021 13:44:23 +0800 Subject: [PATCH 010/141] trivial cleanups (#3403) --- .../broker/processor/AbstractSendMessageProcessor.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java index 9d26e9982b9..3303d70e4ad 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java @@ -77,8 +77,7 @@ protected SendMessageContext buildMsgContext(ChannelHandlerContext ctx, return null; } String namespace = NamespaceUtil.getNamespaceFromResource(requestHeader.getTopic()); - SendMessageContext mqtraceContext; - mqtraceContext = new SendMessageContext(); + SendMessageContext mqtraceContext = new SendMessageContext(); mqtraceContext.setProducerGroup(requestHeader.getProducerGroup()); mqtraceContext.setNamespace(namespace); mqtraceContext.setTopic(requestHeader.getTopic()); From 78443c3a7cf1db54b1e5f1c9b2361b89a632147d Mon Sep 17 00:00:00 2001 From: panzhi Date: Tue, 26 Oct 2021 13:58:29 +0800 Subject: [PATCH 011/141] =?UTF-8?q?[ISSUE=203420]rocketmq=5Fclient.log=20w?= =?UTF-8?q?ill=20not=20record=20the=20asynchronous=20sending=20failure=20e?= =?UTF-8?q?xception=EF=BC=8Cwill=20only=20throw=20out=20the=20reason=20for?= =?UTF-8?q?=20the=20final=20time=20failure=20to=20the=20business=20(#3421)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index b7694295359..b189c82370c 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -605,8 +605,8 @@ private void onExceptionImpl(final String brokerName, retryBrokerName = mqChosen.getBrokerName(); } String addr = instance.findBrokerAddressInPublish(retryBrokerName); - log.info("async send msg by retry {} times. topic={}, brokerAddr={}, brokerName={}", tmp, msg.getTopic(), addr, - retryBrokerName); + log.warn(String.format("async send msg by retry {} times. topic={}, brokerAddr={}, brokerName={}", tmp, msg.getTopic(), addr, + retryBrokerName), e); try { request.setOpaque(RemotingCommand.createNewRequestId()); sendMessageAsync(addr, retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, From d2776fca95a5f154c7e716522ebb510b5dff7e50 Mon Sep 17 00:00:00 2001 From: meateggmilk <87265072+meateggmilk@users.noreply.github.com> Date: Tue, 26 Oct 2021 17:08:36 +0800 Subject: [PATCH 012/141] [ISSUE #3370] group same Exceptions Co-authored-by: shizhili --- .../rocketmq/client/impl/MQClientAPIImpl.java | 9 +------- .../client/impl/factory/MQClientInstance.java | 6 +---- .../impl/producer/DefaultMQProducerImpl.java | 23 ++----------------- 3 files changed, 4 insertions(+), 34 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index b189c82370c..8d127706004 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -611,14 +611,7 @@ private void onExceptionImpl(final String brokerName, request.setOpaque(RemotingCommand.createNewRequestId()); sendMessageAsync(addr, retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, timesTotal, curTimes, context, producer); - } catch (InterruptedException e1) { - onExceptionImpl(retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, timesTotal, curTimes, e1, - context, false, producer); - } catch (RemotingConnectException e1) { - producer.updateFaultItem(brokerName, 3000, true); - onExceptionImpl(retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, timesTotal, curTimes, e1, - context, true, producer); - } catch (RemotingTooMuchRequestException e1) { + } catch (InterruptedException | RemotingTooMuchRequestException e1) { onExceptionImpl(retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, timesTotal, curTimes, e1, context, false, producer); } catch (RemotingException e1) { diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java index e897d4959f7..9651943c0ba 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java @@ -896,11 +896,7 @@ private void unregisterClient(final String producerGroup, final String consumerG try { this.mQClientAPIImpl.unregisterClient(addr, this.clientId, producerGroup, consumerGroup, clientConfig.getMqClientApiTimeout()); log.info("unregister client[Producer: {} Consumer: {}] from broker[{} {} {}] success", producerGroup, consumerGroup, brokerName, entry1.getKey(), addr); - } catch (RemotingException e) { - log.error("unregister client exception from broker: " + addr, e); - } catch (InterruptedException e) { - log.error("unregister client exception from broker: " + addr, e); - } catch (MQBrokerException e) { + } catch (RemotingException | InterruptedException | MQBrokerException e) { log.error("unregister client exception from broker: " + addr, e); } } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java index bdc103f1328..2a784b55cbd 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java @@ -622,14 +622,7 @@ private SendResult sendDefaultImpl( default: break; } - } catch (RemotingException e) { - endTimestamp = System.currentTimeMillis(); - this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true); - log.warn(String.format("sendKernelImpl exception, resend at once, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq), e); - log.warn(msg.toString()); - exception = e; - continue; - } catch (MQClientException e) { + } catch (RemotingException | MQClientException e) { endTimestamp = System.currentTimeMillis(); this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true); log.warn(String.format("sendKernelImpl exception, resend at once, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq), e); @@ -888,19 +881,7 @@ private SendResult sendKernelImpl(final Message msg, } return sendResult; - } catch (RemotingException e) { - if (this.hasSendMessageHook()) { - context.setException(e); - this.executeSendMessageHookAfter(context); - } - throw e; - } catch (MQBrokerException e) { - if (this.hasSendMessageHook()) { - context.setException(e); - this.executeSendMessageHookAfter(context); - } - throw e; - } catch (InterruptedException e) { + } catch (RemotingException | MQBrokerException | InterruptedException e) { if (this.hasSendMessageHook()) { context.setException(e); this.executeSendMessageHookAfter(context); From 1657475997733133fea6a1be14b8a22cc5730698 Mon Sep 17 00:00:00 2001 From: haozhijie9527 <47975723+haozhijie9527@users.noreply.github.com> Date: Wed, 27 Oct 2021 14:59:12 +0800 Subject: [PATCH 013/141] [ISSUE#3430] fix the problem that setting parameter mqClientApiTimeout doesn't take effect (#3431) * [maven-release-plugin] prepare release rocketmq-all-4.9.2 * [maven-release-plugin] prepare for next development iteration * fix the problem that setting parameter mqClientApiTimeout doesn't take effect Co-authored-by: tigerlee Co-authored-by: haozhijie --- .../src/main/java/org/apache/rocketmq/client/ClientConfig.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java index b2c043ee7d9..8d7f5a136b7 100644 --- a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java +++ b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java @@ -158,6 +158,7 @@ public void resetClientConfig(final ClientConfig cc) { this.useTLS = cc.useTLS; this.namespace = cc.namespace; this.language = cc.language; + this.mqClientApiTimeout = cc.mqClientApiTimeout; } public ClientConfig cloneClientConfig() { @@ -176,6 +177,7 @@ public ClientConfig cloneClientConfig() { cc.useTLS = useTLS; cc.namespace = namespace; cc.language = language; + cc.mqClientApiTimeout = mqClientApiTimeout; return cc; } From 2385f0446832d4e799c16d280a88cafc548fab39 Mon Sep 17 00:00:00 2001 From: shendongsd Date: Wed, 3 Nov 2021 14:12:29 +0800 Subject: [PATCH 014/141] [ISSUE #3381] Fix bug of when role change but not register again (#3442) --- .../java/org/apache/rocketmq/broker/BrokerController.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 194f2850fa5..3e146b754a9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -1182,7 +1182,7 @@ public void changeToSlave(int brokerId) { handleSlaveSynchronize(BrokerRole.SLAVE); try { - this.registerBrokerAll(true, true, brokerConfig.isForceRegister()); + this.registerBrokerAll(true, true, true); } catch (Throwable ignored) { } @@ -1219,7 +1219,7 @@ public void changeToMaster(BrokerRole role) { messageStoreConfig.setBrokerRole(role); try { - this.registerBrokerAll(true, true, brokerConfig.isForceRegister()); + this.registerBrokerAll(true, true, true); } catch (Throwable ignored) { } From fe72057abc21cbdc17b46548971a3101defd172c Mon Sep 17 00:00:00 2001 From: lushilin <897401919@qq.com> Date: Thu, 28 Oct 2021 08:51:26 +0800 Subject: [PATCH 015/141] fix(broker): can not clear topic unitSubFlag (#3429) Co-authored-by: lushilin --- .../apache/rocketmq/broker/topic/TopicConfigManager.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 1f2bb4d8ba4..e09080afaea 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 @@ -320,7 +320,7 @@ public void updateTopicUnitFlag(final String topic, final boolean unit) { topicConfig.setTopicSysFlag(TopicSysFlag.clearUnitFlag(oldTopicSysFlag)); } - log.info("update topic sys flag. oldTopicSysFlag={}, newTopicSysFlag", oldTopicSysFlag, + log.info("update topic sys flag. oldTopicSysFlag={}, newTopicSysFlag={}", oldTopicSysFlag, topicConfig.getTopicSysFlag()); this.topicConfigTable.put(topic, topicConfig); @@ -338,9 +338,11 @@ public void updateTopicUnitSubFlag(final String topic, final boolean hasUnitSub) int oldTopicSysFlag = topicConfig.getTopicSysFlag(); if (hasUnitSub) { topicConfig.setTopicSysFlag(TopicSysFlag.setUnitSubFlag(oldTopicSysFlag)); + } else { + topicConfig.setTopicSysFlag(TopicSysFlag.clearUnitSubFlag(oldTopicSysFlag)); } - log.info("update topic sys flag. oldTopicSysFlag={}, newTopicSysFlag", oldTopicSysFlag, + log.info("update topic sys flag. oldTopicSysFlag={}, newTopicSysFlag={}", oldTopicSysFlag, topicConfig.getTopicSysFlag()); this.topicConfigTable.put(topic, topicConfig); From c6aeb782e59587c610ff8293462f028993fb8206 Mon Sep 17 00:00:00 2001 From: zhichen Date: Thu, 4 Nov 2021 17:18:05 +0800 Subject: [PATCH 016/141] [ISSUE #3459] Fix the problem of Put messages to commitLog always failure after encountering any exception (#3460) --- .../java/org/apache/rocketmq/store/CommitLog.java | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 3a89dc84995..36db2f5146b 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -658,7 +658,6 @@ public CompletableFuture asyncPutMessage(final MessageExtBroke } if (null == mappedFile) { log.error("create mapped file1 error, topic: " + msg.getTopic() + " clientAddr: " + msg.getBornHostString()); - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, null)); } @@ -673,26 +672,22 @@ public CompletableFuture asyncPutMessage(final MessageExtBroke if (null == mappedFile) { // XXX: warn and notify me log.error("create mapped file2 error, topic: " + msg.getTopic() + " clientAddr: " + msg.getBornHostString()); - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, result)); } result = mappedFile.appendMessage(msg, this.appendMessageCallback, putMessageContext); break; case MESSAGE_SIZE_EXCEEDED: case PROPERTIES_SIZE_EXCEEDED: - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, result)); case UNKNOWN_ERROR: - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result)); default: - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result)); } elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginLockTimestamp; - beginTimeInLock = 0; } finally { + beginTimeInLock = 0; putMessageLock.unlock(); } @@ -777,7 +772,6 @@ public CompletableFuture asyncPutMessages(final MessageExtBatc } if (null == mappedFile) { log.error("Create mapped file1 error, topic: {} clientAddr: {}", messageExtBatch.getTopic(), messageExtBatch.getBornHostString()); - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, null)); } @@ -792,24 +786,21 @@ public CompletableFuture asyncPutMessages(final MessageExtBatc if (null == mappedFile) { // XXX: warn and notify me log.error("Create mapped file2 error, topic: {} clientAddr: {}", messageExtBatch.getTopic(), messageExtBatch.getBornHostString()); - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.CREATE_MAPEDFILE_FAILED, result)); } result = mappedFile.appendMessages(messageExtBatch, this.appendMessageCallback, putMessageContext); break; case MESSAGE_SIZE_EXCEEDED: case PROPERTIES_SIZE_EXCEEDED: - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, result)); case UNKNOWN_ERROR: default: - beginTimeInLock = 0; return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result)); } elapsedTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginLockTimestamp; - beginTimeInLock = 0; } finally { + beginTimeInLock = 0; putMessageLock.unlock(); } From a51598dea55fb88ee60ab0194d0ff06139d83aa2 Mon Sep 17 00:00:00 2001 From: BingCoke <81607010+BingCoke@users.noreply.github.com> Date: Mon, 8 Nov 2021 08:58:50 +0800 Subject: [PATCH 017/141] Validators.checkMessage() is used twice. (#3435) * fix(broker): can not clear topic unitSubFlag (#3429) Co-authored-by: lushilin * Validators.checkMessage() is used twice.The other is used in sendDefaultImpl Co-authored-by: lushilin <897401919@qq.com> Co-authored-by: lushilin --- .../org/apache/rocketmq/client/producer/DefaultMQProducer.java | 1 - 1 file changed, 1 deletion(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java b/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java index 1af416b48c7..9f91b411e88 100644 --- a/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java +++ b/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java @@ -331,7 +331,6 @@ public List fetchPublishMessageQueues(String topic) throws MQClien @Override public SendResult send( Message msg) throws MQClientException, RemotingException, MQBrokerException, InterruptedException { - Validators.checkMessage(msg, this); msg.setTopic(withNamespace(msg.getTopic())); return this.defaultMQProducerImpl.send(msg); } From 7b05208618cd718b4bde86f1b657784754d97691 Mon Sep 17 00:00:00 2001 From: JiangHaiting Date: Mon, 8 Nov 2021 10:27:01 +0800 Subject: [PATCH 018/141] [ISSUE #3467] Fail fast on loading files with error mappedFileSize Co-authored-by: Jiang Haiting --- .../main/java/org/apache/rocketmq/store/MappedFileQueue.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java index 1f25e61225a..ddfe65bb9f1 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java @@ -162,8 +162,8 @@ public boolean doLoad(List files) { for (File file : files) { if (file.length() != this.mappedFileSize) { log.warn(file + "\t" + file.length() - + " length not matched message store config value, ignore it"); - return true; + + " length not matched message store config value, please check it manually"); + return false; } try { From 2d9a7e1215059d5d650ba16dc6483964536de411 Mon Sep 17 00:00:00 2001 From: Git_Yang Date: Wed, 10 Nov 2021 19:39:25 +0800 Subject: [PATCH 019/141] [ISSUE #3463] Fix the issue of commitLog path error (#3464) --- .../broker/processor/SendMessageProcessor.java | 10 +++++++++- .../org/apache/rocketmq/store/DefaultMessageStore.java | 5 ++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java index 7f861e7ce0c..b31c71e06d9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java @@ -54,7 +54,9 @@ import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; import org.apache.rocketmq.remoting.netty.RemotingResponseCallback; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.config.StorePathConfigHelper; @@ -644,7 +646,13 @@ public SocketAddress getStoreHost() { private String diskUtil() { double physicRatio = 100; - String storePath = this.brokerController.getMessageStoreConfig().getStorePathCommitLog(); + String storePath; + MessageStore messageStore = this.brokerController.getMessageStore(); + if (messageStore instanceof DefaultMessageStore) { + storePath = ((DefaultMessageStore) messageStore).getStorePathPhysic(); + } else { + storePath = this.brokerController.getMessageStoreConfig().getStorePathCommitLog(); + } String[] paths = storePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER); for (String storePathPhysic : paths) { physicRatio = Math.min(physicRatio, UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic)); diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java index 09e153494df..ed882aa681f 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -1672,7 +1672,7 @@ private boolean isSpaceToDelete() { cleanImmediately = false; { - String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog(); + String commitLogStorePath = DefaultMessageStore.this.getStorePathPhysic(); String[] storePaths = commitLogStorePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER); Set fullStorePath = new HashSet<>(); double minPhysicRatio = 100; @@ -1714,8 +1714,7 @@ private boolean isSpaceToDelete() { } { - String storePathLogics = StorePathConfigHelper - .getStorePathConsumeQueue(DefaultMessageStore.this.getMessageStoreConfig().getStorePathRootDir()); + String storePathLogics = DefaultMessageStore.this.getStorePathLogic(); double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics); if (logicsRatio > diskSpaceWarningLevelRatio) { boolean diskok = DefaultMessageStore.this.runningFlags.getAndMakeDiskFull(); From b35f3e4459e8cf32116bb15133aaa27d5ff63cfb Mon Sep 17 00:00:00 2001 From: Git_Yang Date: Wed, 10 Nov 2021 19:43:42 +0800 Subject: [PATCH 020/141] [ISSUE #3286] Replace Timer with ScheduledExecutorService (#3287) Signed-off-by: zhangyang21 --- .../schedule/ScheduleMessageService.java | 48 ++++++++++--------- 1 file changed, 26 insertions(+), 22 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java index e0e7b9565b6..c45287ff659 100644 --- a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java +++ b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java @@ -19,13 +19,15 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.Timer; -import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.common.ConfigManager; import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.TopicFilterType; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.topic.TopicValidator; @@ -60,7 +62,8 @@ public class ScheduleMessageService extends ConfigManager { new ConcurrentHashMap(32); private final DefaultMessageStore defaultMessageStore; private final AtomicBoolean started = new AtomicBoolean(false); - private Timer timer; + private ScheduledExecutorService deliverExecutorService; + private int deliverThreadPoolNums = Runtime.getRuntime().availableProcessors(); private MessageStore writeMessageStore; private int maxDelayLevel; @@ -113,7 +116,7 @@ public long computeDeliverTimestamp(final int delayLevel, final long storeTimest public void start() { if (started.compareAndSet(false, true)) { super.load(); - this.timer = new Timer("ScheduleMessageTimerThread", true); + this.deliverExecutorService = new ScheduledThreadPoolExecutor(deliverThreadPoolNums, new ThreadFactoryImpl("ScheduleMessageTimerThread_")); for (Map.Entry entry : this.delayLevelTable.entrySet()) { Integer level = entry.getKey(); Long timeDelay = entry.getValue(); @@ -123,11 +126,11 @@ public void start() { } if (timeDelay != null) { - this.timer.schedule(new DeliverDelayedMessageTimerTask(level, offset), FIRST_DELAY_TIME); + this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask(level, offset), FIRST_DELAY_TIME, TimeUnit.MILLISECONDS); } } - this.timer.scheduleAtFixedRate(new TimerTask() { + this.deliverExecutorService.scheduleAtFixedRate(new Runnable() { @Override public void run() { @@ -139,16 +142,14 @@ public void run() { log.error("scheduleAtFixedRate flush exception", e); } } - }, 10000, this.defaultMessageStore.getMessageStoreConfig().getFlushDelayOffsetInterval()); + }, 10000, this.defaultMessageStore.getMessageStoreConfig().getFlushDelayOffsetInterval(), TimeUnit.MILLISECONDS); } } public void shutdown() { - if (this.started.compareAndSet(true, false)) { - if (null != this.timer) - this.timer.cancel(); + if (this.started.compareAndSet(true, false) && null != this.deliverExecutorService) { + this.deliverExecutorService.shutdownNow(); } - } public boolean isStarted() { @@ -159,10 +160,12 @@ public int getMaxDelayLevel() { return maxDelayLevel; } + @Override public String encode() { return this.encode(false); } + @Override public boolean load() { boolean result = super.load(); result = result && this.parseDelayLevel(); @@ -223,6 +226,7 @@ public void decode(String jsonString) { } } + @Override public String encode(final boolean prettyFormat) { DelayOffsetSerializeWrapper delayOffsetSerializeWrapper = new DelayOffsetSerializeWrapper(); delayOffsetSerializeWrapper.setOffsetTable(this.offsetTable); @@ -261,7 +265,7 @@ public boolean parseDelayLevel() { return true; } - class DeliverDelayedMessageTimerTask extends TimerTask { + class DeliverDelayedMessageTimerTask implements Runnable { private final int delayLevel; private final long offset; @@ -279,8 +283,8 @@ public void run() { } catch (Exception e) { // XXX: warn and notify me log.error("ScheduleMessageService, executeOnTimeup exception", e); - ScheduleMessageService.this.timer.schedule(new DeliverDelayedMessageTimerTask( - this.delayLevel, this.offset), DELAY_FOR_A_PERIOD); + ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask( + this.delayLevel, this.offset), DELAY_FOR_A_PERIOD, TimeUnit.MILLISECONDS); } } @@ -372,9 +376,9 @@ public void executeOnTimeup() { log.error( "ScheduleMessageService, a message time up, but reput it failed, topic: {} msgId {}", msgExt.getTopic(), msgExt.getMsgId()); - ScheduleMessageService.this.timer.schedule( + ScheduleMessageService.this.deliverExecutorService.schedule( new DeliverDelayedMessageTimerTask(this.delayLevel, - nextOffset), DELAY_FOR_A_PERIOD); + nextOffset), DELAY_FOR_A_PERIOD, TimeUnit.MILLISECONDS); ScheduleMessageService.this.updateOffset(this.delayLevel, nextOffset); return; @@ -388,17 +392,17 @@ public void executeOnTimeup() { } } } else { - ScheduleMessageService.this.timer.schedule( + ScheduleMessageService.this.deliverExecutorService.schedule( new DeliverDelayedMessageTimerTask(this.delayLevel, nextOffset), - countdown); + countdown, TimeUnit.MILLISECONDS); ScheduleMessageService.this.updateOffset(this.delayLevel, nextOffset); return; } } // end of for nextOffset = offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE); - ScheduleMessageService.this.timer.schedule(new DeliverDelayedMessageTimerTask( - this.delayLevel, nextOffset), DELAY_FOR_A_WHILE); + ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask( + this.delayLevel, nextOffset), DELAY_FOR_A_WHILE, TimeUnit.MILLISECONDS); ScheduleMessageService.this.updateOffset(this.delayLevel, nextOffset); return; } finally { @@ -424,8 +428,8 @@ public void executeOnTimeup() { } } // end of if (cq != null) - ScheduleMessageService.this.timer.schedule(new DeliverDelayedMessageTimerTask(this.delayLevel, - failScheduleOffset), DELAY_FOR_A_WHILE); + ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask(this.delayLevel, + failScheduleOffset), DELAY_FOR_A_WHILE, TimeUnit.MILLISECONDS); } private MessageExtBrokerInner messageTimeup(MessageExt msgExt) { From 4bb99e656b1acc99d3a953f84ae0abcb74737af5 Mon Sep 17 00:00:00 2001 From: zhangyang Date: Wed, 17 Nov 2021 13:56:46 +0800 Subject: [PATCH 021/141] [ISSUE #3487] Benchmark supports custom AK/SK Signed-off-by: zhangyang21 --- .../rocketmq/example/benchmark/AclClient.java | 12 ++++++++---- .../rocketmq/example/benchmark/Consumer.java | 15 ++++++++++++++- .../rocketmq/example/benchmark/Producer.java | 15 ++++++++++++++- .../example/benchmark/TransactionProducer.java | 17 ++++++++++++++++- 4 files changed, 52 insertions(+), 7 deletions(-) diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/AclClient.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/AclClient.java index 04ef5d5ef6b..b3d6fb47e9b 100644 --- a/example/src/main/java/org/apache/rocketmq/example/benchmark/AclClient.java +++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/AclClient.java @@ -23,11 +23,15 @@ public class AclClient { - private static final String ACL_ACCESS_KEY = "rocketmq2"; + public static final String ACL_ACCESS_KEY = "rocketmq2"; - private static final String ACL_SECRET_KEY = "12345678"; + public static final String ACL_SECRET_KEY = "12345678"; - static RPCHook getAclRPCHook() { - return new AclClientRPCHook(new SessionCredentials(ACL_ACCESS_KEY,ACL_SECRET_KEY)); + public static RPCHook getAclRPCHook() { + return getAclRPCHook(ACL_ACCESS_KEY, ACL_SECRET_KEY); + } + + public static RPCHook getAclRPCHook(String ak, String sk) { + return new AclClientRPCHook(new SessionCredentials(ak, sk)); } } diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java index 7d26509d2bc..d08795d6b03 100644 --- a/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/Consumer.java @@ -123,7 +123,12 @@ public void run() { } }, 10000, 10000, TimeUnit.MILLISECONDS); - RPCHook rpcHook = aclEnable ? AclClient.getAclRPCHook() : null; + RPCHook rpcHook = null; + if (aclEnable) { + String ak = commandLine.hasOption("ak") ? String.valueOf(commandLine.getOptionValue("ak")) : AclClient.ACL_ACCESS_KEY; + String sk = commandLine.hasOption("sk") ? String.valueOf(commandLine.getOptionValue("sk")) : AclClient.ACL_SECRET_KEY; + rpcHook = AclClient.getAclRPCHook(ak, sk); + } DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(group, rpcHook, new AllocateMessageQueueAveragely(), msgTraceEnable, null); if (commandLine.hasOption('n')) { String ns = commandLine.getOptionValue('n'); @@ -218,6 +223,14 @@ public static Options buildCommandlineOptions(final Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("ak", "accessKey", true, "Acl access key, Default: 12345678"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("sk", "secretKey", true, "Acl secret key, Default: rocketmq2"); + opt.setRequired(false); + options.addOption(opt); + return options; } diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java index feb25cd910c..c32e00e97b8 100644 --- a/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java +++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/Producer.java @@ -126,7 +126,12 @@ public void run() { } }, 10000, 10000, TimeUnit.MILLISECONDS); - RPCHook rpcHook = aclEnable ? AclClient.getAclRPCHook() : null; + RPCHook rpcHook = null; + if (aclEnable) { + String ak = commandLine.hasOption("ak") ? String.valueOf(commandLine.getOptionValue("ak")) : AclClient.ACL_ACCESS_KEY; + String sk = commandLine.hasOption("sk") ? String.valueOf(commandLine.getOptionValue("sk")) : AclClient.ACL_SECRET_KEY; + rpcHook = AclClient.getAclRPCHook(ak, sk); + } final DefaultMQProducer producer = new DefaultMQProducer("benchmark_producer", rpcHook, msgTraceEnable, null); producer.setInstanceName(Long.toString(System.currentTimeMillis())); @@ -277,6 +282,14 @@ public static Options buildCommandlineOptions(final Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("ak", "accessKey", true, "Acl access key, Default: 12345678"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("sk", "secretKey", true, "Acl secret key, Default: rocketmq2"); + opt.setRequired(false); + options.addOption(opt); + opt = new Option("q", "messageQuantity", true, "Send message quantity, Default: 0, running forever"); opt.setRequired(false); options.addOption(opt); diff --git a/example/src/main/java/org/apache/rocketmq/example/benchmark/TransactionProducer.java b/example/src/main/java/org/apache/rocketmq/example/benchmark/TransactionProducer.java index 767a96b40bb..5e2f287e44b 100644 --- a/example/src/main/java/org/apache/rocketmq/example/benchmark/TransactionProducer.java +++ b/example/src/main/java/org/apache/rocketmq/example/benchmark/TransactionProducer.java @@ -31,6 +31,7 @@ import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.srvutil.ServerUtil; import java.io.UnsupportedEncodingException; @@ -128,11 +129,17 @@ public void run() { } }, 10000, 10000, TimeUnit.MILLISECONDS); + RPCHook rpcHook = null; + if (config.aclEnable) { + String ak = commandLine.hasOption("ak") ? String.valueOf(commandLine.getOptionValue("ak")) : AclClient.ACL_ACCESS_KEY; + String sk = commandLine.hasOption("sk") ? String.valueOf(commandLine.getOptionValue("sk")) : AclClient.ACL_SECRET_KEY; + rpcHook = AclClient.getAclRPCHook(ak, sk); + } final TransactionListener transactionCheckListener = new TransactionListenerImpl(statsBenchmark, config); final TransactionMQProducer producer = new TransactionMQProducer( null, "benchmark_transaction_producer", - config.aclEnable ? AclClient.getAclRPCHook() : null, + rpcHook, config.msgTraceEnable, null); producer.setInstanceName(Long.toString(System.currentTimeMillis())); @@ -268,6 +275,14 @@ public static Options buildCommandlineOptions(final Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("ak", "accessKey", true, "Acl access key, Default: 12345678"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("sk", "secretKey", true, "Acl secret key, Default: rocketmq2"); + opt.setRequired(false); + options.addOption(opt); + opt = new Option("m", "msgTraceEnable", true, "Message Trace Enable, Default: false"); opt.setRequired(false); options.addOption(opt); From 0c85c7b4d668f37aa0c3ff48987ad94f40d9265a Mon Sep 17 00:00:00 2001 From: dingshuangxi888 Date: Tue, 23 Nov 2021 21:06:45 +0800 Subject: [PATCH 022/141] [ISSUE #3527] Fix some request header setting errors and naming issues (#3530) --- .../org/apache/rocketmq/client/impl/MQClientAPIImpl.java | 9 +++++---- ...der.java => DeleteTopicFromNamesrvRequestHeader.java} | 2 +- .../namesrv/processor/DefaultRequestProcessor.java | 6 +++--- 3 files changed, 9 insertions(+), 8 deletions(-) rename common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/{DeleteTopicInNamesrvRequestHeader.java => DeleteTopicFromNamesrvRequestHeader.java} (94%) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index 8d127706004..89c998b7f4e 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -138,6 +138,7 @@ import org.apache.rocketmq.common.protocol.header.namesrv.AddWritePermOfBrokerRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.AddWritePermOfBrokerResponseHeader; import org.apache.rocketmq.common.protocol.header.namesrv.DeleteKVConfigRequestHeader; +import org.apache.rocketmq.common.protocol.header.namesrv.DeleteTopicFromNamesrvRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigResponseHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVListByNamespaceRequestHeader; @@ -1446,7 +1447,7 @@ public int addWritePermOfBroker(final String nameSrvAddr, String brokerName, fin } public void deleteTopicInBroker(final String addr, final String topic, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + throws RemotingException, InterruptedException, MQClientException { DeleteTopicRequestHeader requestHeader = new DeleteTopicRequestHeader(); requestHeader.setTopic(topic); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_BROKER, requestHeader); @@ -1466,8 +1467,8 @@ public void deleteTopicInBroker(final String addr, final String topic, final lon } public void deleteTopicInNameServer(final String addr, final String topic, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException, MQClientException { - DeleteTopicRequestHeader requestHeader = new DeleteTopicRequestHeader(); + throws RemotingException, InterruptedException, MQClientException { + DeleteTopicFromNamesrvRequestHeader requestHeader = new DeleteTopicFromNamesrvRequestHeader(); requestHeader.setTopic(topic); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_NAMESRV, requestHeader); @@ -1485,7 +1486,7 @@ public void deleteTopicInNameServer(final String addr, final String topic, final } public void deleteSubscriptionGroup(final String addr, final String groupName, final boolean removeOffset, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + throws RemotingException, InterruptedException, MQClientException { DeleteSubscriptionGroupRequestHeader requestHeader = new DeleteSubscriptionGroupRequestHeader(); requestHeader.setGroupName(groupName); requestHeader.setRemoveOffset(removeOffset); diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicInNamesrvRequestHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java similarity index 94% rename from common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicInNamesrvRequestHeader.java rename to common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java index ab768ee33c2..ed8e7a3970a 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicInNamesrvRequestHeader.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/namesrv/DeleteTopicFromNamesrvRequestHeader.java @@ -20,7 +20,7 @@ import org.apache.rocketmq.remoting.annotation.CFNotNull; import org.apache.rocketmq.remoting.exception.RemotingCommandException; -public class DeleteTopicInNamesrvRequestHeader implements CommandCustomHeader { +public class DeleteTopicFromNamesrvRequestHeader implements CommandCustomHeader { @CFNotNull private String topic; diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java index 98e96dfe7db..bde03489d36 100644 --- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java +++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java @@ -39,7 +39,7 @@ import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.common.protocol.header.GetTopicsByClusterRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.DeleteKVConfigRequestHeader; -import org.apache.rocketmq.common.protocol.header.namesrv.DeleteTopicInNamesrvRequestHeader; +import org.apache.rocketmq.common.protocol.header.namesrv.DeleteTopicFromNamesrvRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigRequestHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVConfigResponseHeader; import org.apache.rocketmq.common.protocol.header.namesrv.GetKVListByNamespaceRequestHeader; @@ -438,8 +438,8 @@ private RemotingCommand getAllTopicListFromNameserver(ChannelHandlerContext ctx, private RemotingCommand deleteTopicInNamesrv(ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException { final RemotingCommand response = RemotingCommand.createResponseCommand(null); - final DeleteTopicInNamesrvRequestHeader requestHeader = - (DeleteTopicInNamesrvRequestHeader) request.decodeCommandCustomHeader(DeleteTopicInNamesrvRequestHeader.class); + final DeleteTopicFromNamesrvRequestHeader requestHeader = + (DeleteTopicFromNamesrvRequestHeader) request.decodeCommandCustomHeader(DeleteTopicFromNamesrvRequestHeader.class); this.namesrvController.getRouteInfoManager().deleteTopic(requestHeader.getTopic()); From be6ded9049c97b69d1f037aa67446b07d54d9d79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Wed, 24 Nov 2021 10:43:37 +0800 Subject: [PATCH 023/141] [ISSUE #3528] The value of ChannelOption.SO_BACKLOG can not be hard-coded. (#3529) According to the official Linux documentation, the meaning of this value(ChannelOption.SO_BACKLOG) has changed since Linux 2.2. --- .../remoting/netty/NettyRemotingServer.java | 2 +- .../remoting/netty/NettyServerConfig.java | 9 +++++ .../remoting/netty/NettySystemConfig.java | 5 +++ .../remoting/netty/NettyServerConfigTest.java | 37 +++++++++++++++++++ 4 files changed, 52 insertions(+), 1 deletion(-) create mode 100644 remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyServerConfigTest.java diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java index f94d5ba9ca5..1586472e042 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java @@ -198,7 +198,7 @@ public Thread newThread(Runnable r) { ServerBootstrap childHandler = this.serverBootstrap.group(this.eventLoopGroupBoss, this.eventLoopGroupSelector) .channel(useEpoll() ? EpollServerSocketChannel.class : NioServerSocketChannel.class) - .option(ChannelOption.SO_BACKLOG, 1024) + .option(ChannelOption.SO_BACKLOG, nettyServerConfig.getServerSocketBacklog()) .option(ChannelOption.SO_REUSEADDR, true) .option(ChannelOption.SO_KEEPALIVE, false) .childOption(ChannelOption.TCP_NODELAY, true) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java index a5e2a232dd5..8708471642f 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java @@ -27,6 +27,7 @@ public class NettyServerConfig implements Cloneable { private int serverSocketSndBufSize = NettySystemConfig.socketSndbufSize; private int serverSocketRcvBufSize = NettySystemConfig.socketRcvbufSize; + private int serverSocketBacklog = NettySystemConfig.socketBacklog; private boolean serverPooledByteBufAllocatorEnable = true; /** @@ -110,6 +111,14 @@ public void setServerSocketRcvBufSize(int serverSocketRcvBufSize) { this.serverSocketRcvBufSize = serverSocketRcvBufSize; } + public int getServerSocketBacklog() { + return serverSocketBacklog; + } + + public void setServerSocketBacklog(int serverSocketBacklog) { + this.serverSocketBacklog = serverSocketBacklog; + } + public boolean isServerPooledByteBufAllocatorEnable() { return serverPooledByteBufAllocatorEnable; } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java index d4c3749a0f3..85f30f515bf 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java @@ -24,6 +24,8 @@ public class NettySystemConfig { "com.rocketmq.remoting.socket.sndbuf.size"; public static final String COM_ROCKETMQ_REMOTING_SOCKET_RCVBUF_SIZE = "com.rocketmq.remoting.socket.rcvbuf.size"; + public static final String COM_ROCKETMQ_REMOTING_SOCKET_BACKLOG = + "com.rocketmq.remoting.socket.backlog"; public static final String COM_ROCKETMQ_REMOTING_CLIENT_ASYNC_SEMAPHORE_VALUE = "com.rocketmq.remoting.clientAsyncSemaphoreValue"; public static final String COM_ROCKETMQ_REMOTING_CLIENT_ONEWAY_SEMAPHORE_VALUE = @@ -47,6 +49,8 @@ public class NettySystemConfig { Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_SNDBUF_SIZE, "65535")); public static int socketRcvbufSize = Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_RCVBUF_SIZE, "65535")); + public static int socketBacklog = + Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_BACKLOG, "1024")); public static int clientWorkerSize = Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_CLIENT_WORKER_SIZE, "4")); public static int connectTimeoutMillis = @@ -55,4 +59,5 @@ public class NettySystemConfig { Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_CLIENT_CHANNEL_MAX_IDLE_SECONDS, "120")); public static boolean clientCloseSocketIfTimeout = Boolean.parseBoolean(System.getProperty(COM_ROCKETMQ_REMOTING_CLIENT_CLOSE_SOCKET_IF_TIMEOUT, "true")); + } diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyServerConfigTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyServerConfigTest.java new file mode 100644 index 00000000000..c07025d63f9 --- /dev/null +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyServerConfigTest.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.remoting.netty; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.assertj.core.api.Assertions.assertThat; + +@RunWith(MockitoJUnitRunner.class) +public class NettyServerConfigTest { + + @Test + public void testChangeConfigBySystemProperty() { + System.setProperty(NettySystemConfig.COM_ROCKETMQ_REMOTING_SOCKET_BACKLOG, "65535"); + NettySystemConfig.socketBacklog = + Integer.parseInt(System.getProperty(NettySystemConfig.COM_ROCKETMQ_REMOTING_SOCKET_BACKLOG, "1024")); + NettyServerConfig changedConfig = new NettyServerConfig(); + assertThat(changedConfig.getServerSocketBacklog()).isEqualTo(65535); + } +} From 87b0be00012dff86727766081d4243eeee280699 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Wed, 24 Nov 2021 10:44:42 +0800 Subject: [PATCH 024/141] [ISSUE #2516]: Fix the value of sendThreadPoolQueueHeadWaitTimeMills is 0 most of the time (#3509) --- .../rocketmq/broker/BrokerController.java | 13 ++++--- .../rocketmq/broker/BrokerControllerTest.java | 35 +++++++++++++++++++ 2 files changed, 44 insertions(+), 4 deletions(-) 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 3e146b754a9..bce21c52093 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -33,6 +33,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.Optional; +import java.util.Objects; import org.apache.rocketmq.acl.AccessValidator; import org.apache.rocketmq.broker.client.ClientHousekeepingService; import org.apache.rocketmq.broker.client.ConsumerIdsChangeListener; @@ -650,10 +652,13 @@ public void protectBroker() { public long headSlowTimeMills(BlockingQueue q) { long slowTimeMills = 0; - final Runnable peek = q.peek(); - if (peek != null) { - RequestTask rt = BrokerFastFailure.castRunnable(peek); - slowTimeMills = rt == null ? 0 : this.messageStore.now() - rt.getCreateTimestamp(); + Optional op = q.stream() + .map(BrokerFastFailure::castRunnable) + .filter(Objects::nonNull) + .findFirst(); + if (op.isPresent()) { + RequestTask rt = op.get(); + slowTimeMills = this.messageStore.now() - rt.getCreateTimestamp(); } if (slowTimeMills < 0) { diff --git a/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java index dae1335540b..e8442a4d4b5 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java @@ -18,10 +18,16 @@ package org.apache.rocketmq.broker; import java.io.File; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.rocketmq.broker.latency.FutureTaskExt; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyServerConfig; +import org.apache.rocketmq.remoting.netty.RequestTask; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.junit.After; import org.junit.Ignore; @@ -47,4 +53,33 @@ public void testBrokerRestart() throws Exception { public void destroy() { UtilAll.deleteFile(new File(new MessageStoreConfig().getStorePathRootDir())); } + + @Test + public void testHeadSlowTimeMills() throws Exception { + BrokerController brokerController = new BrokerController( + new BrokerConfig(), + new NettyServerConfig(), + new NettyClientConfig(), + new MessageStoreConfig()); + brokerController.initialize(); + BlockingQueue queue = new LinkedBlockingQueue<>(); + + //create task is not instance of FutureTaskExt; + Runnable runnable = new Runnable() { + @Override + public void run() { + + } + }; + queue.add(runnable); + + RequestTask requestTask = new RequestTask(runnable, null, null); + // the requestTask is not the head of queue; + queue.add(new FutureTaskExt<>(requestTask, null)); + + long headSlowTimeMills = 100; + TimeUnit.MILLISECONDS.sleep(headSlowTimeMills); + assertThat(brokerController.headSlowTimeMills(queue)).isGreaterThanOrEqualTo(headSlowTimeMills); + //Attention: if we use the previous version method BrokerController#headSlowTimeMills, it will return 0; + } } From 2624763eab8675f203af5443aae53609cc18fed2 Mon Sep 17 00:00:00 2001 From: Smile <1494445739@qq.com> Date: Tue, 23 Nov 2021 11:11:21 +0800 Subject: [PATCH 025/141] doc-optimization (#3523) --- .../java/API_Reference_DefaultMQProducer.md | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/cn/client/java/API_Reference_DefaultMQProducer.md b/docs/cn/client/java/API_Reference_DefaultMQProducer.md index c45f7d4e50c..4a0b9d5f6c1 100644 --- a/docs/cn/client/java/API_Reference_DefaultMQProducer.md +++ b/docs/cn/client/java/API_Reference_DefaultMQProducer.md @@ -54,7 +54,7 @@ public class Producer { |int|retryTimesWhenSendAsyncFailed|异步模式下内部尝试发送消息的最大次数| |boolean|retryAnotherBrokerWhenNotStoreOK|是否在内部发送失败时重试另一个broker| |int|maxMessageSize|消息的最大长度| -|TraceDispatcher|traceDispatcher|消息追踪器。使用rcpHook来追踪消息| +|TraceDispatcher|traceDispatcher|基于RPCHooK实现的消息轨迹插件| ### 构造方法摘要 @@ -62,11 +62,11 @@ public class Producer { |-------|------------| |DefaultMQProducer()|由默认参数值创建一个生产者 | |DefaultMQProducer(final String producerGroup)|使用指定的分组名创建一个生产者| -|DefaultMQProducer(final String producerGroup, boolean enableMsgTrace)|使用指定的分组名创建一个生产者,并设置是否开启消息追踪| -|DefaultMQProducer(final String producerGroup, boolean enableMsgTrace, final String customizedTraceTopic)|使用指定的分组名创建一个生产者,并设置是否开启消息追踪及追踪topic的名称| +|DefaultMQProducer(final String producerGroup, boolean enableMsgTrace)|使用指定的分组名创建一个生产者,并设置是否开启消息轨迹| +|DefaultMQProducer(final String producerGroup, boolean enableMsgTrace, final String customizedTraceTopic)|使用指定的分组名创建一个生产者,并设置是否开启消息轨迹及追踪topic的名称| |DefaultMQProducer(RPCHook rpcHook)|使用指定的hook创建一个生产者| |DefaultMQProducer(final String producerGroup, RPCHook rpcHook)|使用指定的分组名及自定义hook创建一个生产者| -|DefaultMQProducer(final String producerGroup, RPCHook rpcHook, boolean enableMsgTrace,final String customizedTraceTopic)|使用指定的分组名及自定义hook创建一个生产者,并设置是否开启消息追踪及追踪topic的名称| +|DefaultMQProducer(final String producerGroup, RPCHook rpcHook, boolean enableMsgTrace,final String customizedTraceTopic)|使用指定的分组名及自定义hook创建一个生产者,并设置是否开启消息轨迹及追踪topic的名称| ### 使用方法摘要 @@ -204,7 +204,7 @@ public class Producer { `private TraceDispatcher traceDispatcher = null` - 在开启消息追踪后,该类通过hook的方式把消息生产者,消息存储的broker和消费者消费消息的信息像链路一样记录下来。在构造生产者时根据构造入参enableMsgTrace来决定是否创建该对象。 + 在开启消息轨迹后,该类通过hook的方式把消息生产者,消息存储的broker和消费者消费消息的信息像链路一样记录下来。在构造生产者时根据构造入参enableMsgTrace来决定是否创建该对象。 ### 构造方法详细信息 @@ -230,20 +230,20 @@ public class Producer { `DefaultMQProducer(final String producerGroup, boolean enableMsgTrace)` - 使用指定的分组名创建一个生产者,并设置是否开启消息追踪。 + 使用指定的分组名创建一个生产者,并设置是否开启消息轨迹。 - 入参描述: 参数名 | 类型 | 是否必须 | 缺省值 |描述 ---|---|---|---|--- producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 - enableMsgTrace | boolean | 是 | false |是否开启消息追踪 + enableMsgTrace | boolean | 是 | false |是否开启消息轨迹 4. DefaultMQProducer `DefaultMQProducer(final String producerGroup, boolean enableMsgTrace, final String customizedTraceTopic)` - 使用指定的分组名创建一个生产者,并设置是否开启消息追踪及追踪topic的名称。 + 使用指定的分组名创建一个生产者,并设置是否开启消息轨迹及追踪topic的名称。 - 入参描述: @@ -251,8 +251,8 @@ public class Producer { ---|---|---|---|--- producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 rpcHook | RPCHook | 否 | null |每个远程命令执行后会回调rpcHook - enableMsgTrace | boolean | 是 | false |是否开启消息追踪 - customizedTraceTopic | String | 否 | RMQ_SYS_TRACE_TOPIC | 消息跟踪topic的名称 + enableMsgTrace | boolean | 是 | false |是否开启消息轨迹 + customizedTraceTopic | String | 否 | RMQ_SYS_TRACE_TOPIC | 消息轨迹topic的名称 5. DefaultMQProducer @@ -283,7 +283,7 @@ public class Producer { `DefaultMQProducer(final String producerGroup, RPCHook rpcHook, boolean enableMsgTrace,final String customizedTraceTopic)` - 使用指定的分组名及自定义hook创建一个生产者,并设置是否开启消息追踪及追踪topic的名称。 + 使用指定的分组名及自定义hook创建一个生产者,并设置是否开启消息轨迹及追踪topic的名称。 - 入参描述: @@ -291,8 +291,8 @@ public class Producer { ---|---|---|---|--- producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 rpcHook | RPCHook | 否 | null |每个远程命令执行后会回调rpcHook - enableMsgTrace | boolean | 是 | false |是否开启消息追踪 - customizedTraceTopic | String | 否 | RMQ_SYS_TRACE_TOPIC | 消息跟踪topic的名称 + enableMsgTrace | boolean | 是 | false |是否开启消息轨迹 + customizedTraceTopic | String | 否 | RMQ_SYS_TRACE_TOPIC | 消息轨迹topic的名称 ### 使用方法详细信息 From a82306853d5a5902a180ed07f660e45e4bab588e Mon Sep 17 00:00:00 2001 From: mrhbj <1825636423@qq.com> Date: Tue, 30 Nov 2021 11:00:48 +0800 Subject: [PATCH 026/141] fix notes wrong (#3543) (#3544) Co-authored-by: huangbj --- .../java/org/apache/rocketmq/example/simple/PullConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java b/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java index 4bc63383954..ff9ef9ca34e 100644 --- a/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java @@ -56,7 +56,7 @@ public Thread newThread(Runnable r) { executors.execute(new Runnable() { public void doSomething(List msgs) { - //do you business + //do your business } From 5a0da0260b213bd22129fd3119459edfce92b036 Mon Sep 17 00:00:00 2001 From: Naman rai Date: Wed, 1 Dec 2021 11:26:30 +0530 Subject: [PATCH 027/141] changed 'item' to 'items' (#3563) --- .../main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java index 9b49567f322..f6a26d6888b 100644 --- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java +++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvStartup.java @@ -165,7 +165,7 @@ public static Options buildCommandlineOptions(final Options options) { opt.setRequired(false); options.addOption(opt); - opt = new Option("p", "printConfigItem", false, "Print all config item"); + opt = new Option("p", "printConfigItem", false, "Print all config items"); opt.setRequired(false); options.addOption(opt); From eff953060839ffb788bebc1394de43c219b71710 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B5=B5=E7=A6=8F=E5=BB=BA?= <2573259572@qq.com> Date: Wed, 1 Dec 2021 14:00:55 +0800 Subject: [PATCH 028/141] typo int readme[ecosystem] (#3557) Co-authored-by: fujian.zfj --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index eb20e798ea9..79b316c2987 100644 --- a/README.md +++ b/README.md @@ -18,7 +18,7 @@ It offers a variety of features: * A variety of cross language clients, such as Java, [C/C++](https://github.com/apache/rocketmq-client-cpp), [Python](https://github.com/apache/rocketmq-client-python), [Go](https://github.com/apache/rocketmq-client-go), [Node.js](https://github.com/apache/rocketmq-client-nodejs) * Pluggable transport protocols, such as TCP, SSL, AIO * Built-in message tracing capability, also support opentracing -* Versatile big-data and streaming ecosytem integration +* Versatile big-data and streaming ecosystem integration * Message retroactivity by time or offset * Reliable FIFO and strict ordered messaging in the same queue * Efficient pull and push consumption model From d6f3ed6c1ace2dc622f6ddbe5f46da3fda8a6162 Mon Sep 17 00:00:00 2001 From: XiaoyiPeng <644120242@qq.com> Date: Wed, 1 Dec 2021 15:54:21 +0800 Subject: [PATCH 029/141] [ISSUE #3567] fix: The value of configuration item (PushConsumer#consumeThreadMin) in document and code are different. --- docs/cn/best_practice.md | 2 +- docs/en/Configuration_Client.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/cn/best_practice.md b/docs/cn/best_practice.md index b24c6eca57b..07b4ba41ac9 100755 --- a/docs/cn/best_practice.md +++ b/docs/cn/best_practice.md @@ -287,7 +287,7 @@ DefaultMQProducer、TransactionMQProducer、DefaultMQPushConsumer、DefaultMQPul | subscription | | 订阅关系 | | messageListener | | 消息监听器 | | offsetStore | | 消费进度存储 | -| consumeThreadMin | 10 | 消费线程池最小线程数 | +| consumeThreadMin | 20 | 消费线程池最小线程数 | | consumeThreadMax | 20 | 消费线程池最大线程数 | | consumeConcurrentlyMaxSpan | 2000 | 单队列并行消费允许的最大跨度 | | pullThresholdForQueue | 1000 | 拉消息本地队列缓存消息最大数 | diff --git a/docs/en/Configuration_Client.md b/docs/en/Configuration_Client.md index dedb4240b7c..e026a68a257 100644 --- a/docs/en/Configuration_Client.md +++ b/docs/en/Configuration_Client.md @@ -82,7 +82,7 @@ HTTP static server addressing is recommended, because it is simple client deploy | subscription | | subscription relation | | messageListener | | message listener | | offsetStore | | Consumption progress store | -| consumeThreadMin | 10 | Minimum of thread in consumption thread pool | +| consumeThreadMin | 20 | Minimum of thread in consumption thread pool | | consumeThreadMax | 20 | Maximum of thread in consumption thread pool | | | | | | consumeConcurrentlyMaxSpan | 2000 | Maximum span allowed for single queue parallel consumption | From 16cabf020cb220b6180954f2963a6548f9d06734 Mon Sep 17 00:00:00 2001 From: Naman rai Date: Thu, 2 Dec 2021 13:22:06 +0530 Subject: [PATCH 030/141] removed typos and spelling errors from comments in quickstart (#3572) --- .../java/org/apache/rocketmq/example/quickstart/Consumer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 6d3b936507e..07e572402d2 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 @@ -50,12 +50,12 @@ public static void main(String[] args) throws InterruptedException, MQClientExce */ /* - * Specify where to start in case the specified consumer group is a brand new one. + * Specify where to start in case the specific consumer group is a brand-new one. */ consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); /* - * Subscribe one more more topics to consume. + * Subscribe one more topic to consume. */ consumer.subscribe("TopicTest", "*"); From 5d5d5f5f4c43d8cb2a45aee8cd0ad758902d348b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Thu, 2 Dec 2021 15:52:30 +0800 Subject: [PATCH 031/141] optimizing: For a big set, replace sequential iteration addition with a parallel stream when calculating results. (#3540) refactor: Renaming variables lockXXX to xxxLock may be better. --- .../rocketmq/store/StoreStatsService.java | 58 +++++++++---------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java index 395f5e30057..e8e7e048fa8 100644 --- a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java +++ b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java @@ -62,13 +62,13 @@ public class StoreStatsService extends ServiceThread { private volatile long putMessageEntireTimeMax = 0; private volatile long getMessageEntireTimeMax = 0; // for putMessageEntireTimeMax - private ReentrantLock lockPut = new ReentrantLock(); + private ReentrantLock putLock = new ReentrantLock(); // for getMessageEntireTimeMax - private ReentrantLock lockGet = new ReentrantLock(); + private ReentrantLock getLock = new ReentrantLock(); private volatile long dispatchMaxBuffer = 0; - private ReentrantLock lockSampling = new ReentrantLock(); + private ReentrantLock samplingLock = new ReentrantLock(); private long lastPrintTimestamp = System.currentTimeMillis(); public StoreStatsService() { @@ -138,10 +138,10 @@ else if (value < 10000) { } if (value > this.putMessageEntireTimeMax) { - this.lockPut.lock(); + this.putLock.lock(); this.putMessageEntireTimeMax = value > this.putMessageEntireTimeMax ? value : this.putMessageEntireTimeMax; - this.lockPut.unlock(); + this.putLock.unlock(); } } @@ -151,10 +151,10 @@ public long getGetMessageEntireTimeMax() { public void setGetMessageEntireTimeMax(long value) { if (value > this.getMessageEntireTimeMax) { - this.lockGet.lock(); + this.getLock.lock(); this.getMessageEntireTimeMax = value > this.getMessageEntireTimeMax ? value : this.getMessageEntireTimeMax; - this.lockGet.unlock(); + this.getLock.unlock(); } } @@ -193,11 +193,11 @@ public String toString() { } public long getPutMessageTimesTotal() { - long rs = 0; - for (LongAdder data : putMessageTopicTimesTotal.values()) { - rs += data.longValue(); - } - return rs; + Map map = putMessageTopicTimesTotal; + return map.values() + .parallelStream() + .mapToLong(LongAdder::longValue) + .sum(); } private String getFormatRuntime() { @@ -217,11 +217,11 @@ private String getFormatRuntime() { } public long getPutMessageSizeTotal() { - long rs = 0; - for (LongAdder data : putMessageTopicSizeTotal.values()) { - rs += data.longValue(); - } - return rs; + Map map = putMessageTopicSizeTotal; + return map.values() + .parallelStream() + .mapToLong(LongAdder::longValue) + .sum(); } private String getPutMessageDistributeTimeStringInfo(Long total) { @@ -315,7 +315,7 @@ private String putMessageDistributeTimeToString() { private String getPutTps(int time) { String result = ""; - this.lockSampling.lock(); + this.samplingLock.lock(); try { CallSnapshot last = this.putTimesList.getLast(); @@ -325,14 +325,14 @@ private String getPutTps(int time) { } } finally { - this.lockSampling.unlock(); + this.samplingLock.unlock(); } return result; } private String getGetFoundTps(int time) { String result = ""; - this.lockSampling.lock(); + this.samplingLock.lock(); try { CallSnapshot last = this.getTimesFoundList.getLast(); @@ -342,7 +342,7 @@ private String getGetFoundTps(int time) { result += CallSnapshot.getTPS(lastBefore, last); } } finally { - this.lockSampling.unlock(); + this.samplingLock.unlock(); } return result; @@ -350,7 +350,7 @@ private String getGetFoundTps(int time) { private String getGetMissTps(int time) { String result = ""; - this.lockSampling.lock(); + this.samplingLock.lock(); try { CallSnapshot last = this.getTimesMissList.getLast(); @@ -361,14 +361,14 @@ private String getGetMissTps(int time) { } } finally { - this.lockSampling.unlock(); + this.samplingLock.unlock(); } return result; } private String getGetTotalTps(int time) { - this.lockSampling.lock(); + this.samplingLock.lock(); double found = 0; double miss = 0; try { @@ -392,7 +392,7 @@ private String getGetTotalTps(int time) { } } finally { - this.lockSampling.unlock(); + this.samplingLock.unlock(); } return Double.toString(found + miss); @@ -400,7 +400,7 @@ private String getGetTotalTps(int time) { private String getGetTransferedTps(int time) { String result = ""; - this.lockSampling.lock(); + this.samplingLock.lock(); try { CallSnapshot last = this.transferedMsgCountList.getLast(); @@ -411,7 +411,7 @@ private String getGetTransferedTps(int time) { } } finally { - this.lockSampling.unlock(); + this.samplingLock.unlock(); } return result; @@ -469,7 +469,7 @@ public String getServiceName() { } private void sampling() { - this.lockSampling.lock(); + this.samplingLock.lock(); try { this.putTimesList.add(new CallSnapshot(System.currentTimeMillis(), getPutMessageTimesTotal())); if (this.putTimesList.size() > (MAX_RECORDS_OF_SAMPLING + 1)) { @@ -495,7 +495,7 @@ private void sampling() { } } finally { - this.lockSampling.unlock(); + this.samplingLock.unlock(); } } From 4daade3c1ac4ad0650b6e77430cd93ed9eeba0a7 Mon Sep 17 00:00:00 2001 From: NAMANIND Date: Wed, 1 Dec 2021 16:18:59 +0530 Subject: [PATCH 032/141] Removed if-else statements of ROLLBACK and COMMIT --- .../org/apache/rocketmq/example/ordermessage/Consumer.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/example/src/main/java/org/apache/rocketmq/example/ordermessage/Consumer.java b/example/src/main/java/org/apache/rocketmq/example/ordermessage/Consumer.java index 3304de38742..257c6a045f2 100644 --- a/example/src/main/java/org/apache/rocketmq/example/ordermessage/Consumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/ordermessage/Consumer.java @@ -45,10 +45,6 @@ public ConsumeOrderlyStatus consumeMessage(List msgs, ConsumeOrderly this.consumeTimes.incrementAndGet(); if ((this.consumeTimes.get() % 2) == 0) { return ConsumeOrderlyStatus.SUCCESS; - } else if ((this.consumeTimes.get() % 3) == 0) { - return ConsumeOrderlyStatus.ROLLBACK; - } else if ((this.consumeTimes.get() % 4) == 0) { - return ConsumeOrderlyStatus.COMMIT; } else if ((this.consumeTimes.get() % 5) == 0) { context.setSuspendCurrentQueueTimeMillis(3000); return ConsumeOrderlyStatus.SUSPEND_CURRENT_QUEUE_A_MOMENT; From 4d495515acab8f2f2fbbd76872c007f2d01c891e Mon Sep 17 00:00:00 2001 From: cnScarb Date: Tue, 7 Dec 2021 10:14:56 +0800 Subject: [PATCH 033/141] add single quotes for namesrvAddr option example, to avoid command splits caused by semicolons (#3592) --- .../src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java b/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java index 421eedb4273..b00bad3c5f9 100644 --- a/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java +++ b/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java @@ -33,7 +33,7 @@ public static Options buildCommandlineOptions(final Options options) { opt = new Option("n", "namesrvAddr", true, - "Name server address list, eg: 192.168.0.1:9876;192.168.0.2:9876"); + "Name server address list, eg: '192.168.0.1:9876;192.168.0.2:9876'"); opt.setRequired(false); options.addOption(opt); From 5694db1c4a52afd8505c1d1b078d578872225eee Mon Sep 17 00:00:00 2001 From: wangfan <42178996+ferrirW@users.noreply.github.com> Date: Tue, 7 Dec 2021 11:17:21 +0800 Subject: [PATCH 034/141] =?UTF-8?q?=20[ISSUE=20#3579]=EF=BC=9AFix=20spelli?= =?UTF-8?q?ng=20mistake=20in=20getter/setter=20method=20of=20mQClientFacto?= =?UTF-8?q?ry=20SSUE?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: wangfan --- .../impl/consumer/ConsumeMessageOrderlyService.java | 2 +- .../impl/consumer/DefaultLitePullConsumerImpl.java | 2 +- .../impl/consumer/DefaultMQPullConsumerImpl.java | 2 +- .../impl/consumer/DefaultMQPushConsumerImpl.java | 6 +++--- .../rocketmq/client/impl/consumer/RebalanceImpl.java | 4 ++-- .../client/impl/consumer/RebalancePushImpl.java | 4 ++-- .../client/impl/producer/DefaultMQProducerImpl.java | 10 +++++----- .../rocketmq/client/trace/AsyncTraceDispatcher.java | 2 +- .../client/trace/hook/EndTransactionTraceHookImpl.java | 2 +- .../ConsumeMessageConcurrentlyServiceTest.java | 4 ++-- .../client/producer/DefaultMQProducerTest.java | 7 ++----- .../client/trace/DefaultMQConsumerWithTraceTest.java | 8 ++++---- .../trace/DefaultMQLitePullConsumerWithTraceTest.java | 3 +-- .../trace/DefaultMQProducerWithOpenTracingTest.java | 4 ++-- .../client/trace/DefaultMQProducerWithTraceTest.java | 4 ++-- .../TransactionMQProducerWithOpenTracingTest.java | 4 ++-- .../trace/TransactionMQProducerWithTraceTest.java | 4 ++-- .../apache/rocketmq/example/simple/PullConsumer.java | 2 +- 18 files changed, 35 insertions(+), 39 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java index 8d92b5718a6..cff0d6795e5 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java @@ -378,7 +378,7 @@ public boolean sendMessageBack(final MessageExt msg) { MessageAccessor.clearProperty(newMsg, MessageConst.PROPERTY_TRANSACTION_PREPARED); newMsg.setDelayTimeLevel(3 + msg.getReconsumeTimes()); - this.defaultMQPushConsumer.getDefaultMQPushConsumerImpl().getmQClientFactory().getDefaultMQProducer().send(newMsg); + this.defaultMQPushConsumer.getDefaultMQPushConsumerImpl().getMqClientFactory().getDefaultMQProducer().send(newMsg); return true; } catch (Exception e) { log.error("sendMessageBack exception, group: " + this.consumerGroup + " msg: " + msg.toString(), e); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java index 46a72fb5e84..fecf07645ed 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java @@ -329,7 +329,7 @@ private void initRebalanceImpl() { this.rebalanceImpl.setConsumerGroup(this.defaultLitePullConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultLitePullConsumer.getMessageModel()); this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultLitePullConsumer.getAllocateMessageQueueStrategy()); - this.rebalanceImpl.setmQClientFactory(this.mQClientFactory); + this.rebalanceImpl.setMqClientFactory(this.mQClientFactory); } private void initPullAPIWrapper() { diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java index eed5fa43f04..6792d4d609c 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java @@ -637,7 +637,7 @@ public synchronized void start() throws MQClientException { this.rebalanceImpl.setConsumerGroup(this.defaultMQPullConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultMQPullConsumer.getMessageModel()); this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultMQPullConsumer.getAllocateMessageQueueStrategy()); - this.rebalanceImpl.setmQClientFactory(this.mQClientFactory); + this.rebalanceImpl.setMqClientFactory(this.mQClientFactory); this.pullAPIWrapper = new PullAPIWrapper( mQClientFactory, diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java index 59b8deb3fba..a61a1a6ac2f 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java @@ -591,7 +591,7 @@ public synchronized void start() throws MQClientException { this.rebalanceImpl.setConsumerGroup(this.defaultMQPushConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultMQPushConsumer.getMessageModel()); this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultMQPushConsumer.getAllocateMessageQueueStrategy()); - this.rebalanceImpl.setmQClientFactory(this.mQClientFactory); + this.rebalanceImpl.setMqClientFactory(this.mQClientFactory); this.pullAPIWrapper = new PullAPIWrapper( mQClientFactory, @@ -1089,11 +1089,11 @@ public ConsumerRunningInfo consumerRunningInfo() { return info; } - public MQClientInstance getmQClientFactory() { + public MQClientInstance getMqClientFactory() { return mQClientFactory; } - public void setmQClientFactory(MQClientInstance mQClientFactory) { + public void setMqClientFactory(MQClientInstance mQClientFactory) { this.mQClientFactory = mQClientFactory; } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java index 7677d8b685f..41327547b0e 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java @@ -472,11 +472,11 @@ public void setAllocateMessageQueueStrategy(AllocateMessageQueueStrategy allocat this.allocateMessageQueueStrategy = allocateMessageQueueStrategy; } - public MQClientInstance getmQClientFactory() { + public MQClientInstance getMqClientFactory() { return mQClientFactory; } - public void setmQClientFactory(MQClientInstance mQClientFactory) { + public void setMqClientFactory(MQClientInstance mQClientFactory) { this.mQClientFactory = mQClientFactory; } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java index 666b696ffa8..464bbf2a1d3 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java @@ -78,7 +78,7 @@ public void messageQueueChanged(String topic, Set mqAll, Set nsList = this.getmQClientFactory().getMQClientAPIImpl().getNameServerAddressList(); + List nsList = this.getMqClientFactory().getMQClientAPIImpl().getNameServerAddressList(); if (null == nsList || nsList.isEmpty()) { throw new MQClientException( "No name server address, please set it." + FAQUrl.suggestTodo(FAQUrl.NAME_SERVER_ADDR_NOT_EXIST_URL), null).setResponseCode(ClientErrorCode.NO_NAME_SERVER_EXCEPTION); @@ -896,7 +896,7 @@ private SendResult sendKernelImpl(final Message msg, throw new MQClientException("The broker[" + mq.getBrokerName() + "] not exist", null); } - public MQClientInstance getmQClientFactory() { + public MQClientInstance getMqClientFactory() { return mQClientFactory; } @@ -1568,16 +1568,16 @@ private void requestFail(final String correlationId) { private void prepareSendRequest(final Message msg, long timeout) { String correlationId = CorrelationIdUtil.createCorrelationId(); - String requestClientId = this.getmQClientFactory().getClientId(); + String requestClientId = this.getMqClientFactory().getClientId(); MessageAccessor.putProperty(msg, MessageConst.PROPERTY_CORRELATION_ID, correlationId); MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT, requestClientId); MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_TTL, String.valueOf(timeout)); - boolean hasRouteData = this.getmQClientFactory().getTopicRouteTable().containsKey(msg.getTopic()); + boolean hasRouteData = this.getMqClientFactory().getTopicRouteTable().containsKey(msg.getTopic()); if (!hasRouteData) { long beginTimestamp = System.currentTimeMillis(); this.tryToFindTopicPublishInfo(msg.getTopic()); - this.getmQClientFactory().sendHeartbeatToAllBrokerWithLock(); + this.getMqClientFactory().sendHeartbeatToAllBrokerWithLock(); long cost = System.currentTimeMillis() - beginTimestamp; if (cost > 500) { log.warn("prepare send request for <{}> cost {} ms", msg.getTopic(), cost); diff --git a/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java b/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java index 7ff8bd77e03..0626e36bbf6 100644 --- a/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java +++ b/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java @@ -413,7 +413,7 @@ private Set tryGetMessageQueueBrokerSet(DefaultMQProducerImpl producer, TopicPublishInfo topicPublishInfo = producer.getTopicPublishInfoTable().get(topic); if (null == topicPublishInfo || !topicPublishInfo.ok()) { producer.getTopicPublishInfoTable().putIfAbsent(topic, new TopicPublishInfo()); - producer.getmQClientFactory().updateTopicRouteInfoFromNameServer(topic); + producer.getMqClientFactory().updateTopicRouteInfoFromNameServer(topic); topicPublishInfo = producer.getTopicPublishInfoTable().get(topic); } if (topicPublishInfo.isHaveTopicRouterInfo() || topicPublishInfo.ok()) { diff --git a/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java b/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java index cbd755ba39e..8172d0459d5 100644 --- a/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java @@ -63,7 +63,7 @@ public void endTransaction(EndTransactionContext context) { traceBean.setKeys(context.getMessage().getKeys()); traceBean.setStoreHost(context.getBrokerAddr()); traceBean.setMsgType(MessageType.Trans_msg_Commit); - traceBean.setClientHost(((AsyncTraceDispatcher)localDispatcher).getHostProducer().getmQClientFactory().getClientId()); + traceBean.setClientHost(((AsyncTraceDispatcher)localDispatcher).getHostProducer().getMqClientFactory().getClientId()); traceBean.setMsgId(context.getMsgId()); traceBean.setTransactionState(context.getTransactionState()); traceBean.setTransactionId(context.getTransactionId()); diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java index 6fa76e0380b..6c331db0593 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java @@ -102,7 +102,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, pushConsumer.subscribe(topic, "*"); pushConsumer.start(); - mQClientFactory = spy(pushConsumerImpl.getmQClientFactory()); + mQClientFactory = spy(pushConsumerImpl.getMqClientFactory()); field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory"); field.setAccessible(true); field.set(pushConsumerImpl, mQClientFactory); @@ -116,7 +116,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, field.setAccessible(true); field.set(pushConsumerImpl, pullAPIWrapper); - pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory); + pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setMqClientFactory(mQClientFactory); mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl); when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), diff --git a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java index 5f29fe113c5..e161d4ab784 100644 --- a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java @@ -48,7 +48,6 @@ import org.apache.rocketmq.common.protocol.route.QueueData; import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.remoting.exception.RemotingException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; import org.apache.rocketmq.remoting.netty.NettyRemotingClient; import org.junit.After; import org.junit.Before; @@ -56,9 +55,7 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.Spy; -import org.mockito.invocation.InvocationOnMock; import org.mockito.junit.MockitoJUnitRunner; -import org.mockito.stubbing.Answer; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown; @@ -105,7 +102,7 @@ public void init() throws Exception { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -361,7 +358,7 @@ public void testSetCallbackExecutor() throws MQClientException { producer.setCallbackExecutor(customized); NettyRemotingClient remotingClient = (NettyRemotingClient) producer.getDefaultMQProducerImpl() - .getmQClientFactory().getMQClientAPIImpl().getRemotingClient(); + .getMqClientFactory().getMQClientAPIImpl().getRemotingClient(); assertThat(remotingClient.getCallbackExecutor()).isEqualTo(customized); } diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java index 976380b2723..d142e94d76a 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java @@ -154,8 +154,8 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, pushConsumer.start(); - mQClientFactory = spy(pushConsumerImpl.getmQClientFactory()); - mQClientTraceFactory = spy(pushConsumerImpl.getmQClientFactory()); + mQClientFactory = spy(pushConsumerImpl.getMqClientFactory()); + mQClientTraceFactory = spy(pushConsumerImpl.getMqClientFactory()); field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory"); field.setAccessible(true); @@ -178,7 +178,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, field.setAccessible(true); field.set(pushConsumerImpl, pullAPIWrapper); - pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory); + pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setMqClientFactory(mQClientFactory); mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl); when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), @@ -214,7 +214,7 @@ public void terminate() { @Test public void testPullMessage_WithTrace_Success() throws InterruptedException, RemotingException, MQBrokerException, MQClientException { - traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); final CountDownLatch countDownLatch = new CountDownLatch(1); final AtomicReference messageAtomic = new AtomicReference<>(); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java index ce3b832b006..884a2f6880e 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java @@ -25,7 +25,6 @@ import org.apache.rocketmq.client.consumer.PullStatus; import org.apache.rocketmq.client.consumer.store.OffsetStore; import org.apache.rocketmq.client.consumer.store.ReadOffsetType; -import org.apache.rocketmq.client.hook.SendMessageContext; import org.apache.rocketmq.client.impl.CommunicationMode; import org.apache.rocketmq.client.impl.FindBrokerResult; import org.apache.rocketmq.client.impl.MQAdminImpl; @@ -219,7 +218,7 @@ private void initDefaultLitePullConsumer(DefaultLitePullConsumer litePullConsume field.setAccessible(true); field.set(litePullConsumerImpl, offsetStore); - traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(PullCallback.class))) diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java index 5d64a93f039..cc57a03164c 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java @@ -100,7 +100,7 @@ public void init() throws Exception { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -112,7 +112,7 @@ public void init() throws Exception { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); producer.send(message); assertThat(tracer.finishedSpans().size()).isEqualTo(1); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java index 234e32e6807..b951ae88deb 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java @@ -109,7 +109,7 @@ public void init() throws Exception { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -121,7 +121,7 @@ public void init() throws Exception { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); final CountDownLatch countDownLatch = new CountDownLatch(1); try { diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java index dd6d1083ce0..5de15f451ed 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java @@ -118,7 +118,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -130,7 +130,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); producer.sendMessageInTransaction(message, null); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java index f838817bf9d..02289505d9a 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java @@ -123,7 +123,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); Field fieldHooks = DefaultMQProducerImpl.class.getDeclaredField("endTransactionHookList"); fieldHooks.setAccessible(true); @@ -141,7 +141,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); AtomicReference context = new AtomicReference<>(); doAnswer(mock -> { diff --git a/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java b/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java index ff9ef9ca34e..69067c90c49 100644 --- a/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java @@ -136,7 +136,7 @@ public long consumeFromOffset(MessageQueue messageQueue) throws MQClientExceptio } public void incPullTPS(String topic, int pullSize) { - consumer.getDefaultMQPullConsumerImpl().getRebalanceImpl().getmQClientFactory() + consumer.getDefaultMQPullConsumerImpl().getRebalanceImpl().getMqClientFactory() .getConsumerStatsManager().incPullTPS(consumer.getConsumerGroup(), topic, pullSize); } }); From 3084713807adfb71fd9140100a8e9914d63e296e Mon Sep 17 00:00:00 2001 From: lushilin <897401919@qq.com> Date: Tue, 7 Dec 2021 11:28:34 +0800 Subject: [PATCH 035/141] [ISSUE #3551] Fix admin cloneoffset Co-authored-by: lushilin --- .../rocketmq/tools/command/offset/CloneGroupOffsetCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/CloneGroupOffsetCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/CloneGroupOffsetCommand.java index 460f59c377d..285327fe882 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/CloneGroupOffsetCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/CloneGroupOffsetCommand.java @@ -87,7 +87,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t break; } } - long offset = consumeStats.getOffsetTable().get(mq).getBrokerOffset(); + long offset = consumeStats.getOffsetTable().get(mq).getConsumerOffset(); if (offset >= 0) { defaultMQAdminExt.updateConsumeOffset(addr, destGroup, mq, offset); } From 83accb6f12d314be9d164a940009584091652570 Mon Sep 17 00:00:00 2001 From: huangli Date: Tue, 7 Dec 2021 11:42:43 +0800 Subject: [PATCH 036/141] [Issue #3476] Fix last separator of properties string is missing when using batch send. (#3479) This problem introduced since 4.9.1, may cause tag incorrect. --- .../java/org/apache/rocketmq/store/CommitLog.java | 12 +++++++++--- .../apache/rocketmq/store/BatchPutMessageTest.java | 2 +- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 36db2f5146b..112e9bced48 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -1607,13 +1607,16 @@ protected ByteBuffer encode(final MessageExtBatch messageExtBatch, PutMessageCon short propertiesLen = messagesByteBuff.getShort(); int propertiesPos = messagesByteBuff.position(); messagesByteBuff.position(propertiesPos + propertiesLen); + boolean needAppendLastPropertySeparator = propertiesLen > 0 && batchPropLen > 0 + && messagesByteBuff.get(messagesByteBuff.position() - 1) != MessageDecoder.PROPERTY_SEPARATOR; final byte[] topicData = messageExtBatch.getTopic().getBytes(MessageDecoder.CHARSET_UTF8); final int topicLength = topicData.length; - final int msgLen = calMsgLength(messageExtBatch.getSysFlag(), bodyLen, topicLength, - propertiesLen + batchPropLen); + int totalPropLen = needAppendLastPropertySeparator ? propertiesLen + batchPropLen + 1 + : propertiesLen + batchPropLen; + final int msgLen = calMsgLength(messageExtBatch.getSysFlag(), bodyLen, topicLength, totalPropLen); // Exceeds the maximum message if (msgLen > this.maxMessageSize) { @@ -1666,11 +1669,14 @@ protected ByteBuffer encode(final MessageExtBatch messageExtBatch, PutMessageCon this.encoderBuffer.put((byte) topicLength); this.encoderBuffer.put(topicData); // 17 PROPERTIES - this.encoderBuffer.putShort((short) (propertiesLen + batchPropLen)); + this.encoderBuffer.putShort((short) totalPropLen); if (propertiesLen > 0) { this.encoderBuffer.put(messagesByteBuff.array(), propertiesPos, propertiesLen); } if (batchPropLen > 0) { + if (needAppendLastPropertySeparator) { + this.encoderBuffer.put((byte) MessageDecoder.PROPERTY_SEPARATOR); + } this.encoderBuffer.put(batchPropData, 0, batchPropLen); } } diff --git a/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java index 2c1fd25f01c..3bc52e3b1d3 100644 --- a/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java @@ -105,7 +105,7 @@ public void testPutMessages() throws Exception { short propertiesLength = (short) propertiesBytes.length; final byte[] topicData = msg.getTopic().getBytes(MessageDecoder.CHARSET_UTF8); final int topicLength = topicData.length; - msgLengthArr[j] = calMsgLength(msg.getBody().length, topicLength, propertiesLength+batchPropLen) + msgLengthArr[j - 1]; + msgLengthArr[j] = calMsgLength(msg.getBody().length, topicLength, propertiesLength+batchPropLen+1) + msgLengthArr[j - 1]; j++; } byte[] batchMessageBody = MessageDecoder.encodeMessages(messages); From a779d69862c69db0d84fc27b35cfe5d08dc96377 Mon Sep 17 00:00:00 2001 From: zhaohai <33314633+zhaohai1299002788@users.noreply.github.com> Date: Tue, 7 Dec 2021 12:35:43 +0800 Subject: [PATCH 037/141] Unnecessary boxing of primitives (#3596) Co-authored-by: zh378814 --- .../main/java/org/apache/rocketmq/acl/common/AclUtils.java | 2 +- .../rocketmq/acl/plain/RemoteAddressStrategyFactory.java | 4 ++-- .../transaction/queue/TransactionalMessageServiceImpl.java | 4 ++-- .../org/apache/rocketmq/store/dledger/DLedgerCommitLog.java | 2 +- .../tools/command/queue/QueryConsumeQueueCommand.java | 6 +++--- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/acl/src/main/java/org/apache/rocketmq/acl/common/AclUtils.java b/acl/src/main/java/org/apache/rocketmq/acl/common/AclUtils.java index b801c69c707..b385338c6d3 100644 --- a/acl/src/main/java/org/apache/rocketmq/acl/common/AclUtils.java +++ b/acl/src/main/java/org/apache/rocketmq/acl/common/AclUtils.java @@ -163,7 +163,7 @@ public static boolean isColon(String netaddress) { } public static boolean isScope(String num) { - return isScope(Integer.valueOf(num.trim())); + return isScope(Integer.parseInt(num.trim())); } public static boolean isScope(int num) { diff --git a/acl/src/main/java/org/apache/rocketmq/acl/plain/RemoteAddressStrategyFactory.java b/acl/src/main/java/org/apache/rocketmq/acl/plain/RemoteAddressStrategyFactory.java index 1c10fae4b94..50e6c734ccc 100644 --- a/acl/src/main/java/org/apache/rocketmq/acl/plain/RemoteAddressStrategyFactory.java +++ b/acl/src/main/java/org/apache/rocketmq/acl/plain/RemoteAddressStrategyFactory.java @@ -187,8 +187,8 @@ private boolean analysis(String[] strArray, int index) { } String[] valueArray = StringUtils.split(value, "-"); - this.start = Integer.valueOf(valueArray[0]); - this.end = Integer.valueOf(valueArray[1]); + this.start = Integer.parseInt(valueArray[0]); + this.end = Integer.parseInt(valueArray[1]); if (!(AclUtils.isScope(end) && AclUtils.isScope(start) && start <= end)) { throw new AclException(String.format("RangeRemoteAddressStrategy netaddress examine scope Exception start is %s , end is %s", start, end)); } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java index 74df47a088e..3f244f4a240 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java @@ -393,7 +393,7 @@ private PullResult pullOpMsg(MessageQueue mq, long offset, int nums) { private Long getLong(String s) { long v = -1; try { - v = Long.valueOf(s); + v = Long.parseLong(s); } catch (Exception e) { log.error("GetLong error", e); } @@ -404,7 +404,7 @@ private Long getLong(String s) { private Integer getInt(String s) { int v = -1; try { - v = Integer.valueOf(s); + v = Integer.parseInt(s); } catch (Exception e) { log.error("GetInt error", e); } diff --git a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java index 493917587ac..37886730c2d 100644 --- a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java @@ -94,7 +94,7 @@ public DLedgerCommitLog(final DefaultMessageStore defaultMessageStore) { dLedgerConfig.setPreferredLeaderId(defaultMessageStore.getMessageStoreConfig().getPreferredLeaderId()); dLedgerConfig.setEnableBatchPush(defaultMessageStore.getMessageStoreConfig().isEnableBatchPush()); - id = Integer.valueOf(dLedgerConfig.getSelfId().substring(1)) + 1; + id = Integer.parseInt(dLedgerConfig.getSelfId().substring(1)) + 1; dLedgerServer = new DLedgerServer(dLedgerConfig); dLedgerFileStore = (DLedgerMmapFileStore) dLedgerServer.getdLedgerStore(); DLedgerMmapFileStore.AppendHook appendHook = (entry, buffer, bodyOffset) -> { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java index 24767cc46f7..fafe71c63bc 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java @@ -92,9 +92,9 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) { defaultMQAdminExt.start(); String topic = commandLine.getOptionValue("t").trim(); - int queueId = Integer.valueOf(commandLine.getOptionValue("q").trim()); - long index = Long.valueOf(commandLine.getOptionValue("i").trim()); - int count = Integer.valueOf(commandLine.getOptionValue("c", "10").trim()); + int queueId = Integer.parseInt(commandLine.getOptionValue("q").trim()); + long index = Long.parseLong(commandLine.getOptionValue("i").trim()); + int count = Integer.parseInt(commandLine.getOptionValue("c", "10").trim()); String broker = null; if (commandLine.hasOption("b")) { broker = commandLine.getOptionValue("b").trim(); From b5ade623804e88095f69c1576b7191019a0014ac Mon Sep 17 00:00:00 2001 From: Takshak Ramteke Date: Wed, 8 Dec 2021 12:16:41 +0530 Subject: [PATCH 038/141] [Issue #3565] Removing message filter impldev (#3575) * fix(broker): can not clear topic unitSubFlag (#3429) Co-authored-by: lushilin * doc-optimization (#3523) * Removed the Useless class file MessageFilterImpl. Co-authored-by: lushilin <897401919@qq.com> Co-authored-by: lushilin Co-authored-by: Smile <1494445739@qq.com> --- .../src/main/resources/MessageFilterImpl.java | 39 ------------------- 1 file changed, 39 deletions(-) delete mode 100644 example/src/main/resources/MessageFilterImpl.java diff --git a/example/src/main/resources/MessageFilterImpl.java b/example/src/main/resources/MessageFilterImpl.java deleted file mode 100644 index 6cb5d158a38..00000000000 --- a/example/src/main/resources/MessageFilterImpl.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.rocketmq.example.filter; - -import org.apache.rocketmq.common.filter.FilterContext; -import org.apache.rocketmq.common.filter.MessageFilter; -import org.apache.rocketmq.common.message.MessageExt; - -public class MessageFilterImpl implements MessageFilter { - - @Override - public boolean match(MessageExt msg, FilterContext context) { - String property = msg.getProperty("SequenceId"); - if (property != null) { - int id = Integer.parseInt(property); - if (((id % 10) == 0) && - (id > 100)) { - return true; - } - } - - return false; - } -} From a47f1ef60e24b82a44621b29b82d5e052a8db9e5 Mon Sep 17 00:00:00 2001 From: yuz10 <845238369@qq.com> Date: Wed, 8 Dec 2021 15:07:31 +0800 Subject: [PATCH 039/141] [Issue #3394] fix duplicate keys in trace message (#3395) --- .../client/trace/TraceDataEncoder.java | 5 ++- .../client/trace/TraceDataEncoderTest.java | 33 ++++++++++++++++++- 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/trace/TraceDataEncoder.java b/client/src/main/java/org/apache/rocketmq/client/trace/TraceDataEncoder.java index b4a49a0564b..f0c685e0a64 100644 --- a/client/src/main/java/org/apache/rocketmq/client/trace/TraceDataEncoder.java +++ b/client/src/main/java/org/apache/rocketmq/client/trace/TraceDataEncoder.java @@ -17,9 +17,11 @@ package org.apache.rocketmq.client.trace; import org.apache.rocketmq.client.producer.LocalTransactionState; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageType; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; /** @@ -218,7 +220,8 @@ public static TraceTransferBean encoderFromContextBean(TraceContext ctx) { transferBean.getTransKey().add(bean.getMsgId()); if (bean.getKeys() != null && bean.getKeys().length() > 0) { - transferBean.getTransKey().add(bean.getKeys()); + String[] keys = bean.getKeys().split(MessageConst.KEY_SEPARATOR); + transferBean.getTransKey().addAll(Arrays.asList(keys)); } } return transferBean; diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/TraceDataEncoderTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/TraceDataEncoderTest.java index 03381d01ffc..fed8c4ef767 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/TraceDataEncoderTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/TraceDataEncoderTest.java @@ -25,6 +25,9 @@ import java.util.ArrayList; import java.util.List; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; public class TraceDataEncoderTest { @@ -129,7 +132,7 @@ public void testEncoderFromContextBean_EndTransaction() { Assert.assertEquals(before.getTransactionState(), after.getTransactionState()); Assert.assertEquals(before.isFromTransactionCheck(), after.isFromTransactionCheck()); } - + @Test public void testPubTraceDataFormatTest() { TraceContext pubContext = new TraceContext(); @@ -234,4 +237,32 @@ public void testEndTrxTraceDataFormatTest() { Assert.assertEquals(13, items.length); } + + @Test + public void testTraceKeys() { + TraceContext endTrxContext = new TraceContext(); + endTrxContext.setTraceType(TraceType.EndTransaction); + endTrxContext.setGroupName("PID-test"); + endTrxContext.setRegionId("DefaultRegion"); + endTrxContext.setTimeStamp(time); + TraceBean endTrxTraceBean = new TraceBean(); + endTrxTraceBean.setTopic("topic-test"); + endTrxTraceBean.setKeys("Keys Keys2"); + endTrxTraceBean.setTags("Tags"); + endTrxTraceBean.setMsgId("AC1415116D1418B4AAC217FE1B4E0000"); + endTrxTraceBean.setStoreHost("127.0.0.1:10911"); + endTrxTraceBean.setMsgType(MessageType.Trans_msg_Commit); + endTrxTraceBean.setTransactionId("transactionId"); + endTrxTraceBean.setTransactionState(LocalTransactionState.COMMIT_MESSAGE); + endTrxTraceBean.setFromTransactionCheck(false); + List traceBeans = new ArrayList(); + traceBeans.add(endTrxTraceBean); + endTrxContext.setTraceBeans(traceBeans); + + TraceTransferBean traceTransferBean = TraceDataEncoder.encoderFromContextBean(endTrxContext); + + Set keys = traceTransferBean.getTransKey(); + assertThat(keys).contains("Keys"); + assertThat(keys).contains("Keys2"); + } } From b5a8334e8ac003c62a5e44fd26e9d764f8d2b284 Mon Sep 17 00:00:00 2001 From: tianliuliu <643422162@qq.com> Date: Wed, 8 Dec 2021 16:21:39 +0800 Subject: [PATCH 040/141] [ISSUE #3604] Some important interface methods add comment --- .../apache/rocketmq/store/CommitLogDispatcher.java | 4 ++++ .../rocketmq/store/MessageArrivingListener.java | 11 +++++++++++ 2 files changed, 15 insertions(+) diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java b/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java index e1564a9ef58..9d6fa6ad98b 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java @@ -22,5 +22,9 @@ */ public interface CommitLogDispatcher { + /** + * Dispatch messages from store to build consume queues, indexes, and filter data + * @param request dispatch message request + */ void dispatch(final DispatchRequest request); } diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java b/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java index bae7a161689..ceca98f042e 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageArrivingListener.java @@ -20,6 +20,17 @@ import java.util.Map; public interface MessageArrivingListener { + + /** + * Notify that a new message arrives in a consume queue + * @param topic topic name + * @param queueId consume queue id + * @param logicOffset consume queue offset + * @param tagsCode message tags hash code + * @param msgStoreTime message store time + * @param filterBitMap message bloom filter + * @param properties message properties + */ void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime, byte[] filterBitMap, Map properties); } From f395fa68373c7552ad18fb522da69197bf151669 Mon Sep 17 00:00:00 2001 From: coding Date: Wed, 8 Dec 2021 17:14:26 +0800 Subject: [PATCH 041/141] [ISSUE #3601] Fix dledger put batch msg stats bug (#3600) Co-authored-by: zanglei --- .../org/apache/rocketmq/store/dledger/DLedgerCommitLog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java index 37886730c2d..12b8ec7f878 100644 --- a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java @@ -629,7 +629,7 @@ public CompletableFuture asyncPutMessages(MessageExtBatch mess PutMessageResult putMessageResult = new PutMessageResult(putMessageStatus, appendResult); if (putMessageStatus == PutMessageStatus.PUT_OK) { // Statistics - storeStatsService.getSinglePutMessageTopicTimesTotal(messageExtBatch.getTopic()).add(1); + storeStatsService.getSinglePutMessageTopicTimesTotal(messageExtBatch.getTopic()).add(appendResult.getMsgNum()); storeStatsService.getSinglePutMessageTopicSizeTotal(messageExtBatch.getTopic()).add(appendResult.getWroteBytes()); } return putMessageResult; From e2a4b3778519ef761dac9ce4963eb20905fb35f4 Mon Sep 17 00:00:00 2001 From: lushilin Date: Wed, 8 Dec 2021 15:58:42 +0800 Subject: [PATCH 042/141] fix(client): fetch and commit offset use master broker firstly --- .../store/RemoteBrokerOffsetStore.java | 8 ++--- .../client/impl/factory/MQClientInstance.java | 30 ------------------- .../store/RemoteBrokerOffsetStoreTest.java | 3 +- 3 files changed, 6 insertions(+), 35 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStore.java b/client/src/main/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStore.java index 6b762383717..15b5becfd48 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStore.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStore.java @@ -199,10 +199,10 @@ private void updateConsumeOffsetToBroker(MessageQueue mq, long offset) throws Re @Override public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean isOneway) throws RemotingException, MQBrokerException, InterruptedException, MQClientException { - FindBrokerResult findBrokerResult = this.mQClientFactory.findBrokerAddressInAdmin(mq.getBrokerName()); + FindBrokerResult findBrokerResult = this.mQClientFactory.findBrokerAddressInSubscribe(mq.getBrokerName(), MixAll.MASTER_ID, true); if (null == findBrokerResult) { this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic()); - findBrokerResult = this.mQClientFactory.findBrokerAddressInAdmin(mq.getBrokerName()); + findBrokerResult = this.mQClientFactory.findBrokerAddressInSubscribe(mq.getBrokerName(), MixAll.MASTER_ID, false); } if (findBrokerResult != null) { @@ -226,11 +226,11 @@ public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean is private long fetchConsumeOffsetFromBroker(MessageQueue mq) throws RemotingException, MQBrokerException, InterruptedException, MQClientException { - FindBrokerResult findBrokerResult = this.mQClientFactory.findBrokerAddressInAdmin(mq.getBrokerName()); + FindBrokerResult findBrokerResult = this.mQClientFactory.findBrokerAddressInSubscribe(mq.getBrokerName(), MixAll.MASTER_ID, true); if (null == findBrokerResult) { this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic()); - findBrokerResult = this.mQClientFactory.findBrokerAddressInAdmin(mq.getBrokerName()); + findBrokerResult = this.mQClientFactory.findBrokerAddressInSubscribe(mq.getBrokerName(), MixAll.MASTER_ID, false); } if (findBrokerResult != null) { diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java index 9651943c0ba..db81143396c 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java @@ -967,36 +967,6 @@ public MQConsumerInner selectConsumer(final String group) { return this.consumerTable.get(group); } - public FindBrokerResult findBrokerAddressInAdmin(final String brokerName) { - String brokerAddr = null; - boolean slave = false; - boolean found = false; - - HashMap map = this.brokerAddrTable.get(brokerName); - if (map != null && !map.isEmpty()) { - for (Map.Entry entry : map.entrySet()) { - Long id = entry.getKey(); - brokerAddr = entry.getValue(); - if (brokerAddr != null) { - found = true; - if (MixAll.MASTER_ID == id) { - slave = false; - } else { - slave = true; - } - break; - - } - } // end of for - } - - if (found) { - return new FindBrokerResult(brokerAddr, slave, findBrokerVersion(brokerName, brokerAddr)); - } - - return null; - } - public String findBrokerAddressInPublish(final String brokerName) { HashMap map = this.brokerAddrTable.get(brokerName); if (map != null && !map.isEmpty()) { diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStoreTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStoreTest.java index f762910a401..ec7a4cf0e27 100644 --- a/client/src/test/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStoreTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/store/RemoteBrokerOffsetStoreTest.java @@ -23,6 +23,7 @@ import org.apache.rocketmq.client.impl.FindBrokerResult; import org.apache.rocketmq.client.impl.MQClientAPIImpl; import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.header.QueryConsumerOffsetRequestHeader; import org.apache.rocketmq.common.protocol.header.UpdateConsumerOffsetRequestHeader; @@ -58,7 +59,7 @@ public void init() { System.setProperty("rocketmq.client.localOffsetStoreDir", System.getProperty("java.io.tmpdir") + ".rocketmq_offsets"); String clientId = new ClientConfig().buildMQClientId() + "#TestNamespace" + System.currentTimeMillis(); when(mQClientFactory.getClientId()).thenReturn(clientId); - when(mQClientFactory.findBrokerAddressInAdmin(brokerName)).thenReturn(new FindBrokerResult("127.0.0.1", false)); + when(mQClientFactory.findBrokerAddressInSubscribe(brokerName, MixAll.MASTER_ID, false)).thenReturn(new FindBrokerResult("127.0.0.1", false)); when(mQClientFactory.getMQClientAPIImpl()).thenReturn(mqClientAPI); } From 50d45f23a186051d177990fd00a89d152c5c4dae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=99=E4=BB=A3=E7=A0=81=E7=9A=84=E7=8C=AB?= <33629004+shuangchengsun@users.noreply.github.com> Date: Thu, 9 Dec 2021 10:50:52 +0800 Subject: [PATCH 043/141] [ISSUE 3585] [Part F] eliminate regex match in topic/group name check (#3594) --- .../apache/rocketmq/client/Validators.java | 52 +++++-------------- .../rocketmq/client/ValidatorsTest.java | 2 +- .../rocketmq/common/topic/TopicValidator.java | 47 ++++++++++++----- 3 files changed, 49 insertions(+), 52 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/Validators.java b/client/src/main/java/org/apache/rocketmq/client/Validators.java index e712e2f5e46..19208c31553 100644 --- a/client/src/main/java/org/apache/rocketmq/client/Validators.java +++ b/client/src/main/java/org/apache/rocketmq/client/Validators.java @@ -17,8 +17,8 @@ package org.apache.rocketmq.client; -import java.util.regex.Matcher; -import java.util.regex.Pattern; +import static org.apache.rocketmq.common.topic.TopicValidator.isTopicOrGroupIllegal; + import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.common.UtilAll; @@ -30,23 +30,9 @@ * Common Validator */ public class Validators { - public static final String VALID_PATTERN_STR = "^[%|a-zA-Z0-9_-]+$"; - public static final Pattern PATTERN = Pattern.compile(VALID_PATTERN_STR); public static final int CHARACTER_MAX_LENGTH = 255; public static final int TOPIC_MAX_LENGTH = 127; - /** - * @return The resulting {@code String} - */ - public static String getGroupWithRegularExpression(String origin, String patternStr) { - Pattern pattern = Pattern.compile(patternStr); - Matcher matcher = pattern.matcher(origin); - while (matcher.find()) { - return matcher.group(0); - } - return null; - } - /** * Validate group */ @@ -59,27 +45,15 @@ public static void checkGroup(String group) throws MQClientException { throw new MQClientException("the specified group is longer than group max length 255.", null); } - if (!regularExpressionMatcher(group, PATTERN)) { - throw new MQClientException(String.format( - "the specified group[%s] contains illegal characters, allowing only %s", group, - VALID_PATTERN_STR), null); - } - - } - /** - * @return true if, and only if, the entire origin sequence matches this matcher's pattern - */ - public static boolean regularExpressionMatcher(String origin, Pattern pattern) { - if (pattern == null) { - return true; + if (isTopicOrGroupIllegal(group)) { + throw new MQClientException(String.format( + "the specified group[%s] contains illegal characters, allowing only %s", group, + "^[%|a-zA-Z0-9_-]+$"), null); } - Matcher matcher = pattern.matcher(origin); - return matcher.matches(); } - public static void checkMessage(Message msg, DefaultMQProducer defaultMQProducer) - throws MQClientException { + public static void checkMessage(Message msg, DefaultMQProducer defaultMQProducer) throws MQClientException { if (null == msg) { throw new MQClientException(ResponseCode.MESSAGE_ILLEGAL, "the message is null"); } @@ -107,16 +81,16 @@ public static void checkTopic(String topic) throws MQClientException { throw new MQClientException("The specified topic is blank", null); } - if (!regularExpressionMatcher(topic, PATTERN)) { - throw new MQClientException(String.format( - "The specified topic[%s] contains illegal characters, allowing only %s", topic, - VALID_PATTERN_STR), null); - } - if (topic.length() > TOPIC_MAX_LENGTH) { throw new MQClientException( String.format("The specified topic is longer than topic max length %d.", TOPIC_MAX_LENGTH), null); } + + if (isTopicOrGroupIllegal(topic)) { + throw new MQClientException(String.format( + "The specified topic[%s] contains illegal characters, allowing only %s", topic, + "^[%|a-zA-Z0-9_-]+$"), null); + } } public static void isSystemTopic(String topic) throws MQClientException { diff --git a/client/src/test/java/org/apache/rocketmq/client/ValidatorsTest.java b/client/src/test/java/org/apache/rocketmq/client/ValidatorsTest.java index 343fe4bca69..aa448dc07fb 100644 --- a/client/src/test/java/org/apache/rocketmq/client/ValidatorsTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/ValidatorsTest.java @@ -44,7 +44,7 @@ public void testCheckTopic_HasIllegalCharacters() { Validators.checkTopic(illegalTopic); failBecauseExceptionWasNotThrown(MQClientException.class); } catch (MQClientException e) { - assertThat(e).hasMessageStartingWith(String.format("The specified topic[%s] contains illegal characters, allowing only %s", illegalTopic, Validators.VALID_PATTERN_STR)); + assertThat(e).hasMessageStartingWith(String.format("The specified topic[%s] contains illegal characters, allowing only %s", illegalTopic, "^[%|a-zA-Z0-9_-]+$")); } } diff --git a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java index 7b0a8394a11..c0525fa1585 100644 --- a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java +++ b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java @@ -22,8 +22,6 @@ import java.util.HashSet; import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; public class TopicValidator { @@ -38,9 +36,7 @@ public class TopicValidator { public static final String RMQ_SYS_OFFSET_MOVED_EVENT = "OFFSET_MOVED_EVENT"; public static final String SYSTEM_TOPIC_PREFIX = "rmq_sys_"; - - private static final String VALID_PATTERN_STR = "^[%|a-zA-Z0-9_-]+$"; - private static final Pattern PATTERN = Pattern.compile(VALID_PATTERN_STR); + public static final boolean[] VALID_CHAR_BIT_MAP = new boolean[128]; private static final int TOPIC_MAX_LENGTH = 127; private static final Set SYSTEM_TOPIC_SET = new HashSet(); @@ -62,14 +58,41 @@ public class TopicValidator { SYSTEM_TOPIC_SET.add(RMQ_SYS_OFFSET_MOVED_EVENT); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_SCHEDULE_TOPIC); + + // regex: ^[%|a-zA-Z0-9_-]+$ + // % + VALID_CHAR_BIT_MAP['%'] = true; + // - + VALID_CHAR_BIT_MAP['-'] = true; + // _ + VALID_CHAR_BIT_MAP['_'] = true; + // | + VALID_CHAR_BIT_MAP['|'] = true; + for (int i = 0; i < VALID_CHAR_BIT_MAP.length; i++) { + if (i >= '0' && i <= '9') { + // 0-9 + VALID_CHAR_BIT_MAP[i] = true; + } else if (i >= 'A' && i <= 'Z') { + // A-Z + VALID_CHAR_BIT_MAP[i] = true; + } else if (i >= 'a' && i <= 'z') { + // a-z + VALID_CHAR_BIT_MAP[i] = true; + } + } } - private static boolean regularExpressionMatcher(String origin, Pattern pattern) { - if (pattern == null) { - return true; + public static boolean isTopicOrGroupIllegal(String str) { + int strLen = str.length(); + int len = VALID_CHAR_BIT_MAP.length; + boolean[] bitMap = VALID_CHAR_BIT_MAP; + for (int i = 0; i < strLen; i++) { + char ch = str.charAt(i); + if (ch >= len || !bitMap[ch]) { + return true; + } } - Matcher matcher = pattern.matcher(origin); - return matcher.matches(); + return false; } public static boolean validateTopic(String topic, RemotingCommand response) { @@ -80,9 +103,9 @@ public static boolean validateTopic(String topic, RemotingCommand response) { return false; } - if (!regularExpressionMatcher(topic, PATTERN)) { + if (isTopicOrGroupIllegal(topic)) { response.setCode(ResponseCode.SYSTEM_ERROR); - response.setRemark("The specified topic contains illegal characters, allowing only " + VALID_PATTERN_STR); + response.setRemark("The specified topic contains illegal characters, allowing only ^[%|a-zA-Z0-9_-]+$"); return false; } From 1e314f4100e0207198e4d132d9f0040230403b74 Mon Sep 17 00:00:00 2001 From: zhaohai <33314633+zhaohai1299002788@users.noreply.github.com> Date: Thu, 9 Dec 2021 15:13:13 +0800 Subject: [PATCH 044/141] [ISSUE #3593] entrySet() replace keySet() (#3595) * entrySet() replace keySet() * update variable name * Update AllocateMachineRoomNearby.java * remove set Co-authored-by: zh378814 Co-authored-by: yuz10 <845238369@qq.com> --- .../broker/filter/ConsumerFilterManager.java | 17 +++++++------- .../filter/MessageEvaluationContext.java | 5 +++-- .../consumer/DefaultMQPushConsumer.java | 5 +++-- .../rebalance/AllocateMachineRoomNearby.java | 7 +++--- .../consumer/store/LocalFileOffsetStore.java | 11 +++++----- .../apache/rocketmq/common/Configuration.java | 19 ++++++++-------- .../rocketmq/utils/BeanUtils.java | 9 ++++---- .../test/message/MessageQueueMsg.java | 8 ++++--- .../apache/rocketmq/test/util/FileUtil.java | 7 +++--- .../apache/rocketmq/test/util/MQAdmin.java | 9 ++++---- .../rocketmq/tools/command/CommandUtil.java | 7 +++--- .../broker/GetBrokerConfigCommand.java | 22 ++++++++++--------- .../consumer/GetConsumerConfigSubCommand.java | 10 +++++---- .../command/export/ExportConfigsCommand.java | 7 +++--- .../namesrv/GetNamesrvConfigCommand.java | 10 +++++---- 15 files changed, 86 insertions(+), 67 deletions(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java index 5ccabb9f6d0..5f3b7eb7a6c 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/ConsumerFilterManager.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.broker.filter; +import java.util.Map.Entry; import java.util.concurrent.ConcurrentMap; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.BrokerPathConfigHelper; @@ -158,8 +159,8 @@ public boolean register(final String topic, final String consumerGroup, final St } public void unRegister(final String consumerGroup) { - for (String topic : filterDataByTopic.keySet()) { - this.filterDataByTopic.get(topic).unRegister(consumerGroup); + for (Entry entry : filterDataByTopic.entrySet()) { + entry.getValue().unRegister(consumerGroup); } } @@ -230,15 +231,15 @@ public void decode(final String jsonString) { ConsumerFilterManager load = RemotingSerializable.fromJson(jsonString, ConsumerFilterManager.class); if (load != null && load.filterDataByTopic != null) { boolean bloomChanged = false; - for (String topic : load.filterDataByTopic.keySet()) { - FilterDataMapByTopic dataMapByTopic = load.filterDataByTopic.get(topic); + for (Entry entry : load.filterDataByTopic.entrySet()) { + FilterDataMapByTopic dataMapByTopic = entry.getValue(); if (dataMapByTopic == null) { continue; } - for (String group : dataMapByTopic.getGroupFilterData().keySet()) { + for (Entry groupEntry : dataMapByTopic.getGroupFilterData().entrySet()) { - ConsumerFilterData filterData = dataMapByTopic.getGroupFilterData().get(group); + ConsumerFilterData filterData = groupEntry.getValue(); if (filterData == null) { continue; @@ -246,7 +247,7 @@ public void decode(final String jsonString) { try { filterData.setCompiledExpression( - FilterFactory.INSTANCE.get(filterData.getExpressionType()).compile(filterData.getExpression()) + FilterFactory.INSTANCE.get(filterData.getExpressionType()).compile(filterData.getExpression()) ); } catch (Exception e) { log.error("load filter data error, " + filterData, e); @@ -266,7 +267,7 @@ public void decode(final String jsonString) { // we think all consumers are dead when load long deadTime = System.currentTimeMillis() - 30 * 1000; filterData.setDeadTime( - deadTime <= filterData.getBornTime() ? filterData.getBornTime() : deadTime + deadTime <= filterData.getBornTime() ? filterData.getBornTime() : deadTime ); } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java b/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java index 879d17908f6..84921f0d657 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/filter/MessageEvaluationContext.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; /** * Evaluation context from message. @@ -49,8 +50,8 @@ public Map keyValues() { Map copy = new HashMap(properties.size(), 1); - for (String key : properties.keySet()) { - copy.put(key, properties.get(key)); + for (Entry entry : properties.entrySet()) { + copy.put(entry.getKey(), entry.getValue()); } return copy; diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java index 58cf1346ffa..7fb6dc0999b 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumer.java @@ -18,6 +18,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.QueryResult; @@ -640,8 +641,8 @@ public Map getSubscription() { @Deprecated public void setSubscription(Map subscription) { Map subscriptionWithNamespace = new HashMap(); - for (String topic : subscription.keySet()) { - subscriptionWithNamespace.put(withNamespace(topic), subscription.get(topic)); + for (Entry topicEntry : subscription.entrySet()) { + subscriptionWithNamespace.put(withNamespace(topicEntry.getKey()), topicEntry.getValue()); } this.subscription = subscriptionWithNamespace; } diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMachineRoomNearby.java b/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMachineRoomNearby.java index ec0f7f6489d..8e9267459aa 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMachineRoomNearby.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMachineRoomNearby.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.TreeMap; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.consumer.AllocateMessageQueueStrategy; @@ -115,9 +116,9 @@ public List allocate(String consumerGroup, String currentCID, List } //2.allocate the rest mq to each machine room if there are no consumer alive in that machine room - for (String machineRoom : mr2Mq.keySet()) { - if (!mr2c.containsKey(machineRoom)) { // no alive consumer in the corresponding machine room, so all consumers share these queues - allocateResults.addAll(allocateMessageQueueStrategy.allocate(consumerGroup, currentCID, mr2Mq.get(machineRoom), cidAll)); + for (Entry> machineRoomEntry : mr2Mq.entrySet()) { + if (!mr2c.containsKey(machineRoomEntry.getKey())) { // no alive consumer in the corresponding machine room, so all consumers share these queues + allocateResults.addAll(allocateMessageQueueStrategy.allocate(consumerGroup, currentCID, machineRoomEntry.getValue(), cidAll)); } } diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/store/LocalFileOffsetStore.java b/client/src/main/java/org/apache/rocketmq/client/consumer/store/LocalFileOffsetStore.java index b758ac70609..d380ba058a4 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/store/LocalFileOffsetStore.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/store/LocalFileOffsetStore.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -64,12 +65,12 @@ public void load() throws MQClientException { if (offsetSerializeWrapper != null && offsetSerializeWrapper.getOffsetTable() != null) { offsetTable.putAll(offsetSerializeWrapper.getOffsetTable()); - for (MessageQueue mq : offsetSerializeWrapper.getOffsetTable().keySet()) { - AtomicLong offset = offsetSerializeWrapper.getOffsetTable().get(mq); + for (Entry mqEntry : offsetSerializeWrapper.getOffsetTable().entrySet()) { + AtomicLong offset = mqEntry.getValue(); log.info("load consumer's offset, {} {} {}", - this.groupName, - mq, - offset.get()); + this.groupName, + mqEntry.getKey(), + offset.get()); } } } diff --git a/common/src/main/java/org/apache/rocketmq/common/Configuration.java b/common/src/main/java/org/apache/rocketmq/common/Configuration.java index b9eb46d3802..064b5e61d11 100644 --- a/common/src/main/java/org/apache/rocketmq/common/Configuration.java +++ b/common/src/main/java/org/apache/rocketmq/common/Configuration.java @@ -24,6 +24,7 @@ import java.lang.reflect.Modifier; import java.util.ArrayList; import java.util.List; +import java.util.Map.Entry; import java.util.Properties; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -276,26 +277,26 @@ private String getAllConfigsInternal() { } private void merge(Properties from, Properties to) { - for (Object key : from.keySet()) { - Object fromObj = from.get(key), toObj = to.get(key); + for (Entry next : from.entrySet()) { + Object fromObj = next.getValue(), toObj = to.get(next.getKey()); if (toObj != null && !toObj.equals(fromObj)) { - log.info("Replace, key: {}, value: {} -> {}", key, toObj, fromObj); + log.info("Replace, key: {}, value: {} -> {}", next.getKey(), toObj, fromObj); } - to.put(key, fromObj); + to.put(next.getKey(), fromObj); } } private void mergeIfExist(Properties from, Properties to) { - for (Object key : from.keySet()) { - if (!to.containsKey(key)) { + for (Entry next : from.entrySet()) { + if (!to.containsKey(next.getKey())) { continue; } - Object fromObj = from.get(key), toObj = to.get(key); + Object fromObj = next.getValue(), toObj = to.get(next.getKey()); if (toObj != null && !toObj.equals(fromObj)) { - log.info("Replace, key: {}, value: {} -> {}", key, toObj, fromObj); + log.info("Replace, key: {}, value: {} -> {}", next.getKey(), toObj, fromObj); } - to.put(key, fromObj); + to.put(next.getKey(), fromObj); } } diff --git a/openmessaging/src/main/java/io/openmessaging/rocketmq/utils/BeanUtils.java b/openmessaging/src/main/java/io/openmessaging/rocketmq/utils/BeanUtils.java index ef9236f09c1..9853da815c5 100644 --- a/openmessaging/src/main/java/io/openmessaging/rocketmq/utils/BeanUtils.java +++ b/openmessaging/src/main/java/io/openmessaging/rocketmq/utils/BeanUtils.java @@ -21,6 +21,7 @@ import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; import java.util.Properties; import java.util.Set; import org.apache.commons.lang3.StringUtils; @@ -50,10 +51,10 @@ public final class BeanUtils { private static Map, Class> wrapperMap = new HashMap, Class>(); static { - for (final Class primitiveClass : primitiveWrapperMap.keySet()) { - final Class wrapperClass = primitiveWrapperMap.get(primitiveClass); - if (!primitiveClass.equals(wrapperClass)) { - wrapperMap.put(wrapperClass, primitiveClass); + for (Entry, Class> primitiveClass : primitiveWrapperMap.entrySet()) { + final Class wrapperClass = primitiveClass.getValue(); + if (!primitiveClass.getKey().equals(wrapperClass)) { + wrapperMap.put(wrapperClass, primitiveClass.getKey()); } } wrapperMap.put(String.class, String.class); diff --git a/test/src/main/java/org/apache/rocketmq/test/message/MessageQueueMsg.java b/test/src/main/java/org/apache/rocketmq/test/message/MessageQueueMsg.java index 38c2b6b7cee..b500bd53c39 100644 --- a/test/src/main/java/org/apache/rocketmq/test/message/MessageQueueMsg.java +++ b/test/src/main/java/org/apache/rocketmq/test/message/MessageQueueMsg.java @@ -22,6 +22,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; + import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.test.factory.MQMessageFactory; @@ -54,9 +56,9 @@ public Collection getMsgBodys() { } private void init() { - for (MessageQueue mq : msgsWithMQ.keySet()) { - msgsWithMQId.put(mq.getQueueId(), msgsWithMQ.get(mq)); - msgBodys.addAll(MQMessageFactory.getMessageBody(msgsWithMQ.get(mq))); + for (Entry> mqEntry : msgsWithMQ.entrySet()) { + msgsWithMQId.put(mqEntry.getKey().getQueueId(), mqEntry.getValue()); + msgBodys.addAll(MQMessageFactory.getMessageBody(mqEntry.getValue())); } } } diff --git a/test/src/main/java/org/apache/rocketmq/test/util/FileUtil.java b/test/src/main/java/org/apache/rocketmq/test/util/FileUtil.java index e08967eb91d..b6106e00a6c 100644 --- a/test/src/main/java/org/apache/rocketmq/test/util/FileUtil.java +++ b/test/src/main/java/org/apache/rocketmq/test/util/FileUtil.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; +import java.util.Map.Entry; import java.util.Properties; public class FileUtil { @@ -68,9 +69,9 @@ public void writeProperties(Properties properties) { private String getPropertiesAsString(Properties properties) { StringBuilder sb = new StringBuilder(); - for (Object key : properties.keySet()) { - sb.append(key).append("=").append(properties.getProperty((String) key)) - .append(lineSeperator); + for (Entry keyEnty : properties.entrySet()) { + sb.append(keyEnty.getKey()).append("=").append((String) keyEnty.getValue()) + .append(lineSeperator); } return sb.toString(); } diff --git a/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java b/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java index 8863ee3e52d..f336ddd2bb4 100644 --- a/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java +++ b/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java @@ -18,6 +18,7 @@ package org.apache.rocketmq.test.util; import java.util.HashMap; +import java.util.Map.Entry; import java.util.Set; import java.util.UUID; import org.apache.log4j.Logger; @@ -123,10 +124,10 @@ public static boolean isBrokerExist(String ns, String ip) { return false; } else { HashMap brokers = clusterInfo.getBrokerAddrTable(); - for (String brokerName : brokers.keySet()) { - HashMap brokerIps = brokers.get(brokerName).getBrokerAddrs(); - for (long brokerId : brokerIps.keySet()) { - if (brokerIps.get(brokerId).contains(ip)) + for (Entry brokerEntry : brokers.entrySet()) { + HashMap brokerIps = brokerEntry.getValue().getBrokerAddrs(); + for (Entry brokerIdEntry : brokerIps.entrySet()) { + if (brokerIdEntry.getValue().contains(ip)) return true; } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java b/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java index 8984ca67b20..cdf5f329afe 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/CommandUtil.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.common.MixAll; @@ -63,12 +64,12 @@ public class CommandUtil { String masterAddr = brokerData.getBrokerAddrs().get(MixAll.MASTER_ID); masterAndSlaveMap.put(masterAddr, new ArrayList()); - for (Long id : brokerData.getBrokerAddrs().keySet()) { - if (brokerData.getBrokerAddrs().get(id) == null || id == MixAll.MASTER_ID) { + for (Entry brokerAddrEntry : brokerData.getBrokerAddrs().entrySet()) { + if (brokerAddrEntry.getValue() == null || brokerAddrEntry.getKey() == MixAll.MASTER_ID) { continue; } - masterAndSlaveMap.get(masterAddr).add(brokerData.getBrokerAddrs().get(id)); + masterAndSlaveMap.get(masterAddr).add(brokerAddrEntry.getValue()); } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java index 11346bf6f7f..6bacd3cb650 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java @@ -20,7 +20,9 @@ import java.io.UnsupportedEncodingException; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Properties; + import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -83,19 +85,19 @@ public void execute(final CommandLine commandLine, final Options options, Map> masterAndSlaveMap = CommandUtil.fetchMasterAndSlaveDistinguish(defaultMQAdminExt, clusterName); - for (String masterAddr : masterAndSlaveMap.keySet()) { + for (Entry> masterAndSlaveEntry : masterAndSlaveMap.entrySet()) { getAndPrint( - defaultMQAdminExt, - String.format("============Master: %s============\n", masterAddr), - masterAddr + defaultMQAdminExt, + String.format("============Master: %s============\n", masterAndSlaveEntry.getKey()), + masterAndSlaveEntry.getKey() ); - for (String slaveAddr : masterAndSlaveMap.get(masterAddr)) { + for (String slaveAddr : masterAndSlaveEntry.getValue()) { getAndPrint( - defaultMQAdminExt, - String.format("============My Master: %s=====Slave: %s============\n", masterAddr, slaveAddr), - slaveAddr + defaultMQAdminExt, + String.format("============My Master: %s=====Slave: %s============\n", masterAndSlaveEntry.getKey(), slaveAddr), + slaveAddr ); } } @@ -121,8 +123,8 @@ protected void getAndPrint(final MQAdminExt defaultMQAdminExt, final String prin return; } - for (Object key : properties.keySet()) { - System.out.printf("%-50s= %s\n", key, properties.get(key)); + for (Entry entry : properties.entrySet()) { + System.out.printf("%-50s= %s\n", entry.getKey(), entry.getValue()); } System.out.printf("%n"); diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java index be2b9466881..61e3611a5b4 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java @@ -20,12 +20,14 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.commons.collections.CollectionUtils; import org.apache.rocketmq.common.protocol.body.ClusterInfo; +import org.apache.rocketmq.common.protocol.route.BrokerData; import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; @@ -63,14 +65,14 @@ public void execute(CommandLine commandLine, Options options, List consumerConfigInfoList = new ArrayList<>(); ClusterInfo clusterInfo = adminExt.examineBrokerClusterInfo(); Map> clusterAddrTable = clusterInfo.getClusterAddrTable(); - for (String brokerName : clusterInfo.getBrokerAddrTable().keySet()) { - String clusterName = this.getClusterName(brokerName, clusterAddrTable); - String brokerAddress = clusterInfo.getBrokerAddrTable().get(brokerName).selectBrokerAddr(); + for (Entry brokerEntry : clusterInfo.getBrokerAddrTable().entrySet()) { + String clusterName = this.getClusterName(brokerEntry.getKey(), clusterAddrTable); + String brokerAddress = brokerEntry.getValue().selectBrokerAddr(); SubscriptionGroupConfig subscriptionGroupConfig = adminExt.examineSubscriptionGroupConfig(brokerAddress, groupName); if (subscriptionGroupConfig == null) { continue; } - consumerConfigInfoList.add(new ConsumerConfigInfo(clusterName, brokerName, subscriptionGroupConfig)); + consumerConfigInfoList.add(new ConsumerConfigInfo(clusterName, brokerEntry.getKey(), subscriptionGroupConfig)); } if (CollectionUtils.isEmpty(consumerConfigInfoList)) { return; diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java index c3ca9d31902..b2d3fdd92e6 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java @@ -19,6 +19,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Properties; import com.alibaba.fastjson.JSON; @@ -79,10 +80,10 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) Map brokerConfigs = new HashMap<>(); Map> masterAndSlaveMap = CommandUtil.fetchMasterAndSlaveDistinguish(defaultMQAdminExt, clusterName); - for (String masterAddr : masterAndSlaveMap.keySet()) { - Properties masterProperties = defaultMQAdminExt.getBrokerConfig(masterAddr); + for (Entry> masterAndSlaveEntry : masterAndSlaveMap.entrySet()) { + Properties masterProperties = defaultMQAdminExt.getBrokerConfig(masterAndSlaveEntry.getKey()); masterBrokerSize++; - slaveBrokerSize += masterAndSlaveMap.get(masterAddr).size(); + slaveBrokerSize += masterAndSlaveEntry.getValue().size(); brokerConfigs.put(masterProperties.getProperty("brokerName"), needBrokerProprties(masterProperties)); } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommand.java index 22ce867b227..25c8d084922 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommand.java @@ -20,7 +20,9 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Properties; + import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.rocketmq.remoting.RPCHook; @@ -66,11 +68,11 @@ public void execute(final CommandLine commandLine, final Options options, Map nameServerConfigs = defaultMQAdminExt.getNameServerConfig(serverList); - for (String server : nameServerConfigs.keySet()) { + for (Entry nameServerConfigEntry : nameServerConfigs.entrySet()) { System.out.printf("============%s============\n", - server); - for (Object key : nameServerConfigs.get(server).keySet()) { - System.out.printf("%-50s= %s\n", key, nameServerConfigs.get(server).get(key)); + nameServerConfigEntry.getKey()); + for (Entry entry : nameServerConfigEntry.getValue().entrySet()) { + System.out.printf("%-50s= %s\n", entry.getKey(), entry.getValue()); } } } catch (Exception e) { From 38b20fb6a1a984ddee9b69161e1a2710c404fc08 Mon Sep 17 00:00:00 2001 From: coding Date: Fri, 10 Dec 2021 10:06:40 +0800 Subject: [PATCH 045/141] [ISSUE 3585] [Part H] Avoid unnecessary StringBuilder resizing on critical path (#3612) Co-authored-by: zanglei --- .../rocketmq/broker/longpolling/PullRequestHoldService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java index 417ec0dfd67..866f357d897 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java @@ -56,7 +56,7 @@ public void suspendPullRequest(final String topic, final int queueId, final Pull } private String buildKey(final String topic, final int queueId) { - StringBuilder sb = new StringBuilder(); + StringBuilder sb = new StringBuilder(topic.length() + 5); sb.append(topic); sb.append(TOPIC_QUEUEID_SEPARATOR); sb.append(queueId); From 7f1c14c3c15ebd734cc08755b171a0003c9916b9 Mon Sep 17 00:00:00 2001 From: panzhi Date: Fri, 10 Dec 2021 10:57:35 +0800 Subject: [PATCH 046/141] [ISSUE #3453]Producer cannot shutdown completely (#3454) * [ISSUE #3453]Producer cannot shutdown completely * [ISSUE #3453]Producer cannot shutdown completely * [ISSUE #3453]Producer cannot shutdown completely Co-authored-by: panzhi33 --- .../client/impl/ClientRemotingProcessor.java | 6 +-- .../impl/producer/DefaultMQProducerImpl.java | 40 ++++++++----------- ...ureTable.java => RequestFutureHolder.java} | 36 +++++++++++++---- .../producer/DefaultMQProducerTest.java | 6 +-- 4 files changed, 51 insertions(+), 37 deletions(-) rename client/src/main/java/org/apache/rocketmq/client/producer/{RequestFutureTable.java => RequestFutureHolder.java} (66%) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/ClientRemotingProcessor.java b/client/src/main/java/org/apache/rocketmq/client/impl/ClientRemotingProcessor.java index a541b2d379c..b90a5b7908e 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/ClientRemotingProcessor.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/ClientRemotingProcessor.java @@ -25,7 +25,7 @@ import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.client.impl.producer.MQProducerInner; import org.apache.rocketmq.client.log.ClientLogger; -import org.apache.rocketmq.client.producer.RequestFutureTable; +import org.apache.rocketmq.client.producer.RequestFutureHolder; import org.apache.rocketmq.client.producer.RequestResponseFuture; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.message.MessageAccessor; @@ -274,11 +274,11 @@ private RemotingCommand receiveReplyMessage(ChannelHandlerContext ctx, private void processReplyMessage(MessageExt replyMsg) { final String correlationId = replyMsg.getUserProperty(MessageConst.PROPERTY_CORRELATION_ID); - final RequestResponseFuture requestResponseFuture = RequestFutureTable.getRequestFutureTable().get(correlationId); + final RequestResponseFuture requestResponseFuture = RequestFutureHolder.getInstance().getRequestFutureTable().get(correlationId); if (requestResponseFuture != null) { requestResponseFuture.putResponseMessage(replyMsg); - RequestFutureTable.getRequestFutureTable().remove(correlationId); + RequestFutureHolder.getInstance().getRequestFutureTable().remove(correlationId); if (requestResponseFuture.getRequestCallback() != null) { requestResponseFuture.getRequestCallback().onSuccess(replyMsg); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java index 9bbde41568b..b27117fbbf5 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java @@ -28,10 +28,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -58,7 +56,7 @@ import org.apache.rocketmq.client.producer.LocalTransactionState; import org.apache.rocketmq.client.producer.MessageQueueSelector; import org.apache.rocketmq.client.producer.RequestCallback; -import org.apache.rocketmq.client.producer.RequestFutureTable; +import org.apache.rocketmq.client.producer.RequestFutureHolder; import org.apache.rocketmq.client.producer.RequestResponseFuture; import org.apache.rocketmq.client.producer.SendCallback; import org.apache.rocketmq.client.producer.SendResult; @@ -106,12 +104,6 @@ public class DefaultMQProducerImpl implements MQProducerInner { private final RPCHook rpcHook; private final BlockingQueue asyncSenderThreadPoolQueue; private final ExecutorService defaultAsyncSenderExecutor; - private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { - @Override - public Thread newThread(Runnable r) { - return new Thread(r, "RequestHouseKeepingService"); - } - }); protected BlockingQueue checkRequestQueue; protected ExecutorService checkExecutor; private ServiceState serviceState = ServiceState.CREATE_JUST; @@ -236,12 +228,12 @@ public void start(final boolean startFactory) throws MQClientException { } private void startScheduledTask() { - if (RequestFutureTable.getProducerNum().incrementAndGet() == 1) { - this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { + if (RequestFutureHolder.getInstance().getProducerNum().incrementAndGet() == 1) { + RequestFutureHolder.getInstance().getScheduledExecutorService().scheduleAtFixedRate(new Runnable() { @Override public void run() { try { - RequestFutureTable.scanExpiredRequest(); + RequestFutureHolder.getInstance().scanExpiredRequest(); } catch (Throwable e) { log.error("scan RequestFutureTable exception", e); } @@ -277,8 +269,8 @@ public void shutdown(final boolean shutdownFactory) { if (shutdownFactory) { this.mQClientFactory.shutdown(); } - if (RequestFutureTable.getProducerNum().decrementAndGet() == 0) { - scheduledExecutorService.shutdown(); + if (RequestFutureHolder.getInstance().getProducerNum().decrementAndGet() == 0) { + RequestFutureHolder.getInstance().getScheduledExecutorService().shutdown(); } log.info("the producer [{}] shutdown OK", this.defaultMQProducer.getProducerGroup()); this.serviceState = ServiceState.SHUTDOWN_ALREADY; @@ -1380,7 +1372,7 @@ public Message request(Message msg, try { final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, null); - RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture); + RequestFutureHolder.getInstance().getRequestFutureTable().put(correlationId, requestResponseFuture); long cost = System.currentTimeMillis() - beginTimestamp; this.sendDefaultImpl(msg, CommunicationMode.ASYNC, new SendCallback() { @@ -1399,7 +1391,7 @@ public void onException(Throwable e) { return waitResponse(msg, timeout, requestResponseFuture, cost); } finally { - RequestFutureTable.getRequestFutureTable().remove(correlationId); + RequestFutureHolder.getInstance().getRequestFutureTable().remove(correlationId); } } @@ -1410,7 +1402,7 @@ public void request(Message msg, final RequestCallback requestCallback, long tim final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID); final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, requestCallback); - RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture); + RequestFutureHolder.getInstance().getRequestFutureTable().put(correlationId, requestResponseFuture); long cost = System.currentTimeMillis() - beginTimestamp; this.sendDefaultImpl(msg, CommunicationMode.ASYNC, new SendCallback() { @@ -1436,7 +1428,7 @@ public Message request(final Message msg, final MessageQueueSelector selector, f try { final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, null); - RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture); + RequestFutureHolder.getInstance().getRequestFutureTable().put(correlationId, requestResponseFuture); long cost = System.currentTimeMillis() - beginTimestamp; this.sendSelectImpl(msg, selector, arg, CommunicationMode.ASYNC, new SendCallback() { @@ -1455,7 +1447,7 @@ public void onException(Throwable e) { return waitResponse(msg, timeout, requestResponseFuture, cost); } finally { - RequestFutureTable.getRequestFutureTable().remove(correlationId); + RequestFutureHolder.getInstance().getRequestFutureTable().remove(correlationId); } } @@ -1467,7 +1459,7 @@ public void request(final Message msg, final MessageQueueSelector selector, fina final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID); final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, requestCallback); - RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture); + RequestFutureHolder.getInstance().getRequestFutureTable().put(correlationId, requestResponseFuture); long cost = System.currentTimeMillis() - beginTimestamp; this.sendSelectImpl(msg, selector, arg, CommunicationMode.ASYNC, new SendCallback() { @@ -1493,7 +1485,7 @@ public Message request(final Message msg, final MessageQueue mq, final long time try { final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, null); - RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture); + RequestFutureHolder.getInstance().getRequestFutureTable().put(correlationId, requestResponseFuture); long cost = System.currentTimeMillis() - beginTimestamp; this.sendKernelImpl(msg, mq, CommunicationMode.ASYNC, new SendCallback() { @@ -1512,7 +1504,7 @@ public void onException(Throwable e) { return waitResponse(msg, timeout, requestResponseFuture, cost); } finally { - RequestFutureTable.getRequestFutureTable().remove(correlationId); + RequestFutureHolder.getInstance().getRequestFutureTable().remove(correlationId); } } @@ -1536,7 +1528,7 @@ public void request(final Message msg, final MessageQueue mq, final RequestCallb final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID); final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, requestCallback); - RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture); + RequestFutureHolder.getInstance().getRequestFutureTable().put(correlationId, requestResponseFuture); long cost = System.currentTimeMillis() - beginTimestamp; this.sendKernelImpl(msg, mq, CommunicationMode.ASYNC, new SendCallback() { @@ -1554,7 +1546,7 @@ public void onException(Throwable e) { } private void requestFail(final String correlationId) { - RequestResponseFuture responseFuture = RequestFutureTable.getRequestFutureTable().remove(correlationId); + RequestResponseFuture responseFuture = RequestFutureHolder.getInstance().getRequestFutureTable().remove(correlationId); if (responseFuture != null) { responseFuture.setSendRequestOk(false); responseFuture.putResponseMessage(null); diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureTable.java b/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java similarity index 66% rename from client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureTable.java rename to client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java index 9bdc3da45ef..24b3a906a56 100644 --- a/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureTable.java +++ b/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java @@ -22,6 +22,9 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; import org.apache.rocketmq.client.common.ClientErrorCode; @@ -29,16 +32,24 @@ import org.apache.rocketmq.client.log.ClientLogger; import org.apache.rocketmq.logging.InternalLogger; -public class RequestFutureTable { +public class RequestFutureHolder { private static InternalLogger log = ClientLogger.getLog(); - private static ConcurrentHashMap requestFutureTable = new ConcurrentHashMap(); - private static final AtomicInteger PRODUCER_NUM = new AtomicInteger(0); + private static final RequestFutureHolder INSTANCE = new RequestFutureHolder(); + private ConcurrentHashMap requestFutureTable = new ConcurrentHashMap(); + private final AtomicInteger producerNum = new AtomicInteger(0); + private final ScheduledExecutorService scheduledExecutorService = Executors + .newSingleThreadScheduledExecutor(new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(r, "RequestHouseKeepingService"); + } + }); - public static ConcurrentHashMap getRequestFutureTable() { + public ConcurrentHashMap getRequestFutureTable() { return requestFutureTable; } - public static void scanExpiredRequest() { + public void scanExpiredRequest() { final List rfList = new LinkedList(); Iterator> it = requestFutureTable.entrySet().iterator(); while (it.hasNext()) { @@ -63,7 +74,18 @@ public static void scanExpiredRequest() { } } - public static AtomicInteger getProducerNum() { - return PRODUCER_NUM; + private RequestFutureHolder() { + } + + public AtomicInteger getProducerNum() { + return producerNum; + } + + public ScheduledExecutorService getScheduledExecutorService() { + return scheduledExecutorService; + } + + public static RequestFutureHolder getInstance() { + return INSTANCE; } } diff --git a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java index e161d4ab784..a890e721dcd 100644 --- a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java @@ -369,7 +369,7 @@ public void testRequestMessage() throws RemotingException, RequestTimeoutExcepti final AtomicBoolean finish = new AtomicBoolean(false); new Thread(new Runnable() { @Override public void run() { - ConcurrentHashMap responseMap = RequestFutureTable.getRequestFutureTable(); + ConcurrentHashMap responseMap = RequestFutureHolder.getInstance().getRequestFutureTable(); assertThat(responseMap).isNotNull(); while (!finish.get()) { try { @@ -411,7 +411,7 @@ public void testAsyncRequest_OnSuccess() throws Exception { } }; producer.request(message, requestCallback, 3 * 1000L); - ConcurrentHashMap responseMap = RequestFutureTable.getRequestFutureTable(); + ConcurrentHashMap responseMap = RequestFutureHolder.getInstance().getRequestFutureTable(); assertThat(responseMap).isNotNull(); for (Map.Entry entry : responseMap.entrySet()) { RequestResponseFuture future = entry.getValue(); @@ -447,7 +447,7 @@ public MessageQueue select(List mqs, Message msg, Object arg) { producer.request(message, requestCallback, 3 * 1000L); failBecauseExceptionWasNotThrown(Exception.class); } catch (Exception e) { - ConcurrentHashMap responseMap = RequestFutureTable.getRequestFutureTable(); + ConcurrentHashMap responseMap = RequestFutureHolder.getInstance().getRequestFutureTable(); assertThat(responseMap).isNotNull(); for (Map.Entry entry : responseMap.entrySet()) { RequestResponseFuture future = entry.getValue(); From cbf9b2a290a44669a8dceaadda994a345c5f41eb Mon Sep 17 00:00:00 2001 From: zhaohai <33314633+zhaohai1299002788@users.noreply.github.com> Date: Fri, 10 Dec 2021 11:19:19 +0800 Subject: [PATCH 047/141] [ISSUE #3602] add AdminBrokerProcessor Test (#3599) * add test * add testGetAllDelayOffset * remove test Co-authored-by: zh378814 --- .../processor/AdminBrokerProcessorTest.java | 264 ++++++++++++++++++ 1 file changed, 264 insertions(+) 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 2764761d39f..a2abdc02fed 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 @@ -16,21 +16,39 @@ */ package org.apache.rocketmq.broker.processor; +import com.alibaba.fastjson.JSON; import com.google.common.collect.Sets; +import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import org.apache.rocketmq.broker.BrokerController; +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.topic.TopicConfigManager; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.TopicFilterType; import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; +import org.apache.rocketmq.common.protocol.body.LockBatchRequestBody; +import org.apache.rocketmq.common.protocol.body.UnlockBatchRequestBody; import org.apache.rocketmq.common.protocol.header.CreateTopicRequestHeader; import org.apache.rocketmq.common.protocol.header.DeleteTopicRequestHeader; +import org.apache.rocketmq.common.protocol.header.GetAllTopicConfigResponseHeader; +import org.apache.rocketmq.common.protocol.header.GetEarliestMsgStoretimeRequestHeader; +import org.apache.rocketmq.common.protocol.header.GetMaxOffsetRequestHeader; +import org.apache.rocketmq.common.protocol.header.GetMinOffsetRequestHeader; import org.apache.rocketmq.common.protocol.header.ResumeCheckHalfMessageRequestHeader; +import org.apache.rocketmq.common.protocol.header.SearchOffsetRequestHeader; +import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType; +import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; +import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.netty.NettyClientConfig; @@ -38,6 +56,7 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.store.AppendMessageResult; import org.apache.rocketmq.store.AppendMessageStatus; +import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.store.MappedFile; import org.apache.rocketmq.store.MessageExtBrokerInner; import org.apache.rocketmq.store.MessageStore; @@ -45,6 +64,8 @@ import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.SelectMappedBufferResult; import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.schedule.ScheduleMessageService; +import org.apache.rocketmq.store.stats.BrokerStats; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,12 +73,19 @@ import org.mockito.Spy; import org.mockito.junit.MockitoJUnitRunner; +import java.net.SocketAddress; import java.net.UnknownHostException; import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; import java.util.Set; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) @@ -78,6 +106,23 @@ public class AdminBrokerProcessorTest { private Set systemTopicSet; + @Mock + private Channel channel; + @Mock + private SocketAddress socketAddress; + @Mock + private BrokerStats brokerStats; + @Mock + private TopicConfigManager topicConfigManager; + @Mock + private ConsumerManager consumerManager; + @Mock + private ConsumerOffsetManager consumerOffsetManager; + @Mock + private DefaultMessageStore defaultMessageStore; + @Mock + private ScheduleMessageService scheduleMessageService; + @Before public void init() { brokerController.setMessageStore(messageStore); @@ -155,6 +200,225 @@ public void testDeleteTopic() throws Exception { assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); } + @Test + public void testGetAllTopicConfig() throws Exception { + GetAllTopicConfigResponseHeader getAllTopicConfigResponseHeader = new GetAllTopicConfigResponseHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_TOPIC_CONFIG, getAllTopicConfigResponseHeader); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testUpdateBrokerConfig() throws Exception { + handlerContext = mock(ChannelHandlerContext.class); + channel = mock(Channel.class); + when(handlerContext.channel()).thenReturn(channel); + socketAddress = mock(SocketAddress.class); + when(channel.remoteAddress()).thenReturn(socketAddress); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_BROKER_CONFIG, null); + Map bodyMap = new HashMap<>(); + bodyMap.put("key", "value"); + request.setBody(bodyMap.toString().getBytes()); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetBrokerConfig() throws Exception { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CONFIG, null); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testSearchOffsetByTimestamp() throws Exception { + messageStore = mock(MessageStore.class); + when(messageStore.getOffsetInQueueByTime(anyString(), anyInt(), anyLong())).thenReturn(Long.MIN_VALUE); + when(brokerController.getMessageStore()).thenReturn(messageStore); + SearchOffsetRequestHeader searchOffsetRequestHeader = new SearchOffsetRequestHeader(); + searchOffsetRequestHeader.setTopic("topic"); + searchOffsetRequestHeader.setQueueId(0); + searchOffsetRequestHeader.setTimestamp(System.currentTimeMillis()); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, searchOffsetRequestHeader); + request.addExtField("topic", "topic"); + request.addExtField("queueId", "0"); + request.addExtField("timestamp", System.currentTimeMillis() + ""); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetMaxOffset() throws Exception { + messageStore = mock(MessageStore.class); + when(messageStore.getMaxOffsetInQueue(anyString(), anyInt())).thenReturn(Long.MIN_VALUE); + when(brokerController.getMessageStore()).thenReturn(messageStore); + GetMaxOffsetRequestHeader getMaxOffsetRequestHeader = new GetMaxOffsetRequestHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_MAX_OFFSET, getMaxOffsetRequestHeader); + request.addExtField("topic", "topic"); + request.addExtField("queueId", "0"); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetMinOffset() throws Exception { + messageStore = mock(MessageStore.class); + when(messageStore.getMinOffsetInQueue(anyString(), anyInt())).thenReturn(Long.MIN_VALUE); + when(brokerController.getMessageStore()).thenReturn(messageStore); + GetMinOffsetRequestHeader getMinOffsetRequestHeader = new GetMinOffsetRequestHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_MIN_OFFSET, getMinOffsetRequestHeader); + request.addExtField("topic", "topic"); + request.addExtField("queueId", "0"); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetEarliestMsgStoretime() throws Exception { + messageStore = mock(MessageStore.class); + when(brokerController.getMessageStore()).thenReturn(messageStore); + GetEarliestMsgStoretimeRequestHeader getEarliestMsgStoretimeRequestHeader = new GetEarliestMsgStoretimeRequestHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_EARLIEST_MSG_STORETIME, getEarliestMsgStoretimeRequestHeader); + request.addExtField("topic", "topic"); + request.addExtField("queueId", "0"); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetBrokerRuntimeInfo() throws Exception { + brokerStats = mock(BrokerStats.class); + when(brokerController.getBrokerStats()).thenReturn(brokerStats); + when(brokerStats.getMsgPutTotalYesterdayMorning()).thenReturn(Long.MIN_VALUE); + when(brokerStats.getMsgPutTotalTodayMorning()).thenReturn(Long.MIN_VALUE); + when(brokerStats.getMsgPutTotalTodayNow()).thenReturn(Long.MIN_VALUE); + when(brokerStats.getMsgGetTotalTodayMorning()).thenReturn(Long.MIN_VALUE); + when(brokerStats.getMsgGetTotalTodayNow()).thenReturn(Long.MIN_VALUE); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_RUNTIME_INFO, null); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testLockBatchMQ() throws Exception { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.LOCK_BATCH_MQ, null); + LockBatchRequestBody lockBatchRequestBody = new LockBatchRequestBody(); + lockBatchRequestBody.setClientId("1111"); + lockBatchRequestBody.setConsumerGroup("group"); + request.setBody(JSON.toJSON(lockBatchRequestBody).toString().getBytes()); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testUnlockBatchMQ() throws Exception { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UNLOCK_BATCH_MQ, null); + UnlockBatchRequestBody unlockBatchRequestBody = new UnlockBatchRequestBody(); + unlockBatchRequestBody.setClientId("11111"); + unlockBatchRequestBody.setConsumerGroup("group"); + request.setBody(JSON.toJSON(unlockBatchRequestBody).toString().getBytes()); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testUpdateAndCreateSubscriptionGroup() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_SUBSCRIPTIONGROUP, null); + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setBrokerId(1); + subscriptionGroupConfig.setGroupName("groupId"); + subscriptionGroupConfig.setConsumeEnable(Boolean.TRUE); + subscriptionGroupConfig.setConsumeBroadcastEnable(Boolean.TRUE); + subscriptionGroupConfig.setRetryMaxTimes(111); + subscriptionGroupConfig.setConsumeFromMinEnable(Boolean.TRUE); + request.setBody(JSON.toJSON(subscriptionGroupConfig).toString().getBytes()); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetAllSubscriptionGroup() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_SUBSCRIPTIONGROUP_CONFIG, null); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testDeleteSubscriptionGroup() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_SUBSCRIPTIONGROUP, null); + request.addExtField("groupName", "GID-Group-Name"); + request.addExtField("removeOffset", "true"); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetTopicStatsInfo() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPIC_STATS_INFO, null); + request.addExtField("topic", "topicTest"); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.TOPIC_NOT_EXIST); + topicConfigManager = mock(TopicConfigManager.class); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("topicTest"); + when(topicConfigManager.selectTopicConfig(anyString())).thenReturn(topicConfig); + RemotingCommand responseSuccess = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(responseSuccess.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetConsumerConnectionList() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_CONNECTION_LIST, null); + request.addExtField("consumerGroup", "GID-group-test"); + consumerManager = mock(ConsumerManager.class); + when(brokerController.getConsumerManager()).thenReturn(consumerManager); + ConsumerGroupInfo consumerGroupInfo = new ConsumerGroupInfo("GID-group-test", ConsumeType.CONSUME_ACTIVELY, MessageModel.CLUSTERING, ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + when(consumerManager.getConsumerGroupInfo(anyString())).thenReturn(consumerGroupInfo); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetProducerConnectionList() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_PRODUCER_CONNECTION_LIST, null); + request.addExtField("producerGroup", "ProducerGroupId"); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR); + } + + @Test + public void testGetConsumeStats() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUME_STATS, null); + request.addExtField("topic", "topicTest"); + request.addExtField("consumerGroup", "GID-test"); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetAllConsumerOffset() throws RemotingCommandException { + consumerOffsetManager = mock(ConsumerOffsetManager.class); + when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); + ConsumerOffsetManager consumerOffset = new ConsumerOffsetManager(); + when(consumerOffsetManager.encode()).thenReturn(JSON.toJSONString(consumerOffset, false)); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_CONSUMER_OFFSET, null); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + + @Test + public void testGetAllDelayOffset() throws Exception { + defaultMessageStore = mock(DefaultMessageStore.class); + scheduleMessageService = mock(ScheduleMessageService.class); + when(brokerController.getMessageStore()).thenReturn(defaultMessageStore); + when(defaultMessageStore.getScheduleMessageService()).thenReturn(scheduleMessageService); + when(scheduleMessageService.encode()).thenReturn("content"); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_DELAY_OFFSET, null); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); + } + private RemotingCommand buildCreateTopicRequest(String topic) { CreateTopicRequestHeader requestHeader = new CreateTopicRequestHeader(); requestHeader.setTopic(topic); From 8b5832864518970150a83c026f5f5c0d352ca70e Mon Sep 17 00:00:00 2001 From: Frank Jia <474165491@qq.com> Date: Fri, 10 Dec 2021 15:27:56 +0800 Subject: [PATCH 048/141] Polish the doc for tracing parts. (#3573) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * fix(broker): can not clear topic unitSubFlag (#3429) Co-authored-by: lushilin * doc-optimization (#3523) * 完善BrokerServer部分描述细节 完善BrokerServer部分描述细节 Co-authored-by: lushilin <897401919@qq.com> Co-authored-by: lushilin Co-authored-by: Smile <1494445739@qq.com> --- docs/cn/architecture.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/cn/architecture.md b/docs/cn/architecture.md index 42df9f07b29..942464f94ac 100644 --- a/docs/cn/architecture.md +++ b/docs/cn/architecture.md @@ -9,11 +9,11 @@ RocketMQ架构上主要分为四部分,如上图所示: - Consumer:消息消费的角色,支持分布式集群方式部署。支持以push推,pull拉两种模式对消息进行消费。同时也支持集群方式和广播方式的消费,它提供实时消息订阅机制,可以满足大多数用户的需求。 -- NameServer:NameServer是一个非常简单的Topic路由注册中心,其角色类似Dubbo中的zookeeper,支持Broker的动态注册与发现。主要包括两个功能:Broker管理,NameServer接受Broker集群的注册信息并且保存下来作为路由信息的基本数据。然后提供心跳检测机制,检查Broker是否还存活;路由信息管理,每个NameServer将保存关于Broker集群的整个路由信息和用于客户端查询的队列信息。然后Producer和Conumser通过NameServer就可以知道整个Broker集群的路由信息,从而进行消息的投递和消费。NameServer通常也是集群的方式部署,各实例间相互不进行信息通讯。Broker是向每一台NameServer注册自己的路由信息,所以每一个NameServer实例上面都保存一份完整的路由信息。当某个NameServer因某种原因下线了,Broker仍然可以向其它NameServer同步其路由信息,Producer,Consumer仍然可以动态感知Broker的路由的信息。 +- NameServer:NameServer是一个非常简单的Topic路由注册中心,其角色类似Dubbo中的zookeeper,支持Broker的动态注册与发现。主要包括两个功能:Broker管理,NameServer接受Broker集群的注册信息并且保存下来作为路由信息的基本数据。然后提供心跳检测机制,检查Broker是否还存活;路由信息管理,每个NameServer将保存关于Broker集群的整个路由信息和用于客户端查询的队列信息。然后Producer和Conumser通过NameServer就可以知道整个Broker集群的路由信息,从而进行消息的投递和消费。NameServer通常也是集群的方式部署,各实例间相互不进行信息通讯。Broker是向每一台NameServer注册自己的路由信息,所以每一个NameServer实例上面都保存一份完整的路由信息。当某个NameServer因某种原因下线了,Broker仍然可以向其它NameServer同步其路由信息,Producer和Consumer仍然可以动态感知Broker的路由的信息。 - BrokerServer:Broker主要负责消息的存储、投递和查询以及服务高可用保证,为了实现这些功能,Broker包含了以下几个重要子模块。 - 1. Remoting Module:整个Broker的实体,负责处理来自clients端的请求。 - 2. Client Manager:负责管理客户端(Producer/Consumer)和维护Consumer的Topic订阅信息 + 1. Remoting Module:整个Broker的实体,负责处理来自Client端的请求。 + 2. Client Manager:负责管理客户端(Producer/Consumer)和维护Consumer的Topic订阅信息。 3. Store Service:提供方便简单的API接口处理消息存储到物理硬盘和查询功能。 4. HA Service:高可用服务,提供Master Broker 和 Slave Broker之间的数据同步功能。 5. Index Service:根据特定的Message key对投递到Broker的消息进行索引服务,以提供消息的快速查询。 From 4593ad4a59464c9b550ea89dd4675d94fb831d19 Mon Sep 17 00:00:00 2001 From: ltamber Date: Fri, 10 Dec 2021 21:22:54 +0800 Subject: [PATCH 049/141] bump up log4j-core version to 2.15.0 (#3621) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e17a5899164..59813ba0a40 100644 --- a/pom.xml +++ b/pom.xml @@ -587,7 +587,7 @@ org.apache.logging.log4j log4j-core - 2.7 + 2.15.0 org.apache.logging.log4j From dcbf276e919709a160464fdf89dd8439dd9e0a77 Mon Sep 17 00:00:00 2001 From: haozhijie9527 <47975723+haozhijie9527@users.noreply.github.com> Date: Fri, 10 Dec 2021 22:03:08 +0800 Subject: [PATCH 050/141] [ISSUE 3585] [Part I] Avoid unnecessary StringBuffer resizing and String Formatting (#3619) Co-authored-by: haozhijie --- .../store/stats/BrokerStatsManager.java | 42 +++++++++++++------ 1 file changed, 29 insertions(+), 13 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java index b9e11fd5929..466bf5e7629 100644 --- a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java +++ b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java @@ -156,24 +156,24 @@ public void onGroupDeleted(final String group) { } public void incQueuePutNums(final String topic, final Integer queueId) { - this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, String.valueOf(queueId)), 1, 1); + this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, queueId), 1, 1); } public void incQueuePutNums(final String topic, final Integer queueId, int num, int times) { - this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, String.valueOf(queueId)), num, times); + this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, queueId), num, times); } public void incQueuePutSize(final String topic, final Integer queueId, final int size) { - this.statsTable.get(QUEUE_PUT_SIZE).addValue(buildStatsKey(topic, String.valueOf(queueId)), size, 1); + this.statsTable.get(QUEUE_PUT_SIZE).addValue(buildStatsKey(topic, queueId), size, 1); } public void incQueueGetNums(final String group, final String topic, final Integer queueId, final int incValue) { - final String statsKey = buildStatsKey(buildStatsKey(topic, String.valueOf(queueId)), group); + final String statsKey = buildStatsKey(topic, queueId, group); this.statsTable.get(QUEUE_GET_NUMS).addValue(statsKey, incValue, 1); } public void incQueueGetSize(final String group, final String topic, final Integer queueId, final int incValue) { - final String statsKey = buildStatsKey(buildStatsKey(topic, String.valueOf(queueId)), group); + final String statsKey = buildStatsKey(topic, queueId, group); this.statsTable.get(QUEUE_GET_SIZE).addValue(statsKey, incValue, 1); } @@ -194,11 +194,27 @@ public void incGroupGetNums(final String group, final String topic, final int in this.statsTable.get(GROUP_GET_NUMS).addValue(statsKey, incValue, 1); } - public String buildStatsKey(String prefix, String suffix) { - StringBuffer strBuilder = new StringBuffer(); - strBuilder.append(prefix); - strBuilder.append("@"); - strBuilder.append(suffix); + public String buildStatsKey(String topic, String group) { + StringBuilder strBuilder = new StringBuilder(topic.length() + group.length() + 1); + strBuilder.append(topic).append("@").append(group); + return strBuilder.toString(); + } + + public String buildStatsKey(String topic, int queueId) { + StringBuilder strBuilder = new StringBuilder(topic.length() + 5); + strBuilder.append(topic).append("@").append(queueId); + return strBuilder.toString(); + } + + public String buildStatsKey(String topic, int queueId, String group) { + StringBuilder strBuilder = new StringBuilder(topic.length() + group.length() + 6); + strBuilder.append(topic).append("@").append(queueId).append("@").append(group); + return strBuilder.toString(); + } + + public String buildStatsKey(int queueId, String topic, String group) { + StringBuilder strBuilder = new StringBuilder(topic.length() + group.length() + 6); + strBuilder.append(queueId).append("@").append(topic).append("@").append(group); return strBuilder.toString(); } @@ -208,7 +224,7 @@ public void incGroupGetSize(final String group, final String topic, final int in } public void incGroupGetLatency(final String group, final String topic, final int queueId, final int incValue) { - final String statsKey = String.format("%d@%s@%s", queueId, topic, group); + final String statsKey = buildStatsKey(queueId, topic, group); this.statsTable.get(GROUP_GET_LATENCY).addValue(statsKey, incValue, 1); } @@ -236,13 +252,13 @@ public double tpsGroupGetNums(final String group, final String topic) { public void recordDiskFallBehindTime(final String group, final String topic, final int queueId, final long fallBehind) { - final String statsKey = String.format("%d@%s@%s", queueId, topic, group); + final String statsKey = buildStatsKey(queueId, topic, group); this.momentStatsItemSetFallTime.getAndCreateStatsItem(statsKey).getValue().set(fallBehind); } public void recordDiskFallBehindSize(final String group, final String topic, final int queueId, final long fallBehind) { - final String statsKey = String.format("%d@%s@%s", queueId, topic, group); + final String statsKey = buildStatsKey(queueId, topic, group); this.momentStatsItemSetFallSize.getAndCreateStatsItem(statsKey).getValue().set(fallBehind); } From 4f18b0d7754d995daa949b167ea84e886d6b7f44 Mon Sep 17 00:00:00 2001 From: huangli Date: Fri, 10 Dec 2021 22:04:21 +0800 Subject: [PATCH 051/141] [ISSUE 3585] [Part A] eliminate reverse DNS lookup in MessageExt (#3586) --- .../java/org/apache/rocketmq/common/message/MessageExt.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java index 577c4f4bc14..133cb93ba0f 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageExt.java @@ -153,6 +153,10 @@ public String getBornHostString() { public String getBornHostNameString() { if (null != this.bornHost) { + if (bornHost instanceof InetSocketAddress) { + // without reverse dns lookup + return ((InetSocketAddress) bornHost).getHostString(); + } InetAddress inetAddress = ((InetSocketAddress) this.bornHost).getAddress(); return null != inetAddress ? inetAddress.getHostName() : null; From ed258b3779c7abb48e5e7c6214ddf657c0d0e8a7 Mon Sep 17 00:00:00 2001 From: huangli Date: Fri, 10 Dec 2021 22:29:57 +0800 Subject: [PATCH 052/141] [ISSUE 3585] [Part D] improve performance of createUniqID(), from 2.41% to 0.42% in flame graph of producer side. (#3590) --- .../org/apache/rocketmq/common/UtilAll.java | 14 ++++++++++ .../common/message/MessageClientIDSetter.java | 27 ++++++++++--------- .../message/MessageClientIDSetterTest.java | 22 +++++++++++++++ 3 files changed, 50 insertions(+), 13 deletions(-) diff --git a/common/src/main/java/org/apache/rocketmq/common/UtilAll.java b/common/src/main/java/org/apache/rocketmq/common/UtilAll.java index ea22aa7397f..a15b4fadf6d 100644 --- a/common/src/main/java/org/apache/rocketmq/common/UtilAll.java +++ b/common/src/main/java/org/apache/rocketmq/common/UtilAll.java @@ -262,6 +262,20 @@ public static String bytes2string(byte[] src) { return new String(hexChars); } + public static void writeInt(char[] buffer, int pos, int value) { + char[] hexArray = HEX_ARRAY; + for (int moveBits = 28; moveBits >= 0; moveBits -= 4) { + buffer[pos++] = hexArray[(value >>> moveBits) & 0x0F]; + } + } + + public static void writeShort(char[] buffer, int pos, int value) { + char[] hexArray = HEX_ARRAY; + for (int moveBits = 12; moveBits >= 0; moveBits -= 4) { + buffer[pos++] = hexArray[(value >>> moveBits) & 0x0F]; + } + } + public static byte[] string2bytes(String hexString) { if (hexString == null || hexString.equals("")) { return null; diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageClientIDSetter.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageClientIDSetter.java index 041bf6bae20..57090c17120 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageClientIDSetter.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageClientIDSetter.java @@ -25,7 +25,7 @@ public class MessageClientIDSetter { private static final String TOPIC_KEY_SPLITTER = "#"; private static final int LEN; - private static final String FIX_STRING; + private static final char[] FIX_STRING; private static final AtomicInteger COUNTER; private static long startTime; private static long nextStartTime; @@ -42,7 +42,7 @@ public class MessageClientIDSetter { tempBuffer.put(ip); tempBuffer.putShort((short) UtilAll.getPid()); tempBuffer.putInt(MessageClientIDSetter.class.getClassLoader().hashCode()); - FIX_STRING = UtilAll.bytes2string(tempBuffer.array()); + FIX_STRING = UtilAll.bytes2string(tempBuffer.array()).toCharArray(); setStartTime(System.currentTimeMillis()); COUNTER = new AtomicInteger(0); } @@ -112,21 +112,22 @@ public static int getPidFromID(String msgID) { } public static String createUniqID() { - StringBuilder sb = new StringBuilder(LEN * 2); - sb.append(FIX_STRING); - sb.append(UtilAll.bytes2string(createUniqIDBuffer())); - return sb.toString(); - } - - private static byte[] createUniqIDBuffer() { - ByteBuffer buffer = ByteBuffer.allocate(4 + 2); + char[] sb = new char[LEN * 2]; + System.arraycopy(FIX_STRING, 0, sb, 0, FIX_STRING.length); long current = System.currentTimeMillis(); if (current >= nextStartTime) { setStartTime(current); } - buffer.putInt((int) (System.currentTimeMillis() - startTime)); - buffer.putShort((short) COUNTER.getAndIncrement()); - return buffer.array(); + int diff = (int)(current - startTime); + if (diff < 0 && diff > -1000_000) { + // may cause by NTP + diff = 0; + } + int pos = FIX_STRING.length; + UtilAll.writeInt(sb, pos, diff); + pos += 8; + UtilAll.writeShort(sb, pos, COUNTER.getAndIncrement()); + return new String(sb); } public static void setUniqID(final Message msg) { diff --git a/common/src/test/java/org/apache/rocketmq/common/message/MessageClientIDSetterTest.java b/common/src/test/java/org/apache/rocketmq/common/message/MessageClientIDSetterTest.java index 0a17c36b57d..1734cbdf755 100644 --- a/common/src/test/java/org/apache/rocketmq/common/message/MessageClientIDSetterTest.java +++ b/common/src/test/java/org/apache/rocketmq/common/message/MessageClientIDSetterTest.java @@ -22,8 +22,30 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.nio.charset.StandardCharsets; + public class MessageClientIDSetterTest { + @Test + public void testGetTimeFromID() { + long t = System.currentTimeMillis(); + String uniqID = MessageClientIDSetter.createUniqID(); + long t2 = MessageClientIDSetter.getNearlyTimeFromID(uniqID).getTime(); + assertThat(t2 - t < 20); + } + + @Test + public void testGetCountFromID() { + String uniqID = MessageClientIDSetter.createUniqID(); + String uniqID2 = MessageClientIDSetter.createUniqID(); + String idHex = uniqID.substring(uniqID.length() - 4); + String idHex2 = uniqID2.substring(uniqID2.length() - 4); + int s1 = Integer.parseInt(idHex, 16); + int s2 = Integer.parseInt(idHex2, 16); + assertThat(s1 == s2 - 1); + } + + @Test public void testGetIPStrFromID() { byte[] ip = UtilAll.getIP(); From 1635856fea87a1a98056e8f2ab6f2c94f299573d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B5=B5=E5=BB=B6?= Date: Fri, 10 Dec 2021 23:04:08 +0800 Subject: [PATCH 053/141] [ISSUE #2535] Polish log variable. (#2536) * [ISSUE #2535] Use one variable value to log info when eventQueue's size more than 10000. * add the queue limit value when evenQueue size is enough. --- .../rocketmq/remoting/netty/NettyRemotingAbstract.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java index b2e7294bf32..f02518bebe5 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java @@ -570,10 +570,11 @@ class NettyEventExecutor extends ServiceThread { private final int maxSize = 10000; public void putNettyEvent(final NettyEvent event) { - if (this.eventQueue.size() <= maxSize) { + int currentSize = this.eventQueue.size(); + if (currentSize <= maxSize) { this.eventQueue.add(event); } else { - log.warn("event queue size[{}] enough, so drop this event {}", this.eventQueue.size(), event.toString()); + log.warn("event queue size [{}] over the limit [{}], so drop this event {}", currentSize, maxSize, event.toString()); } } From 67225fdabee62700e28f7ef911bb72daaac081e3 Mon Sep 17 00:00:00 2001 From: huangli Date: Sat, 11 Dec 2021 09:22:22 +0800 Subject: [PATCH 054/141] [ISSUE 3585] [Part E] eliminate duplicated getNamespace() call when where is no namespace (#3591) Co-authored-by: colin <39513055@qq.com> --- .../java/org/apache/rocketmq/client/ClientConfig.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java index 8d7f5a136b7..4452bbdfa1d 100644 --- a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java +++ b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java @@ -39,6 +39,7 @@ public class ClientConfig { private String instanceName = System.getProperty("rocketmq.client.name", "DEFAULT"); private int clientCallbackExecutorThreads = Runtime.getRuntime().availableProcessors(); protected String namespace; + private boolean namespaceInitialized = false; protected AccessChannel accessChannel = AccessChannel.LOCAL; /** @@ -195,6 +196,7 @@ public String getNamesrvAddr() { */ public void setNamesrvAddr(String namesrvAddr) { this.namesrvAddr = namesrvAddr; + this.namespaceInitialized = false; } public int getClientCallbackExecutorThreads() { @@ -278,20 +280,26 @@ public void setLanguage(LanguageCode language) { } public String getNamespace() { + if (namespaceInitialized) { + return namespace; + } + if (StringUtils.isNotEmpty(namespace)) { return namespace; } if (StringUtils.isNotEmpty(this.namesrvAddr)) { if (NameServerAddressUtils.validateInstanceEndpoint(namesrvAddr)) { - return NameServerAddressUtils.parseInstanceIdFromEndpoint(namesrvAddr); + namespace = NameServerAddressUtils.parseInstanceIdFromEndpoint(namesrvAddr); } } + namespaceInitialized = true; return namespace; } public void setNamespace(String namespace) { this.namespace = namespace; + this.namespaceInitialized = true; } public AccessChannel getAccessChannel() { From 3a5cdfa59a4674b8e742433f096a07c280b8c9c3 Mon Sep 17 00:00:00 2001 From: panzhi Date: Sat, 11 Dec 2021 14:06:54 +0800 Subject: [PATCH 055/141] [ISSUE #3624]DefaultMQPushConsumerTest.testPullMessage_ExceptionOccursWhenComputePullFromWhere throws RejectedExecution (#3626) --- .../client/consumer/DefaultMQPushConsumerTest.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java index 6c8463542fa..df87932b2ed 100644 --- a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java @@ -57,7 +57,7 @@ import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.exception.RemotingException; -import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -75,7 +75,6 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.nullable; import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -90,7 +89,7 @@ public class DefaultMQPushConsumerTest { private MQClientAPIImpl mQClientAPIImpl; private RebalanceImpl rebalanceImpl; private RebalancePushImpl rebalancePushImpl; - private DefaultMQPushConsumer pushConsumer; + private static DefaultMQPushConsumer pushConsumer; @Before public void init() throws Exception { @@ -157,8 +156,8 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, pushConsumer.start(); } - @After - public void terminate() { + @AfterClass + public static void terminate() { pushConsumer.shutdown(); } From fcb2486f1b77a5a83fd27a792e01fd6f83dabbcd Mon Sep 17 00:00:00 2001 From: panzhi Date: Sat, 11 Dec 2021 16:31:57 +0800 Subject: [PATCH 056/141] [ISSUE #3627]org.apache.rocketmq.broker.processor.SendMessageProcessorTest#testProcessRequest_WithMsgBack failed (#3629) --- .../rocketmq/broker/processor/SendMessageProcessorTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java index b9344e90ed6..1f81bdb8cc8 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/SendMessageProcessorTest.java @@ -88,7 +88,9 @@ public void init() { Channel mockChannel = mock(Channel.class); when(mockChannel.remoteAddress()).thenReturn(new InetSocketAddress(1024)); when(handlerContext.channel()).thenReturn(mockChannel); - when(messageStore.lookMessageByOffset(anyLong())).thenReturn(new MessageExt()); + MessageExt messageExt = new MessageExt(); + messageExt.setTopic(topic); + when(messageStore.lookMessageByOffset(anyLong())).thenReturn(messageExt); sendMessageProcessor = new SendMessageProcessor(brokerController); } From e1571eedb5e73abb4e9f1530e08494b4088af5a3 Mon Sep 17 00:00:00 2001 From: Chengyang He <21126659+Aaron-He@users.noreply.github.com> Date: Sat, 11 Dec 2021 17:43:56 +0800 Subject: [PATCH 057/141] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=E5=8D=8F=E8=AE=AE?= =?UTF-8?q?=E8=A7=A3=E6=9E=90=E6=BC=8F=E6=B4=9E=20(#3475)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../acl/plain/PlainAccessValidatorTest.java | 276 ++++++++++++------ .../remoting/common/RemotingHelper.java | 3 +- .../remoting/protocol/RemotingCommand.java | 29 +- .../protocol/RocketMQSerializable.java | 42 +-- .../protocol/RemotingCommandTest.java | 61 ++-- .../protocol/RocketMQSerializableTest.java | 75 +++-- 6 files changed, 316 insertions(+), 170 deletions(-) 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 a0eb567b694..e6954c24544 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 @@ -39,6 +39,7 @@ import org.apache.rocketmq.common.protocol.heartbeat.HeartbeatData; import org.apache.rocketmq.common.protocol.heartbeat.ProducerData; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.junit.Assert; import org.junit.Before; @@ -49,6 +50,7 @@ public class PlainAccessValidatorTest { private PlainAccessValidator plainAccessValidator; private AclClientRPCHook aclClient; private SessionCredentials sessionCredentials; + @Before public void init() { System.setProperty("rocketmq.home.dir", "src/test/resources"); @@ -72,10 +74,16 @@ public void contentTest() { buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "127.0.0.1"); - String signature = AclUtils.calSignature(accessResource.getContent(), sessionCredentials.getSecretKey()); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "127.0.0.1"); + String signature = AclUtils.calSignature(accessResource.getContent(), sessionCredentials.getSecretKey()); + + Assert.assertEquals(accessResource.getSignature(), signature); + } catch (RemotingCommandException e) { + e.printStackTrace(); - Assert.assertEquals(accessResource.getSignature(), signature); + Assert.fail("Should not throw IOException"); + } } @@ -90,8 +98,14 @@ public void validateTest() { buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @@ -106,8 +120,14 @@ public void validateSendMessageTest() { buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test @@ -121,8 +141,14 @@ public void validateSendMessageV2Test() { buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test @@ -143,74 +169,98 @@ public void validateForAdminCommandWithOutAclRPCHook() { @Test public void validatePullMessageTest() { - PullMessageRequestHeader pullMessageRequestHeader=new PullMessageRequestHeader(); + PullMessageRequestHeader pullMessageRequestHeader = new PullMessageRequestHeader(); pullMessageRequestHeader.setTopic("topicC"); pullMessageRequestHeader.setConsumerGroup("consumerGroupA"); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE,pullMessageRequestHeader); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, pullMessageRequestHeader); aclClient.doBeforeRequest("", remotingCommand); ByteBuffer buf = remotingCommand.encodeHeader(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test public void validateConsumeMessageBackTest() { - ConsumerSendMsgBackRequestHeader consumerSendMsgBackRequestHeader=new ConsumerSendMsgBackRequestHeader(); + ConsumerSendMsgBackRequestHeader consumerSendMsgBackRequestHeader = new ConsumerSendMsgBackRequestHeader(); consumerSendMsgBackRequestHeader.setOriginTopic("topicC"); consumerSendMsgBackRequestHeader.setGroup("consumerGroupA"); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.CONSUMER_SEND_MSG_BACK,consumerSendMsgBackRequestHeader); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.CONSUMER_SEND_MSG_BACK, consumerSendMsgBackRequestHeader); aclClient.doBeforeRequest("", remotingCommand); ByteBuffer buf = remotingCommand.encodeHeader(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test public void validateQueryMessageTest() { - QueryMessageRequestHeader queryMessageRequestHeader=new QueryMessageRequestHeader(); + QueryMessageRequestHeader queryMessageRequestHeader = new QueryMessageRequestHeader(); queryMessageRequestHeader.setTopic("topicC"); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.QUERY_MESSAGE,queryMessageRequestHeader); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.QUERY_MESSAGE, queryMessageRequestHeader); aclClient.doBeforeRequest("", remotingCommand); ByteBuffer buf = remotingCommand.encodeHeader(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test public void validateQueryMessageByKeyTest() { - QueryMessageRequestHeader queryMessageRequestHeader=new QueryMessageRequestHeader(); + QueryMessageRequestHeader queryMessageRequestHeader = new QueryMessageRequestHeader(); queryMessageRequestHeader.setTopic("topicC"); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.QUERY_MESSAGE,queryMessageRequestHeader); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.QUERY_MESSAGE, queryMessageRequestHeader); aclClient.doBeforeRequest("", remotingCommand); remotingCommand.addExtField(MixAll.UNIQUE_MSG_QUERY_FLAG, "false"); ByteBuffer buf = remotingCommand.encodeHeader(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.1.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.1.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test public void validateHeartBeatTest() { - HeartbeatData heartbeatData=new HeartbeatData(); - Set producerDataSet=new HashSet<>(); - Set consumerDataSet=new HashSet<>(); - Set subscriptionDataSet=new HashSet<>(); - ProducerData producerData=new ProducerData(); + HeartbeatData heartbeatData = new HeartbeatData(); + Set producerDataSet = new HashSet<>(); + Set consumerDataSet = new HashSet<>(); + Set subscriptionDataSet = new HashSet<>(); + ProducerData producerData = new ProducerData(); producerData.setGroupName("producerGroupA"); - ConsumerData consumerData=new ConsumerData(); + ConsumerData consumerData = new ConsumerData(); consumerData.setGroupName("consumerGroupA"); - SubscriptionData subscriptionData=new SubscriptionData(); + SubscriptionData subscriptionData = new SubscriptionData(); subscriptionData.setTopic("topicC"); producerDataSet.add(producerData); consumerDataSet.add(consumerData); @@ -218,65 +268,89 @@ public void validateHeartBeatTest() { consumerData.setSubscriptionDataSet(subscriptionDataSet); heartbeatData.setProducerDataSet(producerDataSet); heartbeatData.setConsumerDataSet(consumerDataSet); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT,null); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT, null); remotingCommand.setBody(heartbeatData.encode()); aclClient.doBeforeRequest("", remotingCommand); ByteBuffer buf = remotingCommand.encode(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test public void validateUnRegisterClientTest() { - UnregisterClientRequestHeader unregisterClientRequestHeader=new UnregisterClientRequestHeader(); + UnregisterClientRequestHeader unregisterClientRequestHeader = new UnregisterClientRequestHeader(); unregisterClientRequestHeader.setConsumerGroup("consumerGroupA"); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.UNREGISTER_CLIENT,unregisterClientRequestHeader); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.UNREGISTER_CLIENT, unregisterClientRequestHeader); aclClient.doBeforeRequest("", remotingCommand); ByteBuffer buf = remotingCommand.encodeHeader(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test public void validateGetConsumerListByGroupTest() { - GetConsumerListByGroupRequestHeader getConsumerListByGroupRequestHeader=new GetConsumerListByGroupRequestHeader(); + GetConsumerListByGroupRequestHeader getConsumerListByGroupRequestHeader = new GetConsumerListByGroupRequestHeader(); getConsumerListByGroupRequestHeader.setConsumerGroup("consumerGroupA"); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_LIST_BY_GROUP,getConsumerListByGroupRequestHeader); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_LIST_BY_GROUP, getConsumerListByGroupRequestHeader); aclClient.doBeforeRequest("", remotingCommand); ByteBuffer buf = remotingCommand.encodeHeader(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test public void validateUpdateConsumerOffSetTest() { - UpdateConsumerOffsetRequestHeader updateConsumerOffsetRequestHeader=new UpdateConsumerOffsetRequestHeader(); + UpdateConsumerOffsetRequestHeader updateConsumerOffsetRequestHeader = new UpdateConsumerOffsetRequestHeader(); updateConsumerOffsetRequestHeader.setConsumerGroup("consumerGroupA"); - RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.UPDATE_CONSUMER_OFFSET,updateConsumerOffsetRequestHeader); + RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.UPDATE_CONSUMER_OFFSET, updateConsumerOffsetRequestHeader); aclClient.doBeforeRequest("", remotingCommand); ByteBuffer buf = remotingCommand.encodeHeader(); buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.0.1:9876"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test(expected = AclException.class) public void validateNullAccessKeyTest() { - SessionCredentials sessionCredentials=new SessionCredentials(); + SessionCredentials sessionCredentials = new SessionCredentials(); sessionCredentials.setAccessKey("RocketMQ1"); sessionCredentials.setSecretKey("1234"); - AclClientRPCHook aclClientRPCHook=new AclClientRPCHook(sessionCredentials); + AclClientRPCHook aclClientRPCHook = new AclClientRPCHook(sessionCredentials); SendMessageRequestHeader messageRequestHeader = new SendMessageRequestHeader(); messageRequestHeader.setTopic("topicB"); RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE, messageRequestHeader); @@ -286,16 +360,22 @@ public void validateNullAccessKeyTest() { buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.1.1"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.1.1"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test(expected = AclException.class) public void validateErrorSecretKeyTest() { - SessionCredentials sessionCredentials=new SessionCredentials(); + SessionCredentials sessionCredentials = new SessionCredentials(); sessionCredentials.setAccessKey("RocketMQ"); sessionCredentials.setSecretKey("1234"); - AclClientRPCHook aclClientRPCHook=new AclClientRPCHook(sessionCredentials); + AclClientRPCHook aclClientRPCHook = new AclClientRPCHook(sessionCredentials); SendMessageRequestHeader messageRequestHeader = new SendMessageRequestHeader(); messageRequestHeader.setTopic("topicB"); RemotingCommand remotingCommand = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE, messageRequestHeader); @@ -305,8 +385,14 @@ public void validateErrorSecretKeyTest() { buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.1.1"); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), "192.168.1.1"); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test @@ -318,8 +404,14 @@ public void validateGetAllTopicConfigTest() { buf.getInt(); buf = ByteBuffer.allocate(buf.limit() - buf.position()).put(buf); buf.position(0); - PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), whiteRemoteAddress); - plainAccessValidator.validate(accessResource); + try { + PlainAccessResource accessResource = (PlainAccessResource) plainAccessValidator.parse(RemotingCommand.decode(buf), whiteRemoteAddress); + plainAccessValidator.validate(accessResource); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test @@ -349,7 +441,7 @@ public void updateAccessAclYamlConfigNormalTest() { plainAccessValidator.updateAccessConfig(plainAccessConfig); Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>)readableMap.get("accounts"); + List> accounts = (List>) readableMap.get("accounts"); Map verifyMap = null; for (Map account : accounts) { if (account.get("accessKey").equals(plainAccessConfig.getAccessKey())) { @@ -358,17 +450,17 @@ public void updateAccessAclYamlConfigNormalTest() { } } - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY),"1234567890"); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_TOPIC_PERM),"SUB"); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_GROUP_PERM),"PUB"); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_ADMIN_ROLE),false); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_WHITE_ADDR),"192.168.0.*"); - Assert.assertEquals(((List)verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).size(),2); - Assert.assertEquals(((List)verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).size(),2); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "1234567890"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_TOPIC_PERM), "SUB"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_GROUP_PERM), "PUB"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_ADMIN_ROLE), false); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_WHITE_ADDR), "192.168.0.*"); + Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).size(), 2); + Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).size(), 2); // Verify the dateversion element is correct or not List> dataVersions = (List>) readableMap.get("dataVersion"); - Assert.assertEquals(1,dataVersions.get(0).get("counter")); + Assert.assertEquals(1, dataVersions.get(0).get("counter")); // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); @@ -391,7 +483,7 @@ public void updateAccessAclYamlConfigTest() { plainAccessValidator.updateAccessConfig(plainAccessConfig); Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>)readableMap.get(AclConstants.CONFIG_ACCOUNTS); + List> accounts = (List>) readableMap.get(AclConstants.CONFIG_ACCOUNTS); Map verifyMap = null; for (Map account : accounts) { if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(plainAccessConfig.getAccessKey())) { @@ -399,7 +491,7 @@ public void updateAccessAclYamlConfigTest() { break; } } - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY),"123456789111"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "123456789111"); // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); @@ -433,7 +525,7 @@ public void createAndUpdateAccessAclYamlConfigNormalTest() { plainAccessValidator.updateAccessConfig(plainAccessConfig); Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>)readableMap.get(AclConstants.CONFIG_ACCOUNTS); + List> accounts = (List>) readableMap.get(AclConstants.CONFIG_ACCOUNTS); Map verifyMap = null; for (Map account : accounts) { if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(plainAccessConfig.getAccessKey())) { @@ -441,19 +533,19 @@ public void createAndUpdateAccessAclYamlConfigNormalTest() { break; } } - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY),"123456789111"); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_TOPIC_PERM),"DENY"); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_GROUP_PERM),"PUB"); - Assert.assertEquals(((List)verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).size(),2); - Assert.assertEquals(((List)verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).size(),2); - Assert.assertTrue(((List)verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).contains("topicC=PUB|SUB")); - Assert.assertTrue(((List)verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).contains("topicB=PUB")); - Assert.assertTrue(((List)verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).contains("groupB=PUB|SUB")); - Assert.assertTrue(((List)verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).contains("groupC=DENY")); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "123456789111"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_TOPIC_PERM), "DENY"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_GROUP_PERM), "PUB"); + Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).size(), 2); + Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).size(), 2); + Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).contains("topicC=PUB|SUB")); + Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).contains("topicB=PUB")); + Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).contains("groupB=PUB|SUB")); + Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).contains("groupC=DENY")); // Verify the dateversion element is correct or not List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); - Assert.assertEquals(1,dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); + Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); // Update element in the acl config yaml file PlainAccessConfig plainAccessConfig2 = new PlainAccessConfig(); @@ -464,7 +556,7 @@ public void createAndUpdateAccessAclYamlConfigNormalTest() { plainAccessValidator.updateAccessConfig(plainAccessConfig2); Map readableMap2 = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts2 = (List>)readableMap2.get(AclConstants.CONFIG_ACCOUNTS); + List> accounts2 = (List>) readableMap2.get(AclConstants.CONFIG_ACCOUNTS); Map verifyMap2 = null; for (Map account : accounts2) { if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(plainAccessConfig2.getAccessKey())) { @@ -475,8 +567,8 @@ public void createAndUpdateAccessAclYamlConfigNormalTest() { // Verify the dateversion element after updating is correct or not List> dataVersions2 = (List>) readableMap2.get(AclConstants.CONFIG_DATA_VERSION); - Assert.assertEquals(2,dataVersions2.get(0).get(AclConstants.CONFIG_COUNTER)); - Assert.assertEquals(verifyMap2.get(AclConstants.CONFIG_SECRET_KEY),"1234567890123"); + Assert.assertEquals(2, dataVersions2.get(0).get(AclConstants.CONFIG_COUNTER)); + Assert.assertEquals(verifyMap2.get(AclConstants.CONFIG_SECRET_KEY), "1234567890123"); // Restore the backup file and flush to yaml file @@ -511,7 +603,7 @@ public void deleteAccessAclYamlConfigNormalTest() { plainAccessValidator.deleteAccessConfig(accessKey); Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>)readableMap.get(AclConstants.CONFIG_ACCOUNTS); + List> accounts = (List>) readableMap.get(AclConstants.CONFIG_ACCOUNTS); Map verifyMap = null; for (Map account : accounts) { if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(accessKey)) { @@ -521,10 +613,10 @@ public void deleteAccessAclYamlConfigNormalTest() { } // Verify the specified element is removed or not - Assert.assertEquals(verifyMap,null); + Assert.assertEquals(verifyMap, null); // Verify the dateversion element is correct or not List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); - Assert.assertEquals(1,dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); + Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); @@ -596,21 +688,21 @@ public void updateGlobalWhiteAddrsNormalTest() { Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List globalWhiteAddrList = (List)readableMap.get(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS); + List globalWhiteAddrList = (List) readableMap.get(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS); Assert.assertTrue(globalWhiteAddrList.contains("10.10.154.1")); Assert.assertTrue(globalWhiteAddrList.contains("10.10.154.2")); Assert.assertTrue(globalWhiteAddrList.contains("10.10.154.3")); // Verify the dateversion element is correct or not List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); - Assert.assertEquals(1,dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); + Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } @Test - public void getAllAclConfigTest(){ + public void getAllAclConfigTest() { PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); Assert.assertEquals(aclConfig.getGlobalWhiteAddrs().size(), 2); @@ -619,7 +711,7 @@ public void getAllAclConfigTest(){ @Test - public void updateAccessConfigEmptyPermListTest(){ + public void updateAccessConfigEmptyPermListTest() { PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); String accessKey = "updateAccessConfigEmptyPerm"; @@ -632,14 +724,14 @@ public void updateAccessConfigEmptyPermListTest(){ plainAccessValidator.updateAccessConfig(plainAccessConfig); PlainAccessConfig result = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs() - .stream().filter(c->c.getAccessKey().equals(accessKey)).findFirst().orElse(null); + .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); Assert.assertEquals(0, result.getTopicPerms().size()); plainAccessValidator.deleteAccessConfig(accessKey); } @Test - public void updateAccessConfigEmptyWhiteRemoteAddressTest(){ + public void updateAccessConfigEmptyWhiteRemoteAddressTest() { PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); String accessKey = "updateAccessConfigEmptyWhiteRemoteAddress"; @@ -652,7 +744,7 @@ public void updateAccessConfigEmptyWhiteRemoteAddressTest(){ plainAccessValidator.updateAccessConfig(plainAccessConfig); PlainAccessConfig result = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs() - .stream().filter(c->c.getAccessKey().equals(accessKey)).findFirst().orElse(null); + .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); Assert.assertEquals("", result.getWhiteRemoteAddress()); plainAccessValidator.deleteAccessConfig(accessKey); diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java index 7dacea9c132..1a366669846 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java @@ -24,6 +24,7 @@ import java.nio.channels.SocketChannel; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.exception.RemotingConnectException; import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; @@ -61,7 +62,7 @@ public static SocketAddress string2SocketAddress(final String addr) { public static RemotingCommand invokeSync(final String addr, final RemotingCommand request, final long timeoutMillis) throws InterruptedException, RemotingConnectException, - RemotingSendRequestException, RemotingTimeoutException { + RemotingSendRequestException, RemotingTimeoutException, RemotingCommandException { long beginTime = System.currentTimeMillis(); SocketAddress socketAddress = RemotingUtil.string2SocketAddress(addr); SocketChannel socketChannel = RemotingUtil.connect(socketAddress); diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java index 51b619491b5..0eb01c9a21d 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java @@ -17,6 +17,13 @@ package org.apache.rocketmq.remoting.protocol; import com.alibaba.fastjson.annotation.JSONField; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.common.RemotingHelper; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + import java.lang.annotation.Annotation; import java.lang.reflect.Field; import java.lang.reflect.Modifier; @@ -24,12 +31,6 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.rocketmq.remoting.CommandCustomHeader; -import org.apache.rocketmq.remoting.annotation.CFNotNull; -import org.apache.rocketmq.remoting.common.RemotingHelper; -import org.apache.rocketmq.remoting.exception.RemotingCommandException; -import org.apache.rocketmq.logging.InternalLogger; -import org.apache.rocketmq.logging.InternalLoggerFactory; public class RemotingCommand { public static final String SERIALIZE_TYPE_PROPERTY = "rocketmq.serialize.type"; @@ -39,7 +40,7 @@ public class RemotingCommand { private static final int RPC_TYPE = 0; // 0, REQUEST_COMMAND private static final int RPC_ONEWAY = 1; // 0, RPC private static final Map, Field[]> CLASS_HASH_MAP = - new HashMap, Field[]>(); + new HashMap, Field[]>(); private static final Map CANONICAL_NAME_CACHE = new HashMap(); // 1, Oneway // 1, RESPONSE_COMMAND @@ -111,7 +112,7 @@ public static RemotingCommand createResponseCommand(Class classHeader) { + Class classHeader) { RemotingCommand cmd = new RemotingCommand(); cmd.markResponseType(); cmd.setCode(code); @@ -136,12 +137,12 @@ public static RemotingCommand createResponseCommand(int code, String remark) { return createResponseCommand(code, remark, null); } - public static RemotingCommand decode(final byte[] array) { + public static RemotingCommand decode(final byte[] array) throws RemotingCommandException { ByteBuffer byteBuffer = ByteBuffer.wrap(array); return decode(byteBuffer); } - public static RemotingCommand decode(final ByteBuffer byteBuffer) { + public static RemotingCommand decode(final ByteBuffer byteBuffer) throws RemotingCommandException { int length = byteBuffer.limit(); int oriHeaderLen = byteBuffer.getInt(); int headerLength = getHeaderLength(oriHeaderLen); @@ -166,7 +167,7 @@ public static int getHeaderLength(int length) { return length & 0xFFFFFF; } - private static RemotingCommand headerDecode(byte[] headerData, SerializeType type) { + private static RemotingCommand headerDecode(byte[] headerData, SerializeType type) throws RemotingCommandException { switch (type) { case JSON: RemotingCommand resultJson = RemotingSerializable.decode(headerData, RemotingCommand.class); @@ -232,7 +233,7 @@ public void writeCustomHeader(CommandCustomHeader customHeader) { } public CommandCustomHeader decodeCommandCustomHeader( - Class classHeader) throws RemotingCommandException { + Class classHeader) throws RemotingCommandException { CommandCustomHeader objectHeader; try { objectHeader = classHeader.newInstance(); @@ -529,8 +530,8 @@ public void addExtField(String key, String value) { @Override public String toString() { return "RemotingCommand [code=" + code + ", language=" + language + ", version=" + version + ", opaque=" + opaque + ", flag(B)=" - + Integer.toBinaryString(flag) + ", remark=" + remark + ", extFields=" + extFields + ", serializeTypeCurrentRPC=" - + serializeTypeCurrentRPC + "]"; + + Integer.toBinaryString(flag) + ", remark=" + remark + ", extFields=" + extFields + ", serializeTypeCurrentRPC=" + + serializeTypeCurrentRPC + "]"; } public SerializeType getSerializeTypeCurrentRPC() { diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java index 66119e0eacd..6075d7ab6db 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java @@ -16,6 +16,8 @@ */ package org.apache.rocketmq.remoting.protocol; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.util.HashMap; @@ -85,10 +87,10 @@ public static byte[] mapSerialize(HashMap map) { Map.Entry entry = it.next(); if (entry.getKey() != null && entry.getValue() != null) { kvLength = - // keySize + Key - 2 + entry.getKey().getBytes(CHARSET_UTF8).length - // valSize + val - + 4 + entry.getValue().getBytes(CHARSET_UTF8).length; + // keySize + Key + 2 + entry.getKey().getBytes(CHARSET_UTF8).length + // valSize + val + + 4 + entry.getValue().getBytes(CHARSET_UTF8).length; totalLength += kvLength; } } @@ -117,23 +119,23 @@ public static byte[] mapSerialize(HashMap map) { private static int calTotalLen(int remark, int ext) { // int code(~32767) int length = 2 - // LanguageCode language - + 1 - // int version(~32767) - + 2 - // int opaque - + 4 - // int flag - + 4 - // String remark - + 4 + remark - // HashMap extFields - + 4 + ext; + // LanguageCode language + + 1 + // int version(~32767) + + 2 + // int opaque + + 4 + // int flag + + 4 + // String remark + + 4 + remark + // HashMap extFields + + 4 + ext; return length; } - public static RemotingCommand rocketMQProtocolDecode(final byte[] headerArray) { + public static RemotingCommand rocketMQProtocolDecode(final byte[] headerArray) throws RemotingCommandException { RemotingCommand cmd = new RemotingCommand(); ByteBuffer headerBuffer = ByteBuffer.wrap(headerArray); // int code(~32767) @@ -149,6 +151,9 @@ public static RemotingCommand rocketMQProtocolDecode(final byte[] headerArray) { // String remark int remarkLength = headerBuffer.getInt(); if (remarkLength > 0) { + if (remarkLength > headerArray.length) { + throw new RemotingCommandException("RocketMQ protocol decoding failed, remark length: " + remarkLength + ", but header length: " + headerArray.length); + } byte[] remarkContent = new byte[remarkLength]; headerBuffer.get(remarkContent); cmd.setRemark(new String(remarkContent, CHARSET_UTF8)); @@ -157,6 +162,9 @@ public static RemotingCommand rocketMQProtocolDecode(final byte[] headerArray) { // HashMap extFields int extFieldsLength = headerBuffer.getInt(); if (extFieldsLength > 0) { + if (extFieldsLength > headerArray.length) { + throw new RemotingCommandException("RocketMQ protocol decoding failed, extFields length: " + extFieldsLength + ", but header length: " + headerArray.length); + } byte[] extFieldsBytes = new byte[extFieldsLength]; headerBuffer.get(extFieldsBytes); cmd.setExtFields(mapDeserialize(extFieldsBytes)); diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java index 2bd41cec839..0b19d445724 100644 --- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java @@ -22,6 +22,7 @@ import org.apache.rocketmq.remoting.CommandCustomHeader; import org.apache.rocketmq.remoting.annotation.CFNotNull; import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.junit.Assert; import org.junit.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -118,10 +119,18 @@ public void testEncodeAndDecode_EmptyBody() { buffer.get(bytes, 0, buffer.limit() - 4); buffer = ByteBuffer.wrap(bytes); - RemotingCommand decodedCommand = RemotingCommand.decode(buffer); + RemotingCommand decodedCommand = null; + try { + decodedCommand = RemotingCommand.decode(buffer); + + assertThat(decodedCommand.getSerializeTypeCurrentRPC()).isEqualTo(SerializeType.JSON); + assertThat(decodedCommand.getBody()).isNull(); + } catch (RemotingCommandException e) { + e.printStackTrace(); + Assert.fail("Should not throw IOException"); + } + - assertThat(decodedCommand.getSerializeTypeCurrentRPC()).isEqualTo(SerializeType.JSON); - assertThat(decodedCommand.getBody()).isNull(); } @Test @@ -141,10 +150,16 @@ public void testEncodeAndDecode_FilledBody() { buffer.get(bytes, 0, buffer.limit() - 4); buffer = ByteBuffer.wrap(bytes); - RemotingCommand decodedCommand = RemotingCommand.decode(buffer); + RemotingCommand decodedCommand = null; + try { + decodedCommand = RemotingCommand.decode(buffer); - assertThat(decodedCommand.getSerializeTypeCurrentRPC()).isEqualTo(SerializeType.JSON); - assertThat(decodedCommand.getBody()).isEqualTo(new byte[] {0, 1, 2, 3, 4}); + assertThat(decodedCommand.getSerializeTypeCurrentRPC()).isEqualTo(SerializeType.JSON); + assertThat(decodedCommand.getBody()).isEqualTo(new byte[] {0, 1, 2, 3, 4}); + } catch (RemotingCommandException e) { + e.printStackTrace(); + Assert.fail("Should not throw IOException"); + } } @Test @@ -165,22 +180,30 @@ public void testEncodeAndDecode_FilledBodyWithExtFields() throws RemotingCommand buffer.get(bytes, 0, buffer.limit() - 4); buffer = ByteBuffer.wrap(bytes); - RemotingCommand decodedCommand = RemotingCommand.decode(buffer); + RemotingCommand decodedCommand = null; + try { + decodedCommand = RemotingCommand.decode(buffer); + + assertThat(decodedCommand.getExtFields().get("stringValue")).isEqualTo("bilibili"); + assertThat(decodedCommand.getExtFields().get("intValue")).isEqualTo("2333"); + assertThat(decodedCommand.getExtFields().get("longValue")).isEqualTo("23333333"); + assertThat(decodedCommand.getExtFields().get("booleanValue")).isEqualTo("true"); + assertThat(decodedCommand.getExtFields().get("doubleValue")).isEqualTo("0.618"); + + assertThat(decodedCommand.getExtFields().get("key")).isEqualTo("value"); - assertThat(decodedCommand.getExtFields().get("stringValue")).isEqualTo("bilibili"); - assertThat(decodedCommand.getExtFields().get("intValue")).isEqualTo("2333"); - assertThat(decodedCommand.getExtFields().get("longValue")).isEqualTo("23333333"); - assertThat(decodedCommand.getExtFields().get("booleanValue")).isEqualTo("true"); - assertThat(decodedCommand.getExtFields().get("doubleValue")).isEqualTo("0.618"); + CommandCustomHeader decodedHeader = decodedCommand.decodeCommandCustomHeader(ExtFieldsHeader.class); + assertThat(((ExtFieldsHeader) decodedHeader).getStringValue()).isEqualTo("bilibili"); + assertThat(((ExtFieldsHeader) decodedHeader).getIntValue()).isEqualTo(2333); + assertThat(((ExtFieldsHeader) decodedHeader).getLongValue()).isEqualTo(23333333l); + assertThat(((ExtFieldsHeader) decodedHeader).isBooleanValue()).isEqualTo(true); + assertThat(((ExtFieldsHeader) decodedHeader).getDoubleValue()).isBetween(0.617, 0.619); + } catch (RemotingCommandException e) { + e.printStackTrace(); - assertThat(decodedCommand.getExtFields().get("key")).isEqualTo("value"); + Assert.fail("Should not throw IOException"); + } - CommandCustomHeader decodedHeader = decodedCommand.decodeCommandCustomHeader(ExtFieldsHeader.class); - assertThat(((ExtFieldsHeader) decodedHeader).getStringValue()).isEqualTo("bilibili"); - assertThat(((ExtFieldsHeader) decodedHeader).getIntValue()).isEqualTo(2333); - assertThat(((ExtFieldsHeader) decodedHeader).getLongValue()).isEqualTo(23333333l); - assertThat(((ExtFieldsHeader) decodedHeader).isBooleanValue()).isEqualTo(true); - assertThat(((ExtFieldsHeader) decodedHeader).getDoubleValue()).isBetween(0.617, 0.619); } @Test diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java index f1db54fa303..149ec721c8b 100644 --- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java @@ -17,6 +17,9 @@ package org.apache.rocketmq.remoting.protocol; import java.util.HashMap; + +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.junit.Assert; import org.junit.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -42,15 +45,22 @@ public void testRocketMQProtocolEncodeAndDecode_WithoutRemarkWithoutExtFields() assertThat(parseToInt(result, 13)).isEqualTo(0); //empty remark assertThat(parseToInt(result, 17)).isEqualTo(0); //empty extFields - RemotingCommand decodedCommand = RocketMQSerializable.rocketMQProtocolDecode(result); - - assertThat(decodedCommand.getCode()).isEqualTo(code); - assertThat(decodedCommand.getLanguage()).isEqualTo(LanguageCode.JAVA); - assertThat(decodedCommand.getVersion()).isEqualTo(2333); - assertThat(decodedCommand.getOpaque()).isEqualTo(opaque); - assertThat(decodedCommand.getFlag()).isEqualTo(0); - assertThat(decodedCommand.getRemark()).isNull(); - assertThat(decodedCommand.getExtFields()).isNull(); + RemotingCommand decodedCommand = null; + try { + decodedCommand = RocketMQSerializable.rocketMQProtocolDecode(result); + + assertThat(decodedCommand.getCode()).isEqualTo(code); + assertThat(decodedCommand.getLanguage()).isEqualTo(LanguageCode.JAVA); + assertThat(decodedCommand.getVersion()).isEqualTo(2333); + assertThat(decodedCommand.getOpaque()).isEqualTo(opaque); + assertThat(decodedCommand.getFlag()).isEqualTo(0); + assertThat(decodedCommand.getRemark()).isNull(); + assertThat(decodedCommand.getExtFields()).isNull(); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test @@ -80,15 +90,21 @@ public void testRocketMQProtocolEncodeAndDecode_WithRemarkWithoutExtFields() { assertThat(parseToInt(result, 30)).isEqualTo(0); //empty extFields - RemotingCommand decodedCommand = RocketMQSerializable.rocketMQProtocolDecode(result); - - assertThat(decodedCommand.getCode()).isEqualTo(code); - assertThat(decodedCommand.getLanguage()).isEqualTo(LanguageCode.JAVA); - assertThat(decodedCommand.getVersion()).isEqualTo(2333); - assertThat(decodedCommand.getOpaque()).isEqualTo(opaque); - assertThat(decodedCommand.getFlag()).isEqualTo(0); - assertThat(decodedCommand.getRemark()).contains("Sample Remark"); - assertThat(decodedCommand.getExtFields()).isNull(); + try { + RemotingCommand decodedCommand = RocketMQSerializable.rocketMQProtocolDecode(result); + + assertThat(decodedCommand.getCode()).isEqualTo(code); + assertThat(decodedCommand.getLanguage()).isEqualTo(LanguageCode.JAVA); + assertThat(decodedCommand.getVersion()).isEqualTo(2333); + assertThat(decodedCommand.getOpaque()).isEqualTo(opaque); + assertThat(decodedCommand.getFlag()).isEqualTo(0); + assertThat(decodedCommand.getRemark()).contains("Sample Remark"); + assertThat(decodedCommand.getExtFields()).isNull(); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test @@ -118,15 +134,20 @@ public void testRocketMQProtocolEncodeAndDecode_WithoutRemarkWithExtFields() { HashMap extFields = RocketMQSerializable.mapDeserialize(extFieldsArray); assertThat(extFields).contains(new HashMap.SimpleEntry("key", "value")); - RemotingCommand decodedCommand = RocketMQSerializable.rocketMQProtocolDecode(result); - - assertThat(decodedCommand.getCode()).isEqualTo(code); - assertThat(decodedCommand.getLanguage()).isEqualTo(LanguageCode.JAVA); - assertThat(decodedCommand.getVersion()).isEqualTo(2333); - assertThat(decodedCommand.getOpaque()).isEqualTo(opaque); - assertThat(decodedCommand.getFlag()).isEqualTo(0); - assertThat(decodedCommand.getRemark()).isNull(); - assertThat(decodedCommand.getExtFields()).contains(new HashMap.SimpleEntry("key", "value")); + try { + RemotingCommand decodedCommand = RocketMQSerializable.rocketMQProtocolDecode(result); + assertThat(decodedCommand.getCode()).isEqualTo(code); + assertThat(decodedCommand.getLanguage()).isEqualTo(LanguageCode.JAVA); + assertThat(decodedCommand.getVersion()).isEqualTo(2333); + assertThat(decodedCommand.getOpaque()).isEqualTo(opaque); + assertThat(decodedCommand.getFlag()).isEqualTo(0); + assertThat(decodedCommand.getRemark()).isNull(); + assertThat(decodedCommand.getExtFields()).contains(new HashMap.SimpleEntry("key", "value")); + } catch (RemotingCommandException e) { + e.printStackTrace(); + + Assert.fail("Should not throw IOException"); + } } @Test From 7d1b001c2fe50f9c22096efdc42a42b60d3ee2d6 Mon Sep 17 00:00:00 2001 From: haozhijie9527 <47975723+haozhijie9527@users.noreply.github.com> Date: Mon, 13 Dec 2021 11:14:59 +0800 Subject: [PATCH 058/141] fix npe of SendMessageProcessorTest (#3632) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: haozhijie <“haozhijie_9527@163.com”> --- .../store/stats/BrokerStatsManager.java | 28 ++++++++++++++++--- 1 file changed, 24 insertions(+), 4 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java index 466bf5e7629..91de4a2c5d4 100644 --- a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java +++ b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java @@ -195,25 +195,45 @@ public void incGroupGetNums(final String group, final String topic, final int in } public String buildStatsKey(String topic, String group) { - StringBuilder strBuilder = new StringBuilder(topic.length() + group.length() + 1); + StringBuilder strBuilder; + if (topic != null && group != null) { + strBuilder = new StringBuilder(topic.length() + group.length() + 1); + } else { + strBuilder = new StringBuilder(); + } strBuilder.append(topic).append("@").append(group); return strBuilder.toString(); } public String buildStatsKey(String topic, int queueId) { - StringBuilder strBuilder = new StringBuilder(topic.length() + 5); + StringBuilder strBuilder; + if (topic != null) { + strBuilder = new StringBuilder(topic.length() + 5); + } else { + strBuilder = new StringBuilder(); + } strBuilder.append(topic).append("@").append(queueId); return strBuilder.toString(); } public String buildStatsKey(String topic, int queueId, String group) { - StringBuilder strBuilder = new StringBuilder(topic.length() + group.length() + 6); + StringBuilder strBuilder; + if (topic != null && group != null) { + strBuilder = new StringBuilder(topic.length() + group.length() + 6); + } else { + strBuilder = new StringBuilder(); + } strBuilder.append(topic).append("@").append(queueId).append("@").append(group); return strBuilder.toString(); } public String buildStatsKey(int queueId, String topic, String group) { - StringBuilder strBuilder = new StringBuilder(topic.length() + group.length() + 6); + StringBuilder strBuilder; + if (topic != null && group != null) { + strBuilder = new StringBuilder(topic.length() + group.length() + 6); + } else { + strBuilder = new StringBuilder(); + } strBuilder.append(queueId).append("@").append(topic).append("@").append(group); return strBuilder.toString(); } From 4a8263b4609cd16b5e7c194528bd875711e01852 Mon Sep 17 00:00:00 2001 From: coding Date: Mon, 13 Dec 2021 15:27:48 +0800 Subject: [PATCH 059/141] [ISSUE #3635] remove log4j dependency in client pom (#3636) Co-authored-by: zanglei --- client/pom.xml | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 82a053cf903..07ba7d1e483 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -38,11 +38,6 @@ - - org.slf4j - slf4j-api - test - org.apache.commons commons-lang3 @@ -59,15 +54,5 @@ 0.33.0 test - - org.apache.logging.log4j - log4j-core - test - - - org.apache.logging.log4j - log4j-slf4j-impl - test - From 5fea69bd399014764c74fa583a1a223713f2ce75 Mon Sep 17 00:00:00 2001 From: zhaohai <33314633+zhaohai1299002788@users.noreply.github.com> Date: Wed, 15 Dec 2021 17:32:24 +0800 Subject: [PATCH 060/141] [ISSUE #3644] exchange parameters --- .../org/apache/rocketmq/store/ScheduleMessageServiceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java index 8502521fe8e..0eafe4c2f7f 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java @@ -66,7 +66,7 @@ public void testCorrectDelayOffset_whenInit() throws Exception { offsetTable = (ConcurrentMap) field.get(scheduleMessageService); for (long offset : offsetTable.values()) { - assertEquals(offset, 0); + assertEquals(0, offset); } } From fe98bed001c3d62e45940a365fe6d98d6d38f35f Mon Sep 17 00:00:00 2001 From: zanglei Date: Wed, 15 Dec 2021 14:08:22 +0800 Subject: [PATCH 061/141] [ISSUE #3645] Remove TBW102 topic not found warn log in rocketmq-client.log --- .../apache/rocketmq/client/impl/factory/MQClientInstance.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java index db81143396c..6d35ed63028 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java @@ -65,6 +65,7 @@ import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.filter.ExpressionType; import org.apache.rocketmq.common.protocol.NamespaceUtil; +import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageQueue; @@ -671,7 +672,7 @@ public boolean updateTopicRouteInfoFromNameServer(final String topic, boolean is log.warn("updateTopicRouteInfoFromNameServer, getTopicRouteInfoFromNameServer return null, Topic: {}. [{}]", topic, this.clientId); } } catch (MQClientException e) { - if (!topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX)) { + if (!topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX) && !topic.equals(TopicValidator.AUTO_CREATE_TOPIC_KEY_TOPIC)) { log.warn("updateTopicRouteInfoFromNameServer Exception", e); } } catch (RemotingException e) { From 64ad7908b89bec5190e14c80f044b7060ac2774d Mon Sep 17 00:00:00 2001 From: panzhi Date: Wed, 15 Dec 2021 20:00:21 +0800 Subject: [PATCH 062/141] [ISSUE #3624]Fix producer/consumer re-start may fail introduced in #3454 (#3639) Fix producer/consumer re-start may fail introduced in #3454 1. start a producer 2. shutdown producer 3. start the second producer, exception throws --- .../impl/producer/DefaultMQProducerImpl.java | 21 +------- .../client/producer/RequestFutureHolder.java | 50 ++++++++++++------- 2 files changed, 35 insertions(+), 36 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java index b27117fbbf5..cedbbdb6ca3 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java @@ -223,25 +223,10 @@ public void start(final boolean startFactory) throws MQClientException { this.mQClientFactory.sendHeartbeatToAllBrokerWithLock(); - this.startScheduledTask(); + RequestFutureHolder.getInstance().startScheduledTask(this); } - private void startScheduledTask() { - if (RequestFutureHolder.getInstance().getProducerNum().incrementAndGet() == 1) { - RequestFutureHolder.getInstance().getScheduledExecutorService().scheduleAtFixedRate(new Runnable() { - @Override - public void run() { - try { - RequestFutureHolder.getInstance().scanExpiredRequest(); - } catch (Throwable e) { - log.error("scan RequestFutureTable exception", e); - } - } - }, 1000 * 3, 1000, TimeUnit.MILLISECONDS); - } - } - private void checkConfig() throws MQClientException { Validators.checkGroup(this.defaultMQProducer.getProducerGroup()); @@ -269,9 +254,7 @@ public void shutdown(final boolean shutdownFactory) { if (shutdownFactory) { this.mQClientFactory.shutdown(); } - if (RequestFutureHolder.getInstance().getProducerNum().decrementAndGet() == 0) { - RequestFutureHolder.getInstance().getScheduledExecutorService().shutdown(); - } + RequestFutureHolder.getInstance().shutdown(this); log.info("the producer [{}] shutdown OK", this.defaultMQProducer.getProducerGroup()); this.serviceState = ServiceState.SHUTDOWN_ALREADY; break; diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java b/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java index 24b3a906a56..8fe9abcc697 100644 --- a/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java +++ b/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java @@ -17,39 +17,36 @@ package org.apache.rocketmq.client.producer; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.TimeUnit; import org.apache.rocketmq.client.common.ClientErrorCode; import org.apache.rocketmq.client.exception.RequestTimeoutException; +import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl; import org.apache.rocketmq.client.log.ClientLogger; +import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.logging.InternalLogger; public class RequestFutureHolder { private static InternalLogger log = ClientLogger.getLog(); private static final RequestFutureHolder INSTANCE = new RequestFutureHolder(); private ConcurrentHashMap requestFutureTable = new ConcurrentHashMap(); - private final AtomicInteger producerNum = new AtomicInteger(0); - private final ScheduledExecutorService scheduledExecutorService = Executors - .newSingleThreadScheduledExecutor(new ThreadFactory() { - @Override - public Thread newThread(Runnable r) { - return new Thread(r, "RequestHouseKeepingService"); - } - }); + private final Set producerSet = new HashSet<>(); + private ScheduledExecutorService scheduledExecutorService = null; public ConcurrentHashMap getRequestFutureTable() { return requestFutureTable; } - public void scanExpiredRequest() { + private void scanExpiredRequest() { final List rfList = new LinkedList(); Iterator> it = requestFutureTable.entrySet().iterator(); while (it.hasNext()) { @@ -74,17 +71,36 @@ public void scanExpiredRequest() { } } - private RequestFutureHolder() { - } + public synchronized void startScheduledTask(DefaultMQProducerImpl producer) { + this.producerSet.add(producer); + if (null == scheduledExecutorService) { + this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("RequestHouseKeepingService")); + + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + RequestFutureHolder.getInstance().scanExpiredRequest(); + } catch (Throwable e) { + log.error("scan RequestFutureTable exception", e); + } + } + }, 1000 * 3, 1000, TimeUnit.MILLISECONDS); - public AtomicInteger getProducerNum() { - return producerNum; + } } - public ScheduledExecutorService getScheduledExecutorService() { - return scheduledExecutorService; + public synchronized void shutdown(DefaultMQProducerImpl producer) { + this.producerSet.remove(producer); + if (this.producerSet.size() <= 0 && null != this.scheduledExecutorService) { + ScheduledExecutorService executorService = this.scheduledExecutorService; + this.scheduledExecutorService = null; + executorService.shutdown(); + } } + private RequestFutureHolder() {} + public static RequestFutureHolder getInstance() { return INSTANCE; } From 5081d155f8e66ef1b6f22a99bbb6639e8171607a Mon Sep 17 00:00:00 2001 From: huangli Date: Wed, 15 Dec 2021 20:03:23 +0800 Subject: [PATCH 063/141] [ISSUE #3637] Add enableDetailStat in BrokerConfig so we can disable stat of queue level. (#3638) [ISSUE #3637] Add enableDetailStat in BrokerConfig so we can disable stat of queue level. --- .../rocketmq/broker/BrokerController.java | 2 +- .../processor/PullMessageProcessor.java | 2 +- .../filter/MessageStoreWithFilterTest.java | 2 +- .../apache/rocketmq/common/BrokerConfig.java | 10 ++++ .../store/stats/BrokerStatsManager.java | 59 +++++++++++++------ .../rocketmq/store/BatchPutMessageTest.java | 2 +- .../rocketmq/store/ConsumeQueueTest.java | 2 +- .../DefaultMessageStoreCleanFilesTest.java | 2 +- .../DefaultMessageStoreShutDownTest.java | 2 +- .../store/DefaultMessageStoreTest.java | 2 +- .../org/apache/rocketmq/store/HATest.java | 2 +- .../store/ScheduleMessageServiceTest.java | 2 +- .../store/dledger/MessageStoreTestBase.java | 4 +- .../schedule/ScheduleMessageServiceTest.java | 2 +- .../java/stats/BrokerStatsManagerTest.java | 2 +- 15 files changed, 65 insertions(+), 32 deletions(-) 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 bce21c52093..662ec498144 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -206,7 +206,7 @@ public BrokerController( this.heartbeatThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getHeartbeatThreadPoolQueueCapacity()); this.endTransactionThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getEndTransactionPoolQueueCapacity()); - this.brokerStatsManager = new BrokerStatsManager(this.brokerConfig.getBrokerClusterName()); + this.brokerStatsManager = new BrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()); this.setStoreHost(new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), this.getNettyServerConfig().getListenPort())); this.brokerFastFailure = new BrokerFastFailure(this); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java index 8879a722fcf..20665a3c9f9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PullMessageProcessor.java @@ -91,6 +91,7 @@ public boolean rejectRequest() { private RemotingCommand processRequest(final Channel channel, RemotingCommand request, boolean brokerAllowSuspend) throws RemotingCommandException { + final long beginTimeMills = this.brokerController.getMessageStore().now(); RemotingCommand response = RemotingCommand.createResponseCommand(PullMessageResponseHeader.class); final PullMessageResponseHeader responseHeader = (PullMessageResponseHeader) response.readCustomHeader(); final PullMessageRequestHeader requestHeader = @@ -379,7 +380,6 @@ private RemotingCommand processRequest(final Channel channel, RemotingCommand re this.brokerController.getBrokerStatsManager().incBrokerGetNums(getMessageResult.getMessageCount()); if (this.brokerController.getBrokerConfig().isTransferMsgByHeap()) { - final long beginTimeMills = this.brokerController.getMessageStore().now(); final byte[] r = this.readGetMessageResult(getMessageResult, requestHeader.getConsumerGroup(), requestHeader.getTopic(), requestHeader.getQueueId()); this.brokerController.getBrokerStatsManager().incGroupGetLatency(requestHeader.getConsumerGroup(), requestHeader.getTopic(), requestHeader.getQueueId(), diff --git a/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java b/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java index 34dc640aeab..23b38e25d92 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java @@ -146,7 +146,7 @@ protected DefaultMessageStore gen(ConsumerFilterManager filterManager) throws Ex DefaultMessageStore master = new DefaultMessageStore( messageStoreConfig, - new BrokerStatsManager(brokerConfig.getBrokerClusterName()), + new BrokerStatsManager(brokerConfig.getBrokerClusterName(), brokerConfig.isEnableDetailStat()), new MessageArrivingListener() { @Override public void arriving(String topic, int queueId, long logicOffset, long tagsCode, 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 f710cdb4030..a45efbe2f17 100644 --- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java @@ -185,6 +185,8 @@ public class BrokerConfig { private boolean storeReplyMessageEnable = true; + private boolean enableDetailStat = true; + private boolean autoDeleteUnusedStats = false; public static String localHostName() { @@ -797,6 +799,14 @@ public void setStoreReplyMessageEnable(boolean storeReplyMessageEnable) { this.storeReplyMessageEnable = storeReplyMessageEnable; } + public boolean isEnableDetailStat() { + return enableDetailStat; + } + + public void setEnableDetailStat(boolean enableDetailStat) { + this.enableDetailStat = enableDetailStat; + } + public boolean isAutoDeleteUnusedStats() { return autoDeleteUnusedStats; } diff --git a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java index 91de4a2c5d4..531d3fd3947 100644 --- a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java +++ b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java @@ -72,16 +72,20 @@ public class BrokerStatsManager { "CommercialStatsThread")); private final HashMap statsTable = new HashMap(); private final String clusterName; + private final boolean enableQueueStat; private final MomentStatsItemSet momentStatsItemSetFallSize = new MomentStatsItemSet(GROUP_GET_FALL_SIZE, scheduledExecutorService, log); private final MomentStatsItemSet momentStatsItemSetFallTime = new MomentStatsItemSet(GROUP_GET_FALL_TIME, scheduledExecutorService, log); - public BrokerStatsManager(String clusterName) { + public BrokerStatsManager(String clusterName, boolean enableQueueStat) { this.clusterName = clusterName; + this.enableQueueStat = enableQueueStat; - this.statsTable.put(QUEUE_PUT_NUMS, new StatsItemSet(QUEUE_PUT_NUMS, this.scheduledExecutorService, log)); - this.statsTable.put(QUEUE_PUT_SIZE, new StatsItemSet(QUEUE_PUT_SIZE, this.scheduledExecutorService, log)); - this.statsTable.put(QUEUE_GET_NUMS, new StatsItemSet(QUEUE_GET_NUMS, this.scheduledExecutorService, log)); - this.statsTable.put(QUEUE_GET_SIZE, new StatsItemSet(QUEUE_GET_SIZE, this.scheduledExecutorService, log)); + if (enableQueueStat) { + this.statsTable.put(QUEUE_PUT_NUMS, new StatsItemSet(QUEUE_PUT_NUMS, this.scheduledExecutorService, log)); + this.statsTable.put(QUEUE_PUT_SIZE, new StatsItemSet(QUEUE_PUT_SIZE, this.scheduledExecutorService, log)); + this.statsTable.put(QUEUE_GET_NUMS, new StatsItemSet(QUEUE_GET_NUMS, this.scheduledExecutorService, log)); + this.statsTable.put(QUEUE_GET_SIZE, new StatsItemSet(QUEUE_GET_SIZE, this.scheduledExecutorService, log)); + } this.statsTable.put(TOPIC_PUT_NUMS, new StatsItemSet(TOPIC_PUT_NUMS, this.scheduledExecutorService, log)); this.statsTable.put(TOPIC_PUT_SIZE, new StatsItemSet(TOPIC_PUT_SIZE, this.scheduledExecutorService, log)); this.statsTable.put(GROUP_GET_NUMS, new StatsItemSet(GROUP_GET_NUMS, this.scheduledExecutorService, log)); @@ -132,8 +136,10 @@ public StatsItem getStatsItem(final String statsName, final String statsKey) { public void onTopicDeleted(final String topic) { this.statsTable.get(TOPIC_PUT_NUMS).delValue(topic); this.statsTable.get(TOPIC_PUT_SIZE).delValue(topic); - this.statsTable.get(QUEUE_PUT_NUMS).delValueByPrefixKey(topic, "@"); - this.statsTable.get(QUEUE_PUT_SIZE).delValueByPrefixKey(topic, "@"); + if (enableQueueStat) { + this.statsTable.get(QUEUE_PUT_NUMS).delValueByPrefixKey(topic, "@"); + this.statsTable.get(QUEUE_PUT_SIZE).delValueByPrefixKey(topic, "@"); + } this.statsTable.get(GROUP_GET_NUMS).delValueByPrefixKey(topic, "@"); this.statsTable.get(GROUP_GET_SIZE).delValueByPrefixKey(topic, "@"); this.statsTable.get(QUEUE_GET_NUMS).delValueByPrefixKey(topic, "@"); @@ -147,8 +153,10 @@ public void onTopicDeleted(final String topic) { public void onGroupDeleted(final String group) { this.statsTable.get(GROUP_GET_NUMS).delValueBySuffixKey(group, "@"); this.statsTable.get(GROUP_GET_SIZE).delValueBySuffixKey(group, "@"); - this.statsTable.get(QUEUE_GET_NUMS).delValueBySuffixKey(group, "@"); - this.statsTable.get(QUEUE_GET_SIZE).delValueBySuffixKey(group, "@"); + if (enableQueueStat) { + this.statsTable.get(QUEUE_GET_NUMS).delValueBySuffixKey(group, "@"); + this.statsTable.get(QUEUE_GET_SIZE).delValueBySuffixKey(group, "@"); + } this.statsTable.get(SNDBCK_PUT_NUMS).delValueBySuffixKey(group, "@"); this.statsTable.get(GROUP_GET_LATENCY).delValueBySuffixKey(group, "@"); this.momentStatsItemSetFallSize.delValueBySuffixKey(group, "@"); @@ -156,25 +164,35 @@ public void onGroupDeleted(final String group) { } public void incQueuePutNums(final String topic, final Integer queueId) { - this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, queueId), 1, 1); + if (enableQueueStat) { + this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, queueId), 1, 1); + } } public void incQueuePutNums(final String topic, final Integer queueId, int num, int times) { - this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, queueId), num, times); + if (enableQueueStat) { + this.statsTable.get(QUEUE_PUT_NUMS).addValue(buildStatsKey(topic, queueId), num, times); + } } public void incQueuePutSize(final String topic, final Integer queueId, final int size) { - this.statsTable.get(QUEUE_PUT_SIZE).addValue(buildStatsKey(topic, queueId), size, 1); + if (enableQueueStat) { + this.statsTable.get(QUEUE_PUT_SIZE).addValue(buildStatsKey(topic, queueId), size, 1); + } } public void incQueueGetNums(final String group, final String topic, final Integer queueId, final int incValue) { - final String statsKey = buildStatsKey(topic, queueId, group); - this.statsTable.get(QUEUE_GET_NUMS).addValue(statsKey, incValue, 1); + if (enableQueueStat) { + final String statsKey = buildStatsKey(topic, queueId, group); + this.statsTable.get(QUEUE_GET_NUMS).addValue(statsKey, incValue, 1); + } } public void incQueueGetSize(final String group, final String topic, final Integer queueId, final int incValue) { - final String statsKey = buildStatsKey(topic, queueId, group); - this.statsTable.get(QUEUE_GET_SIZE).addValue(statsKey, incValue, 1); + if (enableQueueStat) { + final String statsKey = buildStatsKey(topic, queueId, group); + this.statsTable.get(QUEUE_GET_SIZE).addValue(statsKey, incValue, 1); + } } public void incTopicPutNums(final String topic) { @@ -244,8 +262,13 @@ public void incGroupGetSize(final String group, final String topic, final int in } public void incGroupGetLatency(final String group, final String topic, final int queueId, final int incValue) { - final String statsKey = buildStatsKey(queueId, topic, group); - this.statsTable.get(GROUP_GET_LATENCY).addValue(statsKey, incValue, 1); + String statsKey; + if (enableQueueStat) { + statsKey = buildStatsKey(queueId, topic, group); + } else { + statsKey = buildStatsKey(topic, group); + } + this.statsTable.get(GROUP_GET_LATENCY).addRTValue(statsKey, incValue, 1); } public void incBrokerPutNums() { diff --git a/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java index 3bc52e3b1d3..0d1e2f35b30 100644 --- a/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java @@ -76,7 +76,7 @@ private MessageStore buildMessageStore() throws Exception { messageStoreConfig.setFlushIntervalConsumeQueue(1); messageStoreConfig.setStorePathRootDir(System.getProperty("user.home") + File.separator + "putmessagesteststore"); messageStoreConfig.setStorePathCommitLog(System.getProperty("user.home") + File.separator + "putmessagesteststore" + File.separator + "commitlog"); - return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest"), new MyMessageArrivingListener(), new BrokerConfig()); + return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), new MyMessageArrivingListener(), new BrokerConfig()); } @Test diff --git a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java index 7c57813da7e..7654e0a288d 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java @@ -130,7 +130,7 @@ protected DefaultMessageStore gen() throws Exception { DefaultMessageStore master = new DefaultMessageStore( messageStoreConfig, - new BrokerStatsManager(brokerConfig.getBrokerClusterName()), + new BrokerStatsManager(brokerConfig.getBrokerClusterName(), brokerConfig.isEnableDetailStat()), new MessageArrivingListener() { @Override public void arriving(String topic, int queueId, long logicOffset, long tagsCode, diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java index 69c1673ee48..d8202eb539f 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java @@ -486,7 +486,7 @@ private MessageStoreConfig genMessageStoreConfig(String deleteWhen, int diskMaxU private void initMessageStore(MessageStoreConfig messageStoreConfig, double diskSpaceCleanForciblyRatio) throws Exception { messageStore = new DefaultMessageStore(messageStoreConfig, - new BrokerStatsManager("test"), new MyMessageArrivingListener(), new BrokerConfig()); + new BrokerStatsManager("test", true), new MyMessageArrivingListener(), new BrokerConfig()); cleanCommitLogService = getCleanCommitLogService(diskSpaceCleanForciblyRatio); cleanConsumeQueueService = getCleanConsumeQueueService(); diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java index db7d367e021..788bdbdea9d 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java @@ -70,7 +70,7 @@ public DefaultMessageStore buildMessageStore() throws Exception { messageStoreConfig.setMaxHashSlotNum(10000); messageStoreConfig.setMaxIndexNum(100 * 100); messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); - return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest"), null, new BrokerConfig()); + return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), null, new BrokerConfig()); } diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java index f3e619d7784..96451e3c6bf 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java @@ -112,7 +112,7 @@ private MessageStore buildMessageStore() throws Exception { messageStoreConfig.setMaxIndexNum(100 * 100); messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); messageStoreConfig.setFlushIntervalConsumeQueue(1); - return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest"), new MyMessageArrivingListener(), new BrokerConfig()); + return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), new MyMessageArrivingListener(), new BrokerConfig()); } @Test diff --git a/store/src/test/java/org/apache/rocketmq/store/HATest.java b/store/src/test/java/org/apache/rocketmq/store/HATest.java index a2702a0f67a..c82a237774c 100644 --- a/store/src/test/java/org/apache/rocketmq/store/HATest.java +++ b/store/src/test/java/org/apache/rocketmq/store/HATest.java @@ -60,7 +60,7 @@ public class HATest { private MessageStore slaveMessageStore; private MessageStoreConfig masterMessageStoreConfig; private MessageStoreConfig slaveStoreConfig; - private BrokerStatsManager brokerStatsManager = new BrokerStatsManager("simpleTest"); + private BrokerStatsManager brokerStatsManager = new BrokerStatsManager("simpleTest", true); private String storePathRootParentDir = System.getProperty("user.home") + File.separator + UUID.randomUUID().toString().replace("-", ""); private String storePathRootDir = storePathRootParentDir + File.separator + "store"; diff --git a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java index 0eafe4c2f7f..bfcebd7eb85 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java @@ -79,6 +79,6 @@ private MessageStore buildMessageStore() throws Exception { messageStoreConfig.setMaxIndexNum(100 * 100); messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); messageStoreConfig.setFlushIntervalConsumeQueue(1); - return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest"), null, new BrokerConfig()); + return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), null, new BrokerConfig()); } } diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java b/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java index 5864b282c7b..7a77e9545f3 100644 --- a/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java +++ b/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java @@ -56,7 +56,7 @@ protected DefaultMessageStore createDledgerMessageStore(String base, String grou storeConfig.setdLegerGroup(group); storeConfig.setdLegerPeers(peers); storeConfig.setdLegerSelfId(selfId); - DefaultMessageStore defaultMessageStore = new DefaultMessageStore(storeConfig, new BrokerStatsManager("DLedgerCommitlogTest"), (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { + DefaultMessageStore defaultMessageStore = new DefaultMessageStore(storeConfig, new BrokerStatsManager("DLedgerCommitlogTest", true), (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { }, new BrokerConfig()); DLedgerServer dLegerServer = ((DLedgerCommitLog) defaultMessageStore.getCommitLog()).getdLedgerServer(); @@ -106,7 +106,7 @@ protected DefaultMessageStore createMessageStore(String base, boolean createAbor storeConfig.setStorePathRootDir(base); storeConfig.setStorePathCommitLog(base + File.separator + "commitlog"); storeConfig.setFlushDiskType(FlushDiskType.ASYNC_FLUSH); - DefaultMessageStore defaultMessageStore = new DefaultMessageStore(storeConfig, new BrokerStatsManager("CommitlogTest"), (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { + DefaultMessageStore defaultMessageStore = new DefaultMessageStore(storeConfig, new BrokerStatsManager("CommitlogTest", true), (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { }, new BrokerConfig()); diff --git a/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java index d375fb0c89c..de3cf7f5945 100644 --- a/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java @@ -103,7 +103,7 @@ public void init() throws Exception { messageStoreConfig.setStorePathCommitLog(storePath + File.separator + "commitlog"); brokerConfig = new BrokerConfig(); - BrokerStatsManager manager = new BrokerStatsManager(brokerConfig.getBrokerClusterName()); + BrokerStatsManager manager = new BrokerStatsManager(brokerConfig.getBrokerClusterName(), brokerConfig.isEnableDetailStat()); messageStore = new DefaultMessageStore(messageStoreConfig, manager, new MyMessageArrivingListener(), new BrokerConfig()); assertThat(messageStore.load()).isTrue(); diff --git a/store/src/test/java/stats/BrokerStatsManagerTest.java b/store/src/test/java/stats/BrokerStatsManagerTest.java index 2b6d0f8b68d..137c37d0f68 100644 --- a/store/src/test/java/stats/BrokerStatsManagerTest.java +++ b/store/src/test/java/stats/BrokerStatsManagerTest.java @@ -47,7 +47,7 @@ public class BrokerStatsManagerTest { @Before public void init() { - brokerStatsManager = new BrokerStatsManager("DefaultCluster"); + brokerStatsManager = new BrokerStatsManager("DefaultCluster", true); brokerStatsManager.start(); } From a2d94242f3632dc71ce29e2a7bba0a91b20a6b8f Mon Sep 17 00:00:00 2001 From: rongtong Date: Wed, 15 Dec 2021 20:09:34 +0800 Subject: [PATCH 064/141] Fix bug that the broker will hang after merge the pr that fix the headWaitTimeMills of sendThreadPoolQueue (#3631) --- .../rocketmq/broker/BrokerController.java | 32 ++++++++++++------- .../processor/SendMessageProcessor.java | 2 +- .../rocketmq/broker/BrokerControllerTest.java | 2 -- .../apache/rocketmq/common/BrokerConfig.java | 18 +++++++++++ 4 files changed, 40 insertions(+), 14 deletions(-) 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 662ec498144..1f72e27f673 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -33,8 +33,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.Optional; -import java.util.Objects; import org.apache.rocketmq.acl.AccessValidator; import org.apache.rocketmq.broker.client.ClientHousekeepingService; import org.apache.rocketmq.broker.client.ConsumerIdsChangeListener; @@ -134,6 +132,7 @@ public class BrokerController { "BrokerControllerScheduledThread")); private final SlaveSynchronize slaveSynchronize; private final BlockingQueue sendThreadPoolQueue; + private final BlockingQueue putThreadPoolQueue; private final BlockingQueue pullThreadPoolQueue; private final BlockingQueue replyThreadPoolQueue; private final BlockingQueue queryThreadPoolQueue; @@ -150,6 +149,7 @@ public class BrokerController { private RemotingServer fastRemotingServer; private TopicConfigManager topicConfigManager; private ExecutorService sendMessageExecutor; + private ExecutorService putMessageFutureExecutor; private ExecutorService pullMessageExecutor; private ExecutorService replyMessageExecutor; private ExecutorService queryMessageExecutor; @@ -198,6 +198,7 @@ public BrokerController( this.slaveSynchronize = new SlaveSynchronize(this); this.sendThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getSendThreadPoolQueueCapacity()); + this.putThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getPutThreadPoolQueueCapacity()); this.pullThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getPullThreadPoolQueueCapacity()); this.replyThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getReplyThreadPoolQueueCapacity()); this.queryThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getQueryThreadPoolQueueCapacity()); @@ -275,6 +276,14 @@ public boolean initialize() throws CloneNotSupportedException { this.sendThreadPoolQueue, new ThreadFactoryImpl("SendMessageThread_")); + this.putMessageFutureExecutor = new BrokerFixedThreadPoolExecutor( + this.brokerConfig.getPutMessageFutureThreadPoolNums(), + this.brokerConfig.getPutMessageFutureThreadPoolNums(), + 1000 * 60, + TimeUnit.MILLISECONDS, + this.putThreadPoolQueue, + new ThreadFactoryImpl("PutMessageThread_")); + this.pullMessageExecutor = new BrokerFixedThreadPoolExecutor( this.brokerConfig.getPullMessageThreadPoolNums(), this.brokerConfig.getPullMessageThreadPoolNums(), @@ -652,13 +661,10 @@ public void protectBroker() { public long headSlowTimeMills(BlockingQueue q) { long slowTimeMills = 0; - Optional op = q.stream() - .map(BrokerFastFailure::castRunnable) - .filter(Objects::nonNull) - .findFirst(); - if (op.isPresent()) { - RequestTask rt = op.get(); - slowTimeMills = this.messageStore.now() - rt.getCreateTimestamp(); + final Runnable peek = q.peek(); + if (peek != null) { + RequestTask rt = BrokerFastFailure.castRunnable(peek); + slowTimeMills = rt == null ? 0 : this.messageStore.now() - rt.getCreateTimestamp(); } if (slowTimeMills < 0) { @@ -787,6 +793,10 @@ public void shutdown() { this.sendMessageExecutor.shutdown(); } + if (this.putMessageFutureExecutor != null) { + this.putMessageFutureExecutor.shutdown(); + } + if (this.pullMessageExecutor != null) { this.pullMessageExecutor.shutdown(); } @@ -1245,7 +1255,7 @@ private void shutdownProcessorByHa() { } } - public ExecutorService getSendMessageExecutor() { - return sendMessageExecutor; + public ExecutorService getPutMessageFutureExecutor() { + return putMessageFutureExecutor; } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java index b31c71e06d9..f5ebf3a6a35 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java @@ -84,7 +84,7 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, @Override public void asyncProcessRequest(ChannelHandlerContext ctx, RemotingCommand request, RemotingResponseCallback responseCallback) throws Exception { - asyncProcessRequest(ctx, request).thenAcceptAsync(responseCallback::callback, this.brokerController.getSendMessageExecutor()); + asyncProcessRequest(ctx, request).thenAcceptAsync(responseCallback::callback, this.brokerController.getPutMessageFutureExecutor()); } public CompletableFuture asyncProcessRequest(ChannelHandlerContext ctx, diff --git a/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java index e8442a4d4b5..9706334187e 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java @@ -71,7 +71,6 @@ public void run() { } }; - queue.add(runnable); RequestTask requestTask = new RequestTask(runnable, null, null); // the requestTask is not the head of queue; @@ -80,6 +79,5 @@ public void run() { long headSlowTimeMills = 100; TimeUnit.MILLISECONDS.sleep(headSlowTimeMills); assertThat(brokerController.headSlowTimeMills(queue)).isGreaterThanOrEqualTo(headSlowTimeMills); - //Attention: if we use the previous version method BrokerController#headSlowTimeMills, it will return 0; } } 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 a45efbe2f17..0e472fb16dc 100644 --- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java @@ -60,6 +60,7 @@ public class BrokerConfig { * thread numbers for send message thread pool. */ private int sendMessageThreadPoolNums = Math.min(Runtime.getRuntime().availableProcessors(), 4); + private int putMessageFutureThreadPoolNums = Math.min(Runtime.getRuntime().availableProcessors(), 4); private int pullMessageThreadPoolNums = 16 + Runtime.getRuntime().availableProcessors() * 2; private int processReplyMessageThreadPoolNums = 16 + Runtime.getRuntime().availableProcessors() * 2; private int queryMessageThreadPoolNums = 8 + Runtime.getRuntime().availableProcessors(); @@ -84,6 +85,7 @@ public class BrokerConfig { @ImportantField private boolean fetchNamesrvAddrByAddressServer = false; private int sendThreadPoolQueueCapacity = 10000; + private int putThreadPoolQueueCapacity = 10000; private int pullThreadPoolQueueCapacity = 100000; private int replyThreadPoolQueueCapacity = 10000; private int queryThreadPoolQueueCapacity = 20000; @@ -375,6 +377,14 @@ public void setSendMessageThreadPoolNums(int sendMessageThreadPoolNums) { this.sendMessageThreadPoolNums = sendMessageThreadPoolNums; } + public int getPutMessageFutureThreadPoolNums() { + return putMessageFutureThreadPoolNums; + } + + public void setPutMessageFutureThreadPoolNums(int putMessageFutureThreadPoolNums) { + this.putMessageFutureThreadPoolNums = putMessageFutureThreadPoolNums; + } + public int getPullMessageThreadPoolNums() { return pullMessageThreadPoolNums; } @@ -479,6 +489,14 @@ public void setSendThreadPoolQueueCapacity(int sendThreadPoolQueueCapacity) { this.sendThreadPoolQueueCapacity = sendThreadPoolQueueCapacity; } + public int getPutThreadPoolQueueCapacity() { + return putThreadPoolQueueCapacity; + } + + public void setPutThreadPoolQueueCapacity(int putThreadPoolQueueCapacity) { + this.putThreadPoolQueueCapacity = putThreadPoolQueueCapacity; + } + public int getPullThreadPoolQueueCapacity() { return pullThreadPoolQueueCapacity; } From ecb061ae425f76ad28a597293b98f374ceba14ec Mon Sep 17 00:00:00 2001 From: Chengyang He <21126659+Aaron-He@users.noreply.github.com> Date: Wed, 15 Dec 2021 20:47:55 +0800 Subject: [PATCH 065/141] fix code style of PR #3475 (#3650) --- .../acl/plain/PlainAccessValidatorTest.java | 19 +++++----- .../remoting/common/RemotingHelper.java | 2 +- .../remoting/protocol/RemotingCommand.java | 23 ++++++------ .../protocol/RocketMQSerializable.java | 35 +++++++++---------- .../protocol/RemotingCommandTest.java | 1 - .../protocol/RocketMQSerializableTest.java | 1 - 6 files changed, 39 insertions(+), 42 deletions(-) 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 e6954c24544..fb8eba1e920 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 @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.acl.plain; - import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; @@ -24,7 +23,6 @@ import java.util.List; import java.util.Map; import java.util.Set; - import org.apache.rocketmq.acl.common.AclClientRPCHook; import org.apache.rocketmq.acl.common.AclConstants; import org.apache.rocketmq.acl.common.AclException; @@ -34,7 +32,14 @@ import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.PlainAccessConfig; import org.apache.rocketmq.common.protocol.RequestCode; -import org.apache.rocketmq.common.protocol.header.*; +import org.apache.rocketmq.common.protocol.header.ConsumerSendMsgBackRequestHeader; +import org.apache.rocketmq.common.protocol.header.GetConsumerListByGroupRequestHeader; +import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader; +import org.apache.rocketmq.common.protocol.header.QueryMessageRequestHeader; +import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader; +import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeaderV2; +import org.apache.rocketmq.common.protocol.header.UnregisterClientRequestHeader; +import org.apache.rocketmq.common.protocol.header.UpdateConsumerOffsetRequestHeader; import org.apache.rocketmq.common.protocol.heartbeat.ConsumerData; import org.apache.rocketmq.common.protocol.heartbeat.HeartbeatData; import org.apache.rocketmq.common.protocol.heartbeat.ProducerData; @@ -497,7 +502,6 @@ public void updateAccessAclYamlConfigTest() { AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } - @Test public void createAndUpdateAccessAclYamlConfigNormalTest() { System.setProperty("rocketmq.home.dir", "src/test/resources"); @@ -570,7 +574,6 @@ public void createAndUpdateAccessAclYamlConfigNormalTest() { Assert.assertEquals(2, dataVersions2.get(0).get(AclConstants.CONFIG_COUNTER)); Assert.assertEquals(verifyMap2.get(AclConstants.CONFIG_SECRET_KEY), "1234567890123"); - // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } @@ -597,7 +600,6 @@ public void deleteAccessAclYamlConfigNormalTest() { String targetFileName = "src/test/resources/conf/plain_acl_delete.yml"; Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - String accessKey = "rocketmq2"; PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); plainAccessValidator.deleteAccessConfig(accessKey); @@ -709,7 +711,6 @@ public void getAllAclConfigTest() { Assert.assertEquals(aclConfig.getPlainAccessConfigs().size(), 2); } - @Test public void updateAccessConfigEmptyPermListTest() { PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); @@ -724,7 +725,7 @@ public void updateAccessConfigEmptyPermListTest() { plainAccessValidator.updateAccessConfig(plainAccessConfig); PlainAccessConfig result = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs() - .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); + .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); Assert.assertEquals(0, result.getTopicPerms().size()); plainAccessValidator.deleteAccessConfig(accessKey); @@ -744,7 +745,7 @@ public void updateAccessConfigEmptyWhiteRemoteAddressTest() { plainAccessValidator.updateAccessConfig(plainAccessConfig); PlainAccessConfig result = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs() - .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); + .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); Assert.assertEquals("", result.getWhiteRemoteAddress()); plainAccessValidator.deleteAccessConfig(accessKey); diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java index 1a366669846..b738e1718a0 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java @@ -62,7 +62,7 @@ public static SocketAddress string2SocketAddress(final String addr) { public static RemotingCommand invokeSync(final String addr, final RemotingCommand request, final long timeoutMillis) throws InterruptedException, RemotingConnectException, - RemotingSendRequestException, RemotingTimeoutException, RemotingCommandException { + RemotingSendRequestException, RemotingTimeoutException, RemotingCommandException { long beginTime = System.currentTimeMillis(); SocketAddress socketAddress = RemotingUtil.string2SocketAddress(addr); SocketChannel socketChannel = RemotingUtil.connect(socketAddress); diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java index 0eb01c9a21d..d43fe8ddadd 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java @@ -17,13 +17,6 @@ package org.apache.rocketmq.remoting.protocol; import com.alibaba.fastjson.annotation.JSONField; -import org.apache.rocketmq.logging.InternalLogger; -import org.apache.rocketmq.logging.InternalLoggerFactory; -import org.apache.rocketmq.remoting.CommandCustomHeader; -import org.apache.rocketmq.remoting.annotation.CFNotNull; -import org.apache.rocketmq.remoting.common.RemotingHelper; -import org.apache.rocketmq.remoting.exception.RemotingCommandException; - import java.lang.annotation.Annotation; import java.lang.reflect.Field; import java.lang.reflect.Modifier; @@ -31,6 +24,12 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.common.RemotingHelper; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; public class RemotingCommand { public static final String SERIALIZE_TYPE_PROPERTY = "rocketmq.serialize.type"; @@ -40,7 +39,7 @@ public class RemotingCommand { private static final int RPC_TYPE = 0; // 0, REQUEST_COMMAND private static final int RPC_ONEWAY = 1; // 0, RPC private static final Map, Field[]> CLASS_HASH_MAP = - new HashMap, Field[]>(); + new HashMap, Field[]>(); private static final Map CANONICAL_NAME_CACHE = new HashMap(); // 1, Oneway // 1, RESPONSE_COMMAND @@ -112,7 +111,7 @@ public static RemotingCommand createResponseCommand(Class classHeader) { + Class classHeader) { RemotingCommand cmd = new RemotingCommand(); cmd.markResponseType(); cmd.setCode(code); @@ -233,7 +232,7 @@ public void writeCustomHeader(CommandCustomHeader customHeader) { } public CommandCustomHeader decodeCommandCustomHeader( - Class classHeader) throws RemotingCommandException { + Class classHeader) throws RemotingCommandException { CommandCustomHeader objectHeader; try { objectHeader = classHeader.newInstance(); @@ -530,8 +529,8 @@ public void addExtField(String key, String value) { @Override public String toString() { return "RemotingCommand [code=" + code + ", language=" + language + ", version=" + version + ", opaque=" + opaque + ", flag(B)=" - + Integer.toBinaryString(flag) + ", remark=" + remark + ", extFields=" + extFields + ", serializeTypeCurrentRPC=" - + serializeTypeCurrentRPC + "]"; + + Integer.toBinaryString(flag) + ", remark=" + remark + ", extFields=" + extFields + ", serializeTypeCurrentRPC=" + + serializeTypeCurrentRPC + "]"; } public SerializeType getSerializeTypeCurrentRPC() { diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java index 6075d7ab6db..7c222bf4529 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializable.java @@ -16,13 +16,12 @@ */ package org.apache.rocketmq.remoting.protocol; -import org.apache.rocketmq.remoting.exception.RemotingCommandException; - import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; public class RocketMQSerializable { private static final Charset CHARSET_UTF8 = Charset.forName("UTF-8"); @@ -87,10 +86,10 @@ public static byte[] mapSerialize(HashMap map) { Map.Entry entry = it.next(); if (entry.getKey() != null && entry.getValue() != null) { kvLength = - // keySize + Key - 2 + entry.getKey().getBytes(CHARSET_UTF8).length - // valSize + val - + 4 + entry.getValue().getBytes(CHARSET_UTF8).length; + // keySize + Key + 2 + entry.getKey().getBytes(CHARSET_UTF8).length + // valSize + val + + 4 + entry.getValue().getBytes(CHARSET_UTF8).length; totalLength += kvLength; } } @@ -119,18 +118,18 @@ public static byte[] mapSerialize(HashMap map) { private static int calTotalLen(int remark, int ext) { // int code(~32767) int length = 2 - // LanguageCode language - + 1 - // int version(~32767) - + 2 - // int opaque - + 4 - // int flag - + 4 - // String remark - + 4 + remark - // HashMap extFields - + 4 + ext; + // LanguageCode language + + 1 + // int version(~32767) + + 2 + // int opaque + + 4 + // int flag + + 4 + // String remark + + 4 + remark + // HashMap extFields + + 4 + ext; return length; } diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java index 0b19d445724..dd77c32e64b 100644 --- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingCommandTest.java @@ -130,7 +130,6 @@ public void testEncodeAndDecode_EmptyBody() { Assert.fail("Should not throw IOException"); } - } @Test diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java index 149ec721c8b..5732b9596d1 100644 --- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RocketMQSerializableTest.java @@ -17,7 +17,6 @@ package org.apache.rocketmq.remoting.protocol; import java.util.HashMap; - import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.junit.Assert; import org.junit.Test; From ed710abf85fee3b660381c4225f45435345eec74 Mon Sep 17 00:00:00 2001 From: ltamber Date: Thu, 16 Dec 2021 20:19:00 +0800 Subject: [PATCH 066/141] bump up log4j-slf4j-impl version to 2.15.0 (#3623) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 59813ba0a40..475918c932e 100644 --- a/pom.xml +++ b/pom.xml @@ -592,7 +592,7 @@ org.apache.logging.log4j log4j-slf4j-impl - 2.7 + 2.15.0 commons-validator From 17d25ea56b44625ae04e7c3181d24b0e80acaf77 Mon Sep 17 00:00:00 2001 From: tianliuliu <643422162@qq.com> Date: Thu, 16 Dec 2021 21:15:12 +0800 Subject: [PATCH 067/141] [ISSUE #3651] for add netty channel option WRITE_BUFFER_WATER_MARK --- .../remoting/netty/NettyClientConfig.java | 19 +++++++++++++++++++ .../remoting/netty/NettyRemotingClient.java | 3 +++ .../remoting/netty/NettyRemotingServer.java | 3 +++ .../remoting/netty/NettyServerConfig.java | 18 ++++++++++++++++++ .../remoting/netty/NettySystemConfig.java | 8 ++++++++ 5 files changed, 51 insertions(+) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java index 5ba3534cb8a..c1b9345c3f4 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java @@ -40,6 +40,9 @@ public class NettyClientConfig { private boolean useTLS; + private int writeBufferHighWaterMark = NettySystemConfig.writeBufferHighWaterMark; + private int writeBufferLowWaterMark = NettySystemConfig.writeBufferLowWaterMark; + public boolean isClientCloseSocketIfTimeout() { return clientCloseSocketIfTimeout; } @@ -135,4 +138,20 @@ public boolean isUseTLS() { public void setUseTLS(boolean useTLS) { this.useTLS = useTLS; } + + public int getWriteBufferLowWaterMark() { + return writeBufferLowWaterMark; + } + + public void setWriteBufferLowWaterMark(int writeBufferLowWaterMark) { + this.writeBufferLowWaterMark = writeBufferLowWaterMark; + } + + public int getWriteBufferHighWaterMark() { + return writeBufferHighWaterMark; + } + + public void setWriteBufferHighWaterMark(int writeBufferHighWaterMark) { + this.writeBufferHighWaterMark = writeBufferHighWaterMark; + } } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java index 5ba6cfab0a7..569746554d6 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java @@ -27,6 +27,7 @@ import io.netty.channel.ChannelPromise; import io.netty.channel.EventLoopGroup; import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.channel.WriteBufferWaterMark; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioSocketChannel; @@ -167,6 +168,8 @@ public Thread newThread(Runnable r) { .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, nettyClientConfig.getConnectTimeoutMillis()) .option(ChannelOption.SO_SNDBUF, nettyClientConfig.getClientSocketSndBufSize()) .option(ChannelOption.SO_RCVBUF, nettyClientConfig.getClientSocketRcvBufSize()) + .option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(nettyClientConfig.getWriteBufferLowWaterMark(), + nettyClientConfig.getWriteBufferHighWaterMark())) .handler(new ChannelInitializer() { @Override public void initChannel(SocketChannel ch) throws Exception { diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java index 1586472e042..2b7413f7a90 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java @@ -28,6 +28,7 @@ import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.channel.WriteBufferWaterMark; import io.netty.channel.epoll.Epoll; import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.epoll.EpollServerSocketChannel; @@ -204,6 +205,8 @@ public Thread newThread(Runnable r) { .childOption(ChannelOption.TCP_NODELAY, true) .childOption(ChannelOption.SO_SNDBUF, nettyServerConfig.getServerSocketSndBufSize()) .childOption(ChannelOption.SO_RCVBUF, nettyServerConfig.getServerSocketRcvBufSize()) + .childOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark( + nettyServerConfig.getWriteBufferLowWaterMark(), nettyServerConfig.getWriteBufferHighWaterMark())) .localAddress(new InetSocketAddress(this.nettyServerConfig.getListenPort())) .childHandler(new ChannelInitializer() { @Override diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java index 8708471642f..bd87e5b94e3 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java @@ -27,6 +27,8 @@ public class NettyServerConfig implements Cloneable { private int serverSocketSndBufSize = NettySystemConfig.socketSndbufSize; private int serverSocketRcvBufSize = NettySystemConfig.socketRcvbufSize; + private int writeBufferHighWaterMark = NettySystemConfig.writeBufferHighWaterMark; + private int writeBufferLowWaterMark = NettySystemConfig.writeBufferLowWaterMark; private int serverSocketBacklog = NettySystemConfig.socketBacklog; private boolean serverPooledByteBufAllocatorEnable = true; @@ -139,4 +141,20 @@ public void setUseEpollNativeSelector(boolean useEpollNativeSelector) { public Object clone() throws CloneNotSupportedException { return (NettyServerConfig) super.clone(); } + + public int getWriteBufferLowWaterMark() { + return writeBufferLowWaterMark; + } + + public void setWriteBufferLowWaterMark(int writeBufferLowWaterMark) { + this.writeBufferLowWaterMark = writeBufferLowWaterMark; + } + + public int getWriteBufferHighWaterMark() { + return writeBufferHighWaterMark; + } + + public void setWriteBufferHighWaterMark(int writeBufferHighWaterMark) { + this.writeBufferHighWaterMark = writeBufferHighWaterMark; + } } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java index 85f30f515bf..4290baea7a9 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java @@ -38,6 +38,10 @@ public class NettySystemConfig { "com.rocketmq.remoting.client.channel.maxIdleTimeSeconds"; public static final String COM_ROCKETMQ_REMOTING_CLIENT_CLOSE_SOCKET_IF_TIMEOUT = "com.rocketmq.remoting.client.closeSocketIfTimeout"; + public static final String COM_ROCKETMQ_REMOTING_WRITE_BUFFER_HIGH_WATER_MARK_VALUE = + "com.rocketmq.remoting.write.buffer.high.water.mark"; + public static final String COM_ROCKETMQ_REMOTING_WRITE_BUFFER_LOW_WATER_MARK = + "com.rocketmq.remoting.write.buffer.low.water.mark"; public static final boolean NETTY_POOLED_BYTE_BUF_ALLOCATOR_ENABLE = // Boolean.parseBoolean(System.getProperty(COM_ROCKETMQ_REMOTING_NETTY_POOLED_BYTE_BUF_ALLOCATOR_ENABLE, "false")); @@ -59,5 +63,9 @@ public class NettySystemConfig { Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_CLIENT_CHANNEL_MAX_IDLE_SECONDS, "120")); public static boolean clientCloseSocketIfTimeout = Boolean.parseBoolean(System.getProperty(COM_ROCKETMQ_REMOTING_CLIENT_CLOSE_SOCKET_IF_TIMEOUT, "true")); + public static int writeBufferHighWaterMark = + Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_WRITE_BUFFER_HIGH_WATER_MARK_VALUE, "4194304"));//4M + public static int writeBufferLowWaterMark = + Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_WRITE_BUFFER_LOW_WATER_MARK, "1048576")); //1MB } From 8cb14687f3946c95092631d9f9d1447024a741de Mon Sep 17 00:00:00 2001 From: rongtong Date: Fri, 17 Dec 2021 09:53:09 +0800 Subject: [PATCH 068/141] [ISSUE #3654] Polish travis.yml to run integration tests (#3655) * 1.Make travis to run integration tests 2.Fix running testSynSendNullMessage failed * Remove run 'mvn clean install -Pit-test' after success * Change package to install when running UT --- .travis.yml | 4 ++-- .../client/producer/exception/msg/MessageExceptionIT.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index bcf058cb5c9..669d9e9bd3f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -45,9 +45,9 @@ before_script: script: - mvn verify -DskipTests - travis_retry mvn -B clean apache-rat:check - - travis_retry mvn -B package jacoco:report coveralls:report + - travis_retry mvn -B install jacoco:report coveralls:report + - travis_retry mvn -B clean install -pl test -Pit-test after_success: - - mvn clean install -Pit-test - mvn sonar:sonar -Psonar-apache - bash <(curl -s https://codecov.io/bash) || echo 'Codecov failed to upload' diff --git a/test/src/test/java/org/apache/rocketmq/test/client/producer/exception/msg/MessageExceptionIT.java b/test/src/test/java/org/apache/rocketmq/test/client/producer/exception/msg/MessageExceptionIT.java index 716ac511afa..7d96f20b91d 100644 --- a/test/src/test/java/org/apache/rocketmq/test/client/producer/exception/msg/MessageExceptionIT.java +++ b/test/src/test/java/org/apache/rocketmq/test/client/producer/exception/msg/MessageExceptionIT.java @@ -59,7 +59,7 @@ public void testProducerSmoke() { assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK); } - @Test(expected = org.apache.rocketmq.client.exception.MQClientException.class) + @Test(expected = java.lang.NullPointerException.class) public void testSynSendNullMessage() throws Exception { producer.send((Message) null); } From 7591227a8364f7a731f5e4e79d81a04313ddc51e Mon Sep 17 00:00:00 2001 From: ChaosYjh <763900954@qq.com> Date: Mon, 20 Dec 2021 18:45:48 +0800 Subject: [PATCH 069/141] [ISSUE #1486] fix CleanUnusedTopicCommand performs wrong (#1487) Co-authored-by: yuanjunhui --- .../java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java | 2 +- .../rocketmq/tools/command/broker/CleanUnusedTopicCommand.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java index c3e3a30d6bf..3ffe33f6fbe 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java @@ -424,7 +424,7 @@ public boolean cleanExpiredConsumerQueueByAddr( @Override public boolean cleanUnusedTopic(String cluster) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQClientException, InterruptedException { - return defaultMQAdminExtImpl.cleanUnusedTopicByAddr(cluster); + return defaultMQAdminExtImpl.cleanUnusedTopic(cluster); } @Override diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommand.java index 2aabeeb4b81..045ba8e44f1 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommand.java @@ -66,7 +66,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t String cluster = commandLine.getOptionValue('c'); if (null != cluster) cluster = cluster.trim(); - result = defaultMQAdminExt.cleanUnusedTopicByAddr(cluster); + result = defaultMQAdminExt.cleanUnusedTopic(cluster); } System.out.printf(result ? "success" : "false"); } catch (Exception e) { From 6ac5f4989750f392d2be9e5976d13a0bc6034292 Mon Sep 17 00:00:00 2001 From: Jun <62043001+coder-zzzz@users.noreply.github.com> Date: Mon, 20 Dec 2021 19:45:40 +0800 Subject: [PATCH 070/141] [ISSUE #2472] Add producer best practice for english --- docs/en/best_practice.md | 66 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/docs/en/best_practice.md b/docs/en/best_practice.md index 213dc3174d9..c50c5df415b 100755 --- a/docs/en/best_practice.md +++ b/docs/en/best_practice.md @@ -1,6 +1,72 @@ # Best practices ## 1 Producer +### 1.1 Attention of send message + +#### 1 Uses of tags +An application should use one topic as far as possible, but identify the message's subtype with tags.Tags can be set freely by the application. +Only when producers set tags while sending messages, can consumers to filter messages through broker with tags when subscribing messages: message.setTags("TagA"). + +#### 2 Uses of keys +The unique identifier for each message at the business level set to the Keys field to help locate message loss problems in the future. +The server creates an index(hash index) for each message, and the application can query the message content via Topic,key,and who consumed the message. +Since it is a hash index, make sure that the key is as unique as possible to avoid potential hash conflicts. + +```java + // order id + String orderId = "20034568923546"; + message.setKeys(orderId); +``` +#### 3 Log print +Print the message log when send success or failed, make sure to print the SendResult and key fields. +Send messages is successful as long as it does not throw exception. Send successful will have multiple states defined in sendResult. +Each state is describing below: + +- **SEND_OK** + +Message send successfully.Note that even though message send successfully, but it doesn't mean than it is reliable. +To make sure nothing lost, you should also enable the SYNC_MASTER or SYNC_FLUSH. + +- **FLUSH_DISK_TIMEOUT** + +Message send successfully, but the server flush messages to disk timeout.At this point, the message has entered the server's memory, and the message will be lost only when the server is down. +Flush mode and sync flush time interval can be set in the configuration parameters. It will return FLUSH_DISK_TIMEOUT when Broker server doesn't finish flush message to disk in timout(default is 5s +) when sets FlushDiskType=SYNC_FLUSH(default is async flush). + +- **FLUSH_SLAVE_TIMEOUT** + +Message send successfully, but sync to slave timeout.At this point, the message has entered the server's memory, and the message will be lost only when the server is down. +It will return FLUSH_SLAVE_TIMEOUT when Broker server role is SYNC_MASTER(default is ASYNC_MASTER),and it doesn't sync message to slave successfully in the timeout(default 5s). + +- **SLAVE_NOT_AVAILABLE** + +Message send successfully, but slave is not available.It will return SLAVE_NOT_AVAILABLE when Broker role is SYNC_MASTER(default is ASYNC_MASTER), and it doesn't have a slave server available. + +### 1.2 Handling of message send failure +Send method of producer itself supports internal retry. The logic of retry is as follows: +- At most twice. +- Try next broker when sync send mode, try current broker when async mode. The total elapsed time of this method does not exceed the value of sendMsgTimeout(default is 10s). +- It will not be retried when the message is sent to the Broker with a timeout exception. + +The strategy above ensures the success of message sending to some extent.If the business has a high requirement for message reliability, it is recommended to add the corresponding retry logic: +for example, if the sync send method fails, try to store the message in DB, and then retry periodically by the bg thread to ensure the message must send to broker successfully. + +Why the above DB retry strategy is not integrated into the MQ client, but requires the application to complete it by itself is mainly based on the following considerations: +First, the MQ client is designed to be stateless mode, convenient for arbitrary horizontal expansion, and only consumes CPU, memory and network resources. +Second, if the MQ client internal integration a KV storage module, the data can only be relatively reliable when sync flush to disk, but the sync flush will cause performance lose, so it's usually + use async flush.Because the application shutdown is not controlled by the MQ operators, A violent shutdown like kill -9 may often occur, resulting in data not flushed to disk and being lost. +Thirdly, the producer is a virtual machine with low reliability, which is not suitable for storing important data. +In conclusion, it is recommended that the retry process must be controlled by the application. + +### 1.3 Send message by oneway +Typically, this is the process by which messages are sent: + +- Client send request to server +- Server process request +- Server response to client +So, the time taken to send a message is the sum of the three steps above.Some scenarios require very little time, but not much reliability, such as log collect application. +This type application can use oneway to send messages. Oneway only send request without waiting for a reply, and send a request at the client implementation level is simply the overhead of an + operating system call that writes data to the client's socket buffer, this process that typically takes microseconds. ## 2 Consumer From e425c097ad6ab2e31786fc343805356363ddcf79 Mon Sep 17 00:00:00 2001 From: ssssssnake <157384081@qq.com> Date: Mon, 20 Dec 2021 20:07:45 +0800 Subject: [PATCH 071/141] [ISSUE #1097] Fix null pointer problem when consumption start time is null (#1098) --- .../rocketmq/client/impl/consumer/ProcessQueue.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ProcessQueue.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ProcessQueue.java index 21798d8c6ec..ba00aaef994 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ProcessQueue.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ProcessQueue.java @@ -27,6 +27,8 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.lang3.StringUtils; + import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.log.ClientLogger; import org.apache.rocketmq.logging.InternalLogger; @@ -85,10 +87,14 @@ public void cleanExpiredMsg(DefaultMQPushConsumer pushConsumer) { try { this.treeMapLock.readLock().lockInterruptibly(); try { - if (!msgTreeMap.isEmpty() && System.currentTimeMillis() - Long.parseLong(MessageAccessor.getConsumeStartTimeStamp(msgTreeMap.firstEntry().getValue())) > pushConsumer.getConsumeTimeout() * 60 * 1000) { - msg = msgTreeMap.firstEntry().getValue(); + if (!msgTreeMap.isEmpty()) { + String consumeStartTimeStamp = MessageAccessor.getConsumeStartTimeStamp(msgTreeMap.firstEntry().getValue()); + if (StringUtils.isNotEmpty(consumeStartTimeStamp) && System.currentTimeMillis() - Long.parseLong(consumeStartTimeStamp) > pushConsumer.getConsumeTimeout() * 60 * 1000) { + msg = msgTreeMap.firstEntry().getValue(); + } else { + break; + } } else { - break; } } finally { From e4394b6cbe682c233effc5b6c1ea7895699ad057 Mon Sep 17 00:00:00 2001 From: huangli Date: Mon, 20 Dec 2021 21:00:46 +0800 Subject: [PATCH 072/141] [ISSUE 3585] [Part C] cache the result of parseChannelRemoteAddr() and eliminate this method in flame graph. (used 4.84% cpu before this commit in producer side) (#3589) --- .../remoting/common/RemotingHelper.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java index b738e1718a0..74dfacc8059 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java @@ -17,6 +17,9 @@ package org.apache.rocketmq.remoting.common; import io.netty.channel.Channel; +import io.netty.util.Attribute; +import io.netty.util.AttributeKey; + import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; @@ -35,6 +38,7 @@ public class RemotingHelper { public static final String DEFAULT_CHARSET = "UTF-8"; private static final InternalLogger log = InternalLoggerFactory.getLogger(ROCKETMQ_REMOTING); + private static final AttributeKey REMOTE_ADDR_KEY = AttributeKey.valueOf("RemoteAddr"); public static String exceptionSimpleDesc(final Throwable e) { StringBuilder sb = new StringBuilder(); @@ -156,6 +160,20 @@ public static String parseChannelRemoteAddr(final Channel channel) { if (null == channel) { return ""; } + Attribute att = channel.attr(REMOTE_ADDR_KEY); + if (att == null) { + // mocked in unit test + return parseChannelRemoteAddr0(channel); + } + String addr = att.get(); + if (addr == null) { + addr = parseChannelRemoteAddr0(channel); + att.set(addr); + } + return addr; + } + + private static String parseChannelRemoteAddr0(final Channel channel) { SocketAddress remote = channel.remoteAddress(); final String addr = remote != null ? remote.toString() : ""; From 86c1216957d6aaa7021943815f5fbfbf0d649c0c Mon Sep 17 00:00:00 2001 From: xujianhai666 <52450794+xujianhai666@users.noreply.github.com> Date: Mon, 20 Dec 2021 21:54:02 +0800 Subject: [PATCH 073/141] [ISSUE #1843] fix: broker panic when producer send to retry topic (#1844) producer send message to retry topic directly, broker will panic because maxReconsumeTimes is null and convert to int. add null check for maxReconsumeTimes Closes #1843 --- .../apache/rocketmq/broker/processor/SendMessageProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java index f5ebf3a6a35..f925364e489 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java @@ -358,7 +358,7 @@ private boolean handleRetryAndDLQ(SendMessageRequestHeader requestHeader, Remoti } int maxReconsumeTimes = subscriptionGroupConfig.getRetryMaxTimes(); - if (request.getVersion() >= MQVersion.Version.V3_4_9.ordinal()) { + if (request.getVersion() >= MQVersion.Version.V3_4_9.ordinal() && requestHeader.getMaxReconsumeTimes() != null) { maxReconsumeTimes = requestHeader.getMaxReconsumeTimes(); } int reconsumeTimes = requestHeader.getReconsumeTimes() == null ? 0 : requestHeader.getReconsumeTimes(); From f620a50bc7a556844501e9474dae6c603e454cfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=97=AD?= Date: Mon, 20 Dec 2021 22:34:08 +0800 Subject: [PATCH 074/141] [ISSUE #2383] Export putMessageFailedTimes with broker status (#2385) Co-authored-by: zhangxu16 --- .../main/java/org/apache/rocketmq/store/StoreStatsService.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java index e8e7e048fa8..2b78f271e41 100644 --- a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java +++ b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java @@ -177,6 +177,7 @@ public String toString() { sb.append("\truntime: " + this.getFormatRuntime() + "\r\n"); sb.append("\tputMessageEntireTimeMax: " + this.putMessageEntireTimeMax + "\r\n"); sb.append("\tputMessageTimesTotal: " + totalTimes + "\r\n"); + sb.append("\tgetPutMessageFailedTimes: " + this.getPutMessageFailedTimes() + "\r\n"); sb.append("\tputMessageSizeTotal: " + this.getPutMessageSizeTotal() + "\r\n"); sb.append("\tputMessageDistributeTime: " + this.getPutMessageDistributeTimeStringInfo(totalTimes) + "\r\n"); @@ -429,6 +430,7 @@ public HashMap getRuntimeInfo() { result.put("runtime", this.getFormatRuntime()); result.put("putMessageEntireTimeMax", String.valueOf(this.putMessageEntireTimeMax)); result.put("putMessageTimesTotal", String.valueOf(totalTimes)); + result.put("putMessageFailedTimes", String.valueOf(this.putMessageFailedTimes)); result.put("putMessageSizeTotal", String.valueOf(this.getPutMessageSizeTotal())); result.put("putMessageDistributeTime", String.valueOf(this.getPutMessageDistributeTimeStringInfo(totalTimes))); From 0bc011cef3ba5ff6c1a5af9fe89109f4f0ab414d Mon Sep 17 00:00:00 2001 From: xijiu <422766572@qq.com> Date: Tue, 21 Dec 2021 14:12:31 +0800 Subject: [PATCH 075/141] [ISSUE 3613] bug fix, solve message hash conflict in index file #3616 --- .../rocketmq/client/impl/MQAdminImpl.java | 5 +- .../apache/rocketmq/test/base/BaseConf.java | 7 ++- .../producer/querymsg/QueryMsgByKeyIT.java | 57 +++++++++++++++++++ 3 files changed, 66 insertions(+), 3 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java index 8884e4adfef..ba4eafae97e 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java @@ -22,6 +22,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Set; +import java.util.Objects; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReadWriteLock; @@ -406,12 +407,14 @@ public void operationComplete(ResponseFuture responseFuture) { } } else { String keys = msgExt.getKeys(); + String msgTopic = msgExt.getTopic(); if (keys != null) { boolean matched = false; String[] keyArray = keys.split(MessageConst.KEY_SEPARATOR); if (keyArray != null) { for (String k : keyArray) { - if (key.equals(k)) { + // both topic and key must be equal at the same time + if (Objects.equals(key, k) && Objects.equals(topic, msgTopic)) { matched = true; break; } diff --git a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java index c6a835fd67f..0f1c4bfc36c 100644 --- a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java +++ b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java @@ -70,9 +70,12 @@ public BaseConf() { public static String initTopic() { String topic = MQRandomUtils.getRandomTopic(); - IntegrationTestBase.initTopic(topic, nsAddr, clusterName); + return initTopicWithName(topic); + } - return topic; + public static String initTopicWithName(String topicName) { + IntegrationTestBase.initTopic(topicName, nsAddr, clusterName); + return topicName; } public static String initConsumerGroup() { diff --git a/test/src/test/java/org/apache/rocketmq/test/client/producer/querymsg/QueryMsgByKeyIT.java b/test/src/test/java/org/apache/rocketmq/test/client/producer/querymsg/QueryMsgByKeyIT.java index 827d4f99316..d7c4364deb4 100644 --- a/test/src/test/java/org/apache/rocketmq/test/client/producer/querymsg/QueryMsgByKeyIT.java +++ b/test/src/test/java/org/apache/rocketmq/test/client/producer/querymsg/QueryMsgByKeyIT.java @@ -18,7 +18,9 @@ package org.apache.rocketmq.test.client.producer.querymsg; import java.util.List; + import org.apache.log4j.Logger; +import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.test.base.BaseConf; import org.apache.rocketmq.test.client.rmq.RMQNormalProducer; @@ -101,4 +103,59 @@ public void testQueryMax() { assertThat(queryMsgs).isNotNull(); assertThat(queryMsgs.size()).isEqualTo(max); } + + + @Test(expected = MQClientException.class) + public void testQueryMsgWithSameHash1() throws Exception { + int msgSize = 1; + String topicA = "AaTopic"; + String keyA = "Aa"; + String topicB = "BBTopic"; + String keyB = "BB"; + + initTopicWithName(topicA); + initTopicWithName(topicB); + + RMQNormalProducer producerA = getProducer(nsAddr, topicA); + RMQNormalProducer producerB = getProducer(nsAddr, topicB); + + List msgA = MQMessageFactory.getKeyMsg(topicA, keyA, msgSize); + List msgB = MQMessageFactory.getKeyMsg(topicB, keyB, msgSize); + + producerA.send(msgA); + producerB.send(msgB); + + long begin = System.currentTimeMillis() - 500000; + long end = System.currentTimeMillis() + 500000; + producerA.getProducer().queryMessage(topicA, keyB, msgSize * 10, begin, end).getMessageList(); + } + + + @Test + public void testQueryMsgWithSameHash2() throws Exception { + int msgSize = 1; + String topicA = "AaAaTopic"; + String keyA = "Aa"; + String topicB = "BBBBTopic"; + String keyB = "Aa"; + + initTopicWithName(topicA); + initTopicWithName(topicB); + + RMQNormalProducer producerA = getProducer(nsAddr, topicA); + RMQNormalProducer producerB = getProducer(nsAddr, topicB); + + List msgA = MQMessageFactory.getKeyMsg(topicA, keyA, msgSize); + List msgB = MQMessageFactory.getKeyMsg(topicB, keyB, msgSize); + + producerA.send(msgA); + producerB.send(msgB); + + long begin = System.currentTimeMillis() - 500000; + long end = System.currentTimeMillis() + 500000; + List list = producerA.getProducer().queryMessage(topicA, keyA, msgSize * 10, begin, end).getMessageList(); + + assertThat(list).isNotNull(); + assertThat(list.size()).isEqualTo(1); + } } From f1b2c4f360fdac3c4205d7275f389c4dbff44748 Mon Sep 17 00:00:00 2001 From: cnScarb Date: Thu, 23 Dec 2021 14:26:50 +0800 Subject: [PATCH 076/141] Fix spelling error in DefaultMessageStore, use getServiceName for GroupCommitService logging (#3663) --- store/src/main/java/org/apache/rocketmq/store/CommitLog.java | 2 +- .../java/org/apache/rocketmq/store/DefaultMessageStore.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 112e9bced48..05ea526c99e 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -1228,7 +1228,7 @@ public void run() { try { Thread.sleep(10); } catch (InterruptedException e) { - CommitLog.log.warn("GroupCommitService Exception, ", e); + CommitLog.log.warn(this.getServiceName() + " Exception, ", e); } synchronized (this) { diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java index ed882aa681f..1f24dc7771b 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -924,7 +924,7 @@ public boolean appendToCommitLog(long startOffset, byte[] data, int dataStart, i @Override public void executeDeleteFilesManually() { - this.cleanCommitLogService.excuteDeleteFilesManualy(); + this.cleanCommitLogService.executeDeleteFilesManually(); } @Override @@ -1590,7 +1590,7 @@ class CleanCommitLogService { private volatile boolean cleanImmediately = false; - public void excuteDeleteFilesManualy() { + public void executeDeleteFilesManually() { this.manualDeleteFileSeveralTimes = MAX_MANUAL_DELETE_FILE_TIMES; DefaultMessageStore.log.info("executeDeleteFilesManually was invoked"); } From 75557b8700bef395002a6b5b7c2bb083c120b632 Mon Sep 17 00:00:00 2001 From: rushsky518 Date: Sat, 25 Dec 2021 17:15:37 +0800 Subject: [PATCH 077/141] [ISSUE #1750] Fix display of SocketAddress is 'ocalhost/127.0.0.1:9876' * #1750 lastIndex of / * optimize code --- .../org/apache/rocketmq/remoting/common/RemotingHelper.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java index 74dfacc8059..34efc4eef38 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java @@ -192,10 +192,8 @@ private static String parseChannelRemoteAddr0(final Channel channel) { public static String parseSocketAddressAddr(SocketAddress socketAddress) { if (socketAddress != null) { final String addr = socketAddress.toString(); - - if (addr.length() > 0) { - return addr.substring(1); - } + int index = addr.lastIndexOf("/"); + return (index != -1) ? addr.substring(index + 1) : addr; } return ""; } From 6d247cb3c6bbe77166ad2fb3b7f9a953b57940f9 Mon Sep 17 00:00:00 2001 From: huangli Date: Mon, 27 Dec 2021 20:02:57 +0800 Subject: [PATCH 078/141] [ISSUE 3585] [Part J] Use MappedByteBuffer instead of FileChannel to write consume queue and slave commitlog. (#3657) This commit improve reput performance and speed up consume qps greatly. In our test, produce about 200,000 tps. 72 queue consume qps: from about 70,000 to 200,000 600 queue consume qps: from about 70,000 to 110,000 --- .../java/org/apache/rocketmq/store/MappedFile.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFile.java b/store/src/main/java/org/apache/rocketmq/store/MappedFile.java index 297271d38bb..774896bb90c 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFile.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFile.java @@ -236,8 +236,9 @@ public boolean appendMessage(final byte[] data) { if ((currentPos + data.length) <= this.fileSize) { try { - this.fileChannel.position(currentPos); - this.fileChannel.write(ByteBuffer.wrap(data)); + ByteBuffer buf = this.mappedByteBuffer.slice(); + buf.position(currentPos); + buf.put(data); } catch (Throwable e) { log.error("Error occurred when append message to mappedFile.", e); } @@ -259,8 +260,9 @@ public boolean appendMessage(final byte[] data, final int offset, final int leng if ((currentPos + length) <= this.fileSize) { try { - this.fileChannel.position(currentPos); - this.fileChannel.write(ByteBuffer.wrap(data, offset, length)); + ByteBuffer buf = this.mappedByteBuffer.slice(); + buf.position(currentPos); + buf.put(data, offset, length); } catch (Throwable e) { log.error("Error occurred when append message to mappedFile.", e); } From 8540da4449fdffd65a85e305ad8d0428703216e0 Mon Sep 17 00:00:00 2001 From: xijiu <422766572@qq.com> Date: Tue, 28 Dec 2021 09:02:16 +0800 Subject: [PATCH 079/141] [ISSUE #3674] Improve the test cases of the tools module (#3672) * move the filtering logic to client * fix error test * fix all junit test except QueryMsgTraceByIdSubCommand * fix some junit test exception * Run through all test cases * Code optimization * extract the startNameServer, eliminate duplicate code --- tools/pom.xml | 4 - .../broker/BrokerStatusSubCommandTest.java | 68 +++----- .../broker/CleanExpiredCQSubCommandTest.java | 56 ++----- .../broker/CleanUnusedTopicCommandTest.java | 56 ++----- .../broker/GetBrokerConfigCommandTest.java | 73 +++------ .../UpdateBrokerConfigSubCommandTest.java | 52 ++---- .../ConsumerConnectionSubCommandTest.java | 87 ++++------ .../ProducerConnectionSubCommandTest.java | 80 ++++----- .../ConsumerProgressSubCommandTest.java | 103 +++++------- .../ConsumerStatusSubCommandTest.java | 113 ++++--------- .../GetConsumerConfigSubCommandTest.java | 104 +++++++----- .../message/ConsumeMessageCommandTest.java | 43 ++++- .../QueryMsgByUniqueKeySubCommandTest.java | 6 +- .../QueryMsgTraceByIdSubCommandTest.java | 126 ++++++++++----- .../namesrv/AddWritePermSubCommandTest.java | 38 +++++ .../namesrv/GetNamesrvConfigCommandTest.java | 77 +++------ .../namesrv/UpdateKvConfigCommandTest.java | 56 +++---- .../namesrv/WipeWritePermSubCommandTest.java | 82 ++++------ .../offset/GetConsumerStatusCommandTest.java | 70 +++----- .../offset/ResetOffsetByTimeCommandTest.java | 88 +++------- .../command/server/NameServerMocker.java | 67 ++++++++ .../command/server/ServerResponseMocker.java | 153 ++++++++++++++++++ 22 files changed, 771 insertions(+), 831 deletions(-) create mode 100644 tools/src/test/java/org/apache/rocketmq/tools/command/server/NameServerMocker.java create mode 100644 tools/src/test/java/org/apache/rocketmq/tools/command/server/ServerResponseMocker.java diff --git a/tools/pom.xml b/tools/pom.xml index 3eda8523c0c..2bd2a8b5ac4 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,10 +27,6 @@ rocketmq-tools rocketmq-tools ${project.version} - - true - - ${project.groupId} diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommandTest.java index c850d71d881..ad2a76b3a9a 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommandTest.java @@ -16,75 +16,43 @@ */ package org.apache.rocketmq.tools.command.broker; -import java.lang.reflect.Field; -import java.util.HashMap; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.common.protocol.body.KVTable; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; +import org.apache.rocketmq.common.protocol.body.BrokerStatsData; +import org.apache.rocketmq.common.protocol.body.BrokerStatsItem; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +public class BrokerStatusSubCommandTest extends ServerResponseMocker { -public class BrokerStatusSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; + private static final int PORT = 45678; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); - - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); - - KVTable kvTable = new KVTable(); - kvTable.setTable(new HashMap()); - when(mQClientAPIImpl.getBrokerRuntimeInfo(anyString(), anyLong())).thenReturn(kvTable); + @Override + protected int getPort() { + return PORT; } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @Override + protected byte[] getBody() { + BrokerStatsData brokerStatsData = new BrokerStatsData(); + BrokerStatsItem item = new BrokerStatsItem(); + brokerStatsData.setStatsDay(item); + return brokerStatsData.encode(); } - @Ignore @Test public void testExecute() throws SubCommandException { BrokerStatusSubCommand cmd = new BrokerStatusSubCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); - String[] subargs = new String[] {"-b 127.0.0.1:10911", "-c default-cluster"}; + String[] subargs = new String[] {"-b 127.0.0.1:" + PORT, "-c default-cluster"}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); + cmd.execute(commandLine, options, null); } + + } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanExpiredCQSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanExpiredCQSubCommandTest.java index 241ae8829ab..a5c070bf939 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanExpiredCQSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanExpiredCQSubCommandTest.java @@ -16,69 +16,33 @@ */ package org.apache.rocketmq.tools.command.broker; -import java.lang.reflect.Field; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +public class CleanExpiredCQSubCommandTest extends ServerResponseMocker { -public class CleanExpiredCQSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; + private static final int PORT = 45678; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); - - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); - - when(mQClientAPIImpl.cleanExpiredConsumeQueue(anyString(), anyLong())).thenReturn(true); + @Override + protected int getPort() { + return PORT; } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @Override + protected byte[] getBody() { + return null; } - @Ignore @Test public void testExecute() throws SubCommandException { CleanExpiredCQSubCommand cmd = new CleanExpiredCQSubCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); - String[] subargs = new String[] {"-b 127.0.0.1:10911", "-c default-cluster"}; + String[] subargs = new String[] {"-b 127.0.0.1:" + PORT, "-c default-cluster"}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommandTest.java index 759f783e495..b893a765cf2 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/CleanUnusedTopicCommandTest.java @@ -16,69 +16,33 @@ */ package org.apache.rocketmq.tools.command.broker; -import java.lang.reflect.Field; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +public class CleanUnusedTopicCommandTest extends ServerResponseMocker { -public class CleanUnusedTopicCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; + private static final int PORT = 45678; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); - - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); - - when(mQClientAPIImpl.cleanUnusedTopicByAddr(anyString(), anyLong())).thenReturn(true); + @Override + protected int getPort() { + return PORT; } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @Override + protected byte[] getBody() { + return null; } - @Ignore @Test public void testExecute() throws SubCommandException { CleanUnusedTopicCommand cmd = new CleanUnusedTopicCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); - String[] subargs = new String[] {"-b 127.0.0.1:10911", "-c default-cluster"}; + String[] subargs = new String[] {"-b 127.0.0.1:" + PORT, "-c default-cluster"}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java index 8bb40793790..7673a7827bf 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java @@ -16,75 +16,48 @@ */ package org.apache.rocketmq.tools.command.broker; -import java.io.UnsupportedEncodingException; -import java.lang.reflect.Field; -import java.util.Properties; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import java.io.UnsupportedEncodingException; +import java.util.Properties; -public class GetBrokerConfigCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException, UnsupportedEncodingException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); +public class GetBrokerConfigCommandTest extends ServerResponseMocker { - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private static final int PORT = 45678; - Properties properties = new Properties(); - properties.setProperty("maxMessageSize", "5000000"); - properties.setProperty("flushDelayOffsetInterval", "15000"); - properties.setProperty("serverSocketRcvBufSize", "655350"); - when(mQClientAPIImpl.getBrokerConfig(anyString(), anyLong())).thenReturn(properties); + @Override + protected int getPort() { + return PORT; } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @Override + protected byte[] getBody() { + StringBuilder sb = new StringBuilder(); + Properties properties = new Properties(); + properties.setProperty("stat", "123"); + properties.setProperty("ip", "192.168.1.1"); + properties.setProperty("broker_name", "broker_101"); + sb.append(MixAll.properties2String(properties)); + try { + return sb.toString().getBytes(MixAll.DEFAULT_CHARSET); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } } - @Ignore @Test public void testExecute() throws SubCommandException { GetBrokerConfigCommand cmd = new GetBrokerConfigCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); - String[] subargs = new String[] {"-b 127.0.0.1:10911", "-c default-cluster"}; + String[] subargs = new String[] {"-b 127.0.0.1:" + PORT, "-c default-cluster"}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommandTest.java index c74107edf92..120ab9e39cf 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommandTest.java @@ -16,65 +16,33 @@ */ package org.apache.rocketmq.tools.command.broker; -import java.io.UnsupportedEncodingException; -import java.lang.reflect.Field; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; import org.junit.Test; -import static org.mockito.Mockito.mock; +public class UpdateBrokerConfigSubCommandTest extends ServerResponseMocker { -public class UpdateBrokerConfigSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; + private static final int PORT = 45678; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException, UnsupportedEncodingException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); - - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + @Override + protected int getPort() { + return PORT; } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @Override + protected byte[] getBody() { + return null; } - @Ignore @Test public void testExecute() throws SubCommandException { UpdateBrokerConfigSubCommand cmd = new UpdateBrokerConfigSubCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); - String[] subargs = new String[] {"-b 127.0.0.1:10911", "-c default-cluster", "-k topicname", "-v unit_test"}; + String[] subargs = new String[] {"-b 127.0.0.1:" + PORT, "-c default-cluster", "-k topicname", "-v unit_test"}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java index 584943ce442..6ad311ad291 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java @@ -16,80 +16,45 @@ */ package org.apache.rocketmq.tools.command.connection; -import java.lang.reflect.Field; -import java.util.HashSet; -import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.protocol.body.Connection; import org.apache.rocketmq.common.protocol.body.ConsumerConnection; -import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType; -import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; -import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; +import java.util.HashSet; + import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class ConsumerConnectionSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private static final int BROKER_PORT = 45676; - ConsumerConnection consumerConnection = new ConsumerConnection(); - consumerConnection.setConsumeType(ConsumeType.CONSUME_PASSIVELY); - consumerConnection.setMessageModel(MessageModel.CLUSTERING); - HashSet connections = new HashSet<>(); - connections.add(new Connection()); - consumerConnection.setConnectionSet(connections); - consumerConnection.setSubscriptionTable(new ConcurrentHashMap()); - consumerConnection.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); - when(mQClientAPIImpl.getConsumerConnectionList(anyString(), anyString(), anyLong())).thenReturn(consumerConnection); + private ServerResponseMocker brokerMocker; + + private ServerResponseMocker nameServerMocker; + + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { ConsumerConnectionSubCommand cmd = new ConsumerConnectionSubCommand(); @@ -99,4 +64,16 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + ConsumerConnection consumerConnection = new ConsumerConnection(); + HashSet connectionSet = new HashSet<>(); + Connection connection = mock(Connection.class); + connectionSet.add(connection); + consumerConnection.setConnectionSet(connectionSet); + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, consumerConnection.encode()); + } + + } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java index 060ba9383d5..4cab7ccb7c7 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java @@ -16,74 +16,45 @@ */ package org.apache.rocketmq.tools.command.connection; -import java.lang.reflect.Field; -import java.util.HashSet; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.common.protocol.body.Connection; import org.apache.rocketmq.common.protocol.body.ProducerConnection; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; +import java.util.HashSet; + import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class ProducerConnectionSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private static final int BROKER_PORT = 45676; - ProducerConnection producerConnection = new ProducerConnection(); - Connection connection = new Connection(); - connection.setClientAddr("127.0.0.1:9898"); - connection.setClientId("PID_12345"); - HashSet connectionSet = new HashSet<>(); - connectionSet.add(connection); - producerConnection.setConnectionSet(connectionSet); - when(mQClientAPIImpl.getProducerConnectionList(anyString(), anyString(), anyLong())).thenReturn(producerConnection); + private ServerResponseMocker brokerMocker; + + private ServerResponseMocker nameServerMocker; + + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { ProducerConnectionSubCommand cmd = new ProducerConnectionSubCommand(); @@ -93,4 +64,15 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + ProducerConnection producerConnection = new ProducerConnection(); + HashSet connectionSet = new HashSet<>(); + Connection connection = mock(Connection.class); + connectionSet.add(connection); + producerConnection.setConnectionSet(connectionSet); + + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, producerConnection.encode()); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommandTest.java index 19d903cea6f..b234ef25be0 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommandTest.java @@ -16,92 +16,44 @@ */ package org.apache.rocketmq.tools.command.consumer; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.common.admin.ConsumeStats; import org.apache.rocketmq.common.admin.OffsetWrapper; import org.apache.rocketmq.common.message.MessageQueue; -import org.apache.rocketmq.common.protocol.route.BrokerData; -import org.apache.rocketmq.common.protocol.route.QueueData; -import org.apache.rocketmq.common.protocol.route.TopicRouteData; -import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import java.util.HashMap; public class ConsumerProgressSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingException, MQClientException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private static final int BROKER_PORT = 45676; - TopicRouteData topicRouteData = new TopicRouteData(); - List brokerDatas = new ArrayList<>(); - HashMap brokerAddrs = new HashMap<>(); - brokerAddrs.put(1234l, "127.0.0.1:10911"); - BrokerData brokerData = new BrokerData(); - brokerData.setCluster("default-cluster"); - brokerData.setBrokerName("default-broker"); - brokerData.setBrokerAddrs(brokerAddrs); - brokerDatas.add(brokerData); - topicRouteData.setBrokerDatas(brokerDatas); - topicRouteData.setQueueDatas(new ArrayList()); - topicRouteData.setFilterServerTable(new HashMap>()); - when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(topicRouteData); + private ServerResponseMocker brokerMocker; - ConsumeStats consumeStats = new ConsumeStats(); - consumeStats.setConsumeTps(1234); - MessageQueue messageQueue = new MessageQueue(); - OffsetWrapper offsetWrapper = new OffsetWrapper(); - HashMap stats = new HashMap<>(); - stats.put(messageQueue, offsetWrapper); - consumeStats.setOffsetTable(stats); - when(mQClientAPIImpl.getConsumeStats(anyString(), anyString(), anyString(), anyLong())).thenReturn(consumeStats); + private ServerResponseMocker nameServerMocker; + + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { ConsumerProgressSubCommand cmd = new ConsumerProgressSubCommand(); @@ -111,4 +63,23 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + ConsumeStats consumeStats = new ConsumeStats(); + HashMap offsetTable = new HashMap<>(); + MessageQueue messageQueue = new MessageQueue(); + messageQueue.setBrokerName("mockBrokerName"); + messageQueue.setQueueId(1); + messageQueue.setBrokerName("mockTopicName"); + + OffsetWrapper offsetWrapper = new OffsetWrapper(); + offsetWrapper.setBrokerOffset(1); + offsetWrapper.setConsumerOffset(1); + offsetWrapper.setLastTimestamp(System.currentTimeMillis()); + + offsetTable.put(messageQueue, offsetWrapper); + consumeStats.setOffsetTable(offsetTable); + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, consumeStats.encode()); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java index 7f44af8225c..82baaeb981c 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java @@ -16,112 +16,45 @@ */ package org.apache.rocketmq.tools.command.consumer; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.common.consumer.ConsumeFromWhere; -import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.body.Connection; -import org.apache.rocketmq.common.protocol.body.ConsumeStatus; import org.apache.rocketmq.common.protocol.body.ConsumerConnection; -import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; -import org.apache.rocketmq.common.protocol.body.ProcessQueueInfo; -import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType; -import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; -import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; -import org.apache.rocketmq.common.protocol.route.BrokerData; -import org.apache.rocketmq.common.protocol.route.QueueData; -import org.apache.rocketmq.common.protocol.route.TopicRouteData; -import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; +import java.util.HashSet; + import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class ConsumerStatusSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingException, MQClientException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private static final int BROKER_PORT = 45676; - TopicRouteData topicRouteData = new TopicRouteData(); - List brokerDatas = new ArrayList<>(); - HashMap brokerAddrs = new HashMap<>(); - brokerAddrs.put(1234l, "127.0.0.1:10911"); - BrokerData brokerData = new BrokerData(); - brokerData.setCluster("default-cluster"); - brokerData.setBrokerName("default-broker"); - brokerData.setBrokerAddrs(brokerAddrs); - brokerDatas.add(brokerData); - topicRouteData.setBrokerDatas(brokerDatas); - topicRouteData.setQueueDatas(new ArrayList()); - topicRouteData.setFilterServerTable(new HashMap>()); - when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(topicRouteData); + private ServerResponseMocker brokerMocker; - ConsumerConnection consumerConnection = new ConsumerConnection(); - consumerConnection.setConsumeType(ConsumeType.CONSUME_PASSIVELY); - consumerConnection.setMessageModel(MessageModel.CLUSTERING); - HashSet connections = new HashSet<>(); - connections.add(new Connection()); - consumerConnection.setConnectionSet(connections); - consumerConnection.setSubscriptionTable(new ConcurrentHashMap()); - consumerConnection.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); - when(mQClientAPIImpl.getConsumerConnectionList(anyString(), anyString(), anyLong())).thenReturn(consumerConnection); + private ServerResponseMocker nameServerMocker; - ConsumerRunningInfo consumerRunningInfo = new ConsumerRunningInfo(); - consumerRunningInfo.setJstack("test"); - consumerRunningInfo.setMqTable(new TreeMap()); - consumerRunningInfo.setStatusTable(new TreeMap()); - consumerRunningInfo.setSubscriptionSet(new TreeSet()); - when(mQClientAPIImpl.getConsumerRunningInfo(anyString(), anyString(), anyString(), anyBoolean(), anyLong())).thenReturn(consumerRunningInfo); + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { ConsumerStatusSubCommand cmd = new ConsumerStatusSubCommand(); @@ -131,4 +64,14 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + ConsumerConnection consumerConnection = new ConsumerConnection(); + HashSet connectionSet = new HashSet<>(); + Connection connection = mock(Connection.class); + connectionSet.add(connection); + consumerConnection.setConnectionSet(connectionSet); + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, consumerConnection.encode()); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java index 1ec68ff02ba..c21df3d1ac6 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java @@ -16,63 +16,51 @@ */ package org.apache.rocketmq.tools.command.consumer; -import java.io.UnsupportedEncodingException; -import java.lang.reflect.Field; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.protocol.body.ClusterInfo; +import org.apache.rocketmq.common.protocol.body.Connection; +import org.apache.rocketmq.common.protocol.body.ConsumerConnection; +import org.apache.rocketmq.common.protocol.route.BrokerData; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Set; + import static org.mockito.Mockito.mock; public class GetConsumerConfigSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException, UnsupportedEncodingException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); - - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + + private static final int NAME_SERVER_PORT = 45677; + + private static final int BROKER_PORT = 45676; + + private ServerResponseMocker brokerMocker; + + private ServerResponseMocker nameServerMocker; + + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = startNameServer(); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { - System.setProperty("rocketmq.namesrv.addr", "127.0.0.1:9876"); GetConsumerConfigSubCommand cmd = new GetConsumerConfigSubCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); String[] subargs = new String[] {"-g group_test"}; @@ -80,4 +68,38 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startNameServer() { + System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:" + NAME_SERVER_PORT); + ClusterInfo clusterInfo = new ClusterInfo(); + + HashMap brokerAddressTable = new HashMap<>(); + BrokerData brokerData = new BrokerData(); + brokerData.setBrokerName("mockBrokerName"); + HashMap brokerAddress = new HashMap<>(); + brokerAddress.put(1L, "127.0.0.1:" + BROKER_PORT); + brokerData.setBrokerAddrs(brokerAddress); + brokerData.setCluster("mockCluster"); + brokerAddressTable.put("mockBrokerName", brokerData); + clusterInfo.setBrokerAddrTable(brokerAddressTable); + + HashMap> clusterAddressTable = new HashMap<>(); + Set brokerNames = new HashSet<>(); + brokerNames.add("mockBrokerName"); + clusterAddressTable.put("mockCluster", brokerNames); + clusterInfo.setClusterAddrTable(clusterAddressTable); + + // start name server + return ServerResponseMocker.startServer(NAME_SERVER_PORT, clusterInfo.encode()); + } + + private ServerResponseMocker startOneBroker() { + ConsumerConnection consumerConnection = new ConsumerConnection(); + HashSet connectionSet = new HashSet<>(); + Connection connection = mock(Connection.class); + connectionSet.add(connection); + consumerConnection.setConnectionSet(connectionSet); + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, consumerConnection.encode()); + } } \ No newline at end of file diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommandTest.java index 11543958c04..98621e6a228 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommandTest.java @@ -51,18 +51,24 @@ public class ConsumeMessageCommandTest { private static ConsumeMessageCommand consumeMessageCommand; + private static PullResult PULL_RESULT = mockPullResult(); + + private static PullResult mockPullResult() { + MessageExt msg = new MessageExt(); + msg.setBody(new byte[] {'a'}); + List msgFoundList = new ArrayList<>(); + msgFoundList.add(msg); + return new PullResult(PullStatus.FOUND, 2, 0, 1, msgFoundList); + } + + @BeforeClass public static void init() throws MQClientException, RemotingException, MQBrokerException, InterruptedException, NoSuchFieldException, IllegalAccessException { consumeMessageCommand = new ConsumeMessageCommand(); DefaultMQPullConsumer defaultMQPullConsumer = mock(DefaultMQPullConsumer.class); - MessageExt msg = new MessageExt(); - msg.setBody(new byte[] {'a'}); - List msgFoundList = new ArrayList<>(); - msgFoundList.add(msg); - final PullResult pullResult = new PullResult(PullStatus.FOUND, 2, 0, 1, msgFoundList); - when(defaultMQPullConsumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())).thenReturn(pullResult); + assignPullResult(defaultMQPullConsumer); when(defaultMQPullConsumer.minOffset(any(MessageQueue.class))).thenReturn(Long.valueOf(0)); when(defaultMQPullConsumer.maxOffset(any(MessageQueue.class))).thenReturn(Long.valueOf(1)); @@ -79,6 +85,25 @@ public static void init() throws MQClientException, RemotingException, MQBrokerE public static void terminate() { } + private static void assignPullResult() { + assignPullResult(null); + } + + private static void assignPullResult(DefaultMQPullConsumer defaultMQPullConsumer) { + try { + if (defaultMQPullConsumer == null) { + Field producerField = ConsumeMessageCommand.class.getDeclaredField("defaultMQPullConsumer"); + producerField.setAccessible(true); + defaultMQPullConsumer = (DefaultMQPullConsumer) producerField.get(consumeMessageCommand); + } + when(defaultMQPullConsumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())) + .thenReturn(PULL_RESULT); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + @Test public void testExecuteDefault() throws SubCommandException { PrintStream out = System.out; @@ -86,6 +111,7 @@ public void testExecuteDefault() throws SubCommandException { System.setOut(new PrintStream(bos)); Options options = ServerUtil.buildCommandlineOptions(new Options()); String[] subargs = new String[] {"-t mytopic", "-n localhost:9876"}; + assignPullResult(); CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + consumeMessageCommand.commandName(), subargs, consumeMessageCommand.buildCommandlineOptions(options), new PosixParser()); consumeMessageCommand.execute(commandLine, options, null); @@ -104,6 +130,7 @@ public void testExecuteByCondition() throws SubCommandException { String[] subargs = new String[] {"-t mytopic", "-b localhost", "-i 0", "-n localhost:9876"}; CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + consumeMessageCommand.commandName(), subargs, consumeMessageCommand.buildCommandlineOptions(options), new PosixParser()); + assignPullResult(); consumeMessageCommand.execute(commandLine, options, null); System.setOut(out); String s = new String(bos.toByteArray()); @@ -113,7 +140,7 @@ public void testExecuteByCondition() throws SubCommandException { @Test public void testExecuteDefaultWhenPullMessageByQueueGotException() throws SubCommandException, InterruptedException, RemotingException, MQClientException, MQBrokerException, NoSuchFieldException, IllegalAccessException { DefaultMQPullConsumer defaultMQPullConsumer = mock(DefaultMQPullConsumer.class); - when(defaultMQPullConsumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())).thenThrow(Exception.class); + when(defaultMQPullConsumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())).thenThrow(MQClientException.class); Field producerField = ConsumeMessageCommand.class.getDeclaredField("defaultMQPullConsumer"); producerField.setAccessible(true); producerField.set(consumeMessageCommand, defaultMQPullConsumer); @@ -135,7 +162,7 @@ public void testExecuteDefaultWhenPullMessageByQueueGotException() throws SubCom @Test public void testExecuteByConditionWhenPullMessageByQueueGotException() throws IllegalAccessException, InterruptedException, RemotingException, MQClientException, MQBrokerException, NoSuchFieldException, SubCommandException { DefaultMQPullConsumer defaultMQPullConsumer = mock(DefaultMQPullConsumer.class); - when(defaultMQPullConsumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())).thenThrow(Exception.class); + when(defaultMQPullConsumer.pull(any(MessageQueue.class), anyString(), anyLong(), anyInt())).thenThrow(MQClientException.class); Field producerField = ConsumeMessageCommand.class.getDeclaredField("defaultMQPullConsumer"); producerField.setAccessible(true); producerField.set(consumeMessageCommand, defaultMQPullConsumer); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java index 504b4656780..e21a66fed24 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommandTest.java @@ -20,6 +20,7 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; import org.apache.rocketmq.client.ClientConfig; +import org.apache.rocketmq.client.QueryResult; import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.impl.MQAdminImpl; @@ -44,6 +45,7 @@ import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; +import org.assertj.core.util.Lists; import org.junit.Before; import org.junit.Test; @@ -114,6 +116,9 @@ public void before() throws NoSuchFieldException, IllegalAccessException, Interr when(mQAdminImpl.queryMessageByUniqKey(anyString(), anyString())).thenReturn(retMsgExt); + QueryResult queryResult = new QueryResult(0, Lists.newArrayList(retMsgExt)); + when(defaultMQAdminExtImpl.queryMessageByUniqKey(anyString(), anyString(), anyInt(), anyLong(), anyLong())).thenReturn(queryResult); + TopicRouteData topicRouteData = new TopicRouteData(); List brokerDataList = new ArrayList(); BrokerData brokerData = new BrokerData(); @@ -231,7 +236,6 @@ public void testExecute() throws SubCommandException { System.setProperty("rocketmq.namesrv.addr", "127.0.0.1:9876"); - QueryMsgByUniqueKeySubCommand cmd = new QueryMsgByUniqueKeySubCommand(); String[] args = new String[]{"-t myTopicTest", "-i 0A3A54F7BF7D18B4AAC28A3FA2CF0000"}; Options options = ServerUtil.buildCommandlineOptions(new Options()); CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin ", args, cmd.buildCommandlineOptions(options), new PosixParser()); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommandTest.java index f61c71d1092..f22acdead7b 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommandTest.java @@ -19,66 +19,104 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.QueueData; +import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import java.io.UnsupportedEncodingException; -import java.lang.reflect.Field; - -import static org.mockito.Mockito.mock; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; public class QueryMsgTraceByIdSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException, UnsupportedEncodingException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); - - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + + private static final int NAME_SERVER_PORT = 45677; + + private static final int BROKER_PORT = 45676; + + private ServerResponseMocker brokerMocker; + + private ServerResponseMocker nameServerMocker; + + private static final String MSG_ID = "AC1FF54E81C418B4AAC24F92E1E00000"; + + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = startNameServer(); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { - System.setProperty("rocketmq.namesrv.addr", "127.0.0.1:9876"); QueryMsgTraceByIdSubCommand cmd = new QueryMsgTraceByIdSubCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); - String[] subargs = new String[] {"-i AC1FF54E81C418B4AAC24F92E1E00000"}; + String[] subargs = new String[] {"-i " + MSG_ID}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startNameServer() { + int nameServerPort = NAME_SERVER_PORT; + System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:" + nameServerPort); + TopicRouteData topicRouteData = new TopicRouteData(); + List dataList = new ArrayList<>(); + HashMap brokerAddress = new HashMap<>(); + brokerAddress.put(1L, "127.0.0.1:" + BROKER_PORT); + BrokerData brokerData = new BrokerData("mockCluster", "mockBrokerName", brokerAddress); + brokerData.setBrokerName("mockBrokerName"); + dataList.add(brokerData); + topicRouteData.setBrokerDatas(dataList); + + List queueDatas = new ArrayList<>(); + QueueData queueData = new QueueData(); + queueData.setBrokerName("mockBrokerName"); + queueData.setPerm(1); + queueData.setReadQueueNums(1); + queueData.setTopicSysFlag(1); + queueData.setWriteQueueNums(1); + queueDatas.add(queueData); + topicRouteData.setQueueDatas(queueDatas); + + return ServerResponseMocker.startServer(nameServerPort, topicRouteData.encode()); + } + + private ServerResponseMocker startOneBroker() { + try { + MessageExt messageExt = new MessageExt(); + messageExt.setTopic(TopicValidator.RMQ_SYS_TRACE_TOPIC); + messageExt.setBody(new byte[100]); + // topic RMQ_SYS_TRACE_TOPIC which built-in rocketMQ, set msg id as msg key + messageExt.setKeys(MSG_ID); + messageExt.setBornHost(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0)); + messageExt.setStoreHost(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0)); + byte[] body = MessageDecoder.encode(messageExt, false); + + HashMap extMap = new HashMap<>(); + extMap.put("indexLastUpdateTimestamp", String.valueOf(System.currentTimeMillis())); + extMap.put("indexLastUpdatePhyoffset", String.valueOf(System.currentTimeMillis())); + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, body, extMap); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } } \ No newline at end of file diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommandTest.java index 901b8bbd3be..c65a6c3d9ca 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommandTest.java @@ -21,10 +21,36 @@ import org.apache.commons.cli.PosixParser; import org.apache.rocketmq.srvutil.ServerUtil; import org.apache.rocketmq.tools.command.SubCommandException; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; +import java.util.HashMap; + public class AddWritePermSubCommandTest { + private static final int NAME_SERVER_PORT = 45677; + + private static final int BROKER_PORT = 45676; + + private ServerResponseMocker brokerMocker; + + private ServerResponseMocker nameServerMocker; + + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = startNameServer(); + } + + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); + } + @Test public void testExecute() throws SubCommandException { AddWritePermSubCommand cmd = new AddWritePermSubCommand(); @@ -34,4 +60,16 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startNameServer() { + HashMap extMap = new HashMap<>(); + extMap.put("addTopicCount", "1"); + // start name server + return NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT, extMap); + } + + private ServerResponseMocker startOneBroker() { + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, null); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommandTest.java index dde80eb3e67..e94aba9221f 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/GetNamesrvConfigCommandTest.java @@ -16,78 +16,51 @@ */ package org.apache.rocketmq.tools.command.namesrv; -import java.io.UnsupportedEncodingException; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; -import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import org.mockito.ArgumentMatchers; - -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class GetNamesrvConfigCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException, UnsupportedEncodingException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; + + private static final int BROKER_PORT = 45676; + + private ServerResponseMocker brokerMocker; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private ServerResponseMocker nameServerMocker; - Map propertiesMap = new HashMap<>(); - List nameServers = new ArrayList<>(); - when(mQClientAPIImpl.getNameServerConfig(ArgumentMatchers.anyList(), anyLong())).thenReturn(propertiesMap); + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - // @Ignore @Test - public void testExecute() throws SubCommandException { + public void testExecute() throws Exception { GetNamesrvConfigCommand cmd = new GetNamesrvConfigCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); String[] subargs = new String[] {}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); + cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, null); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/UpdateKvConfigCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/UpdateKvConfigCommandTest.java index c4edcafe6f2..c7480d1f9cf 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/UpdateKvConfigCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/UpdateKvConfigCommandTest.java @@ -19,52 +19,35 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import java.lang.reflect.Field; +public class UpdateKvConfigCommandTest { + private static final int NAME_SERVER_PORT = 45677; -import static org.mockito.Mockito.mock; + private static final int BROKER_PORT = 45676; -public class UpdateKvConfigCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; + private ServerResponseMocker brokerMocker; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private ServerResponseMocker nameServerMocker; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { UpdateKvConfigCommand cmd = new UpdateKvConfigCommand(); @@ -74,4 +57,9 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName() + cmd.commandDesc(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, null); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommandTest.java index 9befdf8948a..b065c31d181 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommandTest.java @@ -16,71 +16,41 @@ */ package org.apache.rocketmq.tools.command.namesrv; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.List; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.remoting.exception.RemotingCommandException; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import java.util.HashMap; public class WipeWritePermSubCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingTimeoutException, MQClientException, RemotingSendRequestException, RemotingConnectException, MQBrokerException, RemotingCommandException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private static final int BROKER_PORT = 45676; - List result = new ArrayList<>(); - result.add("default-name-one"); - result.add("default-name-two"); - when(mqClientInstance.getMQClientAPIImpl().getNameServerAddressList()).thenReturn(result); - when(mQClientAPIImpl.wipeWritePermOfBroker(anyString(), anyString(), anyLong())).thenReturn(6); + private ServerResponseMocker brokerMocker; + + private ServerResponseMocker nameServerMocker; + + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = startNameServer(); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { WipeWritePermSubCommand cmd = new WipeWritePermSubCommand(); @@ -90,4 +60,18 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startNameServer() { + HashMap extMap = new HashMap<>(); + extMap.put("wipeTopicCount", "1"); + // start name server + return NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT, extMap); + } + + private ServerResponseMocker startOneBroker() { + // start broker + HashMap extMap = new HashMap<>(); + extMap.put("wipeTopicCount", "1"); + return ServerResponseMocker.startServer(BROKER_PORT, new byte[0], extMap); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/offset/GetConsumerStatusCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/offset/GetConsumerStatusCommandTest.java index a01bf8167ae..aece90feec6 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/offset/GetConsumerStatusCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/offset/GetConsumerStatusCommandTest.java @@ -16,66 +16,40 @@ */ package org.apache.rocketmq.tools.command.offset; -import java.lang.reflect.Field; -import java.util.HashMap; -import java.util.Map; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.common.message.MessageQueue; -import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.common.protocol.body.GetConsumerStatusBody; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - public class GetConsumerStatusCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingException, MQClientException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; + + private static final int BROKER_PORT = 45676; + + private ServerResponseMocker brokerMocker; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private ServerResponseMocker nameServerMocker; - Map> consumerStatus = new HashMap<>(); - when(mQClientAPIImpl.invokeBrokerToGetConsumerStatus(anyString(), anyString(), anyString(), anyString(), anyLong())).thenReturn(consumerStatus); + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { GetConsumerStatusCommand cmd = new GetConsumerStatusCommand(); @@ -85,4 +59,10 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + GetConsumerStatusBody getConsumerStatusBody = new GetConsumerStatusBody(); + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, getConsumerStatusBody.encode()); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommandTest.java index d73a996b3fb..03e8943ebf1 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/offset/ResetOffsetByTimeCommandTest.java @@ -16,86 +16,40 @@ */ package org.apache.rocketmq.tools.command.offset; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.rocketmq.client.ClientConfig; -import org.apache.rocketmq.client.exception.MQBrokerException; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.impl.MQClientAPIImpl; -import org.apache.rocketmq.client.impl.MQClientManager; -import org.apache.rocketmq.client.impl.factory.MQClientInstance; -import org.apache.rocketmq.common.message.MessageQueue; -import org.apache.rocketmq.common.protocol.route.BrokerData; -import org.apache.rocketmq.common.protocol.route.QueueData; -import org.apache.rocketmq.common.protocol.route.TopicRouteData; -import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.common.protocol.body.ResetOffsetBody; import org.apache.rocketmq.srvutil.ServerUtil; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; -import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; import org.apache.rocketmq.tools.command.SubCommandException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; +import org.apache.rocketmq.tools.command.server.NameServerMocker; +import org.apache.rocketmq.tools.command.server.ServerResponseMocker; +import org.junit.After; +import org.junit.Before; import org.junit.Test; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - public class ResetOffsetByTimeCommandTest { - private static DefaultMQAdminExt defaultMQAdminExt; - private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); - private static MQClientAPIImpl mQClientAPIImpl; - @BeforeClass - public static void init() throws NoSuchFieldException, IllegalAccessException, InterruptedException, RemotingException, MQClientException, MQBrokerException { - mQClientAPIImpl = mock(MQClientAPIImpl.class); - defaultMQAdminExt = new DefaultMQAdminExt(); - defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + private static final int NAME_SERVER_PORT = 45677; + + private static final int BROKER_PORT = 45676; - Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); - field.setAccessible(true); - field.set(defaultMQAdminExtImpl, mqClientInstance); - field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); - field.setAccessible(true); - field.set(mqClientInstance, mQClientAPIImpl); - field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); - field.setAccessible(true); - field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + private ServerResponseMocker brokerMocker; - TopicRouteData topicRouteData = new TopicRouteData(); - List brokerDatas = new ArrayList<>(); - HashMap brokerAddrs = new HashMap<>(); - brokerAddrs.put(1234l, "127.0.0.1:10911"); - BrokerData brokerData = new BrokerData(); - brokerData.setCluster("default-cluster"); - brokerData.setBrokerName("default-broker"); - brokerData.setBrokerAddrs(brokerAddrs); - brokerDatas.add(brokerData); - topicRouteData.setBrokerDatas(brokerDatas); - topicRouteData.setQueueDatas(new ArrayList()); - topicRouteData.setFilterServerTable(new HashMap>()); - when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(topicRouteData); + private ServerResponseMocker nameServerMocker; - Map messageQueueLongMap = new HashMap<>(); - when(mQClientAPIImpl.invokeBrokerToResetOffset(anyString(), anyString(), anyString(), anyLong(), anyBoolean(), anyLong())).thenReturn(messageQueueLongMap); + @Before + public void before() { + brokerMocker = startOneBroker(); + nameServerMocker = NameServerMocker.startByDefaultConf(NAME_SERVER_PORT, BROKER_PORT); } - @AfterClass - public static void terminate() { - defaultMQAdminExt.shutdown(); + @After + public void after() { + brokerMocker.shutdown(); + nameServerMocker.shutdown(); } - @Ignore @Test public void testExecute() throws SubCommandException { ResetOffsetByTimeCommand cmd = new ResetOffsetByTimeCommand(); @@ -105,4 +59,10 @@ public void testExecute() throws SubCommandException { ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); cmd.execute(commandLine, options, null); } + + private ServerResponseMocker startOneBroker() { + ResetOffsetBody resetOffsetBody = new ResetOffsetBody(); + // start broker + return ServerResponseMocker.startServer(BROKER_PORT, resetOffsetBody.encode()); + } } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/server/NameServerMocker.java b/tools/src/test/java/org/apache/rocketmq/tools/command/server/NameServerMocker.java new file mode 100644 index 00000000000..32718f1c382 --- /dev/null +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/server/NameServerMocker.java @@ -0,0 +1,67 @@ +/* + * 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.tools.command.server; + +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.TopicRouteData; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +/** + * tools class + */ +public class NameServerMocker { + + /** + * use the specified port to start the nameserver + * + * @param nameServerPort nameServer port + * @param brokerPort broker port + * @return ServerResponseMocker + */ + public static ServerResponseMocker startByDefaultConf(int nameServerPort, int brokerPort) { + return startByDefaultConf(nameServerPort, brokerPort, null); + } + + /** + * use the specified port to start the nameserver + * + * @param nameServerPort nameServer port + * @param brokerPort broker port + * @param extMap extend config + * @return ServerResponseMocker + */ + public static ServerResponseMocker startByDefaultConf(int nameServerPort, int brokerPort, + HashMap extMap) { + + System.setProperty(MixAll.NAMESRV_ADDR_PROPERTY, "127.0.0.1:" + nameServerPort); + TopicRouteData topicRouteData = new TopicRouteData(); + List dataList = new ArrayList<>(); + HashMap brokerAddress = new HashMap<>(); + brokerAddress.put(1L, "127.0.0.1:" + brokerPort); + BrokerData brokerData = new BrokerData("mockCluster", "mockBrokerName", brokerAddress); + brokerData.setBrokerName("mockBrokerName"); + dataList.add(brokerData); + topicRouteData.setBrokerDatas(dataList); + // start name server + return ServerResponseMocker.startServer(nameServerPort, topicRouteData.encode(), extMap); + } + +} diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/server/ServerResponseMocker.java b/tools/src/test/java/org/apache/rocketmq/tools/command/server/ServerResponseMocker.java new file mode 100644 index 00000000000..20ddf62ed4b --- /dev/null +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/server/ServerResponseMocker.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.server; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.ChannelDuplexHandler; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.handler.timeout.IdleStateHandler; +import io.netty.util.concurrent.Future; +import org.apache.rocketmq.remoting.netty.NettyDecoder; +import org.apache.rocketmq.remoting.netty.NettyEncoder; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode; +import org.junit.After; +import org.junit.Before; + +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.concurrent.ExecutionException; + +/** + * mock server response for command + */ +public abstract class ServerResponseMocker { + + private final NioEventLoopGroup eventLoopGroup = new NioEventLoopGroup(); + + @Before + public void before() { + start(); + } + + @After + public void shutdown() { + if (eventLoopGroup.isShutdown()) { + return; + } + Future future = eventLoopGroup.shutdownGracefully(); + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + e.printStackTrace(); + } + } + + protected abstract int getPort(); + + protected abstract byte[] getBody(); + + public void start() { + start(null); + } + + public void start(HashMap extMap) { + ServerBootstrap serverBootstrap = new ServerBootstrap(); + serverBootstrap.group(eventLoopGroup) + .channel(NioServerSocketChannel.class) + .option(ChannelOption.SO_BACKLOG, 1024) + .option(ChannelOption.SO_REUSEADDR, true) + .option(ChannelOption.SO_KEEPALIVE, false) + .childOption(ChannelOption.TCP_NODELAY, true) + .childOption(ChannelOption.SO_SNDBUF, 65535) + .childOption(ChannelOption.SO_RCVBUF, 65535) + .localAddress(new InetSocketAddress(getPort())) + .childHandler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) throws Exception { + ch.pipeline() + .addLast(eventLoopGroup, + new NettyEncoder(), + new NettyDecoder(), + new IdleStateHandler(0, 0, 120), + new ChannelDuplexHandler(), + new NettyServerHandler(extMap) + ); + } + }); + try { + ChannelFuture sync = serverBootstrap.bind().sync(); + InetSocketAddress addr = (InetSocketAddress) sync.channel().localAddress(); + } catch (InterruptedException e1) { + throw new RuntimeException("this.serverBootstrap.bind().sync() InterruptedException", e1); + } + } + + @ChannelHandler.Sharable + private class NettyServerHandler extends SimpleChannelInboundHandler { + private HashMap extMap; + + public NettyServerHandler(HashMap extMap) { + this.extMap = extMap; + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, RemotingCommand msg) throws Exception { + String remark = "mock data"; + final RemotingCommand response = + RemotingCommand.createResponseCommand(RemotingSysResponseCode.SUCCESS, remark); + response.setOpaque(msg.getOpaque()); + response.setBody(getBody()); + + if (extMap != null && extMap.size() > 0) { + response.setExtFields(extMap); + } + ctx.writeAndFlush(response); + } + } + + public static ServerResponseMocker startServer(int port, byte[] body) { + return startServer(port, body, null); + } + + + public static ServerResponseMocker startServer(int port, byte[] body, HashMap extMap) { + ServerResponseMocker mocker = new ServerResponseMocker() { + @Override + protected int getPort() { + return port; + } + + @Override + protected byte[] getBody() { + return body; + } + }; + mocker.start(extMap); + // add jvm hook, close connection when jvm down + Runtime.getRuntime().addShutdownHook(new Thread(mocker::shutdown)); + return mocker; + } +} From 28d78498544c07524fe70454337862c7c43a781a Mon Sep 17 00:00:00 2001 From: zhaohai <33314633+zhaohai1299002788@users.noreply.github.com> Date: Tue, 28 Dec 2021 10:40:16 +0800 Subject: [PATCH 080/141] [ISSUE #3539] Add parameter verification --- .../rebalance/AllocateMessageQueueByMachineRoom.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueByMachineRoom.java b/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueByMachineRoom.java index 31f4e3522ec..42a0be1dd04 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueByMachineRoom.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueByMachineRoom.java @@ -19,6 +19,8 @@ import java.util.ArrayList; import java.util.List; import java.util.Set; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.consumer.AllocateMessageQueueStrategy; import org.apache.rocketmq.common.message.MessageQueue; @@ -31,6 +33,15 @@ public class AllocateMessageQueueByMachineRoom implements AllocateMessageQueueSt @Override public List allocate(String consumerGroup, String currentCID, List mqAll, List cidAll) { + if (StringUtils.isBlank(currentCID)) { + throw new IllegalArgumentException("currentCID is empty"); + } + if (CollectionUtils.isEmpty(mqAll)) { + throw new IllegalArgumentException("mqAll is null or mqAll empty"); + } + if (CollectionUtils.isEmpty(cidAll)) { + throw new IllegalArgumentException("cidAll is null or cidAll empty"); + } List result = new ArrayList(); int currentIndex = cidAll.indexOf(currentCID); if (currentIndex < 0) { From 9f9b1091722189eedcf07a4dfdaa4e2f061385d3 Mon Sep 17 00:00:00 2001 From: von gosling Date: Thu, 30 Dec 2021 09:27:55 +0800 Subject: [PATCH 081/141] fix(test): rename error logger name --- .../java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java b/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java index b97b0637b04..1550cf9497c 100644 --- a/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java +++ b/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java @@ -31,7 +31,7 @@ import org.junit.Test; public class NormalMsgDelayIT extends DelayConf { - private static Logger logger = Logger.getLogger(NormalMsgStaticBalanceIT.class); + private static Logger logger = Logger.getLogger(NormalMsgDelayIT.class); protected int msgSize = 100; private RMQNormalProducer producer = null; private RMQNormalConsumer consumer = null; From 1be3cc363fea0404a3b9c25ec74161d79e0ac42a Mon Sep 17 00:00:00 2001 From: duhenglucky Date: Fri, 31 Dec 2021 10:47:14 +0800 Subject: [PATCH 082/141] chore(client) change client jdk version to 1.6 --- client/pom.xml | 5 +++ .../rocketmq/client/impl/MQClientAPIImpl.java | 27 ++++++++------ .../consumer/DefaultLitePullConsumerImpl.java | 2 +- .../client/impl/factory/MQClientInstance.java | 8 +++-- .../impl/producer/DefaultMQProducerImpl.java | 33 +++++++++++++---- .../client/producer/RequestFutureHolder.java | 2 +- .../ConsumeMessageOpenTracingHookImpl.java | 2 +- .../consumer/DefaultLitePullConsumerTest.java | 19 +++++----- .../consumer/DefaultMQPushConsumerTest.java | 22 +++++++----- ...ConsumeMessageConcurrentlyServiceTest.java | 2 +- .../producer/DefaultMQProducerTest.java | 2 +- .../DefaultMQConsumerWithOpenTracingTest.java | 30 ++++++++++------ .../trace/DefaultMQConsumerWithTraceTest.java | 7 ++-- .../TransactionMQProducerWithTraceTest.java | 36 +++++++++++-------- 14 files changed, 129 insertions(+), 68 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 07ba7d1e483..4f6f1959346 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -27,6 +27,11 @@ rocketmq-client rocketmq-client ${project.version} + + 1.6 + 1.6 + + ${project.groupId} diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index 89c998b7f4e..8c9af52e36c 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -397,7 +397,8 @@ public ClusterAclVersionInfo getBrokerClusterAclInfo(final String addr, } - public AclConfig getBrokerClusterConfig(final String addr, final long timeoutMillis) throws RemotingCommandException, InterruptedException, RemotingTimeoutException, + public AclConfig getBrokerClusterConfig(final String addr, + final long timeoutMillis) throws RemotingCommandException, InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CLUSTER_ACL_CONFIG, null); @@ -407,7 +408,7 @@ public AclConfig getBrokerClusterConfig(final String addr, final long timeoutMil case ResponseCode.SUCCESS: { if (response.getBody() != null) { GetBrokerClusterAclConfigResponseBody body = - GetBrokerClusterAclConfigResponseBody.decode(response.getBody(), GetBrokerClusterAclConfigResponseBody.class); + GetBrokerClusterAclConfigResponseBody.decode(response.getBody(), GetBrokerClusterAclConfigResponseBody.class); AclConfig aclConfig = new AclConfig(); aclConfig.setGlobalWhiteAddrs(body.getGlobalWhiteAddrs()); aclConfig.setPlainAccessConfigs(body.getPlainAccessConfigs()); @@ -505,7 +506,7 @@ private SendResult sendMessageSync( ) throws RemotingException, MQBrokerException, InterruptedException { RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; - return this.processSendResponse(brokerName, msg, response,addr); + return this.processSendResponse(brokerName, msg, response, addr); } private void sendMessageAsync( @@ -612,7 +613,10 @@ private void onExceptionImpl(final String brokerName, request.setOpaque(RemotingCommand.createNewRequestId()); sendMessageAsync(addr, retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, timesTotal, curTimes, context, producer); - } catch (InterruptedException | RemotingTooMuchRequestException e1) { + } catch (InterruptedException e1) { + onExceptionImpl(retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, timesTotal, curTimes, e1, + context, false, producer); + } catch (RemotingTooMuchRequestException e1) { onExceptionImpl(retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, timesTotal, curTimes, e1, context, false, producer); } catch (RemotingException e1) { @@ -664,7 +668,7 @@ private SendResult processSendResponse( } SendMessageResponseHeader responseHeader = - (SendMessageResponseHeader) response.decodeCommandCustomHeader(SendMessageResponseHeader.class); + (SendMessageResponseHeader) response.decodeCommandCustomHeader(SendMessageResponseHeader.class); //If namespace not null , reset Topic without namespace. String topic = msg.getTopic(); @@ -683,8 +687,8 @@ private SendResult processSendResponse( uniqMsgId = sb.toString(); } SendResult sendResult = new SendResult(sendStatus, - uniqMsgId, - responseHeader.getMsgId(), messageQueue, responseHeader.getQueueOffset()); + uniqMsgId, + responseHeader.getMsgId(), messageQueue, responseHeader.getQueueOffset()); sendResult.setTransactionId(responseHeader.getTransactionId()); String regionId = response.getExtFields().get(MessageConst.PROPERTY_MSG_REGION); String traceOn = response.getExtFields().get(MessageConst.PROPERTY_TRACE_SWITCH); @@ -1425,8 +1429,8 @@ public int wipeWritePermOfBroker(final String namesrvAddr, String brokerName, } public int addWritePermOfBroker(final String nameSrvAddr, String brokerName, final long timeoutMillis) - throws RemotingCommandException, - RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException { + throws RemotingCommandException, + RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException { AddWritePermOfBrokerRequestHeader requestHeader = new AddWritePermOfBrokerRequestHeader(); requestHeader.setBrokerName(brokerName); @@ -1437,7 +1441,7 @@ public int addWritePermOfBroker(final String nameSrvAddr, String brokerName, fin switch (response.getCode()) { case ResponseCode.SUCCESS: { AddWritePermOfBrokerResponseHeader responseHeader = - (AddWritePermOfBrokerResponseHeader) response.decodeCommandCustomHeader(AddWritePermOfBrokerResponseHeader.class); + (AddWritePermOfBrokerResponseHeader) response.decodeCommandCustomHeader(AddWritePermOfBrokerResponseHeader.class); return responseHeader.getAddTopicCount(); } default: @@ -1485,7 +1489,8 @@ public void deleteTopicInNameServer(final String addr, final String topic, final throw new MQClientException(response.getCode(), response.getRemark()); } - public void deleteSubscriptionGroup(final String addr, final String groupName, final boolean removeOffset, final long timeoutMillis) + public void deleteSubscriptionGroup(final String addr, final String groupName, final boolean removeOffset, + final long timeoutMillis) throws RemotingException, InterruptedException, MQClientException { DeleteSubscriptionGroupRequestHeader requestHeader = new DeleteSubscriptionGroupRequestHeader(); requestHeader.setGroupName(groupName); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java index fecf07645ed..c0d6e97e921 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java @@ -147,7 +147,7 @@ private enum SubscriptionType { private final MessageQueueLock messageQueueLock = new MessageQueueLock(); - private final ArrayList consumeMessageHookList = new ArrayList<>(); + private final ArrayList consumeMessageHookList = new ArrayList(); public DefaultLitePullConsumerImpl(final DefaultLitePullConsumer defaultLitePullConsumer, final RPCHook rpcHook) { this.defaultLitePullConsumer = defaultLitePullConsumer; diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java index 6d35ed63028..eea264b9eff 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java @@ -897,8 +897,12 @@ private void unregisterClient(final String producerGroup, final String consumerG try { this.mQClientAPIImpl.unregisterClient(addr, this.clientId, producerGroup, consumerGroup, clientConfig.getMqClientApiTimeout()); log.info("unregister client[Producer: {} Consumer: {}] from broker[{} {} {}] success", producerGroup, consumerGroup, brokerName, entry1.getKey(), addr); - } catch (RemotingException | InterruptedException | MQBrokerException e) { - log.error("unregister client exception from broker: " + addr, e); + } catch (RemotingException e) { + log.warn("unregister client RemotingException from broker: {}, {}", addr, e.getMessage()); + } catch (InterruptedException e) { + log.warn("unregister client InterruptedException from broker: {}, {}", addr, e.getMessage()); + } catch (MQBrokerException e) { + log.warn("unregister client MQBrokerException from broker: {}, {}", addr, e.getMessage()); } } } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java index cedbbdb6ca3..f980d9d5d66 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java @@ -490,7 +490,7 @@ public void send(Message msg, * * @param msg * @param sendCallback - * @param timeout the sendCallback will be invoked at most time + * @param timeout the sendCallback will be invoked at most time * @throws RejectedExecutionException */ @Deprecated @@ -597,7 +597,14 @@ private SendResult sendDefaultImpl( default: break; } - } catch (RemotingException | MQClientException e) { + } catch (RemotingException e) { + endTimestamp = System.currentTimeMillis(); + this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true); + log.warn(String.format("sendKernelImpl exception, resend at once, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq), e); + log.warn(msg.toString()); + exception = e; + continue; + } catch (MQClientException e) { endTimestamp = System.currentTimeMillis(); this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true); log.warn(String.format("sendKernelImpl exception, resend at once, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq), e); @@ -856,7 +863,19 @@ private SendResult sendKernelImpl(final Message msg, } return sendResult; - } catch (RemotingException | MQBrokerException | InterruptedException e) { + } catch (RemotingException e) { + if (this.hasSendMessageHook()) { + context.setException(e); + this.executeSendMessageHookAfter(context); + } + throw e; + } catch (MQBrokerException e) { + if (this.hasSendMessageHook()) { + context.setException(e); + this.executeSendMessageHookAfter(context); + } + throw e; + } catch (InterruptedException e) { if (this.hasSendMessageHook()) { context.setException(e); this.executeSendMessageHookAfter(context); @@ -969,6 +988,7 @@ public void doExecuteEndTransactionHook(Message msg, String msgId, String broker executeEndTransactionHook(context); } } + /** * DEFAULT ONEWAY ------------------------------------------------------- */ @@ -1021,7 +1041,7 @@ public void send(Message msg, MessageQueue mq, SendCallback sendCallback) * @param msg * @param mq * @param sendCallback - * @param timeout the sendCallback will be invoked at most time + * @param timeout the sendCallback will be invoked at most time * @throws MQClientException * @throws RemotingException * @throws InterruptedException @@ -1151,7 +1171,7 @@ public void send(Message msg, MessageQueueSelector selector, Object arg, SendCal * @param selector * @param arg * @param sendCallback - * @param timeout the sendCallback will be invoked at most time + * @param timeout the sendCallback will be invoked at most time * @throws MQClientException * @throws RemotingException * @throws InterruptedException @@ -1491,7 +1511,8 @@ public void onException(Throwable e) { } } - private Message waitResponse(Message msg, long timeout, RequestResponseFuture requestResponseFuture, long cost) throws InterruptedException, RequestTimeoutException, MQClientException { + private Message waitResponse(Message msg, long timeout, RequestResponseFuture requestResponseFuture, + long cost) throws InterruptedException, RequestTimeoutException, MQClientException { Message responseMessage = requestResponseFuture.waitResponseMessage(timeout - cost); if (responseMessage == null) { if (requestResponseFuture.isSendRequestOk()) { diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java b/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java index 8fe9abcc697..df0706f1f8f 100644 --- a/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java +++ b/client/src/main/java/org/apache/rocketmq/client/producer/RequestFutureHolder.java @@ -39,7 +39,7 @@ public class RequestFutureHolder { private static InternalLogger log = ClientLogger.getLog(); private static final RequestFutureHolder INSTANCE = new RequestFutureHolder(); private ConcurrentHashMap requestFutureTable = new ConcurrentHashMap(); - private final Set producerSet = new HashSet<>(); + private final Set producerSet = new HashSet(); private ScheduledExecutorService scheduledExecutorService = null; public ConcurrentHashMap getRequestFutureTable() { diff --git a/client/src/main/java/org/apache/rocketmq/client/trace/hook/ConsumeMessageOpenTracingHookImpl.java b/client/src/main/java/org/apache/rocketmq/client/trace/hook/ConsumeMessageOpenTracingHookImpl.java index 28fccae06f8..fe97c773949 100644 --- a/client/src/main/java/org/apache/rocketmq/client/trace/hook/ConsumeMessageOpenTracingHookImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/trace/hook/ConsumeMessageOpenTracingHookImpl.java @@ -51,7 +51,7 @@ public void consumeMessageBefore(ConsumeMessageContext context) { if (context == null || context.getMsgList() == null || context.getMsgList().isEmpty()) { return; } - List spanList = new ArrayList<>(); + List spanList = new ArrayList(); for (MessageExt msg : context.getMsgList()) { if (msg == null) { continue; diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java index 04b760eec22..227ea441719 100644 --- a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java @@ -23,6 +23,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; import org.apache.commons.lang3.reflect.FieldUtils; @@ -95,7 +96,9 @@ public class DefaultLitePullConsumerTest { @Before public void init() throws Exception { ConcurrentMap factoryTable = (ConcurrentMap) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true); - factoryTable.forEach((s, instance) -> instance.shutdown()); + for (Map.Entry entry : factoryTable.entrySet()) { + entry.getValue().shutdown(); + } factoryTable.clear(); Field field = MQClientInstance.class.getDeclaredField("rebalanceService"); @@ -481,7 +484,7 @@ public void testCheckConfig_Exception() { } @Test - public void testComputePullFromWhereReturnedNotFound() throws Exception{ + public void testComputePullFromWhereReturnedNotFound() throws Exception { DefaultLitePullConsumer defaultLitePullConsumer = createStartLitePullConsumer(); defaultLitePullConsumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); MessageQueue messageQueue = createMessageQueue(); @@ -491,7 +494,7 @@ public void testComputePullFromWhereReturnedNotFound() throws Exception{ } @Test - public void testComputePullFromWhereReturned() throws Exception{ + public void testComputePullFromWhereReturned() throws Exception { DefaultLitePullConsumer defaultLitePullConsumer = createStartLitePullConsumer(); defaultLitePullConsumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); MessageQueue messageQueue = createMessageQueue(); @@ -500,9 +503,8 @@ public void testComputePullFromWhereReturned() throws Exception{ assertThat(offset).isEqualTo(100); } - @Test - public void testComputePullFromLast() throws Exception{ + public void testComputePullFromLast() throws Exception { DefaultLitePullConsumer defaultLitePullConsumer = createStartLitePullConsumer(); defaultLitePullConsumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); MessageQueue messageQueue = createMessageQueue(); @@ -513,13 +515,13 @@ public void testComputePullFromLast() throws Exception{ } @Test - public void testComputePullByTimeStamp() throws Exception{ + public void testComputePullByTimeStamp() throws Exception { DefaultLitePullConsumer defaultLitePullConsumer = createStartLitePullConsumer(); defaultLitePullConsumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP); defaultLitePullConsumer.setConsumeTimestamp("20191024171201"); MessageQueue messageQueue = createMessageQueue(); when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-1L); - when(mQClientFactory.getMQAdminImpl().searchOffset(any(MessageQueue.class),anyLong())).thenReturn(100L); + when(mQClientFactory.getMQAdminImpl().searchOffset(any(MessageQueue.class), anyLong())).thenReturn(100L); long offset = rebalanceImpl.computePullFromWhere(messageQueue); assertThat(offset).isEqualTo(100); } @@ -660,7 +662,8 @@ private PullResultExt createPullResult(PullMessageRequestHeader requestHeader, P return new PullResultExt(pullStatus, requestHeader.getQueueOffset() + messageExtList.size(), 123, 2048, messageExtList, 0, outputStream.toByteArray()); } - private static void suppressUpdateTopicRouteInfoFromNameServer(DefaultLitePullConsumer litePullConsumer) throws IllegalAccessException { + private static void suppressUpdateTopicRouteInfoFromNameServer( + DefaultLitePullConsumer litePullConsumer) throws IllegalAccessException { DefaultLitePullConsumerImpl defaultLitePullConsumerImpl = (DefaultLitePullConsumerImpl) FieldUtils.readDeclaredField(litePullConsumer, "defaultLitePullConsumerImpl", true); if (litePullConsumer.getMessageModel() == MessageModel.CLUSTERING) { litePullConsumer.changeInstanceNameToPID(); diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java index df87932b2ed..bfc87f13eaa 100644 --- a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPushConsumerTest.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -94,7 +95,9 @@ public class DefaultMQPushConsumerTest { @Before public void init() throws Exception { ConcurrentMap factoryTable = (ConcurrentMap) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true); - factoryTable.forEach((s, instance) -> instance.shutdown()); + for (Map.Entry entry : factoryTable.entrySet()) { + entry.getValue().shutdown(); + } factoryTable.clear(); when(mQClientAPIImpl.pullMessage(anyString(), any(PullMessageRequestHeader.class), @@ -117,7 +120,6 @@ public PullResult answer(InvocationOnMock mock) throws Throwable { } }); - consumerGroup = "FooBarGroup" + System.currentTimeMillis(); pushConsumer = new DefaultMQPushConsumer(consumerGroup); pushConsumer.setNamesrvAddr("127.0.0.1:9876"); @@ -169,7 +171,7 @@ public void testStart_OffsetShouldNotNUllAfterStart() { @Test public void testPullMessage_Success() throws InterruptedException, RemotingException, MQBrokerException { final CountDownLatch countDownLatch = new CountDownLatch(1); - final AtomicReference messageAtomic = new AtomicReference<>(); + final AtomicReference messageAtomic = new AtomicReference(); pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() { @Override public ConsumeConcurrentlyStatus consumeMessage(List msgs, @@ -192,7 +194,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, @Test public void testPullMessage_SuccessWithOrderlyService() throws Exception { final CountDownLatch countDownLatch = new CountDownLatch(1); - final AtomicReference messageAtomic = new AtomicReference<>(); + final AtomicReference messageAtomic = new AtomicReference(); MessageListenerOrderly listenerOrderly = new MessageListenerOrderly() { @Override @@ -329,11 +331,13 @@ public void testPullMessage_ExceptionOccursWhenComputePullFromWhere() throws MQC final CountDownLatch countDownLatch = new CountDownLatch(1); final MessageExt[] messageExts = new MessageExt[1]; pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService( - new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), - (msgs, context) -> { - messageExts[0] = msgs.get(0); - return null; - })); + new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() { + @Override public ConsumeConcurrentlyStatus consumeMessage(List msgs, + ConsumeConcurrentlyContext context) { + messageExts[0] = msgs.get(0); + return null; + } + })); pushConsumer.getDefaultMQPushConsumerImpl().setConsumeOrderly(true); PullMessageService pullMessageService = mQClientFactory.getPullMessageService(); diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java index 6c331db0593..8798c2d9ea0 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java @@ -149,7 +149,7 @@ public PullResult answer(InvocationOnMock mock) throws Throwable { @Test public void testPullMessage_ConsumeSuccess() throws InterruptedException, RemotingException, MQBrokerException, NoSuchFieldException,Exception { final CountDownLatch countDownLatch = new CountDownLatch(1); - final AtomicReference messageAtomic = new AtomicReference<>(); + final AtomicReference messageAtomic = new AtomicReference(); ConsumeMessageConcurrentlyService normalServie = new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() { @Override diff --git a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java index a890e721dcd..5ce761c30f5 100644 --- a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java @@ -257,7 +257,7 @@ public MessageQueue select(List mqs, Message msg, Object arg) { } }; - List msgs = new ArrayList<>(); + List msgs = new ArrayList(); for (int i = 0; i < 5; i++) { Message message = new Message(); message.setTopic("test"); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithOpenTracingTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithOpenTracingTest.java index ecf72ae44cf..4864522c66c 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithOpenTracingTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithOpenTracingTest.java @@ -25,7 +25,9 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -97,12 +99,14 @@ public class DefaultMQConsumerWithOpenTracingTest { private PullAPIWrapper pullAPIWrapper; private RebalancePushImpl rebalancePushImpl; private DefaultMQPushConsumer pushConsumer; - private MockTracer tracer = new MockTracer(); + private final MockTracer tracer = new MockTracer(); @Before public void init() throws Exception { ConcurrentMap factoryTable = (ConcurrentMap) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true); - factoryTable.forEach((s, instance) -> instance.shutdown()); + for (Map.Entry entry : factoryTable.entrySet()) { + entry.getValue().shutdown(); + } factoryTable.clear(); when(mQClientAPIImpl.pullMessage(anyString(), any(PullMessageRequestHeader.class), @@ -115,7 +119,7 @@ public PullResult answer(InvocationOnMock mock) throws Throwable { messageClientExt.setTopic(topic); messageClientExt.setQueueId(0); messageClientExt.setMsgId("123"); - messageClientExt.setBody(new byte[]{'a'}); + messageClientExt.setBody(new byte[] {'a'}); messageClientExt.setOffsetMsgId("234"); messageClientExt.setBornHost(new InetSocketAddress(8080)); messageClientExt.setStoreHost(new InetSocketAddress(8080)); @@ -125,11 +129,10 @@ public PullResult answer(InvocationOnMock mock) throws Throwable { } }); - consumerGroup = "FooBarGroup" + System.currentTimeMillis(); pushConsumer = new DefaultMQPushConsumer(consumerGroup); pushConsumer.getDefaultMQPushConsumerImpl().registerConsumeMessageHook( - new ConsumeMessageOpenTracingHookImpl(tracer)); + new ConsumeMessageOpenTracingHookImpl(tracer)); pushConsumer.setNamesrvAddr("127.0.0.1:9876"); pushConsumer.setPullInterval(60 * 1000); @@ -140,7 +143,7 @@ public PullResult answer(InvocationOnMock mock) throws Throwable { pushConsumer.registerMessageListener(new MessageListenerConcurrently() { @Override public ConsumeConcurrentlyStatus consumeMessage(List msgs, - ConsumeConcurrentlyContext context) { + ConsumeConcurrentlyContext context) { return null; } }); @@ -173,11 +176,11 @@ public void terminate() { @Test public void testPullMessage_WithTrace_Success() throws InterruptedException, RemotingException, MQBrokerException, MQClientException { final CountDownLatch countDownLatch = new CountDownLatch(1); - final AtomicReference messageAtomic = new AtomicReference<>(); + final AtomicReference messageAtomic = new AtomicReference(); pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() { @Override public ConsumeConcurrentlyStatus consumeMessage(List msgs, - ConsumeConcurrentlyContext context) { + ConsumeConcurrentlyContext context) { messageAtomic.set(msgs.get(0)); countDownLatch.countDown(); return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; @@ -190,10 +193,15 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, MessageExt msg = messageAtomic.get(); assertThat(msg).isNotNull(); assertThat(msg.getTopic()).isEqualTo(topic); - assertThat(msg.getBody()).isEqualTo(new byte[]{'a'}); + assertThat(msg.getBody()).isEqualTo(new byte[] {'a'}); // wait until consumeMessageAfter hook of tracer is done surely. - waitAtMost(1, TimeUnit.SECONDS).until(() -> tracer.finishedSpans().size() == 1); + waitAtMost(1, TimeUnit.SECONDS).until(new Callable() { + @Override public Object call() throws Exception { + return tracer.finishedSpans().size() == 1; + } + }); + MockSpan span = tracer.finishedSpans().get(0); assertThat(span.tags().get(Tags.MESSAGE_BUS_DESTINATION.getKey())).isEqualTo(topic); assertThat(span.tags().get(Tags.SPAN_KIND.getKey())).isEqualTo(Tags.SPAN_KIND_CONSUMER); @@ -219,7 +227,7 @@ private PullRequest createPullRequest() { } private PullResultExt createPullResult(PullMessageRequestHeader requestHeader, PullStatus pullStatus, - List messageExtList) throws Exception { + List messageExtList) throws Exception { ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); for (MessageExt messageExt : messageExtList) { outputStream.write(MessageDecoder.encode(messageExt, false)); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java index d142e94d76a..50e4456b868 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -116,7 +117,9 @@ public class DefaultMQConsumerWithTraceTest { @Before public void init() throws Exception { ConcurrentMap factoryTable = (ConcurrentMap) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true); - factoryTable.forEach((s, instance) -> instance.shutdown()); + for (Map.Entry entry : factoryTable.entrySet()) { + entry.getValue().shutdown(); + } factoryTable.clear(); consumerGroup = "FooBarGroup" + System.currentTimeMillis(); @@ -217,7 +220,7 @@ public void testPullMessage_WithTrace_Success() throws InterruptedException, Rem traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); final CountDownLatch countDownLatch = new CountDownLatch(1); - final AtomicReference messageAtomic = new AtomicReference<>(); + final AtomicReference messageAtomic = new AtomicReference(); pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() { @Override public ConsumeConcurrentlyStatus consumeMessage(List msgs, diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java index 02289505d9a..3454bf078cc 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java @@ -17,6 +17,12 @@ package org.apache.rocketmq.client.trace; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQClientException; @@ -52,17 +58,16 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.Spy; +import org.mockito.invocation.InvocationOnMock; import org.mockito.junit.MockitoJUnitRunner; - -import java.lang.reflect.Field; -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.concurrent.atomic.AtomicReference; +import org.mockito.stubbing.Answer; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.nullable; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.when; @@ -127,7 +132,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { Field fieldHooks = DefaultMQProducerImpl.class.getDeclaredField("endTransactionHookList"); fieldHooks.setAccessible(true); - Listhooks = new ArrayList<>(); + List hooks = new ArrayList(); hooks.add(endTransactionHook); fieldHooks.set(producer.getDefaultMQProducerImpl(), hooks); @@ -143,11 +148,14 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); - AtomicReference context = new AtomicReference<>(); - doAnswer(mock -> { - context.set(mock.getArgument(0)); - return null; - }).when(endTransactionHook).endTransaction(any()); + final AtomicReference context = new AtomicReference(); + doAnswer(new Answer() { + @Override public Object answer(InvocationOnMock mock) throws Throwable { + context.set((EndTransactionContext) mock.getArgument(0)); + return null; + } + + }).when(endTransactionHook).endTransaction(any(EndTransactionContext.class)); producer.sendMessageInTransaction(message, null); EndTransactionContext ctx = context.get(); From 92bf68cb77ed59bb659d3d93e3d7c078a87571dd Mon Sep 17 00:00:00 2001 From: XiaoyiPeng <644120242@qq.com> Date: Fri, 31 Dec 2021 17:23:26 +0800 Subject: [PATCH 083/141] [ISSUE #3687] Fix wrong method comment about transaction message. --- .../transaction/queue/TransactionalMessageBridge.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java index 012b9def0d0..9d0152d55bb 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java @@ -303,13 +303,13 @@ private TopicConfig selectTopicConfig(String topic) { * Use this function while transaction msg is committed or rollback write a flag 'd' to operation queue for the * msg's offset * - * @param messageExt Op message - * @param messageQueue Op message queue + * @param prepareMessage Half message + * @param messageQueue Half message queue * @return This method will always return true. */ - private boolean addRemoveTagInTransactionOp(MessageExt messageExt, MessageQueue messageQueue) { + private boolean addRemoveTagInTransactionOp(MessageExt prepareMessage, MessageQueue messageQueue) { Message message = new Message(TransactionalMessageUtil.buildOpTopic(), TransactionalMessageUtil.REMOVETAG, - String.valueOf(messageExt.getQueueOffset()).getBytes(TransactionalMessageUtil.charset)); + String.valueOf(prepareMessage.getQueueOffset()).getBytes(TransactionalMessageUtil.charset)); writeOp(message, messageQueue); return true; } From 6ebb4e40d4943679d5e37dbc96919e8cfc5808f5 Mon Sep 17 00:00:00 2001 From: huitong Date: Fri, 31 Dec 2021 14:36:41 +0800 Subject: [PATCH 084/141] fix topicRouteMap maintain error relationship --- .../apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java index 80999c24249..1e835f79f77 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java @@ -484,7 +484,9 @@ public List resetOffsetByTimestampOld(String consumerGroup, Strin Map topicRouteMap = new HashMap(); for (BrokerData bd : topicRouteData.getBrokerDatas()) { for (QueueData queueData : topicRouteData.getQueueDatas()) { - topicRouteMap.put(bd.selectBrokerAddr(), queueData.getReadQueueNums()); + if (StringUtils.equals(queueData.getBrokerName(), bd.getBrokerName())) { + topicRouteMap.put(bd.selectBrokerAddr(), queueData.getReadQueueNums()); + } } } for (BrokerData bd : topicRouteData.getBrokerDatas()) { From 3f4329ce7fbec8b2c4530043bc6ce1d67e0f3a17 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Thu, 6 Jan 2022 09:14:40 +0800 Subject: [PATCH 085/141] [ISSUE #3692] docs: Modify the title content about the message query documentation. (#3693) --- docs/en/Design_Query.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/Design_Query.md b/docs/en/Design_Query.md index 6988948d844..983d48df9b0 100644 --- a/docs/en/Design_Query.md +++ b/docs/en/Design_Query.md @@ -5,7 +5,7 @@ RocketMQ supports message queries by two dimensions, which are "Query Message by ## 1. Query Message by Message Id The MessageId in RocketMQ has a total length of 16 bytes, including the broker address (IP address and port) and CommitLog offset. In RocketMQ, the specific approach is that the Client resolves the Broker's address (IP address and port) and the CommitLog's offset address from the MessageId. Then both of them are encapsulated into an RPC request, and finally it will be sent through the communication layer (business request code: VIEW_MESSAGE_BY_ID). The Broker reads a message by using the CommitLog offset and size to find the real message in the CommitLog and then return, which is how QueryMessageProcessor works. -## 2. Query Message by Message Id +## 2. Query Message by Message Key "Query Messages by Message Key" is mainly based on RocketMQ's IndexFile. The logical structure of the IndexFile is similar to the implementation of HashMap in JDK. The specific structure of the IndexFile is as follows: ![](images/rocketmq_design_message_query.png) From 0ceb9e6ceb761cc26d26fae3f8818fe03ddf0769 Mon Sep 17 00:00:00 2001 From: zhangjidi2016 <1017543663@qq.com> Date: Thu, 6 Jan 2022 09:34:33 +0800 Subject: [PATCH 086/141] [ISSUE #3696]Optimize the query consumer connection display (#3697) --- .../connection/ConsumerConnectionSubCommand.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java index 4e53caf6f54..1276712dee6 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java @@ -64,10 +64,9 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t ConsumerConnection cc = defaultMQAdminExt.examineConsumerConnectionInfo(group); - int i = 1; + System.out.printf("%-36s %-22s %-10s %s%n", "#ClientId", "#ClientAddr", "#Language", "#Version"); for (Connection conn : cc.getConnectionSet()) { - System.out.printf("%03d %-32s %-22s %-8s %s%n", - i++, + System.out.printf("%-36s %-22s %-10s %s%n", conn.getClientId(), conn.getClientAddr(), conn.getLanguage(), @@ -75,20 +74,19 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t ); } - System.out.printf("%nBelow is subscription:"); + System.out.printf("%nBelow is subscription:\n"); Iterator> it = cc.getSubscriptionTable().entrySet().iterator(); - i = 1; + System.out.printf("%-20s %s%n", "#Topic", "#SubExpression"); while (it.hasNext()) { Entry entry = it.next(); SubscriptionData sd = entry.getValue(); - System.out.printf("%03d Topic: %-40s SubExpression: %s%n", - i++, + System.out.printf("%-20s %s%n", sd.getTopic(), sd.getSubString() ); } - System.out.printf(""); + System.out.printf("\n"); System.out.printf("ConsumeType: %s%n", cc.getConsumeType()); System.out.printf("MessageModel: %s%n", cc.getMessageModel()); System.out.printf("ConsumeFromWhere: %s%n", cc.getConsumeFromWhere()); From 43009b76ac79643ff222af6c13dfe24a593dfa43 Mon Sep 17 00:00:00 2001 From: Kobold <2807645688@qq.com> Date: Thu, 6 Jan 2022 09:51:20 +0800 Subject: [PATCH 087/141] [ISSUE #3685] Fix param spelling error (#3678) * Update .travis.yml Depracated MODERATE noisy mail for dev@rocketmq.apache.org. Still Failing: apache/rocketmq#5670 (5.0.0-alpha-static-topic - 5ebc327) @dongeforever * fix class[RemotingHelper] method[exceptionSimpleDesc] var[element] spelling error --- .travis.yml | 2 ++ .../org/apache/rocketmq/remoting/common/RemotingHelper.java | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 669d9e9bd3f..c55d9291e92 100644 --- a/.travis.yml +++ b/.travis.yml @@ -4,8 +4,10 @@ notifications: email: recipients: - dev@rocketmq.apache.org + if: branch = develop OR branch = master on_success: change on_failure: always + language: java diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java index 34efc4eef38..39bbb0da488 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingHelper.java @@ -47,9 +47,9 @@ public static String exceptionSimpleDesc(final Throwable e) { StackTraceElement[] stackTrace = e.getStackTrace(); if (stackTrace != null && stackTrace.length > 0) { - StackTraceElement elment = stackTrace[0]; + StackTraceElement element = stackTrace[0]; sb.append(", "); - sb.append(elment.toString()); + sb.append(element.toString()); } } From 3738214a355d937441ea3854bc8a80cd5b2b4e3c Mon Sep 17 00:00:00 2001 From: zhangjidi2016 Date: Fri, 7 Jan 2022 01:34:33 +0800 Subject: [PATCH 088/141] add comment --- .../src/main/java/org/apache/rocketmq/common/BrokerConfig.java | 3 +++ 1 file changed, 3 insertions(+) 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 a6169d74d11..e3d47c393d0 100644 --- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java @@ -187,6 +187,9 @@ public class BrokerConfig { private boolean autoDeleteUnusedStats = false; + /** + * Whether to distinguish log paths when multiple brokers are deployed on the same machine + */ private boolean isolateLogEnable = false; public static String localHostName() { From d564dcd31a7e5740693e5faef8863ea9fd739a9c Mon Sep 17 00:00:00 2001 From: weibubli <46190832+weibubli@users.noreply.github.com> Date: Fri, 7 Jan 2022 20:23:15 +0800 Subject: [PATCH 089/141] [ISSUE #3709] Resolve export metadata errors while multiple brokers. (#3727) * Update ExportMetadataCommand.java * Update ExportMetadataCommand.java --- .../command/export/ExportMetadataCommand.java | 73 +++++++++---------- 1 file changed, 35 insertions(+), 38 deletions(-) diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java index 1909436467b..397be8a20d0 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java @@ -25,7 +25,6 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; -import org.apache.rocketmq.common.MQVersion; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.protocol.body.SubscriptionGroupWrapper; @@ -119,6 +118,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) Map topicConfigMap = new HashMap<>(); Map subGroupConfigMap = new HashMap<>(); + Map result = new HashMap<>(); for (String addr : masterSet) { TopicConfigSerializeWrapper topicConfigSerializeWrapper = defaultMQAdminExt.getUserTopicConfig( @@ -127,50 +127,47 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) SubscriptionGroupWrapper subscriptionGroupWrapper = defaultMQAdminExt.getUserSubscriptionGroup( addr, 10000); - if (commandLine.hasOption('t')) { - filePath = filePath + "/topic.json"; - MixAll.string2FileNotSafe(JSON.toJSONString(topicConfigSerializeWrapper, true), filePath); - System.out.printf("export %s success", filePath); - return; - } else if (commandLine.hasOption('g')) { - filePath = filePath + "/subscriptionGroup.json"; - MixAll.string2FileNotSafe(JSON.toJSONString(subscriptionGroupWrapper, true), filePath); - System.out.printf("export %s success", filePath); - return; - } else { - for (Map.Entry entry : topicConfigSerializeWrapper.getTopicConfigTable().entrySet()) { - TopicConfig topicConfig = topicConfigMap.get(entry.getKey()); - if (null != topicConfig) { - entry.getValue().setWriteQueueNums( - topicConfig.getWriteQueueNums() + entry.getValue().getWriteQueueNums()); - entry.getValue().setReadQueueNums( - topicConfig.getReadQueueNums() + entry.getValue().getReadQueueNums()); - } - topicConfigMap.put(entry.getKey(), entry.getValue()); + for (Map.Entry entry : topicConfigSerializeWrapper.getTopicConfigTable() + .entrySet()) { + TopicConfig topicConfig = topicConfigMap.get(entry.getKey()); + if (null != topicConfig) { + entry.getValue().setWriteQueueNums( + topicConfig.getWriteQueueNums() + entry.getValue().getWriteQueueNums()); + entry.getValue().setReadQueueNums( + topicConfig.getReadQueueNums() + entry.getValue().getReadQueueNums()); } + topicConfigMap.put(entry.getKey(), entry.getValue()); + } - for (Map.Entry entry : subscriptionGroupWrapper.getSubscriptionGroupTable().entrySet()) { + for (Map.Entry entry : subscriptionGroupWrapper.getSubscriptionGroupTable() + .entrySet()) { - SubscriptionGroupConfig subscriptionGroupConfig = subGroupConfigMap.get(entry.getKey()); - if (null != subscriptionGroupConfig) { - entry.getValue().setRetryQueueNums( - subscriptionGroupConfig.getRetryQueueNums() + entry.getValue().getRetryQueueNums()); - } - subGroupConfigMap.put(entry.getKey(), entry.getValue()); + SubscriptionGroupConfig subscriptionGroupConfig = subGroupConfigMap.get(entry.getKey()); + if (null != subscriptionGroupConfig) { + entry.getValue().setRetryQueueNums( + subscriptionGroupConfig.getRetryQueueNums() + entry.getValue().getRetryQueueNums()); } - - Map result = new HashMap<>(); - result.put("topicConfigTable", topicConfigMap); - result.put("subscriptionGroupTable", subGroupConfigMap); - result.put("rocketmqVersion", MQVersion.getVersionDesc(MQVersion.CURRENT_VERSION)); - result.put("exportTime", System.currentTimeMillis()); - - filePath = filePath + "/metadata.json"; - MixAll.string2FileNotSafe(JSON.toJSONString(result, true), filePath); - System.out.printf("export %s success", filePath); + subGroupConfigMap.put(entry.getKey(), entry.getValue()); } } + + String exportPath; + if (commandLine.hasOption('t')) { + result.put("topicConfigTable", topicConfigMap); + exportPath = filePath + "/topic.json"; + } else if (commandLine.hasOption('g')) { + result.put("subscriptionGroupTable", subGroupConfigMap); + exportPath = filePath + "/subscriptionGroup.json"; + } else { + result.put("topicConfigTable", topicConfigMap); + result.put("subscriptionGroupTable", subGroupConfigMap); + exportPath = filePath + "/metadata.json"; + } + result.put("exportTime", System.currentTimeMillis()); + MixAll.string2FileNotSafe(JSON.toJSONString(result, true), exportPath); + System.out.printf("export %s success%n", exportPath); + } else { ServerUtil.printCommandLineHelp("mqadmin " + this.commandName(), options); } From 938e201d1a9cded4fb5deb6bd1330f7a6c425f20 Mon Sep 17 00:00:00 2001 From: rongtong Date: Sat, 8 Jan 2022 20:11:33 +0800 Subject: [PATCH 090/141] [ISSUE #3733] Upgrade dledger version to 0.2.3 (#3734) --- store/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/pom.xml b/store/pom.xml index a37c986f7c6..46aee036231 100644 --- a/store/pom.xml +++ b/store/pom.xml @@ -31,7 +31,7 @@ io.openmessaging.storage dledger - 0.2.2 + 0.2.3 org.apache.rocketmq From 1ef1af6a995ce61d434af52e2f498c9f3559e7d0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Sat, 8 Jan 2022 21:42:03 +0800 Subject: [PATCH 091/141] [ISSUE #3728] docs: Make the 'Uses of keys' entry in the document 'Best practices' more complete. (#3729) --- docs/en/best_practice.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/best_practice.md b/docs/en/best_practice.md index c50c5df415b..e9d0ee4e307 100755 --- a/docs/en/best_practice.md +++ b/docs/en/best_practice.md @@ -17,6 +17,19 @@ Since it is a hash index, make sure that the key is as unique as possible to avo String orderId = "20034568923546"; message.setKeys(orderId); ``` +If you have multiple keys for a message, please concatenate them with 'KEY_SEPARATOR' char, as shown below: +```java + // order id + String orderId = "20034568923546"; + String otherKey = "19101121210831"; + String keys = new StringBuilder(orderId) + .append(org.apache.rocketmq.common.message.MessageConst.KEY_SEPARATOR) + .append(otherKey).toString(); + message.setKeys(keys); +``` +And if you want to query the message, please use `orderId` and `otherKey` to query respectively instead of `keys`, +because the server will unwrap `keys` with `KEY_SEPARATOR` and create corresponding index. +In the above example, the server will create two indexes, one for `orderId` and one for `otherKey`. #### 3 Log print Print the message log when send success or failed, make sure to print the SendResult and key fields. Send messages is successful as long as it does not throw exception. Send successful will have multiple states defined in sendResult. From acac49f148b5c1070471d72a10b881cc15c58d6b Mon Sep 17 00:00:00 2001 From: Kealdish <12494239+kealdishx@users.noreply.github.com> Date: Mon, 10 Jan 2022 09:59:39 +0800 Subject: [PATCH 092/141] [ISSUE #3738] Remove useless callee of String.valueOf (#3739) --- .../org/apache/rocketmq/store/StoreStatsService.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java index 2b78f271e41..7bd1a23758b 100644 --- a/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java +++ b/store/src/main/java/org/apache/rocketmq/store/StoreStatsService.java @@ -438,11 +438,11 @@ public HashMap getRuntimeInfo() { String.valueOf(this.getPutMessageSizeTotal() / totalTimes.doubleValue())); result.put("dispatchMaxBuffer", String.valueOf(this.dispatchMaxBuffer)); result.put("getMessageEntireTimeMax", String.valueOf(this.getMessageEntireTimeMax)); - result.put("putTps", String.valueOf(this.getPutTps())); - result.put("getFoundTps", String.valueOf(this.getGetFoundTps())); - result.put("getMissTps", String.valueOf(this.getGetMissTps())); - result.put("getTotalTps", String.valueOf(this.getGetTotalTps())); - result.put("getTransferedTps", String.valueOf(this.getGetTransferedTps())); + result.put("putTps", this.getPutTps()); + result.put("getFoundTps", this.getGetFoundTps()); + result.put("getMissTps", this.getGetMissTps()); + result.put("getTotalTps", this.getGetTotalTps()); + result.put("getTransferedTps", this.getGetTransferedTps()); return result; } From 0b612ff813051cb3d4272bbd50ed280cbe4da244 Mon Sep 17 00:00:00 2001 From: ltamber Date: Wed, 12 Jan 2022 11:16:10 +0800 Subject: [PATCH 093/141] [ISSUE #3223]Optimize startup script to support greater jdk version (#3224) * support jdk9 and later version * support jdk9 and later version --- broker/pom.xml | 4 ++++ distribution/benchmark/runclass.sh | 3 +-- distribution/bin/runbroker.cmd | 3 +-- distribution/bin/runbroker.sh | 25 ++++++++++++++++++++----- distribution/bin/runserver.cmd | 3 +-- distribution/bin/runserver.sh | 3 +-- distribution/bin/tools.cmd | 3 +-- distribution/bin/tools.sh | 3 +-- namesrv/pom.xml | 4 ++++ pom.xml | 7 +++++++ 10 files changed, 41 insertions(+), 17 deletions(-) diff --git a/broker/pom.xml b/broker/pom.xml index 1de386de27e..d36ad8b73ef 100644 --- a/broker/pom.xml +++ b/broker/pom.xml @@ -66,6 +66,10 @@ org.slf4j slf4j-api + + org.bouncycastle + bcpkix-jdk15on + diff --git a/distribution/benchmark/runclass.sh b/distribution/benchmark/runclass.sh index 12802ddf90d..565ad0275eb 100644 --- a/distribution/benchmark/runclass.sh +++ b/distribution/benchmark/runclass.sh @@ -22,7 +22,7 @@ then fi BASE_DIR=$(dirname $0)/.. -CLASSPATH=.:${BASE_DIR}/conf:${CLASSPATH} +CLASSPATH=.:${BASE_DIR}/conf:${BASE_DIR}/lib/*:${CLASSPATH} # The RAMDisk initializing size in MB on Darwin OS for gc-log DIR_SIZE_IN_MB=600 @@ -58,7 +58,6 @@ JAVA_OPT="${JAVA_OPT} -verbose:gc -Xloggc:${GC_LOG_DIR}/rmq_run_class_gc_%p_%t.l JAVA_OPT="${JAVA_OPT} -XX:-OmitStackTraceInFastThrow" JAVA_OPT="${JAVA_OPT} -XX:-UseLargePages" JAVA_OPT="${JAVA_OPT} -XX:+PerfDisableSharedMem" -JAVA_OPT="${JAVA_OPT} -Djava.ext.dirs=${JAVA_HOME}/jre/lib/ext:${BASE_DIR}/lib:${JAVA_HOME}/lib/ext" #JAVA_OPT="${JAVA_OPT} -Xdebug -Xrunjdwp:transport=dt_socket,address=9555,server=y,suspend=n" JAVA_OPT="${JAVA_OPT} -cp ${CLASSPATH}" diff --git a/distribution/bin/runbroker.cmd b/distribution/bin/runbroker.cmd index 6cbf1c031ae..e52230708e3 100644 --- a/distribution/bin/runbroker.cmd +++ b/distribution/bin/runbroker.cmd @@ -23,7 +23,7 @@ set BASE_DIR=%~dp0 set BASE_DIR=%BASE_DIR:~0,-1% for %%d in (%BASE_DIR%) do set BASE_DIR=%%~dpd -set CLASSPATH=.;%BASE_DIR%conf;%CLASSPATH% +set CLASSPATH=.;%BASE_DIR%conf;%BASE_DIR%lib\*;%CLASSPATH% rem =========================================================================================== rem JVM Configuration @@ -36,7 +36,6 @@ set "JAVA_OPT=%JAVA_OPT% -XX:-OmitStackTraceInFastThrow" set "JAVA_OPT=%JAVA_OPT% -XX:+AlwaysPreTouch" set "JAVA_OPT=%JAVA_OPT% -XX:MaxDirectMemorySize=15g" set "JAVA_OPT=%JAVA_OPT% -XX:-UseLargePages -XX:-UseBiasedLocking" -set "JAVA_OPT=%JAVA_OPT% -Djava.ext.dirs=%BASE_DIR%lib;%JAVA_HOME%\jre\lib\ext" set "JAVA_OPT=%JAVA_OPT% -cp %CLASSPATH%" "%JAVA%" %JAVA_OPT% %* \ No newline at end of file diff --git a/distribution/bin/runbroker.sh b/distribution/bin/runbroker.sh index d1a148b36c6..b9048ec3c6c 100644 --- a/distribution/bin/runbroker.sh +++ b/distribution/bin/runbroker.sh @@ -31,7 +31,7 @@ error_exit () export JAVA_HOME export JAVA="$JAVA_HOME/bin/java" export BASE_DIR=$(dirname $0)/.. -export CLASSPATH=.:${BASE_DIR}/conf:${CLASSPATH} +export CLASSPATH=.:${BASE_DIR}/conf:${BASE_DIR}/lib/*:${CLASSPATH} #=========================================================================================== # JVM Configuration @@ -62,17 +62,32 @@ choose_gc_log_directory() esac } +choose_gc_options() +{ + JAVA_MAJOR_VERSION=$("$JAVA" -version 2>&1 | head -1 | cut -d'"' -f2 | sed 's/^1\.//' | cut -d'.' -f1) + if [ -z "$JAVA_MAJOR_VERSION" ] || [ "$JAVA_MAJOR_VERSION" -lt "8" ] ; then + JAVA_OPT="${JAVA_OPT} -XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC" + else + JAVA_OPT="${JAVA_OPT} -XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0" + fi + + if [ -z "$JAVA_MAJOR_VERSION" ] || [ "$JAVA_MAJOR_VERSION" -lt "9" ] ; then + JAVA_OPT="${JAVA_OPT} -verbose:gc -Xloggc:${GC_LOG_DIR}/rmq_srv_gc_%p_%t.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintAdaptiveSizePolicy" + JAVA_OPT="${JAVA_OPT} -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m" + else + JAVA_OPT="${JAVA_OPT} -XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0" + JAVA_OPT="${JAVA_OPT} -Xlog:gc*:file=${GC_LOG_DIR}/rmq_srv_gc_%p_%t.log:time,tags:filecount=5,filesize=30M" + fi +} + choose_gc_log_directory JAVA_OPT="${JAVA_OPT} -server -Xms8g -Xmx8g" -JAVA_OPT="${JAVA_OPT} -XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0" -JAVA_OPT="${JAVA_OPT} -verbose:gc -Xloggc:${GC_LOG_DIR}/rmq_broker_gc_%p_%t.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintAdaptiveSizePolicy" -JAVA_OPT="${JAVA_OPT} -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m" +choose_gc_options JAVA_OPT="${JAVA_OPT} -XX:-OmitStackTraceInFastThrow" JAVA_OPT="${JAVA_OPT} -XX:+AlwaysPreTouch" JAVA_OPT="${JAVA_OPT} -XX:MaxDirectMemorySize=15g" JAVA_OPT="${JAVA_OPT} -XX:-UseLargePages -XX:-UseBiasedLocking" -JAVA_OPT="${JAVA_OPT} -Djava.ext.dirs=${JAVA_HOME}/jre/lib/ext:${BASE_DIR}/lib:${JAVA_HOME}/lib/ext" #JAVA_OPT="${JAVA_OPT} -Xdebug -Xrunjdwp:transport=dt_socket,address=9555,server=y,suspend=n" JAVA_OPT="${JAVA_OPT} ${JAVA_OPT_EXT}" JAVA_OPT="${JAVA_OPT} -cp ${CLASSPATH}" diff --git a/distribution/bin/runserver.cmd b/distribution/bin/runserver.cmd index ddd9b5fc70a..2bea8edd9c4 100644 --- a/distribution/bin/runserver.cmd +++ b/distribution/bin/runserver.cmd @@ -24,14 +24,13 @@ set BASE_DIR=%~dp0 set BASE_DIR=%BASE_DIR:~0,-1% for %%d in (%BASE_DIR%) do set BASE_DIR=%%~dpd -set CLASSPATH=.;%BASE_DIR%conf;%CLASSPATH% +set CLASSPATH=.;%BASE_DIR%conf;%BASE_DIR%lib\*;%CLASSPATH% set "JAVA_OPT=%JAVA_OPT% -server -Xms2g -Xmx2g -Xmn1g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m" set "JAVA_OPT=%JAVA_OPT% -XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC" set "JAVA_OPT=%JAVA_OPT% -verbose:gc -Xloggc:"%USERPROFILE%\rmq_srv_gc.log" -XX:+PrintGCDetails -XX:+PrintGCDateStamps" set "JAVA_OPT=%JAVA_OPT% -XX:-OmitStackTraceInFastThrow" set "JAVA_OPT=%JAVA_OPT% -XX:-UseLargePages" -set "JAVA_OPT=%JAVA_OPT% -Djava.ext.dirs=%BASE_DIR%lib;%JAVA_HOME%\jre\lib\ext" set "JAVA_OPT=%JAVA_OPT% -cp "%CLASSPATH%"" "%JAVA%" %JAVA_OPT% %* \ No newline at end of file diff --git a/distribution/bin/runserver.sh b/distribution/bin/runserver.sh index cfa577345b0..47cf0f009b2 100644 --- a/distribution/bin/runserver.sh +++ b/distribution/bin/runserver.sh @@ -31,7 +31,7 @@ error_exit () export JAVA_HOME export JAVA="$JAVA_HOME/bin/java" export BASE_DIR=$(dirname $0)/.. -export CLASSPATH=.:${BASE_DIR}/conf:${CLASSPATH} +export CLASSPATH=.:${BASE_DIR}/conf:${BASE_DIR}/lib/*:${CLASSPATH} #=========================================================================================== # JVM Configuration @@ -83,7 +83,6 @@ choose_gc_log_directory choose_gc_options JAVA_OPT="${JAVA_OPT} -XX:-OmitStackTraceInFastThrow" JAVA_OPT="${JAVA_OPT} -XX:-UseLargePages" -JAVA_OPT="${JAVA_OPT} -Djava.ext.dirs=${JAVA_HOME}/jre/lib/ext:${BASE_DIR}/lib:${JAVA_HOME}/lib/ext" #JAVA_OPT="${JAVA_OPT} -Xdebug -Xrunjdwp:transport=dt_socket,address=9555,server=y,suspend=n" JAVA_OPT="${JAVA_OPT} ${JAVA_OPT_EXT}" JAVA_OPT="${JAVA_OPT} -cp ${CLASSPATH}" diff --git a/distribution/bin/tools.cmd b/distribution/bin/tools.cmd index c61fbcf8a47..263700dc3e8 100644 --- a/distribution/bin/tools.cmd +++ b/distribution/bin/tools.cmd @@ -23,13 +23,12 @@ set BASE_DIR=%~dp0 set BASE_DIR=%BASE_DIR:~0,-1% for %%d in (%BASE_DIR%) do set BASE_DIR=%%~dpd -set CLASSPATH=.;%BASE_DIR%conf;%CLASSPATH% +set CLASSPATH=.;%BASE_DIR%conf;%BASE_DIR%lib\*;%CLASSPATH% rem =========================================================================================== rem JVM Configuration rem =========================================================================================== set "JAVA_OPT=%JAVA_OPT% -server -Xms1g -Xmx1g -Xmn256m -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=128m" -set "JAVA_OPT=%JAVA_OPT% -Djava.ext.dirs="%BASE_DIR%\lib";"%JAVA_HOME%\jre\lib\ext";"%JAVA_HOME%\lib\ext"" set "JAVA_OPT=%JAVA_OPT% -cp "%CLASSPATH%"" "%JAVA%" %JAVA_OPT% %* diff --git a/distribution/bin/tools.sh b/distribution/bin/tools.sh index a13dc7e620a..d3fa7eb65e0 100644 --- a/distribution/bin/tools.sh +++ b/distribution/bin/tools.sh @@ -31,13 +31,12 @@ error_exit () export JAVA_HOME export JAVA="$JAVA_HOME/bin/java" export BASE_DIR=$(dirname $0)/.. -export CLASSPATH=.:${BASE_DIR}/conf:${CLASSPATH} +export CLASSPATH=.:${BASE_DIR}/conf:${BASE_DIR}/lib/*:${CLASSPATH} #=========================================================================================== # JVM Configuration #=========================================================================================== JAVA_OPT="${JAVA_OPT} -server -Xms1g -Xmx1g -Xmn256m -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=128m" -JAVA_OPT="${JAVA_OPT} -Djava.ext.dirs=${BASE_DIR}/lib:${JAVA_HOME}/jre/lib/ext:${JAVA_HOME}/lib/ext" JAVA_OPT="${JAVA_OPT} -cp ${CLASSPATH}" $JAVA ${JAVA_OPT} "$@" diff --git a/namesrv/pom.xml b/namesrv/pom.xml index 32ebda33f56..8f650c8da29 100644 --- a/namesrv/pom.xml +++ b/namesrv/pom.xml @@ -48,5 +48,9 @@ org.slf4j slf4j-api + + org.bouncycastle + bcpkix-jdk15on + diff --git a/pom.xml b/pom.xml index 475918c932e..19abca03605 100644 --- a/pom.xml +++ b/pom.xml @@ -539,6 +539,13 @@ netty-all 4.1.65.Final + + org.bouncycastle + bcpkix-jdk15on + runtime + jar + 1.69 + com.alibaba fastjson From 5233f12f3ad6cdd8b08b3dc399724bf1d1017324 Mon Sep 17 00:00:00 2001 From: HuiTong Date: Thu, 13 Jan 2022 13:47:37 +0800 Subject: [PATCH 094/141] rollback spelling fix to avoid incompatile (#3749) --- .../impl/consumer/ConsumeMessageOrderlyService.java | 2 +- .../impl/consumer/DefaultLitePullConsumerImpl.java | 2 +- .../impl/consumer/DefaultMQPullConsumerImpl.java | 2 +- .../impl/consumer/DefaultMQPushConsumerImpl.java | 6 +++--- .../rocketmq/client/impl/consumer/RebalanceImpl.java | 4 ++-- .../client/impl/consumer/RebalancePushImpl.java | 4 ++-- .../client/impl/producer/DefaultMQProducerImpl.java | 10 +++++----- .../rocketmq/client/trace/AsyncTraceDispatcher.java | 2 +- .../client/trace/hook/EndTransactionTraceHookImpl.java | 2 +- .../ConsumeMessageConcurrentlyServiceTest.java | 4 ++-- .../client/producer/DefaultMQProducerTest.java | 4 ++-- .../client/trace/DefaultMQConsumerWithTraceTest.java | 8 ++++---- .../trace/DefaultMQLitePullConsumerWithTraceTest.java | 2 +- .../trace/DefaultMQProducerWithOpenTracingTest.java | 4 ++-- .../client/trace/DefaultMQProducerWithTraceTest.java | 4 ++-- .../TransactionMQProducerWithOpenTracingTest.java | 4 ++-- .../trace/TransactionMQProducerWithTraceTest.java | 4 ++-- .../apache/rocketmq/example/simple/PullConsumer.java | 2 +- 18 files changed, 35 insertions(+), 35 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java index cff0d6795e5..8d92b5718a6 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java @@ -378,7 +378,7 @@ public boolean sendMessageBack(final MessageExt msg) { MessageAccessor.clearProperty(newMsg, MessageConst.PROPERTY_TRANSACTION_PREPARED); newMsg.setDelayTimeLevel(3 + msg.getReconsumeTimes()); - this.defaultMQPushConsumer.getDefaultMQPushConsumerImpl().getMqClientFactory().getDefaultMQProducer().send(newMsg); + this.defaultMQPushConsumer.getDefaultMQPushConsumerImpl().getmQClientFactory().getDefaultMQProducer().send(newMsg); return true; } catch (Exception e) { log.error("sendMessageBack exception, group: " + this.consumerGroup + " msg: " + msg.toString(), e); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java index fecf07645ed..46a72fb5e84 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java @@ -329,7 +329,7 @@ private void initRebalanceImpl() { this.rebalanceImpl.setConsumerGroup(this.defaultLitePullConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultLitePullConsumer.getMessageModel()); this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultLitePullConsumer.getAllocateMessageQueueStrategy()); - this.rebalanceImpl.setMqClientFactory(this.mQClientFactory); + this.rebalanceImpl.setmQClientFactory(this.mQClientFactory); } private void initPullAPIWrapper() { diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java index 6792d4d609c..eed5fa43f04 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java @@ -637,7 +637,7 @@ public synchronized void start() throws MQClientException { this.rebalanceImpl.setConsumerGroup(this.defaultMQPullConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultMQPullConsumer.getMessageModel()); this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultMQPullConsumer.getAllocateMessageQueueStrategy()); - this.rebalanceImpl.setMqClientFactory(this.mQClientFactory); + this.rebalanceImpl.setmQClientFactory(this.mQClientFactory); this.pullAPIWrapper = new PullAPIWrapper( mQClientFactory, diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java index a61a1a6ac2f..59b8deb3fba 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPushConsumerImpl.java @@ -591,7 +591,7 @@ public synchronized void start() throws MQClientException { this.rebalanceImpl.setConsumerGroup(this.defaultMQPushConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultMQPushConsumer.getMessageModel()); this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultMQPushConsumer.getAllocateMessageQueueStrategy()); - this.rebalanceImpl.setMqClientFactory(this.mQClientFactory); + this.rebalanceImpl.setmQClientFactory(this.mQClientFactory); this.pullAPIWrapper = new PullAPIWrapper( mQClientFactory, @@ -1089,11 +1089,11 @@ public ConsumerRunningInfo consumerRunningInfo() { return info; } - public MQClientInstance getMqClientFactory() { + public MQClientInstance getmQClientFactory() { return mQClientFactory; } - public void setMqClientFactory(MQClientInstance mQClientFactory) { + public void setmQClientFactory(MQClientInstance mQClientFactory) { this.mQClientFactory = mQClientFactory; } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java index 41327547b0e..7677d8b685f 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceImpl.java @@ -472,11 +472,11 @@ public void setAllocateMessageQueueStrategy(AllocateMessageQueueStrategy allocat this.allocateMessageQueueStrategy = allocateMessageQueueStrategy; } - public MQClientInstance getMqClientFactory() { + public MQClientInstance getmQClientFactory() { return mQClientFactory; } - public void setMqClientFactory(MQClientInstance mQClientFactory) { + public void setmQClientFactory(MQClientInstance mQClientFactory) { this.mQClientFactory = mQClientFactory; } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java index 464bbf2a1d3..666b696ffa8 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImpl.java @@ -78,7 +78,7 @@ public void messageQueueChanged(String topic, Set mqAll, Set nsList = this.getMqClientFactory().getMQClientAPIImpl().getNameServerAddressList(); + List nsList = this.getmQClientFactory().getMQClientAPIImpl().getNameServerAddressList(); if (null == nsList || nsList.isEmpty()) { throw new MQClientException( "No name server address, please set it." + FAQUrl.suggestTodo(FAQUrl.NAME_SERVER_ADDR_NOT_EXIST_URL), null).setResponseCode(ClientErrorCode.NO_NAME_SERVER_EXCEPTION); @@ -871,7 +871,7 @@ private SendResult sendKernelImpl(final Message msg, throw new MQClientException("The broker[" + mq.getBrokerName() + "] not exist", null); } - public MQClientInstance getMqClientFactory() { + public MQClientInstance getmQClientFactory() { return mQClientFactory; } @@ -1543,16 +1543,16 @@ private void requestFail(final String correlationId) { private void prepareSendRequest(final Message msg, long timeout) { String correlationId = CorrelationIdUtil.createCorrelationId(); - String requestClientId = this.getMqClientFactory().getClientId(); + String requestClientId = this.getmQClientFactory().getClientId(); MessageAccessor.putProperty(msg, MessageConst.PROPERTY_CORRELATION_ID, correlationId); MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT, requestClientId); MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_TTL, String.valueOf(timeout)); - boolean hasRouteData = this.getMqClientFactory().getTopicRouteTable().containsKey(msg.getTopic()); + boolean hasRouteData = this.getmQClientFactory().getTopicRouteTable().containsKey(msg.getTopic()); if (!hasRouteData) { long beginTimestamp = System.currentTimeMillis(); this.tryToFindTopicPublishInfo(msg.getTopic()); - this.getMqClientFactory().sendHeartbeatToAllBrokerWithLock(); + this.getmQClientFactory().sendHeartbeatToAllBrokerWithLock(); long cost = System.currentTimeMillis() - beginTimestamp; if (cost > 500) { log.warn("prepare send request for <{}> cost {} ms", msg.getTopic(), cost); diff --git a/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java b/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java index 0626e36bbf6..7ff8bd77e03 100644 --- a/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java +++ b/client/src/main/java/org/apache/rocketmq/client/trace/AsyncTraceDispatcher.java @@ -413,7 +413,7 @@ private Set tryGetMessageQueueBrokerSet(DefaultMQProducerImpl producer, TopicPublishInfo topicPublishInfo = producer.getTopicPublishInfoTable().get(topic); if (null == topicPublishInfo || !topicPublishInfo.ok()) { producer.getTopicPublishInfoTable().putIfAbsent(topic, new TopicPublishInfo()); - producer.getMqClientFactory().updateTopicRouteInfoFromNameServer(topic); + producer.getmQClientFactory().updateTopicRouteInfoFromNameServer(topic); topicPublishInfo = producer.getTopicPublishInfoTable().get(topic); } if (topicPublishInfo.isHaveTopicRouterInfo() || topicPublishInfo.ok()) { diff --git a/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java b/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java index 8172d0459d5..cbd755ba39e 100644 --- a/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/trace/hook/EndTransactionTraceHookImpl.java @@ -63,7 +63,7 @@ public void endTransaction(EndTransactionContext context) { traceBean.setKeys(context.getMessage().getKeys()); traceBean.setStoreHost(context.getBrokerAddr()); traceBean.setMsgType(MessageType.Trans_msg_Commit); - traceBean.setClientHost(((AsyncTraceDispatcher)localDispatcher).getHostProducer().getMqClientFactory().getClientId()); + traceBean.setClientHost(((AsyncTraceDispatcher)localDispatcher).getHostProducer().getmQClientFactory().getClientId()); traceBean.setMsgId(context.getMsgId()); traceBean.setTransactionState(context.getTransactionState()); traceBean.setTransactionId(context.getTransactionId()); diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java index 6c331db0593..6fa76e0380b 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java @@ -102,7 +102,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, pushConsumer.subscribe(topic, "*"); pushConsumer.start(); - mQClientFactory = spy(pushConsumerImpl.getMqClientFactory()); + mQClientFactory = spy(pushConsumerImpl.getmQClientFactory()); field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory"); field.setAccessible(true); field.set(pushConsumerImpl, mQClientFactory); @@ -116,7 +116,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, field.setAccessible(true); field.set(pushConsumerImpl, pullAPIWrapper); - pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setMqClientFactory(mQClientFactory); + pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory); mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl); when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), diff --git a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java index a890e721dcd..7347c2ab24e 100644 --- a/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java @@ -102,7 +102,7 @@ public void init() throws Exception { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -358,7 +358,7 @@ public void testSetCallbackExecutor() throws MQClientException { producer.setCallbackExecutor(customized); NettyRemotingClient remotingClient = (NettyRemotingClient) producer.getDefaultMQProducerImpl() - .getMqClientFactory().getMQClientAPIImpl().getRemotingClient(); + .getmQClientFactory().getMQClientAPIImpl().getRemotingClient(); assertThat(remotingClient.getCallbackExecutor()).isEqualTo(customized); } diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java index d142e94d76a..976380b2723 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQConsumerWithTraceTest.java @@ -154,8 +154,8 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, pushConsumer.start(); - mQClientFactory = spy(pushConsumerImpl.getMqClientFactory()); - mQClientTraceFactory = spy(pushConsumerImpl.getMqClientFactory()); + mQClientFactory = spy(pushConsumerImpl.getmQClientFactory()); + mQClientTraceFactory = spy(pushConsumerImpl.getmQClientFactory()); field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory"); field.setAccessible(true); @@ -178,7 +178,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, field.setAccessible(true); field.set(pushConsumerImpl, pullAPIWrapper); - pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setMqClientFactory(mQClientFactory); + pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory); mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl); when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), @@ -214,7 +214,7 @@ public void terminate() { @Test public void testPullMessage_WithTrace_Success() throws InterruptedException, RemotingException, MQBrokerException, MQClientException { - traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); final CountDownLatch countDownLatch = new CountDownLatch(1); final AtomicReference messageAtomic = new AtomicReference<>(); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java index 884a2f6880e..3f7031e8fe6 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQLitePullConsumerWithTraceTest.java @@ -218,7 +218,7 @@ private void initDefaultLitePullConsumer(DefaultLitePullConsumer litePullConsume field.setAccessible(true); field.set(litePullConsumerImpl, offsetStore); - traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(PullCallback.class))) diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java index cc57a03164c..5d64a93f039 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithOpenTracingTest.java @@ -100,7 +100,7 @@ public void init() throws Exception { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -112,7 +112,7 @@ public void init() throws Exception { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); producer.send(message); assertThat(tracer.finishedSpans().size()).isEqualTo(1); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java index b951ae88deb..234e32e6807 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/DefaultMQProducerWithTraceTest.java @@ -109,7 +109,7 @@ public void init() throws Exception { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -121,7 +121,7 @@ public void init() throws Exception { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); final CountDownLatch countDownLatch = new CountDownLatch(1); try { diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java index 5de15f451ed..dd6d1083ce0 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithOpenTracingTest.java @@ -118,7 +118,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); @@ -130,7 +130,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); producer.sendMessageInTransaction(message, null); diff --git a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java index 02289505d9a..f838817bf9d 100644 --- a/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/trace/TransactionMQProducerWithTraceTest.java @@ -123,7 +123,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - producer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); + producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl()); Field fieldHooks = DefaultMQProducerImpl.class.getDeclaredField("endTransactionHookList"); fieldHooks.setAccessible(true); @@ -141,7 +141,7 @@ public LocalTransactionState checkLocalTransaction(MessageExt msg) { @Test public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { - traceProducer.getDefaultMQProducerImpl().getMqClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); + traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl()); when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute()); AtomicReference context = new AtomicReference<>(); doAnswer(mock -> { diff --git a/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java b/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java index 69067c90c49..ff9ef9ca34e 100644 --- a/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/simple/PullConsumer.java @@ -136,7 +136,7 @@ public long consumeFromOffset(MessageQueue messageQueue) throws MQClientExceptio } public void incPullTPS(String topic, int pullSize) { - consumer.getDefaultMQPullConsumerImpl().getRebalanceImpl().getMqClientFactory() + consumer.getDefaultMQPullConsumerImpl().getRebalanceImpl().getmQClientFactory() .getConsumerStatsManager().incPullTPS(consumer.getConsumerGroup(), topic, pullSize); } }); From 8d28d3fa20b85ba21362bd3f6ad4b118733710c2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Thu, 13 Jan 2022 13:48:04 +0800 Subject: [PATCH 095/141] [ISSUE #3751] docs: Fix typos and other little error in the document 'Configuration_Client.md' (#3752) --- docs/en/Configuration_Client.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/Configuration_Client.md b/docs/en/Configuration_Client.md index 41b573b7ed4..0fcd95fcffd 100644 --- a/docs/en/Configuration_Client.md +++ b/docs/en/Configuration_Client.md @@ -42,7 +42,7 @@ HTTP static server addressing is recommended, because it is simple client deploy #### 2.1 Client Common Configuration -| Pamater Name | Default Value | Description | +| Parameter Name | Default Value | Description | | ----------------------------- | ------- | ------------------------------------------------------------ | | namesrvAddr | | Name Server address list, multiple NameServer addresses are separated by semicolons | | clientIP | local IP | Client local ip address, some machines will fail to recognize the client IP address, which needs to be enforced in the code | @@ -54,7 +54,7 @@ HTTP static server addressing is recommended, because it is simple client deploy #### 2.2 Producer Configuration -| Pamater Name | Default Value | Description | +| Parameter Name | Default Value | Description | | -------------------------------- | ---------------- | ------------------------------------------------------------ | | producerGroup | DEFAULT_PRODUCER | The name of the Producer group. If multiple producers belong to one application and send the same message, they should be grouped into the same group | | createTopicKey | TBW102 | When a message is sent, topics that do not exist on the server are automatically created and a Key is specified that can be used to configure the default route to the topic where the message is sent.| @@ -72,7 +72,7 @@ HTTP static server addressing is recommended, because it is simple client deploy #### 2.3 PushConsumer Configuration -| Pamater Name | Default Value | Description | +| Parameter Name | Default Value | Description | | ---------------------------- | ----------------------------- | ------------------------------------------------------------ | | consumerGroup | DEFAULT_CONSUMER | Consumer group name. If multi Consumer belong to an application, subscribe the same message and consume logic as the same, they should be gathered together | | messageModel | CLUSTERING | Message support two mode: cluster consumption and broadcast consumption | @@ -93,13 +93,13 @@ HTTP static server addressing is recommended, because it is simple client deploy #### 2.4 PullConsumer Configuration -| Pamater Name | Default Value | Description | +| Parameter Name | Default Value | Description | | -------------------------------- | ----------------------------- | ------------------------------------------------------------ | | consumerGroup | DEFAULT_CONSUMER | Consumer group name. If multi Consumer belong to an application, subscribe the same message and consume logic as the same, they should be gathered together | | brokerSuspendMaxTimeMillis | 20000 | Long polling, Consumer pull message request suspended for the longest time in the Broker in milliseconds | | consumerTimeoutMillisWhenSuspend | 30000 | Long polling, Consumer pull message request suspend in the Broker over this time value, client think timeout. Unit is milliseconds | | consumerPullTimeoutMillis | 10000 | Not long polling, timeout time of pull message in milliseconds | -| messageModel | BROADCASTING | Message support two mode: cluster consumption and broadcast consumption | +| messageModel | CLUSTERING | Message support two mode: cluster consumption and broadcast consumption | | messageQueueListener | | Listening changing of queue | | offsetStore | | Consumption schedule store | | registerTopics | | Collection of registered topics | From 14b63ee5503671cdd394694f55509aba3ce8241a Mon Sep 17 00:00:00 2001 From: tianliuliu <643422162@qq.com> Date: Thu, 13 Jan 2022 19:15:42 +0800 Subject: [PATCH 096/141] [RIP-28] light message queue(LMQ) (#3694) --- .../rocketmq/broker/BrokerController.java | 16 +- .../broker/BrokerPathConfigHelper.java | 4 + .../LmqPullRequestHoldService.java | 62 ++ .../broker/longpolling/ManyPullRequest.java | 4 + .../longpolling/PullRequestHoldService.java | 8 +- .../broker/offset/ConsumerOffsetManager.java | 6 +- .../offset/LmqConsumerOffsetManager.java | 109 ++++ .../plugin/AbstractPluginMessageStore.java | 539 +++++++++--------- .../processor/AdminBrokerProcessor.java | 7 + .../processor/SendMessageProcessor.java | 4 + .../LmqSubscriptionGroupManager.java | 46 ++ .../broker/topic/LmqTopicConfigManager.java | 49 ++ .../broker/BrokerPathConfigHelperTest.java | 42 ++ .../offset/LmqConsumerOffsetManagerTest.java | 81 +++ .../org/apache/rocketmq/common/MixAll.java | 5 + .../rocketmq/common/message/MessageConst.java | 3 + .../apache/rocketmq/common/MixAllTest.java | 12 + docs/cn/Example_LMQ.md | 85 +++ docs/cn/image/LMQ_1.png | Bin 0 -> 304040 bytes .../org/apache/rocketmq/store/CommitLog.java | 24 + .../apache/rocketmq/store/ConsumeQueue.java | 56 +- .../rocketmq/store/DefaultMessageStore.java | 114 +++- .../apache/rocketmq/store/MessageStore.java | 10 + .../apache/rocketmq/store/MultiDispatch.java | 157 +++++ .../rocketmq/store/PutMessageStatus.java | 1 + .../store/config/MessageStoreConfig.java | 28 + .../store/stats/LmqBrokerStatsManager.java | 120 ++++ .../rocketmq/store/ConsumeQueueTest.java | 139 +++++ .../store/DefaultMessageStoreTest.java | 16 + .../rocketmq/store/MultiDispatchTest.java | 98 ++++ test/pom.xml | 21 + .../test/lmq/benchmark/BenchLmqStore.java | 305 ++++++++++ .../apache/rocketmq/test/util/StatUtil.java | 478 ++++++++++++++++ .../rocketmq/test/lmq/TestBenchLmqStore.java | 100 ++++ 34 files changed, 2466 insertions(+), 283 deletions(-) create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/longpolling/LmqPullRequestHoldService.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManager.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/subscription/LmqSubscriptionGroupManager.java create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/topic/LmqTopicConfigManager.java create mode 100644 broker/src/test/java/org/apache/rocketmq/broker/BrokerPathConfigHelperTest.java create mode 100644 broker/src/test/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManagerTest.java create mode 100644 docs/cn/Example_LMQ.md create mode 100644 docs/cn/image/LMQ_1.png create mode 100644 store/src/main/java/org/apache/rocketmq/store/MultiDispatch.java create mode 100644 store/src/main/java/org/apache/rocketmq/store/stats/LmqBrokerStatsManager.java create mode 100644 store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java create mode 100644 test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java create mode 100644 test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java create mode 100644 test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java 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 1f72e27f673..7bfc618b922 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -47,11 +47,13 @@ import org.apache.rocketmq.broker.filtersrv.FilterServerManager; import org.apache.rocketmq.broker.latency.BrokerFastFailure; import org.apache.rocketmq.broker.latency.BrokerFixedThreadPoolExecutor; +import org.apache.rocketmq.broker.longpolling.LmqPullRequestHoldService; import org.apache.rocketmq.broker.longpolling.NotifyMessageArrivingListener; import org.apache.rocketmq.broker.longpolling.PullRequestHoldService; import org.apache.rocketmq.broker.mqtrace.ConsumeMessageHook; import org.apache.rocketmq.broker.mqtrace.SendMessageHook; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; +import org.apache.rocketmq.broker.offset.LmqConsumerOffsetManager; import org.apache.rocketmq.broker.out.BrokerOuterAPI; import org.apache.rocketmq.broker.plugin.MessageStoreFactory; import org.apache.rocketmq.broker.plugin.MessageStorePluginContext; @@ -64,7 +66,9 @@ import org.apache.rocketmq.broker.processor.ReplyMessageProcessor; import org.apache.rocketmq.broker.processor.SendMessageProcessor; import org.apache.rocketmq.broker.slave.SlaveSynchronize; +import org.apache.rocketmq.broker.subscription.LmqSubscriptionGroupManager; import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; +import org.apache.rocketmq.broker.topic.LmqTopicConfigManager; import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.broker.transaction.AbstractTransactionalMessageCheckListener; import org.apache.rocketmq.broker.transaction.TransactionalMessageCheckService; @@ -106,6 +110,7 @@ import org.apache.rocketmq.store.dledger.DLedgerCommitLog; import org.apache.rocketmq.store.stats.BrokerStats; import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.apache.rocketmq.store.stats.LmqBrokerStatsManager; public class BrokerController { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -180,10 +185,10 @@ public BrokerController( this.nettyServerConfig = nettyServerConfig; this.nettyClientConfig = nettyClientConfig; this.messageStoreConfig = messageStoreConfig; - this.consumerOffsetManager = new ConsumerOffsetManager(this); - this.topicConfigManager = new TopicConfigManager(this); + this.consumerOffsetManager = messageStoreConfig.isEnableLmq() ? new LmqConsumerOffsetManager(this) : new ConsumerOffsetManager(this); + this.topicConfigManager = messageStoreConfig.isEnableLmq() ? new LmqTopicConfigManager(this) : new TopicConfigManager(this); this.pullMessageProcessor = new PullMessageProcessor(this); - this.pullRequestHoldService = new PullRequestHoldService(this); + this.pullRequestHoldService = messageStoreConfig.isEnableLmq() ? new LmqPullRequestHoldService(this) : new PullRequestHoldService(this); this.messageArrivingListener = new NotifyMessageArrivingListener(this.pullRequestHoldService); this.consumerIdsChangeListener = new DefaultConsumerIdsChangeListener(this); this.consumerManager = new ConsumerManager(this.consumerIdsChangeListener); @@ -191,7 +196,7 @@ public BrokerController( this.producerManager = new ProducerManager(); this.clientHousekeepingService = new ClientHousekeepingService(this); this.broker2Client = new Broker2Client(this); - this.subscriptionGroupManager = new SubscriptionGroupManager(this); + this.subscriptionGroupManager = messageStoreConfig.isEnableLmq() ? new LmqSubscriptionGroupManager(this) : new SubscriptionGroupManager(this); this.brokerOuterAPI = new BrokerOuterAPI(nettyClientConfig); this.filterServerManager = new FilterServerManager(this); @@ -207,7 +212,8 @@ public BrokerController( this.heartbeatThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getHeartbeatThreadPoolQueueCapacity()); this.endTransactionThreadPoolQueue = new LinkedBlockingQueue(this.brokerConfig.getEndTransactionPoolQueueCapacity()); - this.brokerStatsManager = new BrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()); + this.brokerStatsManager = messageStoreConfig.isEnableLmq() ? new LmqBrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()) : new BrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()); + this.setStoreHost(new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), this.getNettyServerConfig().getListenPort())); this.brokerFastFailure = new BrokerFastFailure(this); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java index 42c8da9f3fb..321c800b7ce 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerPathConfigHelper.java @@ -39,6 +39,10 @@ public static String getConsumerOffsetPath(final String rootDir) { return rootDir + File.separator + "config" + File.separator + "consumerOffset.json"; } + public static String getLmqConsumerOffsetPath(final String rootDir) { + return rootDir + File.separator + "config" + File.separator + "lmqConsumerOffset.json"; + } + public static String getSubscriptionGroupPath(final String rootDir) { return rootDir + File.separator + "config" + File.separator + "subscriptionGroup.json"; } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/LmqPullRequestHoldService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/LmqPullRequestHoldService.java new file mode 100644 index 00000000000..42b44b6ab37 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/LmqPullRequestHoldService.java @@ -0,0 +1,62 @@ +/* + * 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.longpolling; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; + + +public class LmqPullRequestHoldService extends PullRequestHoldService { + private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); + + public LmqPullRequestHoldService(BrokerController brokerController) { + super(brokerController); + } + + @Override + public String getServiceName() { + return LmqPullRequestHoldService.class.getSimpleName(); + } + + @Override + public void checkHoldRequest() { + for (String key : pullRequestTable.keySet()) { + int idx = key.lastIndexOf(TOPIC_QUEUEID_SEPARATOR); + if (idx <= 0 || idx >= key.length() - 1) { + pullRequestTable.remove(key); + continue; + } + String topic = key.substring(0, idx); + int queueId = Integer.parseInt(key.substring(idx + 1)); + final long offset = brokerController.getMessageStore().getMaxOffsetInQueue(topic, queueId); + try { + this.notifyMessageArriving(topic, queueId, offset); + } catch (Throwable e) { + LOGGER.error("check hold request failed. topic={}, queueId={}", topic, queueId, e); + } + if (MixAll.isLmq(topic)) { + ManyPullRequest mpr = pullRequestTable.get(key); + if (mpr == null || mpr.getPullRequestList() == null || mpr.getPullRequestList().isEmpty()) { + pullRequestTable.remove(key); + } + } + } + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java index d956c223c0e..170dae2939c 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/ManyPullRequest.java @@ -39,4 +39,8 @@ public synchronized List cloneListAndClear() { return null; } + + public ArrayList getPullRequestList() { + return pullRequestList; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java index 866f357d897..85ca9f73b4c 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PullRequestHoldService.java @@ -31,10 +31,10 @@ public class PullRequestHoldService extends ServiceThread { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); - private static final String TOPIC_QUEUEID_SEPARATOR = "@"; - private final BrokerController brokerController; + protected static final String TOPIC_QUEUEID_SEPARATOR = "@"; + protected final BrokerController brokerController; private final SystemClock systemClock = new SystemClock(); - private ConcurrentMap pullRequestTable = + protected ConcurrentMap pullRequestTable = new ConcurrentHashMap(1024); public PullRequestHoldService(final BrokerController brokerController) { @@ -93,7 +93,7 @@ public String getServiceName() { return PullRequestHoldService.class.getSimpleName(); } - private void checkHoldRequest() { + protected void checkHoldRequest() { for (String key : this.pullRequestTable.keySet()) { String[] kArray = key.split(TOPIC_QUEUEID_SEPARATOR); if (2 == kArray.length) { 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 bd057587597..f09522a7bd7 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 @@ -35,12 +35,12 @@ public class ConsumerOffsetManager extends ConfigManager { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); - private static final String TOPIC_GROUP_SEPARATOR = "@"; + protected static final String TOPIC_GROUP_SEPARATOR = "@"; - private ConcurrentMap> offsetTable = + protected ConcurrentMap> offsetTable = new ConcurrentHashMap>(512); - private transient BrokerController brokerController; + protected transient BrokerController brokerController; public ConsumerOffsetManager() { } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManager.java new file mode 100644 index 00000000000..7e5d77425ab --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManager.java @@ -0,0 +1,109 @@ +/* + * 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.broker.BrokerPathConfigHelper; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +public class LmqConsumerOffsetManager extends ConsumerOffsetManager { + private ConcurrentHashMap lmqOffsetTable = new ConcurrentHashMap<>(512); + + public LmqConsumerOffsetManager(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 String configFilePath() { + return BrokerPathConfigHelper.getLmqConsumerOffsetPath(brokerController.getMessageStoreConfig().getStorePathRootDir()); + } + + @Override + public void decode(String jsonString) { + if (jsonString != null) { + LmqConsumerOffsetManager obj = RemotingSerializable.fromJson(jsonString, LmqConsumerOffsetManager.class); + if (obj != null) { + super.offsetTable = obj.offsetTable; + 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/plugin/AbstractPluginMessageStore.java b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java index b95bab62490..62fd1c5d5e4 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/plugin/AbstractPluginMessageStore.java @@ -1,267 +1,272 @@ -/* - * 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.plugin; - -import java.util.HashMap; -import java.util.LinkedList; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import org.apache.rocketmq.common.message.MessageExt; -import org.apache.rocketmq.common.message.MessageExtBatch; -import org.apache.rocketmq.store.CommitLogDispatcher; -import org.apache.rocketmq.store.ConsumeQueue; -import org.apache.rocketmq.store.GetMessageResult; -import org.apache.rocketmq.store.MessageExtBrokerInner; -import org.apache.rocketmq.store.MessageFilter; -import org.apache.rocketmq.store.MessageStore; -import org.apache.rocketmq.store.PutMessageResult; -import org.apache.rocketmq.store.QueryMessageResult; -import org.apache.rocketmq.store.SelectMappedBufferResult; -import org.apache.rocketmq.store.stats.BrokerStatsManager; - -public abstract class AbstractPluginMessageStore implements MessageStore { - protected MessageStore next = null; - protected MessageStorePluginContext context; - - public AbstractPluginMessageStore(MessageStorePluginContext context, MessageStore next) { - this.next = next; - this.context = context; - } - - @Override - public long getEarliestMessageTime() { - return next.getEarliestMessageTime(); - } - - @Override - public long lockTimeMills() { - return next.lockTimeMills(); - } - - @Override - public boolean isOSPageCacheBusy() { - return next.isOSPageCacheBusy(); - } - - @Override - public boolean isTransientStorePoolDeficient() { - return next.isTransientStorePoolDeficient(); - } - - @Override - public boolean load() { - return next.load(); - } - - @Override - public void start() throws Exception { - next.start(); - } - - @Override - public void shutdown() { - next.shutdown(); - } - - @Override - public void destroy() { - next.destroy(); - } - - @Override - public PutMessageResult putMessage(MessageExtBrokerInner msg) { - return next.putMessage(msg); - } - - @Override - public CompletableFuture asyncPutMessage(MessageExtBrokerInner msg) { - return next.asyncPutMessage(msg); - } - - @Override - public CompletableFuture asyncPutMessages(MessageExtBatch messageExtBatch) { - return next.asyncPutMessages(messageExtBatch); - } - - @Override - public GetMessageResult getMessage(String group, String topic, int queueId, long offset, - int maxMsgNums, final MessageFilter messageFilter) { - return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter); - } - - @Override - public long getMaxOffsetInQueue(String topic, int queueId) { - return next.getMaxOffsetInQueue(topic, queueId); - } - - @Override - public long getMinOffsetInQueue(String topic, int queueId) { - return next.getMinOffsetInQueue(topic, queueId); - } - - @Override - public long getCommitLogOffsetInQueue(String topic, int queueId, long consumeQueueOffset) { - return next.getCommitLogOffsetInQueue(topic, queueId, consumeQueueOffset); - } - - @Override - public long getOffsetInQueueByTime(String topic, int queueId, long timestamp) { - return next.getOffsetInQueueByTime(topic, queueId, timestamp); - } - - @Override - public MessageExt lookMessageByOffset(long commitLogOffset) { - return next.lookMessageByOffset(commitLogOffset); - } - - @Override - public SelectMappedBufferResult selectOneMessageByOffset(long commitLogOffset) { - return next.selectOneMessageByOffset(commitLogOffset); - } - - @Override - public SelectMappedBufferResult selectOneMessageByOffset(long commitLogOffset, int msgSize) { - return next.selectOneMessageByOffset(commitLogOffset, msgSize); - } - - @Override - public String getRunningDataInfo() { - return next.getRunningDataInfo(); - } - - @Override - public HashMap getRuntimeInfo() { - return next.getRuntimeInfo(); - } - - @Override - public long getMaxPhyOffset() { - return next.getMaxPhyOffset(); - } - - @Override - public long getMinPhyOffset() { - return next.getMinPhyOffset(); - } - - @Override - public long getEarliestMessageTime(String topic, int queueId) { - return next.getEarliestMessageTime(topic, queueId); - } - - @Override - public long getMessageStoreTimeStamp(String topic, int queueId, long consumeQueueOffset) { - return next.getMessageStoreTimeStamp(topic, queueId, consumeQueueOffset); - } - - @Override - public long getMessageTotalInQueue(String topic, int queueId) { - return next.getMessageTotalInQueue(topic, queueId); - } - - @Override - public SelectMappedBufferResult getCommitLogData(long offset) { - return next.getCommitLogData(offset); - } - - @Override - public boolean appendToCommitLog(long startOffset, byte[] data, int dataStart, int dataLength) { - return next.appendToCommitLog(startOffset, data, dataStart, dataLength); - } - - @Override - public void executeDeleteFilesManually() { - next.executeDeleteFilesManually(); - } - - @Override - public QueryMessageResult queryMessage(String topic, String key, int maxNum, long begin, - long end) { - return next.queryMessage(topic, key, maxNum, begin, end); - } - - @Override - public void updateHaMasterAddress(String newAddr) { - next.updateHaMasterAddress(newAddr); - } - - @Override - public long slaveFallBehindMuch() { - return next.slaveFallBehindMuch(); - } - - @Override - public long now() { - return next.now(); - } - - @Override - public int cleanUnusedTopic(Set topics) { - return next.cleanUnusedTopic(topics); - } - - @Override - public void cleanExpiredConsumerQueue() { - next.cleanExpiredConsumerQueue(); - } - - @Override - public boolean checkInDiskByConsumeOffset(String topic, int queueId, long consumeOffset) { - return next.checkInDiskByConsumeOffset(topic, queueId, consumeOffset); - } - - @Override - public long dispatchBehindBytes() { - return next.dispatchBehindBytes(); - } - - @Override - public long flush() { - return next.flush(); - } - - @Override - public boolean resetWriteOffset(long phyOffset) { - return next.resetWriteOffset(phyOffset); - } - - @Override - public long getConfirmOffset() { - return next.getConfirmOffset(); - } - - @Override - public void setConfirmOffset(long phyOffset) { - next.setConfirmOffset(phyOffset); - } - - @Override - public LinkedList getDispatcherList() { - return next.getDispatcherList(); - } - - @Override - public ConsumeQueue getConsumeQueue(String topic, int queueId) { - return next.getConsumeQueue(topic, queueId); - } - - @Override - public BrokerStatsManager getBrokerStatsManager() { - return next.getBrokerStatsManager(); - }; -} +/* + * 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.plugin; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageExtBatch; +import org.apache.rocketmq.store.CommitLogDispatcher; +import org.apache.rocketmq.store.ConsumeQueue; +import org.apache.rocketmq.store.GetMessageResult; +import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.MessageFilter; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.QueryMessageResult; +import org.apache.rocketmq.store.SelectMappedBufferResult; +import org.apache.rocketmq.store.stats.BrokerStatsManager; + +public abstract class AbstractPluginMessageStore implements MessageStore { + protected MessageStore next = null; + protected MessageStorePluginContext context; + + public AbstractPluginMessageStore(MessageStorePluginContext context, MessageStore next) { + this.next = next; + this.context = context; + } + + @Override + public long getEarliestMessageTime() { + return next.getEarliestMessageTime(); + } + + @Override + public long lockTimeMills() { + return next.lockTimeMills(); + } + + @Override + public boolean isOSPageCacheBusy() { + return next.isOSPageCacheBusy(); + } + + @Override + public boolean isTransientStorePoolDeficient() { + return next.isTransientStorePoolDeficient(); + } + + @Override + public boolean load() { + return next.load(); + } + + @Override + public void start() throws Exception { + next.start(); + } + + @Override + public void shutdown() { + next.shutdown(); + } + + @Override + public void destroy() { + next.destroy(); + } + + @Override + public PutMessageResult putMessage(MessageExtBrokerInner msg) { + return next.putMessage(msg); + } + + @Override + public CompletableFuture asyncPutMessage(MessageExtBrokerInner msg) { + return next.asyncPutMessage(msg); + } + + @Override + public CompletableFuture asyncPutMessages(MessageExtBatch messageExtBatch) { + return next.asyncPutMessages(messageExtBatch); + } + + @Override + public GetMessageResult getMessage(String group, String topic, int queueId, long offset, + int maxMsgNums, final MessageFilter messageFilter) { + return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter); + } + + @Override + public long getMaxOffsetInQueue(String topic, int queueId) { + return next.getMaxOffsetInQueue(topic, queueId); + } + + @Override + public long getMinOffsetInQueue(String topic, int queueId) { + return next.getMinOffsetInQueue(topic, queueId); + } + + @Override + public long getCommitLogOffsetInQueue(String topic, int queueId, long consumeQueueOffset) { + return next.getCommitLogOffsetInQueue(topic, queueId, consumeQueueOffset); + } + + @Override + public long getOffsetInQueueByTime(String topic, int queueId, long timestamp) { + return next.getOffsetInQueueByTime(topic, queueId, timestamp); + } + + @Override + public MessageExt lookMessageByOffset(long commitLogOffset) { + return next.lookMessageByOffset(commitLogOffset); + } + + @Override + public SelectMappedBufferResult selectOneMessageByOffset(long commitLogOffset) { + return next.selectOneMessageByOffset(commitLogOffset); + } + + @Override + public SelectMappedBufferResult selectOneMessageByOffset(long commitLogOffset, int msgSize) { + return next.selectOneMessageByOffset(commitLogOffset, msgSize); + } + + @Override + public String getRunningDataInfo() { + return next.getRunningDataInfo(); + } + + @Override + public HashMap getRuntimeInfo() { + return next.getRuntimeInfo(); + } + + @Override + public long getMaxPhyOffset() { + return next.getMaxPhyOffset(); + } + + @Override + public long getMinPhyOffset() { + return next.getMinPhyOffset(); + } + + @Override + public long getEarliestMessageTime(String topic, int queueId) { + return next.getEarliestMessageTime(topic, queueId); + } + + @Override + public long getMessageStoreTimeStamp(String topic, int queueId, long consumeQueueOffset) { + return next.getMessageStoreTimeStamp(topic, queueId, consumeQueueOffset); + } + + @Override + public long getMessageTotalInQueue(String topic, int queueId) { + return next.getMessageTotalInQueue(topic, queueId); + } + + @Override + public SelectMappedBufferResult getCommitLogData(long offset) { + return next.getCommitLogData(offset); + } + + @Override + public boolean appendToCommitLog(long startOffset, byte[] data, int dataStart, int dataLength) { + return next.appendToCommitLog(startOffset, data, dataStart, dataLength); + } + + @Override + public void executeDeleteFilesManually() { + next.executeDeleteFilesManually(); + } + + @Override + public QueryMessageResult queryMessage(String topic, String key, int maxNum, long begin, + long end) { + return next.queryMessage(topic, key, maxNum, begin, end); + } + + @Override + public void updateHaMasterAddress(String newAddr) { + next.updateHaMasterAddress(newAddr); + } + + @Override + public long slaveFallBehindMuch() { + return next.slaveFallBehindMuch(); + } + + @Override + public long now() { + return next.now(); + } + + @Override + public int cleanUnusedTopic(Set topics) { + return next.cleanUnusedTopic(topics); + } + + @Override + public void cleanExpiredConsumerQueue() { + next.cleanExpiredConsumerQueue(); + } + + @Override + public boolean checkInDiskByConsumeOffset(String topic, int queueId, long consumeOffset) { + return next.checkInDiskByConsumeOffset(topic, queueId, consumeOffset); + } + + @Override + public long dispatchBehindBytes() { + return next.dispatchBehindBytes(); + } + + @Override + public long flush() { + return next.flush(); + } + + @Override + public boolean resetWriteOffset(long phyOffset) { + return next.resetWriteOffset(phyOffset); + } + + @Override + public long getConfirmOffset() { + return next.getConfirmOffset(); + } + + @Override + public void setConfirmOffset(long phyOffset) { + next.setConfirmOffset(phyOffset); + } + + @Override + public LinkedList getDispatcherList() { + return next.getDispatcherList(); + } + + @Override + public ConsumeQueue getConsumeQueue(String topic, int queueId) { + return next.getConsumeQueue(topic, queueId); + } + + @Override + public BrokerStatsManager getBrokerStatsManager() { + return next.getBrokerStatsManager(); + } + + @Override + public void cleanUnusedLmqTopic(String topic) { + next.cleanUnusedLmqTopic(topic); + } +} diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java index 86aab63479c..d7d7b63bbf4 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java @@ -292,6 +292,13 @@ private synchronized RemotingCommand deleteTopic(ChannelHandlerContext ctx, return response; } + if (MixAll.isLmq(topic)) { + this.brokerController.getMessageStore().cleanUnusedLmqTopic(topic); + response.setCode(ResponseCode.SUCCESS); + response.setRemark(null); + return response; + } + this.brokerController.getTopicConfigManager().deleteTopicConfig(topic); this.brokerController.getMessageStore() .cleanUnusedTopic(this.brokerController.getTopicConfigManager().getTopicConfigTable().keySet()); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java index f925364e489..c8ea4d3b5a9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java @@ -514,6 +514,10 @@ private RemotingCommand handlePutMessageResult(PutMessageResult putMessageResult response.setCode(ResponseCode.SYSTEM_ERROR); response.setRemark("[PC_SYNCHRONIZED]broker busy, start flow control for a while"); break; + case LMQ_CONSUME_QUEUE_NUM_EXCEEDED: + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("[LMQ_CONSUME_QUEUE_NUM_EXCEEDED]broker config enableLmq and enableMultiDispatch, lmq consumeQueue num exceed maxLmqConsumeQueueNum config num, default limit 2w."); + break; case UNKNOWN_ERROR: response.setCode(ResponseCode.SYSTEM_ERROR); response.setRemark("UNKNOWN_ERROR"); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/subscription/LmqSubscriptionGroupManager.java b/broker/src/main/java/org/apache/rocketmq/broker/subscription/LmqSubscriptionGroupManager.java new file mode 100644 index 00000000000..635b935b823 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/subscription/LmqSubscriptionGroupManager.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.common.subscription.SubscriptionGroupConfig; + +public class LmqSubscriptionGroupManager extends SubscriptionGroupManager { + + public LmqSubscriptionGroupManager(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/topic/LmqTopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/LmqTopicConfigManager.java new file mode 100644 index 00000000000..d021758b2fd --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/LmqTopicConfigManager.java @@ -0,0 +1,49 @@ +/* + * 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 LmqTopicConfigManager extends TopicConfigManager { + public LmqTopicConfigManager(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); + } + + private TopicConfig simpleLmqTopicConfig(String topic) { + return new TopicConfig(topic, 1, 1, PermName.PERM_READ | PermName.PERM_WRITE); + } + +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/BrokerPathConfigHelperTest.java b/broker/src/test/java/org/apache/rocketmq/broker/BrokerPathConfigHelperTest.java new file mode 100644 index 00000000000..01e7c365928 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/BrokerPathConfigHelperTest.java @@ -0,0 +1,42 @@ +/* + * 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; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class BrokerPathConfigHelperTest { + + @Test + public void testGetLmqConsumerOffsetPath() { + String lmqConsumerOffsetPath = BrokerPathConfigHelper.getLmqConsumerOffsetPath("/home/admin/store"); + assertEquals("/home/admin/store/config/lmqConsumerOffset.json", lmqConsumerOffsetPath); + + + String consumerOffsetPath = BrokerPathConfigHelper.getConsumerOffsetPath("/home/admin/store"); + assertEquals("/home/admin/store/config/consumerOffset.json", consumerOffsetPath); + + String topicConfigPath = BrokerPathConfigHelper.getTopicConfigPath("/home/admin/store"); + assertEquals("/home/admin/store/config/topics.json", topicConfigPath); + + String subscriptionGroupPath = BrokerPathConfigHelper.getSubscriptionGroupPath("/home/admin/store"); + assertEquals("/home/admin/store/config/subscriptionGroup.json", subscriptionGroupPath); + + } +} \ No newline at end of file diff --git a/broker/src/test/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManagerTest.java new file mode 100644 index 00000000000..6ec20c618df --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/offset/LmqConsumerOffsetManagerTest.java @@ -0,0 +1,81 @@ +/* + * 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.Map; + +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.subscription.LmqSubscriptionGroupManager; +import org.apache.rocketmq.broker.topic.LmqTopicConfigManager; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.remoting.netty.NettyClientConfig; +import org.apache.rocketmq.remoting.netty.NettyServerConfig; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.junit.After; +import org.junit.Test; +import org.mockito.Spy; + +import static org.assertj.core.api.Assertions.assertThat; + +public class LmqConsumerOffsetManagerTest { + + @Spy + private BrokerController brokerController = new BrokerController(new BrokerConfig(), new NettyServerConfig(), + new NettyClientConfig(), new MessageStoreConfig()); + + @Test + public void testOffsetManage() { + LmqConsumerOffsetManager lmqConsumerOffsetManager = new LmqConsumerOffsetManager(brokerController); + LmqTopicConfigManager lmqTopicConfigManager = new LmqTopicConfigManager(brokerController); + LmqSubscriptionGroupManager lmqSubscriptionGroupManager = new LmqSubscriptionGroupManager(brokerController); + + String lmqTopicName = "%LMQ%1111"; + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName(lmqTopicName); + lmqTopicConfigManager.updateTopicConfig(topicConfig); + TopicConfig topicConfig1 = lmqTopicConfigManager.selectTopicConfig(lmqTopicName); + assertThat(topicConfig1.getTopicName()).isEqualTo(topicConfig.getTopicName()); + + String lmqGroupName = "%LMQ%GID_test"; + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setGroupName(lmqGroupName); + lmqSubscriptionGroupManager.updateSubscriptionGroupConfig(subscriptionGroupConfig); + SubscriptionGroupConfig subscriptionGroupConfig1 = lmqSubscriptionGroupManager.findSubscriptionGroupConfig( + lmqGroupName); + assertThat(subscriptionGroupConfig1.getGroupName()).isEqualTo(subscriptionGroupConfig.getGroupName()); + + lmqConsumerOffsetManager.commitOffset("127.0.0.1", lmqGroupName, lmqTopicName, 0, 10L); + Map integerLongMap = lmqConsumerOffsetManager.queryOffset(lmqGroupName, lmqTopicName); + assertThat(integerLongMap.get(0)).isEqualTo(10L); + long offset = lmqConsumerOffsetManager.queryOffset(lmqGroupName, lmqTopicName, 0); + assertThat(offset).isEqualTo(10L); + + long offset1 = lmqConsumerOffsetManager.queryOffset(lmqGroupName, lmqTopicName + "test", 0); + assertThat(offset1).isEqualTo(-1L); + } + + @After + public void destroy() { + UtilAll.deleteFile(new File(new MessageStoreConfig().getStorePathRootDir())); + } + +} \ No newline at end of file diff --git a/common/src/main/java/org/apache/rocketmq/common/MixAll.java b/common/src/main/java/org/apache/rocketmq/common/MixAll.java index ec1e1f0245f..c2300d362c8 100644 --- a/common/src/main/java/org/apache/rocketmq/common/MixAll.java +++ b/common/src/main/java/org/apache/rocketmq/common/MixAll.java @@ -83,6 +83,8 @@ public class MixAll { public static final String CID_SYS_RMQ_TRANS = "CID_RMQ_SYS_TRANS"; public static final String ACL_CONF_TOOLS_FILE = "/conf/tools.yml"; public static final String REPLY_MESSAGE_FLAG = "reply"; + public static final String LMQ_PREFIX = "%LMQ%"; + public static final String MULTI_DISPATCH_QUEUE_SPLITTER = ","; private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME); public static String getWSAddr() { @@ -444,4 +446,7 @@ public static String humanReadableByteCount(long bytes, boolean si) { return String.format("%.1f %sB", bytes / Math.pow(unit, exp), pre); } + public static boolean isLmq(String lmqMetaData) { + return lmqMetaData != null && lmqMetaData.startsWith(LMQ_PREFIX); + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java index 5bdc846562d..ba9b7443cbb 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java @@ -52,6 +52,8 @@ public class MessageConst { public static final String PROPERTY_PUSH_REPLY_TIME = "PUSH_REPLY_TIME"; public static final String PROPERTY_CLUSTER = "CLUSTER"; public static final String PROPERTY_MESSAGE_TYPE = "MSG_TYPE"; + public static final String PROPERTY_INNER_MULTI_DISPATCH = "INNER_MULTI_DISPATCH"; + public static final String PROPERTY_INNER_MULTI_QUEUE_OFFSET = "INNER_MULTI_QUEUE_OFFSET"; public static final String KEY_SEPARATOR = " "; @@ -88,5 +90,6 @@ public class MessageConst { STRING_HASH_SET.add(PROPERTY_PUSH_REPLY_TIME); STRING_HASH_SET.add(PROPERTY_CLUSTER); STRING_HASH_SET.add(PROPERTY_MESSAGE_TYPE); + STRING_HASH_SET.add(PROPERTY_INNER_MULTI_QUEUE_OFFSET); } } diff --git a/common/src/test/java/org/apache/rocketmq/common/MixAllTest.java b/common/src/test/java/org/apache/rocketmq/common/MixAllTest.java index 8d86544be69..4f2a341553e 100644 --- a/common/src/test/java/org/apache/rocketmq/common/MixAllTest.java +++ b/common/src/test/java/org/apache/rocketmq/common/MixAllTest.java @@ -95,4 +95,16 @@ public void testGetLocalhostByNetworkInterface() throws Exception { assertThat(MixAll.LOCALHOST).isNotNull(); assertThat(MixAll.getLocalhostByNetworkInterface()).isNotNull(); } + + @Test + public void testIsLmq() { + String testLmq = null; + assertThat(MixAll.isLmq(testLmq)).isFalse(); + testLmq = "lmq"; + assertThat(MixAll.isLmq(testLmq)).isFalse(); + testLmq = "%LMQ%queue123"; + assertThat(MixAll.isLmq(testLmq)).isTrue(); + testLmq = "%LMQ%GID_TEST"; + assertThat(MixAll.isLmq(testLmq)).isTrue(); + } } diff --git a/docs/cn/Example_LMQ.md b/docs/cn/Example_LMQ.md new file mode 100644 index 00000000000..85a3db5005c --- /dev/null +++ b/docs/cn/Example_LMQ.md @@ -0,0 +1,85 @@ +# Light message queue (LMQ) +LMQ采用的读放大的策略,写一份数据,多个LMQ队列分发, +因为存储的成本和效率对用户的体感最明显。写多份不仅加大了存储成本,同时也对性能和数据准确一致性提出了挑战。 + +![](image/LMQ_1.png) + +上图描述的是LMQ的队列存储模型,消息可以来自各个接入场景 +(如服务端的MQ/AMQP,客户端的MQTT),但只会写一份存到commitlog里面,然后分发出多个需求场景的队列索引(ConsumerQueue),如服务端场景(MQ/AMQP)可以按照一级Topic队列进行传统的服务端消费,客户端MQTT场景可以按照MQTT多级Topic(也即 LMQ)进行消费消息。 + +## 一、broker启动配置 + + +broker.conf文件需要增加以下的配置项,开启LMQ开关,这样就可以识别LMQ相关属性的消息,进行原子分发消息到LMQ队列 +```properties +enableLmq = true +enableMultiDispatch = true +``` +## 二、发送消息 +发送消息的时候通过设置 INNER_MULTI_DISPATCH 属性,LMQ queue使用逗号分割,queue前缀必须是 %LMQ%,这样broker就可以识别LMQ queue. +以下代码只是demo伪代码 具体逻辑参照执行即可 +```java +DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); +producer.setNamesrvAddr("name-server1-ip:9876;name-server2-ip:9876"); +producer.start(); + + +/* +* Create a message instance, specifying topic, tag and message body. +*/ +Message msg = new Message("TopicTest" /* Topic */, + "TagA" /* Tag */, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) /* Message body */ + ); +/* +* INNER_MULTI_DISPATCH property and PREFIX must start as "%LMQ%", +* If it is multiple LMQ, need to use “,” split +*/ +message.putUserProperty("INNER_MULTI_DISPATCH", "%LMQ%123,%LMQ%456"); +/* +* Call send message to deliver message to one of brokers. +*/ +SendResult sendResult = producer.send(msg); +``` +## 三、拉取消息 +LMQ queue在每个broker上只有一个queue,也即queueId为0, 指明轻量级的MessageQueue,就可以拉取消息进行消费。 +以下代码只是demo伪代码 具体逻辑参照执行即可 +```java +DefaultMQPullConsumer defaultMQPullConsumer = new DefaultMQPullConsumer(); +defaultMQPullConsumer.setNamesrvAddr("name-server1-ip:9876;name-server2-ip:9876"); +defaultMQPullConsumer.setVipChannelEnabled(false); +defaultMQPullConsumer.setConsumerGroup("CID_RMQ_SYS_LMQ_TEST"); +defaultMQPullConsumer.setInstanceName("CID_RMQ_SYS_LMQ_TEST"); +defaultMQPullConsumer.setRegisterTopics(new HashSet<>(Arrays.asList("TopicTest"))); +defaultMQPullConsumer.setBrokerSuspendMaxTimeMillis(2000); +defaultMQPullConsumer.setConsumerTimeoutMillisWhenSuspend(3000); +defaultMQPullConsumer.start(); + +String brokerName = "set broker Name"; +MessageQueue mq = new MessageQueue("%LMQ%123", brokerName, 0); +defaultMQPullConsumer.getDefaultMQPullConsumerImpl().getRebalanceImpl().getmQClientFactory().updateTopicRouteInfoFromNameServer("TopicTest"); + +Thread.sleep(30000); +Long offset = defaultMQPullConsumer.maxOffset(mq); + +defaultMQPullConsumer.pullBlockIfNotFound( + mq, "*", offset, 32, + new PullCallback() { + @Override + public void onSuccess(PullResult pullResult) { + List list = pullResult.getMsgFoundList(); + if (list == null || list.isEmpty()) { + return; + } + for (MessageExt messageExt : list) { + System.out.println(messageExt); + } + } + @Override + public void onException(Throwable e) { + + } +}); +``` +​ + diff --git a/docs/cn/image/LMQ_1.png b/docs/cn/image/LMQ_1.png new file mode 100644 index 0000000000000000000000000000000000000000..3afd0885f16db3d10ba57811daaf547caec13ee4 GIT binary patch literal 304040 zcmeEuS5%Yj)-6R8L3&fVA{|7f*C0{^qy<6=2nf=~H%XS7&QG2P-_hXJH8jHw<;V=+ljXs0e#Kga%)vhx^9xkZR4w4`ozj7RC0 zIqj|1q!-rl#DV?oO$7naoPg*9{kdDB#oE8_x4W@VvszF*2f2s?%K`- zkAdz{s^j(FeahU)uyc)4MM37ikn>0Cr_!^rJ1klw014si63~Vs{Gwtd&IRol4T?syb@V-`1#qq@p1$ll#W@jUYkx8xJV=!*k$7 zPQI_?je}bc?|Ra0E99_?g-DouWTHV1*t`n=T1)wrE|@-iuQrTvhDZTw82r8hEXFHW z#b(nj;Sl94tCb&)s06?|RW0m~oWtHt`d?HlWq+gf_<0*8|Am>bOzhipsHQ^v`1J~2 zY8Q(W$5-KRtS-bTPj7Nk-G%r&DC7&sm|6<;d^#MB5Tp#NeZ*+;Sx7B%czbja`(oZMQce{@U&QZkgvR-1w%X$=4Kjxbjy+~oBF4^%D?2Q^4o7a2k^bY7uR0eBc ze=;9&W9cgqY^lRC_t+&2BeN6fFx@HPA@}C)hwIFgB!=?%D%X>*l?TM#2`pJ^?K&Yz~ZNOxkw zLTB+~*<9zJPhP8;{rd9080$4l9|b{`PqpPIY_*po%FCeLl>6jXhx zTE>egFPQOqmS*2BZzf473}|2W9ks)6P&u~^6KpxKuqmA%lU%19AvPS-DY{wzfkl5V z2(+(=3O~vDj@a!Al9|hx+Yxz`kQQG*QtMgguiv*wN}8~x((?I!8Xt{K*shVo(Go*> z4fRF;T>&oxFLAv187izjhD;ahC!f6Byt;%WWH&c6Gn;93t}C?0J4nQrZ#>qp0T4Ek zM-#rT3NMMJEA*(@Rlw8BB+THkG$$_#A}s1)d4nH7d8@7E86_$86TW1;dV%0IqwD#B ztLEf9_*Uk;hD3a=-laDk2&G$D`-#be=4VA}@E*Kfn|)v+%Q8!}a(zo)D46~JyXc3M zmfr}G{#=I>b;1Qu>iLuW5G;pu-_KQgUOwkqcN_^R9Q~8pdjTO?$ z8-&U9?cW3*G5okOZ(e4|m>AYo`q_|IKN@Y0H55L}D6!ZuWH_MWp1buWq`rlYIgIa}(Rw&+hxv1$^+DH@oQ`%-i~TR%MpDowA%*mzeXNYa4x=e;e7+QvhKz(j2^-4xy=9+r4h zI4e4xQC=h0T6bBSzYsgDJM1$o4hDnu!A;;C)HiSiim%esJ`L?%h>gqFa?5tHowp?! zw;0zSe>Cnm&LoxQq(YHjFtE28v5MNK*}t(*F@9?!Yuv;!=s;kDZ{zv~|G1C`k5rYE zL)uL#C89G&X|LhNBfie7TDc=`c=nK>4HO(j9lje zwtS&}p^=%~0ENp(gqDWF61U_mSuKU+~?+L}AJh<1wc2#?wIL;4*Hoh2q8{wnP+ z?cOeZDNSQJ{;2kc$rAMv>k`>g!4vDZV{hNRr6-58xm)%7-fFT8Pm%iEo~kK4^~!so zvg603K-+UjRr~roqa?{8nK79O-@Ie0Jx$X0N|_;J7J1)wA^=A31;qH1K&?_;m?{xe zWTl>=Q>6oH9aUbfR-R&y>cKRLqQ^U=hP?AUB0Q?2LV9^pSH(QUyc~F|g zW+r}32ORg>MlwZn4n4p4X@pR_k*12_v@-hIvEX1m{C`;;lPiP>6Mz;UenrOkUk z53GBy$JDy_sK$(A?u1FTb@k|`=BDo<-=RILd(7(?BUTO8lK0XS4CH3wXNlZj-Qt&h z`{?_GOs7pBmw1T7e%x_iE|yqL6A-Ld5yz;MZY zxpC2_G8RJ_Jti~f5(vT*GPjiXVdsA+`}(Q5PplQCP$AcfPN_#J`y2Ib;VKU4)W)~x zkH3j3(w74kWIJMn-?|CVl132=k~xwMQK*w3Zuw9;(WTRwGEB4lzVk?M&hY--$9%3v z3mV0DlKPN~o-zBZl9mfyJYYQqJ*(jy+X7o7_dOo>;Z7ZX<*A6WsJL$;?ZJjl>)~Uk zE(b23=EE$Jkev(F6r}X?Nhg%0Zp&ZYHxDUxNLev|6w@|$NB=O*IG5uVv$5ASb5h$ZmT@)pp2iPry07Bu`tdloP1O-- zV;5HGS3a`<`c!YFQ!ePLC}%ny?~EtD&Ok>VZbnN%NVK^7V0Zzuq<0;pZFRK(br8W*q zL-Wd6iBq%w*?6^to68z?3FlfOb&^`*v>|<^!-Q+&qEZiA$!%2jOHO;M)$VEpSFx0V zT!Zg91OxewIS5|~&leT5Pb@w*;e|Jt(g9J3@k;NP*i%$6%3y@31Yzb})#PwqKsHBK zEvD??P*{fGol%_BD_vb(TIjX$-#bQnMsy{X3z}7_RaH4h%y^cM)K_`duO4uE*rsI{ zyLHsCBze8xbbx!oW@;j85f0*G$g%U~t<~n4F&b2XRY~9UiC@(z{80EHNsOILm#L0P zDQZIlrr~%)nbp?rzH&b^JJZnPzOn$%hS;W+4ZdUbtrAQUxzOP`%h}=g%|}x#C9g_OOohBH&#c!bemfg` z&v~;p^!SXPHtoON33X)ilS@C2-(5A@ito$qqcdY}772(t@h%Z6Dbecww4-N?e~Vm1 zm?m)47(cUsprz*4A%!J+StfxUQvo9hx_pPWd6Mu3Uz@_YdqZmZHeUVjQquR6+r^V` z?(2lZUoFjT71LLL7cXVE(op<WJe|rZDnOUkCK@ z1V0}h5p)E`&Dg(exyIxYiI0CTS-|q<-FO|t++L_nem#Ff^X1-F@#?Qt7G$OL_;kAS zqj-3RHwIQ92O6pvYKFq+$^|&Mu2IV4;s56s@@je48tIk`s(*YJ@{2IIED17qf`8Zn zO8LOqaQlDSBT9L5o@>O#@8?ANZ~o&J{Mn>`ywZO+33p)s-K76U(tp$KznJt7Q2Jls z#?{jQR=59BfPdsS|K&+IR`&lJnasS7Pe9BbC~uI-<8YVU&A9*alZ;w74=~QvpYFcS z%kMg@uwjVzSqAvC5W>VPMWOL=1J9fZkzJRuQ>^6k3Vr5PnMI}HXKh_&LwaLtHIZ`? zd|ze0pkAj(c!DwO5E}j=yXupdJ!4DWpfrC`b6u~A(jT|3CY`MDcp2GpJYr%BctLTr z+Ux^l9NxbYxNqys!4k-&s@&lp)V|*VB`?zRQRpU^^86?b{Q5L1_d|Fk7>(=>UOGJ*!^ zODR^BPl+X^aDYFHN93RvAEKd_x)jWx5U+3j$x6FhP~=ejC*KG$DLNIVY!vcHMt^FT zis?sA?0vW~6Qr9)Xo&(9n8qL-@cS|xsn+=XJS0QfqY$i(Qht-A=9W5^4Y4a%zgM(s z%9Is*=g;Qj{q2Rh2(K00N>VOMKpYtkvdzO`Ou8_I z4%5gxnt7VVY0sOVwFPV_9-5I)%eh@$-rl?>xMo1{GYW|l71xNsQ=Q#!`6+VU4dhwu zo@G_`Ysl7hr1u5DiqxsX03=-y+Lvz)#S1H>uVapleh)kwNw2cJ=qFWFqW}42lcey8 zyZ?O4rX<<28dbv2$VH%BOhgKaObv!DYIF*C!(4ZcZa>^-tB&~UwU8?Z_WdGPj&Y=t zH_yE$6RYhxz@lZ@rD*mF9RZ`&vjsK{XgpZj6$rQwq_Tq2^6#w>G1n=*sChX%d7b0C z=Qrl(8r(_+f?v!5UfwkcTx5kq^BwB}zn}ji?Dcif--(~v_mes1vsua z7WTjvLOSzrXhOOiIRat<3VbcGgPYuKui<>9vO@?yx-DR<2$&lv5@6T8bF!?Dh# zGPI!^JhHJs6D!MNky}5t!Z+Dh=JQBo_uvdFyu)z25Y8kizYvvVRtN@_VO26&&sX zPqW%ztQl8doNbaBS8b$iIQ&ALGBAldj8)oAv~Q3a=Plf&;Te?Z+Y`~i`Xa!{D%Zts zI+}TezGm9%ACKm-uX=U(V_sBXe&}k+tyQ9X-*4irWF=sW)nJvAWNe3B#b>vPxu$|7 z{}#J{;`zp1`PUwlZydt$2spU3F}w!hnIf*%{+@aA3Qt0U5x_&d$lZHdA4OuiQ&?^3 z>}xc@TxHjHe!Lpk)ZZ7%xE%2S)Sq*UPfhal?dJYkA6s$(53Af-@09u?BPa^% zlN&Wc+l_qc`gkuULKE2uae0*fO{FBIVWo{UeRqLdc6Aaa@hw)zDdyWC?B`=n)4RC> z3Be&g*H0>XXu)vF_a-bx4A&Z!uGr+;YJXI?jG#LS{x$NLmghn>gBo*n9EG&HJReuw zrLf-M1r{q-J3%r@C1x8@pA-P8qBVl;!7$)c*_^CV*zwi*dO)0FKH9dZg2erhZm9c#r-JQwta)e^LzaCFV8;KL_y zJe|(aW5NI6p-9}u-PhXzOXF*lw;$n?7I#0Z)f2DT9ag)e3+K$E^WM}+@f_bs+2kw` zL42rkz%5?O;QUm)NjO!k;8R(HBx)&lXp}i%cR|OVah?WhwBTv;gbA5&U$tkK{2t;f zS1bckLXFNOr0cy3Z|mUkgNUNy71bV>U!Olu*ZgkkW!mhKN^7r@9ug?&*tk4c=rB8} z{>`wQ}=nGHtVC{48#(|;bq$WbMc;wODt240mCZ!3>7dB&#z!u`0@o`gp z+Pok;w^W%L5~|QSGM^QltWXn99fajuXSrQi#`!$JFWXkcW~jZJMoadhC{&9zpzS$* zN7SQ5_95i2`(Y_q?`b>xoX35;k0R=m$&*@=-*+!E-pPAd)-Om%XU3~BY{Jd0|HL%6 zj_6fqUc7ZwRMcYXi2bdRsp_d3zAjCZfg0N>qk#R;W-)zIuWQ3B_j2wFVfjDUpMSzXLoDNjixsO`?9l7UXed#+-;2!t+iW0us%={Z)B@C#O?zRr^(0JxgBi zO6rW+co=!r$$q5RihVRZWN;8&6Aj@+ zzPl?)xAD^OgPU}7qMqF@RASWgRAuTyNXY%NMHyUzUVmi=l|fJ4typzT6`OJC8JTbf zQ+a8=5O(Pr%Gd^6oPM&eTh4T1GK^Pi3V)V^iMQs8YT{ph$jn-=cty&{o8QM#8|66> zS4v{`*F4wOHT@e^jO|)Ojg5lRp|l;|a)ar=3YjCO_oMm!edk$}25mFkH@~u8d`ReE z7M_BP=5yIp$k>h8SWl=uaahpIlDPk%JbF`c)L{f0U-bNGReTWdUNT@*5Oct5WVkG^ zw%h6CTV2YvQ6ENtaDvcno)$KA7l@EZyT> zd1};bx>xFO$ht*w766#3;Z8Bx`{2g*DZ|y`%ZSHXE%-&P9_uKEF+b;}QHldc)WD$M zvRYJk*e4M*JmvlAN|+kN;bKx5zgOt5TOmQo7P5?cHBN3_7FJ~I0AKz{sEOCJf*R)( zxlgp1=<0kcbGnmk7@r^_xW>0z1|A%*a1(ftFMct%$yChm=_KdT8-BTrZuS|dMYMhG z@8(HWWy10lBaHqRFd!IIYtC`2Yrv47lQ}$NecBau*>JphdE9()_|w&5F>LwExw=*) zldf^Tg*v>=?c=UyR;6W;Tz(u#Y(-V~i?UsIr{dFM_ncM<4W=}0Ik2@#|9~y=mo)k` znknvwRnLF`0@2ZJ32A$&4$+>hcvB;zlqxqSoxEi;-giZLI=URCQo>#7FT7@6JKE2m z&~VKeG%kpzANRzRzSOtsc#x0_Q~KZB1%3pZKwV{ zN8Em2yx3no8 z-EHH)6_Ru@QX&gCQdX<_RS_BHpI4Pkbg1F zlge&PUvDUZ_B)ie-4S+E3z-;tMG?(hLumbuMyie$LUknWha0vRjx%?>S_ii(YqS8k6}$zqxrB?-w3srwE2UtCWSkyGojyOUQ3 z&1JPsShi|}j_0s7Zgy#<*hL5If66$;RBzhS>ClJp)m4~h-AFU)&Z{RM`AP>u+2#F* zy($gBm8SEF((Cxw5%zW-Y;h{NCS-RbdaqQ4krA2(TDIrl3xq~01V>>>p{>*D^hw=!}BZ$2&PJKweu zb`hEG2FgBC*AgEZcB*EoQ^}0_J+I=(5PNwQ2WNlXOyHa1*{rCCYZfft)8_K#(TTs0 z642GBTDzkZfURbmpg{`cN82&G4LX3ghV7N8xv?pm_Y65|7_n>j1l9e1>F~kUnRwA% z**PPu;w+r?^W%3IWU8tTax=Ep&lW=g{>vw6s)8eX2$4Cp)M4NY?d~G#UPpW9ET&*1 zRxyk}GuxHWM4=owi)k$^Bf}w6t~_frUr$fvNdGR=<-+A;qg9>fepn3jq#|O6PT1w~ z)TW|`)tx*i{L#XgsQUyrKsS$~@I|}Ts6Nf{SQ#ATvGb`OJnyr%Vie`Bs&ch5;Mep! zLQdoZ4gvtuvP`SpT_hptN+0u$3M?t=7#~uZkjZBeUn$>kRoUgvpM89RKQ)V-;++?9 zQW>CQyBybYnuv`mf#M>giCeXrH^^2jLh;ZqBEfjrzV}xZTYk}RU~vMAHF`JhTT~WAxzbkw`n#I^{Zp!%3 z^dhTom6~r(gbzs8&%a+&3BlCUAFUe0O0OhZ^38Z`X;}NxNIZfba3+uI_vcL2N|*k< zhX?fY1;x_MLk>Ub>sUL=40 zuC9j)8L!A)V1CCy6$<3A=WEU}!{Y}zQulhtwTi{His0D|V$3|OcYcKQ4r ziqz57|80qMPm*Bb(dFtE>^2mCSDyj9HFT~g^#NVkEHVA$#P(Z!)4nagMt5GDEMYxq zH-grTH*xSmr9Nn1G*wK~X93!^FfJ6ZaXNbO>QlOV!tj3HhI@U>h4gi>&+i_vwZc_w z3@^wZw>W>YL|n`pk@6(g&CO?IM0G?uq57g~+>a6aUt`}~3IG;b1)jatC1@@dCF6ts zN+Mmmo#kL4$<+tdp%+kq*@+IHr; zgrp1(QYD%Bv*qh@ZG5`nbU)$QMc_N3KG(rkbl#p;aU(~D(wd`R-gx`D_6C@GeNr}A zeIHcEX`Zz&yzphJ)o9PNjr5A##06h6cZ->C-~22Qu5jA~^-koNIgBTars~>9M`H$- zSW?@$F;U9B*E{0F`DhJ!>PxG8YfJ=cz(3#TxUcs%6iu+;?1KS=NliL~aa%-XKSzm! z)7!0eBO4diwMT|K?&-_e_M>hTwhT1kx!(A@z!+5qL`<&~$uZfZwm2iM3^(?E8ul8D z>bBW6O2Y#Jg8*#4W?VXUJJc&&MkpMlf*h;0u@Ukj-b(bYLw}*JCizlvyDLvGPEX?D zqAv8dQ65pfU2Ypc)4NewTQ-+G#9giI==bKFSP93fiC+fIC)ijmqej!IIQO~CE00nl zo&fdq*KNl?5c`yK*5wH?TlX_BrwZO^2$VkWV4E*b7YWWOE*F5qMw)&1RO*@dV~pJ9 z4$5`ZxrQ1>^cG?#9P#HRv>f9>z;8--lI|kLoVU#q30=d3LZNU@C#x3t|DFCYEu zmjS0{^yy8bipX*w%?fuXBXv)zqm=S55Gxee(Px|yupC+MRdRXJCth7{YXuwy<>;x$ z)bS}`hl7P)M2$ZmW&IC~{s+z2){T-q$j+bs`t7^;>g2SNxv)(_LuD)OwRgd`^NGE7 zja3jdysJwU)LAYNJ>)$#NW=!ROnr2>lnp;mzs}nICepA>VRUrL3)#F8{es@`KdDrF zQk+VCNJmE{KikQ$9jKO5tQHk?Q%EEkZfb80-H&oU;}Ob+(eX3&TI(cwo5Zm>CH9eJ zlpm?})su*a@3o4l$lX+V-FeMb_NE|+FC|(dvc#xfOl?jhRg9CdO4O^Dx!iAn@=AJ; z!zce*6mF9^kMmK32&TjvC35xa7BhgnQ!N@L{yPyqOqRV}3H-DGcPW;sJ0*IYQ};@f z%(3D&xZ8YamdO)dlW1*~Rj^hewbf(a|kOGb&S@ zApbye>rH)xV6z&n0;ei%LGNe%g6Z43rU}CsRpgSE$KdUhK~{}dqNmD6oMmJ=FBQx+ zrn zgcp)7@FU$A?X_1SFRTs%{*%sN#~~@9MK(17F>@S2*=|1mC*|+9gSPxpFw83!L4NzO zDqKj~K7Wqn`J~r}>Tb)gs^y2rKA zHqX7;x?L`KMdY+RrdsjOu6iDw_*D@v$co=Q<_S1-Iu2GuYs9pwkJC zjV-PAo}_VZzH)D1A8EZBB+A=VxdIv82u{`)6fkO7E4s=|lsErHyzWKZR!%i5kE_Oa zh}VS^A+1yfYdwZ839=EL7nXYv4X@PT&X+-~L=R%y4>&+tb%T0>wnudT0mPVVaJEE{ zu?`LXwP=Z;dot1_7}eb5!yq_=2&M_)K>xmF6|-R zA%$0YOxNjaNj~03dfByw=vf(tKRwh?tj8WvToDd9+V=uC*i>tV&kP{w(QNrz&_oP* z{mz{3APUxPx~skK*tCu(IeSiQn+Zzy60z25O@NPmEk0mz2EEWhxA~7}-@Xzq7WrC) zixbyAFV|4X%YU`JF5}l3?%**>9}p2_pY_!SIdl#oAMBB_Mapc~W)HU$h^#BYBP9Cv8+$DH>1H7?ZN@v!qadC#I z9Tm9SPxnkW1Qd7O^w?f0!7mFWw5v#Z)k^WN9sHr`0Zbz}w~p$8wlbq0+^d%I_PYgG zCB9x*$W1HiBy437I{OFYn@&2dU6ulin@&44eFgF3qeRS*44keE8Ll2xR?evUS=nl9 zM~O6^*5K9`dOhD*B0?sE-)|k9TCFRa17Df8Uij|X$_(XY*QrupO6 zcy8F0V;MUOgnYicQxX^@t{y}}GKW@e(?c8jo2eR1Lf}J}$VbxB6U|>Cpqk!!lgUD) zUhn9MNtAuocu1ls^faaK#}7U1DMO{q>%mPCDCx8{2s!2CQ@hkO1WBMjfK4`sUy&*1 zuddaAW;?&Akzf03XM(KmW)7=RP$PJ~$ajE^rUqdYUVBVRkPXM`se@i;|Gj0v<#E17 zs^{h;wCyJMmp5}^3@_XVXwtXeS>uKpr$6l8b)1Xg)U_S9713$2tD16$e0-ZH6RB>!wogtySepBkFGKS+;5U* zkd*?wQHic@!XljmPA5nfxVX6d*Q)*1(+-(?Vg_Qs*uzn|#lM3}3zZfF(AIR@@@*8(YH65~>s1m;oWc4&=S;x(+zmpF%q7R4Fll%WBHS4D=#<55A zc-*1dBI9EanvJ-G1XCfNU0VebO|Y%Qz!9uFA5?fyz>EYibY}P^;M&$(=anA9i2};M zXPm?!PO3-m3qQO3*L0_$GJzbAT2^#63rkg~RCpS$>p0caw^dpIB6%ais|94wCF(0{ zXaZ|*?@b>z9rl=Al=Im>{ItjW^rT(9X{X3WcC7<&QHEK`6Wc#X5VMQ95a34+>VtF{ zQ}gbc;BH%=$T3^hzA;$5zRx7pCl%b!7n51#GdSAo^l9YvL{s2iHyZz?lqmsPdA z2mKDGmP5dZWEHJ!;O}|1%cF`uxzj(9e||6_n!CDYc3D?fpr}@``cryKk2cpTq3~kL zYv#ORS=QOxyN)DkcwF7=1h}YLS2A1H!HQ_rS{p6791S=hZN5BhHh9~Fl-jQC^V_T( z(yTw*?P;{?MIT+1a|#&{&6F&$9ktw;KB%f*QtpX8gKqi;fdg8xAUGqt%-H^QP17Wkf89*#ctCf|W4PPUy)3n9(1-!4AsbADL~~UM zv)udSxd$>4DzX@zl;%y3k|A3-?m(hOP28dk5Y${m3R}L*(jJ8J3H%~JT@@COVTR_T z?;SErkJ2CVih#>3zH_i0KMOGoI0_9onqz1-B&@2e#URRY$=5JT)O^;{T$IErWcPA* z0SOuy#+|Xtl!Lb5kv?C*BEQ-3abJLcPokidRO6TAWNrT?dxRsy%=eaaAkOvte<6XH zmtc}_`J}Vgk=pjyJcPOz_OILIR1%!hxH4jVo5DutWyUCejU^di~y1=O&}5LL$M- zL80@(82^0^D|5(j4 zH}t5{h_8dgdN_Zv#Nnh{{Cj}hh!Bi%zI^{)z=OA z)g`8|rW5TAr??lTR|pJ=>_4cmGNiKnZ&WZPkQ-5>!l6Q*1Y_Agy2Kyt#uD;eE!Dk8 zF04=ifgb1P#3b!z_Au2RjN1dwSGD+kfkcbSNv7VbI~{B#^diySZ|~7T2FLko2Ae9+ z>YGpN!MEAiMx*tN`uc%yn+`AhutEqU$LySbZ9YeAxIu%$+iL%ViVV9aDp_%c<2l~N z8v7=Q3o@g>GX4J^c}fh!v9rgL1I%3iinSc}*`QA|5Uq~At;>t86n^~OvDFRdfcj-w zc)s+7AhijvJiO{;)2kVwOsl~wGPti2u&9qxcJ2Z8_#KVr8{E>66HF)1gk~nifz(SkP?=;`7&1XRIP;%zv7!g( zleYY(>*pR$Ni3$QY_8de+pIVm5S&1zFX|olk2YU;aDt>p`7%;cyt~v*vpdy<8iQlL zG1}%qKmxhOXRDjO@cc4un{?>XQ2K#LUtVRcnG#t0ExH_XGLNE_klljz?H!;tSiB~6 z93@^C@C|FCqrILdV1ehO!4`bDjW0h~+%S-xG_#MtV#Xqe8z)}9m*YA94U;x&Kacuj_LAH zQ?P!o+fT`k`rN|RL}RCca%zEuLuXW9`TGcH7#v2}#DF#FfecX~8G%)ORSj44HI+?q z{C7d?(G47OP@q~`o=_8#vYGV#aiL-l_=7k)3oyc@5u`_wjfc}djVTJ56+flaL&AMU z3XK?7H8Ro^#R;7OyY*Fzw@^_hd}*iL07{ zppXks-(S-Cn((GjoAkYOUt1=*O1jG+X908AAq%63UPyS14$Ic zyR{Xp0cSg4?J&=R6R&A#!PfZbrgQO&H;_D%aqU$5!SX6l@S@P+FYHB1_7&bNZkQk# zK_#8VO#ZKs=Xfv+{pgw|>vO<^9SFWIxtff9!$;vSV!-kp0Imm+vRUMz)S7X$#9)Z4 zgVm?VxtQAOS$NYvuxA>4`GJQHe!jEgVzzzU9>oMwNx-_!=7J#lj^yj9BPV4?U^CV+ z@{lVwq>yC&@NI&Qta=@)nhWO z(Mluql`Mq?h1?jMl|%nI)7Kef6x)k`%tlu8x48HG@ud93WLlC4QyC541d;(a@Neo3 zpG@yhtU+mWlfZefI^B#!hMTdK`lled1txet{l2TUwROhDn_3bC(TskoZ5E@%qRV)9 z0zYVoe|<<3JLqGG?Pc)YWP%j?!J632QFlqNxEEL4G%c9i=CkF$V$yv08tr2ZO-)Rp z%)E}1`;dnCarw-O7Q%OOu!Z9=B|KuJ_{G}B8{1092{Gg@S;9kT795&9%=Xw$Xngeq zyM}H`vVRGIsJYERX8n1uh;vFq6I`IXu>FskYU#hpV+^#dwC-ix`pkMKuT|B!%0cIN zeZ5E@X3_&n7um*>F$GHHE$cynd1M|X0MR^0)Cj{#HCDqnfp4wLFjd@nBv|aQ(gBM> zH?3VD4X;2-N{QEhtd;8QeSJJ-cMM%lTb9LbFD^wk$5A~ha>otuhSKVE0L&z8l(*nZ zHW4Zl>A(G!&DQ8CEZ)G5CqYebGZpb|Z6z1(wAV;2tfr(4VAXmbX27w=R7XJ-% z9NArWT63BmAcLBrwDRIvF~ zy@5BBf9ClJqBQ=!JG0xwJj%ef1=WxDvI}s^>e`sjjd$LB@_B?K?h{o^N9X{gBtzhbzQ? zFlzIfVV+sKo*hjyP3CCBQi^?X9fnVkpIbNN51$Ux^36Udq}HmX|0RvMDh>{CRR#sP z(~YxaYKh(1%qsRn5iu1vG(#bMIAwl!qrL@rYAe)4I+LctU@cXPs8CD4 zn5DRA(d94MNx@xrw?}BSR{PrGYJ4|JJ4E#mSBbXHr?-$MwYxcN=Y`5vuzaVJ!#@-P zU0GBMkvk}hX~apf%31OIdb;Q(c;+gp*-=|XI#z(JWfwPnfzvHIi*Tb})a=vN%1l*J zzuiS`F@sj0mDE$sMqyBr-jh#NyS2_guYzG&TsjR%7cf)(8;1EaBh|X1l;u{5J8&r# zFqAr7=`Sznl!Ro(6|bxF}7;A+erV2P@eMfTDJma zD?22n;e5jxH-Zo%Hp<2KtIRT#)ewuL*wecAK-z)1Ad#4Ho&kTo$+h{dHu9yhGO-79^ z%l+#g6hVPKpSuB?og<-?NyutBzU)OHkckWXg9NR9^N0&b|UhBJX|EzUHsXPBOgTN@JWQD7KA?Di=g(?K0mv> zsGHzsJ+E9gDX*upg2}RTkLbT#S7J?qaHD+OmB4&+JY@tvbW1;Q&p^K5@`Nht|Cp98G^A|GVyG(cY664@pAf@$8;|ysmE7Isqiuf6I0@`>W5t z>bef^@W5e-#a*->6eY~5#YzNI{2dGT!x8!cL452W0*0c%!C^_j`5G#VD*^G1LC?(N zEU$X27h=@Q7cyvz{G)-WlGGZWU|Z{#CG}AZqKlkro$kaL7yAWf``HZ8)9HMn!z8__ zZIbRX+6uvoYeFSgHGd_jiH?repUK$o-}y_V1ugPt-m>%cHF1fiW*0m){7< zB506+pw-}<7JwxbCI9U#&Gr5$X6z4FlYhZ~WM<&w`-|cvp)%_Od z1{4Kp&n{qPId_x%Zvcd5Dr{m{BSmFp{YGDS@$v&XY%=_++AWWD%NwFC;9kwAQ{B}v zOhbzWBk^zic0Xk_><_9pMtd19ro^kmw2`nrOalg?D1^!+e-pdp0Y~bY*`MbnqCc2= z_x$+#lf<}OER34I>!SGY^8FA_0CJco?S3QkeIQsZ9`E=zFUv)p$6CBFlrguB@5W#- zi74n!0=TgJUNdRN4RuRPORYC;G`_!N{fq>0=9A9uez8}h`XTqQ27z<)`2ie9C~N?y zMij-kN}g27pfMc{y)oG&MhbyRV_9OYtHEQ5C#M$bj(@>6z$Xi3fpt-yxx;fpu5aHxi89vPj<`2x|5<*S|Pzk9JGP1*YYk zg4eg0P5xX6%W(U_e8>rDB>bX{$Z^fl8wD}G%*wA7dQ+Q-5Nk9c!`z8I@71s0O96v4 znxLR3i|9?blmPofcY?gljVz(44~Is%TReWN6VJ_>nDrq@|F!iIlQSM~Y(1G=X43hD zz+}G<$m2)Tu_I4&uLtSdhX$YhS@Z$Y2cUN)DQHX(r zAglV^2ltI07cL>lI(<*hv96yv{jHllo0~+IVoEaxHEO=0O>U0GNsTOPbGwVCPf+>w z=Ap9Uz1Yc;C(b}%6Z8-OLY8Lr`1C@NebV-5G4fw&eH))Sag@;94mU`f*q)&O@31t7 z{^4ioxmgA{pDzGcW!>h&xG~i$XoP`%v8&(p0wVN)&|uHkw>f|DdI=eV6WyC3vKK=t zvfCNbXbSerxP^eAERbn2AZ+`DRU69Uh#*QRj2_oJ%1weV1srD)0c4QqTr1RS0!TyX zyhN&Uuih2tA*|@S)S7%ewjVfw_l6tRlPJe}q(s>gJ9TuN}mxyn{0xx-Uo#lY!pO$iD?ei`o07Ho<1X`l&A3}1b3s3XOz+JLtgC#=s_`pWn7b}mkU9U>)7 zH*jZUTA+|LFEVIA-82WL9qr~p8w8{_6^&yGmQLW`!iag2}f z!+*)}3w(onQoTW!J2p-5n}}DB0fE|7PVg z@CpSgquFKTUr&uEJc!5)@N{ z+0V4t$jh`rl=LE)&Vo`K<6C~fp9~ih(x&|6B6b!V?m-SPY3S(v3I6^@@9%^N zo<8vInJVn*#_~*-HuAj_w*gpHsR(ka5&F4)nw~69gzOIKH{LQwBTQx2bS!dU4cw~h z#>YYL3$-IKu)eQLjW0qOrZ3?}e_HUkM{01&8)(X?^2ftBj3zkoaFk>{D3`Iml0vTw>q(Qk8hdZt-Aqjm1*Cdxox zXZKEAT1c#>`N38N0Cf1`z}xEX2d|zA_C}o*mZvk05T< zow6?XN*n2|{XP;h8wc979Q3bQIKSPrj~Icw4I4AqWhNic$tx^a^EW6yNz)R$*1NXi zLA~hTROuDY6t1*C-@lmPsom_>2DpRI2LgPYz>Wn)r{&EcvtmsHujZ+PzxvqQ zl(OT4A+yvR@2snZs5e6t!&&*r0`E|XW8IE3IUCA#j;P?T%e1S z(dH)T-cnAC+{s7v2a>C~pN2E}-bChF;YwRCsz@VVX=AKx9@m@N#dUrP7ICwWu(c7S zt~40I1~oNWoVmO>6nB~|l!a>}q0%_VQq+Fr3v}soT}AQuaG#pvTrWI@Uc>-(w=9o| zcwH-#`0(2v>bE!OKILH0=H%!*J@!|G&>5vE2A0B<0=6bcXXi4DTj&&nvXUTE)Wzii zH8KBMzFAe_w#tk)YxYWB9>-lI@EfoMjhMh+G1fxpm5o~wn-vjdkW7)6R_3pFnL^rG z*KHn1PD$GM^?2*mMl|lRqa3efh0B||#ce;y zt;9*(;`+t4Q?rQRRcBE4L^z$k=Hcj=^!*>Y-YP22K6=)~A$Wp&fZ*=#?(Xgm!QI`1TX1)G_u%dp zf;TRW%QWBo&zZB%T64h-*S&sw@2aQj9fa&w6+&KCllidKb3k*DD5aVfmKZeOs+!;? zD(M;TF={T+2>$qd$z?uu0+*I15CaS1PVu4dY(|`T&&+%#bF`m4QF`6h9j0!s{z&&e zS19(z+w==W>Wrx3bAKM-eac@v=5N#dpVItaTKy|6=)K^IbfsTZq>G3|I`(!xoxk4a z%3jcY8m(HRI(BL)BvAV4V)`D75GfPz%>jU*%gQWGuLP#Kn!g)JWKmI-YyWOC&G9s! z)^i8?Q>Ck%UbD!3TgZJ^1*(nWt=wQ}=w82oY(Q;Af{(GX_VsJ)9-noq((Y%FN3w^v zbBv*?(hV~G*3-x@!_pWf3@xDt!k^si!l{PX$7}AT;0CyT4YX|MtGOO>1A7H1p))JL zwp3JD#cS)ciXPinTKl1J_uA=MDk?=4W0#AgpU~Lo5UT96(@XaL@i;TxszM$@saEVT z5n4R6Wo>Njc31E=S77H+aWm&(@#bmt)}hW=8jvkzTd<#6-;j47N*pye!){hBo00Cd z)znb34oT^G7__Gob@)N&yYfKP0IxPbn-v$t39Su3fsRrh$S5by{(F&P;aG0C$=xF- zr=q#F_EK%dzjG`nu{sW-l+nuKxptkNRkDK8eM#MIqB{2;mG;yZJw8?e;Hh7JQ$M2V z>{GRdU#qpeWQpG`eiOa4^rG@Q5xwxq!E766jv9;$68MU_;MDxeKO z%7lMauCI2nn-5KHWl(Ds`(^u4j(gW00mw@Bf13k&j$AXKB}s+a$GQq>$-=+<8#z>% zN?+K}8IR{Gn)JET#RQIv3?DRIvGEt*qf` zDYlM34PpA@f4gVHmSXoRN(y{h>1=e9pH14)%zFV_tW^908&@%Zw9n}^VYi*>(>gsY| zlJ%coB7sZBY-C#My_Wa>G|MbT zAU0)hgiX<&xwu-HXI~wbJjQ@{DtG*Aif^|@HnE9jl8?Y`IE;#EATC|{ID(5{rLz|<5pgoD0+JMQ5r6Jv@o=!n9n zrK`Kd6eGDu+8tBJcrDY0QxyJ@Z=2t0+*cpw3tZNDE@tbBWP<#ZU;>a+^Z(rzY^Db#g#?KO`z z2OMkjF)BFx*!|N*wA>N)(BedAo0(iDu}S8mwHB>>h=k^4S>xF>FsVfm)AWE^ zn$}U-r^sLKbZfZ6QZ<$DUwIYPM5Oc_f(OzlCq=j-EN7aStz&2X{;l0?ZhSB|g+W81 zUK2Hvi z`S1AsYJ7X6A7YyefBd!RdN!@rE~|n1xU)+}&M^58L;@KnAiMZ$0%r1$Pr>DNh=M<# z-gEI&!8%^{|Mc32hC&F3JV0)W-?D1gU&)TFb~H;|^~ZJK2%VD%Xf zJKLXYp<7#)JLl{CXY01IELW|!@o=%azTK&Fl6(<_C60mtU&y;MmwmGwrZ}qHqC2JY zm>A}3iLbgEf6jpTHdmsSYIGezP>EF4G$1(3QB{7n9i&jXsQC>>baz^)z4i9CrEZC~K{CX?m9p$_vl35%zAkQL zqEp&zkG&1kJv1m(J=X)YFZl?^#OT$sK-X9^uc$E6F+$aR!1RYPCE%z*Z79~By_eo( zjgnsK9yM!Xw-RH~nb3+}U)R~IR%%!(kR(VY_-Xq|US2g(_Cv@4d?5uYB_7;da z2##r`JQE#vXiYrPsIA>7qL%9_xGSuZT-K(T^7h7CW%~Js(WZdxIOVZ<`p=E&|DHO)`@wm$_fq+?1BfEMno4NpwiF%IYPu^Ov|qwFx^_VhM$b1|y9$%1 z_>V^vF&lvYQX+sCO0U{&6aoT8v+p+6m%UMw1PU*oS5$VKm=9?i%_~PZ^8}r z@E78gxpb1=*c>}MyyIsSHs`IM3PzDI=OGxO>|?a+a2b&YU%cHew!6e05VAA1daVXW z&UH9Wg;XNLWDmT;lqLF?JiZl+sn1x9j(628j*q-r{#DqniPe%Ac+9WRAl_2kduCrs zu&hg9Kx^?=hCE7olsHA9iAQW#2o6ca_k}^KUYsvcqxcsKZAqc3}e%#@$(Q-MQS!ND> zuWoL%M!++{)OUd-_B=6lbBHSAU&m(N5@-Ca>Akj9N9gJxR<=S+lUzZ_O0E@)L^Dn+ zrES$tx?H!W)mbgl(avAoOY>*1XgeK5!pGF|dKn)8M)t=| zqO1ZM9Zt=LOBkWR`C92fXwXsj_t$5-!butT|2DG?rJ$1ELT2#K)tlb68XIz{_dIX2 zcDlTP)*q5eY-}e+sui_Rrws4^7*uy|i^xv23}d+47jyQ7bx=0X;^LM5 zlTg%K)h1ZW8=-B~Sv^T`QCCobv!(6*Q#uG)w)lHKly*M*xW#U=)n2w?`N9sbb<=&O z#xCo2&5F-iYAr;2HUA-`D_xb`$i$dinmAdxXk4ekt>SK28Dp!0{+c6>vwB0NwQACQ zt7k27$?Nw^QC+ul^YQCrsQ+ zjP=$fenv^1Y2W}rEng`juBw1sp>d!9$%QK4q>1CINu=LUXZ=-y*It=BB4#Q@I2F-bz&ZDRj7owa9mZWUdv~G z&NQkS*L?rm(pGrT_MCZ3(|0@1l>U7{eRx97UFQx$Z z{{|eG(G@ih^5f#R!s$=U13n?Kw_^Tz-A=b^joD)wF$oax!jwo|Ci)=`_>gp`!ftjy zP+QV(%FBM<28z;{S!q}Ly^q(vMqLCKkLs2OEx(^x2zXCB=M?VEHy}Y93jh7{P0{D8 zQ>4Ci@yz33EC+^th-6`k6>Id?U%9n7wWd0fg$MDeoa`bdH9DFGCQVFNtJM}Gl|m9aZP(ZoYMtf=7K{V0Z)(ae@Ui(+==_vT4Ow=84Z7Aqt=%4b zctKt|r42LN@|3M<70-~Kg+m|j!9LZ3X-&^`PoycJwB99*MvLvjfWy2eSx2ybnaM40 z5kEVaBvZ=Po;*^8&UOz*#WcT2X47`&=@cIcbJ_O3NcuqKtk&gCmuIat<>C|=A7Z66 zmOC1zn{)5>*56 zqhcY|YaN=YQ$*plLBgAit=PUpsL-CjtOE^bu0D-k-IgB9)9F`#z5U3~GjV-rIkj#Q zmt_SqLT`=EDY@QuPq`hd9si9v+R-i*HZIo0lQfo?8HmTK7S^2!S?(?e<>;A-)qK?e zr0Z?%{#i@`D4WA`q zJ$+hd4E|3O&3~m*{8uZgo7A-u#+%Rb6H?Rc=eGxss-7pC@wBWAwmpm0weMZzy61n= zm^n}@X|P?4qf8lCPlxn<{f=r(>{}$j;TlXGitwzCgcEYHR3a*k5_@YZf^j#`oUX3g zQdj_LQZ8KHwsccyGC8e5df0x6_GWijuVdb+zJx={n12`*rI3lzeF#`kFYhttvlvSi z0oBP^Q97lMtz*uv>H$DnNy)D6v$;l;ZLkL|SiRJsU2;%Jo-o^BH9)xcjY)D#NRn88 zp0R9JONW5Hek-&il^Y0SYPE)YDYpf`-1F|x`mt~hZl|vzt1M^45o*TDFC~Q_e1buH zztiJ35R2JuPscjfn_b}4jkufoV6c~UrXR{(Vcn5Su+9Y!V?7M#l@&IH-V%Oa&D0-e zmJ;$PM-C70u|M^_*=vp%zjG1jv8>|8cIdyLFI!MYVM?^WDrFp_CpL3$QMF+OZab{V zm0U6WR`k}0y^X6O@h)9*uBTmw58iH?U^_W9eQ6o%TD|Jd1&j_Mt@xGeuas4dHL$xa zGEopZuj9;JQ%FeE#%7{n8e4VVXm)=M&-bhk)6x1k7hSI}cYND{(dEi8MbBJ1I|=(_ z2>Rwhav)tc9UUm1#*8uBOM6b_c3T=4dRvKqTp1{bM9Vvid9?B$?De!k@c;9#^15fS zLqWU(!vD3XT!BH~DIohIR?kqM?+_Zz-+=I;QdrTa2?d{jXtjR8${FrcTD4h=W$jA; zv}XZ1405Z!)QUBUY#Cmwc!0%f<6EUBc#69wncApo+56dM`$A{wE)fW-EO96}AiMZJ zMGC4>v4u)BsJyLNy^`{;*;rZQr#+iXBSW29zOX+mL=@YrbhFJ`Yx0}9pL%$EO})=K zf4aM9V@XOqH_u!yaFW}`iPzrsUWSn_k(Re+XJoQzJ_TlyAju4MaMUhreYnu{Z$g?Z z%{}-n&v9h?c2eQB?8pI|NDw#%6iesBOsz6kPW(`a2+Xb%LovFkcta~%l~Oc-)S3=n zY(N79E90#kc@_7h_hxn5w@Z2Auh+g>DBcM(uV!1s3NuFl0w;$B*vFlA-|5SXzKw6% zlx0-5dxOiu|IBGoezxjwb!|<>^OZB&b=TnLm&aY4m7R{e&-orL;xwa`eAO-|ArB4M zLe{KP7kBlBNx7yTU4$;+d0LC$2E~movn}JAwHItgMs-+pM)9V)c`U41rbx;!H~nTt zbUqp}iakvlJQwGZB&rE>b?cw2MgiX{vVRIEFN*l6MK|!4$Dbflb0c)ZrNy}NAmTOI zvwX%{Whg;XUq~Bk?Ekrke*S)l(PuTud;=_Vjl!g!W<#;`AG3%6KzrtATxGoe z4vNYb>cmymAlJ-BYr}e%I3l!3;mO{pIFm=flrnR(D|-%izqn zGEN@oi{T6vrYyJdu>VDZ(ls)j&1m)t8;Mj`%JhBm`jXlazGM^4;wU3jBg zr0Ra?D--51ZnIi*{^5J-d|tufxjc`cTeV1nB}bSTYJPs`(NYNRj6uae7YIxoXvY*z zH@&1&U*vGzKq2dK%I#%6yNpTvVWn}a&$6VobW!cg)85TFwG-s+-AX5ndcX&oG$}K7 zX6`rDuVQ+n=M_@@itgFd-NmBZzU*P+qD{7>W-NH-rBavwFfFYb(mIUt?1?J_T{49; ztuF8L2Z{)lQ&PO()k~{y30rSS=7v?G8>@;{E6XJc$1L5**V}yE7W5jGy5356>@1#E z_bbZy{U*ms37khr3(A+~u1iCXOkyRZNTW_WB3y<~hbnZOEn2zZ|MP%EK=D~uNiM7Z zyB;u*40Bc@bv;o$byhx+!|k73m!(X-qY{U)Je+Sr>(K z`NImwU6pA%g^9B3m=e7%Pz;wpN%OJDQg#Og*J|%WMXD__HxY89e4zMO*(ytAvQfw- zfm;2`{z0lT2K(ZC=DY_T0&bJsQjqx7T}PE1AA1IZx))cJ#xG+5A?BIdl8Kecvc^ZUT(<1Y*cONXeh5 ztIPkQRK-ipo0#SHTG?%-hmm&DTUqVo++6VENRMy{#%gi70XCPvU}7fTe!#*mq_GS( zZOxJgT9ex~_1H>Ry{$uac!w;hA%mTEvfKf;9K%fSLEU@iNubTTT9(x zS#6Hc)U32%2~GiT0TX`pjIOiUyMk$Xt=XGB-oJjYhsVClp4^&TWweU#?QE=>qK5w@ z2E#@^CJPM^UBew(^i?c%KaH9psc=4_VMr@GqwLSoqT6;JnEYRlL10r$-SmGw z2AlVS(lG4B1KUWwC`GYKWl3UQmUVI5O9tNn4^|&`vi=km20?dpMH`Q&SoA3X4P2es zT#rh($6@UU+ZFbA?>OnZiL@S^s)Sdy=&t`aT)#EsBb^l2B3IuILn0Y;*Oz2D z1<4jaw_I$)?Bkw{kn{7Fhs-Jt`%;nC-eOs8Yc<~m^f*0tCuP$0TEE)GeAfocDtlUh zd8Z2NshbG)hLz#m?rE{#!D5kt;<*r<7ZrDBc=-FT#vF@xChFbtt5;Zer^3)fO(Aj7KRa8-6b_ zg&T|jtr7@I))&PJde)>)^gFjE#A{z=;ssZZgt)owDhiij*{tgbY{jY(G*=?sIPxvP zR|{*;^ImViZiEsB6VRK`oq7v7+f|mT9{*E~SJUv1*Kr7o)-5eN9(oR;k>=)>EBCCZ zZS-6$6FXODU5;?d2v_KGS=2U@Gt2g>u7Q5BaoA|gDK%Qylwi*yRJfA^q-A&& zU^XGV_?`5=zcq!_SR1SlNH{w`H8cU0qz`sjTEIeSttam$yo99p&5D(UKu#97 zjUHv{oTL@udq(yYz=Vy>oGx0)?rEv)Qj5Te)6LsH;=EcW8#=WmbLqj*QrDtlgUQz< zY)VaMdaA4%X8rlpqT6AW5QNB#NO0ge}StlXMf zN?f+PMQZPuY_&_=SQ|KSzQ@wScW-H-0Qq+6+t)d)x40@&D@uH^%9`pP?3?vMF4d{v zD{5Lf_1NDn?($1^swG#tTIO4xQ&${uI*Bysba|iS;2aJ+xYx5AsD>EVc$|;haaSJ7 zd$7(RkSWT(LS=HB?eYlgd*vy~J=7^9K+M5y^BPX6mZ6`~O!0zCn!i1z>UO`+w=`8#$5%my9ND6(f#7f)FwmHF5k&sY{ z@Ua8@1Xk?iUPMfhSjbih-4xZ*7EW>i6(&#+hFSekRave4dfT1*_Ti=0cl%*mS~%m1 zO-k+HcZch?^ZnZ3x=n5O%-U)1%DU+R=zDIDt}A*ECkp=S#by$fAV4&+yA96KQPOYI zaa8YTKQw)%)>rzC`%Ndq6i7A$Bp2o9)-YV2yQMxjiR6{cu#-H|>1Y#O}I? zNxW<@DP)vI7s+5Rk?GbTlI!BAvK!aI)2t+d`>;iNn<8@BiyPx%fxjdxPxigra^&0l zsn#oIo@$p71$t5p7r;2q!`0)Wq(UUTuEE@*OLF~*G!MmRz$=c?W_MfvJ!v5U}ZmdFrBRlNYn-qp!jMI##`eKru^R_p|Yu z$9~t`CxKj<#OZAJtyqcVia&*nZ>tgv5!Z49n>?aKaW$NiUH=#uUb7hVd_0xhMv`b_ z&@e+s0&|N^!0>uy8M?0c!mD#Dd4FsEawKs5CzR21Ie_ASlISm|a>!2EpmFGRyC%x= zVgJqSf0M|oUiL?2+H45L+DF6FR?(3sil4HgmKA{?Av@g#`{IJq15>YY5C(Me>dR&t{rg1kqafiF6Aer4o$@|24dUzK?JX!nr>5Y4v!_`6 zoKGG)pFFZ{reW|@(yC{P#$_IB$zi+B5K9Hv->rR1GkAXZxbR^X(una&J;9>xSRLB-QrF9=RZ?rO`QD&I5STE!HEbfRg z!N1)_kfzdmTy{zT>jnX!;;9VRhN8yXlAX8QGy_lX1ozdi9aS)a2K608{%oA%3{fcx zzGwbVewm(Dn3(k|g>Oo1y+;h&51|ZtxQ}|@={RHNSs1>fz|8q@Rg9UW-CO(N43j#@ zE-3Cjyf1}%%53s zpmps=wElI4!CQH*hw@47s&4w4EBbTv%9htC-uB~8BubegI-G@Ko#^@{jEBt^^4=>V z|NAum>m#8}>OKjEssU`{7J7!>e<`)mN$rm65{?#$SP4YLuflKzXI&NohaCD(#`%mr zzrG5Ji54S!&Afzy!XYtNDng>TPek!yZ~B@C!K1v-UmZTSK1%OLW_6&8g2rr}>aK29 zdi-9tXt>tHA}Z>>2CGj55uraFADgiEJn;ETN=$Tvw?XT|bv-f5ZolsMePmX0q%rw~ zr8O4*ym^g`f3+hBfHSqVW4s?v&pCMVe-G?^J;SOcu88M{2+cXcQVjO+gMrR-m*>Nce2S;5*8*{anT(kLL+#7!$~G z4CS~Be7`9Cc#KrgZ)*5WPfLr=C4JMdlRHWwuaqeDbd&aRbnAZ;Srpr$1Y;$H_4Y>$ z%l|0|<#lRYnlQv!XjMch5o_&L*MfIFYv{@0bwp;y|8SxI?`+a{Mj?*LZU8LD=G_R# zyZg-Lk}E7zv}J8WpUt#s7mrkh!Mg&(zjQjJ5HZ7~`Hr_UAE5?9=Ht|&nnT<^C83me3-%1= z2)XWW+m#7xi!ytH>9Mf;Cu9u1Qw|sFY}cAW44!TirO#oxnbzb` zzX3fbbiMu)QyVbBByk;x<^h>z*&|Y`(9Sg!L{FnAUfY`43LZU{m2d66kAHW3?i;Z* zA_jC($RVXkSd0kokN)jiuE6p?=8!H!q6L!_BTn1KFw<5b(7#?`^GCy!HB zMseOzB=LdOUwXXw^Bu)3yh7Cr4G+-}Qw>~^ems^d_y7uaq4acMGrh9gCKGhHjp1_* zfa6$vM{6^(ayh<&whq`{&IXU`EA(A|FJVc&$XxqvDXEnb`-OVLTGG1qLj_*Wa<5*M z07xf4(71KFJl0U`A8S9(9|rR@`g*xz2BC~jN5+0Oxf>C`KKZ+E*LGb!h5ckg3=@OX z!D_wSdg*j{?{>|K6Zm&UI?er}6i(E#`@1o|=!>ww{C@slNHz0E6Dn}&{_k@%c5(vw zbjbUAu?{|MN^$T>!D)z`)5zqO%_zo^Me4jqaZRII>d_yeF{ybsq0byXUDRtG3<8>?O4g zo25m}EG(EE^(kaU0H+@KOQ(ymaL-RybMAq3J07=q%KziHH};JLpK$mSeiH%^wN6b) zM)Japm!oiG~Y?J0(WPBTkr=j0&?bEUYW>K5B5 zlA1&5{?hn9DyvB2gueew{r9)tsx{Z4;?zUMREenKf(}7GQoGzv>*#s;=g<_1g4B+8 zlvFFLmi?Tdbg(7(hAxc^I6dWO8>9T;nxgk1lj)AFZyQzuyy>Z@ozc_HD7_Us> z{4z49-vp_~R$kFP^VUwWd|>|_2fZrD0^;wltEfu*4JTN2P8rLny|yIGJz64nV=zov zN;lpYEd{_#V;deHB$r{y!oy&(ga1(nojY=Y$5?G=&wap5#u^l{FY8nzpMoEN3y;0w z*a7Z)|G;gB^q<#~DIE@j-q_+QSK46e-VyQdv^VI^=q@i70)U~x_@O#8d>J_nGn^|a z_YFOo6+Lg0R3ffy`MEes+S!nTb$o%FNl?&h_lnkl_>29oJa9#e(fiKcYq{%T7$W*d zm2We-$xAm@@tpTVXvh-YI_hX;KMBt?)Z=;||JDA;=0YEL%+1*6fd#N6deuv~a}y!XJhT01QP z*`FBl*wsN=kQTHf$?}sQQTLh4Yquk=zV=#Nh!gN*?deE>jUmm5Tv~^-ui(-Cvd+Q( zG_`wT&JvwGkDz}Q_K;?wA^mRlFIK1jVS80G{?j(Rv_3WuzGwG^(N6DK7@f{T*t^eg z$!4CT+4jkhC)anoRAhfZX-;#KN8{O#knFq?1w}anzbDs?OG4vT@&iunr1TM@@aO4; z+>3k%u_F<_*9PD}y-}dtN3+*N)%P2N-9<6H75dmd`ifkMw)&jclXk_?YX( zja$d`VpT0b^rj8reLPt7>SF4lKM3hsP*MBM(qkhZW`>Vlfo9Av4fh38F)j$JTR^+M z#X_+`yVK|vjcZ%nLq@EmD(o)Q7WjtrjQ70?^%qD^Om>`fbucv8Gy}hbR64w@LrZ01 zMVuSm6q+MY`Y_6P_VkewOo?yS8txV&M+ljj;$kC80|KmZe9EbmXs^|%;` zb@t;l`pT{D8G~Zw_bA6<-ws?RSPSv#GYE7zWrR8}6Rf}WS=l3W2RM#0DB^GU%ub9` zM;5m8*!R9vRN5m$d@q()BO{jonMgS$t~vAgel-`*|8!i4W#g|331miwZ8O$hZ7{n` z(b(2j^WE>IZMsBWfBbH^@+*ZBcOgxI3yAGSwpm!s^sgMCpE~V(g7Bfej3BAkPXSqN zFe?NBSExedI5co#@q6*qRxBqn*9FU&9WnCoV@AvXT2Y+&B+K%rY0sCK$;J-;vn!-{ zUmy5F?BkKW#nMc%*>n5Hak3qKoU7saVPEt+T+tzl@coY~*H%7YxdOc@QqjHe?a7PY z&Z++q&&tInlQ1sN82*wV5ft|yauOjvW7(IJBi`bwdk^p}>j6v(sU|So$6y-!ogO~3 zJ8U^8%ebZJ=o;I{`GQsFZ3O``sqa#Ihr#*e%LkFNmNVGmPwm=P7qR=ksi#Hn^F3vAc>dTeZ}qF}q+5@jm;J5=6}r zU23l4R3{q3Z6FtnD35f>(43hflU{$-33w*C#JL2N;U2nbT+ z>h-pGuXfO#f``mg7q0WcL%P*3trbIGx;ZIz;NIPLm0}=3c3ZIVkP88V=nvoX%AK2e zl|93nKPTzgCh>f)7!2NXCNp{{x ze-)dYPaOPg&q>4EEzR))09OoVLWA4znLaa}pyH|r>6>cW#5Ue`GTb$%iDm@<5tbZq z&(dr+-1sU7_9GmebFm3<&e=b+$yk-n9aa&3Co@Q|GY}ye!|_09s~HP*Dmh%LAz;q} z#1Euh<$|Q(Faw?@=CDhhJild~Ix-A^7lVP)>vfd-h8mAU*U7RDJJv~A;gTjT@ffKz z<%o|l;Ke|HEB(jf8Edf^xxF^`RL7eg*H1ymK6C={uLzaq*>c)n4TzAOG;qvZFqfC1 zY4`u71X$jhU5sWAd)QkoI8r$90%aMruHtyc5Ym-*xdSw)-U5pm!!+6z6dn?b3)?r$ z^F%V9zah=YMyW@@1>h3o0dukUoU-Ps0|jYRX+wL z0gwUXT*Q(GRj1*rq`d3No;+fz{x&@K^5ptzNY%aB$*RvSVnGrP7LpMvYa6{Aa4p%gWl!vUnB zp${Z?f^xz%=TwV}qim@NQi{b{;cU2G_Ow?Xx9W$|Vm@P);K2829Dlmh*$?1imMw?lRu^e=gKZ-kd{N`%-GxS!zbp8~tEJ}%DLx(wb6rScomvK5IVFo7Cs<+m zf&CkYmoV0B3K$KbP*PCGs-jK<|3-cS_DU6ZitpNPzAnZ6g~VSEd(sC`{`F~=rpG)?PmLDNp-xYT z@wJBq5`7u&Th;U}?Fp2S zrZG}R9i9g=6g01T0FLlaCA&Ng5*AGwVw6QOYlJZ&S{goo%_Wzkt?Z)dn=ghil!&Q6 zglP>Edo54hk0NJa1o}*F-x#MJM87V}^rc3$qTeMs5{e;sf-5%U^oR)K(!@Ciq^g8+ zf@|Tb56`013l>z8%wQuhv){dX%{`B@9Am8bXyLBOBG2-9 znNwxx79!cR%!&HoT@$<@IFaT<(*s8Xj3I#e0`r35WZ$nWn`!U53CG!~ur&^#3OT@t zf7>sUg{cpmGscO=)Z6^vpGtBWXfW@Rqgbo?L2ki50A@{QQ4vr%V6Wy14V=qPqW`8h z5s`5HJHZX5@cG9^Gy$@$(92M|m6YXIOtP|KWA5}YiDiyOvHIVlKTRSS72#pAO+&M_ zmf{DX-E4xUzn5%&E5_+YoTJu!Lye#Ju}Cg*gtDFamA-hOOR>$E6T&0{+s5dT^vm0) zS6NxU6KhF77aHxoZ2`$NCTz`aY$6B)ygTI_FPgSzZ@IsQV6fqjm(s(TM9!@Z!|Ovl zzfIC+rTVW4McS3nbV5Q=SYcr_!K>+P9)ZVxlpatS1W~LwGJXMzQ<>c0RZ6W7Ssr>T zJwJrt_j0hBcR=lX6h2%AxQ6xj#scyn)#S3RsbA581-}2#ay>wWfyXzW5h-?uZQf`R z*0`Pb^BupH&g}z12I)(;PHhQe7biEzP1IdWlgmm_FIO+9RU&7U)HsN9`j2zMhVw*V z#vt2R!R`0uh{Th+6L5A9Y2Z1sb5*HK=y*Rcl=;}FIT zeQYv>3zs)aY)_TqUmHW_ZJ&ej`;FEeueWWzjb()kahmvh!uXkSsJ~G&cm6Twpubw5b_s4Y2!}#9}kwIC-TKeT$kALkai6m>LK~aMptO%K`jvb@DsQy}E;80M>Iv>va}^f_AYagCnlGeo zwQ?aL4`hvQniJDh?G>%Q_Z&zj?aS{A%uxQTSSG6z{(0?v4xCQ`^>6&3Xa+crVT_k(Ix}#Q9NjYn!;B)Ot{+wxR~Iil*dDNfJI9*1k8VFV3PD8=eg`oCg&y?{-y+D`3pfKpZ?n$B0qf=#x1osrP#S#wrCcg)2(@FvA&aCsDVnY*}1hKW3-e3PI zhh>X%jU?%&&Lwlcu@ui&pvu_9KY!V1<1DylgzpbeM`bf_HpB*}2gZ&`=i}52k3!Kx zMlB#_od0C>W^NTwb0?%UCX;iwTC#|Mppvzuxq>)(W#Trh`2DXJ%CfEi-KuIAKIefK zk^kp5c3`ioRd2VZetzP(%}}j4t`SVngd(aq+G2np2*}KU z3-$Q^Axu#%Ash>%;j;aMk1 z)%#p4CFX*>ej%>#`a7!@lqa&{h16Zck2W&HTh@-p!td8QaFkjlIA=$e6iA^hE@j5}W`{1Vs8JHDh%jHlzUir})$7k7|7=1g(8`8F`?0IY>KQbv(BTX; z^w4SDfJFqKVB%GK#D0smSzZ0wG@vfdLWs(#&i}TZ2>TP#4ciN??cixlcL2g` zJ!o2cS#16sIebE@m4PUL>;?4gKD>6L3s|-oCcZVsL)SWKG4Tpb*)5~(N9c69f=F>t`-MT7OZ&Zmy1GH%fbjmq4Dx?<0Z3T+D$?*i zj-}+a%6e?-m{uY8m+hjFLcYgbLF0LDML42`#30XtCQ6XS`JRkkP3-UHU8r<%R7SpIyEPf)gc0Zbi2K7sP9!u*EiKE)^mB z4J$S<8i7zLWFjP8w-dGIQ#tIt&A$A>|Mb&)!>-kCr{}G6=j{TP|0=NGQ1E-g*Zj}w zQBdF3Fw-bLr1b%_3slolDr6jFi!ZgXjnf1cj!5>Ga(Q{}b6W<^pkExR_N${6J*a02 zGmk%1c=wa8e%$*he-ME)|Vh zhjx1x@_WLqS3IT;)`Z~c6H?xJ+TG62_m|cDerg+l%}-_G`xPAWfRpWW$?_Tq)ADqg zYl6b@jvaIgs?C_YvJZ!GNcV`cg0`Pnsv>R~lR%RNgwf)hNMuarx>MWbetT z)qfw6A&ZZ38-03zu^tGM0(|_dUk0YRY_YAbxG&$-hLKBe`F5@OHFYp0+u0f6n6yy7 zc16rR;2qBf<&5B)OA)1f{bN6Kc`|7~v9^h9kLG4bDC$zcj2f!9^TPSVa?%Sdmz%S3b#{MhWx&C)|9@ko$>F7|4> zhL8j-eF#yue?<7wCMszEkb8qIA08bEtv@?+Zmo+`+Z|p~?_m^xjp?tMK8|>xzZAsME$E6<{CSHalg%wu zh}>aLE;K6*Kc6*NW;w7dXL)x`&XI#iiHrby zSm~JdmOw7W@KuH<-MSx8C9qOTvW0Bz+qGREn$%QKPlN)S9{lHwD#*R<5R!G^4;1@a z|10`iXZG+z*#!dw!_3q}T^X42&_A*%#{<~VGuove;Lt|&Q#pcB3|v%}(l-)|iYY+` zn81i8RzB$u`~{ZZ*=l|{sb!Y+yt{%9nQ~5=P;!GQMztK_7v=XXL*KHfn_e}MK+*X$ zILb22qa}K)@i41CQDwD^X)GP2E21u4ajW_mJaN0ZkX<|&*5NY4Q0njKsa8W{ubv}LP_}-h7_ufk`Xm_Z4jO^h=`R)cp~tgvGg=hyq-84OhR{t^6O2`?$NhK zdRguS^QUDfSln1hS&u`;#q`-MxMU|WfO{8*(FR+=X2*6`1C3Oi0$)|(Y8{d#6#z)$ zD#;nMudtYBNYVE3VoKu5(*jaaA6)m`VNL_NqF@GSGm2RK>T}n%v83$MV{ViHG4ZwVQRtrYk@DF zU^7&RPf)i-N{Zrh*&MdVQj?Za&ylnIiX61Mq(FCWiXJF4ACU#aAJTetoWEfUfC+Y< z_$jN{p^w{8Ea(>sViR9q`N43yOm5|$_BRZ#w{}zBoMaPBCYL%a zQ%Tcek+kfms` z;7fHx2_lT_wyRg4DKJnl9!T^yM^rzB(LkZpgj5MCXW;;~I(x=@43U=MCyG!bC;pWA zy@mr$&L5AD^25aK`xN`;So%D)lZ=zG)9JsgGMS}5(&-;ufU-h)-fPiVdG&D|i{C<8 z*tXrEY_=b$K=%ejyJR+rZi6_TjqQ7O9~h@oYGbpO(o{$!*pggjM3_X)us3gf_7K2? z-~<6Gd$i$IGM*r+q}8TbZw07qh(=T6#=)dc15s6!Ygg~0`i#)7*LfmVW6SeR&DY^~ zav>#=T_49~#EhhDz(g$HkJ*rsmw0kP;pr?(d`(L^$Gi*3!2618Rt)=kHcNdh3 zjg3@DSbEsAVBLZ0o`HuzeR%so1cM|$_Wsli6eY~GIy<@x)+bx+snP71P@dtu>3iAx z?Pj0RW_P{Y^Xbjk%`pMTJIB%R(VV}-d|u(UbtIn9iU%izP?k$*e982c?ywV3Y)A9? zIo0wu9F-J^SMzNq5r0SAEYt&T zSRtz#gs`iXW68!jPYT<@Ve)b89!o>QSDFRcE;t1(=7ixC&{-%?xqUsmIx`ksp@PU^ zk;IyZYjy@^*{dFWc1^1B<9|Tpo1D6sxza-=LVW~xMlu({7l`pTsAC3SD+t^`ONS#! zXT#;9FE!Jk2>{;?IwGwNr=Xs zb(qHs6qlNru?>&FaY>Mx@6 z58kiZ1Zess@%D_uVx>3>KiCng>Di9Fub#BkME9t$ery$j8Ks5@BaG z5O@an@nOSL$%ik1KZYiOIP;8hZNehv$TLra=Lm-fN(rj_#(*{Hi3()|GyV?DtswSB z;?6fWXF3nuIUdHCup!Z*Zpa+MEQp~f#O*J3LV;ET9wc-Y&zLmIIOdCL#UpMod zkk;65R?Jwk>L^EdZ6r!=LhX;z_CSLA(2m2WG3SS{h5hRkHN-NHpf$q3{Rd z1)ZYa%*rZA_xIn&>F%^{yC>{tDsIIq8%Pj-aOPq|oCnZD(3AMIaCl$N)}r9Rk)x)B ztsXp{mYhM2*oC#2%gpwQ&M6B7ye7{e{1=vk z6JZs>h}zI?d~Iv)`@&%ou49i--;cNiJ>p- zR4V8F*CPWnuVcr;I}$}a+Cs_e>h;DQCTz965*XNcj35grYEA|M#{;oCc<;7KN-O>o zIsDf`1N;FPA(5F|GADk{ce{;W7idnuu5}EzK>qZI3x(dYtRZNKOgxZ?%0$v+Du#%q zRa~{3M=-)MqL~Bu&O~Ow$;>NLq4sSs|1z<}ZYf<-2rg+TBUBf@7g2D0NS98IP=M_Q zZw^kh1E~Tr=kt2%w)5L%#~g>@S^Lf)ZCr}|=2qGb6R(=q32 zs&IjMW}WC}#$h6*fS`d8ThC`S&}Mq$#uOVBX7?^2piDQ?W|!rXFN_nHwU?t>Lv& zXn@mRyfl(iH@05CWi&g2wHo49Xp9nzJlBQjMRhC5B%UlAaGdyK&Sp`^?ln3knlmmf z9VgpDgjs}Q5~o?fXgMgfW5H_AFofHU^7;kay-}n>>9mkmEm=|;*$Dy(K}gI4IBO`> zO|*e$M3@tO&yyKfU{ntjrH5T3q*g72{E9gXqHw-jXn?bNVYV|xQlJUxCM}2%fSKoC~5 z3Zz-rXGntx%jf&v`1 z)8F%F#GLIvA(P>Th_oiYDOK63{po~IolnFc)Gp_F4d)lt7LLwB$^dP5-YJ?4hFV}V zaaQ)NlGhEeol0-8w&ikj6c)ya$n8*qrgv1?&6Or$s zb`qe9l&D&g{1vm!EJrR#VGxRQHsB9D@1{&_{L=Z~{2kd^7xkoGsta$9#Vvt4W-@oqZFCBeo}X zR`BT#kxut61-dYt!r{pU1I^^*86)Ir37uv~(-`zR+Q0-5di2k468RosLUv1m!Rn5b zIWMe0^$pyk?K`)6pIkaupc#rAjUNccjXW1ksR1UdxskY^TYI}bH!pfi>mgyup$Q5) zMVYUac4x2l6`=^4Z{)d`rl(-#!*ymrRr@Y)2Ima68UG&EU3+*X3&0!qqSiM*a}ODV zKxfxOu|zZD*uSzHKO@ysU_6)&Wv7s2-Y4vwGw3&Cfx)i(i0!rFg@nF8;s%B`-PJ%l z&hS2iOFs%1{p(EW0XtLd38|)07A$MM8JWx<4dCm2KaLis*`KPR4|Q`WoQ49tULWS% z`smD}>u?dVE==_y9s>tWP5YZ7E+>ac6M~eJwAN5z6@G`8v_3slbpJ|C|Ah-cTDmw4cPc) zdL}h>;79ppOw{)SVs{?!LR35ZRd|A2x(vbQkEZQ_!skKyoSp*KMn_&uy{c+nB&{K|sGBrFAiN>y4o$QGON{lR+TMHu zJQt3hIhC>2x&wCcNg7K{_*=zDC#*S|+l^JUwjGyS81ZKs_l%aZqCK`4;GuF_Z z$~d8LMV3a-j~^=KNETl5CY6+2-0zw>e)fM z6t$Rv#(SI$OiTKDWqZD)hG3DfeXwm?5zK^ejNd7weauwhV85cJOHtcNzj7EV9oOp# ziph@~lKdm;R^F#H_t9Ua(FO%i2I3rR0@S${0mj9ts!V{h#RSUL0Hl#3^d;VnY7mEg zHuQD(2FMIf6$bWeF5s1=REivBDozZPPfn#snRIn|g9^2^NE~T)v^RnHL$tI@FKk2c zsJH-KhcOb0caj;)TXj?>JG>W1T+=0rBv6G=AMeYnNS*;N6#gUtheScq`FX;-Wg?J( zbVV0JrwG@aD7Y61VR_CziB#T}8&_gK3quNw)hP#*v-qS?F+~}vEbb5v3`Pb4d!NCG z#w0ifClVF_3!&&rOWI7%KW0oZ#`taDwn(bW+ucjGUY&mDz)9h6xWE_=CuO^0)%AUX zzzG?5Dt7{g8MRg-&V2Sn`w8IhOySW}Aif0ddsSoQ$kd{zM1+mRy*|$?(ygyCg0o6@8^D!Cy54Q0to*V^LMM?bR3`` zF&~iD^qgr$YGUbdv@FLf?!QR##1iFUu`6V#4zS^4Q4d2EnLD= zQWQ@h{HmiohXzBBQ6FA3GFoW59C#5QP_E&M3}cE-;5>GsSZo~@Tpw8UB48Gj?peKl zwL2K)GNu64Y}^RH%uZ>TnTFoYvwsE>(R*+H@_p}~oTjqN-b&^ql}UumzR0 zmabzPpu@G$IhSTLzB;B($lbAv@$jk4E%z_7J-{0oDvlPvfG!Jb8<5_%AO>HKA7rlWgjzyOJ zX2=6?B?Oy?s$mv=I#+mt=Z{$*-F3TQRdM=#b1;YB_ZVD15*^M&;Jb8RF2u`yj=*FD zibaP9?*4NH8s=2IHf0vYwJPwJyz?M>b~7bBqNza11od!%pN)`%bX}U zULYV2VqIt5Jk6w=F4FNsWQj71B)C_tOEO|G?#fjieQuG|k1&d(v^*JJTyS6$$}5~n zIMs5}=XtSZ;Yk!5ZsFMM3O_#5UQanR(Hq4S4XeYOfpxRRB#T8YUJLXYX?Bh!U%u0( zJd9Xjq#dwwN0(3$<6J&0Ia_bl-F_029W}1x2wVv*9(ib|((LM1EpT?gTQt8^?SwYl zzfjCZTQ(PMq2vOqrPh+uVk+uHBe2I(Gf$``n@+Se?~Ky^$Vkif)7Rm`D7upr_1Y>q zvXLdZ3qO=YJtfs6j6<$ILD`FHBthr| zO%=(uNKCXu0Yg)?rCp+lSi|lqig_A|0DV!HJrSU*2#m9+Of_;(*cmMs*9bAqa!hO1 z&{Bd4Jiw~k4)gPKTRs+wrVl&`B$GsSrM6!_Z2ELYpe)6R)uhqEI+q1BFc)R;*V8hU zK`8kI9?Im>5J;)nMyb>)Y>ITG&?~sZus128LTu{GR7#|(>Mic*K83aEboo{}AU{t` zxb#cjw7B0`Tzetz(f2i)P<|Cv9TfiI#Kd9EKocgVXrx%WgZ8r0R)C-}5n%$>~in!7>3u76uiMAS{-lHS~WAGCzqzg?d3w-Yi z(ai*@#G13FRlzH(nOxy9I4x335TLj!Zfx=^oN^Wcq}QUB-pk@I)9CYql50{nm?XvW zMfT_XbUXLx`?EvUzmcer?ws@sjfmJ%8DG zi3-8;4Qzo6cAR;ff%JbJ2ibiGHM793JGxCVOZ@WL_5Pjgf5nIbuLbI~d%_hPfQs?~DI z?ZNW(bqi>=Ty-mrpv4Ww@v@yGg{{Vnc6i;nJhov+)coWWvjxjR+YIFyY=E|?3Q!cc zL9*t|vL+nSlTO~)u);tm+QyS2^ifAN2$5`Ljb(T_Yk0U`N{@p|i?XhY{J;yZ)puHH z;^I(9PN*nJ%Ij|WQ`bljK0ThH)U6EXpQew+sb7J0g%cVZ0wD|`&qKn|K1v=FY&)96 z89og$iTdHFr@mE^!Jjg9QX1mE)4aCqNXiKTHyY?iId4|R&j9zDjE)e92MKeUs3vvV zW%Es^frhe%65!T$ey`C%m(Qk%bZwP%=}kMY$#>*GY$sC%X_HhB&&St?tMcNi@u`8L z(psxUApJ?;&U(^Z9pPDJ#Vf9z+#JwyesKKl*`$$cichx0bvnnY2tr~f>(k`gEQ`b9 zJx(jmpibQU68gycrmNX@G*kA|RTOkj9g?0w^GNB4deP=9>!p{*0^~!vqYO(^aybXh zCxj?ozIoU9`UE4|gkA@~`k=qNqnw{F?)-|o%&N#WOIbd94FR{p?AGeN9C?857`Kwr zC6>yaTil*jNPFeH08aSmu(k=ELK4%XB_+Cmtxzp)(;i(iIP<-uq4mC3_S)(o?p68w zyaq%SU@nLuHn1}h5%K%+NS2Vo*x?2Z3&bgYVL3);FdhTlG0xbGc)}@S6Z*u9#F45K zvG8+oc2ZOaMn*Z2SF9uLBa?+4KlbKOb7ZM^xd8Rc)KuB5pP8EIeK+W zx!LG-xoka;O(%3(VY$qI0Q3KltY06l&_!+St(1Q}feVahT^CvPzg`$0zOumE_-q4) z?Hm(ajGiwaiR-21{Qk3Y^>13`8Nsvf%81N@imZ8W*A|5J=zIHhNevq>J1|c>qiR8)`P#UvL|T>9%H4zw z!x%McltdvQT3>pVn8>#Ra^aVgyIl($pyys9A_yckb`zR{N@b8GOYIq&L4%woMWv#rg?Uwmd zV&If{O$#opLOU-|_{n5AxCE(3?l&uv`|(9x*;m(V6(ni_p?rdLvMy(lM{n%@WJYhE z7AhqmRv7kaovng06NGP04Pv#EyBFsCjajr{P~NgVY2+)s>gA2mE=r|E?E=F!OR`4L zHR_*PnA=`>HxV@pj53xSh^;j-)vD4Q6ttE{{C{$ef+g5AlNTnD<)mo(Y2?DZk>pAI z87s1Eqa`OHc|yA3^pGY{y+om2<#an&m&@i5y|9|nRyH|U47yQBaU0%Y$&}Jb+_c!D z-pUwh7t>`hEvGX8S$K(2O#Zx#fbRixrA=r#18QVi1yOn~)LbVHr5Rm=1u$KaWXg#e zO3!Mtq4(M?+G$A|T4{&UpV{Q3nf%$VWj~aa$zC-D5)TK^RF`6@1L65Dqc#);J5>uN znw34!&{O-7VBxPz<*=J09ayr&5tm0n*$bV-`m0Q@hPLxgB<+gP=dwW}s62Ps_m%8_ zMs!DIJJlQzLU1rzHuk z$Ka<}glvMRn(c>AhDa$%@V?bHbD_RF{Phk3Dig-QSWNQ>n4c+`&tC7Ky5vL&iC z5|%638up|jtd)m*n=I>;iP-wpHgDi>_NBkmj%Rvk9PW5yNI4O$U(yYq9JF2D?4Wye z-C436mNXpH>mNE{ zoRjyLao=yu4+?p3suc3H*)7CKBZ;KqX=E6%XMy1o+VfrLfLQ_XN(>vm_KUSrHv|>_ zQ^P1Gc;e~J2X@Yj%Ak@#uFsCuv7&Jbk19iKg~^uNq%)6Y9_(31T=UzFZcf-tlyJ)% z(Jb4aomo+^%@eeWGp&Edh{|IqGhQ#mPk1!t7VA?O=~RkBszT-z6k_|`>!&!}3Rx6* z>i&dqVu~Y<3Z5|{n=b$y!BtE5Da)p1ln_>z3FS~s$28`==g~+A?}W%pVV+!Ucl1Lw zw><5<*`Hanay4_CW7;uD1?VDEcXM;Fc4<20VWUlav%ni8phLmN3Z#xBLEX$;Y<+5! zH3FL_h1OfJc$);Iqkgp_6VjHq@1#V#$K!6ir9-Nvj=H7g z2Ej%8&{+?Eu2qh8cKMlLU6N9$it>u}QlrY+c_Pz(-dZ7zk4=(<5UlHc%w*L`cT!xJQv=OEB+nFdVvATm;p zs<1bp6K~dkY}Nnt`+GeCVN+D|&%TtGpHJn24Xqum+Hrzv8Fe%i+Ck1pM3~41vx&d= zXUw469W^`{VEDUhoyRd+Pbx#eT8ROA1a!dM+j;r*=U-dzc}9;X8>pMEi949t(+pN8 zyr0KE9e@68|LOYpEEExS54Kloj^|}mhX5+Ljo4IA3tX0f3&yDyW+g-Kox|DwsZ7>r zHe#kHKBz^jPnP7@86nq8vm zW)pd(FJecO_mb(YCzD`Js%5eKy}$ahvU^3MujTic;|l(D*EMu@zyF`eaw4Y})7~S+ z`=mbmpwzpyIop?Rh%!yC7UT$>G~l`)aQQqO23mLj0>yfw4e$SMCLDvPMfHE&rN3A3 zzx;_cZ={Nl>?M~}7^?xTHKNf&Uhd~)FEp@aUo2yHrIq0NX8ds5GTbz;jG{(-j-LJ6 zJNqY-kp-2NgPIM$l0df1|ZOgO~1F^=P-CvdF)!sirLvYG@UpBdVKD|1}GmVUTvl8KTp zW#o)SmEmKCE{iotU6thP?4X3V;sZvG9v+&Q6(wlQSseXmYYhpwCbkwCDb6eyrh48< zr5v_@mOip;#P%4>dL=$92B(1Lbn-hp9IE$V3ze*HMc0{nJ$6cIqR10hPpPE2(ZJ)1 z48gr6W@cogEs~qr@#druJC1LhmLjR=1v=5~ZNc)RHZ3)+|Db~rSo z5_vMwwp58nNX}$gr28xrkhu5XOP07z19`iz#8>e^NouF74k+%`$qr#CTb8)1M_*-Em%(;_<=01Z*NQKe*54=j+yb2WC7JcTTj3LDO>&|lD3f%(Bc9v_%WeS zZ^Bn_0JN{nF?}pCp=QD;1|mt0ils+RXIf?a2$Xh`V^3f(+4a7xJ$47`FhySJw}5); zhYPL#t)EGeWZOU6fB|+#VNF%l7+aJURLSD7%R7<+^-;bc34HW=+1_it+pQlTbM=0r zun+fdDEP6?tbms;mMtgWrZJTYbM*nO7`GTwyVWRkm6lVyyq!OzFv|eE1#K6TZ&I3N zh*r*elwemiOq^CtS|s0wUW6?+v1<5we_L?DS;Yf$g?P{?=95;BU~TZ;pe8?V#qy`$ zS{mj|3<9`V_E5wv42e~KDSb1F8sWnC;!U17_khxQSX)~cMH#`&NZqyNwyJCV0Tmr} zL_Py$$f+0*TYXS>`C`|5U*-5wwp(VZvYh0~W5oK$K}`oFxX6>i`K zDqJ_Z?b-TYlbPgKKz7SsHUd|8zsDbNMC->rp>aai&pX7v+usk!#b#emMD$9Td3Jl9 ztq)L_>kgs>b1fm}PTUa-uAzEDOebtH=lt7CR)f`**NbfI6KM-z-JCE%$GY(j$zEmr z72Njc0wP&2=*i8?3`uOsNtob%hfO zY+^wY!50P8SnClfczP(IQOj8-l{OvjiPpdZX7NfX8s(zQMBnReH8~9YM)z0d*0_0* z_*&W9Nyb`GIY#)KS6Z=jN$&`ePHcgTdzUqsfkA;gL7sCRK)G<%F(^6F1GU6+GUvMRMr5rlrobY{9rND>ZXxDLYc> z;-aCa1phVdHOB_&RZZEW0KY7|luT8g(h9ijhh$q)+b8DD{VjV>D9Z+vrsFhij4FOD zfaN&HvF^wju2cfeT<>vbWVK^@FkN&;9X$odL(YgUKv%-7%TzW@Qj6;Ay@GQY8lfBu z{rlO_Dva0iqcz}=Pt~B?v5h>6q+1LhIrm+UUBdmrp~vYc#%cQjn<(pQg;C+u%sN&& zPn=kLQU9olSqDNi`FN8fN18Oi``qrRa4Tr79SgNU0**rC#|2%L{KNTl6jRui$PZi- zM@ggAPH|y1RM#~S9vS}hx)|hbATPQtb(*pJswW>~z(ttg=2EnGaDnXU7O2#H0q(*` z43M1tddc2hlhY)0w=KI{bZaCd4HXX5_ zhZZZG{wq<65SI3|W<hzi;zq)$SouMN!b;G=z8~&J*;57Z_SkLtcyRc> z{lhI4X~;Fl^-z&&axA!LBF)JqCo%e+gL0GQ86b)_b%H%E=NECEzzLiDn z45$~wBRM)@l;rX&&*A=<*|Thx9*_95hbo0dIaE|M;|AP`Dp+Unf6ClA%>zjj(sLC$ zX=(#2OCwGFIa8CW1|OV&h%=1R>0-%Jed=4z85bb=AaR zv7*-w%|O+zN-@c@7zx{wx5XbTbUcD4Z&ZBmbzQC?J|uNkV9&j$t+C z!ID_ORh-{9QE1}emv3%44C6}mQ-}MRM}tU1>95)7I0Xb>@F7)TuIwo8)%?*UI4U%# zhSVS_k94op&(7l4j;?U!NN9~KiPnVyIR9`w9I3qr`!PyBuRHQ|B&oDas_0zyKxU{o zZ8ITxk)<0UQ4|ZEa8$K4 z(j;2`P`_K4rGvm7kC1W9PQnO%l$W~Sgc(XJCqaE}gTzQwJk_eQTsEdBHP7DHq$>E< z`lChD;5se@*qb_@=N85dQL9KJ6F`AfFCWXQ`V|phjNnBXG7+yG)B|Xm2w{-9|O2AAVOW zl7#vjtVl%_Gfzl%EqK&?VInGBtQ=5XpNsE}SEUb$k!iJ7!)_&(^gw+HHj1yT(7rCm zn_z$h6>MDuezq!8-GXDmxvICy zV-q{x(^?BWwpBWBr&dMxs*k$X+tfbv)l|~i$ad$`=BGM=aoU7$c_&i{_xYX9aJveM zln4%HsXo+LZiru%;dwK6lk|HFk-30FXZE#j6|`R6+g!TPTcN|paJajnTWNT{@>*zX zJ^Xz?FdYW40T_vH5dNn$^VD`Oc%3`MpY#p8M;%?q&DUApCQ7A~_w`#T`Ar9`B|kS^ ziSM8VP{l@?MBq zCKuAm1GeNEDwMrmv-vFErGS(@A5OSS(o)vVvy%VRdDyX4L4KY2kW1L)XoECu<4raZ zV#-m*ULxOI=2Lg3Nip|Ouf5ZI`}Q4tW5If+$&L|7h_|XKyL!!{<>tkIGf>ca6pb#- zUO4r{;^&Cej&&i@`hH&L6Cmc9iITBWE7?jC*E2gwQXQt!xgqw5RlZN;jB^S)9`h?| z*-qvAo2A?`LSBP>yV^%nou!g1TXp3=E`cZ%H=d+TiGd!xeX-*5SqCywhd}Jrd0)9t zTTm(oLt`R>dksp@SEl-6W=4R% zi7uH_WLB7-NQdf<2nK_F2Im<%*;QN&Mx^0eYr$FO~%Y6c&I6q&^7KaYUpypH! zjMNvs0=nfC9GP=0fA^ar*}2CSdg}aBWp%Lt2h^^b;Pc4gC2-(fcc~J+V)y+jlW%rw z#b!mCApXa;!SU|H9(k^@V)aA4_o1vqYOlh|d3^DdXHlCi?I9aIHHE1J8*itY7RgpL)WvSAG*fL-BLnarKbhJaco19UXtZ)A~^HHMBz}2RY47RY9yn2fILv4Aiy_v9_`HNCu|43h$%`6n-2Y z*4%xqeDsv=o}$$S5+ zw$|GFo@dzR$pLSHaXX=W-FQ6Y)=p0M2DjeWJ|?I{N43}6?ZE~-1CzP&0ziN*>Y>CP zOT5<%J&|*UIS1V#1(Vk#(Bow6K72^C!ixcS=tYtEYiVg|EEZo$@b&oDR$#KQ)Yz50 zJaqP~=vK5?0(f1r&B~z?5gh`%U373p5CuuCQF>u7;0{!~v139i-9YNQsnn9DSH|`8 zRPV0Fx9-!!dCASWWm2JON|lrnobTj68*m0&6y*;QLaL0EBaZ1)igB?ylqH%Ihif>m z?)%-G)h2x1*;{V|eI=DnY%dX*Io{h@e+bY2B)$ts;xCx6|NLvYUO1IsiT!PL*Ei{k zu081ns>9x}em1^wda_U8SLvKbub}N}bN0OAUt)68XcBO#Qr+ypn_4DTw3ML2rVi6Y zvRuS-)msMb70wIqWJuY~F7sbP{-#f>qepXsd_QK2d-peM;uF-AOdj2^jz}_3@Lv|g zseF7zC--x&TIb@i`%`bcSyqf5B+_0EjB&CBC{m=}ow%~?qn7@2V-YzR0DZyzELiDe z_}|(5f8IQQz9vEY-hW?h;g0U=GmZU5fj{hWs-J+jbI$o0Mni1(V4Upkd(|%We`aeF zJOl9Z_2etcW!tRaOtq%Yr&1jV@E4t(gnXIy+g%)-h-+es4<97^<<~~_l|V(a*EBFg z+x~s@=bj8z?udn!u!4~UlG0(2>6fk+NR`XHz;B|@1ERaFcFTD}ghC*5B<2;PV;#7j z7pNutYaF@C!f3UZrUWJ`4CjtnmK^LCZW-jE1h-R1K?hbr7Z*5@A4x?SXyQy_ZOJIO zYFV{pMU4$IdL>M-=w*{r%`>rKVEoPuo#w-3-&g&&_opANXeGg&ima2Afkf*fLFO8{ zzvdD*hvPHtWA%^#Z~*3&DPtYAftjk&%isf0I!1(WmI+-Q!@kq(y{m6-8t)-*XXApq zbdF$2o~YlqZ2^m{a-$(QAk}rp3S5d^tP*V^l4OzBo|T1CNAY|wBKfeYW?ap=V^;qE zh6kPt|7u@kyz1?s|8CXoE1!Z)VOyKO2;3NhH{T8i|E-%*VJLXcK7M7sX;$U1U@gGB zdOQ0SFEsF{4v^M@0$k$+QV@VF#V8Iev|Lm)}=e^~%vMdVIG zzlZEtBR0M{s7@dq6MOmcFvF^!w=662UDh-kfd-&V3Cz8THwMFdT*Zsr4b?O`u1T?9 z73Dpol=C0q&O6HFtYQ$a(SH%8VL}Gxpe`qhu!|N1vAw#uALzThH^^_@g@ZzyQCsK7 zM+Axwx^;{VTgJjG(ZBLJVVK^?dX&^_TH{G{YF$tEf3?yvef(hFaTAXaSnWu*x>paQ zkjR**ts(-8XyTh24hXl$1pWkB03!>T^j}seHRopCX2TLa`!;l%ygvZ1`bVg)j_dvJ z&vmy3t}krsr-8T5h!Z%b`1#F)vVrdOlA%nezQO=>9 z{Hcp+W2M2zdIg1DAJ#mM6OILtbx4_Ic63Ysy}?7nK`%@Ls7u~A&V+L{(&<9m`EJat zc`pt#0f-_1cEgsZK_)GPl=$jUB798yvH#Ol4Rj7*uuKW@NO1`rp_AKU*^BRSUT{-)hSGH z)VQFme7@dT5p!t)+K(qiPRV~c>hL7#Pkgf3e}K60H*L*rNmf;X#uui!6pTQBeg zl$`=HOM+_(wt71VUjO82aE+%M1!7|M35mv(e(>_o=rKZ{a0z{o_@_}MRPCLb@P(fy!@qTq6dZRFiUzuhIdNH zDS)p~JDibiE35UtpBo$w^nK}bzNMrYiR!w5Fji4CYgf7Ee@zLs%zKu9I)GjTeLt#j z!&1@j&Mggy0C+ZRA=Mmbmp z=Xb6V#&xqpU;wsaTbJR;&~T545v(1HvSct%l7%`~PDfg0Mo2+1h;==#6^N9#U3Nfv zc+ao8?BpK{lP_~_i8s5RDNV*m1sW)nI8UjgGb?RGB(#*P6kNEK$WXo zJyUHggT)5JJe;ZU#EoO-b$Er=v@r4Ms8kY^-j04c_5L`L@Ssf3b=+FcL(lb$(~>kN zZ;2{|lGejNV^M)@ZQ3c5U+!7;MYlpx)mc7}F2qaGoNaYm{-28^KhD3?h7O*#$Nlfq zd3SdZAXrQUB^7^SyIb;-#+U1yqoDmuGbMAL)}0b$D|@R}itQKE+nocjC-~nFIqo|O zOKN?Wl3Y_M?siV;%w;}oAT;Zq=Y8z>_B#4i5sCvnuQ#*FRtp&p5|U$#ZiNngmKZ4h zw|WU-6Rd-jwG;KQjv$>n6dufrSFH1<**fxw^wCukQyaTdUsU>yi>-4$XL~>_4EzM6 zLp3&w$%-bTxh5)^o359<=Dq8#Dp8Bx;WeFU$9MGWW>iIpz=M=Yw&>n7CMHxSp1q<$ zMc-7ySg}`IjUt#qq#hWr1FjehP6qlEX)f#D86Ig2+4f*TjOy`51lEJSe@ueestL(P z_loo-VHZy`>E00ejAkkcGHrR*V2hh;hB>>@Nd@*C;Bnj37Ou8LXk6LgnPj|Ol>sBc zz~-dLp)kC@fOO3F=zqP+{~_ax<$+UwJ7)WP;KZyT?1K4rV${;UeBB)Ao)u53W|7GT zd96JF5#;-w4r?-s>6&VMSd4W`_g_{8_;mxP0k;S4Gn!fj%}uLo^h*|n99s_RMm;~9 z?3OMe-SQ5(&%&)|k!EOlnIoeD(10*~P*!o`3m>pf3cyt7Mto^XQl>GMwakf)EF~a> z?>$mVpW9MFC@c;+ zT@MBmh_Syv@iochLYpdZIh+KETPSc&X>X>L>3N#&T4Qohuaq{R{Mvqq5AOG~XX7u( zlUD;n{uAb%-cEo+zs)vcyyu}3rKKQA3|3Fo-48>V)EYLHWp?*jC^(OU(?^ay5H<@N z0}J3@q7}~_ietelsQ`q$L$uw04;|RB=7at-lkW-X&8!Z0maUA!ZbQRk*dcYI4Ws4h z_IJ_=xvi!BzYp+Tg5$u`k*@J<=ymCEC!DQPSDj&fdYP&IE|$4$Nbp23fij9NKQvP( zyeNQm1Tw~58f_g(U<>)uG84tpf>?sMy$<6c-&pbQfpezgpa`)}VsN)R?(Wyl z=j`DY3^X?M+7C8e{~SN@q6Q?LS9OJ+`xkt{-}l`Pe?H$7agW8APwuT80KW$%z8# zT$9C6!|yOx{diDOHMaNhWFKze@oYW+uH$fYpSNckNHM{kfPLOcYSFAX8b+@81FOb3 zv&sH{{+9{SkXm$0!b7g@3;&%9AOhgHC!lPRFFcz?2)XrZI$rqR&iRw9q*q90>;3n9 zDRjP&Vu*g{-|}k|&nPr{xXsVrh+n+PE#k{s$EIH9#)Dl|;qCWjbmvHWh(hA8T$dhE zZF@)Iqpm#dukG)f^?#)CZrMcXgyI+&{BP9wEdm+*EowUEqs0P*)CaFe&)C25WSf!0 z{%vWd2Z#J4;=~>L^U8i=^lD4Mc|W0h*FRQY)WbawDzg8@J?+u&!O2O#uvuRJy_I?O zqtH^Qv?U(N9m!}}t7%>%Bukspw<+w{%^*dPTfOt}r|WCG;N$O~cJq{g;8NVKvqo$m zZS_A}f1&GQ{2`ito`npKA?Nt$}w}z`VY&zX0+SHPFm)nY2z~26n0M^*J zs}JQT^WfLtZ0&20(G!OIesSpEX)eW0^<6Nd$|IW zj-+q-$k@o@dAsnr9D??Jk(BNz!+#@KO6vYx^EA8Ft@cgRYHC$VTXR@%IyN=;DOs&E zt6y)j%Dx)q}Z#4@zIKE1fqU;Etd*xo)c_NwUEo1Z6jL%_3Y`@`OkZt6Ub zq?dF$=b~w4=%?^J%A36Ca_r$jAb=5JgNo_<>O;)lB7#AI1_y`?5I2CJ3R{Pwhq2SR zMVqfj;eZF;ZcR;=v-Pr;JDLSK!K~SU9$G}%p)kYX0KXug9gG8AqhE=!vJGy#$qDUD zxWsC^8O#;$$NKM--!i{8^!JFFzo8aH^WfdkE=yCE%%7O=G%~A69O3tUeo^x6>8E)7 zpqm=Xt4X9SNU}x8B9lKKI-GBI_?Gq&^*N+&kDV4leaY?qh{>zQ7(CD;RnVW}smoIy z*D!1lpa&I#ZbGw}81JkXFi_>ar?SFThF-sjS3hikUWh+H{!^It@E zJGAX56dF~o5t>`$w>&dnnZz-e5v%WQ_tuH_!mEv7+B@?6dj$I5o0|jyb-?I5U+Rs& z{iW4_qkkclzw+eWZ^#4`dn!2Qt1(0x;I&)*sub!j@1g|y-`Cjf5qiZ*aQ%%>olK7Q z@Mg%fwN3I^G-04-lbR8b(8?&anRIyVE!Pg-h5360QA{*2@5;&V2_2nG+wwe~9$?^n zMld$p`Xv&6EkIx&u9+hun#oL8vX+)%OoTiGE3s52{bE6{Oh>O^>NkeP1eFB^g$-p| zMdOBGVQGVi6P7!M*K?pd9jU6^SE$1kaZ<;WT!(Us-+fc7V8*n2$2rf&^|_v!u<4lK zx~A38#V}ce3QbUfM@F31dZw}=dI&neiI=uv8|=^QtbMWlJ_Kwg(Bh{TBNqing8#?{ zK721;e;+MEwEmdPYeRs0{yvlEDetXq8u9zQ?`MY`&qH_x+EYHW1mf}#5{RreX+yAp zfE6U;G#k_csQ|;*f2C{(Nz4x%Ww^|TRllk{ie>31Ts?cVHBM=Lmhsf=EqM<2CDKge zn4f^Y525_d23%3%FGB1M5Ytw0ua~l#4>DITOCqf1efuPmNSa&vBX!mJbz8RVi#GTf z>ZCKQk;HiJe0{^nuu9zva%A_O7)Wuv=g|!^g=U1;t|44U%OKa}f z(P#U~3h??3NF_1O!GAuQ`|sV+hi6wnz{l^asOjFeRwX4e-g75gHyu$Srn?{VmNwew3ipIoQFU1R7ktT(Ub)6h8b#vaNJK z!Sj8i`%=bEO`F31{trn2dbmslNAatl`h5d@uF79z{a^qu`nB^q}zb~b7?(8x0@CvzDyIdM`Y74 zmt^CT?sT4@)t)R#j><1}1MdgU+CnB7o|sFf<~hQvskK-sZ~_E<@ngW zCIuFhv!+_PSV9zM!uhLcUb=A2+%rmOb!&<0w1{VH;$&oZeg%Yd7b(jf5=4ubkjr!6 zCK~SyS){2v@TR$V|Gc;P=jsxHM(&-V16k7={dZSj{x5<2EX6HElMouG9)^zkka2a0 zEITs6k;fcYq>7TKGj0oUC^MVUF^a2IU z0NG*x?b~tuXNi67IWc(zB%&laR?$19c~5$@e>e4VN0NM^p1CDLk&9I~7RjExS~(ExCj<%3)03geJEb26u8_n*nV{}kK6wvah0=mBbI-1!tkR5AVjde}v*bMCcKoKb#QmN-u7v$_ zsm%Li%K)Os1`Je5JOk(3ItulYS4ZP49ZCLkFt9K4}TdKj(8VN)iRp3JP zUwQTuVu?ngBBPPue1v&IUOYyMOt{>_zJl>9;!sh|i$|Or$LP zeKF!E@Om?;$lz5T-pd3&EMKr@PEaGsKkZD?k~HR>Jq*}a(8b47-GC8K%I{dBNn%MZ zq+v}5VbotYFY)~*Us#C2axg=JmTrC_{cf3A_q&F$x#JMvhZK1Z;}%80*9=UOJmCAlaMeU?Z1sP^R%ULvNkN=waVG1JeYQCgU` z%FfrBY55?i+n1}1d(+)U6xKFzp*!FE2d9uCz%$XvUI1}LygO{PwDE$VaM603{9I_P z$zqh)D?I+jGc&MkKwrlx{CkmjQz%>Xm^ZU=16k)(FJLB(W>B5trZoNs-YU#DHs ziheg?auulOGC>^_mrF3~6V6S@vOK#{jAas|;-pR8z_J{1zcJEaTs2Nmiy%UW19PY$ zXe|kKifXgdA7(1d`1kJ5I1U+LIo?982$(Gq=> z{wa~`645{4XRC{7;0%SqP11*Dk8ZIQe!@~*NP}md9^+K$%Xcl@8I@p+gw-%vNNLy+ zn^@7&G>kS8r9dEIAlry6^aSl|Xg5Hg(uyRTuftMj4b~lkD!R(1p>jH7C02`b$*eo5 zC?1J~S8%Q8RUSy9W0|}Wz_F19y#)>U=asb=W|_%!Q)g8(0cIyOg~ddoQ8dDjbsx&MlJo$%DCT}lcRVb_&_^j!ft;QId6qQx* zSO;;*!2!P&O&J+wg!o)7H|qNLzz@+xz!DY!{atk-y%dKyH(~w9=B*IwhgKLxw6vvv zgY|zDD>~@WPvA+1)LCaEQz!LL0Cy}jG&G3<_f%cKF~<->!s@q1@*m*GCuxj6r?lR$ zralX*HZlkY%ukk~h8y}KI0mZA-Pk6}top(310N{|^a_gll_*-PfP-d2sHoa&_z=R{ zHhd3KC$gc7WN1U}QjXL&9qx?Pns}%7+|KsIuTrX07c1{f%5!IAXu}4-=?$Wit+O%0 zmor+X_@h}?lD*boi3Sd#cPP+uqhAq-kkIqd`c;C}%4eg=^zr$tB*ti7bUva0wj0SP z6fiP#H+gT6amOe{jZG;jtwA-T7<_2H(x9Wkts1)8)UojS6N*rRs?))esZ(yYP{7_C8K~mFNASM>Re~p_v{7<^9~plt%Ebh`-*yUTd_5 zm4(XgQ_AVSX84o_?dPA&(K!lmcb%zcTnp%Lb@-;c212 za9HjIezKAfQB)fwu;1&yVYAF$zE%mGbR#2&YlG-(5ZePJB<&duxJT;?gHThDSCFol zrOe0qFIM_<1Q;O7AY(?%;qCvBBA4%J{CfDgk)PDp81*Cy4hE)-DPe+w2{}I2;qybN zsDZei2Bx3eX)ZBI(mkFu+O{O}1HaB{^U@`ZCbKrbKS1n4V20skuKLoNHHfEg~|%MqRu^9;0=&s`?{Y1r!9nl4?M2O!N=SlA(nF z*H2;_&*~t(Y4XQ9G8JG^4JJw{i@3;x=x`ks*DkydhoclQ|DQoWg6*Lb@Msw!4XK-| z*%-El&Q4yOI%MO9R_Lcp;ZyYhf8GxX6uSm-JH7@!9aKlytKzvX__&|C#uxZjr-)h9 zDbcXpqyKxRJk?uewts$LtrYKhls< zDJ_VJaBd0rtL&{QJQs=5|0?6>0(q;bvymlCO-T9m+d>kmDYFVc^8nYXfE&~8^PwtZ zc%fA>PI3R;#2$o{`XU!1V3vSCZizEesb&^%&;2)rxGz3z?h;isac2O#ID~b$Q>Yjh zyygPp<|kSZymB@ga)GcM8WO%VQ zlT~88wo8rb5KPEoY6|dxW=oVp5eS1=ixOvyc4o|GNoC$TWBJ3~t>EyxE%tt)Wys`} zP0_i!y*~=+sW$c&LlV0d27rw0bJ8|> zCVp^iLjOAX4LY|6y`czQ*f2L$)7_Teb-!J|zOyI61w)a5fY|*XlW_;~(dm3`JD^NE zd_XF0e><^Cb_Q`*RSh?aja#s+@Pv^#J|VNl0wI;U*DaORzsgjd$ZjR_wQf4ME&Knp zDKzj19x$VqYh^0{YcjqmrNK?!a|P>8!cu=o$Dmx_$VRhHZA6bkI{N!TGGO**3Hz^v zdp&L2H4>?6>)#7v)OM(1WP2Tj4*H5|riH#3SlhQ638ZQxB*5&Ehy__r1r`It?^Vj0--XED75d1Wa;d$ou`5f^|~5tpdDTM$91ubb(F z)=weRt9hW5X{Oc14C}RRB%;Y?-k5HXzLL#`SfXKMo_W6V@F!P3$UYt(QT}~TgI4HO8N#n4P*kG=Z&$A;bq_=2H`e7_+S4-uM5$+eJj~Ob){}s zd_KRsg_M}W0^oP)GKqwG=E|*yEUFd=`Ha^kTjty{tzw~n??eGWv$8##nJ<2?s+)Tu zW!>t*0X*G|aAs&f`;rrbra@6$!b)gX%6w2AP=2S(!Hw{TP+&2RKZ`U+Mbpp8zoA=^ z#ZmtLeL;{Wr>RO9Re;=mQl79o&n};-Fw_J=Os8fRG6DEq$rE3N)`7`Mw(PRV0W3d9 zk?Vl%b`F=b3LpKf&Iy8|Hs+mrYA>}^=F&f}lnW7RmT^HMVt6N9M@O9p?-mWV=+v%1 z4?-ZRfCuaDhV~>rxK(`;@p&J>mayQ1JDvb%*0zVEz-=cOHmhZ?st(e`zZLhgn}rOQ z;QV&4(*ga|_4yY>EvrrvCX1HE8r+Q|S2S){Aq5H)RY^La*FMOPpgOwZj6UuEypi58 z{~rk%+-c$y2FZ7I;#005(B95VY2XZ8l^j$Ygt|+skp?>Q^ce#Bw1B|97cqoTv2%Sd zZcHnhA%EG_!ZzwdlmdKiRg{ao`evjk1KO~3$A5qVr2jY#q9=Fy!71^PIKyFt2;`e( z!)mnA0d!qO0ry3y`Ha;+j%2UX{CVRMjmu|Q%mc@K3^pq8;*4d!U$sd2-45ZLN_Iqs zIm3C$XjOBy=XBPJ|0tOpL^<(+hz#@%kM5lMcYQKBb1jO*3P=4nvCrN6&uE&0tfpV5%Xab_JizY#O<0h)M0tc!C8Qq^OA zosAe*!qL4iZ8tYfN#?l{}BlG`(PXtHA?`KY4fH{4I|c zx@hQ(cMXri2}_2n%_b9P9QYdL8fS<1LkXt)Tp$KXr*hF2k%7%lJ-I*6NggGhc>#vT zXE6`a_4OtKMJm_k(pf!}YID^JT|alUW;b1YUE-)>5q%}A&rcI!(r{Pw$Z`R~If_lu zE7m8{$F&FBb(e;{^7sbTZ1=hd&Zu6UKgx}6Zd8Z@QA5~cF!LLh&@L0XMxGs9tMfHh zYiy`dBUxRZqBuQuPP2P?!H%Ef3A9|tiAB{7;%I-&B}rt|BHjd3N2r5v0uj%J5f*uH zk99_T&jY&Nwl+s@ec;yP(m1rOggCJp27gY#Zw{d%U- z!PKmfwE3z0tU1W*h091)jwNaSd|qw`=t9mom2A}q^D$q zG!pwW8_Ry1#x;`ngTp9pO5h*hOHS|&d<#G;C6@jRd!d2vF@TOJ^j^k$zUHq#bhR&T zy}_@~CdZ96$+vWj5jEvx(=Nhf>|Ujvc?%53P2t%Y0XvR4Kbr^5^#$(rjodNnc2e1) zk{t@h|KNvLe2}s+@SvZajpExXY5)NG!-M@H5@;Yf+_i%jsyxlH9tR`}LdvJ@HXcV9 z5p=vS9`jA8f(i5{$e^&ze0W~Q)8Z<8Rpenl*MuK}nVUllvth~@Hol*AJQ-}%!xUM+ z9^3^gLmj|hz~@qeNM2J>R1TEF4tt)D9=AW{)o!xIS}6fEKccG9w zA5uO^eNAN@=R|HngsCCoM}_D6n^Fy)-b!OhlMteT8s>{5YDV>H(B2i3>3M|gNVPd& zmbcz8zbzDjnAevGpp#|p)X7e_bSNYz4=vSQU4QV+Tu`V`W5njmI&t#(%pV4{a$;|9 z+JKJ!31r<_18<}X)p?FcJ8YwKqhE$DsKh5y1Z9>y+bE42+SvQSLE${4>5k7s>&rOLduEc zFy|)Ff8u%U7n5eq)3)_=5{9w>~?6!It!Rm9Lnx&oT1r>MQQLDbUfxuIWna+}41 zTQ1!{cMq4h3mVOY@|O{<&1FoPzMZ&e7U1H29LvF}hyOB2z4-KS@PC>$yDP2FYa2Z( zmG8NOA3%jo!8!%nr%f5uInq!Q=TriI1*%x zI-!3cC!kn~bP_VKh~bFw7#24MV-*GS9}J6MG9EHjt}OiFL|&)M$*ArJvSeG(w(cD zBQ#t~q#5#@T)uW_e1q5`qx@~Zv0klo(4TuI&C08Jmpu2wkJXBHG=ZDzP!a_T(TX1- zJzn~B3Tf|o5QOR+GJ9D5I>neLYxqY3aGyWJi#*(G_2O_}18R61BXYDXq{|FQt~;AA zE~TCF^HlvRLw3?nitJRaJfp;a+Fo3J3}5?RZhIeJs)-%uANa#C2JiB!NCqMNhQzcA zQG~OW@O^;8d%n}#;JctXUA-K%U~f;uzeu-Io2OL51SLw5Lq;T+cacGQ2mG zTrkt9AE_Ck&V*irJC=L#j%R3|Nd=swyZ8PqiziV(AIn|kq-2$o{2DL0vN+@i^CB{Y zd(w){hUVwUwtH~~v3{;KMrGuRWt;L?#h7_N3KA(@QXdXq-qU}Ij(npjG zH+o@NWr!+DGJoZzM$L4}o6ar&l76wmdLDMYbnGjC^H^bftjP{7Js(Zy$5KXBHwKyg z@g&2a8=YW-$kX_C*o3__EB{2;{s}Dp#}|Z!11eTuG_E8wu)HPvodaQSGu&wtUwKCN zobl@BcAC)u0}O=$1kLljF_TY?G4VCC{bH#x5`kmZ_FA?5#?>}h+57vqO!-|?3{CO% zmni{=>9`I2GXi|Cs2bhKh}C#7i(*^0(u=kvCQQY}Ln9$=DD@!gHwZGo=hu5)8EBS( z;v3#+w|vM$vMDEzUvQao#q+Jrism>KRzlmP7TX2r1^o%J6B9H$xa%M~w73v2ax4Y2 zYn)H=*&b3?3}Y-8A8Iw29MA@gLX{V zUrMIx&}u#3Uca>wW6w@4P0L0rMi>Ar2#Pmj!jMUcN@f65WsysgmvLn-mR%x0m(*~B zGyJFcafOHCPjUfD#yvLVvC{Cwdj>FQd(E%AuQr)pA9$bdF>H^P0fX~U6iM&_2>G_SC)Y}S#&OZutB+?N!89r z9mE`-cAIR)1%z*t1W9x8*OpYyw>BIWF{IWs%{U@8P-vp0ut$gZB{g)Z6pieY%tYkL z!4+BD-yd#saAHzuev3P3kC3Gz)5M#&m;W6tKu!-WVEkJz+->?cW8-!Fprhzg$`#ZC z5z0;xlODP+(5k)=B<^~9pp6K(BI|UYMC18+nZsQtQy(!fup%@IW-@3?Vvz*zuMGl& zy;f$U0Q1`T`jx#hi{o>X;`1J@5`lJ{g>cAePuaa{knPc>>v&U0j}(T|Gh-*L{*meG zAfV`6R#t#u$XsT)Df=Mj^JV#Uk#g;n^{7vTuZ@(-5P8us@DUWe09fYhy!_SjRgl%q zET9?ld|W&apAPf6P|$zrmRW0Y&G$5l;rf**NOSClF^#t?TXdWLZ^W$_fWCK@+HXy3 z`c1fDhXAk}c1QBSbeX6j-{SNcPiJ7)XNN;79K*_vl+`NSic>k6JwIg|ZPRj5Q17)C z$2_u^hpRB=dB-hRp=n_2J)|XSU@4E{EWW@=Hu&`C_>dVN>;qrd8l>D~7EwCllh zS8eRYDrqO)Qj?vCptDCw2W2m&yM(W@05Wd?y>$F5hKVB?8a?#jk;(*;q%mLO-&1sB z;HuW9Hsc0;zU418fD8 zop$&5wrJUg)KYOux-f#|HjpR2TXJEkYOjmGCW0&wB|j~N1`DVYYGv$b+%APo7pUG{ zhr_pD#s|1K!1>HIApEzR7g^M@Z2rzfS&i-l8?Tu(J||yGa5?)R4OY{gR(OYKrrYVWD@Dr&gDh{`st^L&LidPQST$2* z2Y^A+7Sf?O4HCmEM=6M454tCwIcl$>4ENj@VKh=TuZfOeY3%A8!KaL?b!!tTO_u+ zQu;CJET-r2=K5opuGPM!>d(0no&1FHdT}gcBr$1$tlqmuk35Ft zp!VRPTKk6vR0I4i3p6*OBqoF>wbLpMl&lm&D|Ic-Vkyzt6{r32OSSMgd(y@>Y;eS+0?o0pfmnGTzM^=3M96@@q>P#R=;R7V+ zvX6oWYY~J>7toOaAv(+r^1H$gbEfkh^T`Bgkd`|H^6g4D>lX8Dy-!=+Nhj;-kfr!9SSxPfOOCKGcHD9>LXt&#~cIdot2d}wc?JVD( z^d)YZ+ua-rWRetkjc(I>^1a%I*YsWoEytnlw)>&&XN%tRre3GZ=$KO) z`8v+T;_&8K*h7XB2m=-en%;b&IvL5B#5jwqink`ZtukR`Yb$tSt(`bXUw{wkq{
@wj(Rv-`4KiJuB5`ldo;2_m(Q2Wonh-T6iG!tMICemX~SUh@44q3Vk8dwY* z=sltKJ&hk$gWyUk92z9Nf3E*bNAIm$YSJNWc>fi1*ND)7Zt@@FGl>F(EB{cY1SW&4 zZb)W4)g$yPD;<4FIDKClE1>i{)^iW=m$W7Cq&C^(09K>A)E44YcL%bsc##b3TxoKD z#pbo|$U!lJoXZIUSPlh&4iWm8Z9+{kF1ZW$?0bOoQ#ql99I7bjfqO+@3W6d)Ss`^3 zPXHuAQG|Bg^}KH}B6#Q#{U_~;vvr?H3w~yva^UiU*gGbl6E>j$UozsG!kue{Pg*zP zdfJ0f11Q`myLQ9xQKcLhdv|&)kpjO5XV$FL?rFVFJ-P5c-#v=+uG)zoH!b3Q9?5Ap zFX*1S?1bljH&O*I!M0w0)s>UG<-IeqU-!Sud@+k#>`{C9|Q4w=z=qOLn?WJpj?wCpVyHHH>l}oXJY$H zxs(?4TIw187U60mv>jNg@tR}PIibC$H*xp%Vpk^I6Ikjoox;~%3ZL)gX+YJlg&0ym z_QBC@eRFfaLC*=TV?W(j44>nS$fW>9ZY6kN{@d9$H}7tvJ#d=9(_%#OXqSUSgbh_W zUGCqAQWYn#xwt;3YkAIkc07)TD%BjvIor=m&lEsOssrZP2br#iGvV00MIHf)g1s6G zlIV6qxrY8`gk)T>U_$Y8ql>?CZIecNSV({fC)R3nT6TVxq==+<>c8;k2!KtG%zmWC zhQ|EAWQg8(fgQkH;0Pc2b2S-c<0G0S&&i&k|EYZZXsWO8Qrf#RSg`pt=cBLr<3F^b z5K->Di0Xn%gg{cep;^CS00;39eO+n8x@wmIpI-Jp6xqGT`}r@X1YYLEBdma4Xn?Zm z?byCoY!$Lm7y2>{l%sL>{&ATCrTw5WYd|lz5dPGk198uk9a_REe;RlYT0^wFdU{8w z0IDlccRoRr8jvw!i-Uk)Zd{UJZGJAHL{5QoIU{4CW3Z^k_>#N#4}I z=I0f450&GDQmKPN&C@b;R!>gG9+hHNzNwqYs2{irG4f}9(_XOlGG_fCQrXzknMQ_950}o$TqV{y=?n= zD5v`*F;5j?t@U%?t5j4lKhrAn51(yO3{5Iq_GUkyhJ4rBUEIuV%R>lZ^GhlR0wAMt3wU-~_#If& ze}X&z^Ejb_LqdK~J$+t(y)Uw8ZSj7EAG#o*1wtj8<7RJ|T#;2^x}S$I?|?AX{NzOA z+`|b5K!5#J`;u1$aH&cmwx{kdGvoHwd5=B@o*#k``KKV% zD8wLfEgHMQgdGOi0E-*7)!Q#mOO>VFWUTeB$R8!FxCVRP5e50Kr`GO9z0vo?bU0{p zkXMG`nyIfMT}mTOXV-WPpH^QY#Wkz}$g5vN2y0x5r%Z{%6l|Y=iS-VU+DWc99VZ@U zdR}S^vPwtUtYWseczpT{r&4}g9*4PXC=(imQBqBfZ)n{;N7!D;kQkEko~TyW&hR?^ zR3>C33i6#u9CaG;S=6l+7T|8WOr?2@`%4?d?ReGLx&TQs7jI!jZTd=_CVjP6eO**_Y z|M@XTP3L9CO`*HNENMlyP4R)vdSbzLj*tOCxMZTk~H!Llux)2xj1I+8?q z;rC8CD=p*~lmjY4sYlVm#sA8U48~p(xd(d z1p3O_SdTUz0sIDH%jB#GRhaHNpG+{&7B)%iL|JqaP|2!ne|`SLQKmRu{7I!@zI8QXiMVFHu+-HAz({8y6lG@m$QQ>q0yg zP3#4T68ys$Y9}aP|Gyln+^F}`MQ#s_Yd+<@<+Z8>acO1wvG@s>Bf_iP*&gr z>4TX>K>*tR81tc`Zd2k%k-(Rn*N zzY&e4vdPX`Qbo&hE+=h8*&{(XO3HVwlSEI0K;drl(c7fQz}&RoEJ^0rl+$w;*o=fX zg>Zcs2fJjmd~9rAT;JF&xyXju-hZTZ~2a&42H8R2-4QMp*|tHU?IbXc>jX*HQbvaHr?{<~?oP zgs1;|Xp>Ew_g9b%l{t7mff6gu)Ce2VD{iB`8#Q$Vw&d8%eMj}j0;1{<2NXTWHX6JO zlwc02qJHB0cd{dtwd&7cFxm2M!s(M~hR@<#OLBj#h`c4=&dT3}kZ7{@L05<~ zZVUPk)WjP7#fxQf#clMT^Dx5ZL!{oqg=+6~JgIUnIKf-f&)r${0UHEaubn~u5fnuN zJI@08XQHm}#$yMA|03+n6~HLEYw@Lcl0|XjVX8!r=uSev+3R>sStfRwBnRxgL10d? zbbD}kz6AWQ46{D)}S*{U*69;r-eDi8t!b4Ha7n=pUnW*)ms zg0Na1|5189#|*yRJCXi6e52JzrbSi1I?v{}-#ZF}@jW8sMvYp$=d0V3#D==il)}(xq?uU;F}x}&g$|^ zxg8|eyKWLuM+4m@_Prm2)G!DI`o!t~jG+>%5z8SbA8kSHHr+6frs&4C+-+ShXZC>x zn?u6_;kkV_$p#NnHgP|fbQ1qTxh>BGL#TA9XgbaEuIbY2ud9QxaNZ|5_JBAK6P6b( z1A!7`g=&BRGZmI1WIw&iB7$D!2P&C1qfuGb_EEr$;xI{!yrwFkovj-04;pH^Rmvb$ zjYl@vxJ!3Q&&E8dn1&rS%Qhx%G#a5b)~F>%sfs^moCOWvCkfjJhHo9`4yqQ66A`#4 z4l{?b^U-f=A!F!T;ddJ%3>*51Iahh{`g3Gh?zw;f+r_sAk=K8lQZMockt>bBC}k_( z-FSWCPwLWYoth@sW7ec73QRP6vu}3dn^JTHMcs%xO(n9T{q_CzpUN6AfA`QIMqVS< zwS-Y>nWoW~dw+$N${*O$e8!e6k)2gnil$?uG zzi@xrWZQInddPX{hiUPBJX^o&I%>zNHmP!?WI+Gdlf>o)Rua9j09WFrs%)xOT^cdj zf!m#iZvH0h{oR9qpjv1HFX;kNq#sw1%U<_<`P2EnJYBzi&f62KTI&jLBLIz!tQNPU zhw-xMH}#aOT=m1^A%~dm`3y&iShseAB5(+#L9O(Zw zP@*?_lc~L?QC6+v9Y?DKckHWh9DH75uWv_<#jN|q4-*qT>81diZ)VXZSR$b9|HGhC zQ`UW=Nv%EyH57VtSG}v#z5wevtJcVTw4EP2Z-7R_|B@Nk^Hs#ypbcK-Y>jG~*IZ&f zyaShS_<_(?FyvzUV1(3|S->Bp$m}t)_W2WdtLX~&W1ka{_!j%3O!LG8np0K%o zB6s#g1h8CbW2W4R2cXiFvqdA6FP;SRmxBk=z_9)gz^;Wyh|Pn*lIkJ_w^-!A*^gN_ z2K8ZPOZdDv!G(KX0A_a}w9ea}Go5(|d!pGKR%~6$(O+@&E?=w1k6-E zAE;Uw0i%tWJyRqA3f;^44NrB?mADdy&hgH?E{964S`X*AY}GrlVe@J=^O<@nV}dgK zus4DyQ|>w!5Z%QWdKTI4aQ=b|_4FeWpiqZCZzFyMcqb|UT7{7lLwZX_!?~qP5yEg| zZO1JnZ?Hl?FKH&C1m~!CO)BG-y~Hm2t282IMUKI+Q53j=RHz~!sdpKa!zc1#7lQdc zt3t~6VQAN~<`H%EZNmy+SVry~s$1RUB-Y}!_u>i^2Pu&E;Mk(uPQ6vWSq!}tb^{s{ z;LA%j#jU-{B1uO6(A!*PlBta}ba@iaWxM#I4GZ6Gju-^?X|U43HSTceDN zGqj5I1K9n$35haN^}Gd`;;A4Rr-i53rBUl~GpZTHe8q;v0m(5N!C5Q0ipB~9Cxe1w za_Ha8K(Q1fv$c=$2qeql8}L%@)GEa+&o#v&@OWa|BpxvldG(|L{)6{9w6q=>mYtY3 z$HKL*yn1`kdATqXlQ1*>oMxSyjpOMfM48ISeUs|wJK7}5Cw@HM)Ly&)Wetd33_Dda z*;N0pdg&jlKo-s)hqBjR*z1oGmvE~+m&5735(m|gOT>ooDlBD1YXi@o4AA${&()1u ztgBIm5;*tg(yI;I=6^|M88eh){%@MVL7Z~WnckniXZ4Qi@z5^>A|-JR0A0kKZz4S( zdKla}%d(eTwyS!WBlMp*X*qVEbVgPR0q2e(o(otsLUTMFb9!5YfmeyJi3L=RR@~{z0#H3(<+|>D<`ShnnDwZ2jBn%<){d7B`laQynwg@5dju%u!#n~`#ITi zTMlwZ@XNU$d!6cU;}zH;BfoK5A4D?`IKmpa0gUF!ptF0KD>mHscb0qiuyc*%)4Erc z>Yd8xKV~_km-1cq=TrnK-3WseXR5q8C^sX2h{!XDME67VxNm%q9THKfs9#PX}^wVTCDFAhz;>Z`Y=Qx zlyXRk75l3pXeX?|9Q;@6CJqUeMuR$0F?|;|t`5sBKQkZ+R6;^|I*v&q(06CvrlqJ@ zih!Dfk(2YR9T$#7cTr=pwRSC`{=~(&DdMQ!zM_lHcCaczw9+51+WY>y*(9km_$|Pi z!?cJlJXCF$$NxsxfMZe3R4JL5Qt&YCPGH8GWdV# zk`Z(uA#^;W-l@a?X1jcsdiIXfVa(&U%IJ}K?68r&mk=xakE}^oj1n$4*`W}Ec-_WL z@AW`Jq{+!l2NkwXmj8zjM*y;dR96Iy;ZJK*S^~GAAK-m(G}Njl-DWwSn5#=1rB-wN z!(qZp)o>R-7YJQmbD1aaw1ov7S6{aJbO@&e9run$I;-a%;x#k+4%WaeBdTgG@et*~c@n;yg5*!dCmQ9|%l~NT+Ew^5Q2SLOcq1gFB{hLTb$Wtu-n&;+mU+$vwe!hbD z<1`5b2pgOMb(0!OM;szsnp(|4n=^I@>dP3h7#s+K^%+xv89T{Yn>fmlM9gQ%sDh=n zl8Nz%LpS-*jX>nNN5&i@Ko7Z*`zq+H9+{!p6*C2Ziy2T zM(|sHb&9?nMLb=gq8%k}Qj$>P$z(mFIN=ytB-B`4N}g;J&Si@x@hI92fw6fbRsk&0 zjLBCuIS5ug(Mt$QZRO~kt#7T_1g=7#gUF0Z1zj6}BSp(?jr2e%x`n5EoZ4)|hNt}# zXO!MpNcgP%csydY&s8s4t&do7Xb^`@_hB#@%?KNv@?op{?%}dPEql$v7)LG6osUNG z4*XPTLy%T@E44Z{O1s#0ul_pyUXE`}HIc7=(XATS-QYtdM?Bmh2 z9pWZrdidN%-D`bOw;5WV^#BB9zoM*rYp!-V&aD8bsRBtn5@n6@)FZ!n8z>$KrG{gm zQs3Me2aS*9_lyqthxy`Yb`y@r9sLZVN(5;HR>Z!FlwuGYN#XMJ_A?>MGc(2n1hdds}9k97A8ca1oR6L8)Kkfmwr8r2+IUZVXda=!B+iF*IG?hTl5 z(%9plO+XM)2ZKpN3;UFku6dP`ZHRFQk-C2q$`w<^C*v4 z$;u-xEjNkFXSX08Ht{OmZ)RC?-Y#TXHr(d$|0bbIz|0Us-Sa8NYg#EvI^n;x|*3Fy`xvy9}mj=n}b=@5;(F{C;BytTqXl;5t1lilxlng;pLX#M|f=Xy> z3Y{XaoDgInLg=d4%BuvO#lrrLtw^X{zDo}ce2y<-Y9*{pM_9~JqkS$lQoNma%0xB? zO&SkfQn1hbKo06IYyRwMB`34Go@JSdCx0OTo)0?CtS?>kLvjwX&XY(I|ItQ<*4#s- zc-|at3(k)2g}&wR(#G5?%l6nvsE&!*Ju%H?S?9w*_?Y~xrW(`TxSOBT+GXIn`5WFK zF~T5GT3z097daYxRHTSn5qnRB=HX7OM)TH;?ozgh_^b|X`y;8N#+Jk1?7?dNGVrYF ziPz0l7?Esl1yM}L_x7g^DkT*}y0B2B~Kv#9U80vsU(q(3T{Wa{wDrE3`5KSY`A}tX531DCV>AR z8=t8UpeIUdiWMnjv7o~CfH%^3k3t)SI*mEia0X&P^?p$LAnzOMu+@Q1 zwCk%5@1N~gsgC{1P{}R_sP0oI2`bXs;*6Qo?Is$&F+hky$u*@>MV2v5t`j1%Z_6m< zLzN&G!#*sSSa+TFVl)p4XHG6&YVzE@PTft(*o*ERR6(jg@x35cz4GIRzgekdHuOo~ z+(z=k%a&O4b5rV>ZXX&wN8N|v4m-spt(b#I%af*_I=Z(k`@`-EI}i9$H9*Djq@>83 zKz@@clzWI=PMv?ycG#iG32jv&e2C2D`YYepMkxaCapPijRpOu{>0RDy1^H^NgA4pN z$`o~4PRt`|!hqC|b?f>_(GoP>iW={p!|9*O5@Mq=xe8=3ciT5?=g587b1J!Pbl4C296A4%OA^=>Hz!gG@W^AT+3+Ra zD2gR>Z^19{5JTv7(5=>hyO<#hqbbMVkXEHGP#LLYmpc8URaeW!%{detu&s}##kPGI zo?+kNY}4zufI-iZ%!W&?i8Y2t5sS~7=M=kcT$-eM8vc}V7l=)dJ2CvhFS4V0g3Jp0 zl`Zc?0)pJZbX)Oi{&7YX3U)ozpTLEOLj;EdLfu@*YW9-?%ub(X+3J(+NWEOVg zYE8NP-`s9`jcCV{@u`O-3KbkzHh?q zI93H8@cuHr@mRWU&0$kw#DNtT)+21T#Y4b6(vYk3fY^Vi%~x;?f(4r!6dcg~|2 z@{7)*gzR7@bc|PIFhEqX7TWEJdQH3&3*q25b0XUPC7yPzr|D={Z!Qy`nv9R)5C)UC zT34~~sJ-51Axe|RU_CId<@DfIe7O3Sh%z2ABU2z`N(CybcEWSJZ?RE13Sw7Y)v>c% zF^kt5OCV1(47W!C=_iA!>GW56y3B9lO~$nfawuR;rr_IdNt9QPI)!BsRIH9=oEjgn65Eil(qDfZ9zWAG3xHY*G|XylIVcoa!IbA zjwywEKi*|?Sj@oh>QSGsW*tBP^Dpqkzx((sz&z-LMjE>GS~cils_%{FfJEQ+?*m7a z{DE25@_t<%3-T-I{kJLL85EJh>k0+vujbtUwDum7f-HuK7(>J8OCmpHe?EUbBLF_u zlTI7cwX3;!5t$Y2(&RAkVMJI`<78pL;=$x@ieiRAfj4?Wa0QmsB%ewxp>OGks5wYb zEI$nlNwD0vJbQuvQ(P6~UwV7GP}sc=i=K-qSk&hh8tKIUpJ^JO$+>lcf+VcLAP>0EZyQ12TI9R6_F4QkLL&JA4GJU4zC z0mU45kH4lLPkN8_E)VP-mpXDXMNt-0nxjAX>nkIalX>IS+s>hAwkBX17elApeNo(> zyRzV|$6V~3dO~t8?kpCcMpOiG25bNWqakUX`)T+61b?N)Pm=ijlMqQ^l9d{yk+|zM z;na@4Jtq{tY+^2GZ_8n?(4xCt;6;XhYPp-M&TaTuyf}N&spAdb3*J7g9n8soQ0+pg z7#3{DTEvtooOAIH(Z-4da;%1)8mCMe;d8G}j>~P{5*M-7KIg(Y1lQw=@T-v2{z_@) zlD+M>Rx7x&0#&f#=Z&4He9Ol7TBo;uHTN^A%?JN07<$H2Hvw03>n#=aT>IyDn+7`q zPcwL2=v6MiO^fFVGR#pdX#OrgyjEg3;<#+(P|;HmU&^0Yf0+?nrJd#_OLxn?^fjv? zyVLNVQek>7Br>xaQ=2||9!K@I34FJmM2qP*4qIDmC=HURn;F*7&GPQqTPD#l0>Vgc z&m8jEzsp*UHyRfGu$`%E*S_vnB)p?GL1ix=EN7zT43bWhYMW}38qN*n@)WtMb+e^P zG#ZgcPQgJ#KLo7kHT27~G-HV_Hd?Vcg)bi%T~YoZ_P+!MO!^HG19XWf#Dx_U)dr`u z+S!LIXRJBLY?|{(XjoF_DxWztQ{NqWlB3;*NHv-A(-Q6>KW$QWNYAAWy!h{&Nl5pA}ucr>^ARjIs!+k>eY z0G7p|JIgBip{6O`LOo?4c?3{VfO|Zh;suP0Qo0rcnjWP7EK{dclESw#R2MM?8>FQn zaZ14^5wYM@0c*KAmAzCy)Vs7zrJDC@`RlU!sgpxGi|6R;LBU4m>NX|gd?D^r^l}{G zdtVeNL)5;EFmu?Q;>3SCGQ)53_1+LZM+x^(vJjzpf*ad`>;ypZiZfn-o*Q!Hr6RK= z8nm0m8RMZlQ6K0kgVP}eR&YySBL(Z0_p1I0z2jI8g6a1`P+g2Kb61z59ou=(t?zmr3Qzm) z6=SNyal?G$X~AQIOjkFA$ay+ag?%(2I=v%dMW3TUf&V-o@iI)k?3f%?@XuonL6Z80 zx!K|YjCa>dCp(o>zkPgxC=)-8QuJ9ND|KzuK3FUjG}R4S)pB=dCXi&{-kih~yHaZ) zWV@IsIFr)SS+;|>acieMU#B-a@v@R@30GA+_ek3_=5TD5Y3X&JsDr(~SM1k$n~FF& z3cyoy*xf|-Fw~^ePTuHVffPsOs_$ORjb^g*{pG=$b`fz zwPB?zMVLJzThK=2A83+bZxC`?90PfUWD{I6bxeIzlGA`)sHjTMgBe zI#S`7W!i?HAk;&8R#N~rOo-#e2uxsPVJ=s|WSv3^?|oxQ>-pHxicM@Ts+?n%u1;0c zG|WNPp0q*d zrM`izRes$8~t9g z_O{sr4$`DN4Z&5U<&CPw$^^#39o{e{d7j|yOPDNr5WwDYy`5~0;hl=ayNAZM9c
@=O1yda2J^vn}7XjP2m_NXIdc%5oa#dMjW`>d;zSc?l&;-;wA(JwiO!Qu5Z&Mz!>2*KQj%5Q&8|0x%TqLH`hP8CGbN)$z?~TBhVDLl!~G+;dvSN5KK5+ zuQN}dW$_`;$eFr{h?66j_Nj85Lok#S}?2gG@Vgu7m3_I%`|%)JH*ZA zwXw^RVtDt7L6R5>>T38y+UqLVzI{9O=qCknrVX$ehb#z}Ei|F(JU&mDZN;j0!nDXAPsJqglsx|n`tDpMK%UYxDHQ-wJiDP-N%5l?jt2#TK?IbtZ zl6~FhW#<<RDZuCg#lBgaLf5Hh&<&tF3 zOKB~VfU+Cy;IpcozBTJ~Jy$PBIuaNsy5AQ%5kF~u9j`@l6Yfe02-DszG-710(~!p; z3oaKmVIv7j?)K)k#(BpW?f$79@rash6H9{c!;T^{kQzJhtWtklT1uzq7U9UsBtP!6 zMO~@KW(ZZ2dyX;x^UeD^ehl0!cQx)^L`ei@qcmAArWW&)1(%|sCWt7&$_=Q|B^MNY zSF9PP8D_f)v41zwe?+lhd2>dT)Cs#4&Q_`2TjB7<38nmIXBbHQsFGMB9f~iBEW#Sv zxy%Uppw7>nr=$w*WCjV5i~a=nGo8&i>>_G8Uz#RB1JhB5X%;7Iw$Xb>YMf@o+~{sh%MAZ0NyIeqRjKi0 zxl>$g`^oV2BK`(pm}nUV8m&?)9&>;)b978LKaL2+Jgho+RQJ6vtftI+p#9z~0Ekt< z@;VvtE_lSjc>Fp>%en~Y{E3yf*mr=HH3q`n%Mn#&`B`|?1~waGzYqS47gWP{j|0VJ z!|UMalCvJ{OP3Of__oY6u?6Yrofn!dLVhyBWL6Tz_`-jj>o zmfH(*x9m05EgcFNs=J>+faSf%*Lr+pwVBnzq!iE1=QD!+8$ zLM5BKMDT`|CO2+nyB>0fn5BO>tOzmNLAcWXcHyahd`jmfvY4&k!!frY!TmZlxtix` zvjT5K(;?Y2)z05g`xD#C?O zd4QuZKEd^eBN^k-Ae_zOJ?3)TOIm$k+$b^T+CJ$n-+FDGcE(HZkQJ9p53CwGbBan? zIh{Ikj3)Bhng}auaSKO6-%CYa_-`kRSp)1~N-Qa~LBRQoigouRW2Or>QGlNyHI%&w zUaMEn1k}&E8*!3=E>WC{smf)|*GwnbF~|w|eyxY?aU0&(ISZ`<`6_z_r?-dl4({6w z9H0Z0yoBAOy3lYA_}*qUsgPS=V+On^9rsI#k8|f|-e-!Djw_yrTeRd1lcSTmp!f&` zhr|i1Jf}P%BXPsTnU-E*V>paQFWKt;V1g+wqzJIWwoip4vcEe;Q<+5NIP0T+q$A~R z9#G<74~ctE^an=7r3U4S!u&$;MrgUNtR&HWDmMxXE=uv+A*kJB9+@fcjysmm(h++j z4-hySLfB)d=o0cdXgj*qmd*5CVPv#7RY0pX$}q)7DRFqR|ME>`aPb)b$2T!RGC+*i z-cU|Y4a^6c^47TIK$Vj5;ZqF>X25tsg51p#Ft~t9+*Iuu{zR{l4Y3mZeeRZ4%dFj& zfxbl-K-PO0LOE!MrJwxmqwufH_Lx2%T*KQ>8rlF&YqD>v5|^|kgDOp$?`~XtazICfD_@nMs~Ia-SmUri&%}`4`T1jH&_oun#{6_Z`spEVfSkm~=kKTrwz-rG5l70u@$L-W0IjHD#e; z#{@-O$o{otK{rbkb}hst%T z!@c0g6-d@{zQi=4k>?RKQ@Tt2y6qGd6>`%h5DNh7csjHI2jRMOOb2cL?hfo%1#=+A zcTyr14-n7dD(P?ABAS#zoY14E#o{%slJ^Uty1FTtSpuSFOo2*lYzw57neZR_5D zZCkAhLs2X>A}e%rs;Rq=^>0zWP%VYGVmS!UCNcNNk52#oTcL0e@*z07#~Qb80QuxX zb|)Y9Rd6991PNW@&F$H1am)-lJEaIy{lU}hIH~r11OY+n7ROhzawxO9^6n5az`dgv zDJ2bw+R2jsG~xMmfNT8J*XHhuom^7Kxct;V5lvz-mt81<{*%`YCv2YYoWC%Wr1eZi zW^hhUw6O9w!`ua95sW4%_}`@9RIazaNR{uQz-~WiPbgSK2xRIFtJjKVP}HHJ19o(} z0pi2HaH&mpKpbJW8swE-R^vkt$f)8IkDx$|>$ynmi2bMB-vf<*4-q?}$c?H~b-leR zrQ@&dmasdYQ_z51^aa||w_QuU-HXt2Vfv5x9F;&KAX$!z&c$)YNz~p3wDsTUR%6g& z{&CUlKCjMNr}ln_d@5e(et+8fGy!}uVPwo%uT;)mnuYs{lF^0B0Whr5Y25VtJpk87 zviuD(3CYDHih$ckN%eu+u#lKR5NtN2WY`ezg4P_jI>)nAm8CdFZ1O#Bo5fqg%=5t; z0()vjwHZaqJpyST>a*3z{#Nwb6rB=_X|4}uw%fiLfkx8H%WFBhQ2s+hEx(3e+EkY< zr+uQ@uGs^rF*(QXhX>13T^G9*Oup8COHHZ5q}vOQ3I5FUF)McIbKn)E7T3w@c)T12 zFNv=^eF7dgx!2gtA#-1(6jgWx! znyvbt;I9)7C#4Cp7AQ@7vW-m=&$~`h-rIIr8WKr2`u8)ynaO87-_s^n^=f!`w_9za zDZ0ngdFkP7ZJybk*PZJHTi5mR*88X|Fn=FIv6rvM$B2fFB>J@IWRf$>-~!d71xR59 zYhvk2j)y)|u}oR~cbA}&%U|O%m)=aj3;sITCSl#vGC{gGSps3YehalEH_qHeLhz|C zeJsQT5p1LN3Wp~huZtfxP-(CT#jx(eHlo`hn$!&19~x+8bnr2iMTJCZ%` zGE!&e=*$*ooP*@Q3yHJg8t0Is7E?Wpi*`P>o{esC+@Ulnkhg*S*+DX(O@LH5+##IN zE#_Y_yw9lPb#>G)%2$iEtBZGr9zze9O;BpOJ3u=?r&K{q(#c}y-}r0nf*-MQfHw)l(5;S_N^^@{5%1fqaRYHQF|FslXe+)YfNr< z#XRvxOt~CU4rI*3cHfS=`)(PJQd!dhhwM|SGdRPT|2+Ta#Vi?q=xe1S| z@m6<>){k?V^5r{D%IwW-zOO1at&gp*Wx&4BmB(Z*6k?SW>$B|XhcgrGpCfu;n2at> z7dpig7Wma;p{b;P)YE`+y>SMZ8mSzN%W3#;%8d))NWJbt!(B-&154O4(}R!yL4+)4&)(D3&>nVSc-4PbGWdSe7P7)Yf1F+G|l ztw7qL2O`&)02Ox9xpf2AxlwcKj(c%bd2si_rUfOvKI2ddUi{EdRPtkn3vY9In*${} zf)K*c7lagN%ZTeSq@T}AQm}4}{dbs_mt#=N=Ie|HUK<(0k{{#Vc_o2s8SH0q_ZM~c z>uLNndGMh}_2|-aG=15mYa2`J?a6r=;P$%(BpYz6kYl~TrWGl5)k+9B5_;-|W$gWg z&U)J{jK<)QKn}}Z&?^J11O;LS+n~v5>K!b}*LrF-kP<^%n2R29295aP%0gko+6hc* zT(dr`SE)JPa>Vl;wS8n;yfzF4z3~5?xP_{>_9Glt)6YvRSNRO4_g3mstsm1d z+v2m%q>UEa;!xAqM^y#RaNI(EG&^(6+t#iXq30`yF)nlcwX$qyT11lICxajYT`i@n6F_1!w4 zR-yL0$=-)dra!A%Uyt`>xgSTCyFI7OsI$tFdq^pO)xwY03$7N; zqebLU)v}VDT3OfIy(+X&&##VV5ThE|Es&0Xh@qP}^I4plNUh#%pLDxwXo1}SV+ID*=;zzGUwoSAdQ5y#*n(k=ZVmbDY6o>z*A$;)v=*we zQ_>soca#6Ad{_QdwdHa5O5W8)`4a>cB-&QM)~U+BNdEo7cLT_3L@WdVaz5;)^dSFR z1t(i=NjxoS+$!V!BM@QpqucZODWnPZhq);dC}6Lc*X{2#;I?WVLCfhN@t}@ibbgxq zZFI}y)FW^=!AjkhNGVNj^2*~YTKJ@*qN%yW?gxG@@dl!$gpkd9q-gioie0LZ98OOY z#AOSIIv#{FQi}-5Dij*~SNYvZwfCz7R{ljvv5F3^mdoOf@tGOsyNK$KCbP6cq^Y|; zH?=`bTe%pr(cg-^jz_t}yic$i-q+@ze~jmZ!}A3XxMkzZyM9rFg6KM$ zH~b7%l6JTl+Rwr+?*6Rgs2gr2mC`?OG^B0W3o3si8E>*iR~kg>wnZ=}#U}f)=*kGj zFLXsE$<5;cX&Y)Xm=1sS6@S&fclCZpZv_s21(#Ln|F!e=aSZq7*KW;oTSm=_9?qD3 z`Elry{X9V|%d0d#0OvXCR^e03_nwbWvqnRg|FBvCqAXW@p@}-TOcC;j`cl_SkLX!K zbLK@^$&&x?X)&P=BKlfIQ=YnQ=Ax4U2wi?S5|rphNJ*)w7@9Tg8oQgj_Eq~u)au34 zk0EpBcrC0dLJsx|d(>H@1uRcFdPS&=59W74+R&qM$)5klc89o6r90otFJ!;|iohz?I} zy%*+l^xt}2uxo?<2!9{Kw>~o*f6hsPcHc(>BoWT@;r*zixZSS2st+6=K}cRK>2$j3 z>1gS*B{g_sr~bqeY2z2w?B-fAc@I*Y*IGyPPflLYw4Ey8f6V1)zmx;60AKhznpo`} ziS{du?Dg&N<+11wQde`_*_cML`}yZ5={On8GMzfsx1qCjOhPckf1(ko{e&rioT=S9 z0@*5DK4#K4ZbF za|tn_1c$f{D^0+PGdt=jgUoUrPp8Wt*TLQ=S6>5(bWPpazd;n^dWpDn_LgYE!bpD_ zj%{!q_7oo+wr{;|+jxEG5FS0OI$C&0rVOuJhqAwf31zO4golMFuJEls1CQm66GSn9 z7>loy&4#L|9w`$1x_jDinBN_E{CW-7IVvfpiU_OemA`xr-7e4eDzeJXm7C#o1ScSf z|3jT6A(O72l=1)y_^|l`08WH#yoN2J@zz#CSn-2(k&S(R1SLRPzATvVL70Pq>H2!u zwBByJCrykiKhnsBcC%`;Acl3{Z7}J)!XtrM&k}4oPEX2pE^aR-E{!)7!<9qBlrf6m zCFSSjc@l<`eNO`U$c4CiZ=L>l1#v+{giVuuevZ{Pc>&JiJH)B9f4G!6(=;^ zg+-`+Q0I>1Kr?@Ho2q-=H^}}&LeNw-xhi%|O<3d$QU;8E_q2{l}1?$AHtYJhdCq^MOgWp%>-UN9(GW zykP)jE2qDvDWb3YvHRm5@Vv)&KgCt;;r-_ER8eM&P$uasYbLg49KqU}n zmMV=OPjFTD(`4_x&%O$a-WZ{NT2Q-;L#p@)#ySz1Qt|{X)Zb}6j-~Et}!gShZEzL(K4-TVG7uOq`jI9b;pRLGG8J*#Q_*IGVf zq>qbx^ha5L8t9{SjS2iYhL*3(^=0{t!Q508 z>4%@hjpgQ;W#H5(xadj-JqZRf3E$qbq45_6S6fQ>*I-6~+cck#W>i@#1pyS^4+m>EO%1`&GV(R=u4H1OOE zW!}uqBn7UA*fyra^r`S&*+rrxxY6qpee?%*t9$j&^#)9^!gl6T)Ljwm&EF*{3U;ttGv%A1J#TQCV}tM zjZqD{X{v5Homo6?d&HI}xd;$sx^zG6aKkU%awh|zv?19T`5{d_!w4!u>%D<9uR}sk z6gYEjdd<+2R^mJFB3-`T0Iyb}@Q?!LzsSN_oB9lypqumVU1KsJ1BV#4T1G?w z>1@A{KluQ!DqlOYU-cIvaC9XGlMz^D4edfo0gRZWxswoc~6>!!wDg1J( zLi;k08i~v1k7We$(fK|0(Uoy2)qoVeMy{NTY1NoEv2^*>kDs!6ZLVnzo}&x}!I7R& zg6h$dF|ap&N@^=0_KmN!5Er*ouaU2jUBO7Ujq!+0Fq(vB4*k*=oE1>gx3m6|7^#HE zX#D`x$X`jAFZ0U+VtU+SqES(k=UIwdEI6=HF)ET&GY1uvJuTObG_WL#S!o)08xs}# zyZGs>+WsEY(KSgvpOj4uyY$`O+S#`#t_#K5YC3X+-V~z-yd8aA0zQ3pUIKOAPEDe? z2K{7?cG?@rTVAsK1qM#wkKGEGI3C&*5BAo(%Pn-eW#CQL@!0zVRyQs!*glnS5osVE&q>Q5nr>JK|{D$@*mdOx6 z`xDy4*QD5j+KcjWXAP}2RmMyI9*~Ehn;~qmVu`#`RG^^t; z&ZJ(Rqms60h9;^q2dl)QGdg)VQJTzLxvb|CJzvcY8?QSEu&+$VRo4D-gF3>Q@mnCJ zpT)KLx1o<(x)sAOJ_)M{4)(GxGRUfm=B0Y-qdT{>1o6fRb<5xs<$=}r_OB_6BEDCQ zt?QF%z==V&_xr4g5yC*I!o*ecend@BY6G10Dhjvf)@=Kw`(N3wjcU*L*;7~S3`jpd zNdNAK!QHRtvEr}F=D+Ws-+DBQ+dj*?QJ(lwCh5=cXuP*PLS~P(Qx@^-u5~gqS4*U( zJF?V5?3ee^leMfpv?i~U*Y_1C<7E;5*0Z7Qp@OsqB<)u+xeZj#xUgG%*&-k3c;hUU z;Xrq=QAR?3{_x;?d!YyHI&6J>kYnJW zh}L`2Y!jo{CZRPBeYElP`Xk_U?NxaIZnM9>@ef&g0#zOl8tB(D@$|<&8(^=}1~3lr zd|Hcr;E6v7ZJ@ChV~d?`f;WJ*9$n+nkiC)&D@#+EAAPJFbg!WDLHZu$aq6YZbenKY zbCA&{6KG&cNqVvL*6z_Z^?_J8<=deD8l zM{bjo`n&HGaE;8)hdhHS!cp5eUS!!+WT(btQ@@IN=YB~4j0owy@BcM>HLd;Bgzvo- z3M;!vX^>CG=^A706h`f6Xe>+gNDORC$=`i`X`>Ai=X0;qq(y6S%5(7E%zG>QM4$bk z!uPgSi^@=pW6r;1y#cbDhk=C+^fd$VbgnM+4&+DJVD&+p~+GbQKelc|J-gh z$#$!| z{gLvyC=vlEeJE+W$?tw1JhA_p{+QU(Xcz8UZ-+lPqHJYTaP0 z`Er{5&>AxP`J<H2tPPrzMbQD)TaV zC7G_G{z_Rsd3OKLP%W_D&&bhDHDxQj@U^)7vO1DoQ=;Ei@Gk>~ zLGg{qeHs-%z%*%hPqg6eBzHu+jAN@}vB@{=~3sHIOSq{Nq>XE@z*Pej98(qZnKAfxK>L#MTL{5Gc(@#dY{c!1>F}qP z%=Q4Yx(>Bw=F_K0AOvIDotW95D-;aBF|$!`g3ey!|iwxsgi?O>Q9ti6QGSs zp^3b+M=vB-Z4ml; z&^hcXz+E)iDm~xlplpK^=8_~l#~dbK9hlj!uJ5iIzM!|gAc_EzxhLI&{za_dbadX@ z{W!Sh30m4IUaNld&LGbhs0@9W(HhxXF-i=w@!u66)Pp*xRWr=s7f=C_l}TU0#C11b z4>)$I0Qw>Ja@mFhzZVdbUx`++2df<^#8nR@T9&+SADjO44R@ASeiGf3M>)y@8hOLR z(gb*4j{Au-M2ms})vaw(^(0$NsfBMkUZ??PU!bx9(lB^KzDE4ScV!_yo~QfMpT{27 zfQPHEeNos3t*F0!l?3_M>zlS);7&?( zqqL!9s_naHF`Pq$I&yvrWTNnwxxbk*50q%5c-SV1obMDRoks>$nMGQl2P_C`5R0|J zD2>$#U^$y()Z|*(+gz+{(5v)f2OZ7to&y#oP{|h`Np!e6d>56&^mH^W9oE(pm|E7r z(|K7Kor%DA@~n%*c~?_3l@g&YDmaIpe1taB>a>{klCB1KEn78_)PI5ud_RpAO08^1|NW?4+Yt} zR&TCg1HAJfY(17?f9*_vuDQs7Y7{!Fk;uJDX?#wx;Q>A-m+=c)2J1!Vk8QaPvQdhh z2A4u3glghkt0ssE(wOz^k83c5?bS3xqU`4yo|D*4-x#O%C77scEU8X^v-Hmz$tBp4 zsQ$s}_Zo3H|5^i>UbHQ5-e%#$uI_`Gbr8yq9It_^M0LtwIOdx~d?4{y1IDW((BYiLRS zDwf5>``S|{`C|v*ssifdNpOH1^6i0O>KioES^X1o%K#~2C74GQ$Ln@`ev#{=MYc#~ z#7nr$b}s;rJzFz<1@-B` z1^5|%AJI^=N6rkHmbHWSI?@rdIGEcDvjBGfWF?QIsLmat#0lTMEHUd)H%0e)25Bj( z0fXyLEYPq^m4Rs>?4GaB6Cy}p4fZWV9~*K zi1m?#k9*i~U$1q)7JEOc@IAyv^R;cW|FUJ$LH>P~KLtF3w1qC*E-M$MJbc z1MocjJ%7f{s^CGOcM`D$o3wuJGK-$C^%dCzs>2WYx@VD^!ZW+^`{Mtf=ozE}MpvpZ zjLv4Ca&~-ovb37MQTUZQ0iB^Bc6GD>t;`8e+w@U`VIjw}r44eV1(Fa_TKK zk<)IQ3mN>qS*s6AS`=&TDen90^Ak2%`T)mcWKKmN(Z8rms@;$;# z*iON+wrlu+IZDBRO1krAhUt*G0L3ZbBgkBRS<-`++vr}&QIkaoz_9Xc{;aS(T1dg; zJtsTM2%iK$qW)GThx4Y$)^?(in&F-5lAqTA3#JKDJ?Bv{HANO;Lc7Y;fKZMWod`QC zfc1b~q5*sL5yYHIIy2|{R2f0+Uq)(OGUrku`(6V4!JA6b& z2bGEm?K7-Q4D!)&GSWp@QHLZ8HdFe2 z;Si_XhuJ2=80pK1F>dt-)3&A3h%y$jQTxx|l&RyYFhpylMjSe&L+;7uo8@FfwbpG! z;>QmufAiRKIo+_Pl$nAAzIY0n9^D`8kZi~xPh)iyX+$8^5)yyr8Khzu9O_A?Qe0?l zJ6kI16Mn>v4SMoU_Aa2;78%awrN+?K#I1h-y&gCV+7e-iK?&C=q;Ds&=F-xa%rJ`^ zf)=9pK8utgO57Ttk~0zNz0L6 z<^4H=qL;+kgprX(5;@-M;)N1`@BB>L@0wNp&DZ-yW1hEF*&mV7Z zi3g8;!%JmgK82Sa(5u{0iz&1FgKgtQ^vg5mvJ}{4)k19nN{O($AdNO8D_#fRzK;3B z=ASg~K>1G^mppM_`v*S!Cz4A*Q+n(DGA0|A<5{kUhSVE7x8uqY=@ek<>`Br*HJ<-r zv9~QrVod4>^|750#TddkAHlfVW1y`Rr&u?dC~arxvyCf;R`03#m_GjjFYE?TD=uck z!%z?C8lTeTAw~v^3mNggcR6i>Xs0R@CN!$}v)}(n3WTFDYHx~#*YbIt{$!x}Ub*#- zq4d%B8xrK#@bPa2aei3q9%usU2?j8o1Xy5Gr!|?3sRY5`7!i!* zU~Irf{h}amqx@b?$zVTGfU=s+~=9;%ep*<(sOdia;A4$`GZ@ErfbXr$L$_*VzaKqtY)#)p(fQzp-0fDPzk5Z<@wg907}&LL=! zJIjqf;F5{1Z&MHKl8%hotLX-bM5e5wJSrx3z}YLx!!Z-_Ikf_Zz!5~gSB|1-W06r_ z@7490Q;+te49atZB;oQkxwOSDO)o5ZehK@`)0DBEI|`7oTy;=)TkE97v`>|U^(-(( z)%LZ83+0!nP7Db$`Wn4Af(+1*xv3|D>B3>VU#n2q!mK-BX0>yocR}6uufQJy6@d;Y z*%x6%>@nI}q)Hp#Zm2-_5TK!(^=SKjVeiVRU7Y+o8+Uvwyl=RHRAJ2@aKsCg; zBSZzueljLaAp_-LD+FMP{tATK5T5kOaH|d0U0{2wI^Ze{kwJBJ6FU?-8AQ8P5||AJ zL9X-ysJfIN!-0$3)&^^^B_!JTcCyEyj;k9dKN~1!)Kv7ZrPP0Z zeSXAGHi4*VFOKS;IqcHt%;6_Kp5*pTN=XKw3yMx*R!EHE%Z6})@OYHDnbAZU3rM&Y z*%t@M*7LCG8J=C8Qnyfd5VitQkz>Uu##k*#s39TmjtdH*d%RJgyRmBEoB%N-76feP zY37tyLk_~0{2!CK_9L7UaTj@z$bIo;70YBC-Y-jO9KkG&fg9(zYt1)LpF`Q4QKOO@ zGJXdIt3#|X8ozdnU118~bMxoHHMqiHK7DIShSG+5#kEx@6ghuT^=+T5EM>6zBcUow zmKjLoLV%KnbFYB{;rv!;@l(~%_@URI5)ogHj6+AvbY9q36Sspp1&$KNf7b6~E=N~G z_3i1HABxr!Bmv7ED3_3FM~Y2h*sl-qat&|eDSU`apsWUjFGC@zs_RTfTTVjMkEaHV zEEBRwjOmx-8-BvGEI(wMfH4ET&J@FeD2&*H1K!jFvWL^b2u7nZ<pw#BQ-=?DFb)?lBo4!IuM$FGIF5{5TiOl8~Be548i_8I;X`!di`g!=> zHeCKU_3`Sm;^4siE9oyCn=Ak$ zH^Q&O^LO9M!3|oxtj3Jf$~8?U94BF0bQ1~F$VhsGyHYq&zq^{&#b#p#bM?=!_K{l% zsN47MmW2*aZ~m8n^Wr3#u@g8EVcw#K#Y_bBA(iv9qvN4qyta$#_ml3@%fo+jxC0q* z4iJrd0p60R3>kw!0)$7k_wn%OZZqI1UKa3DKlpy4RyP>S2ZUpbR3zO>Lqrh5Ak2cm z&=V>uY0_krwZ|{VK+2fyR@b1a|8Xb3>`H?>d7wkJirM|emK%K_Jqk+@n&bqgE0h&< zPXJ^o%AW$7&5$w7Wy;)rcPr|kzT2Wv|h3Qfr@lhOh)Fat$Q;p>p& zh}dkF2!F%^2y`_74rVCFKSh-McMGGQ!#G z8Xk#R2Dttup!e5(;=rk`-SxcMx0ZL?=-Gq!#6cdDG0Klf1lEeyC+4+MBjUbC#!ZD) zI#qhP+UsHw{~1M~O?eP|0dz6{RrfdtxC~RSvya9FRU6J;)aPsGp9Sqray_SWbJrg{ zJ^0^di*LY66vvOVQ)9FVy6zUV9A(AoBvT)XKK&sPQMYoRHJHceeMD5II(-^Q=qpo8 zXI#*}GpD>;rEnt9U5L3;0b0sGqyHb^3rm{gK)+$_vfSDtd0|>y17W0QY?(oSXxD}g z9-fzGY}x<#Z~eG6&gX&NYxiERzLnVh#mx3P2z%4*dAqk?=W~Fb?nw6H|1tHB;dO=E z+HkO9+cp~8wr$&X8rw#bCRwpvt^Y@z1e8w2JM%Gkx^?nMy z^{ENVaA?`idaI?qMj$KJ9gGHxRoTNZI>IY!-dJ08)BZkbFLe|w3EL>4oScKijtu4a zO$#HX6|(Yy017L0l&IiahDeLs$rL@DT?mD*nHb0{49s3}5HY5GTacOi{99QoI@pTz z%1_G2G4WL#d&`BrND#gLsIkF97U?yLCHm_^r&vQB1wT|-V)zkPv-pc=P)CV4rDT^5 z%#4X~{Ui60S|Jh_O3|khIJSvsV{o1o@Y%Eh1=_s?4E#NY7-5!Z;|!Jqmtj;iRPg|5naUl&`kPtD=J3z3U6-coq;Ld=~9bI&4e^o#Ep$uGEc*K=JnC~Da; z)uTI}P@yxb|EZ|;04G%o9lY19Gk$%dxe4u0-s13i+x)_Ox|*d;%bci$V+yBLnZTa; zfQ$|XO6B~mfjtm$D|IY4ene--;A&}>ZoAmf^yu1wPAkM&h6b)z-bUAj@=oz={hh3&DNq{KA85r`oTrH0_~$rNYV+FIn_a zZH8G}#AvfA*~!(|zolV=S2g6lAeSV=M%GC>q-G3B2+Jk$VN#KfElQ+0vBQYl>AKxa z;embA%i(fSa05&Jb1bg6Aq71H;I zYbBca&F@6j!-q;+|6@2Q@OEzrIX;KK8-F86Q(1e6bmJx;%mb@ARycb$kgx}NSU!XK zp0SmVHhxzOg(&(wd#D3Z@G%F5lkv=`!;vo2T++eko#@R~m<3J*(RPv<{2XIXrzX29 zu_(oOxq?TSOnI@9v!i1JHgHBDgP%NE7;!dWc{^g+%@qv-=K zqYxEGa=VW{E4_QI#maXa0z*=TX7tSTWy)gD(e zpgb!vwRapt@#Q#7vx?dL4zCcj6*VA}1YaDub;^P?GNo1Y3}sH@t*D9*nbpu2mV5oJ zlBAmm1Y{BQgc?NwW|DfWjL$S*)T$}?m{rAs<-1B^gHm|>=kHUv31FirWR=)Zn0U)Q z(4xHKD>|3Yq+u~b_r)EhZyPyyGQ@0uGM&TBk`coHq~ST#ZOYR)wDZYi*u;G{tr%m5 zB_mDob6%x_8myShH6~OlF}XT)J!TenaEDU)JQI+bGWmP9L++ABfaIn|8zSdwSlfCM zwlvCC*O$K(GUYLtR8@t4o80|Yks4hM$_bBLrxLTfp(*RHFopy*(0{i z_Mv&3TgUa;jH}nK393hn2STk>#zQY$o>%1`+3>4Ie`N^hvV-s#MLhMTr>A;DftAi}76 z+Mo%xV=MOuMJg^K+qgHRPqo)ICTsRA8QoFcFCVJt8Gs=Hb@?hL(%PJ^WCfBJ+pN(pzeZsUvRhW& z_ zd4vx?CZ*+uK}S;BlMe0D%&$DRcpkr9i#UApwi~?j5PYt)mwv@(p-y8>I(4*0FB?E} z)N~wf`EI`nhocCb;1j3IC7KmRaP9xC`&owTv5cFeI z*DVkK*OTBhE)__j%LWsI|GDFwt+_}l-2hxk{XwB$CCkmCg|wEodL^b7_#X{`feM(4 z^h{9EjPog?idF>f(n1NivZHH7w1M+Ya;iVz3wHqZiR(yLJ>zzk%Aw7lF%OV_$*dQP+RyW_LKK=n%f*yHhd48)^wnO z&mPqXLS2im%Y6sI5kcYvFy{=Di-@jQZ-4owA^3360Dg5cOLph*r-W!h%tY=Onz@WO zNY(V6Ud)t6OU9B9y@9?gNEswA5;iy#BLhALP`ZPP>`S>r=EoPj3KE$1j9mrZz6MU; z0ET-+$#I%X9Cr~luVYHJv-B%c4SFm}Bw-u=TH#7ZJB9vk%+g*wF$eZC8%{j!J~Tu` zn8Sc)(v}*yc5+NgG^fl5o)d~Ajqh>g3Xztg^(s5$yV{YEWD`e@_@2D|&}PV^ZGaet z4#Q0LDq{UWL1w(|h+ZRdEy8>r!Ed0^%J$<1Wn0fdGUa_V+wY3KmGAG*FOPfzhdZiy zV}--MTo1(|4wp1V#Ye!C6u3~lLTCkhR0?nZ0NLAXR~7jw+!@97glr$UX00EA7vg^y zF=m(?hvhjh%b8y3B>s1-j4e7!qL)l3w9WYbdFR``zlQXVtAJ<5fOpjd!HXJvU>fhJQC6(Gn={@@ZO2ywKT@ zhXrz|3UGoM8Mf$l0opY4VddCeyM0s#5ZR$O?CWRSM zT#JWa2^uisK}tJ_>{^udFlIIIU>EOHt(t(IYs`jX->aC`NhI(Szj*Pd&+tRL zh3EpI*a@nHt5e~V{eS9F7hrAY>hDEw@5)_V>$ca^Iq}QK2Tp_VebI!0l=^6`}2QA{BfQIF5${Ij?OxLm2=Ov0n{_20qyYHITZkY7E}c zYNbDpoXH@rhCmSAAu+A+1ehfZ6ZJAIEwkov1cK*;o7Bk0uqc!zM<7(ZfdNw`!Iv0_ zj1n5D0%mnHr&o_dB7_daMg=!702|@qRKm$wziFlE@9V1sOi?>Fv_KWsZu3F*y5*;WN{zr@Es{Q$S`mvTry*_xqXh!hnF*Z5mF(!iUU=_Cj9*!f6U3cC zHjDe7%q<~=e^VeeP4aQ*!LIIqFy;<*R>>fdz|IOCuG@BL$i0y=Og?k>Jd z0AVKMVJ479{!E%{yLW8xzt2mLgc(BLI=5&z0mbE)=L~YrVy-UgxmIAG!VeLC{f#Ey zt{AB17_SKLKDj*|2#%Jwdo2vh%4wPO?Pj1te5%?EYvJ2L(s%qL9^fV`aS=l$Aky6A zP+$dj#UR|O$7aJ|H3~KtJaW_7g^$EwxOqcATdhdXD{B!WkD@ok6(Y~$M8aq#=}ZDLR1>P;<%LjZS;L@}@W#|3qd z?q5$?9$EH@*22|qI~U7k3;yxlLg61XH&Y%4^!(_Oo)3b) z2&ZbQ==0NIndMneHvL=XJdr?;Ax%@Q!vS6$=AlDuN$sY~6+Bf~EnW6kiK4+ABRg_S zkW!51AWO-4FOA2`xpgn72w2H5prFEEpN8BS^g6Q2{7W8oGrhTZx>kS_K5cGuz%lQZ z9&hWE-dc?kgIN`9{qsJ)=eTYD6=|os;I!!reVlj3TV~1rkU_ko{;!(g*H`_p^Tz`h zy|k~mn;eGGG{962dU6ezLYyH)?bogCJo8ul{tmbOk*nA|?HAzhZHI~O-5Glgjme=tg(Cxx!ZMoNC>-13 zq=9vkNkMsRaW=SfEJ4LdtHZ&Q9(Ao#LdU}l%M~$=(FZyRnIA)piN9T+0Pw{+Ppm^G z&2lxfJi)oL>#I-NaNf#mEz;9~Q^a~JzBFfxjR>!|;wfOerL;U^e>E0kuWI*%<@se! zRzcSoqa0>+_cm!Tw2wpJNF8!$_x%nzG_Z_E?iy!>3NK@#Z6{1CSWiFOPj5T@7KxI3vTS%JdESb z3B3ev0<70e8cTmq#1fcjJn`o_^rMed#C=ygIg@af zK;SnE#k@ReDnAuv%DMruh1BPWrPD}Nb$Q*Pe*@#_vq0-#2M8qZaFqf zDh<#%t6qZsd)kbw<90KEsjo9fXA;mVsoTDC+W&2812sYO{naHx$It6*?(u29!ast( zNreSp1g=;h%eT%+*MruNn4Rx<(MtRTJ3ZSgUzFwNXjj}%C?24I0?QB%6m2p?hRx0d z#81B~akfxTZ|-2zpAp!d6aC%%VLF;X5f$@Uz<=hI0GP2zNq)_J+)dYp?}yc%hTaDi zd!5_QA&!1GE2aqy6TjFhf_>u2I{ALkV^McHi>y$hs4kaT5W$x+oSajDD!o&O=W)6R zjDb0|lcYF@Ju9^S-f{RtV*D(U%8=XDS{?q&*vr`tuF-F^ z8)V5zRqlbG_cHzb=DF4Bu^poRm`yJ4TyvbreVLuhwzJS9%d`91nb9}=Pnf3f=YGWf z_;sEAhb8`MqAW(Jm_2mV*QIY52J+~^AFuR3LrOW0bfp{O!66l9N6h&_xeU$)N^pEuBXU=VGFh+ncnOKg}Mi(NMfvH?05=jJMX<;*c)vH92sY&|g7t|~MT3yv@c%8S9vfW`RWVLeo zQ?rAo7(snymb#CHiX1mW|5GTqQa>!>q9lZ%_KD@19Hh=s? z%qc_nXWQ~1E`}q?Rv+pUWQID5I@f7a(QaZU(CkwOj2ajM(~6}=XB0MHB8i?1^pbMn zhw8Be?iO7Ah9%|} ztUhGp^+dS49i9;BXjTtZhd>B(U&YCbUtdJLoTYne>(IR=NG5`_kJXQ5Ioy8px<6X&4=y{+wIR~7eySQNTVArN(2Xll4|5HjnL4W zhlcu91UFu@g}8Vs1Pfl^GtpP0mjwWZKfvWF<8~EvnHa`xH+!6+{B5|ZGJsGIwT!>q zLyPN`;GB0nqmZq`&Rbz&hNttv*gj=iLt{8CiEorf5reps59X5j7Ck|!C$Xy0W$7}E z(AxDOZ`-tvAN=mM4Jq@rP-HY_+sjhEV_>8obv>`&7Rm)cXey>vq=#G=%LO!D( zpUU4jg}M&DDxh9E?tbHKiz3%wKx{rSVNHAtbf(%sxq7b493nXGj8H{GL^n79oiOEL zupnZ&J__cohd1G(ki_cW)P!)J5(R^H^5Tf@9QDj{ZXFXXfxV=tBNY(iI8?*hx)bzy z0VaqF=MTHVcimZ_*?^vt@_O`AK#A9Jim|Yjp)4WvA$$_G1#@v!zxUjVf(CF?94Zx2<%Q^gh(4a*btDAb%;oh^#^OHGg_ zi|UqNs4V*uFO6`c=z~4srQ-EDZ{C4g;JbIxw)8q?VsyX3pFEUirN=gwCMmzXD-lzY74Z)ANhRH44uX2tOGo9nO3AR`j*o+?2I_3J@ys+&Y4Mg&(q zn=KeI*@E6P7Th4YJ=GnW*HE_E3CPa`mWPCYN1-+cz*cjDPVPkU$_|bz*q@k;oZ$vl z`u0e)<+;8YQ|rImY52SmMb(=k$I)?s&TUQ3#cfvM#m;v8c&IldYti10j zSQ1rZ@brg#d``w0zIk9Hnn}v*PaE9S}3OvPnU(8Z5K59CJ@@WvA zN4H+=hsK1jLhH)k_hV^{9Qj3djLDzlonlw*7OA7hX8*%nnMJorJ5?cugWu^_rDM;< z!?=7coWIwz~`&udz%bo3&9ZDRsx zh~D}b5FMjVnu_U~Z6#fbBH4$X^E9LtqR!cVDbF3lfKciFFBO`ZsLIdPMS-e#R@_l%bE@3t0|KgiE`=&Ra8?Gx2?6wU zWQ^Wv=V4JSb?vLf#<1GtKKRd(b8SFIQDuC=JVJXLDe^o@DLM;sY~6}#cUc#0=A%1W zfew9!bUM2^p62Y)ojdC9t=l802nolRmmmKr3iP0{IZOpoZbN8=V7^hs|k{U7cYGRP|Lt3W2NCw1e8>uMaq8#sA+ZKyDw3JMztfJ{f zYnS82+K#jS!PX75s*9vFjCMBX!~^T(>!7RHqW-}ZX&B6wff7aY*@%C%lO8_mwdzew z;ZMGd|IPqdF#rZASGrSD#BATIu0jR@zrU+2HKoESjzrsf>zx`IaMcnkDi zeVwYb{$gmFaS;pgGj?5I?~Lb;6y?v_YC09NFmaIML~nQF{;`ra*gHC4-Oyi3Z-*AK zSHlnEDt;_?AqJwHtOUl_^B?q}-U`huTvk3SBwucos#kopld%`MbhWqomq%UYoVk_b zvro(lmNB1`uP}2kopNpCSpL0*&r2(ubR!-PIceVXu&Wmi6~;yHU-KUQO#FpCMb-db zqo;#qna@PNTYxnd{qcNp@-t4)7lt(o^kC5HmoMAkHXFVD9*eI)fn%&RJP6uFcZ4CPWdx@UnVS)4VmjJ0ysN zle+L5&b%iOd-h7aJRd_)G`WEJtXbCLso?e5JV~0@!WzS!m>qyafaRTLsu*!28IP(G`beWeD1?M@dXG?XFiH_pK?Mcxv z#_Q#D6PLQTB1qFpZx(TkZ+a=Ww|wYdQ7q4)FFE$`xh&u2X73)uOEj;VlqV zh+it^jw1MUZ_ysR?exhX$8#0ay2XI>Ir>im%qJCy7=;+K^;=L|naK@Aj$KF=S8)y7 zDVl@_ra{|jvFGpDuiEZQqw5RDuZX3u$ithf&_%w4gL}oy6v#4I(pZ9Aa|)>Sa*)9? zU_Ig;(g@7LS-1{`nJ7H9lOsASuLILz(XOSj!g|kd?%-^muBP+z5voCR8$%8WQn@CnubMed z07tg-f@3K&Dh&_qfCG6N_#b_8o7F#DJJ|k}xp_o)nIvM( zrtj0)_cypa?^mQfI>(KOioo~smLbd4z5&rU^=8>cg$UH2R2U=~>1-VvwF(7S!6Rp% zAMNvA4bddR2}lUE3bXaiNvh5sNd(OTWbdct2-6KpIHW5f$4D75f(qeeh30 zJQp}-`a}MF?G08wlY){3(+lq+^a>C}eMp+0j|YF4sW6jNcoo+I=y4?Z9tdt)L z#Gh|LOM-3ehK3@4tpar1L{~`uIK_5tzYlsn+lw`!oe}4yaIth*mh^ZyqprdYti)BW zKO=v=mUB|fQmxB9Er^*S+Iza!Ol3wmhg&hXfzg=rXbE93R?MN+m(^9i%G1j!&_%Jc z9=*vmF;Qp08l^ahpSlo86-S2*j?a#+sN7Exg7XX_-{@A+!TsB3|ntt-~5UHPtkKy}sQd(uV8j zZg-Y|+5en1BXXuj0|XTc!=S%Vra94rnWKH{`4SmmceDplXQ%M-@cxU&AZ`QlZH8VdF@D)djU(SEyx3Tb2-m1WDnH|6 zoUjs4U1Psm)_94WAKRmU>=)~oQj~#QD4|ZV`Vw)Jz(Ob-s?sx)NMSPP(75Vh*Xbci zTVP2&PP3Of8qFgml_K6Z#V=hj<$QeUN#B2xKfQ4|I6-B_83rmWEX{lPTNc`H)_p7S>_m|@clP_xiwf(A< z|9&h(;C*0zT~p89iX|6g7&GFufw;an=5OhJrO%41ESv^zCC2w!X*PFoWgY@*MNu}E z-M!y}s`%9FL10K#sSA=yZ-twUCGt+Oe1jr#J)P{z6R zPAhb^H;qlD{iYyBPQrEDUHem>g(Rl%y?NsNfb;KV(JX5|_O*0!&@M@P%s$9(6y!Z4 zWeR+>*{j@TX-7#x&5HmjW}o%2c4RN!hcvuHyA_m1#F^j#DmfkWiurmb@9;5X6T!u6 zDPPnZk~Qj${y+NL&QtBJ;Qc=pmO-rU;_XRe_uH)a4{LOAJ|Z5u;}zLr+V{VqNAAC- z7P4+cd_UkK#~hrMZh!KN2CMxlp4-FtqePHUKyBxaS+6e0$qiOxJ@(gIo^Z`## z{4&C&5Lr9^{gFn7ddz)TH#)_RXU>5KpH3!wDvk7b^>u}b!&YwmrNA!k z`js7W`e@Qga^#-{#L8p(err*(A~VOSTU z%$ivgb_(rlAyk_vr>zxTHmM5Xv1B^2GGw5VT*D0bk8HBpRP0Ty%CrhEsuH# z2tZwny2zHyHBWLbuH}UcTnwF>mQFJ*q31GR89%H%bd1JN;tVlqW%wTqX(%XTzX(@q z5Pq=xwx)%EXf~r=tZA44k%{=Z95;mp?4IZ&Ypd6c_0%uS8d~%iXy^gPL4ay@4B4## zn`XJvc|UDC3K&WPt>qJUsuOHjr#0kXH+DJhs%}7KYGzYgU*HYLbg6mNH3cA44uS+f zrckDU*89gF=3hcpmF6(H-x1SY^K2t>&(%O_pL4*@SMDy&GE~5l2svD9JMmN9UHH5LNJc%%7PZnX z3qX7+IY&>FTdjbj`R;+xOd~B~NLD0~kTqkp#y1^P{6O3h+RWTtZLTh1kTM=hPaZ8r z2DxCAaGIZ~AO;QKi5cc_lL|8lbCzH?mjJ7AAC8g{){!8HQYJ{ZkW2X;M*E`@|6^tr z1ADdtCKyX!3Qc9G#CtFdIW=O4jhNVqhdR)UYO_YFo7bCS@ zN<`ZV&rF^k?>l>)5)#Lf`?qu}w~~qym|1-*aOL_UOv-zCg`^-Ob<8pOht>3vme71< zaOR~`Vl(lW-&}?=^aE*ahfOwd zs_gdD`-qVFjg za$GZq(Ad!OyY3ImQ(!L1;|Y?=-U3B;IL+5`Ny*v)=&_1lORbZ2Q;*d|-iz*>8!m_b zj<5r4Qi!_3b`C-T@Gn}O(>*z`I)-z0l8vgHi-#vaH|)0t4F98()w@SM))>a}=k^2r zKP}~#il7jnHLKZX9aY%r(0Nj#`y}wvXs9iwt9w|_7KIfvY{w!TuyJ~BdmqT$Czb7L zISkktasdnDtsz#Dv>A@WU0TAzwq^b%y}}|Z7d>z~ht~(vjJ1R zrNylXP)pEBuD{_Bez6%-(#~4LFp#fct%L`#AdH5Comf)L9_<==kA)imJi|f< z%tD8!wyBlr1d9w-FV8)6J}}p(fp`t}f8!zPkkx{7_yK}3|7nwBEpc>w+wc9YP3n`)Yr8qB^`J;=O`^{fq71xrfy2Hh-b8R-d#E!Fugmv2AyGFv^gd!zI$ZnlaHa<^T&Q}?n=0Ih%5&1NxvhXN}iDEPOqXGx(o2nZR4hzT~ih;k{@ z!bC?n402HAFzOGupV+%)J`puZF`%I~eI22rW%NGi&@!&TU&uHltYT8w;!(2Q0^gB~ z@-vhL@g)D zPVJGqNh9zjh4`-I5{TmU`ML9U&_MwS{n9?5P z3=9n=y-oC;kW`TeD<6iFq8HM{hOz{Ff!l(o&n*8F^`#dab8^!E`OKM6jcUr0enHmf z%B_|Lk<5s+p9P(WZAakvKU)^!h8z|&$AN2Zxh$WBm?|!9zf5@cK8`gi`qk<%JW?v7 zsgaCmH7g8GV5b(SW~eJmtOA($_x5+fwSR~*&7JBpPOq%V?-?+YGDG9yeYusKXk-I( zxw0{t@qTC4Sv&MWWp*<;OlAu-or^VwB|1_1D>cJVj&rqF#LxMqlLKxR=on-0>0%a*~pA*X*|3x@WL4>Bss?ZrmNdnPt34~fKbQ`307=oMvd%cD#X6DA* zpiW~3AghI(ac5a7+YqytXBUepB%$G_TYZ%Kr^)mSLErerKjw)uj~nMEPn7q71C4&R znUScs&A^W1Q?%LS6b7-Z#a1Qx+!gAV_WW`|Oa&VYYKQM|q*7Tdyf@i2w>2{_fCR}i zn*ibw`Otnxz=aKzv{e&jdA12EGhU3Xhm{E}Yn02UiThy@E0i8XDQ{ULt%gAW=;Ta) zIEbmi&yTeDCW}Rwe^LjfgzQ2Ev*4AlC$?jm4JvY-F@e!ix)mxSsNdKcY#+K+0Br3+ z3pE#T_VHUL1f1*jUKj#GqB%;Js9|(H(ON7?K(WCuR3Y|=oR3vhgl9}n zL`R6YPNc$B*p%WX5UYviLXxNSkawMjmk$9Bp67!LoYsJ%{GSkuAX53H{U!+vL2}Pf zDK-?8P-*CYb?q&1Au=U`;Ne)13eozeDIiUnxJ8w$u^hBSPzkby_CbkUSE+X>l|y zf#Jga9C)}pfGBwii!2p(Y*P{zZRJq&)CG#iR2H8Yt4|wnbX0vUX>n?-39!$k-!99e zkkE{Ia#jks`Jn-1O4u@A=}=?ekL|BkGaS%=-Do#aStY{YJh43fPk5*|%uLdQGOl5z zVerAGXQ*PB{0f(?k|8D>`A-QMgg_8wX$}NzOvLjzqz$kA&8x1Pm(m3On|G@3SFAs# z1vc$^>oA!wnY`@wq!O<*SQa5uxjQ*Gs^mNlVwLT48a9KNl^%Oz4?~%}iY>GRk7fjY zVf2zwp{eRIzsq4~=CDfIW@hc227WI<9Egok>5K}K*U#pQR-nh~x7#Morr(44fPTbp zzs%O$^ttZgysch-QHMbzm+Q_Vu4mf|Rpit!Fsxj*b)PrnLm6T0nfsEjkK!0I+}a#- z?S~2g|FIn2>3uboPj^!x)4%^ilI_f;2GYd-HlF2s$HKZCX2`>0+@$+RPlFzjtD4m| z%a7@~iC*7&dqn!EO!!3Qz&S4dV+HA4Kfv6HGKrHk)pJHd6}fzL zZEShE7)-Aa>50J0A9_W4h<&Z0DN)>$!}VgqGl+%2X7}_Nj?5f0TRygn5)A{)bq&MR z;Jgw{PIca#6s-1qu^{>mH!_B%zVzU_s8{?8*KkvY`{7J^C0xunBf_G+CYSQ7L2UMh z@sw!f=ODgBLJdkjXaW=_Na2^>0$`x=7@9s)Qf_d#9Ek$HsNxW#SUzJ@lIGlQqKXho zMp}4pq)s}r2%#MA^|+vQ#|t=R2?Qg;$Wn)evx$ty{IM()`C0U;odr3CN*-wzC#gt%)LNaGC<0#8>>y){2FpUReB5B^Ds&y( zGg5)(n(zq-Dpa^14Jc$n_>UgWed%@-@N{V$$*SxoG+;AgqBubu8D31m0vCo;JWYyq zaOMRsy8;jhyqQsY4s(B{p^#F;*!f7HVTT}a`T_xTWF%Q*GZOLxVDe>>4WbS+saiWa z7fPhT-YUCu2w5UQde*bS@oiQzE4o=g;dk^=#|C4tIyLPuN$z`D57}?TLCG^GU(0>* zu~M9%*#o6N`km%;lvtONq>O|~XZ4fAGYclHI@n-gt82*L7&^(F!qK;6IEAa*!!70G zxuj%pLST#~$upUT0roI=eFXQm-eX0Kg)50~V!KNRELmgnL>D+mYDkbJE)fED;F!r& z*1Lw;qv>)X<^}1H)D&Ej(7-8J?;XU?Au-CaEZ=OSAT}r#;oy52C(%-tVq@$)3XAMv zmFsUq)a;p@OJ#iy2UZt`M!=~$A)^f@ut~69{gPC=A^enyuG@ zIMom;bTz+<&I@=@iY=*l1&aB*_M zv?f%&L0B3!yvC)_FyItEGf^Xl5MzEn9KD)k!Lp14FdtC?vH~o-a%wi;VJvS(^ZRYp zFVNte%|OOM;YvdrZPThm+s+|)GVDt&W2ql7)TBcNB`Ts(kfIvYoqdCV<3WyN9n^0} za6(V?`czQaN|?rMSrY1xWLNJq4^R%#+|q;UjpH!sfB$1I2$B-oNAUL@LeS6c`|x!T z^jdc}AibpkUZE+7!C|#6lE1*4q8PMd8IlMR9de(Tx=vt3Jllevc^e9$8rNr~>nm8^d?2z^@CyoWTcGFzM4d3| zian41NlTv|&3yCgALT(H6S%?ub0%;khnDi^tSi$#b@pK9p_bd{zQiAiuS!*Io?Tc& z?sH{13`Nit47@569)QB3i7hi%Yge!`InUGBbIW19zzBC@`1T)FOGY9bns|Ah(r>IU1-A(aBU@_x)>sPdJg0n zG2`eC3~;zQPPV&SAmnvm&DRDrl`$WX3r6brB~v|>{QaQX_s><|JEQf6&K}3SmmJ=% zmj}KIaaj@65F9adFlcUU*l42&VXz+UnwHGiv_?sF zsL)t*2*$PVamv?PF|#dv$Ioo3@4%5zo?2k_XE8F!dyOkcCB0Q-Ws&1(1n9NW4bpTG z_m{|61AA9DNC-d?-tn!4$>k$M5D8$#Eqw_-B9UD$*FfNlG3|QGpiVziy-w<+Mby3ZmRYO*JDdh%v?I zF5)B~mCRp{w@jKugMDVoaj7vS%b4!(0zEM2$k76p+&-pysux zU7}Aud=UImJP4aW<$TOQ3oqg`i%-M;Miv{hYQE;ijtfwl902gD1aL}+;M>Y+Zn+76 zvF4lju;UO6@$uqn!1tBhDVJtoGKsxKP83mz(oJb4ig+~HW8ba=CD796&=+ia%8adw zn$Q%=^m4_Ttvcx|VCTt>Ud+2#N6VnCWpMR>xkWZamco_lU~wevzU%@h+=1!uU>OLIsre7u}3(;gO6QxR#pA`~qa&E;B{NIU6m&$UODj+#UjVVX z^LW4>R6EF`5KWaNOf}4CSO!T!1Jwiv{#bQJ%GF1T5f0!s3nL{kVEke@i7O46sU6;2 z>Z^!tsW}9tP^h@(i1`Tn9qVRJ%8gsUl3zuy9zrcYNQ~r@qd01#oZF(&3!stDNK+;O z6es4-p+sCpwuB+8EvC?HNXh6ZE>;ye^`70qL`Sv6tS8AGZ;gCc`bEfpoyg)M$%GJY zpiSUX&5V7utlO>8!kXt+I*6!4N6*F7`uf!SPJoo--9+Qc@4&``#%^)gYg7V9L}>)p znksbq&_=W9E2-dBFEtJS5IG)|g08^dB`4`#6v4IWzhaNf^s{|UCE>dc|DUc(N!>R+ zk5{!n_l33ocgvQVjE{nMqtkwSPXbT(RKB^+u5*sJn4t>);G;x?7J1O5imWmuRt?ZocH}sbcwpJf>aG3F>q>;x(R-cRgKlGr@e`DRtUu7 zG_YE=#t_Da@pvPbT8!w&SalDkaHFH}=+w?)pAeaCz{KnXfG2cvXaN1pM& z90bVq0pxagP^DDA(SdRD)P!;$9ZD$U5hTz!QLOxX&?F_ib15u~iyuXAL$&+&K!=jx z-L8xt|DyDgsdD+(sIZ=1aAZ&$PZmQ0w@mnSU2)?LWBdQ3>6^kU>w;~&gN|+6PRB{d zw$-t-W81dvWXHCXj?r;;td4E_=0E4$_0_|AUTe--vqp_kTAwbXh`0|J#Jw5)JkDVz7K57vdGrnY@h3q(0pu`vHAKp> z@yq*|$shS&nPh_X#UYK*Qkk+i)*&JBuD-&jgHVlfs`?ftY+_jttFzv}RJvCk#iAPP zCx}lySvLG>LX#QG)VZ)3*hkgyilEeESIigKf2a?9&tym~HX3gTgNda_T*eW@JGXGf z^dV3wOM2!XM>&>YWq{>eY!S*iwy4}#&Xr@CCT$0{&4C~RB#Tt|2(>O9(Ope5n2oJf zvZDa|oQFm0mhN?Gw%3Wbs@gMIw zRg9F?gjVxHgahHriRDV@-^Gebbe^0YSPAO@_(Nvh%_P$3sEJxw+L5?k=~i>D{6nN+ z7@dkv1)jpKq68BevgVBGo`##6YW;WcIEJiRNgUd!a{Z+)YDN}{$TA)o;{i?+x6BE_ z?GbxL-;Mq|{q?l=D%oGC(>tj|s5?jOz&;+<n1V#FM zEe*empcxJK4sq~VWCAr!527eVg}Cava1}4v0L@^TaZSXi{h0x*`qu1PGIasxQ`nse z)}30H7P~(uq=D{uiFk;(*lJ|;WlHH>={5zxJvt}0^mFkv_%KhTJynRP>Vj5YVzI(} z+CPf4tAbV>mPy~)sI~tk@I+Sk&Tz|7zK`AcdkV!!3$K1KFi1%{*&5=|P^g1odiRha zNmg|qSo+mb5$ItYJ_~(t7Ss-9!E6G|#=d{o0z>~zZL4s% z0}Mx;7c1&d=xyUQFPVlp+)=6` zsQ(@aig2CFRA>rQ2|~yV7i%PRUe&IMMMMO9Q+X|g1fxg}x6lM=waT~z*sNsm(lj&D{5yuBBlTMFcfQi(sKkaWl%#&&YF3k!wRd~e zH34EWzPjttSg7_cx*Q27oQt%3Pe}Yuyw9sy#;UnYp52Ia!K6G#DT4E{qt(4$QO_@2 z8%|2zX#uzUB%;r!yqBA$&!?@o)6STmld?Fbs64odJ+}{rU+0<^%FF=AuR9l1q?;qq zjuIIX;(Likm$>JN)wtL5zoAe>)`Pzu@#QK;jybKu{5VG~ZWQUh7OiVux9y;N3}PUa6Cg{EP;rJr?TB z$pjxuafgw@6sC6BHukx#71y~ez$R6v*3DHuo7-(GDDIr^#ohHIx9XLc$yKXz@;9Fw z+!oM6O)%7>%k_vtvIwS0JziR2TkA|aWm6kWe?5Ku+Hc@*>+z*n^c{dVH=HfV)vHV>b;|a#s3mu<@Z#--To@S>2=vI zehQjmT*F0R76&*#NDN@5_n9={sTA41g4~xgVp>m|Z5WpNA2i^_6f5(8V#hRi_Ns0D zJ$Y?h;m1@Z7fF_Xg+RO1BrPB@0Lu_-X~0=!`oT3(^)OPUq%M&Ms$vgqq)FPK_Zu1X zi3xsreakhi1T*`ak!5;>Hh%&2-lvvq;_WyMQQwmzY8-eH8u9Xy(bw0t!82YT`<8$&+y*QC1d9R35<{(@$(;Dh6k`FQC{KhGK8#EZ;IT`H7LCr1#@W}f~jS*6{V$5xi9>KLr zK7VmUD83R_CW(z|$|Y!%Y;!MPBx!I-O;ZSDM9@`bIUD z0E$mqC2sy~gJHuFLbnZ=HE{g1C*D1_0g}LpopMsk6 z&vzk2cmUiRx<&Qt7BUpDkyo#nv@UIQySn^24`p>^1%9s!M9@}P?8BEeG4Yr=aLv`j zi+f$CTF-9yZFLz-)%3g}yUNS|VN=8o`~D1lzfvZ^?Nl?;9Z?vy(E#K&{_I+va<9vg zb30jKd=YcObcQ@D7o}pump6OO*>wHaeM%%DN3npNlU%=8KKwF8;6 zcje3sRp`^2d87NP`n5#p$w}MVpi1gg;GG*Y@YeTAc=1nmkLd;%5E7x;7dco(Q!bqd zo)W@SO)IbjD+>gTw{ISw>2>qjwaSwkvgMZ-aix5uw(zEyHN}N2Sim24YlmH}Sa5A^ zaC@B(jk#W~2Y>hm8fsu|V8hPB`^?_~svQJ1e&$?5g8!;c6r5hN$?>?pX!l<~$sk(oZ z6U`S$*&*s(&CN^ndJ4>l@b)k!FwuJG#on31$5~44h>+&ecMXfyPL5zvPaSt0g+I}Y zrQ%QOxrP#44zH9i@xKlG9d28tl}-zI?ZZv;eV)xv`+c;(KF5aHA>WCG501KLk?$no z79R{iTmZnRvxdE6hSTVmrxmiQZryBHtuAwO9$hbe?p%<%wZq_xyr%#hMt9ld>mt5kocHh}yUzzC-4I~Uh7`0qx`B?tvJPDzELpbGH;pw(Beq{r?@q_0Dkgp34p052v( zc%ee6@$5t-F57o!?Dm{dTYE=S`~Lsf?R~MwJiB1 zNT#zCVCOy`ba;_qDRxAhA#XO;vU-@?c?Y7(+v?ReKH#|%(QJaJ;pG{fTD+1{ma}k zGBMWbsZkrf^~g(P+M%CG=xTc)Lw{T|lQ-4by%>Q;wRxv5-Y3fw!RA@Fv7r(a^fhd? z@^O0BVLUuC!V9Fq;zoor&G!cSa#7b2jqM3*- zkInblQ9ZTlxft7gV2sT32yKRSr9~Kj>AZaX+V3>mw%QX2uT+?_1MrAmJ)WokZ0tU9 zE6mtD95}>!!x$5&X)?CBrb#(jV^1Uw$zFjKx%KY5FduAc@E843co z6Q#A$p{kRvtf2$S$Z}uyz88yU5mX8SZ_84Yu+)&`?WW6oug9f~eD73C|M+SPe(!RW z%-#v&KAJr|Sxr0IEo$B?Wpx~H4vHQ-<-PV_`klp5dc{e2nW2*8m$OtNbUnB{ZhZbJ z-*hdTOyBlb>F(idd$Ie-Ly~w(_m-7eUeMY`&^_%lu5Evu9l1QVDR?v4{snI}tnKE| z`7L1q$zJi&tQp@HAi=GF%>&%F9O z8^Er~K^3PJ74J~aOv6=vs`tbyZ+X*Hciz{VM*q^a_2o4w&ie>0EtiNLpgdR@pClgU zG4^Fv;#|r5_G0%GsUD^mo;rdWb+6v|b&k&xdEc#IG3Gz>O1T9{Bi#{C38YJg*q7j_PnRjyU~o&WAJJGDMz z4AG7nLiw!>?Q?K^$bS!lte{+VIx-TA|HjjEg2$M{?}LlL_(O`d6X5b=;CVmlwfC_g zP8vn#HH|U#PlpwkHWmN<@-7np+fjykXUGF4E319gy3Y++zVX#e5moP= z%cE;6^Ht%BqL?xW<&1Od)))8Ix?#g6Pc&sHN0!Qe9Khpu{SoGx_l0Zx=Cx7HDLb0A znx8;?DD8dEBFcy@?i^~9%lbXbG&O8b7|oc=zxF&{>x)pHkB@J3AjzoB`>Ih&^zlc9 z=}URDnrT`pRjezZmX+J_`PO~;c47CKYUj&ZpTvF+F)yyF!y+pWP^GI;7P+p4S^ojy zjx>9lnr*N(7$($ZQ!Hh>iSK;W2z2ydsljl(Ck_X@b|>f9IfAVp1d-{5K6oz8f!a1} zR$-B6n@i}hCabHO=eax{&aCv^Ts^N|tTY+~JQY3_KK2@?`GQs)iU!u?qEf()tCIbZ z(Aw5u48BdGE>Husm^=^n%I6<{t#@7)oc-s25bPB7mCoR%m7~0x~>Re+alUCu6eLr{Fp4u3<{J>P@da3=&$s2-y zA}c!!Fm->RF$ZV}6Z>sxb!{C*b-HiN8U&Gn4L+}S&LteP__SQ~&N}>BcF^I7Ts?Bf zw>&lc@1>3Kk&6*e%*L57w&|YvzZac2FZ{HaLVMZEzat6uX~BjkNkLu|pQyT(FHURu z3x?D;v8~+~UcJ{YOgb+uNZ$WMzFb(LUfw_8rjc^}w)^9}|91A?#UQcr&sKpqA`F(9 z_TJF-J|poz3Alxx!4yfY+17?}L|y8)mYA z9Q5+-2-`>Isl%k>uir%`+T7&^Tqt}_to59}_FnR8TERlu8qz|k4eZl>d2vnq0ZV;; zLo-2wU?%tU%%IwY2(BiKF^-BDdeudM_j)1 zdPm9dOvGFKNOfFn=)L)TZ=ODbRFI3KUOO7r+>tvVZhr3kc$;R?2t<#Z2Gym0j5mDV z1qnP({G8V5bY)1DGHL&{2rzDLB1xKTJigixxN?$|CV`pa9HNiChZFeh)%m!p9Uix4 zP?u5Z0j_fEUaBBHzK?8uJYM*Hi;_xElzD!0v#!Pef}oGqm} z65}LG*ZuJ;rjMxJ@d;%#JVuHGW5Mb_4Rtea5uTLqzUAuQwNC)KB6a9jz4x!E9jlggSvuge3K z-@jhDdsGCiP4c^<&lf5ImuGHt+5wvC5_ZF(KO~d4f;@57FBfhfZWVea@rt5k;|gDn zZ11B(TXwFVOEnLjzdP%Hsd(F63#??zSxo`^oSYu`5Qn$~zh za8U?TB~UhtX81@7pv&s>7{51OzwKNvMVfsmI4!Vqzt3X7=Qx2onRH@yacX+Y{MA6K z;Z;xU%riN^r`k^t-qsaFbwbct`8#Lh=i?Ip)h*GRhx9KSKIsKxxSyNG68UCs4fFVu zpJ$J6DSnUV`dBCsAvq_@g~!IdpAl~FFP)T)USu|M)bZ|b(%X*=wa-15?>%;E|5$Lb zdn^v|*3O*rI~mE6#SH<8{9l zx0}+h&t1Zj#OVn6-M9UzviCV}38^M@xMOWzPak`(0LUGuTQ;1 z^wB;2cINfrY7Q;Zq@s*q7#rMC&(i)Q($4Fqv*+@)C#6%ic_kLHIzjg<=hgdt6s_wk zGn-c9!2NItD_p9*&Jg~rw)1$6>)Q3w>Pz?`Wwo78lwbXMA@G89z)rter+7Mp0Y5@! ztpqh-)$MxD-n{g7hV+3|5d*Tzw~bl59r!#8V=>JxRLC(9AQ2gRHd*IOw_BRt>h-!P z?fJBG`U~_zu+7iU7kJAO*u2o^al6P3yVz(y%=Mbe;l2H}b*szr8Iw?q*^AIuw$+l{ zII0&^i|L*IRh-B}WbW%`@aXT$y=!_B4OP3-z_ku3z@jZdkjlT&zV(N>vAXzsD<|xT z#YYP2wrZ}^TQ#H_;NM}I_trcvp2;w)`&{pYlzu+-mwq1IgeXkbbkAhqXIsmk-lZ13 zSHDzERvrNrw7n28N5nbBdxK$^@T{-9BAkzQL9@qcZUWuvvjPD2(;Grr8X>n1*v1qG%dfSSLsVeC#fRc?WDfBcnEs z3fuGCc$sj@@~Wxf6T?E?-8RiI_h1=*f$O`OsS0CnMesQT0w|eyBD)cJ4@I9W16lYO%ha7pse}wPHYsaH`K?-YcGhP4{zF z&-+@6HBJz4cWwJN$@G3JI+%-W`I)m^S9mLBBULZExSv;Ac;%AO#|CQk8$aLc924zyNJ!C%N;a_%%Ur4)<(8N1jZqc z67$&FhYE-q5~E|nMfb7wkBCbrHrok#7e$HMY!zAq$5il_2SVRf?+U&u^dWMlG!$kg zEU$L*ViYe#r2_K73atszB(xh@iG)x+@?d+)wEDwYcX^SKQM~q{Xs8IbM9rr_wOYwQ zW(Z=29G1tVX)J#_=&=rL{P!_W}?Kskd~$Xidc*?Az6Smr$wX!5zML~w**I}iZ3va z1ZuJp{!Z%)fP5!UdVpt}foQ4M-b#jqD0T8PdK2~&Y}D=*DI|Wp#(!ov$jtY z3rKd7M^Dhd5m{hsRAOW3N>*e`Po|bahSzD^je;cRU~8M%ym`iDB!%xqCoxTI6CDU+ zR>cW(wXeXBl<|KuZKdKtz9{dq5q*{T$j@`}^mrH*m}{p{%CnC*&xDEf=B)jlLXZlu z?1gYp`JmQL4o9=FPzjZcVvQ%76RQ@bJ_#(Gbsn<^D+DN?y&+298V!&{VbU6&D5c0a z0~3fCcORlXNUj6?eT!>E?!}d3K=%%f_E1=UP~^TjIYG%WiZW{pXKMN7*p}%>TtadndWynzRjEbvdJ2E93N43 z&Pumv3?lPlY#3&W^Xn-~j7i%hKfXJvJ3_q&6=X%B`4|Nx`n#yE#43?E92yh&AOy*I z;1?ACYwx#By^BOe+c*iG*bz;~o;c_|4Uld|1J~<;D}zwnys=~KZ}z64Eu#Jl+sw7R z=j@P>`zz&CO#bdj)47uZ6!|-{B|g=0_+C09aRLL(Q@i-^_3!0s(wlS*7L7W=JU3BQ zN^Df)oMp_PeshNw8ogwwxE~6O#bgIG{(l0GyGSKhRLg#*pz%)|M_3Zfwz0ie&cgKa z;gsthSKcAO_l~JuQhD0X^|C;?R~hdS?-X}rI>QwCsRsP?sALj=6OQsMtbmk!vO zalf|qf~Ms~NO*aU58Eb#A~`ou-Uh;oqSgEkfA%RD~gKqgz*6mlxgm}596A<`3R zZh}UTjn@^RAtTAGCc}AZL}ayA0+vu*w#s!#S`Gd+gDgG-V%5SqQWj5iDji<2CY@j0 zccKi?k^4OF3q&7anW}Bp2&~{8}3rBZMQRP3+>T2x-3g9V(rt=w0p#8-580Evb z!K?=oenSVBI$lVpHk2K^iOw7q=f6;em}776+oIOZJdM*bo8&7pokulh36o1!Wh5od z1IKDo2L|pBTJU|ticLl#g1)>P#AyMJzOi0pQ>T=@@ThAh77#c*b)bbnRZxQ9GZ9MF z2I+rp>Yy?vXQ9rS;q=4M(Q#;@F-@QD@=|*untG!-%sz@8oj6fsbhLG`M!MU88g(v5vHS;yLJ1sw*TnUz^P%uyqgSUpbEkd zj*kI7K_i*Kh9hCO1B+GgSMeUB8u$-8jpy>}&9NMk2;t176iaTD>#C))#Kd^9eePLk z0h}v*QnVhb7EK}*>>BZ5<9r)KRl3-ZovvJ8a9N{h+p&5~XRIf=<~TW|!a67wFsHno zgc}CxUjJ8cT<~KF$&fHEs~YiE;|VowuGsTXxH0lghNXzF<5DuDMWaZ`P`h+zclu4e z=AVR7S=2vSym(d^9+YpPU_XkF-i~3k_>^FKJvp$;^|+=l-;chJD*SZrw%D~89_LFS z6Of9)E)0a38Z35*r99`LgKaR!b24O|C9TBQO}z!_6DT37m0dt~c)Gg9we(rrP4rwV zrn_~XF_y*8lFro9p>oP^eq%$)9t;-!{BK+SVS%%XBs(#R;YYIMm>czZwK%~LWGb)ibI zPLKXn?gpI4Z5(XOy_rXBq4)IbAJ(p{ySWoQN51^wY)egagi+kB$eF`WO&yuD7TKOZ#{7c-C^3;a>Y`l)aU|e~$$IYi(cM!0Z(brV-7u z{6|gt-!Mq=clDnQPfvgw$Qv4s7GnRK$nqS%WZ^wy2*X|?!Gdl0psCWpFw85<<|Iio zvu^uWoHFChc*o#DkwheX(l0wdr`>M#-t zde8%~x8PfwOS)3mlmRZNxD6?>B}SS!K&{fvGKyHX+NkXGU$g7%`Uv1!D&J)zscUr% z1_gELq9z&znU3>RYi87?QbskpGF8kACGy}^u=?rz-bpzp>2G0JFLHi(3QcNRm532V zOl3CnbP8JCQj&^mg$D{eD@Fj(ljYHiamNdm2MULYB2vT4!qeNN7~iBz4ra{hRm(62 zAf(I62Kc(9Tez<95GXeowTxmqNVG+pWsVKX{CUPz-zk2MMo5wqZiW~42}dDx*OB5m z1&oK;OluQ7a3;!Ow-duc7KrjhtT!m3wavlOj?Q=Sk(y~~*s#Y+hOQqmrccdkPnug7 z0GaF>+tm~I=4~kj$`U~zuVggAqtNAl@ZT|QLV2ZVRqt$N!|Gv@{Z9pdyq1oI{bou2 zHS23uwvqhe9hrdf&C;xyvRNKd)V7d4)c*|%EQtp0Na|4#nR1;iO?kp2ndElVA*P|A zNGuV8LDeVy6j))XF23gTJ+cXpnqTA@y~EsrjK~>l?&4r1ooTUsp7$J)B%sO+Cnkwh zFn7I?g&+&Bn4of*jfR~tnPmc|KL%EeUW}Mj5Eu$CM(|u(_=ne2sjEZP?H@csr)B}K zkndaviVyHLZO~K=0!lRj)ROt*sSHkMV6UdD(;?rZ9tCrVszbAJ8m(d|9VmNZ(f|~x z1~?d)g!VILBHdT2kB(~$rp$3R`%gfP6HQBGQ;XlT2dh51nG`oMi$BdEt+e9e z@CrY}On6L!nDbnA`VuY_Z>n=%SO?T5XaIzdo57WGQXC7 zg?MF}cwf?LA|3}ozneqlkJ7Zq+=uahSDq1y{F-cA89VmKLIl5MsyN0ls0r6a{+t}E zY~Lpt1wV`>t%qkMlRTe;XC|FC&*Nu0CY^LMpJes-G2kRGZmr>+9Ch_wU|Ajh`KmQv zuC1LN5kCIe*K>Y;zucCrfad;tva^w!sZv@lhk->ZWGcJtgjOWfS5ZE>qmul0_u;n0 z6D0z20H)Bqkv|O~jdVuhnkh}SJA)=_rZX8yfJs8*8y<&%q!bKs{1O(8JZ5C8*xcy; z7&>a8Q-Zu_wwkZU68+oGmv}M!SOaM-&1uh)qW`**6oM$CU^$rDuX)C81(g(M6IWT6 zoNJFfcNsElMn>&MMY4oLDeQ#vaZfbjhNIN9CJ2>Z%sH~m)JQTy!~DFVgo$P6mVzDr zT21mTg-b!br=-$S$ZoSpb3${=`A8(s?3OhOvf>%k!Ao`Nb5swN>n`$eWvt~T5tN4; z)pEPGBSO;iUR3P37s&%5!zF;O9$DwC)`rPmi_HH&q|SX5>M$oJrc9Bqmhk^wH7j5S zuT0RsiwkCInL$r4QxLEa=8Wu-6_wy<)A`m18QhYEYNSH~4d9cx>QMeAmOJ`N1=q*H zvtoN&d8d^8h@Y0_-J0dZ?%+gLHGjT0Y|OCsK>tNv6loAH7CF)W^PT6!k}|FQdup}8 zGFDO_F8*}uDo|v{dw6;#qmS^mRyuI7nRf~s4nF3?h*7nlrPKg9oHR1Q5Zj4)zZ%t$ z4<%{otTnhp=Gak@)K{dkK!)-nL*~2~#sI-Zt2?q5kGE`+iz$M};_1em3*VWyA(kYq z%y6$@%nFk!IR#1SsUuOSj5=XjySf_Zw;3B7jEr`$gyjf2H-+$)h-6QvHK?wpFHFnb zlAxvY2z*AHrt(N~rnLDk@1eFx9x3aCxRXc>GF&C`N=W$}PV3K{oHLemkJ*tt z*tm$?#H@H7D0UHR5^~NF*zF+Eba7<&s>K7Gc?{VI^wSQ2zVAm8Wr<1}%Rk7UgXMn8~DGeU#|LDxLow zX$LqvHwqLdoWEheT5CCnU=~#64T}Vvr2T$aHZoEv5Fg?9FR{KwgkP8$1(puws|jML z+y(UZi?9;VNEnnh3a3~hKJq{Ufk;fKagr-#h?{6|r6LPp#j7qXJ^{r|N0YiCU+{`1 z>lqpAOo@{->O__WW-KpNlfj;}w$p`SLj>_lud%XCB&Oqih`G)k7q zVnYe+gwoBD+PFH4K+v#dTM?UBWkKd#pB!*8o$=+}uy$n1ew|jiPr41A8bZuG`)sVO zpiJ`{87vw1KeW)!m{*|<1Qd~wd7+(jX6rVL)wfznxP zP!%VO`QD;@An+oh1ihYSwux?PzJMRy;F86MO|iDTS2?fz0A$B*VRV;~faUsqZEI_F zvqTEbj3durzPO}-5$jg_UTGaUA~7)>EOrEHeWR?;$Q6x?PxQd18S~CtmHM%c$6O14 zXBg98x$K6UyIanO8M}EFZMTqDK17g0JZ~&Qy9Vh$yv>~(Vy+dM=Et;^{GF0=b?WBh z-H@cNo$Z|xz9)CjA>Ff^FAsNK%Uip*;U^r{NmAceM@ho$9jo+J{BW|@N0D%kjF_lDnp=X_EmTubJ9)$#mb4oys$?>#Ta;fZ$Vdl`x_n;uaLez5mvkn0do#nDP+{uzoDb5@rLmeDKRD+9`}w0*4!EatEfIA z0L6@nrGEb*B>ybHp5u3q1RvPH|My@1ivVL(1Lr|a?OjaYfjyC*rQWSXNxdwI5~d*l zsF)u`1&;WW%VB*;;3Rjf@a=4Mi9^sT5t;VYKh^%`^NG)N%(0zjPrpfpMSz|8W=<%f z2I8p6s+?qkVrJL7_;D1nUcwbuq08CWyvC0l`X|6t3XhA_6G7z6?PjVaAQ+K@3>L{y zI9zRGqM&!AMMP*~wOH>uqM(aDgXQywQQGq|{l*2@95C6~zpGp<bnIJq5*euD3gNTZJ&js7t^hkqh*E&+w3 zs9Z(_U;uC*@U#6hhsYJ5nxhl{E9ZVE{QqYGkPx?6$%^l8rk6ck7vjzH7?9hRIU%>E zlM3~~vX|lsPKKfBgQ}Qa77}||Z}uacznd86Qjbw%p!``f$PINb=n)G9PdN(F3q%ba zrt? zhPrD5VJ&^h5-p2SL*U&~)H#8O44qO6>5{Zc#~sNGbCG0X6HJAQgP|n45Wz=+YFf_e zlj^*!I-rUw71na~nn+@~NW#Al2`sj<%klB@duyqV-i!O-kUlSR@3f{JU^Jx^!BaX6 ziE{G-oM zQoyK!2`Ys;_P`u-M;V`sV#HwH7g=)*rfgv4e+y4m=$R#n!z${NB+%0FcHcn9+_ZS! zlvA8-sfQ6??rmRUhm&Wm6IJVUZ|KNVTK6%-ioL+8{#cD}uAcAqT*$QETmPY6m$&6= zZ#H?OA)Nw7Pe9H&qh=(rLEr>n4kr;=dvFqBUeS}1mOWLSUYh~Ka$$f<8 zGkGjJmYwvoRxnc0esL=`EUBE@$!^KCn{nj`i4 zyc;EXvb-`EDn&@LbB_lGc#5h@8=#E%8NVJ7psZ|Q8f`ZU)ZBrI2#q|%!Swb7U)dF5 zTkzjZ7RBE(1K)k#`1zR2=QG_7MR>}0r|Ei8P_+POMeFT$k5A?9=>qa13hTkx&8RSU z!=F96S6nn_zkX9oCEXQWz>^?kM#?dIUiz=7*ZUp-I_(Y&k2U%~I{R0d5Fv;o2;1F+1lJxoQz!d>jej@EW{gtR0eVVgR#_#agRc8f&gJmDMHc z%+Uq9v>1ozKSNz`bFBIpQ^e60yN|i7of8klf)rrnWasU?akT$dBg}}!P%@OnvBM!( z1X7pK^HApm(~8NdNrI0Yur*}%JAaMH(1WMKnV|evu9NVM(yF>{r$20>t}Yc_`ADF| z$r54|87G>>=}1^P0&t|=Zynwck6~t>-k>o~tB%kNixC`w7u6_KRD{G^-yUu=oa_`q zMTVV`FpGOqOqtm<6Yh^RKw$PIv4W3-sbJO#9i~vfGFHZ~me90Kz$P|%daT%&Hsd~*TArXusL;63AinT^c{!t;QQu+CL6v)^V3wNFv-zc`-iNocUhKVS>=|>y~B_VHH?Ge zfHGdBxO4`Oa)7G+`%Zz+cf%Tf#ohoaxi*;mrpVcPWRe6r%UxwK(3y&Ik$9YvY`WNj zsQ8Wq*ijfe%|c4OYhX!)iTZt61V0BhIMx0u|-F$)S)V4UH!nw)Je~#f)j}W0aw)jF#<1Jp`g{|DH zNMI?3L7>@#BVCC}bhr3N@k%wj>7;ma#-gJri1|DPd%wi3$g<3}?HH3wnXF3Z74+XG zMkLE8ct0|sh0<_-I(bQK5!^`CBT=Tt)F>v4CJZx&w^A7KO}z7Ug4b_p*la(WsJcEn zd*kC(K$7+IK5xHvnl{||B6GfXSDU@xe(7wiDmnPw9o#Be@b0p13p}S+(=U{D59kMa zQsk3k+ixnv#4m)WE8ebT$NW4d=LSR>Q&?w;7;~{g;=ry>7D-g@j>L}KF-l8IBxQyb zJicw7MaDmGp$o;;IRmqg60IB99!R@PZ22O~+)wuX<`ed6;UqCPvy75$S;?Zp3Ux$7 zw?RMjw2>{hYyzit z#7G+`UK@oeD?QNym^`n_Iq{77!wfBZ}KVNFTq;2=_ zzc3*#D%l9uGgvxwFxDEgf1#w2_xZv@#S9AQvfvESw8E;#@gKI4jWE z32Iq1Gt1%C(CA=?kKzqW4iF`yAuA+_nqf2j#nTr40O;$&BNKjNbCI5(YRk^%Y8>ojFoOZ zURw6g7?gILBJzo_gbhzBh8PXPF2ZCNFkA85qlQ}vQG^@#gRD9Q-C*KgR5;T@oxQli z@Hf8c4@r`+3e`d#6zyPy7Pu2w;6LQ!oEom|N>{_(cD6_wogwV;}>=-ze$3S@f zZ1g1d@-YiA-3mlCktM`~03Kb3HR2%rxCf&cNZuY{w=qeJE zoR6l)FBkf09+k+XJfc@4yvT2I3VP-kZ2{&=ux!bD+;LM6GIM)U=0$tSbgoHhb67S% zAp3G$L@5|187PNxD-usiOxRp)DUwOkFrJJ%)o~>S12)%)Bo6W^`;d8;qh~J{B%rOV$A- zDzQrat4dji&&LhY72j%gC#BCtjm*S*r*1c3BQF)%xVfo6 zi}%Lr9e3}sQ?E}_v5!>k*Q0lJ-}%n4#`4P#(~_YWZJ#n4R{Oac-JZsMU)b(?pW+@8 zX5a2QOIe4(DY6hs@2QM<8+d@i*abUyp)hLze1m%gmPO@{XE{ct8HtiZk`=D+VlR0_ zVyq;Qk!UpuZLwTy3sknc%IEOAeCWWLE`|0M+|+kW4GGeNmV3!>#h>I7^~!qhU#uVlu%#fuYy57) z2KxU|mtTm2ah@*#-iLl|8*T1&*f+ivv20rK1LaXooun5kIv8?dr?xhBval^!s0kV! z-+B0_4tZ!K_mW1xg(abLbSFZSkP(W@6m(18o<-&-;4xz`#UHq;-pnxT65@}=jolj= zwL8lIU?$(`fTF;v6VACov7&nCDZ4#DLZlh?oPpeT3UbrV@M0Y0bA&QTO)~=?RK!iVFeK{9@%%l`(!;bJ$5?Kj=&jfR4~KW)lR8@yYwDS=`H}p|JdR z4Oy}B@v1K;nfPIFC)5liqs4RD&Lj<=D)jWKdpJk-is?@bAK&!nuRA~;v9c0-ttv4?KvvBB6>!bP>2tyB@A8h+@#_oWy_??6NW;crF3NSn#hFlz&g-oqNj+O6n)n- zOtEaTX2Cu>ck&A@iV`vRDe-k=(Vk>)%~^3UW;L#^ye$h33XI}*D>TuAQ-`QzMzJ&s z)+9R?njbB^ic4CG9>+1@>f~9O12y1iM$)G|Br5$y{h~qDXj#&|L(;V1&ZIebpvpQ> zyqD1PWv`f*8iR4?KuZii4RDdb8+`OTFka1{sUuQaAPDSd;#6hIKhd=8uJCnP58p_g zp^ebk>~GREmL9h=|J!VC?x*lbY1W-@B4N zS$|+tW!c0d4xu3s+$FdZrr1v;8H2#DqS+@kjN6k!O@hf(&!xkkm7oc$Ga5qdKECjQDMhA5z1GG8|vP zte`S$-l;_0h;4mm<}xn$0sIv114tq&LGP6nO9aiiGkT4i%{V7zILg62v~h`|&ByaW z;5WV`iYZw|drbk%@cr`!zOSgE8}H_l2}5t`myU-is~Q+^wapRUkoqztR%m+rRxW&S z-xKboE_D}dBjo9cUt5zcKdXalzrCgMDOmpmh!L8CJJ4-~V~(6Kv&nqzy7t5C;9$|s z^k-MUUg|;Gkc&c)1!G2~`=?Xstz69L58V5jNG@R|6NMjYSuqQp?K9}|W;C(+^~^-& zEJ7IFHvkGh?yA#xwxYC3Tm7gJg6^C)^?!LT8DL&+coYj?^8CwT#un!*DV87Vlqf04 z%PP=Vge)}73xaCWX7G#W&|uJ~Yf3wwR;`J~j$4kBLX7twhLZ7iOO^)`m%^y$V6~s? zUNQD!2K#`an37BeoRo)j>=lyK_)M;y`=jiWNC?3F(mr#bQllF8Ua=vf*>o3Ms3sEjEjdhyh5mk7hyB7bD|2Qd^}>J*44 z1gTRPYRVJL&^=+qV+o9+OYYs)AJ%JedKK<=YF=5A=({}KS^3DHmJBW;SrdeBJgmxA z2$zBA5KABaRWSbGz4q03L(TjyPuni^51`DuBZ+eIyY8UDEs zQsI|p*z~udd3^gL`IX(rMCF(>ywzJJz~$aC+H-u&k5ge48l~_tN^E-!PlX7C>jDtvNRTU*rsK?xjNpyOpjJx6k)s zgd<`=5S)w$Fq+$5cZ<1+Wzv1&$31*cq*t7;xK#(aesnmSyKey~pFe9^75-|Es(oLW zBu^sNq$Nngy>xMLe({al+Qwjcd3oE+;qFhefkVE`3zzc~B(nKq60{Cxz5%UG=}$aL zY78E+i!Qj5HzqS`r1NSzNe6TtWh?Yw*yKYS9R?({E%K=lG!c}WkDXW6=0* zq9psv6u_6_sW)WqG;9#mIhqis6HDnge6012DPONZGOd;sad1;9oE<*$hyrgCk`P8t zSEg|$bLfjI=G*rDRVu)06K(lb#36J;29nkR@2sU}U8v^J)yC`iX!LKUOHj&T5c zwiho&m~|DsP(Z!WBHu3gF5G>!gi!F~C#hAU1ZiMs{k`H6WCDH(n8rV=K_*FUV;Q-WNo3%V^2WDdxW4}#n< z)5JT}BAqXn6B!yYmVC+Zr7+$C1y?UZR80c6m@))vx1lIKRL0=)5|Ivux$cY3=^EN( zT_kO(66-?Z7;Z81X%>I4*Zsg*ZcM}kVm-0oP#v+sjRnJ9^}J&bo3f1C@Gz;0f9Z{; z9hI%beDAWwO3kMTffqgtvTP15Ga0g)A$5-yZPG>z*1E3=dNhIm99@T^@}5kuj43`T5H+PHV8-Z=&KqR`of*F#k%CH04}PvT2rfaV7PW z-(pb zle`&lWT%5^+4$(yZ(i=JG%Bta8|BrlH7w!h)O3rt&q4M3mgh)UFLiC~8M)Lf9=bFy z9$t=JUdp>}8-H#Y6D-c$sVqomf|*7}QbSlOP$^lNfp=DJ(s`FhSm5rhF$HD^ zpIKaQw6CSI4jUlYt-@EQeK58}ASacmssZwZ&}ksbB%DgK>Lk*o#iOuBjL1X>j->sj9N0zm6#E(TT!4~!=-GX zexBEcCg!kh-QiTLnu8p4q9+r~$tIbXEF$n^(tP)m{E%!bx<;EEhOAU;I5u9d4qB!W zjk&C==?7w5Z%v0+5#%Y|h0Cg`yE-1zOn3~&{9&+(5#-fw0}Dw)ILW<%??0@Kb+nF( zdnA&oawg`f(uz;CUb|gyBvV@`@IpP>2d^R4DcU9)mPVUG52uedM@|Az*(afVp>zzxa^_-5?#%!P@Jl+R^U zBp50Amti{|v4a(#HS0^uxio3mQxB?DR=n~$`vN}bw)v+6XO@ZdTiG*_HO=iJ!}nL_ z?d;eIFRO=|KWxH`zlmER2Mlp;DZY)2nb^kGoUv>Umw~f?mrIS45Cr2kp3zWvoio(^ zJU=NXu!`@Nq-|+f9Ss{%s=8_hF(Ifz1JQ$ppl#<_euKf26325&`aVRZ;= z#ZaTF+6^@I9%Zr7UL5hO6Eg1a{NO|_M8IThgzPifVn|*mggEuvB$N9-Zh|bQ9ngn* zaBBJchFLS`>IhMWx;BrR#a$z@APn%#SeTW^>(`Isw9lB=RBk0_a%_yUg^<^#rK<|2 zW?zbyR_1A3+m2M8cFU&~<0}PM>v)+zE!`?(d~wt((bIC1|0yfP^1=aqdrFZ;L#qm_ zS;D*(fwNZT_NslLUMQy4xKSx_)st9P!s6*rE8T27y?05#{Kh0S*9wN0%1$#@4f5d#E@=b=^TPSRYLV zC;BM-j*=|`z(9l5D4-x(!U_%qxNhbpgj$x=*=brlg4tukw+MmJlyb=-3qrIBr)Pl0 z@*ZHBsOV{U=|yov>k5M>SFl6Eytdl8LpXyc0v(%etRB}k_^jI}Sz)!A)L~J=aQ1s& z!9t>&tmx2i0%1pE(kYi8&^<4nF}53G${MJ&D0Bn06PagC%=6O{N}ZBVeUXx6t&3xT zD3Jq=at=tlz54l$^A&&&GX&7Q91HunSZpFSb_iyPG}F&0Wg|$0M8!V(YyL6Lc+T`{ ztxn+}QgxUlYe$@^PBbx$dQGa-CE1hk@CJ4oHwiJc6z>>=NkFVYzSED=)yl)P)oNzv zae=Z&?cu^d(0yw{@|R`;2P$Jy9{`21n4dy=Tai;~nK|+5d5Wm^l#TN4ktfYJ0mv-p zz*1}Tk8+-ac+I7bbaO_CyX8_lh^E%NV%K%nh_=3bb=sLIT3&&Yj%>pC@b4kUjm25exqO4<2|5_Pw4Hb0AP4p2X{4MVHHd2eC@})~)%i_{e#{Px- z>({UA(ih9Vj?U}u<{RS$=`pG?5r~m1)09>Tz>*k&CI>Fn%bH7n=C~svN%3f%&>S_oD_27Q*#;u6xh)|tb zXBE8}`4!Cy?~JuQV8fYvcmm#k6gvT!2xvJIWKd3`UfS`;Y(>5SM$>~~cD7ip$msA+ zt+#pDYEhx@FeJwqBa_lfN8+84h*@fCN{_&m{}ug{F*m!Z{#|{8)KbmCf9ZgdOxxSe^p|%$FFp6+BpH2EuBG4BUFC7(m^n zGI(*D!L?ty6O`cX(hOv*5R*CFK5dlLV%NwqAr$K+Zug@l=cua8(Q_j(_rlbqtb%qJ zz5#p_Vu+J2=!rNLfvOC70t{aqmA6cLd4h@`3Su^o`i^-MVfcyzW%QG`lL3Z|?O-Pd z1R}5D(BW>p$cdX(aZ-A;ZylrGE;JFzkyR(<3zLjeV2*4_4xa@_)78(P zy96}-qS-+sZL5BQ3&K*l%ar|^^-)Vjyc6xz3wM|DX?;zY98$G)Z7ijwz|Y`r(vn3R z=l!=lM?>zQe(4R~iRCFn_m^Zr9V231FWJ#L&=#4TSEt z!}?@53E9zX3Ezjvs!Q@BStLGD^#kE>46gJfgOMRFj+8$=G191xh1`I=R>xFR36{#I z0sB1NJL>l*5khsJn7X?=-1WqH?`#>T!+qj1Ut=2Wy3>@g{c+d_nyG?YxH|r6$Drp+ z*bnOq`01__QHN}^lcyhPU52kA`FZn8gqZ21wjV1_7biBj97>V3)*NUw9z=426(S%J zm*8c@^_~_&53vtXlc%}xY=bv3FRwH=7<<@qHXcl_0ZXsQ*@DzLG3-%)X1x4ZL*cnm zl9K6^L7U)sHI#XAL^Eq#Ffm-tfy3g>6U=r3oYGyx6;0v`(aMjM#_4qf!4)*t%onxXMFoFf&fK;-)h1 zj3^8lT0J9x?oID3vSx-fjbJZdJZ%S2c@$7JiM{tgO0U~kdJb#VB6aK1i%=Dk(vRnvwAGh zC9-j|T3HhjelT{~;$dCjCa-3@i6srUoWFNyUGit;2Hm*SBy8#2Y17l(=(xg+Dx27S zGIKVHSaO~Ot>V8?n;Qhd;Oee>j%Jj?zj>!nz#9tE)|Pl>6;CadQgS+{H7UU?nQPdQ zM}Gkfpm}CG>W+k?gmsqXl!YDE`7$lL@GVKBceOL3)RNxXPxdn3JWUERSTUn1^Jqu#D1h!#?NCRQ6?%Tk2uPu1RyY~&u7QT8g?g%kV&el>W32{FzOoR=OYa= z?saU)_Y82WTiJu86TiZg?R_$(!xLrx(F}uN`LfU3gg16ij@A^~k7r%E0spn0p|@mO zYkOQx81`VmUNNk;LXk8ppM(qmcRD4_v88^atXYeXl2P-mTIInQuF+6}a$3-fS(${I zTFQx}Cp~?P3CpMulmu^X`(#!BC?E1+r$T8^)bS7Oh`F7LgXM}rxg@N!UMl$a!QVLS z-24>)r73IK4vBr5aVesE%wu?>$^&)B3>#6`fjiLlW9zD6Jsz-udef(yJo_N9UT(Zb^6SbJ8T;+&}MLKcV#femIFI6cd%TvNebSceg0f}9l6{HGmf3?&4c z@&uuBd1(XBdlh(|Z@nUEf@9RBCX^tkv(mfx948<-WF4os!F(n9GF}{e_LgXs2^EYU zb)(TF8hIL4SvX5gZy*K&{hqRU5W3>JSP^tc_y1Yp| z4m3jm(I$mJW`A46njb)FMSI)YMnV@nQ|GK+yBx6Rs^_O9aN`Rl8Ri;#-<`sC8JW({fWkzhs-J%%D!D3Ey==n z^BgF~`m;hxKu4Ko%O0jYMANl6RA+n;E(sEm^DY{xqZrAim<=L5zyvplXyt3kE}w4T zsNMotadQA46V?oG7y_*M=g6&9s;0`78>imV5pBTUpd8Ac0M~5H@4k1#!xAti;^H1#F#>z8$@*~_^5)LLAm?4Y=!xUeA8 z-q^3*Z&0-3^!XLs%Ss~r`ziYKo)>2fs7X<9=gri+gnDQA+HV zk?C=a<5$GG@z$aXQvs1<7>Cn#lQ;D)Hh~Z(tXzdtqoaQ5!T{s1ZlJI_rp< z8ZbAc0m3qrN*alYb67Fm-L$ASFT4TwQgGEI#H+avI>wIx;?gi}WX5VLD*Lz!cQ1ge^W8nh~cSG|@WV-)i6X4ep=gU%Y zIAr0R{g-t5A1_zNs0@0+>FTCq)lbdHX8}vgx9xJ*-K*9ct+4ARPk5(hzC1nHzd8JU z#-mpu5JCG(`aoV5VOFh`0KRPh$!~x2&=3jX;RfV3Ebiae_h&o*R&gaQ%o+2#e*F%d zCI2kVa65>p14P#^#hEF4I8;4pPc!n8jpr#_lbjz@*RO0%m?xi-er0Pa=mZ)3%GMP2 zVPrTehnt^NB`e@hhr_=(=AZ1J0E^36pC7OAKdAWIN!-G!THD*=bPfNTG5`4y5j;e@ zLx-pTU#hkdgs_s0pktxge`h<|DnAHSiL$;8{snA9mSqYgA#1;YkN{T1#ZP ze0ESNwU)OK%+ya})_9o2w|zyXOcz!Ytx{KFBU_kzX8L=vWA|hJ+zZn7`;Gqi=VU!i zU#GQ5C$4&f1Q#zBgwmC8Ftfuod+fkOAN=GE@7$ZRytv3pTwDz))3NqHXX1Z9Jk?M*nk-`pkkO^fkz3o;>iO^2MW=06elBlb;1K-mR^OTrHsmYbn)y4aQ-i5* zESWm~1>yfotDP)V1`(y&!Al4bb+ljvQ&j{rK#w#Ibg32&EeHpDa%Pb$a735_K+$8mvbH zhg0qRWlhhtozDt_;ZO8v? z?-2gU>R^C3cNmFw?yUBa+D8_WZ zL;3yJ7U(~R^Y0IbPET#_a_#)GL4U=H|KgfXUC}3lnn$UN{NHC2Vl-bJox|`xKFO2a zcTDU+PmDYrCrZx*7VVxQhsd@KLPe!1k|4sDpud64hMh*X+CjaqV0^!O3A0pHXW)V9I{2O`r zv-*Ek+*umdDKLDU#M0cAGUhy3W4k$IyE%9=IkFws;$Y>ZgW&j2yR`zS`qg)Ff!|-7 zv19PRb=#kBX#9F3t#>!D{yS-(4qQ$a>X^YhOl=1CZ8l*&@ zSA9gIA=5HBW$UWtCEurZckbqXFLrX|!&dg2z~95jKj(|qP8+?d7i_JBG8PCi1*LwS z7d2)4k)Ta7N!q4mGeOAZ!Rw_#&u1o+KSL5+{*IEukUBs_|AgW?WH3BZJT6|R77YJt z=Rf*lrwxI0d-Y|wrhn#i{*8Wz@Uxy5giqWrz)i9cDg2L${@1?^G|)%(&f35T&Zk>p z=&+vAXbk!S%eQ~rY4dyP{2}l*3{Pl|!JF=n{@OL5e{0MIpBAXQ$|~ui{#Irh4WC5N z(2V8VABF@BSdnB?zqZ)ibNJm7XyN|pCqMVU(Y*Sv0^|4EA7V#b#ku(6{~)0&GIXl; z;!Cy7$Umdk|G`xY-cQxv@AdHgA&-rm2v!SQdt=Puj=!_E5Sj-+f&SnYm46*s{C(dv zC?N8Vh^`OmL907L0#mEK@*AIC@v4rdY+Bw#A1CmFr(p5i!URC5dEjr>NUzhMhUSjK z|H}sbFFyPqEMu314qp3b=|vAG07WMuxXTXsFhu6 zza_G`U46j)XqrZ-ke0^*8bTW+i%rp222J0^q{zK>!+m`c`DP}8ge}{8j&zW-2G@e3 z_iY|j9F#~56ru2y?l|93+_s0j>oaO7MBlEysd2;g%eL)|GuF)S;G*u3@he#vYT>8S zzz^+OYU{gC2aQSYmzvo5zOC!ZEQt7Vv1@71bIHrqGYMAO_pghMER3AaVj>w6kHkMJ z`=iRg{M$eZ9bT$^#JL_W;|pu!0Hd`Zq5feo9#7we#*~$Bnnpoy&T?FHju{(G zGnju==iXFr?s089tl9UOZK=2$-RJVQX8y2A0i_SiRUQB&XRz~CR?~8W!mf9ZQ3je? zHZIYwa4~9vbSAEu?H~sak{EMEV?=avkpyt7n>Pwr8@$U|l_*c+sE(QK6t!wne6_xWgB^3w=4^ zI#-`RucNQL^yS1>q0hud`wbsG$pZ_WuzlU;03QWHOs5OYgl6Gbs5B$|4O!HlPtnhi zg^waS(xRfgG0<_hev9#}P#ZmY1NVx|Ifl*5N<6BCW#V54-hAKLnT|8g8#~g0fJ}rd%)i2M5T#TV&tBocP%n>)jkorDO#o;5 zo!@VQ1>V8BRE*F!ELc<-1&G!tES!si&s(R=?kk6c=8gCqKUazp$+Sndj%O`En7j0z zm)|7+1b0b>(1*(i-Kl*|}xE%C0QJK^DB+y58UjlGI+J!2qz&9SP#-ZFJpBD{uJo{{ZRi2suzs= zY}*Nu#vh&JV@V=@Yv^e7bZlpYwj4>gXes%h^c95U+%a=3^FTct)s;uh!>+{Ze8wL z@Dw*dQbv-VZJXt$94v>-)nBWX&QVd{wY~Q@)Qu3e@Ao}JIJE$iWvU6gk>MLQHe4Vr zMyfv*0fQc0IG|)&#EF6dTV?xwb1J_I&Fq>p5r&o!TVeh3q=k_6yCD@5wP!T=2^Y;T zg3|{rIW<-S(NjYO`53fH)pQPIs}h~Sl0aFrwSX>gm|y9^)mab8?{wV)EMx+0{L%bm zA#B)~4J-a$=ZGph9#d41Aqf2#Rw_XnJi#;hp7r*V=}bXP3c?T7Y>N;W-)e4dwC19m zVrzmp-Dy&c@)Gl!8mj`qJ@x_JgHj|sq@yX4(0A4eXsyRyZqX*KXVYA5?F+Cx7IzED z7Kc$RR-ae`KO-ee&2=b&BrOI}T}Jks5d@hW7|KJJ)Xcn}uhq>~Kz64PFwZGrzNAmc z^CeHi-9yb8BUh!2z7)yO+4Y0w@k!%n?)E;L2BCi0|KA+&k2eNCF|!)y+cV#|%EM3} zlVbR``c`(Y%xKY<+Tf;H5vnxH1vSgnWym9~Uoj`5Eqcw8tIK{Vqo3YOAsf9<-R=wd zm>Fn|aKT*Cs(&u^S+hfEIvcgI8ihb4*kIw%Jc(fbmB!IzflasJ{$Sa_a9*mP`NO1w z*Kvjf1K~ALwc{Z~YB?o3=cR-o{A36^1$v$T62CTf8kC?lh&W8js&I^bW2A^p-uyQg*#w;4L>4CC(%IHYr{G z&=x$qW%PwAwsq3OzDAGn(q0O#$SPPJiU@ggAqYzRan&G4HZr$OJRLe z#G$Fd8J7EZ;En(9ZS@KVWN@s{f(xGv`id5UurB#%KudO+-Aeur?YoJP;*k`{^}7+7 zH^*J=gp_g5m8QJi@@wkzh{uF;goV9~^=;iA@5fs!9#-}%3r6Jki_*csS?6*q)WM)e z9E*8pk|MH58~?szNy8{)dJPs$k?8+)NhIg#G>?)J%9SVaSWEc zl)G20u&fzje@E+`OWl>T6$K!_HONvbv&zV7eer$u#;9qHzHrJ@vY; zpB64%;-$8|_x9L>5aePtFN@Fz16zg*w#W6=&tQQ~uYcb($GTTg{qj75Bg;_5vRIK!F``T<-6{5b#)V(< ztb{qd=*J1DPU!Z%i$QVa(8J`{v)n>p+g>=M#b#{^@IA*;_9Zhdmn9O+-c1p`3&T5s zQ;qAaC4}-X9+SJSdVRBh2}=KXIhzTw5ErIH7!U@l=YtN+;kN`k^NEtt3hZ9E`#e^A z9I3A5lt^axPe~WEd2WJ_NMG}~M4YJ__b$D&1iPw_J=|Q$FI9W&VtpjL{Sk-Vj&y;& zTT19ZR5G%ONgcQ5LMQ?Ak~xFiCx~iw*BMitV=q5?U+u+_dz7umW|>Xd*blqqBt6fz zOjFSQxc^)Ct}RW(IyClx_Y4XV_5RE~c-(M5Sk<&b__YFO|=wSKEHigH#6Z#TG*T?GjI) z2?t?2e`SK=7$nJUR%iNjqnJYBe(T3nuSQ=lZBvkT>P?M~)>oH;;T4V~Q450{qpcsr zBo#)BU$a9Nr`L1v{V~PK(|}WMHOV4U%pH2g7_3zhn$GE8{HKvN{tjy7KS2#_9=a!W z7Y1L}AC4YBhmJkPhqz_5zKI4oj(S00FUe(tFkhUoii-6X@D`3IXCIr%HXs3r;vXF8)#V?#ba1!wQU_~A;+hHyR@&L{+*3K@giAw@MyVOKu4p3@KI7XljvigbN;YN=7D}X<)exPUO;*;`*;nd zl!YaSrh`S(k;)ryUA!)$^~ASai`#uKT#bj3ZWcm7#XbjR`b<}4?jLGJ7DqUv_5vTW z!Czh{g|xU{Rf8?c-R{pnly}%sHrm_rHD;8T7pF{#;_V`;7dEn0G$|f2rq-F)Y5hnx zg1A7k3SM+yAErl`yi^=af9<$sRjzMqm40F2bOC@<{miR7W9otf&Td1}oGLZlrW>bl z37NpQL#`dNK5Fh1n@l`D+ns z$8E~s75_Gj)!=v3$@l5}5yi1Xj)SU8cx{3@(&KyVS|l45O{zjC$!H68WtS8}fe!u$ z>xALx_4*1Oyf?#@9ktWTJZ~=qXqj{AJ-!ZNQz0XsmHHe(*G~8wdsyUUc%Bw#Bnv#8 zgb1BvX_=uQ0R!$Pb&5MY`Mfrh)%(=c@KyTeGSZPAcN{b4_!6>M6GNO0so2t0d~Qc` z*)nt`ItS+sXV{cED?N{uoyJQn?v@|EpdFdbULN3F^!q&MbR>CSSAP_+XD6zEe$yQK zjjwE@?mWNJ#{+!H!tW7ew@mU{@a)^0E7k!(9+}3^i=mX;FOR3EynBF+SOX4+e~SG(T13SAyO{K#K-#yu6m zs)6KC$qoD3nvv!wqosiK3aQe>3omG z_LSzga}(3~8sg(mYo8US*cHC$cW_9>mfW(3~Fpa7P`OwDQN$(F^z8o zcguoPOqzD}&}eBlHQc{_y|Ikl& zAdRY_PU=_=AruCo;SDa{nzeNR(_nxh-s0@w=XF|XTW0f6kkKKBH9P5H%jG&?2tn`X zPHX&cL8*;C%&9{0KM+!dpRXD50NSf&Euk#_tg4CrA@W9Sr6~D3>5-^*%cO)>mAoAMSfC`tQj?lvZ>H_)eSGPF%8D33-r+cL@ zjuswwe&QZEVcz%)iWclWv<>&`5ixyM&9wvqaPY3ic{H zJicvyxgz^^)uUGzqw2?=peA&-QY99qP2SLc+C4biZ=r}P^fUWG?*qd!^J~~IBe_=x z52Z($YQ3oC?)<%~Kc}!YmHja+(D}Gvo8gc`Kfm$1L+;ZJkAEF}Iq8uEIeu$R{^jB8 zXIoWGB?+ROa=DWqVUyG6G}OVfp>Knr)Iwn*28POgs_rEJ#vh^f1QNnuZBsx%!W#r( z=BDn`AzZ{g+v_qypcq3_R47Rmy<@x3fMcGkbfCE+32!q6$BzUE6vedR5uKVPGi| zyq^?%ys6tB=BOkDnP{G3IbDlxA+G^A3u9pRl8U9V(JRq~@}_m=MmdE`gV2UrWY7FMZlX9cfT#LQyUq9Tm0i=|w>*6Tp_ zoW&PLyJehws%m~8vy`ASCc5`fJO{?dN}6PRF51-LMt2M_NSm)hk`iVa@$4_ohvSV* z*WK+Rmxdw-FXV(tq>Mj1=|Z65PB6G&cphzwkqp0~8DP+w_FL&1!`gW`x;(4AiTw4I z6-3AxH0Kivf~?C>1QCW}Xbtj?fB_|4v764dvPf`In5G|WEOxHtnHFU`_rYv%3xL7B zkw&~B9RH`qZFLA5sZPwHVmvw5x%U-YG_ju2g~7NqXgwyx)4Z32cgyvGO#gZeA-@1! z9l*kHG1q_f#RpuBIOR4}z;E;tgY&xpVZg}D;~7}R2eO9Id>pH4d`u&hA5QOhoE3X-)o>8Ugz2h{KPHjOhJ}Qm2B#M7j?Vc z@i;D4H|c-8(9TghLt7q?G=-j`>8!}|07(cj zq*>jgmY|BBrajgno2Y5!CHc$NWmF1!Dxk3f(&AMrD~hR@)`u<>qTajte3HYBKXW%H zzKu7Uoz;_^6v5f!LxejVP>q#%^;MLwZO#f;tP|E6N35=7I;p+o^k7!uW_9T^sKUau z#t>_vXqQAJLl!k;VbUW1;XZ!6LijQv;M0Zb5|O&n<~^PsU$`(1rdo{Sgy?VtQWfx< zyV`hOd-ibEYYj?X@>-B+34;3%ODXxXrd|MB;45|;bCmeYBz-}!Fo5Ae3EG|*Z2RWs zG+ZlVCqJyepr)@hj5XtSiOEh&%UunO@0k6=l=j(#@g(HdpSGz6?C(KE)31`Ns}b8A z>^ZKKl0Z_+FPSAa2V!Mn5Sy9ib4*W{%KDJ!pCKU90Z>_1HD7V3RYHZmcKd5j;V%!O z6^7s0qiC}Pmcy^eN24Z!74XA8rOF})kAmJ!iM9__hc1l=^`YdgwDXH|fNKHKjK{K8 zMO4G`!{Qz@H}qmnmi8?_F@C2;S2zVye*4J+k>kNuI{~&^+W6$sAkBF!poytp(C6?N zf%lUDlP+pl6=Vq5~=o^WAs z-@oEJZx}lkM8AsyY8O57@Jwze2c~T~b4lHFTna;fp&aLo*wVpp%xX~!FjhbQ=Goya zp{T>w?obvMr1We({iIo#o^f;CK<{-#(28`7Hd~t}Nv0)rvjB&A-=<+kZgLX%bAn9c}N<1b^O38Ja2KS*R};HXBO z2$Nsc)%)cm(yB3Vv6hxu)>tJvZF<|eAnsvEwOn6>9KQ4FA4FnXar~{vhTP8;;X_d1zOvthu>O}yJeNF@R3K&6K#4E zh?gK#ka75{c8G14vD0((l&@IXA!6=)+XWkNZn7tcOtxxb<@gmNd}sjPmV;kzB%aUv zODbDb>u+!yHfLd~q5-MO3!PF{jn?y+@I+P0eX5m5C$g~AL#53Jb*@DPk!=dAQMsPA zAQP;~=o!`$KSBg`78 zMGaQx5z0z@WDh`~L7}mfr0BlnCcwv|Dthl6tsH0&X`RcO-OA+0H20#Dph@RL{VKDX7#24?2Cf+l{dd)iGRJ&t8#|T^Tzoj%;INBZYQE%a^^}G z#o?+kAnvl5@N4qfM~4>U(xM+bnT|30%KI1b>O^=b6~ACdVGg z`x)6?r%;$x=_-sx?FG^;0_YKEB(c}Au5weU2lL?4ON~W}po-dp7lwMa#3#jGY9_Fy z5p86P7uH#zQnGf=c2PRBbQE&0<#O@UbUm)9iK6gF-C^CK=m`h?An4AIWX=IPM733w>6nIzuhqU#mRw6Q zK37fLQK#{SKU#e`jELaah1FHp!#|$?6Er}#029Q8pS2xd1~o`6O=XS6Z!RGhsu9ae zDwmhPN6H}sFWp5Ua%R9que0-paX(*+Cd{gwj9*kGh}s#-enlxky}O3GFOW@cGCj$# zIxT8K!?I7j-f;9F(t#k+_fq>4`d13PAXl;PnTQcx`qCe;tjn}Wf|Vv;Am^?ke;A@? zA#)4=NpVAdUj;wtl5aDF^e7qs17CjKMJ~*(8cwdM9iIh&$Y#*D@eE6yt_Z`LbEpA< z)wZ^0aOoaOk@(0PWtrwTF4!m6D1V|2&ZX)GX4IL1%(yx7 zN;uXl!vpc`uV#ZgFI`LiCD!SC>V zbeD)A1d8&k`{P}a!a=$sbQrM}r0;(l|I3HHu5{22PmPa-L?kj`Uol7gkV6Jqz80ql z6ZfMK;g;?=l>~FA$CbE8XuWypdXYMZZB@Qhe9L&GqczZ;%KQ;+H2#ym1f04MG533sWO@Dro_f{@E~z8oUA`;fHmetQ1wb= z&v%35Q<5t{J<=GQ$jhdx*AXbSru^|#T^uqFVNr2=p5I&=%+Av=v4Tm4tq=T;QYV?x zI(aDr=TU~8hF0Gb4;p$0rt!NhmVfAhQG!M1f;wq@xJqCyX+y@#rOlQlEh!HAnh}#a zA5`~F8t_&i@dFPtbHrNTyfJ8mB?TSHqA+&UW!Q$STKbyR)7q+>Fvf9Oy=4D^qb{mswKGb0|%(GIj)pfmk`fLCWf%6`np=07+W zBb)!qgcOmXER7;VM@Inb!|f@}WLBGEVq$`L?(VOfA}^`PQpF~LY43bQ;*loaeA;)x z7_X+3D+V&oP*3WM zN`xsJ2K%XtwK7pKYq|jQG)mlr2?1-G$+L|N0VCZ*-wm+D(QubHGx#TIB<|QX)uk27 zK#}hpcXL4=;4eCQ4*K|t%u1HA%?E|PqJDi~+DfaLPYXn^HaV77O4ans2ZpAJ^$%o( zJIOJnEFEoiTbO$asmtL`HSnAW+mD%Lzh#X0t4P<^c?<@i_g0*kaGe8(l#*WN!}z^o zPJZz>9uizENU+anB`q4FAGvABF-V=l{e?t|5Pb4lK;cssh!Xotyv~T(bZP#&emsmI z_eU`_m$ki|yl8sTn`h?69Ohfp2=s3<73?uhPT0|ER7WsabgT(BH+rV!_`qP4UNQO2-*N9RAH0)Go!d89Tpxld~x~~ zM^BURBJi7xhy~Od$1`=~{5-?&41hj#A)2Ffp(;Sx^O9nsY%i;?xQp#c^-LF2%@^ZWx}i)bj-Zqv8nz@bw!Tkm8$~bHU~g_O5NPO>+PVrWjOaxmo9bUm~9RxgUqI#Ocj-6EBurQ$i)mFhX|j1aoCyr?Ph8~$m7vb z@roU7g3My^^y78a1KHv1l`0$RIzmxe?16%n&^Nug!2H2vv4`syrwtF4XyC`o?L_ax zc0=408MhV7elTCTBG5MGZruZ%`?~-BMhQ!O)pcm)h4be`P3tYRjY97Y))O_yck&Du z`Rm?+1?tZ&aJY-KH9mQIeaQ}yRuVNW^X7Z|^5ZD^3~(0EUc#)V`yL6k|C2lE{(RPE z!WS3CgK^{b`t1Y)nGcL2A1`7?2|;rM=B6Gc0KU`rGAO0&zWK70d+ZF-Fmbc(%(TUV z$t^Jw+Nu&^SiTnTuZ4|@95ROt-j2Mw1L7xDkkT)I>{2Vbe(@Y9PcnO>jRV^HBzv@0 zr~#^UmK?hv#>!IluubZ!YJtaX(<3yLs~(pem864xw!*LL#}E%{geW9|Du;FpFPjl4 z`!f*msX{#>s%3)$aGJ9oM+Zq1NB1}J9B(M-yJzJ8z*YM)25m>!IijRp6amIyC;m4M zJ;Xv9hlIxT6!;baDt?vFkn?s}8-2;LF}7VTMN&knEe9qQcAk9+1ZIlO>4 zg3mHLF3XKwWW1%dHB#VG3L+@$obObFC1#|<_%;RR^(o-i-Kj{(#@y&BLvvpr7ze~f)) zR9wrpZbE=y3GNo$-Gc>!ySux)Yp~!F+}+*XgEb!9Ex0>0`kL&s-#zD^os2t1|5&}c zdabIO_D!iOX>W%DxksFU%c*Et=|kDbr-#5%h^uWeKPiPrFYmEYkF0`o(Ej$e-xwe9 zLXH&yw5?4aJ@*scHfRy1xMSXvKk`2Id|u^ZQd!}HYs z3>}MzNW~C`rI%OOoq}k93=HhRV?h2h1_sq;)wOb9gCgLpA6{Mfd_iG_F$@rHqzWQP z`c8}zasm9_e>^Bkh%gE9-+2LJk zd+p?bajviy=wtrg)|2+4_OppI*_a*JA>_+ov+=aS zM7ZUYTwqLqAwL-O4>DBa5uEU9x3+8w%b)!)S8qzNeh%Q-7|xJ3X0we>6U;zARygQt zn&J#&#l|toB_U8!^QmX8qv4HHieVotVNm(0PNdm)f0>9HVnfjRAwEf6m~RU6!G|54 zl{kxno!9jJOeU!w7P}(*j^9{XkvXdUW&`5@z53<%;)W>k*DmYxQ5G?py5e)O0cGcL ztcBE2Spoe1s%0a}+Ase*~)UfGDY{-bU2)6+i=`Rb%s&)e`?9>=vB@b^o7Hjtlx9=ue_gDYP4P3w9M(q?e?yi zC1A>O1!#dBjAAYawQ?QLNA?|Oy(g(KZ7Ro!quGMu_hdt5X*kA&6ZL7dVFxEz#U(maS!qH*HUhE@s(c5M9>8_R@aT9n893Q zDLmJpQ3CED%L)+GyTfecFN?xMF>q%blkP6-k*%>Y>NwkfC>EoG|d zzs_4AzAkm04t%$0V~UDoqKYTGHm+h&gRo=vT~67eLUnMdV|tSWoL{{ktXb_aUoGHu(Rub=f_>|^iEykr64fpj zI95c~arem92cTP#733__pCBI#ESE)00NSmWDG~9lujSDu7&9@u@lVnu%!3)e`N3Ii zI+Z)Bq>kzI{=1&xLLQb8MZemIb*?+9b%Pb}5;^hDQ}0mJ^Fw`-M;W@OW-&vKFjdOn zyxw<_K1kMp$tKukWY){jA~2f;3nv+rUF1hP%g(@`l}@ka_M*&$(5jV>dp?jZ}jXjJD z<2R`rp!zDg_(OBP(d%C^%l@GnJfmq+V90!KE2nT;|ID`q`P)k2HB#ctdG9CE2Zcsr z5F$F@f3=8ZERE37qW#4pnmL7dm*FZ@0u@~QCx)*4{rTh6SJm{` zO;ZNs0mer<E(wA7*96i^jNxBzv`W=L0Q_RTF2Zs4dP*UwI&1fO!@g$t!Nc7AWdhIWK>JQ{3lLx-gu=OdUI~ z{`6{pic%rk-g7ZG=q4*P{=l9b*ucE&x764*!KA?H-!SxJYfv005DRX>U#4H@o+a{5 zDP=_d<$%e0Elu(tlawPr{HU-asJ8TDP&u$%Mud#vskdK& zHjbC@VkJC@(DW` zGb?+L;4N?C(>H)5uks2Up?u*|5meED1v4Zf^4MwFcrFp(B@1hV6Qx(l<`za49^42J zeM12r9>vONLavBzxGIoA;Zgykc=o|Pxv;GYfvcQ1)y>;#zh*(9KWAeOq!_MvJh*PV zl!!VmZxRThPejExNd@d6jb=}OvQaW5Gn1Gaqi0(gBFQH`ryJ9=u_rFbCC)r6PZbJi zR~&H9l8$HD9fM#%$Ie*NIlC?0i*rYeLE;O#R>L~M3g=Nna*KJdFS#66<7IVJ3K&aG z9%xB~h=U!`NtgLaABm@k63}uj;@w5R&ZqEaOHK^3_nil24oN9TUrU*#3LHU(9E09P_*SdX=F>)EZI2+Kc|d56E<h*piPZIiHT+IcWRmuvgfnk1;Gv-O5!F0DcB9OV^h_7b)AEanGX$$71`#A5H(>R zbU{1xbR$zGi;1T3N%#QIAeDTr8{sQvoyk(_5>YrOWSs39xYn@Ck%*zu2h^%WrtaT- z9*pmP`lW2P;A94_H(wk<^ZX&)0YM?eGj`@#%TuVgJC9&((fwy z)D#KZ1bFxqRpCNTk-r`YB(P&c1jf1QpEGw+>t%g#mZ7E7CB!V7WK)6Pkj-u};0YX8 z&__7%MT+K9I7rj+{Ma8*iQ=oOVribV(2A@l!-z9lK;FVi4t(oOVjfS9hG$E)%_t&} z)kDB~Xo!g=t!ZAea3$`|P&%@nOHLC;+C(MmFgK9-E!*n@G%UH57;~ckY-N|z3M67l ztl1Jm6}%-S3YdR9ax1$UV7iEMPci^X*?bQiYUMChr3HAsPvyPqtJK&ItDPpTqX|T4 zjs9&Gu$}=lWtI=?_6H#TVGH_nAR!R-39D`|Z>I5+7|B#K@(@wB@KkIm{MGR(5;Wlt zMZ&^UVI%>@AdM#5&i)>x1n~_qGo(R+2Ed0(frBOzc05tgfu=!EhzvxmP((wC>^8W? z$b&fonKs`Fae;U}-)mvbSXA5@Xi?rx^PE^Eh}+iEeMmF7c0$LX`Fbq|M=}GWRPv4` zON?N95M>`uZ;Memr8N582C5ilx36vidvMk%E^}^eHXQZ+TNnmi@;4Z=2k<*w56Di- z<%s9dkM#xvw59P1J|wEUpI-YBj8@>Fl|9Pzkk*&ZvJuK_j8vJHm2Z=3{k=-+A3`QJ z>7c4nS2T}b6$r|rnT}-29@1o93&rk@Z)b92xLy3e`qZ0EM{Gu&eoc%kWNam{6S9y@ zp@_C~w|kET?OeB>HPa{(ExJ{alwK0-)GB~KKTx^F7GkZ;!cfdIDL&}fCaIRmsq?y<$ ztMumB8Rb=MTO?bfw?tLMSbuO722uGH*|>6+n4feXW}^6Q4)^ej4oFn1Ch{XG$7w!O z!9CWc7E?#-%%%URqg@pH;;YQQ!Pghi!{0!U=)g|a3(dBvt&EOfRi;@#9BHa{&ItBL z|LotidOhhYhKcx9lO)uubri9ct{L*lPIYMZ+=w;pTtQ?l z5iK*&_JI-n3Q*byd?od}+Eki7Vg!5ve8e9EsECP4YKR=ct%+tu42r|)vQ+Rd6(_WE zk@KyreNMVqWTXbj>X1a>Qif@r#8#q|G&ZU>IUi`mQ(gJgAEQ|miv}z_RB!my)NCpz2j0n~`YH1i09yyu@^iDM2sK^q` zPZ8h3wBF{#&nF62wC%#hJGu{2>|No(iu@I1#^qt{lS&^ zN{whlgk}}do2a|c>qKQ%`GaxVylw#lVCMIJ*#HnjLo(WV9!wapR4UOBa-j;9-9=xV zu+!cELsZqD1!hhfm*P$^qUe4L#=n5Gf(j8Vv)7PcX4%CW8vGr_8)>3G0cJ!j!lYMk zASQ`yKQJel>5G<4`XqjHbJH`;oyqVIIgMJ6gQ5*>&{18$WS#3jcL043T@=msHRAM_p0A+@RP~yEyITRE4 zA_-Mr?DXAUJqDqg6e?bV7%PkzbVHASpORfhstE+@=ue3BdOKJ{$X)`1tAM6v^G(e? z)c_iCJs9`5EX+kaMQpvsUWKdGrDG&w)J#O|VzKEv2%oR2 zi=X{k*uCB2Y6y<^58CJNe5<-RC}3;uYR~!-unqiFpuHylfvNfP25Z^drU}82_f5B7 zqMC1sM^(6ca}F#-$Xm==mxE$J-gQOhSDB%)ynIyJTfw%c7PSrGDD4B1*PKmnR>8jg ztU7M5s7hGq)CtDn&mDjuv&MGCmGpZH&7`Z-@x>=LNpEMv#-=vzuJsb+k-G12vsh>7 zL(epTS$v?W%4Jh*5c6QOR4+QN4sQTA!rLrC)i{-C^w8H%(_LtNEnT^>c`-s5NLW1D zu{$$En!Uic5Lt?gmQNYTyvVhXsx&Z52KG&Y#6y=?L=_-b>6)jueURZS-m-Y#M}9Og zpkHT9XI%#38!I@X)s|A!^4N@ktBhLM)tx^j@x?*;fM#a&^p~ulk9x$bFo{?RMRPw^ zrBy&8tw62qdrAlZ$$snw*X>_LM<8VI-=8&70uYRf@j%dkd`Ue#NJ5K@k>&P-3LcHb zq9vkd`7QaFtM|XoXx>w`M$yWnZ^k|4GhKKQe2JZ2MAldq?FN;i{m$cWgaMPA4E`(r z(rNkWl`R!>;C^YG!=%}wXjZj$N*VtN`J~dyf!EQ`qpNezhOR#AZW;|yj;<7&`9CcU zvUzc4P9wj-zP#Ay;PVF)O9mu2IZNhwF-u6OQ8OYK@X+bey7aD%=N22!0eh6+_7L=M z!7tBdu5BWAI4_MFyTG%jpqZQNl;{=Z$WFUaMNeU1#M?J7g&sR$>DDg}_`lee9Qc0o z`yCXskOnPjT)eQ~DF+F?)n_nL%O|l&OV>~HMCleiWj*?QR)Cy%vIYWQ>^73xrsf~w zYY)E_1sikZHC~OCg$#V;Z-R$0(ZKdC|UYUd%>Z+AERr3Z)sS)U5*@azsu5oW4yxu+hWBAgxAinWpUzv z<*Z%M!9+Af<*A_60E&pHuaOU90bvo2@{Nhz5=4Q9L$2d}ag+hhnBmr-MC)%y(K+*y z0dw0<{r1AiE689U32aGhDSs$qzs**p=#dDRkhVXtbvtgG6?YVgcuZs^KjZwv(h|um>ngr7w4(CvORQ$9Us=RJcme*p?0wDFRO|pUeA|0^U^Xs zwX8azp4dHS`@lD1pg2yX`D^gqb% zE|kWTHmzvTFP*U#PlyoLrOa>nsgcs-volqAg$mn6Hwn1K+e)RPJ&I}j;_|50KPA#Z z{X_?$?UaLyHM18b1S-G_%11IM88rJDWjvZR3_2h&r4@#W%a&BlIap<`PxAq^pz&{m z@ZT3~UthUvEMpv}#(^HyhrD|j!EhY;8|+!gCvB1XS9 zz-4Qrv1LSo+)@4ieZCFf54#YIfj53fpERa-cw%+4=Y}qrfBFNfk5p%qMk*Fm-aIBNmiJ1a0DZoC4{9tCO925^_2|}T7 zUC6r$b)_x~RcrmDwxJNxZiV%V$V5)u?TaWqX3W<3-7*u?s(=DY@HOqiC2O=WW9O z$wxSn0HGI7@4F?C=IN{_)xik?pBc^8RSc669`&CxkXm%ppcKfQqL+7%zrgj=Vyc)? zQdXwqObIUqkk%BF#}`9J?wX;RtxicYLqJxlmfBCx(GN;26cq=oPjuVNf7F!hUT|^z zn9%%XH#49gqlWMRlO-{5H?D+hLknbi?%qbH{C=$_NFi-RzJ4JfXt_Q4GFyq!&?95! z9DZsHs943Q3NFy;dewW(=k2jAXBcJNU|j5}pL{Zfqg#18598g!MPPqDTXA#(rOzPN z;=wyl#;WA2OACaVUp?svNY465Hn(AE{xnyh-~nFcAl4 zAzxu;W+_*`5y9noT=lu*q3E$CcAuC0c)$oHV@XkkU*>IW{sE+@zwnEaz zT1`%;LYC5G#G^Lw0Pnq)E__o&k{6#1gTXhXpErdwC`NJFf|Z;I?U$g@ZP1JR;H9M{$ZwARMF1L?RGajf7XgYeZ16mQ*bQP0yAy?4y_zm6nV() zb~sgBp~+xEcrG`crMZO+AOA*#V(M+_bOY5@UWeCNj{#a(i~UJ^$4yb2Gu3qHkB@JW zuUn*Xc~Q}|Il0pt35Hd-3$SPp zfcbvHX2O?*DjaS2&`?*3t_h+9UOzhg? z2Dkmzlj4{kCy}3rc20IxAK;M@S$B4uwoi90;!2evUzaZJ8nt*KEV2CoQIPtGKc5_> zSYJT@YSei}_yuZ|Bodp&%Z66fSdU!(vFFYGNo{mKzmi$*)XBp<(^viB|-jI3%SA>lytKGTKSyR0$8+vQHv;NZl0{c1b zZ@{q6kr+sGwRoNXHA0h6DWy{?T9(KTZ_KMlnOBuc`&UhFPn*=2*I}P|(kXIaQ7x0= zu56yqf`R)N#wqpOierTkC6EELRZa|Z<4aA{M4yNIQ!{vk8Ng9lS+#%JCR@3B#yNGqrDT}QFppE9#6!3 z#Z&cmOQ*UiXC;W95QpgeU>7jkfCfDyNZaC_lBQyJ+GKk1D`u9f?bVMz0kx*S()B(^ zgKqD9)&97CoZDx_n}6w=80uFi>oH&J!}Y{TR}F(Kz()#Xa&IX~M?-YE_!chvfOO>? zF1B(3@-NGX`!XPf!Kq#7u-xHjFR7@-ZyPNzTavU2>QS?{G^-GICZn`P+C+ZB??1C}2o zZ~FlyxE%w7u@0ZU7Y7h1$QS1eE=unYt0S-OYE`hMr^>SmrfjQap(>hEl@w?1mMKxP z6)Q|2n{yoG8RW}KRv*UI@bxI34v9W4M|$b#L}!#`K`M)&=4_jm>NJz31bs7f)+gxBDP{(CvZ`Vgs)P~^4&nw0 zt>OK)*=LV?hAA8ot1)!!dWF!PUj_WlL9wAGTOd17x}hpC6?sTRtg#F z%2jNC=wf)%fNZ}%Qk@ONh)0h=`y1lQH0V;YSUoxHdHL9?7(yp(x(d|MUO~}WiHB0`}33Dr;$ow7aN%u}TVDF5Z4weEoTv;-6Qda6Aymn$-6Z=j5dz*3~}iLXN^ zi`6L;3Mn?Bkd>4)PGg|A_G1shhDs{l+&`}Uh#FK)xOq!52s)*YJxo(FPzgyHQt&DE zqrKH;t-UPG=6nI~nQmaVFOOt&vKq_8d{0xAB61F8!M0Qx`PHDrV<1^ZiC;<7jhW!Q z0c!xSyor79AJ+NkEF$Y{m@Pg-A-t%GBmP$;cbPH1>=dq3&_4>X2&srCI|(S_)L}ze zgV`P3fEmP8M&{Jf%s#}lG|WhO2*L%1C%<7Qv!a{P zc|jy=9BR#$uiLW>ckOPqS+MwW)%1N3k!3F^^!% zV0-T>`9^K*-R*aG`0I;_&8i4pEh_WjO7!~xgO216k9DO98v$72%u>dlDRBoSM>E61 zzp;rgZiS`Tm*5i1yDs~eI_3s^A7_+ezshJ*g^jAU7>|_iFq9-YxW!P}Uif#iUBnO= z)M?yT6IGGR&f3A_adM$zj||P7???)O{`J6esfpn5Bncy(Kebh~Ec+I;TN3e$8rnt?4gLy_Fo8Ym|fHI722% z=4GQCtLP#LM?q$w48G0ROU0b8aq^0cn8{jvI$6Hja{V(i?N7ybOPVmI#f(MVYLEMo zI3CBho4Wu2U~=dB2=93Lah&XgGOEaT9#qSyezlcQuoGKcT2jke?{aYo$T6Odz=c75 zjyg7rcIzpoR~V52OdSkM#EF%iJU)fEowro*I6su)QV+{4Rcfb~^yT%=HC+3!D$Dpa zT;nym{gynSEuBES1d)45{R$VHV7_joQrN1AQr%UU9z@|1$OB~uRO46xIYV(H8kbpv z=uaD*tKNs>Z$>lKbgEfts4mXl`1QYsi(Uv-KW*sE#vtbmD3#lHH=fq^*y$x*RsHUc z$(AIEo^~HY)^Rz<`+&IWAFJBaM2~CLiT-}GjUKFZ_SL=wXjy3yr`Kir>cd3JSR2bs zxi(u(wS$b=9}Ne`OZM^UP3IBF_@$Cns;4e#FpY%NnMQHgG5bPdVF({6{+i-i!6vkwy1tCEhWwt++{qTeam>5^Tcm^ z^Nr$6{T3VST~(zA21yCxL^Z~Sg`BpV+Y!~Owj&y%a%jFQb6B-BKOU3IqYOPVKZ z_Dw_Oz=z(S&KZu*PsxE>;r|`2`PUtgeFWt?cBC`hK@MtT8I(g+@r(*+Z;TxIsMZ*# zdYrL-bL^`-A+fTNm?~G~{LyBO#p1Jfn{w8Brv_Kf?dK2cpT#3r-1~i&^DUMY%kx-2 zY{+Ys>_pyMx7uvP#qAAH?`mPL)e}$0+2P{5-=^OjO_COypvb9qob|7IU2BYS5LZZq zWR0)f&8Km?XFPOfN#kHO&cP|v8%(9Sqmh+3vZTEq#5?{|9IJBVWegYiudFgaI{2C( z3IbZqReyYsvg<>aDhXG1Y(^S-=<@-(*6t48S&=BY5b)S_f?G4c*w!opmh8uVsj=Ic zfn~c#?cII}ow`H8;BtE!C)%JLV@6q7xb$6fk>X@5fCeH|%#U8q^4+I0OVHW3>u-XL z4M|7ifZfU~>|8WE-!(5>H8;8Ax|@Kxt0$5Co|h%7OC6WNf$=cfz-pTo=SwXfCW@Nu zBr?;CKN?5(muTur2Cp*+7oKV%xtxT6JC(GXJk(dj`sc8|$0exb4_jFz)z%2t5`#+4 z{@OXgvrFwy$_6l0I@t8I40qg;L%kYiBUOqiluAn_etxcT#y?HgQf#}GPD&zHpqR)d zj{R~;I+5$YvPsyM@I`?ReKUJ*P5KIkTshbIwJbOv9iiqr7@^BIS}0JXar1%URQspZ zo5zvERt_4b7!G7o;L%v6s{2vhjMn{4D%ZW|jN7%BSEZ+I|2N_Edx3sfygE68Ifpl)ty2#ouc~U4H0Sx@ z&MjaM0s+_6RX8}2CE6OGxYXDpAIkW|`H zrA1!NQN<=G6dytv@Q_c4+c9g5l3(M7AlP4lIJieG0rL`Fq4wVGP0y4E+^ zWOXOAjD>J7-FsP1Wu^FdvYk7cVVZPH+r`0&K~dO?#WAPrm;|p@#`%1f2K{P3F+wv_ zTSKq=?(XB>94W0Tcyrgqh@+N-hUc~Zp%!P0gex_T#8oH%NM)KwgIV~7krDi=#>QIv z*^LOm+bZHhw4*fs%8s=~WilG-!_zhzo?D}t>!xMn6!pf=`kJ(x!lpFjEI^honb(c* z1!u$q;Fv@h{oJ|RlH$Kc_HSceqYmpNE}WP9YYN^X`T(z3;md}&D*0wv0-BgRMhJXD zA~i>(8#K#?B_rNxtkl`-s&fy|wzh)8;701JSmUl2e*T^*fD=MYQ^px$OfbQMY-Pt2WJ1 zvRHJN0G5G^2D!iK}(?H<5-Z1{F?6Y9IA<7rP(M zAIf7a`WLLorZ?*I;(9Y49Cill11uM}osQxUlP#JTKm;^*Jr*WA5h%y0(qAl(X`maX zqoqBJZ~Ev8fGi{0apE+vG4*;P8yeDnI9>wM(mOk4G%c_=ZxL;m+YAkPK62V;@U91v zl~U7LXtO`h??kk9a?sU!9+N+n4yQ5m-sChjb$Fg6NNcTB+3!4j6Shp@)ibraNqPcv z^!?uq{r7->v3e>bn50G;kvHh;Uh#G_t%OKZ7#`)Rd8t$y9C zfm=cRqQ&|7av;ExvwmK-Zn3{?J08uT(TN&#l)c{KF^TaTysBz*tCcgap$n?eYvUso zt)=?b>PqseT5|vz)m|CyB3goy2Sw-zuosup>n>*5+I=jL*7iVqq22R1HMEb%)ynFo zY1HvLH2pqxRWp@1ca)XQIijg4$4R=ucVvvlBISPESnBrB+tczUBFA&h?B-k=9Ub^_ znbc{nQ&Twfp9J{t@v`5(2=XaISuaS`-@}7PlYASVSKpM+b*;U|xo1U9oVZVrQML72 zNv@`~51-;;m;tdq`!n*L_+*9T{Pdhj*n9Aqr6pIB(GgJtPj9Q^9!q=P^pyLJ$Q_>3 zt`pDW1Kx$YpXg4KPF|wLxOMKPhWAUN$ih<2Aby{;bcyr{KOB?JU;LLa|1V(BRGs;3d5#m#dW*{H-7);rp6BY_TwsId z#?F}YH`u2KhMkYVAfi+;cIWfsCj1AN<3(Psy@OR?S1|A0UW3=|-#+5b-_aiZ~_Y1O-P=ks4(_Z0pVA~ra9Q-NkNU}v{BxEaJ3 zh(t?m#p}9brTzV*-Xcf5Gqtzzj_B&+1l!Xi)hd^3@7pR;_1gFZgQo2r^m>|+d!G0l z=t!0MdP`kf#5Q7|y2i@N%G$Xtlhq0Vl0WcFXA-)P5X@NXY&{dzFY!&- z#M6sznTU=`)}G_x(6hBw#XHypX~2u2#fUkc2Fv@K1ZQe$>QNdx>3Wm0+s&DWiPT6v zniKYOsFYGC+N19IcFjm>L!i}6tF&(2-y{ET+XMQ=22B`5sG3`RP~F6fcAirP-%Q0p zWal8Tx!a>Qi&hydk8!5GS4bRDq=Jn}KMHM$cY1c2t@czH;>oV7-egh|u&g>CW2=+8 z+mqfwGiX^|A(FePsC zHN}>n!Ue02q!g^fQbx{me0;nT8rpWl5qmeEuYHU2=$QF_N*DoWU9Ee=qw-=TSGQKl zvbcDQ(}_`!aShs5oy>^((M4UM&YyI04IzjZlCB1ww0`k|Us&fc@Tgimlq#kM3~|yT zmH=c?R_mF8YSXk7B|t&-f^Fq^Wv62SyX%iyJjczvsu#t&&9YfXl*$ejQ*$fpE!9`rdeYy#LC0mC z`?FxG;`xx4kDhyjL1aRuZXs5>?vBTbJ{h$UX4T-icetusdM~C;ZNe%K zwQG`4)UnU|lCaeg-{*37xU&Ib8gTgx7#jei#e4I<7cmyI0uEl$PsvZun^1hBG-w{s z=F4`CYfMQC!z@9IG3G5@tutvBNxz2p&+M>99RZpM<1^kbQXMbwsImge!hUo0;k^9&@9%m8UgX|U9U-elrZ z(xwtoIWw>0psg5EYW3>eihLHt=j(~d8pr|qbW$eT1#ctCX zhd*&$DdO(Xap2Wf$;GcJ`5f4}A%-L!@pd@AXg({iFcpz~-oEZ|_2FTHotaMf)j~m> zJ(#FOwZfnS@=R<}xvG621$VVlV9~Q<%YS0YzqsmOMk;k3%AtQ6(+hh4*NwH)PdauY zYq`1Ush2yP%AmBxc1^IEnDi+8uyUHy=*b7}`7b2FEF*Fxsw{|!$0NyyeaV*}-J`O$ z%9gEJFl+Mu@G2Ca3`g+f#yiyx{;P_YcBf3fo#zh*F`()vMft#dAXPZ97w3j;#6S|9 zI6L2!^1ht7Hl7r;qpQkX%w_UsDQSp1vg%|Ggub=3nYmeu;vZ&@9L!i2^sbMgTDMX! z!!P@+gA?6g7Ubs`v4m;pJ$q@R8eF{BN~)fhn}Md}acL&E)7X(0kVnBV@66Jn7yK!hW59KDYISV&|O>}@-5@TLNyvMGYT6C1y3}vcmTY>Xt zIjLx6(sA@3Bm(p>LqwB}eQNhT&ND)rcwrss;vUwdJt;yLiD-H|O81nvA>&;Os&zkZ zjbk4z?U}T*k~d3Y^Y$2g#F=mtVMB*vD%3~f=SLJK)SHF!DdVr_Ql>d%?!vl_XI2mE zHUJ^P;YFijvA=NHfAGz}ebM0tm$LS7;!fE@{jLAQM+}P9DFO!u&n98N$kA z^fIZ(A0>5STmP%^e<}kT$``t#;#mIOiv1DF;>tDvbY`~NKiaN;2{~^H$cS9kxN7eI znQ#8?(Wu>v(TQ)_Otblm&nzgwmaXdYPG?+x%Xghg*e{e`nr*20WvWqpKvLVB(e%;b zyL0Y;4&y&mkIn51+F18~$LaTnR{!&tR0;$T(Isp#IfFs%-$R1^^k?g#_Ux=b>X@IP zVXx}|=5bu!ZjKp=)7@>$TUr0M!=Dh@mtOeCro;E@&l>UXei9_9W)zWxVp z+uRbo%%k(jzAb3Tyg#9a$HpqIY&5sksyx?A&CN8o+f~%5YRZcolgu4&0)AcG>sgi! zH|Nh7mQOFEkG@`L&0PP;QS_l8GgM>pXQ}uf2cG@=3$@_<+TWo6-{fQX1#eJe z<$*vUI~74CBiqjE5=BE75SrWc^)%0?4Z&IoEH-rDxlm3}LnkmL+NZPM6$UY)nCe;O@rtn~hmM8@_e(%A4lUF$KBe#u8~I!F zhj+vHwZQ2oQ>~&;C{;^E`OYJgQze}~L~B}Ui;faui;js+FfC3;Vkc2>4C*HhM)*oX+_PMJg$Z8iK+fnzLrr@MaL+z%ZB9gH{W>13H?(=uaD0UeX*etUhi z$Z&-;vl+LN@pM^W2KH?U1aMnPFu{1=R0ZdYBw`Rr^agztm-`AzkW`|{*RZ^%R6TS$ zocl(59XH1o&3vCfU$bd>{5o3V5!Hb~wgdceRe~67ENx+x;@7s(%BQ*e{_{g>*K?Ij0c4 z;}H??#6F2Jlj*Y9;&*(I zCP`=akx_1)uoA}SL9wDtIT+W*zHH0p{oER5?}9eOK(i!yKnCZ!&-U4Do#r+kIOM=W z4OK)PAlip15$fhNMw%xncZJzLkoSCmhyl=_8~~Z|iUCKB2=GntEiV;{xk)pFc%ng= z44DMAB%?VjrUYdX5#Sw(#AoA7c7y~dVTPVe7Uj5GWKuON(iN{``wVx0xR!6u?g)a( z&umO)tKn{fwi47e3ASQ0bxP;A1Q8{u)61i<0jFE7~C^c`VJCcMd7|iNnenb z6T5SxVZps6AA)+{p=nS91ocoDpy7#9Bj-=NpYNM_E|SJt*C6PEwr2iWw{>FrcBbEi zjzv{*?yd)XN&Ge*nJDF}VWP{U`$6*c2mie&L{zzJXEQw^xZnco<@9?0PrqI%+?`;;`x3trHfhXh9>U zDi##ZLNZiX3Q%9YoAG{}^?uxD>i~vZd;BCsZwe~oP3U;o0uI|BWXfJ|!NY;+JH*%2 zZ#>LrJSHV0g59*8crx9oI01oqbe?+|PkR~dk4M%ivouz=p`qFj8|GF(7T?QuU%5z6_@=&c9&Z$@{T@@n z`fDckd)S-S9qx?t*aDS1hK!5U9j<-ZNHH^}_qD&lpzldQz8bDrE&R`_9$WB^o~IsL z{}_SfDd-Cl(zrF0q&F$VM83j8qRmhQVkG64-|?TK_^QbCO|ufyPr+KthVQ|NF3K!j*Yw; zT9%J~M0#DKR=D&otrv;M3(QUQy{`gMKTg_xVUp(=IHnG^^Z;FW7O0=x&^%q1e%nL# z(aD-xI?RA2)zP%jA46OePEDKV=`50WPgL6NB4t&AEq{u-GGBRS|HkX&2Jbv7&_FRH z@L~Oda>%p~3f6%MO}^S{%=_trKnKV(3yM=#xGz&_RtbHZvCp_);=Rk^4cWPA?C1Va zS?)P(kdl2L-_W|n+Rzlf9;@S0B(AxC^89o{=RQBg*c%ug=Fe+Hpzr(>dvz`@1V$dc zZC8-@aZfs>&S!(>V@_^T(t;2v8JRtr-rla7YY~#J){MN&zoetXVp#4n`6%H7n&EzvnU~Q?>o<*+>l(WXuc;U z1JA`c=sBk;UmPnGACRR8sWg*~vSvSS{JEKDP`1>t+_^=wiMKQc)N$oEY2QGtXuO-Q zc)IQ9U8#;+Y*ABIFp#1C2=CJQhUb2MOl`YVat5qT^{1X#*}?Ne#^1yoPy!v~m~O^J2&2;%xIw-P7^O)BAo04B1BRv?T-1Z!`kWKfNE|MR5TIVddtV z?lV|yusksm(WGE66P_6GWg8^s?&eg!F^N!+fC4s2oF1V7Lrt9gCy+sqFI&S}ST=Z*2p=e-3587KjFo|o%vD_Y4l)BAVa)SrVHyQfwaP~nkVt^@I23_0@#4o!hr19^Fp9Ei} zeVN*RUH6FPOr&dtaVGt=DNQ^9x8LiL^*!8whz_g-;k3WTN|nT^bHp)RAFr}YF{G{o zCQ5N6{M0s5jjAbRa(hqsgZDn)qFYvQy%{^{=yri~$Ne(Umqu_NYW}9DOzuP>9z?{4Djy81C8Wt zCr~iLi2HYqRPkq&5W5hVYIK}jMlE!{jodBFw#NsT%4Jw4r)6=xdYjnKJ|?-c{E!)k4`3r5|(`{s|C-N?bC`EA^O!8r|sVG;gvtK+r@Q56i&ZHQ-&GuD~ys~ zvA?;G;H3}%Q58~Zp=&kZ$5>7$G?c5W3h&2$(0TNu9x*GGJKONjr|{NxF)1nn0|K*w z-=bRqgXddx-m0|3rbt%EoQw0LP7i1OL*H(9V`UFr?*@c<&Zj^?hcTfkEI*F$;9Jbu z;kGVuOkN|Pv}VFkLp;p!o$wJjZ%c*N^W7{bzVy@Z4<}c9UzTn&e64D12}oM`Ig_Au z$dpe}Xy<2?tbmkc$>9}tT?AZM4sW`)E_+t=IwzDBJJS99F?>mXe)gP$ILglln&t#J zY6R>b+>K3sGDZMUb`UX1g!GcocG_k)Jos0uLt1IItL*ZKJ3Xd{^Uk^|r9C!~og{JH z53Af_&h$>LK_v6uarO%Q)@dRW(8X>&u68*XPT%0u2$pzFkSBiO1=6=r*t+jaH@*f*TNH;gcB&e3NUr#Pg8@`{Y*D=+ zBv(OM*R?nJBBW?jVdlx(&DYSwQe?Dw`2k<17gw%XTN-sXIPs+<7{OD%Veb3C{KNgb zW<~i9_lauq8ket)#_!{NC0Mo2sd{kvD0P9Z7E#m&w`| zR&nw(AUy0+HUOK}>zPLZiykY+y(_lX4HE3Cqa)f~v9M>$L`iU&jcCV<%1wic24#5n zbNx>p&o|EoH7ifo4F}N)b^%KO8tEC7z_)5xzC>lPTBp7fnkZLhNHw{4Qqgr!y72Ca zno403bf*SEypNN-ZQpT^b;2Xy?_C%kLPS;u8md#}ov&*$()ea_)pHZzY}*brrAhnl z1~ON%UvLMmzDOTA_B+014X6(Tx54N8${ioO*;9x2Inb(g-@bMDY2;WQ_AVaL^jNsN_HOv-x+6zBeFT z|1Gxp_$x1(8!S_ZN&vSG{~Fez8*SeobkZ5@_-%EA%gm9LGv{kW9FOij!61i+6VqTq zQ97B51D~KU9kibz@b&j|LfOQ^_@0*?&-Wcnv|%Jzg6P^3AzwN}*v0!cpkJ>iLSY=E zd#$5;1b%k#)Z@~oW z+;-Rl_%r1+c(RJZ!Z}!6L+@7K1bKCUrwTlg5y2c&Is6)?Ubzee0@1VeWi%#OB$uM# z@_$J35$J0$P`{?Dl`go245rtUK>WUB^6~p3(qG8k#s<;m*%^P75@7@P#HD}rVPv(D zU}xpn(SNpeo8<)r_wnoe7%mn&z?yCBdqIx>kFK+hs;d9iJt9g-Z5rv8?(POD0j0YU zq#Fd0+;n%RN|%H*o7i-RbV_&UUEB9}&N%nn`~Kk{42LrIUTe+oGoR-(=YmPxC8Sv5 zytwo>VforM2I-Qr^e6US;&_TlZ4xzaD#-Z$RLD&0WnCb%8&39l0UeFF$qVQxk&+O6 z5+}Gkxp{+z<0D=^d#yn&h}HKbe4v_d(Qek!gCq?jpoS%uu=_*StVN({QU8#=?^)^0 zor$IXIuSdX%gnE(Db_;jr{9-j>L zx|;84e0J@DpuSnodAWNFaGN(57`S~EqSVy;IQsh`CPWAlGNku#-tR3hP~D#PyWu9^ zq@x~E!1*va_Ttm>UejbDOZT&|9+cc7j%Ehc|Gzau?1G?7b?G?TD@cqMra^6V;*a42 zN70k3+{fe?FF$V4q#}pY<4&@{Z|MOcY?+ zpHS)v8x-mz$7pZ8Z3+CeskGU&$3@IDQq&RdnsdH-x3lg95_nKl$)%?sAjq}~XceOb z5BY#q?2$_M3l%H@{b!5&7QE|U{5K*E&hzjILqYm)OU>VL_T#$rP!W9J{@~rx!=U03 ziuR_bIqNu(PAV&mi~_2o{he^IPVT5-4JqW|_h;@4leJ(*nmgP9P?R@QpUzx_G%fLt zybH=rLO}Su!P&!KXZ3$tZ2uT{D>yKt_C>fWy*SDkrXv~!gMV2c^h+|={D zmO@5%n(TsPHfUgri#5u2<#$HBX%b#VcAOK?7+EJ$P}A1C4%uFvPB$|S4GI0=cM)$S zC9g{iAAS&|O1_|yOl}rXMz7h!GCO`+V||nu!bvwUmI;i8qMa0B1Dh5OSt_dL>1lR* zeKX#NXwC3d8TGLnV%`Aq0Jn5-4G@`r!E7MOJ{{!9PiCT$n9oVlC?zNl+8*kkEY^mp zIJ{nt+l*E{F@^2)J2F#yi}Mn&V_xCeC4e%Hw@E3j}RCi0d{*B)h& zCPZqenpUMbJ;Osuf?-Fkm-q5B?;fBAqWhv+4A(e#2@x%J7zh>>a~( zAvm7Js@!-1er= zStK-w;jcFc$AYX?bGFG1T@Eil25E#=wQ5bUx9F@+F5FA7CB<)U$fYb6m_bA7AlubY zH0gLyUDc!+=JZ3Aul&F&#-A~pR~0F0dlEkTreu=p#zT*leZBC*Iu{dN#_y(k=RR9N zqQch{G*cS!2%|V@xu=!>AgsOF_`Lb5Yf1~f*S>~c`rP^;URp!dH07k3(w#6N(zhHv{Z zbM<9VOdwsL0@i)pF{Z>?qLQ=-jTp6062&%fUSd5o?`?kpt&6UsEayAj_nSHW&xcV9xPKg}e5 z_b1v^P84zspm)WLjNJYUJs{v_{Nrj);as%fIB|CVMn;B2`jtPZ=A(n;?Z&>P=r3+c znmf8-m|Uf-8}pjE8CuI*MmJ1!_}9) zOHYqi>r#t0#P5$~Tgro;o2GsB9~>@jL(PRonZo5lBKb0O@UPC_zH0RGp|ZDzxoc$I z-|yn@@jQtUlv&mQV@-Ncu@@sfePM%kzab~~gf}PIVfs8*A^3AAVOnL`A60o0KaLnTa4D8RQ-~D~LhIR{ z0bop4r-UL$LBIx`d30%O^mz;vY}km}GXWr0Y+edNWA0bA34x~Daf75xEj_rTVdDAZ z9EX6KIs;5O71j$jl3x5O?bUCCr-(!71LHh=77+i8LeBSC=IhTXxo}PR%qHX6& zz6uQs0vZTgMF^y%g*80G@r`jZ6)ib;wfDT``PRr0*Isx)KS>HhdEe#^lz^xnqYa+%Cssb`ZM@LDBcUJU(9VspN|GU z&o5pB4N@%&dV{pAOqrQ9IggKX_RQdg$u7`3Ww6BIRJar}vZr6X#}<9wEr+MQduN z`jClsH6tsntGqLS^qgF-XSlxgg1Msk*wQVkiL23AniPHEqTF5hs&Pb#JTV6@+ zX3$aud!QZGuYXjFmD)&5;JO(9It!;haSm3!Qee^~RuW;TvOfoGG=c9*O|`^#icB+H zSrGq!GqR^)iHfx;;~<3*MS1L(ZY~JB{K!i9=TZE9If1v$>odLwGf~2qT8bu=z(Q z7f25wVcF*&?Gj`ZoTHgiT=8^y}VLMPk6wGLTVIKj?aI zke!`4qL;GR*UEBQb7pcW1Oy!l2B%netL3g`HZMxVq#}f#(S$iZANrGlEPqP;hcCpL ze4QFGLm)H?)zet^qBI^Bb^=G3kpigi^=k0`;Xzl($17(V!LWtw8ZEQJKIYdzdgtWQgqem5;sE^R$4dn<_5$5;oYRcMIsr=5ml((vnK$0-e(ruams zTY*>UH`X1mdH1!#%;WI#=NAv%tjnEjr$wGAIkU&G3jVGvB)l2WKXj7m&`cNc4YZuX zNZG3F^n_d5mf=MiiiF*v9eSVmBu!s$-Yq{p+8CU@jygeHetCkb6pN~Z--;y?g6qA` z^}T&a%nQ1{jbfKd|BAkx>MLq1*>eb&`Ajoq7aG^ACM3mDRu(6$Gkm`je9kw&=NV4? zkf;)n{;m7_=TUzCp$b_%auPWT$-#F9|PDJ1zx5rvmpu(!QZla_nO&M9zM@wo9tQxop|mcRsO> z!+-Du#KFoQNaSL59dngnIHZ+$*ty&}Q=zsmeNrSGShvRe#ViM*l7q_f_NH1GUH{@a z>G2n2w}AnxWqV6V&hRm^rT*Q|26kY}C^1la(Mb~uP5XWLnS#4@f+r#{J`!6{ai)Yi zWUxxHV?F>;#w+mp6z61Cf7X`_4jh_+A8goDphl0~sL;c%{03js1svRdmpUkFe!e4v$HRwD>#rmn23$g>D73AIBFjG?dk*R>x{O2M z5ajxGhESZjrzs^3D)JhWmiitUL@qk~bxVbh{V}*?x2yKaH=o&4JG<_`tFK8F4V!s; zU!D_A%Xu%)sdLlKfiBSF;y~%ISDI_SAg&G;_5C3&B4adMZyViZNv&fQXwv0AsCLE6 zv&Fp-9vR< zW!Lks-lO6d;g^8kR*TePWv>w|LJ|qiPhG8=U`=p^W2DV~#pmoJlaQt~^AlM^ve?7% zg$k388|#W&uXvRdmp4{El)k9?(q?$csP|6`iWn?r*X^?C-3Aa+>wc93Op_X>4YehIF)EEsMG`O4F-##wmb&CEgphNHMvgh2{NiT?sT7G^t zb-9LBnvOx0s0gGkOVPVI(+eNqzZ+(3O23vQ`RNN~OO(H9exC@IXI9SiOt%0%dIKDR#NJBR<9 zYGmHO(cuG=SqIHP{ZBbcvj)>}(4@03~C@ESuI_V3e*(`uC2Yx)u1zg_){QLsBd?zNO8IYJcW zUJn^U(K^|>E<)$B*}{jH^Zx>1pu_p2VdEki?J*TnUQ;$xI>oY~NZ2y2*1*EJwby)M z&S!FV*5BUv{-FQ#U%LP*Ev3+pWxvsgCrNqo9beYBJwR1_IppEffFrvMh^(wygcf>B zWcZfpqJ6M5!8-`gD0pJIiCLxYa-@mL?-Biu5x@5}%8YNc+iiDA}%<1Wy#U zV+l5X&PEvSK0Mir+Ks&K9wh-(X7T&-on2SuOkCx^SYDx3DIGp_h0gyvqVF%=z@QBo z^XE-+&K`N2@PbkQn*9+l-94|3-W{n_I>VK8X_XOpLa0bjRG!XNUUr-90-vx&SkZAL zC%B7vU_r(Y8KGGoo0+2s%0cPeLC5CB%Tayg!_pCQa!>5lCrL^!PXU?PTWY!wzD1QSYM>8jEJ zM`0Hc?z?|xkcFwR;TADd=cqL%aYC5$eGgn>vEUjb{e zOR<1yH=S=MaviYc(m`xjrqRMlD4hw|SWQ5h1Izca+KWXWtjXe&A)QyQC9qH<6iGHS z9;pdM0-txZ;`4#cR4x#m?g3s_a;CD?X)w3R@`1{}@_|{xtfiZu!hJz^S~yaD20nx= z`TBOI?J+b2i*H5sa;BvJpRN-Q!D+pv{RIo#;nd*`mS&pQVmz~o-D3K3c1@r*S|+2- zb4=z_-DIFQ&2o}NbbS2M}B^TZ)guEL;Au9V^T1Nk}+N*V2n$>Hn#6UWn zn}wh6%AD0Z^&C8@aH>T-FPrYj-Kkxyc9ehX3$7Y$VV?$Wg_{!ngO9^UuKI9mGqImO zSc!_ZK3-I`-3;0D2);Qv=Jj3YJyoZ+vgi7yT?0%~>JL}E=1_!&d~*6cM>Z2)X|a)C zvJ;Isbl~`z_qA7&3Ok-f>VNdt|3f_mz*slL&|)h&N+z&7l3D0yxt43n7a}?y(&znm z-7jY=Qu_S)&5+nh>}G2>1l5*^|4@)&#UeAaxok2TYU>RAnd^N6POwJVL*3HpbeCe| z1o509V`fm>J7BqXIueZWfkG8fX#M-FC`}lsioWChP-nDQeb{RkG`k!V@%si$H>W#H zZPidU{)kZEK>{;sw};BFpJ0DtN`e+N_RFPPY9rhWN3>-;>ueev`(+6EOYkMnQ8dGN zH%arhP*PsCfQiX!%~rRt;JZfvY*g{gqG@q_IK4iQYJL8b(GpCAoEu zIrz4$DkhtF7g6Nc-#2E(v=sm{9)MQHWstpBPDA^U^F^B)u^tmA{yALuY#*ZAY%QLG zUMjgfa;1{POVuJ7jy7SvIf7oq(oY6Wq4cX53(Z%YRw;;84m}tVk@rmhf^))X*`yM_ zhv^Xro~(cGHfkV0uGX?4Z+eruoh$`^eg#rskBpMK7$(0qs`+-I%q8)V+ z%31V2gWpgG-gn-#14Xj7Ffhi4b%Vvd@l~p#aohfg&gD{*1u^OUXKD}JUJj+<>?5B) z`QcE;qxC;&GPKwWvIDsPp=l=q#O#_scB9<@?*B_0t9lb%-?A^-f|E(nv0rrrO*9a> zp#a~Y#*_orj;^-Sk(!0PuP)si>i?P}9kfBTd@WP^B5Dx8S6Iz+N`K^DRc)1VjZ{4D zR`lz1e^Ovx*K8V3)RJQu5bJMc67VTXoH$-|a{+!rz`Iw`EjM?0S!e8M168iPzYoO`XxU}d)V^yaMU*JTE1SQ zdHoXPvQ903nhr5RaHLdGgI6T%_+Ot=gB=x>Xfa#1xGPq{wB0~D z_;V?TO#jY2+cC^h27VB8zXm+Jj1{s@ciTr5Z6+QsZk3~i=BMc3cc9t&57CK^>hEaj zRQ$AD!u__e>3P{t( zt6h&wOoVn+>UZ6iVe`9uIhXQ3V8%m4&I5YY7|O|&-_w=BEjS|54o2UMt|N-E^f#?@ z44)|&-i~j8KK8YpWgh8i5@5e8rKk4XwA5XS2KzkP_F0v`pzywS8B4q$h<;Oajd}5J z9eEO^L+Chzu0RMgH;Xd6jUN{XTtmG?VUm~=XB75sMm_s>63EZmNWK75vih%OisO5_@cC~BO-H53U$EPVj*QsYe|3- zU0=(AjJwdf2OeF*SbV6p@Y%#0H_(ay3+|{he`HkkJ>@X1d-;kw($zP=br;s~cs{8F zr?Q;G@@;$10o`C)Hazv+lIeyI`Izls3>alOu6U3!KbQE*S>U&sOzs6kO1cUKm6|G_ zHV~VG+)5YMqK@l@J^N;~C01 zagB9M;EKy83*Ek8^>{XbROHZ18K8{qH^{rY3`7h)p;3N{6{^AZk|HwyaissBJO&Zp zU!C$N;>=;{QkqrhwX_LM=ZCHEpU?IHYUB#my1)bqxSP5*_E<@R4ohxq3P+0~My_8@ zora^6yr%0Bsa1;CI0xaK!&jaFK9PBc^D)_Z#3EtjM(}X634*M==Jkdto@hRGMj+uc=kuN>d#e{RAWB4< zSmM(=8`1oX;osXu(7pzQRGike5{gw*JNVx_=efK8B(#-y&%!&k9MAi&^iJcFrB-y) z_mkuD9P)qudegMM@U6E=QVrTl(^6Xy%Hq-SswedKv(K;)SSJ2htFhU7^|8X})-E37 zoiS1G{fi%(=4>`LGKRY95u&LY&kwd$n?F;yv4B7F8Y{&ZAs?MrzVutDFs zn{KG8)p-fJxc`TLO`=QUv_vSI*{3l`74HTQ!=vn;cocz;~j`8coB$jD> z6^4pe*B8a4;Zi;)Wl)WQC}5Bze6PNv(?8r@I;bSw+=`BIgwiE&EXW1MOqSel8a@_M zzn#`2o93AABE4i&dXf3MN_7>L@bS)|=y`|=-)i7J0r^M?;8S#@$&3*ze6l^IYa-I7 zwI>`(JH%GUZ#@n&^>6Y~+*ahoJH|bir^`f$WOCp$n`UOh=9D`9ct2*QLHQ9i5WAKB zivSnyT_O~+A8F^3ZyLoy0{Wm0QI(!Tc5mvle<%*xK;}tm5X#T~s`7e1q<4Oj=V_|B@ zehWLw?oFMzu1CU&$q`!I;Qj?*Rn3*edy2A%mRUh4MT)@(AgqXW zDWn#=AU8-cDRCQ{^Q6SF_!Ib?@^C(E**_Lh%tB6hVC{#i%;?9}bPv9sG#cWBV#0u7 z4ikh-i(LY7XL90clTpP)HV`|J#KA@$EH|+}T@|@%R4J0-b)I9rs}Ui?5jf*8T>WW0 zI&jhWG1Z8ewB8&uBAG8KOZQ72 zgQleUn(6%1o{c2(XM#^TI=gDAt0(^S+iuyp;K))xU-l zIau`TZYbBZe82GjWY@okRCx+_rtE*i&G?sTWZ;{cbx!D?Msu*J%h(3bfemx8gFs`L z4)RfNhS^EftuF6ZZZn2&_SnqZzkyicmM5aA;^Dtk)kbr<>L zJ8?#@&nJ|I;fOy~;F!6m7XtE8!*BMEa)UF27prGz;SbTLg4UUq!=SF}wAlHDEuK?6cc8T_O>A79uRRAkbY=L# zu#wQJHE4X+c>$By^xKj+EZ~jJ+)1dOuXV<$&5+VVZ%xVWQaD5MrQ_&OU+b`T=G=SU zuajj1c|!De@&H?Rb6TLrZcCDJNO^f$SyN$aYHM)FuJ=FpEHl%;OTnQDWO2!T zB`y3?DI`QDqC=6a_XzQOK5z3X+I}y&yyiv^Z>u9Qn-!aU8NcAKS+L3Vu;{sU6(BaF zIzpNaK?W3GBSw5q!$f_-jSH!I~!;!Ew)nHQ3fUpB)CBeOLJ$_M-EnJ^f8) zaFo5`iGL->p4o!UztnhQCRpMwM*V^bA{fXIX_FGr>JX>)$xlU!B0#JHEn@C8^l+Ew z;;ScfULj~i@o%LdNSV;#S(s8%D!uR9i)#q`vPJE{!A$u2J7ZP^2s;$~xn;FDwOxs? zO42>w>G3Q(+sEs>hy23buxUcBxj`=BWO~sm*9rI70zs(p?PIe*(IL>g zxR0?qAzzx7S>eE*BG2zbA8Jln?40xCKQVbLl@YrWquX6wju~zdqu>P$I{pfIOrM+D zU4D94Zn>xUB-!IdBB7KC^88*JXSXwrpI&R52lI?1_+(0DTVK&k#o|)fXzL9zWSHak z_I!s82UOGVZ-)&waX*{Jp~uelppJ+lxikbw?a?)cUb$0F(jTTuGx?+!-}a+)fb`GD z6c4ao3Vd#gKB_Y32hQ;bK%kRh zB4E$Yo%9@?y2>eMM|21sPX)oM~qFUPkPC+N7U@T!U<}d zD2h)9W)9M1nK^EJ4A!k{HCt=qsj3K9;uzHV0YlqJTqjir8APU$^1fM82A*{JD{c+6 zXtf*at_A4NL?H5Zc4aFvD>eCyddCB?H(+;gS@8M3Ft;RHy)PGgGxnAiOu4#h=0(5O z9+*Ui3ed=Vspg_>T#A7&hE@n)rHL2^jp!2Z&ShP|u*9DU=I4AxZuAsS0ER*&++pK5 z#v@{vLW{26C9n9eWjq`l-jo1dm9F?skGH^7g(1lHH-Qq zfH)jWDb-)YN_JR^GI$Y^y&H*AZ*GHiVY&+FaR`C=w-rWOj!vJKuANy`eFQ=MVgb~l zOV>TTVfiPi?T)=9?p5LXOmR#x5t$FwqJbEA9V$i5o!RTbgPnzrmkHKI;n^ooMv+>A z{}JQJ@Y*#@Z}9NDNoJi&zh93h3M(oX0{!A4t*$AE5KPu$d#k)ba_A5OTp4|c6UV3P zCb{aZ9y``3UIrMFcH2TvIi(dVZGraUzp1%^5@&pL)9NGFA_8SfxSr4yCa$6P$(qWH$A@ z+n`tR47=}r@2cjxzJc+$g2A-cWanC<+dW@J9VyKq`Q$gj;evq@3$j##wq6h7w$@#g z*?%r6S9DwO%yR;C3A=Z=r7LgooF{--?JbCsEAW}uyiYn(jPWYEPw5rvapN*^^5?g6 z*X!WpJbnU<{zqWoExTeozLdexYXMS3Z8us`fZFO_j~Ak&vymH)_d#5;9BZwnx9 z8dmi{Znh)vKQky@&o&d4oM5v6Eu-tYR!&AIz0rkm-Jt`rj^vmL!SexzdH*Pl!iw!MN_v2>&2PD9 zOsBD>Mxk<|RzC%%sFc2i*FnB))cbhraf)XI+HUU_#*Iw@K#R9@tc+K1rRf7P&-KlS zpFzbJ{Zkoufg3&qLG~2FvOS-dFmrAv{2YSX-_ke1{fbLMoM1D>H0p!`&7^5?GBg>` z=*_8z%pTd0<2lb0sj7mjwx*Xo8WwF zCdWZ7p0El#pB)85_*dVok!3}Uy!3L#q2K7?Cz3hW=!O%^U1*ACJ18dpvpGhT3Td#M zBng)>QwY0A(DQKjWMa`*uKYqME;jbCssBUynv}RnNrV+g-u<-wL(&;Cu&W-?4?y#W z;y(IU4r-q=EmXx%WrI#_WQ7_sB2sV}YD*&?ER!U{GpACsW5%Ru;qUg~%5sbaY=67F zc#~1Z)XBycMQ;*%_#TU-D10Sq-{CP#*$c%5izU9xO%7b zjU{x#JMutL&0tCU@VFa)) z94KR0YR9GwjH5_c;l;=sjX+do2Hd~z!i?x-C;X?K;#jMpy&G=C)=fEJQ zXb)1A&kx}o^U_r}Fl8k_{G$;MK3`@_T^s)w-AEM~vit_`6~&UFhmS`c;_dogOvcZ$ z7dxN?W_mdv20wIRzkBJNg2LYP($D~=xKDsIpb8cjxEX3!bJ}U#uYSP$HAZ}iArNcM z;=EA?`XmZ?RCeb;PPzEnQo82)w|9X}JQ;JkH#Z#W>zuwc0_MSI%2v-;N7FtKbCmR; zNbPL05sX=v$0>5^@4|^7wV3kFglg-nMTnGJhb+Jl4$ZK6U)sXtW_;}o3i*giLq1Pk zt3~mFe+iKRXpq_O%w3^H)T{^lH>PK*j6SVAh{$CWNjw!)K?u?5?V~HmQ&5z`*#S`V z%190QNb2)=7K;fpH0DYZ#6)2&12Mdg++Y2CWW)v#52yD?zHt=9%04|5^k}dND8Z24 z8~%MuavtOgj$c1LVft40mrJVhSHTDy3n?j?W*|5IbA}6jZtP^8gClR!4-AwVBBMG! zS1-3gxce3pz4=ggRGFg~LP?coF_c5wn#a$A&>E*8MjnUdv_9q0;?wm6m781cc z+!~onFGdwdsY#9VRo+mK3PiXX7daTK$f^#)!#r{O$=VJ~J=a@kLjoO5z?Go|p`HNP zReuKz)ZGG<`BdoJU{n7M0;FXMZzk%+IF>jWaM+nMjE>8{j%!xZAEvI&x8hS2!DfFY zf?898WI{Y_xlvegy|Q=!>6Yv9L?Dy9y$*>snrjN=v@p9B`LZf_SvxqK#7ef{ zi8aOI{53nsgx>xL%&N!+RA+~q1SG2@uCSSt*KF;wr5EiGvA_$h78G&Z-vF9LGy5Z5 z1hPPS7l$D+(jPCvy#EaU4~rozB*nRoUVab}=<9#Hvv>^QWij(zdH^ z*eqnqf&xO zEAV)J@$lRBdmMSX0ZyMK%PKQswA{Usr~8ro!W{&;^Zjt9H0FQA=l|Zu(Vi=wBl{xj z6Ic=Oc$!SM?5nb29%zA;U@XPmyo-W>>S3T=Pwj6qPvY2ba7w5tW1^WA_GeBTKzp&W-B&uuR8UH!Ck3?pga^ zka@0bdzp9X7a0{&^M$;coEY$I>4r?X3{8vni%I$#25pnmd2)r6C1PKqS2y2d94&F{ zLx*#k-}dyq(Ju1^n1(wb;jN&-_yY^@fng$N$UgDa02yD(SZTkG8OJ6Co?qzLZv-^b z)3+4)GD_Htra66tHb*Quc zZ)=luBinZyx-_EQXXjK_0_Lg4DPnT@;kjx+Nr4j6nWpfJ2*H_{`&3xFVdlFv#%X*p zLh2@rgfyM8X5!doj2wJj=xHE*Pb4;nSS>p?6`2a z07q7(l@y)eE_v3XNR6MNP&N?lG0>FV3CuJq=lMcR*s#fl2o~_~g2R)(qcKlw_?Hea z1$SuExtimp^jF;XSIoL3Hulvo!se!ED(hH7yZxsvao`_0Q(EiWgz7oaJ`ayX!u=6q zhz#0Xbbh5Aa64t_z2TR2@Rqk}sLZ#))b46+1KaVew#biK;f!&gN%^sQ>)pVa^DFG0 z0`4+R&6+3N!OJTCc_E{wOWu$})GkYFb35auY-rSGq4R;jPQcTji$9Aune}a#^ln=$ z%8!>tHocIG2_%TU$gfMgsP^B2HS-Ok;1kW*CsbKyDw~?8IvEctX&-oPDy?*VZ06)7 zQX4c8Ud);nBDBx9H;xue`CYtp>z@G(BDVX42MQZ%o|AT1Ml>Jt*|CdS)oBP8O^?PE zqT*DrFE31Hi<$CKcjyUy=I7A2?PMJcw+0(^WZt0$1HSdquS3@g!CHrZTUsdz7IOy{ zsQz9P1g9g7lFNk$2g>M8dT5TZsR&97m|5{mo`lH+Eoac)?ZB0iGV*R)s z2|?LD$EYSYC?6AhJS7`j^l|CK_&>D-SR>@X>$W^hBJ%2OniWm`)nxjeOo|xwn5nc@ zNn&|vfmL`AH>0A<^qf{-TrBRVvdn-!cD-qODks#}X^v1EN*KQoVUEG&n{p@U5`j^s zT}l86(alj1mO+M3LlJ1{8YVxUiV*;!6i=cwVRdk8AqBD;+UB%oIT4PuKvW~y^yuv<FVzXr@gwx17*zX6)zfk^tdc)EDh-%# z19Nvz#pg`rw&F-G72y-B%7ivY+cuAWFSCU|7gtxx_g26!E1u zU5I+tX+!-(%rW^1;M<+RnqBJYAAe$os9X$z0}mFUy)%Y%L4lnwqCI}oITwLj?9iPc zw?y+br+8Rf?>Bv4D*C_gL>&J}OoF+blfg}6<`I+k1%AiEchnexNa`en6&NzA6KAod ziA7#oa&}Pccz>%%eE3n%2$N^xgP66HJk=VpV2m*uwg}GZA8c{eVau@15Qu?ah=W+` zUZIxX@|rSPqh16$hz$j%O}|neUCw_)$oV+*?T|Y>ady2Ril)wA<-$IW+SqeVg(=&_ zZ$zw^qh$uin1}9mz9 zuYQ`L1n|c9r-Osk`={?R*_7@K!yjY6ONlK79F$i5Qx?j^+B1s_kurGTNtuP-121j1 zZR$Tm<|Ye_89O<~#3to`*%GVT1Vg>gy@L54uh(I_bN+zLsctv}v(XG-U?41CRQ$WP zq`qnd(^9|+_@p5(;P;juE|*9!!U-p#|4sd_3Mt zL}$a0EZ%Do1W{jD3#Yz#h3$H=-zJQmk&+p2Xn3_e z2?jp`U!o%mOL>V0Ls7oo^u6wx_Wm4&vbkt}>GyyhuGD&O-;TDz#-PHRMqzG~Tbd(( ztI(a0#?OP^SU}Cx~Zq|AI+I;g~6<#LUtw zTp&4_{`{bj5Euk25-r}{CEw~EbkspLLXj@2U+Fv+ZZO%q3Y&z@av7KZ3vybJ$u(R7 zA;W}Zf=M-?zSjBt(!@UeWenl`hmHb+zWon}CFG^QSwO^Q)vHK3H7B}{sN(+ zZ^-N3lI#p)AF$6HaQyFo6IwvAX)*60TxgCPv9h~rCK+N#%hWP&2*cSZRSZ^TOI(3V ztQ-1;skeuQ9`nh(V*jC@4jH@5rJ{<>SaDoE-pgLYEUo7EK-Q(T;u$4$P0>kX!q1~? z|MfssBi&Biq=K2oR7u*T1;2N&;e}tk_qsT}cd-aN&xPYrYpwim+a}geMK5hv7IjC= z8iRc-1YB7R(uS^O9WS$e5(RJXsrh-Yx$=3r*~{{L-xA@rsdT4Vqr&yJ@1>-~vi?0* z_~ejmJDyb_LTl1)QYFYo=yrv&f$^%N*vS)=rK(=i@O9PR0K||n&&$5`-rw|rD*Ld3 z837eqBn&p2pb@bs+0y(df&QgxMuU?uAHl5@({j%tn=JoFm+k+)ng5I9Y=5L&xltgr zrT{kl7oA1KTe82b(eB57=cJr9%qG0IDLMR8QC@`oD|C1?Z1&wu(6Fh#V$g8t_w|?R zo^Vnzgx)pZ=z@|cC?gtwm@N<(PEGwS6)*g)CCm#1mF2yeY>%Z2TgEbWl#I3;DN1jP zCg0l~g?y)OTHkk(9ZAsBOZuJ5Gc*{XGeZj9aHY+M6nCTD6P|j9ep7LyRT{?C6YNYd zt(n}7RT0x;X>wq6 zYI?=1sAl(sTm|VP1_3NSc}{lS?r=I7^9RL+)SCzXix~CPq;UM~kHmr*83s!v(Ggpo zB;oU1j7_@w-uQXKH5Ei=cQjGB5Wo7!YCCX#3OD$Yc_S9xQ~dkkXb(!zXz6qc1Rjw1*NtY_#7)vs{s$~e;sk?#Lq^!N!QHc zoXlV$RSWD7Y{OsMF)-r=*pr>jv)wFii2yhT>v#d$`_OpEG_6DEk(^5y0pvUz9KQrx z76m%9BNHJcr0mHgrV#E0c(q{0YHlAGq9`b30O8zO()9}U(s*&c(D8kcs6%w zbh2R|vqn$vr8j}WlIQ?o0GxCl>vz_EKUstKvgWS@32VQ8-PtFhcnCMq!!^I!wr%Zf z+@uNZ-6?v0u#ku#!Yt)?*pdR3zQVJj0{ zT>5sk)Y6=xp0}5Wifxmu^kdqc_TC7?m2zs={X*8viz4(49b2dOE5ECgzvGL2w=7hy z5{`ATGAbX%oGTG+{H7kL!u{>Y0u3szs4Y1Nj><=zi#esYN&q%t0qNzoz6IkuVeH@M zVr%lIW={#q*31IVVJa{kyxMj`)CNiw34IYVEp7A)Q;vo5$ump9wd#tXO?WOWwV?Yo z@8z4Uw;S&xTV5AaYtmYm&s8_G&CMhV8|)a14I`*Kn+!e$_it1R3g7VBSV(#`4cT-p z1$8psrcg7Ac!!-1hgUG86v4-_k^MyMTM)gncdEZ~{_9FDk^tmOxP* zeL}r=Yl5zb+D6)`Ijz2NI`?R^Kzu4SRv<(2NM2UR3@IYy4>aIn32{f9_zTga#n1DR zqD$VkHzB0i_jR^A(#t;m2SCyJ?~2ZVuo2(PRr`rkA6Hm4NQ+HDn+oD>7J;h9` zw&2$WbpQD}fpTeCnoc_K%Q8q2fKB=jx~cO&y-yFQS396|MH)%8PnZgif=LC(D#MhJ z43%`Us(BgGY~8f^*4n^I=J8bQR|CJk4}I56q9sgD)c-|yu~1%^n7NzS2)qSnaE{!3 zVAy3wd7LahC%keX%WwWq($67gpqkh5)!P`W&3~my!wL{_yZQpCP`J@`Q+anoBvfzjYw)(vkJ#~p@? zwenMTEHuc8yGU>#>5nE}vO9D6ewLxQ@)Pq>S~iz^1=G<+RFJ=a=C4uUwFlL?%=ng9 z<b>-|`Z=Cga>7$ifS1`JCRa^W7X`kTaZOw`c71dEnWI=tKsKZ7 zi@*Sl_sw6-8=Nk}G~cy@2NZLW7)J2~3Y600T;BesRyy6ve0+M!I=Z?leR@(-8|N8) zKVGdl@o@OGWaBeklhrbPraMw&*Lq{ILbkg~{^qc_xELQFpX4n9Q=sPm;p-iPD~-CY z-PpE`9otSiwr$($*tTukw(WFmr#ntM>F7(I=Y8v(uj1*=cQY!3s1(8LxxR{L(HpfleL!hR*JbqbhXFVY?A5Eu!v8a_`M` zw@|5?B`+OtEd&GBsG;P*e~IO+SZ*EzQI1*20QN~7Nfzdh>C5CsPDmy>nk$?@j1UIV z035C+6L3Jpj*c6!{#ac+$jaqUOCv4gYxlVoIq|{y!Ccx_~?AO-7f~3C;2Fa zSdc?G6z~unA#Bo0s1X5xrh2$Cmvgeoqh-ORx|Y>h224HcR~*K}N(&jV4_u-Kq=`am zX3_6uLPxYf)I~2>(Y(njgER3rLyoqC)`Aj2>3~#_Pen9~fT~Y`RNHYgng!LoQvQFR z(}8Qj{J$Q23c}<^z0HU5KQzSoOk0{QWal~zLVt;YH@|&?bvhj=@4sa^wjaKsK)XIo zO--c|BQ}Bq=O%=F;TG7QKfi=VpP?Mxz~l_|*y0XrR83AAb2=zxg9Z8}QgaoAnY6VOotwE#^r!QeqRtWbE)i6>G zP=YYt6-F2nyZ|eNfs0iTQ~HxgN#%+n7@>ie`lx6*U`+*BfJpTxFmerk0B)YH(4ANW z+E{r6I0X7Q+ZY7|+BnA>U)`=CRI;}F{2Y<%sm30uEQQj!^$-{Kt0jt{cqKV80xAS{c=Im)8k3r7oEeVasIr-KM-6OQ_B0*HKBpTf;UFXJ258{9ivJ`JY!K(Uq&z>+~9&V9tFeZ&@VKWaQ;*^o~m+;;F+(C-i24$cJi) zs>Bhg{qLc@-UYA(5$R@9DG3ATCcm zY!9_-D^0Wi_Z9CYDCC3J5Sw|FbOi z45|EM@0RjDSQoM}T+Pf*IxlOr+wAb=zcNz(y?=aH@iM+Ymfxr3y7q&}Z@V2Xhusl+ z5&*@?^(;XdQou%2oFHHl2@psIWU11RF6C&UqW@)<`@Vn%*6KQp<*7Fkqq5JQ7%R$W<3Hf)foNOoTyxv=Ni&G zGbbl&o1={t@660Cwc%QoCLS(dDH!QNOi4oTYJBW>c4fajY2VO-f-e@DDRL^YBd1K< zK_Cg#3#{Pr6kw?aj7d9^iFhcfKp(Xrh>{mX?^9;d5s0S(eE%7Rw%ajBcxlTyIPQ+a zaGb`f%j2{yb0ZPCF3$MOC`P!%1+FSTQc%it#0@HFOPE1EHq^;U89ZJ%5o^Yd9@y!$X13b zjU0CcC1reyxV%$}WUA);LYffSCH*{QRWBxaDYjZ&Rf4M`L;;rt*diz_844Fl2Gm;< z5xe5j=KpL2;%Ojs4v%MxZ8JUppZmgpSA@4Zz@Pi^tLf#glz+(GC+w!^e=2FF+E~_eWlvgYN9xuN;a=uH= zxj{orri!;zI&BXLnS(BwSfiSaYAqOVdnNy%!W&S^cH#U`ogrxNL~(AJP>@;v#VG%* zPQWCq9GinQBb_LUAo}E6=<+au;nWDpdnH3DScD2Sn$ZQTfAS%Y#A$rCc%dV-JlE8CK~5^XHfQ0C;iWr93LO=gw0&kBO=PW7xy7#tr!v#SjL8p^4O?_`FL1# zlD@mHReQ8|1OJg=u{vz|zpD;`DNH`Z@ftOZp&Ja8%iRQ}bKFKUO{Q`g?=hu)2e&kZ z-TV&yoY{i!-4eOyQ3^RtB2sF3S>Yk9~h&%1{(z3 zN$kgB_qW~-p8oJW{n3XDI@-9j>(4)(fu39g-P#s)j|Knf?W}9TXHGV z8fTSILK{ArL3)|IZd!j&(BFaQjQ^Z-!KXReEM5Fd3FK~Rs z2`$Hc9;{lWmg6zl*+SV623!{KzyYB@FBy*h2)11c35XxzT&<)Phs0q(nfxs*oDhKqlYeofRBvc7fyr1$C zu(Lth8vZC24_#f-piu!zLxdVx&@2W=2c<1+B<5NVkjJu`$SZ`)pC*amK*`dnfO@IW zi&P9)SA-7~k`$3*T0t!g0WJ`$bi;I!Kh!VS?vM{AF#dEd6 zO${pmww?&PD%A7U5?-F=#^BEVy?A?N@cpMEz;|4nakz?QGzKZWhqJubB%}@ZONr)x zw-E8D@9JWq*gDhp|5sfwsr>2hCcdn)Uo=kBegc)G?`HAUy6?%_JkN)HL*6Rn4z1GW#7R!Ri)kMDW334u}lCbRT0JcY1xPs zUyKC^Q&V#y1#bwsy$t%2IF$3X&GgvU7`%|P900Fol ze61p>GngW=L{O#{=UE!>SWpI9O_R0gt^rjf*I4P~v;#O?QRirF@$s&4&oE|04R6QY z4+G*ccFqU9(dPqJAKp<_a@&&H;BP>Imm-R*S^g2|S9P==qSZ)&Z#=?nS~XEIxtQ5}n- zI?qngcEQg$RJk@>%yEa--%4!e^~0q}K5g=9j74SYc%Q+J|DV4{n9s96r}*=Iy(a$C zKC;=fzr}yb5Mlfclpg_m)et}LARHmmpZ|OfibAkIH)J*FTP&o8iq(ejDxmcxni}H78x<`d)?2*rjuf>(tDh`=$O9btB4f z3j~8l%-P;(YK|7ZJad^=I6SwGp$?PQXs+~#q7scXB?OYrkjrrl_!HkGuz<9J@8msX zg2x=alG5Y^B#r(H%Ki_l%o24lUZ-bYMLf>fTo=)G2Yxn&m)+H#jeD-z`kLY@?$bSs zND}i_D`5zBm&tcXbF0nB9Vd3?1khgb!g$%7WG0Bwd24nCV0YQA8uTFb+B;v@pIdSAqd6 zJIIMzR6uPH2Pf37KfB8(o+VgogrMj6Q!tiW@S$qqRKpGLz-9*zQTV?2s`EFhHCu!u8BphUA0ihTljFE;Tb#xV*z=%B}RIOa+s;4O% zazYC53RgJLp-Jr>H;r(Qg$!`jTK-b4K2^pVc=gs1J0M?H&V2}V3~?)8Lud%Zo@erw z+&s*wyj9y=(WUq&HKYJ>tRux{!PMHAnz0OVE84x$mizEYQ(YmdDfwe6voa-Ea+H`J z6WUr6?WY7i^L7~q^T(QSO&TY3VkOKw4s?Zx^0Fsdu`jA(ElGH&m^Qn`lGO`fQNCrF zU`PUvTZ{*X#NuqjE`2kQ7oLUB~SWpT${so(GC=Md_&Q zU@Iv$j|65YoTx`#(+ru3E%0}^Yq~dA#8ei|PcBYKCS4W1=;T(+BQbm|nAL&<3@0_s zBQxaqmETjC$NL&ZR5i^oPNw;n(^2{H)$?00C#@!SG=06Pqw*GSGV~Q{^vm8@?W9o} z**{41zUGN>B5v3-BcCp0>y<3RkEF5HGKCi=sDZjVs5+z6l8Iwdnd2NN_toWhP`1T- z6_Uek5{E8_PXu=scNa{Le+#4NLD~~@nW5aScYM}O({x=|R8O;=5n}vxb(FKu+#GG5 zs>ZsHJDe|1)3-$V2diSj8$dG2=(UX%7M6QlkMEM+aw>m`65V~H#MSw74|Q)6rRhBd z5yUlO*tMhszIxC@;D>nf{_7+QJ72bs-!Z7`n4WJTZg^LY5IUW=Kn@jf6HLaeyr7iP>8Yo6c zIfONF*-y=sH%TB1@uBc&!&4p=#~b|7)tp{minIRPc`&;ji?GE+EW1`g94BE%sswIE z=jDn5NEERz07@(5Rt5=}|V zG1AtQV|{E<&MfFz)XbL7^1mNF%vdBhK5W0Mb~vD(w|dJV z3y!QTe3`)uC36#|x#wR^NLaQGBf)}SC)CxghEIhl1}whza>I>r&u}BQE-#ga>%ned zN!Hv!bGTv@cMVelK9nHIlTl+-eLU&Y@t6&1F@5Q%su8h>QdyJbq6(0cQ5*{1-`im$ z*AN0VE09#5O6s|mWHc#$O`_W=HJ{p^ZWG54vE>ux$jCwoApnFJNol(erTvye@GnE4h;@=zgwbSj}oc8e}>Ql+#B%M_wqxuPF|jUk4Sa;l0uM zDm)6Jj)7y30riJ35%H*y>eKwP9Dk&V48~h4SglObSV%o|YG&r2(UJ{a8e#Y6d%(mO ziNoggFUn<>UR)Lj_QyUa!yAHi2$`ozywV!e;7KYmu7jgRryCt5Q|_qs_m>BqAjljEm)ZRS-#VlRHTfo zs`N^H&J_9P1l2Z{xGH8bD1yOc<%aY5M$wCh$e%BX`71E|2Y_P1x9tfDO<{J zNF;MP@?r2#iy657&@2ayX9dD$ridDgz{X6lH8^-JqblUezCS=%!Zjiimn^D7FI2jh zv>$tNnriL3+HSZE#kKG7yvOkAf2*JTMTT6eqN=gH4TzP(1H$lCCQE-8*!zT#r+C~A~ua}8VCd1d96TB{h%;Lwy!cyun{rTqI?*1#cZoB8Nb!J|_<%;8_ zTAuT&r6uLh)nkG2hnKB;vxgb@9$Li3;yFlu3iVQ;%zup-OW80@%+aQG|Eu z%e7UZOibh;E(A*`_A(=^8|5f`x^;wyF`?bh36ULwSW8=f*@I@k|MbTvRE&BY1ow|N zG;9Bz$_G*PdmHnX37l1AJ#NCvDi@C!cA|5SY(1CICCD(P$A^xNTtoLGaFSIfpp*eCb1ZN_otGQb) zNf2N2zKMHrQtZM<1bK^k)b=)NF?`&HQv7;4Q~D3d#hQ&#byKR;_HyrGDEG?{;>85N z(6YA{c0=p|kp+witN^7%+&&U{!(DRC2O(tH0y`BsO2fawr@-*?rS+2VOx#Bs%@ZA} zz!98wDD^WK+5kl;DI4Y_oV1k9_83Q1i7MF4!4^jfg;j#Bk6-++cr{Xbam zMUtGVm>{SSjqrmn>;pkF+ZRZ(*_nom5oAa0H7-61BIz$278fEl9#Xz4$JFSMi4uh) zQ=-+H5)RQ~o+w=-HS^}0*J*-mSm2{$IdGFsu#1O3|xWbK-WOY`;EAE z1qB&4{2Z%$op`HF^<+BWf_WpnY6I3r6*}{uVDwqy`f7biY%xXo2HO)7kR`M z>jDSeaxaKa_wOiBH90xOFdLc}bw0KqEeXI_NCv-bZ3M)LopBOFfsqj;w0=qc09t&5 zx&qe;O=RK%D^xn97zyKQ%!@)J(?1YSu_xOhozA>5T>cgo+?I{COj3c%778lFIBiTW zTtN+=1fJHx4nl@bk3MB)$2!TAQ@)UY1ch9?6=92A{YE73Auc=zKqhl*9}kd+!=Kp) zp#1oumK39GAwnzVUM*mm9GbEc2#}B_R;AdD|5S21C&w^uP9=n35Eh<5H2!J3I?QMo zTUab95(f{J+XP3_P56DCy}#n)%+Gx9^fffvA6K>ezJN|&&ci$+3gq&|^Rm2;FEC6Q z!YWDZ;c1al276K}NL)OKcWK5{d<5`^aR;sE!Jvjeks9>BU>UMVvV8u9e>E!M%ssAM zaI`oF(s9q;L-%&vvLx2l+|NMp4thw35yM@6b?$Ss>+i5p?z_&r zrP}PfF7Go5dH+gwp}U0EUG7x3u}+=5d6JwW>`l|f_!z9~#h$WMytmo`)8cGFL#eTU!S`3+`%yYeo7x&LDB`-(o6d_r}(sWQH&OK_QpqSTs3MBLfN|>qE;ZH##jDV?r)JvnAmo;*4keChjb)WeTs1 zx+Rw?a@gGWJ&oi{1$+%L#+2-vT`j+RP(vyQ1M?VJRf>CbY4>;~sU+I6zkBd3iH`MM zB8!Y&3p!x}kVdjbZU5QQJse;dM{Q`z)L`~dF&1(iDP@zjuj|5&HM%T-=>AjV-7{oH9yN@0BT&b2gc=Nf2d$7@uWFok4TNj2@wbyu zhG)h~DH`@iU4A;FFY#hasylWAG(-ZyiHHFaM}rF|!YnU*2vO@un9AoU0@4vwB3;ww zl?a_0Om%JPgjud}U4Z9D-y+dkts137C@jr$ctJv{W|YR9OwY%x;5VVny@amHIqLm| zWux^F;iAY%pck;M@gY2fsoF86!$Ts|NK7KhRtP37)^{yzB_%DsB1Ehcyc2*Q^U)i6 zI*$%=<6O^83|E#qRta2Sm9YPoPlL`vufz=&>LByBnHJfrQ7FB0)AhZD3XL+CiA4CT zfs3-lgn$-nEleg|U;`UMI)cp&3juTzf9M+gHfeTp$4Hab`vrYfSsA$0aqj*{W?oI! zlq;gJc`ATj;SxLw2Ow&!5?^1!8Ergk8K0={rnOe5faux8`h3Nna)6W&~mu z8S_Bnyw@LR0-$fnu8-I~tX)gFK9dIQCh&S>aDH(>_O#`V$Ayq>8T1 zDOLf%-}g^gqxBsLi?J}$B#5FiNPZz0GJ6>HRwisiOwj0%bM%?Q2&gUVo7f~ykc^D8 zc#>%_nM*7LV<2${b^kcqa^$~_wIkq2Ox!jLjcWTdFss=4Awz^ASyJV29lr7VSoi=? za6%VI>?ZGVtOz0P924m7X#7c%SPh98bV?$66M9i_gH}Z$0kNE1Y(dQtY_3{?eZL#m z=U`-LwvYp3B@u2-XUPNXV3A8a9~A(pM)WW#U@m^XXASBp3jA@AhAoB^)s1rAHi206 zf_l@oCU}no=cmS@VT1+Bq=Tldv%*;5bONH5azeO&I0c8@M5;Lg)(Lbn1qk+uDph{E z>>ipSA(?ePe47;%o;U?jIvjxGf?BL-j^B<*RZ0c-M}?&0p;FG4+6&Bbp212(*Vx_h zcdp^n@Admw_ij*NVV>tJ2T`fMFWdNglE6#SH({B0@Zo`VaWfR_!N2Rbvt3Xufs-Tw z7yOxz3IEHG>(Ru_cT5!F`vT$d=t>N3r1Rh9{XkfHpkeNpqa@u0yCE?9i2HGxrIYVT zvr@lFoCY%XIQj`6nFuN&g`~$NS)|vTy9Qo0>yuGNPC`I^&y3f$wCx`!ZH{<#sY1_| zYj)6vcTg_=#IPL?6|XOEKU2%+`F4o{vSN^y9h&o> zMKEG}W1XP{`Wq3=%M%lChV~q6H8IJI3vE^P+_he9lh0+MO5p(2=mu6XzqB zi%x_0e~*@#Iv>^24BZwn8XR9P#k66=Qa(eWvduAQF75SaB1TywRLlxMxc{{K&=fpe z?of0kef3WRoAcS4&EL1gZ=GB0SyJ&aX7c}?#$%QFqv8GMV}41$aX5$tL%Ki9!n}+q zHCk4z!~(~voYetI3RoTMLPj+d+Dt`?@Uz-j)(C2=kDYQ7FOkl8V{VdjyY8Py`tPFkgear%`ou6QneY%JN|p7V42r4yd@-JT~;kMwC*PKM%E=P)k!^+U6-%Mpqi8!>15xs%8q?E;B=KH@6&HYlxF##&IJIN(fVd2O zgEag}`)%@x^?V_ooAM4r=^yn0Lu~OhV}@x z20pRcuo;c)GSi|sgubI>oOIYUsfdf{tZm5pTZh-hbSIZr-Oy2HA06SE)Zs&oc zbWsi{plpb+BrRtAxc#{s_?6ui0yq4Lwjqf(Pd+{1wo2(XoHLJ^6a&S{od&<~qs+@x zZ3Q^FhGo<}7aA)xm-|7kha5vT1`?^;G>69FvQnLn>?rtrht2f42=wfF%W#6lA;5OoV_G|DblIL z!XrQnk4TEMJ|3K4HdYWy#)DBG##%Jdj7qu+!ObFalG05^!@Q~?rpCFbDqO!jbYytO zuoZ4B)+*GGJGt{5^UFw3rEhXf%RCq_4yLMfht~}>F?RBE;cJuqj%xi*vi=g8@di}; z3-Utt9;|%F+V*{!90o`UK?6<^2ADOA2P*hEe-7B%IY^}6k_~#={1zSYkP{4$8(AZ0 z@Vr0y{>2#o@7Lbz*YIi7+FB#=Ix$;0I`Cc*7$}U!y0E}t&0_kF(nG*c~_Yap);}=)Gi-t~W0R@Vl-ID;__6u1e*`^xfYo?`OAf z@ZMm=9y`oyjCzRe5RFQ;IWhK797!;EDJrA++}^xKu+JHYI&1^48O6&7JPn479ToD^-$;0bLcikUfE#?9# zk=qMQxk-RM&75xW5;;N?S+dRZ-Jl^lgcyhbvn|RG!hT+L~4b^8XX0L znVN>GqT$s@6b1)K1021ic9jO$SWw8=3+fD;tP)6EZ9&rnS5)`dCNdqMNa-YBGwOo+ zI}E9A0ygdM^8;zbKGIEe^Nooq{7IaaNy{PPBwj}WDW*#k(Vaj0W(H$uRM8;k@L{pVy zF)Kb;6&g`%vv4Xx6v?76XqPIWU(}f!fuC9z7pp6k#b;1%Ni#6uw zEfxpNj6*1`6OF(Jq>r{WdOEOF>=I2_n7R=-z|Jm*iJs(&5i)unrL_QNk0kO?fd=gle03m55eZ7y3VL4P(~NqQ z{t~NAe$dQ}C?gYbF(Uo;+hIZ3a>PI6evX+6ZtzOw3|KPY%=JLSxAr9)1NVad&yM8$ zKEU{rRSgy!^rv`AK6C_M$#k|8&(VL6;E8RcZ%`hjI}OGb8NuG9e=%m8nw+TKrIq`0 zo)h1@;uWlkp&wEj(3b=ik|&_H7$KooQWIwYs-(TSF(u&4u#1RXid)37yX=&+$K>mg zalZ0h7(BlX!zt*c>YT^M-i#mt!j`97TU&GQTXVWTk1O3EcqrJ}`?S@TtLoxW7_}5f z=gh^l4$S<8R}ock=*~-^dbe*;`fmrvY5E^uG>c!~gqvqc1nNi1Wj@9xMjj|%Q#(0S(8h(X?=_X$toI6;kkP#rg6Xj< zp3uu6Jy7mT6p`~;bFq5$mc@PafokYl#3X zaB)BIbyx7)C=b!>ydmRI)L+#*=h%sN_ISPLYQ%cd5dXGAo*Q-=>m?GgQE%BkbbZz~ z%Im6Qe@b0q?ToBGZyN0J^8ctZ@VLOSnC$Q!<}1aLU~o;8{$%P4?8Df=f(IEk z3^0pN*;|roHXE}R>$+New!A?r8Feldm&CA>Q%T9Q=AsSL@G4pWe%TPD{Nxye&+_yj zMOZ`U=1`Msmsn!=t?BL=m>)8LAOpW7L`xSWUi1`e1am6K@+85Nn3Df8fy;wa;L{E; zj*?(N5Li@}@W&gi+sJ48EApXbPuDjuwjFHEo6d5}ad3BysPtU@j1OL*o#_%%7F0}Zp&0|E}vKj zdb=L%Fe(ro|4F!=3bqFU4bhNFJ65G{60lRwNTPT zTT>bJnW-h*EI^L9emhgq5D3};5h$9wa~2o| zFj{WCfY1xs zgcBo4MNdPLU*ZSq8lwq1h>{weI*O79Zm=n z+ic(4qj6-e@ozi-VJ>}o9N%eiw{~sF4qkx#nhesSXupu60LmmB^ti?7G+CE9&C5lC z!TBj2uRR-JNG)YDuRjz?-YyD~ASyCY8%ct3!gGg(P+lD1`Gtf<2Fr}cHT|W zd~2p{cYE>WNDwY0>Pddr(s$nsenY{H7|xgis#EO=xQKOR@dQWsIKJ$4u;2G@PHrn} zE!Mwc@p)=HPBIa__ds9&UY8lIB{2a@L;yHT& zhJisotSn@pK`$inau_iIy<^6Tz!;eh{7P+6 z9}>@LA!F_u0FfxgNpY|TID{$n7}|BALBKQ#$raLiET8E{&eKFZ1*3&I(WI?M21zUZ zxw4Fa)Iin^pVI?OH+$-5`p<35UwD0?*9%{m;b7@J=TS%N)tvU$vFU)2)O>)MR&?%d zxT0#huttiiv+M=Mz>Mefv7Wpb9%6OOVfw$t|0<=8GeG_T_)@*fYPA2OF_i{p-B}zZ zfXXzK+#D2;jeJ(W$bRvxTExxOmK=D3Tad&+$Pqu@ElHIsXS)wxWR{@QUzp|(sJN!Pp zemb9dzRtkEk7HJ%>AigPJug%CAyLw-4btqg%+?tv1u)dZKX0q+d0yZ+kYLm>pY0oj z4#Mufes+JN=Dm6M1H%b#P%L7jSS4J%e`M*s48zhVzMxXOJTff9etuJ3{RD3Q>Lo2Y z+zmv3G1;_Natv;y>)uAD7~X{##`FE*)Pflv_&*tb^J2LY}CGOX{?+)C1KezMV-X0$UpLYY>)n-?R=dQixc;q#@ zS|7yH2jN0uZxro5jYvX@lHT zcO&1@*pEO8-0go|!_8gg5kBr$!5lJrwt<4t-Wk^E_ui%H{mUlwz8KGv<}wM`5TnWO zKdtTkH@EYCY6=P?L36W2wT(CAmrVEYy6^XC&)~UZnuZfjb+&>0=Y($et!DQ_-v%~x zi3n@s#)k9a@sKvF%uV3_An$qN^9}@!B!%NNJ)Q;sYP;)UGOdsP=G8UH0MODB=nwc0;MO4(C}Wn3Cscu#rWo;e)IA_3E`N61bw`Hvm|3pN-)%kT z$2Xk(8~$CP7eGQ%eRq|g4evX;ixO5)(JLW0ynoHJ=X<_F*z!5uW{}c%^uF@>x!i4_7aKuA#i8P)bzgo8e^s|^xup~Xm2`m>+0YWq>D6mU z`=5u_a{}&sKbpt@4f%RE1BjpRVFGvOg-A{mbLBTU2cc5onp67k$Dh70hytJua5@Om zz|o^S%jR_-zufyyL4$;nD3pt(id;D7Z^NGdNiUvY>VL4zd99MiK>kjj@JD3AlG*m& z-0r*oM-!gq+-n%2WMj&n2yLGW&)st}=64{jcXxmZwo`u>HXnkVHLU#i_s(Z2N3ZLM ze?{>#&j_1jFnJwqxk9xxi!{D6rEO-buv0o#_lXrDvC-5j5)z;DBo~V0Bf2~)m zUvhR|@b{_&o`VEDHaB-hQJaGqsegTO?On0ha~wypEvU(RT9Wim6t?itLE_P0;#mfc zyWRe5IM4B=g)Jv>f8}?-Y#Drr!u#!G`tE=8?5pk<7Z<;oY~P%M+ExTD9%@x97E3Az z7eGGoAb0Y9sExCFa8GZ>RKR{4??{7vWfXQMK9^cat7fDwa*6w9jdK?J_FKiAew*{D zoR3j6_^9;zO1`4xvM9p8p?XZ0w{B=@_Sj*WXKua7f{M|Er$?d(E-_@#RSQ-eCdGc< z)U65(bjGfg{3ATt)(eYST*d*TEZDj=-{9fZv1~Jq75qeo&yXpW&*P_NjyMJwT9&GQ zYHJ+Yevy0q3Va$cJn-l~Dk$ga^8l5x9}74A2dRO~i0$LEk%E;?M>@H~bIU>m<- zlN5NR^vtH!a|eEFMOGH*Jp1V4Unky2n)t;FJ3My!+**u3}qzS-MtzNY!Tw`JL_>l7l5=nS0xP6O7eYXnNB_B&gI z?ZofSV=zJ$?8~3Whl{G&yxMg{oXSQp13TM2=8aNLs4U@}3^KSqm)~u-LMM>ptO?#; zm(8rjgA1Z-`?KZyx3_VSP!AC)+}6ib(ZNV9197o=qy2UnRsQ1F4#`;Wcw93W-p9j^loQ6rnFr`Uj>mZ(r&ZhUPp53iRTZL?c zn&O1&#reA>0DBbz{m+9?HtS9gA{IK6>58ti(mu{!?g`n=Ls6%X<0Td3r!H}clZ?^Ls@A+~I_eV8By@Z{&2mQL^+?KBU zZ%vJIpjrbtI5>wSia4u|IYSY18x7|R?5aSCNMFtvRBv9tuNWQJHv0~DOXM^oJ~?FJ zj68GGY_~>lV0NCvhn7o@>T}nH7hw?x&GL=yHhZv~wAB1jv}pCi9JnS12XNZ8!hNc; zk}sLCQXA;6a|QBQLYN{s)xUNBem|kD9|w7ok+Z zAJr0THv~4f8ZF0abv4l*z>5YkEV|=afP<`kA2)&(QQbpL8w}7zBwMTuYdhatz1*)e z>G=Agv9OE96|&-S{LUby({w+QM=`^v;zc_`cE?9T-mZAAJA5vq7gAxTqghFP37+>l zj+OH|9nD7EV!AFmkV5-|y);--l7nCx6n% zR{HyNj?`1Jd*5fd@A~|DXTFtGQlS3{i5MJTJMMqb&BB4_cSogcb1^~!n$?675=Lmh z+iO0RivRm8<}k&Tl-pQv$h~X)@87b_`&pj-;v};`gW$b|+XEnBS@&PHBQsgoIh*=p z6L4?xHRX(YT$ewYWW_Fj5&Hc*zjQHEQ!@XpJr-58yV*tAcl*zFh+L3@_q9MH%_+~H zMy`KMSzf>+R?BLDlTsyn;H`pACfvfrR>x7GOd={*{Jm{v-nm8Zzq zmO6G)M4sytRKP*8ZjMUL>a-&`QGpEI33%zZ5tEYwkEc1 zYhv5>Ol)gndty$UU(fsVz5jvxS+B0OdUaKub6s`n?ETs)FCmXSlJ;}o&YNd=_hJ`U zJhz1{gD?Q>GlQoWt1O#cN1tI~W!W>S{XHCp7Ut>#f8@i;^M66nW14BWd&^DTw64i* z_1bor-}}kNP|Q)=8wHuV)@dUj51XvlDhiqpubt#+aoi_IF4e&LY-Mioo%A|w#SPz# z-~Eni89#qKS9~ose6LsZ-nXd;N+W#LZEjK5gi{_Kl5>%^dRs+AMhFjqX9vG>6YBKf z;@WaWi+={%8m7@I(a&+&&$$Bbx%9JNqCkHG$^HJM_fO^if670BAxq(!(x%V^3@68V zH!0-*&^Ewuz#m=sGrZ?D8xFl&TWAq-6;lrHBMv>lplc5#k@AKI(q-qZjuQ-_TT4&i z+Qk_2n3(aV?d`fZ-^U;+N_HE_NATsMDY(vt7-20txFJ|YW0chEzj?}9i`u*;QHq=@ z|Ka1*38`&*FaPw@$9*Zgebem`tJnFmm1`d&qn3B^g?&V^My!)`$_%pRr!>5&0Kd{7 z^7d2#%{G&x322`e1CNxzT|>mX=NjhE!DqMLm#Hqix&x`Kc|y;Q_ot06?U^rS+}`h> zNLvI*vP9q^siV%CiXBIVUH<$4b`fz0(zv)|eNOU+75R{dA7>aF2P2!3C=Jw_P~le$ z!BBAm?S|VIO9f^trW02w_&(>VihQ53%?cw0ubs1<4?WG6%nuQKFE)6#tb7*w6OP$8 zS7ro-5)`wWj>L7Fv<*;3CiS@-bp=oBKw)eji);{P2d_nUS*_N8&H))v`H)RjO7_V< zj1w-l7wE)=5^-I>={*d*u+lFJ(hw6U47M1E7P6XjWs+O;qfq$f3>g+5(c*VgCg027 zpD#d5Vy(EL|D6$B>=Dk5^VIdlF-yLX+{;f$JqDR0?b3?F+nEwJPqVO4B6xf@fPE*n^Rl@TMWP4+2)ykN{*9G0#AaV6gUGq#<`U?Fz0fNDL0&6$^jVON zf^-Rt+#@l$tk<7L%&S|MoDv+f-b8A=v-2tn5-epB-}Fs7mrQf_OL8 zhu{+QJ2$aBd9Fc$fx==ui>Vkhsd2BrQwe2>eftj~^ZIFT(wg)jilstieD@!CZJ%ae zw`O8UrOg{PAXx$g6s6!Ui>HS{3`v99akRjAK@rdR@z__~%EyAKb%M@f zwR1r9;wl+r-0RKpN}qgb8{WxHL!yk6Yg zm*;Xdo{vURgC!c^PmT}D+Q!u2VYcDYpc*KcT8W3l_~%}KzB0^VmbSi zkVWP8`ywR8W-`2zmGfd85kVXMgY-?cndBZtcrdG+L3l}re#n?Etnx>xonE-|WAVbf zI`rI}1%mK;+`#{_ylJMl3>^1cKp<`|%NuvT6lI*3NE9rV^&y=xIi#va{MvDiX#&p{iQ5C2Xjsl6 z6WN_oPMTeKvtCbKi^Qy&@$BmVX*OrMM1CtHjFKga6*8RlD^EL&7PDEP@E4L{e8&XL zJ7PRb`CIkhOCtG4O$*LpjWLb%97fYbM8k_g^@0RW2Ea?3;o{|tI|NaLJh=@ z);P8)o#2hL-+NYs?vSqo*5s$k*biAshX%ac)s-ESc zlO?Fh1o4KLgIzjueBi{h(a-L4+2OSbEuw%L_yL+LV@*d+a*#-)IC0t&zT7MXsKI%= zXmDXit;?LNw{1D~JxVtn3!6RsT?z)U#9#vKVUeLV>(zy6*w#?EY~naeWUyiq@$&=- zOB1Bg#C}xxLw7BX5Cf#3m?QjC9AVh(t~K)QATBH{sh9mQFOADhL7{FRq2RE4G_Jh% z&)#1Te$YJw)tTv=5!&_l=c}=S6$ci^u>-ncWUd-MO zZ5drUkn%s261WT8J6B2`P}QdsKV>LZX3T z?KUP^7YKcQoOY`Ptm{C*Tt)#19wN4OTixM7xcD+ws6$=d>x1>`*#^nk5R7UT+PG9T z|7})Z=iguRm%3M*eqWzR?~hWn_#l*Yn8%lJ$z=J(7~llg<0UbdJRL!C8Ij{{8ZOLS zRLOtx!GN;MKD0*6&Uf+->(m%#%4N_7xmHpnbBfwt*+P*M6W0CI!ROKEdjC$jt$T;B z*TDDQ4|qaKiR>>iv{U0V)db6Rj`*VH|BnR#-Q5;^wco)EznJGyMMvuStg&RaCjj5OIJQbOpK^}en-yjJ9R+w+sloDRXQXdJ#|ca$gtMu^7cP`k0ZTG0YUkwNqvi( z%|Au52u+{dm<8+84=+?9rF<)kp(HFPR44KdB1$`RvBsd0q%?@p&gLycRTnR?;_;VoH2&+`d|3kB_fGWo?PiIc!bea3j|G3r zSbpw#=vtm?Tcb$l#;z|H&46{o#>*)xtkHZ|{Ljog3o6=`3A`q&Q4xLqur?oyGC53%~2R*ef_I^eyM2Rb1PK}OeqRrFjMrFTN6gtbQq&U3&%*dvIRmjNI! zNy@jd(;=x#5|F&=qgyQxZ8AV_P`c*DuXVTvab96S)u`^7k+P(?(^}`&aJxkIW0!_w zE20Oex9M8A254a9aJB+wf8FktYgS0UV{_0fmQ~1~^yW6Q3ly-Ku)A}VS8w>*z=eVg z#}p$a2D#eHl}Cd_a&CkcVOYTXBxrRgJ$SV-k8s;C%=Y#Gb5y}vfiNhO5`F#kil{Or zZUL8BRE0Q}{uGkPUu$ojeQa9EA`7?9Kp}}8-JwbB-?b=_7-e{?cyZANyvzM8W7Ne; zku*|lPSS>E%f6f6(1fFW1IQm^`Hzg3)l89AqTW>Zj2wk(z()e>dL@5MXL2NR4nZLT z#N!tcV^PU)U1eFyO{*!O8W-$DS>vU_Ap1$p+Uc#89uhQrsEt~PC1Q{m)zyVVPCb?4 z!FfQRVlkT_g&@V;-qOCGO?U)4;h|$sj@t%=tO&rntNvp6gH;>C;*6{Q3HEpOOFAvy zKqy5q05(Y9I;-Uieei?|<0?ZFT(0?jW*atCq=Vp(CpS1YED|lVIh5^ylu?c_?E?Fm zQNf3@(Dz^oK|OqZ@flVKat^iYzEluJcu*?|w;*@sS6p#NX&EsKXvlaD7N`(I(p3lw zLEm7+&I%j>QJGx}9o0!&z43jbnAC~xok)HSu7(OM?@$W_9S9s5dxkp^GTAD<$8sOp zWv;mV)&rkSz8m^P#Z~wio*LiK^cTmtDURp7_^ettQe><=5XHg$c{LC0Nl!OD5Pwlu zWNwA);~~!QordiQ3<3`D8Bn>>&yj^kz-B$%B3B025pcJ5o>_1OKMtZLSWNyYpKCi$ zFMGL?SVIYZ8)DQa=fN1$ZCw?PmC{>G-PONu;kMxB*bH_98Y0-FY0HQ(C>Q5v6ba(@ z21JQQKyCy`3O@LLSl^{Y#t;Idqhm2m2NkS}H^jB8IZ!+VUSK+eq_G)d)7B2_za9pI zQ!%!FhY>Hc;E^g&bSlu-c$0uHmVN*u#2WNf*u;pvv&cuhmtDOg z6Nyr!eU~X!Ee0*qP|p*!L;?M_v}g2KaS#^xdZchkxZmKKq$&|6@Y*{uAu$ne1QG>8 z&yhq;778z5$nl0 zeg1z~Lh8vE>4_OV2N-s=y%DO*{xQXRpVnvrpyi1Dm3vh=fSlx{TL>Xu<*I z9ygS+&AWqLVZWx%yHN7>Eq6Sw0*xMlw~J!vIE5Sf_bo>i3(4P~I=NI@6m zi}QWFt5q2R82l7%a^4G|#DKCn%iLyNOH4I1IA+DtLiC<))I6}#2t}1I$;0Tx_i{dn z&VGMw`(lmmy=31}2>wTP`#rc&&VMTG$aTc_xxZN5`yiOz3qZ3^`iu8Q(Np)^YiK3XX0;p)8aLvC4YosGcEzKt?xritH>wkNgr z+sBqi?A6cn5g?#jGH^YuT0`*YOtJG2=-K}Tf_0yjz$aKoyJ^eDqukbUw5#yuk6kCH zz7{&{lu;{*zt93CGd0Qwxn~e=d@TO1`==;`E(f`H+^8%wwafG>m^L06gbjlHOVoRi zpZ8VF#BB-g;{D`oyCHyv?iwTajC7GLU{&_88`k0ggoWXuYvq{*;8_!PD{F@^NA?-G zb}+6hKFqrOo%#MYq)wzi_4N0*8dVLL@TxkrI__IB|6oIqYRQ^+!nrS#k&v5OYK|v_ z^N_S8%DCY}mJ7BQv&_71AyYhW0RzEBVlwlxyAy%NV3px6f{+&)K9%4i!EHI`4al1v<7F`833j@dK>YDfeR z@WEQkoiT{uOlepXJjC9v%*W0_Bm}n3b%{Z{(pllOItZ(qg3^qI`X#VL-}5RQfDIx} zAfAgL@?wPah+{{9judEXjM{|3N(WBkf6t$bmTWjK6)K@(a%s6tB&-qvYYfpVvW9^L z2xG?g*w@O5A{}7Z#5_c}TkRJnAk!0lS*am2(4>?0xfPKQTRX)ZNcD?>kXQV06d=u- zD_26<4`H$zRtO>Hzy~EWE*6_-4O6Df7y+FPQgO0M#@r#LN!#>^NKmpd(Xu%3Kl=J`}F|hT^X9 z2&M!tzhqRj?%Sm7y(_1L%O!$hqBbnWfGDPQotQG)DH|3F`=NuCiztkaOa?>D!5LOS z1e#Pyt1pk06ciGSt^L5gi}?}cj#3dCOcF`oE;1N6Yb@6@gQEoD@B@E0i~!|lydUu{ z*JGLY!8SBS5bEKS$N@;45qr*F;3l35s3hT15UUMq6{<>qf_g5jFBrlA2xNi4OpF~n zK6w-{;EhjiDfH}F-Li8P@c;w=m^lYWoR>%xG=*bam^*_gIGml`io>4&jhO|f!T~u4 zz#}P8r-cV~6gURvChRt(l%s$Ga!(4r*G@Qqo1!va-f$^a4G~YlDCe zWQwp#DZ<@CQta)smQp<@WPy8|Bv&}C9BFYo&@7=i4J5ek;*-IyDrb?`ybf7p7|iqG zTMoNJ#3vQg^?QzMFbF?HYomwypKbD^ppS#x#lULKkxv(MwQL1#r{jlLDJhN^Xt;4; zZOCAkk3kpLC`2Sr5>H6@zgyduY1HIxX<<5qUIG%w-?Snb{1`IcxCqE*M1BPnnW%T@|`@ z!tPU@m?sNS(24*lX}&8o$idS6S+A0$_@M8lzbpkQ7ah$~DB&(JMKYWQsn>ib_v5(i zUr(hzlQJQXk2_E=JZ&2cFW++Wb|x{yKLCaUPMp{!%3H%%ne`-MVqzjqXI7Yc0iUDvRUqRAO@l#%|Q8nF*nB z+TR| z`)p2ypM|3b(qpmLBK<_Ex}jco|GO>ZcOo_0MeKKVG|THCfS6n2B3vw0gG@kVgE@WA zp>9k&NK|)doZ0r{pA;uP7EF-pht*cokcbq)LvoC+yZ&2dwNMtP$nzWK%e?P1&WnVxV1?s@m8A*(w**Bafa&a!Tstgy!NlsIZLAo{C z^)qwbVw%4HbOOOHVf(M9sAM+};c(-J6|9}o4-@?F^1y5Dc}1Wrmh z9DO82iPDKZ1g!+uu4OD*Fw9vUaXwf)PE9mVwOnO{6AXWku4}HM2m*u`T|8xek!q>G zcAZ9IE*gGIsU7?#2vn(k2>&|=4&(DFer`XbL;J#(%T)G5k7$xKNPK}JrX-zBz6cff zc&UXN)?fQw(S6Ci7EhjwCuIz+U9Q!k)pA)aQxzy*h)&`TtJ5X*L-@dAi}bYSu0sA~ zTaR{=`-I6uN!ftcAb;bILp5ApRaS(PP1eawRFj z@kY#B>}o&1xx4aRU5E>V8A!_RT?rRDg8y}OON~t4x5pZd z{N66I1i4c5X7*Du42QjgM%+J{DpOd41*}P&7aIX>wH#zIwL%$6t4e!E88M@@EBWh8 zfHyKb!#d>$;yndx(7K&!*$VtRmRAbIt5Ptr@;;bMN-E=Eg(4c&1ZyB@B6ZZBu(2Du zY+;8zIc1B|PSYL;o`$NZh*=fcJc)Kcj!vxgx?KGnqmHDkbV4RMfdy+~c`S*Jo?0&A zQ+dg(7CmCF*O>ZIBpwWN!8bsjg!1GJ;%SUUD3}<&SPjdO!1+Zum&s_O~ zGrgDv$sdqo2YNWbZ+b|w!%!U-YvH)Ioi}#;p2?OACja*0)s8@U@n%gtCA=E9k4mB8 z^eWXy^H5>!2FwixX7lK4yNd;l!bVEoSstLa<9706Cn$@TPcuExjG6|)@i+P*Ocn$a z+QNomyXAv+!(6z^{p3SNx5ghEX!)F68IO6oK1cyWF_d=7(ntDU#F9W56+}rAv}a6Z z%D|aw^fXgD4K~8E$#0e^G|irgYpb+DJM|9O$pfHXY_HYno2P(x@6NP8V_HL<@ z`0>4eN-BJ~?AJWbe+P6J4Oag6x*5jG3t>b$3VAZ58@~&cR%I$p{Ve;RJe~#e!ps#^ z@LL3$_Qb~Z#@$-Li)a5gDuWTxPQKwrVgJoN_M~Bha#eJ@uag{d>l&W33C{1$s)S@@ z{1J`3s2LK8gem4Q;UInDFtGBKJIXqR`N^=Efdjf4SSn*pfh19>OI97qt*>g50Kl#G z)TBHvjz-jlabXoc|SzUM% zv9I{`q~Nij3u|gz->WS!v_B?C!uZUT4b(P}ZgW;C*>i~gi%eG4PD^8JhE;6k7EV>F z0AJBtZOSk*`7#eBN>OAvJYw=M?vfZ4VUMxg*8@9w56UoeuMz_a4BAW7S_B(28%gW< zEjCt+EhS!-yFnSL}G=+p+C{@L0Bg$c)Mz zSdh3QloGWCc?S=hidKkK8!cRCMPYMMF_@9*UEp{?;ihpC8*%DG^Aao~@+jEvz=G8~ zy&U>bWGWkVMEA%~19xT{9Na(DaVC!DwkR#ZC1ypG3%tgBsbyi^H)7sOGO&2DOHtCE zOKL|Y<(S^Qf{bE3)JSBMbcN)k~YfeHJirmTx5?0_mWdR8?gU>>Vil;f-V;htXxrKtR2DOc4Aa z^E*CtKg@4Fc1Wxuma#_N#9p$!8lUX0?sJEkFG4t?^dlof-5>RH_}iI z9xa;OKdYs2tB@H2Q61BfVko#TdwL6t0jO0l+#$2 z*H+Up@s2zL{j`fHm>QMY5ua+|R55eJU__gv=)%%_C>GdpF(z;pthX{kXqoMlw^m;F zy78G%Mm1eiZ~LBW;>**IJATW8C-^e1XaFP?)_V@qIj%{O4dqzOL7fq!DjFsKi;P{- z_sETeqT`w##xk5r(_~brU+LZrhO_UvmGsOW1E$hQ^*)p`>3g4*j`&22g^M3E+}Zg1 z6PiuV=bW&yvCVpputlEjBL`{fRYH9ONP}$OB?@cwkWy+oA6x=a_n$XNUvn7Zp=orQ zb^6`uEsimz_Dn!$>Lj`&k>+Z?H?b~`$}f@av3madHhbNki+!&En6Z za`MdlCFN$ddf633tn`enCcMA|YR36hsX?w_>xd^aC#_P!m{(Z7RS+D-Z}3|o*$fmt z0&@{2J#7>iJ6a_jg1f#_h_K0{L#fLR7TVB1y%IObqV9>mZHy+cz*^}DJ8D+Hv4-?P z#ls+dPj*^=PU4QP?9xYSz~tejRI8I?XO~c|f-$7|<+=Ik*ob@cvSQLlNdiwlP5AtJ zgCYEU4&D!eDujT2=z`8d_X{PyO{`4>okD>aq-Elz>2sk@_t$fnZGw>3BT%eym!BW7 z#$!3VI}!Mjx(U!BXmx<)I{hCdV6{w`Z;!9-t-MzM-?l%ra-V#R9c{7pv!}`dsTJ;! zoyAfC%_xCV%Im!A+282-n{eHyE&mOn-0r=S6M>Wc`x(jamwS0qws2f|MEE^^7qGhzuyXcWISo;`n5zqD$l`bXAfj^sDtlfa(09wSub)%s z>bVbk4))UoBM|r4=l*6@LfE*&qiGp1BZr|kLWmI6 zP_*Fl8J{G9*qFZr9wG3`)7;NvPGUmqNzh?xtoOkw;i1u_<`Ao>k0;^RN)w0J_mjquE3^25F!|scb9|jd~b3a64ClOKzO2)KM-X-A;P#b9a z(RZyn&O4X>!x>g8Gn@(NTIPk5XvT7zeipTnp@x80E8ZGG!dOiR3a%KO_o_w;eYeJr zQ)jITRv)nAEMR`p2a5|8Xt2S+HZm$Z+dYFtmC0Frp#D?IJ$8B?Y4MsGs=RbG5@qt) z^h28)<3!e&l0rggmN~z$kzMHq9cy!-AEAKXN;OUiW{m1FoH+P?NMs>YmGuK*%H}6% zfk-_iKWKX}WhTB++0E0x{2XCz=TwhhNj}>?wr3JHZq==_ElVo4TwqWzu5*gjgZ0-Ph*5S&kd*ze?KB;=vUI>NHDHm$$y zD@(Vl|1fs;?qDVT)o@#>RZ*{QuZmZqqjNAbGdmPAec9JlEbD#nkqCXR?6gMaQllMJ$f=|}wo(_`#2av-n;5`M+OIB|h=@p) zo*x+nkSp2@kE~HToaV<)hYZ2B3<01?B!-$vW|22BqL9GlV5CQ|>{|;$`++no3TE-E zd;aydm2BN4qc}=&9CDA(6kC^rr=4mZdXM@R^*cHPP$}@^LGBwFN_=`|W?drkZV_yZ zK%pxLxlC!;0B%}#Afi+~AiZ?ZLqxPnX529%_xB-@2b_*|JXTxCBk z%?o7Lx(yy4<$y#mmD+$*DGhqhkPVs2*;lD*fPt2KT#Y7U?44|DV?~965WWbW0tJC| z&K!OGg35s*g(BF@iDaz$fF5O1>Z4R*qM8aSINph9N6CJ$Nt%~!OWtU2Qk5-NR$QqJ zub^SNXMDXxf`;_+!hJ6|%v_KI^~LaQl<9r{>sB_eJ9y+A>dbz0T9(uAbus_f?FgO_ zu&aGu%zj<~V|)r9&sXkgebF=H%m&n@OST{DJV@k@a&SUA@2aZn_EAq#Sw8cu)yRK5G zB;4Wkn*E8id7~M?hFVLljCX28j#`J9^`iExNjw*B7>xfg`WKwj?mM?x^uPhVtr&WQ zoRcl3>B;ZtMs>lqaSA;7A7pI`9-7!XcD40k;mmzA2q!1zbD(wp2EPBD;>;- z*#pcWsZvU==gh+1tJo+ES6r0fO2&G=9UJFDJ0?*2w zSq2URNiQH)fw86DVfZC3WIlr6yxlu$C3EIqcFla$S@&Uhn%z=GG<&2%iV`M4l^`K5 z_w$HmK}>>2W~0(p=yG7%xK&fzgZx{OWDdGWyOiF7#T8e(K$=~^Ri(rBJ9K$ooA-f) z<*%c-`fkqyZUodKwYE%7Z3y6X;seugbn6Kq*$|8glrX{s^HyXY;}~j46s1ml{hnh> zV6NGKJ-J`KZ^|TADSC9Z z@*7&_54d7|C}A07*mBH`1)5DmT%u9C#-evS3a5~W5a*-ww9Tq5Ty@T2q{OXMFnmoz zG1$djtYzPw`uTRZ){^@rC~#4y#ca}!1h#biqhP?I);X?`uGqlX-nB8lVL3%|e~T4U z78O?t#Scv$(;SM*iu^{d%qAz3k*1xzZx*|P?4{cu2gvEe2h9tmDrElaZ4cg!FLyNb z3^@p4F-BU>Ey}60aODPyFo?@yhLM6IcNBWhxH5z~06^>{*7ILiHZ%E4_Br86e3vn& z=x7Z<(&fV$bf3Jae&j-3SelxC)X@040jJ*<|3-9<&L}%ZmNsW(zo&}*4HBL}9|@$V74OVx88{u7l zA>@eZi#738YplL41#S2o7g5jR1ER5E%QT~OxxW7EwM#PK_!UG~>nC5`8Vf z>}6RUAeOA)T|VO=(;@rKu;5feWRr=P)JkrJtWp>Wed{G0UOrZiZ0Q}D7aA5{%VxEp>niz1|qT~!z;#?oM3YY8afNSQ!%r$=;5%w zv}VRAGQ>RUwsIZ?kzyCz$G1?8=S4GvLSt8>Lpst@(_PY-?rzdK!CQWFpv08R`?dVT#wOd3Wk%oQv@}N zsItidCV&N|6oXf6@>PisdGr)Kk4Q34FkLW-R`F3Rb#j#x;s9aI@X|p z1!Qq0xcyj^hH9k8>Rl`*AoZsGH>i%bFM*NSCB>7e(^O~yY3tZ=Z7lhd9A4UuduXzP z)~zt0QU}O&=+dgle@KNf93RLISz<-WAPbQ~GV&%otK@u}UX21y)N@1oWE<6DGML;q zwxZGvKM#byE*16vI+#08X2QSug;o8qNHz{8@7@4rya~Ml&85e{FrCiVKh?AR_i}cH ziOgHhlMKK-NG=G_tpw_q$=>(L9|qnuAj#v>$&F>`?eFiiU+?+vLO5f+uhqS; zFS7x8>x{piP2`$x5T18KkUl?lz67`s9la-kj^-T&|-x zJK*Si=e^rubCVLa0xKd836~>PZ%`v z@_AWjyMZGVRz%LJ4Y{F0+34TIWKP>kvcBZr3bjSEA{BUj;`ZN5Z?T`0hpA`$ue{Jd z<5Gx1$h~PvGcfYv6X!+iCM2j4(%}Jw73|WK7XCmDgxgd2DV0d4aJ_O)(!h2Ig@rmY zm4IATW0Qkh0nk$hwpY@t zn}103u%*^@#!U0GoGHu2)RYE3=*0U$PT@2;2pMt;0yTr)OP`(3S3p;ztJK_|^q8z> zyQ{EA=ysDqZChaWIUQ$CCAn6)LPf^sgOC4S)bsMnuS$J}A+w%Avf*qTcO+5uBRfmRU0`7WUzo|!CuTSj<(xx-wqv@f4*dYT^I`=aj0R1Of&bD zgP=Oimx;(nNnxWM4>qF|u)xQC!Yu_vSP-gN1PMiGaIm`sVGyd>S*eS$A$2+Z|M@Dp zGGaxpKO?ed&M6gpjH0k{oq8!$0q#q+Ce{m-XbO9R(_X|ad$P`A(>BTB%7f5VsP zm#iZE00b%meUy0>DQ90%>hPSUV1J*U1wa7tsK3Z^UeO|lh+q6o^V0C&BhVgz-ih_S z0BgA!YzY!=BH(>Dzs!dGZxif~SHNbbfzqYP2#*nK=C6RKDj~n`ES6JgVlf-92=tiw zgxhiFvZvPRp>XQNr4jVG<>S&u+oU0wj(Nts@PpwpFjHp(xqZ zN!93e-UUGu_N@_{Fi(|GhkNsxkT2|Fn#xP4Ybp6%m=f8x>gbhXCma-lTceTlO6nXg zf~ZEVK`4jcsfkkZoB>xZ0rNLm zgxlFL>7G?b)t#tNAyfFIET&4j!jlb0C(?xYjI3gG~J1m_@Lu803%A6p;e zmNPHiRN2(I%I}S<*KLDtL3(*{nrr!mLYqdv4{cjujIawj(uURxNjw6Rm}8U z2uWw+vyk3Yu~}>+t=Ny@g=VSj2vVyT5%Te!%Y?v-9wa7aG!9pnA^4{>UnmO^`5IGt zshZkNC$iFU9<;Sj;`2vxYwi$z$J7NYUw56~?=T+LV|x4Zl}K&h<5O<8w&9b~nWxHnt^kJ5|Msy!~1pWS1Vgn1aKIq)dB5K_vXC;gLfJ{S;M5UR*DZ4+v2s#hNg z7k+`_6U`~={9pH%>M?!3fIi(!fyG&su$=nvQa1)Ml_Q8tS52Ku zU5{5Au1zsBpAPx>)6`&yP$8-i?Kov_yu8xKk+#tH$0IDS=CI7vIL`Gx0D81_a`UDoPnG7vuLLM~dO12qp z)5S7*HMgTQO;TOfeTqNgq!KuJvN-pV|18S+Z<{Ap@a7$6tNxNX;r+i?SFvXYVD^A} zVO@WFP6W6)+2!{Q6g;h_DcL)ptWYwbK}WiuxT<)@l*+^LhsPO8Z1S;cVKiJM9ujG; z_s>@qoANru9|89n2N+#LjWrz&n6@zyglF5&NL*X~qx#eKrZrZaTS*4pD>ucgmAL-+ zyB2}Op+)=*E-`J#`^Nl!h#Y^s)`dnX!VfR##MmscbfstJs=_T8OWM}V!~d1Mr=M`+ zI6BL7Q(;S--BbUL&HVBp;SsCG&FV~CxA^hjv9-U9opoq!`M`U zK+qy(KUIgMHJvuea8E^$_@c5QJ%jOvVH#4eJp24nK{Jn{+lUg?wGL*THm+fu)Ck2B z2SD%Dul{kxD#Aj=!pFh~n5}UH$VTb97s4yW8QUAb&ZS^2MKL#Gc@C-5Lh0UgV|j*( zY8y?LA+&wI^o9+O?z6c(ux#V}^rk=fo4v}6t&frli7Q@;+CiF#OpHnzTmYHj1V^E~ z6TC)&!t@-4&~ZT8N{SB7A~;v+cX*3M^znf=BNSkYT~~2}_BKj-(}a>S>3wIe>5H$v z%hPY@cjVChQ^59j`#{CT(^I%OJ?vJhK5xt|=s0~hwAmWb{|#E8N$Cj?2@xY9gL;=t zZY`zE&C1zkBEkD;^15HQ^;ue;T`l|9%8Y--z~|C!ML*)^>~@rBr_pqjZI&NA0qM&J zQ2o1BcXR^>6qU8Ga~s{4*rCJrU?h%f+mmb=HTKW8&-GT5Iow{;?>c>-m!lt}jS%o` z;dTml3?PNslm-yA<=c-4##K1I%MPNeUQ2-hLJUq)*;gx7x{*xc=e&;ifwweZ>pKD*o zU%r1|Rde_;dW*)&RYY4PLG#?rpKVw&8qyLKjh#!W;(y_5$Ml3!(J#hwwDVpNt!Q|f zkTx7bYA&At%2~!kvY`G}qz8Uf8|5&Ly3CWQB{wq=-&N&Zr~BOL-$U|hxZiQZYnk5i zI4f@Y)H!U)Ny+At&M0VywfcQGJ4n-!4&5C73~4!N0d8CP(OjSf=N&ujKpNSQEDXurh;kE`cNv!whl}>H(sN=`$@OY~$bg(qn^;?7;E;Wu5q)`>W2`eHfudX*eIr zl^1*=wO0C^x0i(;hOWsHOY<6*y(WAG3?}yXOL84~nl!&@naslaB;d=ifjpWgB(l_%2Raz$pc`1YU2?^k5}6QXD))^c%O_MQKU=#EHq{LehnKzIia+;r z9)(^;R`h>aQ{b&7M!GSl(#A4I*@b1MDLTp-Q{7TJi?F3V;1>nhGNOSttRm{}Ezf_g zK1EF;22lX7bNXrhXX{dQz8s=K+AFPbSzzw>F%&En(O3rYS@S(9Ri8lw?8_p z)@bL=r3#^rD^4OGEEXz(q+=b&hn{>`@MPMM`R~aISU^ya=l1DVmyp9de$Vk+bf|RN zv`97&-HOdeczeHbwk697kX*BDza+wEdSucBlcE`8l*A-yPE)k*OUmGargd&@4)8>E zPf$q^Kd1b0MZz;t2=`aZPu%bQI;4A_wj>T0Q{)mx(97w@3ItpFz&H#tOl z&#nx0Ox80pFUU;m6vYHWZkr*{Lmp8EKPKf5d6dLEqMI>Cr-y?dSC+ZmXVr+QIUMOb zAEj-lR9+jv>5G}amN7Ne_&oKmOhlF_g@~QWhvV2sZ-HDRRM`wyjX94p=)nt$TXV(_ zIz8=Dk1{juDkxciUBZrtm^YI$GZocG$#T~DZe(x!9&3|j_?iwLyaD%mq*a0y0h~9Vtwi`2iN-Lddp{KSr0H3MbK{Jqb90RtNSaA}-<(+vYE13S%|FJ3%@9Y7 z+W3}(y|%n7O*uS9#4d-`0v*$b53r&l2u;~#EzT5LCo5pAu>|Y6>+>Js21`$3PR6I| zul21+78Pi7`-AuOoU7~5*)8`|y0+q}N?kh^*aW&@ZGfXD!3}B69X$2D{Bg2mMWkID6Q|v40;$t_%EAnGlm z;ly9qsRMx19me9{|BCDXMhJcYK$y;!i#^kwq$!sH7~ZV6D*VBm5XGA~vS6{*b9=c;*UoxSB#Y&p_caz#Cx z%EKdW#l^unb1jQqN(&-Zq5G+Mq;)QCi#EF!2b?#+TVpJ;OMH^EG|o)U13xb%1{ zOj>m})PQ;tqk&X-;QfMq}UgPCh>JPCR_Yi<>V;`7`M`=q>-_2D~?5t5wtR z?~^>Y(BgOur*m%h_^y9RJl=vG+}?-78(nx2DT zkISrt++N-(rSQA7w#Jcetsl89LT8G>X*M?HxBfx|WYkmU9!SJHc91?D*`y1?s=%+j z8M6-_W-XNnY(XW5nV_sUUde`{+5BK>?J_@LuZPF}x;DnG>_k6z#>QD6UIetP6>gP5 zAa#-%^wCoCqd``)H_uAlMpymwP3E_!M?upNox#h*Tc!GoG9(g6M7b05G9z8y4W4Cx zrBm^tPA|!WA`c)YI_!Iu{a-yEbudi(L!g9aRd<}7$;Ros^K+>pU^500V75lL9Uj~j#4)DZ&t)27} z%AfmP`m%O5e;*g>@4Wy4s(Lt^!Zx7GJB5p$6SJ#l5Z}$2*Dc_;Y6cFwbtm7a{q58~ot9MRl|!l>Qpn2XuJByZ`O$>ol?_ z=w}ae0SSP^soghtr6Q;MJcCuIX>QEvxz=COp;Ch__%AU9! z2``#Y{S2StOH#o11!%|lc!qK`ke4YCDb11oF(jWdRu}A$kbO!XQqBEYxZ3p;sGg&u zZ*4dWuFw=hUZBJ~KY_m}+*gT|pKIA)2wFgC+|S@y4z7voaY|4h{%qJ-tv253J^k_L z`w!y=8rT^@r5UU69B7`4!0G)JH?Qp?W;d_Pq=UDWeWunw?HQU^kAZA;@*Jve7*eLm zSZld6f#ThK|F_b*{o{}I*Av-aEQB20REefKr0C~LI^l=imn-F3UfrJRN7AbV&UuUA z7Aj87o-kZgCj%OeeK^6h?!>h}z%foe3?^d;64fvwgQUpO4Yezcz<@Y124c-UOb)8s zR}6<3#hKlq_t9GlMHYMnZ5x0%M-7$hX6;>1L`^5{#ez;7k*veBDaM=ZIBKV_?m9?J z)-u7o+8yRaDe}q{Z?4P&$%J=J`nN-Ta60*)^VM(Fn;$#X2NM%`=_8{J%jnYaRS-*n zG<8qL@c0Sd!HCRt+JW!p|C<&d`{w%qtSs+jO+YHnvma(KDr>(vO`D(0W(ns4KEQo% z{I(#D0@n-~j9SCP%HIQN0j1`~frSy$&?KUsa_o1LueY(D-x`0nsm%{7#sw$(w~vU| z*6l+Nxl#ZsYzt?bnXe6-(pWc7fhw)9XRg<2e z1eE+Q$?+ZB08UQnyg)?<>Yj1QDCkNUy-O!S2lD935T!s!1$>9pL;Xgpqz)%P&n~T7 z?_as!tC}>t*PnHBSS4Kd;ZMc5Cmmr8UHbwW?s@@C-1XpC48FlxS(bE>FNnT}Fy zs38sLwLf_1rrgGx>e6)03bK-t@|Q$~Ga2~xjA2U^^k~zQcr|)Z#4z{u6>Z&NYC%vD zWrPHJp-TISu_~h8NhY%5W8AH@*X*gb^R+9kn4?Q@%tkT_FpbSpS4s=NMp~|eF&RgV zn&yWgRz$c@vg$BL)9TA1V6eFO@9gKzixlntoie%1k<>o*oLywwx$a$WSP;R$Cl|J1 zhJFI0`2T~zh=EnoRdUVnWa0x2WrHG{;|r>^OUmpgmW$gTHMuq6IPb6ILk1CG8!3@} z4I_QDx0ljbEFv7yuGH$6amgeDc@uAFa+f|dK2oKTAb|`+-vhoL{gVba3ZGDNAWDT?J?@6`x6~9!kyFt7UX4uvEKRT7LCPMl(HOMB z7=5q>E>P)lz(f68OnyKc0-3L=AD#qi>rDH3ixno5npLcYx805W=k4g`%fbuH$IYu0 zCwRVPShp#jffAhFUMK>N^YpK-z2kqS70`ctnsJ#%|9SB?RD#*WL%(R*?vZWme`W!A zYcGPenJar0y#Yn>X_#Cu0!5pgil|*pFK}}!A!CP6j&b=&2YrN8eWK3@2$e#yJ$J!@ za28H*>c3TTUq^*cB$5LBM$4uDs?{Y2>J4aao`K!>jCJFS!0Rr4qZysQV6d=#w{(M| zn2CYBDGsw`cO5(w8EDdUSZFm+Vw@J|RGxpESNy*2!au>;04`;?VMkPm7bA;_xTO7w zOX+nzW$D~&ue9$GL%5N#(dp}ZTOxQemsbQnlE=C2Dz1Xew|Ke;g;A}?jg;7*PsZy7 zf7rMx*OXl}o2JLp$CD7)WExFXfw8y*Xs7YA=uV=07LX4ZIPxHz8e^JTEO<^C+^fx~ zOq79*Fw}S8hW-&;FkGHK)>{4U1UaVsH|t;i#jI?THJ%0{@GW7OdO|s-$8a`6BjOUF zer}l8&|uLa+c3Ucx;Cnb_)-Cf)5A^&KV?;)!Ah~bbOx(EpJ1xm9>Toev&?bO6%K&C z0?ZhseENQw54_Qs1bi`e6=3ARZaP?v>&_ss5^W8mJU6E)EFecFwAtl@lb)!elR)f! zQ^@k=1Pzz*S!&bMOYfWIBh@!!eL8yM(%m1!Iik?6F!2q-G8w+DdnlZN#>sTbzSUm$ zKQ48Y0g8F?48&wh5GsuiOE@N%edZn?jiY-lDl8Y?HK#=Pi}AD#c{h04}lci*Do z!==jFX09UixQ0f@`^>kxxZI_0>m95lShTRCp`j?=%HV4eDk5n-sOyKqkZ&{QyG6`X z$#VSXTmSoy93uV`ke_0gZY)sDfrI%f;nYyEDYMjxV6Wg`B#jf~srR+n3bs?!LRP#0 zv5h%2dksH<;l4k7n>S0qtDayFRZ(lzf(`i_@bsD%E1*`0gq8dL z$8&&T-JI%ojrRWW9|sL(I&nyQq5g-tZ@LHthH8k&Y9^ia<5%49+FmP+HPygs(G>QS%2cC5bf6_w2HW~&G)lDW$UM6Vr=)F_s@%t;z|VN zf{+Qbc#zWGe`mrFmu73DvqCLud;E_(sHtyiNQ4(Nw_#Q4@GjF%R(~7>y?IP%;{W|=Z(uZQ8r~2)4Ez!l@IeCjl~0v*IRR-zJMl;ss{zvEk4bs;;Q$5KFs59dK)AUc~?jg zdRdF+*%1H}Z2-~s!`L7xJ^0~Rb&ryf%xF&^ST}W;eeL8I1=%aR``tt#A1#T~b3A2A ze&3zsiTyrw*`i8*REcUoSmwFhjILXK#2(V9!`jA0Yc^7+^JM;DuVt{L2~Mo>ko#u% zdURmZLOUwM@|qGGaz|!h!;N+^6#mNIP?nn-Uf!hb_I(y# zohdFZ|B2sK8(zoXXYp*awz=z{(%tO#TKHW7toI(=`S?|7W^%A|a-NL!t&4yB4u2K* zAfrzhOQGk`Q{4iWBlMRI5UZn4w58acqcl%)!PRoP6Kv6#BJlXx8dz>l&*ctAle=>O z20>L-9OlF>quLA2COQwM-Z9tqx~i_V^ib;jG0NQeOo_t7Xd^AVr8ifX0&JU30YnY@ zElr(twd4M}v&Bh=FXPi@7UpLdBfx)QUhVo$?C`ui^uGdzN?5@}Drs78a7i$k#}in9 z5QTq8aPikORLJTqiB0mWKw})vK&&C3dM<8(&z|NC; z3;_vrTni>y49npIklkr^WJPwg*0RpC^}ng?fFHnb=C%V*Je(ukkUX?4=Xqq+{Cg3$ zQ2-Z!x(xVECys5r`GOVYo1b?m)}8xUiP0~vOJ!+RDEC55;NapmX%y2*$)TqR^vlv> z;L85hQB%<)c=XrSHhe)pXu))hKsDU2K5=T7=ME1{SQPxjUj%>;{v{aUuW9&ca@;>- z6JsGUSd-9FjYwVylVA$5rddEL1}mV;Qj7O9WX zIq_221|)%KvfO+v<^FiXmu|m7*?gS%GP}(CnngX0u7IilCouzwYuU_eKN-h>y;qo}p>+|EsZ}r?I|n1nxmqhDl7XamB*dR?0D|85Ni6bU`>fjb?g@K$&Bo zXeyooL0rYqD)GL7n$)zF(IaNLHIKJ4AsUM<-++*XNB`Ycx&P`VMAjrh{^is#_B>>C ze986EF96LaZYH}lXC6snbeKVl+$>sUMy^e49|B2y3g0wTf~#zug%D*mO(vFML&QRP z>QqK!k|gLCnh(XUnpxnnpO@z=TRX-yCV#N2*w|GQ0}wR%_Vw6fW$ zAv_X%WWSC^%+!?JXjD$F3v!G_<5eDGv4vAHk+WNg^)HkY$fYm`t2#iNUee`~_FIV6bsqL%- zVCVXQw&+z8i?%~Vh4i4*nH~4_5~ruE=S{qJ+hvxet=r281*Yhl&;balQY{v;;00UZ zroxVdOtZmIMV~2&Jy5v`pu74dzH0?|cD`I_fW1r}XYXtTXHUc|?{W*#Ew682Hq>`a z=sC?$wfhbwtICL7hLLO^%*SRmvVzhd3!?9gt@=|4( zQLSRR(tF4fO95e|jQHJirdBZ??fe93Byl|lp)R2%E+s!I3rq1i1QBEN^SD?E8e)j9 z7}|JRQoTB3L>&Fn^4fI6iSOVO&25WLr@c(bNa!c0#9vCRcOyk)OfkXZ3d&=rL=XBj z<`egRSqp0)%)8+83gs*O4AeuqA$NVJlQI;fTj2BL9ffLgxJTG04bfnEdETM$3i2As zO;zv$GyVQSFu8VCaNM4=6)Asp%H0z$&X}SjTHI1 z>FHSj;j&XXSIV{->#S&HTc^tgnm!->sVycqKmR&ctDI*?q=Z`^v-bG~W#mAVm6C-( zNnL{kJmS#BMohcz?LHQ#wt_!If9)p#5Wi4WfB%0|{C9MD1M6P>u==~@5j)+Bc zAI{H)Uz^e`#&=6FbU$Qfh{C%={d^)=IF`oMJBj zQ&!?b7V9{wQyG&#L`oXFJyn_^0QNqlPF6cc2)jF8U#r!18+isBHW)WICJMKw?1Nh# zn-K?!zb9Z%;)r}&B#urbT_XO@KJX&#TH1;+`hr;uLL8npg+fRn&`0cIUn87BpqQ_? zm7u50PsdV^fIt{Ab1M5JdD@{kChG=~t>ex;{TDfq5atj{`g(ai1qKG1bdg&w0;kQT z|L6YV=h1cWalBqee--Q8Ki(~Oc4oQly`5jVWt@PYH}^t-zC z51Z?V;sC^NRb76PU>~ze(O;>%b~4*_WN;?PD#d56>cpE$$R+ieFAJ#wBS-mcIGtpY zW5|x1Y+W*N2a>-8$4cemLqQ4#33n_aP55OjRDgXclm}v7dCgc0Xl_{hZ$Qw7dWC@!&}TB#s8C+CWfilIDoO zPd==lyN5a;tBCj4u9s#GL{-nJ-|uk>0Rj7c$$D@@tP4Y+qefLPJuneU>Qe`Q^t@uQ zra%>%lfNU*aCV!QXBO${kr^0N*wWF!fO6&U)7T&m<;)0>hlZsm;a-@{^XVj2km?UM zK>|%MkSCY5z;+10!dQ&s9uK9eQANq1P{~#zrvL>%%oAo|uQVV#>wU9y;ubTCiQ372 zGG#M7T+kW8(iX3PwaE2|C8ld>ibF=y0c;4U5GtzU0^X9z!bh=6KcAwthT1lk(A)+t zF~dK9QxRct(a(oE(6NC{i^w#g?NnUh@`nY4g^Kz~Xnvku)8OI2OKN61P&bUe;E{Cl zoe5?R2Sk*>=~uVh4rgY+WKN#BU6yZnyeB?k}V*2A<7%*l3o190b?? zX(S8pzTwfKNu`QjthLttADQDT7EJTx3@PGcow@-7UB7jU-;O-$RJ)sCoKO-Hw7~Af z92m(vGprb7>%y^&fQ3AA_qw8TG#}unY%&(>+sH~4K_1oSF_+Rw5olK+F%swT*=Iyr z3VOIugXK{K#|}uJATR~k;5XnSo@2oNtw~>811C$Z(neW`Gw9VlRu2*-zayFVCe_}o zFtj$>n?dYNPVFqatoy2T-0<*Vz7OfdUJHL7Y6ywRnlw5h3b;T@a1^7PN~6PF-nf{* zB7a3FUjR7>WLn*wQCQ@9csMv529X(xU!Hyny5{}+YyO%Pa?!!qE1$XApDKG%uJTeT zF8PK>hXH1Z*VH=*s!oDTxb6F$G&-(0e+V{!rvA|RriPi5MrCD>BT{M$e?TWV3e4yi zglnLOIK}8($9o4w&^G17cCT5PEp!+X;mrbPoN3;jqpHy!%Ks1@CD=oI-Lj&(z9TIn##HY}|n zz1T$}93*w%<&`^W5VrX6Xom!@^bX(}AQ_AV$OZHpQUy@^5hBESZd?pT|GfJ$$*OwR zygxDs|1`ORXiZg^$ySS@!YBvJ?3hkk<8h1~;3R)#&XN;K{< z6qX~@GB2P4R(zYg(ANm9#eK>090;Jofv+*G3dV|>88~(tyTigigI`xG6-}Y&>1L<- z{r1{j{*WxFGmMM#py>H4_8@X(66m98mkr@ub9{g_3PI0}PzWQ4{m-7(G}97ZjRaiD zxsl4WLF()dF^FlPJkLCsf5ixxcR%+c415QJ)rQ8~BFtyjA_jDLbF3h~)K?U-oe#7T zuMVV%W29ESq7y%&6TDbzUjU4mc+w?-=KSRA6)7vl$x{@ijXi2k89fHm7i!8$;xi2s zMA`D!B@!MZhQNQDo{PGU78W3nH9D~jfi+q|6^F2yL9xBQ|1$xr?6@WUE~6QGtllb{ zhf#1$%I__jh0J@+yx0)mP`?mRVvb)-y_TmNCknWJ)av7^ZPC=%3i%j-XAq~vyLVg} zFXQkQuSdn&u&EMvyreASJp*=pnJ`@nQt&}2A`)({;HY>{_h+hm@t=GF^>r8w z-TnScM<{@T{d~Z?cVD?A94soAy!L5!6mv{%AHFinSt-6+Gs8O1<8CT$fO{6VSYH~Ty2*~lFI^k@QWZnhs{K}B z!IvYxp(x9+3Sq^dp%}w=ao|oh&V`eRv-b>@Y`9{1A~t>(f9d-&>yE+z>VBfhmwjyE zGJ&fa$|Cg-nWPx2T9aLCJmu2owNrCro?(0)XdEE7+_Ke3mHZFz(`?%F^6u+C<3GE| zfhm0)Pfu0RZ{FEX@j3+|17&dBKn?S}vO(=BrUdk92-U zNfr`%aNj0M)SrZwd>jSWmVRdeIpJ=Is-zc`hV!O>4w!E|NXvsWtO)xv=HRK{#)Ej= zjTbg#ElhKs0qNIu<>!3*r2drD>(>B}SRNjF934mz6>G{5-COYR+{?d$h-BtLeha%y zE6pqHsYbC!Z3UW&=be)@q$f#w_uCFqFGIo~JPMKSWLm{sZx~%T={r^z0=%S62~_P& z<9~glbFp=ooaegj(Hw+6`nKjQptmof$QFqeB?#>#^+IojE5w?WFAG4=6PjWyzUccX zG+;fGWLKgt_BIfNln!;eodP0bnyQouBC5n}|5@rcff<9FvRXA5o*7+ru-9NwuVYdU zY;4whc00=LzqLL_Xc-otNV7&g!55 zPH{AcaT%BpRd3_1153pP!oXDqGgegU2b&WWgrcK0g@;Q}KSjIpY@x}Kh#2GoW+6I= zpJG)zGbj*M7&gD_1K9yu{6As9|f;`w=8r5FxNg6liwz?=JL?{~AN3E2ta zbTX}boO7v%@+zNUQFE1Wyj}vsP8q%tthi@*88DT@VL?1sO! zGXUUYgH9OAhaCHmV8%E>s4R6{ zex(&WPH`B5WE%?kzd|>CBTFDaPQKz)w+Ly4V%zECk||ptJQlhlv+bKY7wuR0uZn=L z4~5D*7#D^83RN)2_DfRLVMsQUuwmfKkoU)B-ygHWTCq_XK{9fFYKb@r10Y}JJl8^H zj$X!rSE@0Ch1t$6T~!`Oj}BmL@ROfxfzpMgzi^o(5c<>F?f%#ghQSoXZ;0Sf@?jF* z;M#{z$U5mg09pnEw-YBwW$`ocssm(~3Yby8SB7vr#tkf3i;-cm=(jF8^qhLT~lou_` z1U)In$PEdn2^+Y5vFq_{m=k?%x`n!>!h6n-05E!KUk^1suyN|_CzLIwO%DCuH`dBw1Tu~cG87z%XrIp z_J4(CWM6bZ^5#0tzu%JWG#u(wnJ_w7$_qcXZ`YsP=kW0_Q(yE_>_BkwZOC6{rfkK| za~$)J2g*O?7A;{CEwTIrSvYrf5`ulLQjnCbdbdUiu(<<))>LnItgX-6C!hA)#;4hk zXz|w9tO_1Ss|0XY-5FCSy==lPSXrlmB$|2el9$40DnUH7<6JcIe1(DhV`!`cQbS2P zsjgbF$+ajbf{tGe(j{vs6G8GCUc*0CS0L52m@kJT{HR(Ei%gGqd16YiL`Dn-N42TP z#Wo>zL$^RlC*hR?Obi2u!NxF%N$C0T`l=9QyTGcZdhVA^jjfK147c5RlajQ~(T&{> zDSjj)uU?mTJ?gb9jBj&^~08;PYVYN+ZD#w=)F!dDqt<~-B`G;Zz$_fz9QSPx=c~WCe z3k}1rh$1oy&fp%v=|ISS21kp=HSJ7e0{ZMCM30>==KQfNDJLqYiRn+R!wyuP*;Y?d zym*?+Vz*L~k=5$Atsm*&koCpZM^sFR_dwqaw`*|5*%W|@KC$257vTt}n-6nQ)H1k> z?7lrxLflgzVMCJrgSr#upfENYj`!%yrh_OtlPafFU+jBv*a_%GB_~Bzo+R&vpW?Kan{X(?T!6qLd22@gW|j zxh2OWcMxu3TWq?=$eDM0u}H6M@$5IHpC6&0_wLy4*o~@RCZ_r|oEc{GYjSNxqNX1h z2HxLout`u&p8sR=`ab9UJOFI$U^rvq*axh!ewsbth!!ycvQ%uo{(Y<#j=L#dF{WP~ zVWS{`U-R*~s{dHrJI#}JbO+k*AS$XUnQ9pdl$~_dp#AO{(;0-U7g&{rQx{I?+RjNx zWv(^;oQ>ur`~fNzR!-HWC!dcbQTWQSIjWmnvpSwR6)J$#EPyV!M(%=8gQH_sWAAj! zVYrxa7+Q1aK^rd63U$~|ogT~VwAqoa?PBGeEBI zt`CJ-BGMt7lE`$=qhc*;%Z9WL4{X00E7}(G#|J*b+)G@ekoa0g@<|d1?IX)q`OGX!9lvgF?y`s`g~pVG1WN zRzD!ZUCc!0G$xVgBogKO`>GDJlflgOZb2QBU<7bxx{hqyfjK4Ii?;d&bw)E$r^USwB4i^~P#mDcV~_rr;DzWnGavEQKWw*53KG)*@jjfF*M2yzZ;KhZ203-*Da z`DX^Il>bD{7qf;J=CP*VhP;w?s(x>>-N{iTpn_<|I*_rt2rQXE?({#YvOQuZF||r~ zh2ZN2hwKe~LzUd*>3{V2bow&$Bc_;Evd(I@5Q&w)kDQYcn+V{mmW3halS>p$Ed~Ar z8VnF}NyC>%wvksDo+o+e=^k|x-Js+%?sHB*7d;H(em(Jbww5*5S#1(ky%y8f z6GU3p1LOAtNOWo4{kH%&j6-2Gi>s-yzdQA88rkfYkdJ7={j@_RO$OMB% z3NvX&hg9^zt{|c-bUY#Oe^P&R25!{uoh;tP0jf&3`gLw#=}r+WSYhVIwn}q6F7--# zS64nHn$0NV0;L3@w;haRqed^14VA3j_QQvjV$?!kyh1VdA5z^`j56Qx6=$A3q-CB{ zbA5CKTT#v_kOEW;q)~z7Uax#;B~&(fycTAVPzAsvy7F z@_8V7ub0QxX?9D=Xlk;tqH>2T7S}|`TTnZNb4u6G;Q)|$OLjSU>3NhTCpl$dyIWE7 zhpu7;5n$LBam5d-Qq1r^uZw?Jbam)ap;sQiy6x)RTsbS;1Co*Nx+27|BYGKm0|n5Y zjk4u~e%-;-X>64{{;0Y*&TZ2Cg&;cs&$CuZvF;}iABM42f^bJgnnf}5`iK8w*ZsY+ zMh(b26B}LsbJ!Zi-XJf-FfX;Ww;HgN!WW=C5qQ6O==k5gYFjq=Q=5p`7jDs`FG~(y z5B1qkw;0|{HxGljLRcLDzkG~-A3cm1$u)-b0QvYGVz*Iz(*RIFRCrI2WDBL`)E;V) zR#*4DrM7a6{>plh4Nz)B^8Dt69uzOsOqzmg>0heY*!Rr z5u2f%CS_}OKsVjDE<5eat&2AdcZY=7qj|-dA@uOva(+ut)X04p)P2(0-Lf+EQqSV-jTMI#!ci4K!(}g(SG`dTK+0bwgc3c6^K(T}@By3< z(=k?0GmD(_c6Bs%{(0n(>Bt(xY2$ojbQGiHkQG`zf_1sx3%@)abu)Egf}sIrKrMd* zK{Pz$#QA7;1~sz}*s5(@oLiQ22A;4^aA|E`a^@InR*~YtvM5o=0Ak9+Sz5>#O8TFc_gyQV(|?Pqv?ih`~2-<2P0WV<(RuG zwNe%0sV|nDCi$DrrLw4y=)J2dwcaX5On&1v2sjGulUXz?4`Q>`8^>n>S)_z$j z{M<)*8p!h(!gkCc7)Q^2tNXS?q(u54X^ao_9sE~D@5Z}=2jhva)?5d}?-{=^phxvGZ0yGq673vSk{Ayi@7(aW3y(ZL1dN zYK!vc;5i5>un5E@dQsJayv6WnJ{adWRbEOtAu$pALTELVlQ`nbTp#XNO?Y`F&od(XZ7e>(yrl9k%d(iqR=YO1=t^3)D2^^3HX8|g23f*9` zI^@g`rKsQ)yC>r57IpFv&|1dFO?eZF;-;>>c6&#vk73&y0TAQ{@<$|64t5`B+1$l9 zI{d)Wz482r!=EYoX$c|J=V=k)2a(1T?rdK$eXfgtO)`;JfT(;Di@bYLFO(5P;ftQAbx|2)9tpOMz<78fSH1TiOACS zLV(E;&guKkE8tm9Ctw6J(h94SfDklyhdeKasFm>O>i63VVh)XlV~?R*YV{TM`T=w} zft2}wl!zL=$2L>R(m!jUzfufB0&P%$Cn`nm^KRJoFsSF4z0cv#{Zow$7j{7hL8PjMTt>{5k5sYQ60fX zX=>vRcL__odATMF@ch^qgyKjat5&`5qCa0Z?D&|$KZ3EsX<`N9S$c-T!Jx4KTW-X9 zG;lHX0Zu8Ip$osB=RAA~eb9kZAp#qlN-?4cpAMpsivR-de#>@Xip~&ZPDKqJ zNE&uuUHoeL0tHq9G6YXjjx`j!;|q|7QdBFWUbD%U8_$Ul?nn}2ho*jVLwG zSHM`-z1;|{@MwwUd;%;Y5d^0PIc4QoNItlM0>aHn3XWvgPAIU)c*e|(x_ z^ueL{;mp(PcEO!({z8thuxFz5c5E)`DfW7lsT27z@Q%Acqq_4f%~O6gK|5sEXw138 zNGY-O_5x+(%2?g+}S1jA4w z1l~=RIiI5|uYQCMrr$$oFFu`&oU%2)r)+ZtuaBm&qMSAG_imp>1JQPs>ij!Y)1r^Tcnzn(7Waz*00c|g6 z)U|K6S5?y3=0~9&5?h=6Lx_7ut*4(j7E@fQRK6$;7Kd)kU6q{6=F{C4v3mXXE+0zc zH@4*m7>==ZlW$TVt?gq0|E*nbJ&qLn6BCZi6=}lA0JVnnJUU#hjy%FH-(gJBc5bQ3 z`6L&cQ?is{5@)8m^<{8G22Uw8-owubDGMLByZ;YZ1T>IKevW8~tAQ44r)9vqQqLKQ z{f?rm=r)JV=7P1ywbu69l}ZDP=i$gGwX4G|3~}+)+dO6`L~>TodXD1^l(X&jsg3jwbuAbOvQm{q^;8TV+4N7d^*(J5};~xr#o3@V>Ord@UvRI#n<@Jzao4 z)-wEXAj6t2c0Pm8xN2;FGG1j8=*%U!A<&S#=Ak|{vT--Xh2asZ&X0h;Qaz2UpSoGa ziu{V|S>7(Q&x^1it*n|81AZPQa_JEwpR}N0(-X1O&a$|#50t3M6WRihDu+fYi#k`dlif>4Emr)Jg5+zeSh*8{ za$Fmdoc}0RcxqUQQ&!RY3GIFMHm~fPb^I*!B=+e@A^}|CC!*s6xg~Ovv*#nOv5CHB z`N_b7>7a4;z7j} zjq7JVLUJZ|kk52A>;V3cdr8d~EB9|BWf7bXRsH{QRenqCd2k`?Dju>S#;$t!$-J;S z&V{DVB98nG_InxLKm6kbx%)T11T`u+8Uly$AX-X+r2YW~75dh$TaT(@_(Gw@GLyt@tb6Yt$!pO2TXE!mchHYxsD)IhXO!>I1V?|U_l z%#o$q5)Z52;D8*c$h)2o`k^PIo9^a{Fp;S6GRTe9zy2uvcm6+nqip~H{vvGZJNFQM zA7}E*0_EmF*z$GWgn~7N#ZNA?-=)FNkPoC!X82*h#T&p%_OLyS&ETxf)1(N6F{?un z0MnyDfK1c)e)GB<8Bi=q+T=VH$S)XAu~x;y7mFoIbJrtrSEZ%V{*<_V+Txpd0t9;iCl_)2}p;Lqva(}#B=elMvg9sF( z2{voi|C*^GU1d^nJ991Tt?3-y)WiM9Y_ReKasD@2_@cJkcKL*sH(&um7{3E1fk8U9 zXH0jV31mx)wu5H%>YZC;4yN;Mkr+A|96w1hK^(~*GV&!BlV<0vj9|q~kapDbe2I3W z2E|EiVWMQ0nV?x=a&$B6g9_y^1-c=6a%#0F)Qoh(N7L)D(6Z z3k>>Q*v!{B9&4jjiO=L=arctB4OF8Aoyl)j#oOHq!IAsw%M$PFLq5Ha)Q{(aEGz=I z8zAJmH0s~Nn}sic@suvjvz(gj){!-(^3Vp=dquMcE-UPZn=1+}wOluJQ1~gY%@xqg zV}A&Hp*ckP)#ESAhIK`2Pf7_#+0$WXbL^tv zL&lw-JCl~5so3=4lhb{|3OTtCeO7U~1Pa&|mJ4P;VKJD@LE{7; z*5s%O&BT z=J6weS_2x@VW;7f9Y7lK?bxvv_q@ir4yu6xXKF)TPLf`;RXOL;Mg5m7;Wg^~`GjC! zcQBMax=m*hmd{f6_L0g6jIlIrq`RNb6XU@bVp1=LL9F6yf zJB`uS5R|?xp&p_LX`!mgsM(5XL8u~yvD#4zWwMI$l7&*x3lY_ro1O0NlgrsZ5z?L4 zY!_6`)#poH+3oUBrGFKkx;_C(5a)f3{LxWzm{W1Th1?`eL&U?=&U9EFu3r1xgA7fh z5WvB?lT5u)iO6`Gc>U3p2}1q066{$BP;E~V6hMwQdCHk_eCsJO^a zO0tRl3~O5%zXSQ%?^=BBy|J!rMT$t3($*G!0r@(??daft8L5M;4I9D$10-NogVKj9 z$fqP=LbYE!Q@Rd^)ZdYmEx@OqC`#0^oNS|%`XHxn_)A~7O4l?f2WeYm!w$nsFcB&- zL1M6%MZ#H^16Qm%G5@2IyRP>_+-ur6pt#`nWZPiE7&F)@F*lXo3f-=C?)6Y_^%5C& z!M_ifk@6&nuDUq>rB!3+2fi3|A8%1QQ7N_W!WzK*WX1E4=OjZ)mx9cWAWcrP9yUwK za@IG2oCSkg5}j3j_G7*gpQ{L3YWSe_1n2x2gBgI zrDvyvk(RO6%^@O0E+DJ*F$=WwCa_IGxyOmUx#~Df$J9*DbBo&pHAr=;8wMH|WscqJ zAr9Mpo!s4&I$}-An2bn|ANZaSlO6e;4jRkz2rbP3ibL!9L>XbyC+B9o*>>^#apTGO z5rhQ3NieusxG?HOvh4H97pAw?N972YEPQTt)Bf2|SySa<%&TUHJju83M8|Y zNtA->v{geM4yvRiVimuay?ef(PAZNLOG$ggQ(PPz;TTsn#cZsSzFt|lU&AT$=@r80 zUgEjA3h-BcK&?(0OyLfq5_n1|SSPXI;jFVTEmU=B<+6AJ=p_k2GxXgThyBu&;dk}M zQ>GYEU6BPHBpufkdh7#MFbz1awl=hqqDb&&S}02MwD(&SqSOYbT`XTCdV_sVpNuoy zPU_pTnKl+d*a>U@mp35=JzXayv zDqK#1fLMAa{1U6z*f#gai4J*pQfWYqo-8h}G}L|(x+sD`41M^q)b^pbZfS@$7&hZ< z+Xqf*OI_0(wLQ_-$4l;(R6TJ*-9j6_a098a70Ws>%m^QmDkI=|$c!A$>CVHA!|GPM z%qO#&j3s}-`Yy_^W2EwTo&lM74$Yj-oF<&sdVKoHdB=s%*M_5=0=tLdM`T#79W3g=C{TcB`b-7NAI?+FK=P%gJswIu%b;PTT()fJ zG=foCkJO6MauI+MZ9`?ZN(ZsM-)0sqZuE;6&7&Fkq@So&!y0S2lOD^2v0Q-7DWk>Y zoEa!2mZX}=(fn%p`qZir%qk5tj)wT%=Oz;cI6s!^t2+-8O?L65 zqR=IquTiv}oVNZFpS5}|oZsJ2_(c~t4cfa=Yfxnmbp{LSNCE4(r_F;4#fjSV4dbe4 zl+)5Jr$*$-!Z>qTIvo6lVpLg0PW)wu*HGeasrzk%;7ON0PWAy|_(bK63hM=1SH(>A zd_Ua;-??7Yb`}*>NahJd%L5&R;Y^}m2GKuPKfq%>&;X)6DZ4{8=*9~{MK!|B@tZ@O z4)B`DkLJ8^hx3h50Ex~I!=p_ao|(Jc%!QP~23fs2roi%g0(<<9y&20sf!Q$skIUJ3 z!*S>^=>G4yhs@&rLZ+S6u5U1XaWPGz+;aVDHrH{4a*k`Hvpy;AJA^Y#9(zEC*uGIT z_dD#s$(k5uIlc5ctm|$F_zQj0G5!pL32}-}dw&%tJ0mzDT2|Nu`zz2}-v}XX58+6{ zu~RFDW={y0tJHC(PGJWd^ztA0_Rl&_N;n$EN zs>Ya%?{#a4Sk{mJb7!Bqqr}mFcYNF~68SEdnWFeOz*`<_z9cT15<71^zmt2P6$b#p zq4%Za|9Ymn%SX-$=|J7I*+IGN01U|5ERJh3JcqkRjyK}>7i2n!sR4&{R;T%VM2oN zR77^P>`tE4Ppx2?N{#GI#1l;Yl9m@BQOyOja)E@ZUuDu(ojR{n!G12Fo`?hHV>_x^ z>gDmtWfVq1wSeU(E|f6wFJ#9p zFnBUnLZ@%Oqa8(HFpk%-TxEQoxAr((^#1A@Y~OwE2my|v#)*LUqH-eFa|y@FG7kO5ci@IplzB z%gDe3MkEmy|MvoMen*7H;jPwYUX;AXZ8}qGB(yKQT*l;TC}&hd<0pI+^z zmUi%TPF2kcdy91Y@B$s3A1_pvy^EB@s9-^Eyy5b1ER|?IO87V>X{!)U(CEahz}iv- zuKIIUF3Q+t5usOmnSp=V^9S%!=ZXU!o(~*MInHP!>vEGd_JLBR(&>l$14q2cNp2 ziQiAaY@gk)#|&KVo}QX(-Tg^&p+c>J=Pe5~go`a7NdJiqI=Eon#8>z|s)7MO!1GBY zANzP!U{yg!)G7G(s4=88kuKJ!c^iUy%ttpghb9Cu@96u5l|!CSMiwd>YRN2yg1Jmi||W zP{Fb!&F6IX)$|3)l@Iwf>io&@`#46Y*Jk8r`}1KH=i>-Adp+EU3{>A_P6V?OS*(Jo zZX&B+!CZu40q9RPNC*;AAP`EsrF!RV6o!n(Q_(TD0ZQK()Hz<#AX=v1&S2daQ%XcL zG?J-vlm$O<+9+BtmQR_v+TUMU<4A@7l)tZPdF({QGv%#?^N>0S(x3su<&~{{HPDRt z2d3%e`?k!R5?iB&N>7AW!3j*4o&?Y+a@Um{!R;#;cO`*@QYw;bY?Iyv2-Jg8Co&Aq z=8?-1^0k5ClE4*-tjV~VmMKM7yIOJURvjq|5@(ZTWRPGogp}Q>>8e(Ej-6}cf)v2O zJ*l<^|1ZkEsw=K0TzYUHoZ#+mAq?*BZoxgcyE_Ee;O_43?hq_kaCZsLH}Cn@`2lCm z&Aynm_D%QhdaA0cY~qU>lTOkUv|&^)LI9cfpdnqI>KKJ)si46i8+2|6w7SMqgzJRx zZt^UohIv9V+doZKXUo;sN;Pe}QJq&k-dc{C?&;_Ox5Cz;V^LmgtsQT-ldU#u1Cl$K z8=gnmSwc~c<1*Dij>{IF=ssm(!^6w7GQR6oK4_lK>>?PR1pUGxP07gY@GDPyD?^fC37o5Cu&ELzzH z1U+H*&dmtGVMbJegv^^gv!t-jKE`A44jsk4RG#r!6Feg}G;YDbN@_hRO*5{ngjs^5 z{6!frhkk-^R*fPJBh5ANH&0AVi2I!#fUEuRgn_mH%>)wD@F3J=;=)bU7~;9R z;bpP*ZBKB+?>4i>`ywSRdjUSAl|eX2j>_EU^wqWau*$#4i+{=xi6hE+UprQ}h<=v7 zR~VU6`CSZ+$#ty>3+>q78TZK&PJHj@i1bZ)r?;Y+IEbla|MIKj-HWVd7%;h`V-DIRdY8|(W6Z1IU9XuOl!J#ygsX~5XrYU`%MAc|D=6QK{*fH!_Q zwqU#BqWZDlGSyMpxVbKP{s1crV?}1s&c>9`YQ*zD-z7RTx!x+)Vv^kOpV5DN_G?aPLjoOI|%!mWD z_aoWmvsvHgdyaR}0z%c;y-{6~E4hW*=*eCdC)0MrNnT^?oWE_jn61Y(f}V3CnZDGMc36a>g9 z*E5`+ReqdDUEF!jORCvKltmn%`wGzok>X%M7llxFgS9}XGmu5=Rj+w==hb-kkv@hT z{I`tp@r^Z(AInwR-j6wJKc;npb!Fb|2)4Nr*sO&1CH5>5wcfZV=f$!Ze~~qfJ222r zl#|{Uuo^I44_Ie~ROhj&c!T{50Nmmi%Pm1NAb=u)`R`l_l!FXnU#HkvzyY>TNsLBP6F%KJ^q zR45E?Zbac$Ttyi8F*Muso7k@@@B-{WOQ~Bgw50^^-;MN->e9Q(3`X|;y^JGIUO(UA zF16R*i{KB$l$0Ia)_+*pzu%DFt}0_qSUuiK6TtRx7Z|Oxdi(^yYQt^!#nQwA9K%D< zajh9^@9w_8XU86Z@fE0@bRMAr_F83cE287Xh>F z5Q2O+fi7ia2gvohBZj)XYdS{UPS2XfmHHBY(PRWh(03g*_Loj|?rvI{gK09(Vh1#(^T@q@b+mgI!{pQANW}o0+qbw5rLQqS#qxeiwPHV$T6;>Ww2x zzbN$Xd6(i?%`7`>sk(#%A(J)cy=djJ5jeOSBOD`Km=kyf z;S&nmDy_D~KY(-Lw|wrVX))64s1T+wYg||F*pC-QE99q%f4vSlfW1 zf%kbm*kSr{+tR{yit*GI#3dgrtSGhnMk&4|HqRsdt1#04^L1NO-;>Jq;kGDAL4}lx zn2z%DoK_`(z`hGTM+iB+R3Y2zx3Z*i#Wf-nm5XYpXb--t@;?J=A%e}243EJ|HPhT= zl}cCaS*iMkyZt1^EpSvmKR^F{+V$?2UR=8XgCAc_7%cOo$~!(kURgTN*=;5@s&$6t z17$ZNP8Lg@CG{U0(u7e#6HP(kQAO>l?pvNTCCWm(fpAk3*2m$f9OhGfdj(tCWuX;8 zb*b4we#IBysMOCrl20jYElFK_k3oWDGDN_UUH5~AD?R+#=q7M1WR8T!7 zZXmeTURU$WWw0qOM~ESjp-j=E@(EqC=^x)Gx0B*Df%`d8R+NuH+A-2|unp(+qIG*H z60?8KsgCU&uOrGpy4Z@;2!Ah1e~#8_H%LcbDb(RJ6w&^5f{K2D zu3HrVKyB86#^pYtz=huFyH?tt#Shjnl0tK_Nr+b;az>n!;P0=q3K|yVG064*IPP9{ z0}lmAwDOq1ujZP@LP1OsR9q!1916>1nYDZhH#>Y#J+R^vGoz*F-L~(f)t^x87)9(p zlr3>;Kc1Ly5f*ke^|e1Ztm48I_jwz2|avg_FK*W;c5zoJay^_peW5a`CQ>DlwTDrRAtc4D5Ag zFfL!$UMJZPsMMecZ52hn*oZEwpn|N?_`a~ap|a*DeH_P_wP~B#I=JeVpxC8Uu zM5Md+O)XaBAYf6^z)NTX>BU{~TjLv^FQKi?$2H^Fo)YxT^bXl>qFE7vaXu~0FUvyq z5nV+C@r(RORmIQexvC?VYQ@p(3JOOP?)_sBPVd3&F(GcZOVLeIN15F`L|w6{9O6{b zFdP8~I%eM4>&5#LBcr>q#5pvCT?yu1`ZPizJ!XNR-9g1^oZoZWw*D@$-`Ljc1^?Fp z4A?81^XG_0+NwIw!7KwasULjloMasHRd|b^&9~z^)Z3X$Z6o=yx9U=rvyX4FztMf? zBW(i<(N%XY2&Js8mKs=Y_` z>uI*$QWFTdk(ha_$X=QPcb19|bl0H@ASY@YUJL2;U7Ge(NvFdago1ahXufAkBO&8I zGZv9u7aR(81?J2P$dm{IH6Kr5!t@afNsv;h-9oENa1lgkW2TOY#wXYl8Nd=aco(rM z3o)SvB<+sIOUoaozB|^L|0XV(v zu9U!98vkFCA;_Y3c>Yd&=c6`unOxXE1n**#(P!3A+i^YPeC})`5BhKhAS&CeH>AuH zL`)oqu{8<7*H)&4x?;vGhB@3XHw52L6dNsO&GGK5+;gNOgdTd|4p=0KJ@-$SDy5U4 zxRR;j_#bi8D!_zX1stcuVU6MT=f}TUo-;TZ!PSp%oAc`Nc+-{>LBbVLJRTN?BMMoLih}MFcqmr{ z-}}#{6z+vLwbq^Z#+;vuYfkJmMiyYjK#9Yw7Xd_&SCo{5tA^r0%#Md}VjY~)>^@a5 z({F=D7*pQMUti2g)ZV~%5P>pP75e}kW!*K``l7%fgVZRMWRT6B5Awu>-5(BE{5?+= zie)0VfPF&Zi=<@lncHuN=_axB^h2DI0@q3dRDdF*dGD<|Vm0dDyNPmMnrc=G0U1#N zJ2dgUNGLW-Yv2^-mTf=p0|ZVzaMfZR@Q_0XYlmyyCOGxSOctm~!Bu=ArUiroYe*X4 zJlR*M06$I%H;vgr>ym2-i#;}8%8cNPNc!Qu{kpZ%`{CLo8HK5KpO(i@9*A+)2Zj1J z@TI42lDX5N-1JMI)|I2fuL5 zh>M%l6b%%t{K)y!t$v4;M3xbbm0EQkA#!E2xee)*ov?E}{_c)aM z#A>D~!zeKh*rUCG@(7}WJ2qL2k1;%6q8Tc8rFNFsNcNz*VF%KG*Fz)w5vw671)d$I z9GDD4LkE;<3W-Q;8eB+{g-jB>XcFWe((}fV?3IN%G?iYby=}*Tw)MCC_bVUQ?XT9R zGKF#3-iN=26H&+uTW~(_o3BVcS6YnUr+CgoN)Vu;#kJOf=~{37wd#0$<_XUMj~d* z%IU?ZC+$&6S^@R?#(GBF*rQ+N791^3yvlgTB~$udw|y2O`cphy74W2pGtC@Pv}%jr z(X99o8%RaraYD=VI3&Dd>8+o_=g=U}4>U#~@X#gpF6*ag!%S9j9*Qd6b6S}yJ$jPZ zT18N!Jmyi5;;Um6TRKyrD4hEQu!Fml)u9s|f73_INTXGT5CxaI_uOS81_T9`yVU*Z zp!-WF4k7fozWwpIwY4Q;J9M}P6_$aIpoFd( zvWAD1`1cTJ+6r=?wga(;&fE;?S`Cf~;dTdML~C`n-{?5s3q9{5k|yV`N~Nld_yT_L zPZwpGl0$|%{RL>;CbHWdzU#SK)?dp&CyQuuR?M~#K({ucy~!lRWT1RMM9wMW*~3N; zZ52$uT-v$lK12~y!c5~^YIe(180FK?CxH#DChe?LsLWqbQza5S{nq_u5n`EQ_}YZM z{UaH!Y>g9Q733`%eCQG4wC0t?nR{GxcPjsSI!TUoZ8>FnFn_wIp108sCnWl;Ba&C_ zFputt!79KSO(H&RjB{0Q+PE{>kt?s|*mKjjxkVfptc^ zz-riFlh_iwv&ZJcxK(6@+2M;J0dNsReR68Yb%myssvTCw{l;)~@v0Qq>m%}B_lekT8i#?_Zn%P+u zmb1wX_4kiI9t-V)?`M(%&%OTbj|k5$GudUy*HROE6D02$3{Joef6wCchXdbDE7X^W z-!{qj?-tu`d<&R0xy2MIA4ktP{<6fkeiGyqHE*VjA*!RZSs2p}+3+O3k(k?`PqtQb zOowMV>jbr4UH|MZ1u;JRPPDk)Ed;B_5G+8l6q{!hOj!0_wm<~Y<7czl-AvIHTrvLU z8f2D9+G4Sdeq(t`@G&oxP9DXG=g>f!RGgKoQORFbHfFep5lT23ssrhPUYC)Q8Mw8d zKqOe$_yjXR(#V|#;UOh5_ws@gw5o$&{_b}7*3b5!u-2Cd03PkFZh+qNm9-(k~r zC7%W=fU~ddqcE9XFHz}Nq_imDi1q6V70C}=7m)W=ck6&?IyGP!`b`XJ;D|&|y&K(% zUIg@kaMc6OV~+k3th%O8N|HFIGF5HYaY7wcR`ZmvdJ&&z;EdG}aYB@;SYcC$vrh*l zECMM*3dK`iX({dhP-#%C+WWFp3sCB@5;v}jo)cCHbt{*ZTe-)H*Z3tYE^eIlO61T7 zp=Ft`{kvd?O$t})^X*7b{U0ob4(MhGlMG5lT2-9QxqOLEL=*aq6l0y683Om@!y3B{ z;_SM%tSsu%vbLEyS*V2zQMR`CP+*8W-}p&vGS- zoOvEb#>yn4yxKNQi?`^`kdy`Z-yM5 ze0kt5HJ9Mp;7sie*ZNzsE+Y8&hJ=eDmKtecqJ{Ekl$ZC2ZXP~euxvK82vedAX6Yy$_8RyIAQ zSnR6!Nyr7>-3j0^V+*|lk%EdRGMnbU?{{;*U9GkDy36RH{262T9xt*d3M>y)1M z#jbI1@t?rygnriBNsnaCsh5L{62jzG_rL7~?jKrM?F!u0P19`)oUJ&HZkGJ=G6zZ; z!i+xqQi|WXzVb}GCW(5k547aqBT4intyt8;Fk_ zCJz24)#$|B~RZ8eEhGi;K3T$d^{X*_R`I>BueSfOii!Y8O;t!t} zIWwd1R^5FfZ$AM0W24QHwuw9O6pSh#z@UH#qp$38|4Z-FaFuqWQb!pqM;A!JAb6h=iZc%sh*?>uSL^3|A-Ci3Uy)}CUU{EkPRel?ha^guNa<2Sb5bLHvu zCS~LB9mx0Lu@efC=P}KmTWp5~Riy-QZNz$!FHJX? zQ}738+{Q1p>=}wnI)Bmk$Zv=sIp+#8CsxC&9hmtG@;h4kZgE*k9#cH$--GF_&19`w)rj_mldnE;gKJmoj6WWC>t6 zKx(CNlx@RL3zTF-Yla|0P@NSw+VhO(k>5oXQV*KAuMI}@o|mpSUCr)bC`I(oUc0ry z8rw-barBjS@Slqv`o7jYKM^ud(w{C-H)3oi^2{?A^PdQ-BK@NW>gBPQxfuQsfc2b6o)b8(h*k8V;h_F z4ifys`zv#?_0HL5Fwu~l8jODdtbs3YW6QDxx93gzEx&>> zWXzG_sE>3Cd~{p&^ZTojd+BTJ1kOFUz%Lyu5>^{Q+Bin8FJ_?y=7nNtCT4LDBaYlH za_4?^H9FZEcRjKa_y=OU-wM~1)8^@I)$jf{k6pevD?u_#$CBPSB(D^OpMcfCsclhC z8-Nu}s-|2OYg%Ng<$R!^TfSJC5I943Q6unOw&PD1T?1UChO*2FECh~L!|ntou-EC1 z2XatGE806(=iTG{cmHsry|JfL$TJSLKDj`UHTo^GKTO6~JvJKyrMH$L8y#R>tDMH? z;UVVDgorS{ch=pJ#$Au7e*KR`9xabj$4%5fG3b9wibsuG32~zd>!K=0zN@AH?objP zq3`!AptIJZI0xK~hYm>Ik0q52EzJuLJN#WzMIZubNo>Xz*+hjUP<5ilgg-;nvSrfq z=akqgeQ%--d=4`##^4e?UhYo}wR(I&t|=`yd!vOtK>^^RKB7TJ#*RF+B^X@pK?LE? zemIhx})*62g!O9NOg9-EQrpFsvQ zW70%<#o}X=1q(~{Rg+748l?}DLL;aHBwS_wp>qEPn89~Z(YEFj0y+TZUZW@4oTF5d zZrzuzwG5Va3QFiSA>dwVj;SLYS_UCVWN2r*49GjNuEVdj%6}|FHn0QXU$sR zt2}Id{q~2w5zp{_kx99EO3y!QI{v>0o0?{D1525Xd5z$V`PhFCTq`=CF0yK-H533{ ze-{&+L^vm7NnN_K18x6bzyJIJA%i4T$P7Xy;`8{E7);pN&3Ua6S_v)3A+(WDpf@GS z_w@B)@Y_U`$RN=?=~u1E7tk`Y#-l*^S=GD>7+Y?G{0+T{c*x(rsb+)L6=b`KL1?mA z3S%Erv0H)W$bO4BV`e05Ok*1~3<8Q>=8bN4AQq<-qxJbb_1tLrywPFtiwebO`zY2?7wwy83Lf4V`6l>yvomk zh-zi>BGPo%)*Qp3sZp12wu!y#cD$Po>l9y?3*n9Q5;(?=?)ru-E7p!fXCb)Gcl+~0 z7@AdZ^F|pvF^?4fT*g{xSZJe&FBB?n_ifBE-4W<4IA%cU!IW=K6W;6wx&xCJtk#`m^(n>BzZg zol{H%!T+Q~T-zU4XYgKV9wA~32HUf&!U*L}yPon|L{dV-SH1{DGlkC5z_D{B-{9ih!1b778)aQr{*%astz}3CEds+0Zf#`lI3`b4 zf!|iosj)5J72W~OBRVsuvz^-2L0k8yVLMmzW zdi1ahIx;~zP2#&?maXV}TVMsoj1Y2EcLJ8?-=fW$8s=i>iDbGm;*=thjM2?}wec`z zQaBnE$6b86zo`(Y0n7Lpsz}ze0TY}heAUWmAX^ktz+zx%&6wUY)C1pL=%Kaz(i<&><}RvN{2P`zBN=8cjz zwxg11sjwKLuTs$(AlnyV6)lIh5Ra2Ew=OII!D<&+Vr%msa5&)(=ng;PHQFzM2scc- zK{kjxJ1EQgdm1X=Lhd*^&BsgV$vDlR$*9IfdYozG5xRT8+GQE+#P~74X7+r?_|~6G zf*C7ygu5!|GqnUFU9^T2#9 zZbpA=Zq4^f+)itf4~S4(zbSfN^n}2}3NIsEJfafOFf5xFrA9j(K*`C5A&@WAC#DaT zysTC+7NhJK2|Z^@kkwG@@mEG?jkZ5FcYe+CxzY2PLNSfc$Q*jfStcfuW|g zHs@KonCbY!7k`u=q9>1w_3ikx8+NLxzTVgYVffH z7+$2@uP|}051`RWBQp!SGWdKt5|zeJ9eCufcK3WQ##qF7Dfwtv7qs_U_nUP0T!TQO z&l_8cOhxvC%MFyCK!7uM!=z(Pf1pdEb}A+0e!Xg)ZD#OOsdQHamP&Xgo4itSaX8i8 zzrekp5<+gXJHt9_pHckrVFA%6VFcf?1Xd9MA(cRYx3L&xk@O>GfjN%*`ru7`aw z80}h$oXezH$+~US&e%cgu|%!IU@G&-iBXsxsFW}_A|Ph)Ygb;W z|3fh9wmpzs9HRe)B}>2+mem7+lSJU}ikIk16B~GPrAU(YRle^#M;JC{t{0FdCWvUmh1a^d|TVFrbV0Bh5;411=gT zMH|&R!mjmzp@)V(=*ts>Ku%9cm!RvjbjJ#NYFboKz2SNOO%AI_HcAtI4=&%n9Q9*oj>9dc;~31?gI`zlf-Sh>RiRmpTzKXevF;7+V5Fn; z#^<&YBm^D79mUSTq(dR>3W~0gz+6J>{6%^fShT8c4f6R|fDm=1amEp%D!B5X{A~#Z!u^^GCWVMR; z5l2WG4(Xk5PE+@e-G6!~ic~Z>+MLLi zj#5vZv_qN{MF$eW?CN*9DqeUI!Uh(n3YOLPYHl#wO?ra~kBp#{Z>r)7{0C!b24hPH zxX*JF8HZ*q)+^}HZL&-#VB!`MC~GrK7v^g*&@xDXMaWPSel70%OGCo``!+Hv53>DH zVcnK_K>}NjteL4;5>&-HGzajD8e5_F;uVrT+qe7|*wp`@31QJOV1-bS*e(ltf}QYd zsubqFQQBNx1N(yWW94M3D*+>13n{RT5g%{$g^nJd;Z6vb2n`iNkiwGqxGt@@V=TEn zbBXYC+%Rnt6V;`Xm8BNcdDYG!$-!-B48*ozn^O|NmXeBi9|rg(vHXCwsuc#9(bnS+ z(E{+9^CfU?XF%fG)SyM~~R!2acc83Ee?-t;zD78l#h0Jo*QdfR@l4r8PMyTA|*rV9Wgtgw%JY+y$dY?6ljT|Ocg4TNXVM0x~i!9o-`r8Cpg;rnBYo7A! zE*Y^|aLS8>qOEz-??d3GaC(`tlErbow(WCBWUY%9})qQKGk2#zu}&Nv=ucYhy9 zY(t)BtEqblhHslLm>(NbfWb5j3!>>YS)r34?SpI!IFNY-z&kLb2qU z$cO;8KCCmYDLYokMV%q;@LeL^ zKLVIA0IO)uo1EcwB&aj`QbRQ6@_h;qd!z!_I`dd&Jg?3 zgc#0Yoz+_LBq<=>B4V|4dMh+5pZc(YIh~CL~SZ{;T>G97G7DFt+ za0<00mXulf$d8qp_i1~-!soxEgoq>=dv-Tkv9U~yLi)b@p}-2wl}4#(`zMG$>iNIeireGGn~D{}C#&+A){ zT)4&6bIkvu7YbG!B&-(Yh;oY%)X3s#Y3&tg@%AkXG6whO$eWZa?y;sF%i#lASUC%c zewpJ=ND=y=OpGS9V`0EqGGK^S#`%1Y|Bbxxf5G}p94jP8S|O$^6p~gZdZCp3GpjtL zy6BD#LqB5N5#T}%W5Ib^0yRe*vENcxoF)x(T5yzQ4cWtk>HjU0=(AM`b`5V&F--a& z2&w%OO+|?0d8j_LQ@p(2X`rr%`9`fZMNKAC_Vq`SN>1wHrHl>>m3r;+m;L%H@%pUo zzi$__E7w@#L9@;lEIEJJe0}&M&^tnM;=H$2FkYWHFkLYP<#6dLQf_{Q_#1%sC|oKO zR!-}(+Ea?YIgiu_h7%FBUAhEWC#v(a7#BL=7fEZSIX61hc5i9d$V-{97+nsBCURy$ zvyCJ+H+(_no8fk;aD0qNYi+kGO-kb{oqfo$CWe|m+1cwCFh~`gF5C@${ty$*0%b7> zPx<+3gYNRiSENVAc?BMaY}PH#{BT+4he!BiU+cHn;#MWz$h4XK2+(Z zsKCev)INL@-X3}dwm#O#arl5U-lJGUg7vlVZmTC8F8>9q`?c}hp6TmnZaeQaTnBzS z-zV07<_5>=3)t!WbpUYRX%BG$yS#7=tZjM?O<>)I54}-JUNSJR3t{A}S4mWGS{=hl zdb|&7_0&DhFWTB>hxaRn^rj_QAWfi$veM5z9L@clqKevnAB;;T6BaRt37CCjxPvA5 zC~KV}1hxo({$~Uvu^BY_Pfh`KUB2x9C#R^^-#`FFIcVgFy1#T0d-j_Xat(v&Ti6*+ z*#c{f`$NVb`$`*}yV2xmiXsgMVFViDtAdX6F;Em1shLZSc}IBvw%_%CJ|I3eOlJ8W zn(`*U(WxiDezf`ZXRKgMZO965&5X47Aq=UTjp1oo9hQ1>4Lg=fKA*2)rnc~!xl{%& zCh;ihyWPcIMt{z)*O#yy%bGbSu1@b6fe?66~8VcoQVYu04?9tH9=r0#y@7*h|DvKW63$&v#+y5{Dj} zaAbO+uslO)df(B5>7vmM{6;uSroU@DgW#kP+_>lb+o+@bS#{Tlcrdg3_CxA?eU87G*IneVw$zgxyO zrL}TLPms(U_eWaS5>%Js7;rQgHqRYk^FAkvn$2MWv=>^C+H!;{!+sh{Ao7%equAz! zinRB;%cSR|38zQ~Fb`ca>54cOt9Zy{!grEjS?q%9qvI$#f@)PG+|40w;G<~&&3 zkc=zUB=!sSx@Fz26O3n|@|vZZkSBPe$`n>`%@JYMW=wC#oA8F#%M6=Osy~3^M0{BD(zrJ03Q% z2LT6ts|NvH3`EI`8Ao?PF%E*0-g(uG;bJnZvFLIlGop4|3wBYqrXbc9P`sy9SA1yX zkgOk1bBA{M6tNz~1)6q|^Q53go(*DyV<5-0J2gd|5cZIfLvdcBbKla>e*-X}aI#s0 zrcrhhD)i>I{lAAOLsu&UkA{%sJ!5fo9V{t>xJ>M4%Trzxxi+lLBHXBVE;yGOQVS)zi*&wUGVl4*XCMQDOy|G6ZT35Cjr$TKWlvri&n zxG?cL*}4Grba5DQg+6gM%HLbF>j^z_xcewZOhu~%ty0e57a^4yT;S}MroE}<;SK!X zeh9uKEIw|NjDD0rlD~On+b=7Raf+A(DoFnFGFa8w#9D_(vAWcBihUiMmq+NjXVK9-wUa=li zi>Iuk%~$Tix8?BeaXgDU5+F>0^9<8a$)dDxL>Zt$Z?Cfx28E0up#zb~9$#<=we8p{ zAl5<|4;X!{eN$Aln46h&nnaIX3`ruKMM$|%PDgZWeE;5H`*|B|`j?yvTu2+s`+s=g zRQLSr$NpB`rq6Zi_@ohIJf1CJH*&AdfhSTnz!oOeIr=rH%vJJ^W@&FGX}07^*NUd*=AH=HIB}&t@%JILjMLWV?toSR0QmP+zws zLhT1mlVjnV#1{t5WaKU3B-MUE^4$tUir+GZnc$;R?I_D1&znh99YTbrtmF|{bbsyJ z5>s2_;QtmKD-gMhBpiDc`2fIy);JC3=*^j>4S7S!;VgcHy67L0QA=-4FY@zV!j9YGFT=g7ArzGx|v@R^%w$QsVjnn}OQW97ShG2$

6Lct0sR;z`gm7d4u z3Vs!-X?qFa%z!%4Gn&sBS$I6H6n#KA&GoSReu5;udh&prZ788Ut-4oQel;Fc8g!c& z^`ix%VwYh*q*GWckkSrA7@iBz1~bI70N&(YuL__8EUo@-0PY}rQyAdvJGBO)RjdkZ zN~|a|TgVDjJ)i_08K0J130=j$^MVWG9uikiiP-159}8kR5!V(AfRRL_IPz`+5tfia zo{!b^OCt=6mCNWqV#3TJ;0tu@PNAU>ez?V%@3UW2K;h_iz-P-S@lFZ9%WYFMf`b zf}cby>mt2S)k_m_8nu4Or6I0Cdo8?^k6OpDRYNhf)TrOEK7ZOkeJp@l4g58`@xAlo zMhO^e7W_UdqV1mU#NFSX2Y5g4D|u^P9t3N<_FvRz)zCAzZO|@nE;~Nz1@%1kuzNe6 zCy?x4&deA-{2mZ1@;Y}D4*oi7!(TIHo6IvyZv3ai95?`0%-;6*#Sf{W*GAw<(W!3v zoQ4pBL~PLO$9J`NRx6oG_gZMc{*4?czOT3YQh(Qk&+-HvZ>SC8R;56`RJ|6bA#Y2P zjGqgo`W>^k&)8XgH^&f`BD5V*C9}55kjJ$Ubu%RKi28FAI+$1XrM<6BUK_g8-mfyv zmrjDz+NV>6Pkeh_f6!HKmg3K^4%K*c^f zQn2jRgp-fCFMA0yT^IH-6-$TJjXgB9sI+(UIUcsb^ci9?=sK@fBzfERKWSUWT>}!v zbD%gy0fhlB8(!iwqL}ZHvcFX5klFSI+PW^2Cl%aRI2co@UBDyhXqtzDx0&sF6&{6-u8Qz?e|fB%@KufHqv!|X$fE(UlY+# zm~v!?BY95JE%ezV&xrSkFpt-_jRhgf6qTwv5t6%2tZKqLq{Lp*`yjMoY-%oKDU^!4 z0FU~9~TXrKGwhpDlg1eDKAzQWzC$DAc?9ztc2Qs1wsmRC9*IIg!jtOx>1e; zR8|pP;GwJeyVHFDLtj%Qq^>jM?O$A;UKXDnqyS~;L|K;Zd$4JetQra+02VwRmTLX5 zdFTqrkw<|soY>f~h}em8eNw^lU~afaIhwqWQ`k#tP6shBSc%LPjh+<*5NbI+3Nx%r zONB~>GRcOrF?$=XW$wA^p)t4H&N*C4yt>7UtVbX*f|5MJ%Gq-D|50D=Q02b4U=IqA zPnPOQw|sU;(So*%pyvjbwp_`&b)65a-+Un-4IG^j7ksT8yVUA>nla(6-g1m{YIe7^ zQ0lm0X;q36gNocJFjHNpfo-nJY9jBzj^mQZ$xwChtUM;b$iXU;K6a%hD9 z4tP0STg=+icfTM0d>!?&PpRD=e|O--9wMCiW1PoL~ClU&#ANDC5%7}{N(A#v>O9WgTux7AR) zm0E;HmmX!>Qw?8KG6bw%yHmYZwmeg=ny3^E0*+d{IyzpiPs54tvLhgbZ3z+ZOVjev zMe^Sf@2Y0#iv7EHPJL6KIYv#a1j&THps(<6OtQx`q#IEMvJLw~@~7FN89P!@prsJ_ zL5f$0nrU|vMl>SEBe|GtOj^o0D_BR*mzyKNF=IqrORW4YHY)nb1qJ7VmYHC)#jt_a zPIHJ&-kK0VSL!*Oj=I~2I?!^cp0O(9VvYSKa?q#Eq0b;{|!TmDv?>ZSZoNx>I%uu$Rgk(kzHI$t1Ogn&h7I;k~ECaIpFgx>>}m@F;Ir9_+H zF<1wA6yzzYle@rTuSQ6d!W_MLtbPIilB38+3?4@lIXrfi88SVLQZ??hFcc!#V#Xjr z^c!*wBY3(Ad_4mUGcwV;`t^a>yMx(az03iyX%h=Mduh^?~m+*n~IdrRj_s@xvW#6!;q_GjA{x*X`) zHQ(IgudNK@jxK{d5aJ15oPPjsW3Sv)`Ms?=wPaQZ->0hCUB=<}?`_aU1M-Ez;b3XJ-*%Aq!|I_R4N~AWVrs8)!pDh&KO~CU=aE+T6a=GTZYs zK04KUsZFn@?rXii{p(%sCEo)Pi5&jo8qR|II?80=Q6v;kKl+a#JnCsjw=nlAQJ|h5 zIMvOPVZTnCgU#^?JC*WqGN~HtHp3>urn;HE1PGOFIWU_>2RCadg?%_PiMr{%wTtIl zyw%)q4NSGcl}?Afcfdc5z7uUhM(}a+Cls1USxvz3x+R4o_pPRfx)i_kBq}`^KG^kmuS%q zGDCU*0$(~mi248o8KBf3zS2!NZi&Oek|Rz`VIHjkvn`O zt$}!OSb>Ztc_q7=K=WFL@j&%Qe~88HlHkD<@lrGiUX0H0P1Ib)af zPde-1`%;e1T6v36!b+z{IEdt9&d~0Yuv@UvB99ej__+pyi(I<6!hK-1u- zRQ_T3c)mt1jtk;UOVkigTRy;WMPjnHWmh-KE8aglcs|NijJ2*{ZCy}F;i3mvv7#Ax z?1cU*nVd{z0#EA~i1M&J#9iZ7qRC*|R}h*kElSvg$rf+#RzZ3G{YOnZY;ZMXOhx zOMgFTcT+;AkJv2%k(;gK|}gFV1HbUEf8A4o`ge|_voZ<_i(XBt>X91 zCK(IV^DNT3j}SW(*;~sQ${+@TEgd16B54Fy`qflgMb5O)k?{-{{AupNyW&n;-vMt zyBziqxe4efM%G=A69!%rl8#`_8lB!s-Z%64GfV0|sC(ijkhu*>A7nI3`!=yO3Wso+ zfhQ)`K8OqwxP@SUas6D=DJiKN^uboRu@jK?eE)J-u#ZYehgQKgUz7)N*|JWhunRR6 zYQvjLLSzl85okihO zQ^VO38Q+#xS^+61eeTGnasGf)DCx)DDWe=cwndlnS4Tsd9*To1dRWNeTWfQxq!+4I z-;XIGQ45u0Ibb$|GjE#QY5LxN!k6T>Qb==34)&m1r}2zqUQ9g;k>QM>#y%%CVY3vA zvOPTm!+xNDA z438l5*9_iJAWb1h)Y|Ku`{!b-zT;-CUHd0XR;2p0W+!2Qndw(rw)pP0;~YXlqnYb$ zxWd&8^Z82=V`qsk7i} zt82q_aCdiicXxNEKq>B4q%Be;IKkcBwYU`v?k>gMp?F)I3GX+v)~xdblB|=n&)&CO z4=kOwL;v9(uv2FL&jO~@Dm=80%U zoi01wMLcl6QNPkQ^{h(TUA_K04c=-n^BKVP1nLX#xBDlb7opK_7OgsyXt zXyzZbx`a`>T%=}pu0<5c@!rluS1M`Y*RNqZF=)IA+D`yRNQ4Rr9-bWwHU|j^*%rvF zj3NmAmPSR2w1cxI0UEUL=whate3_L?GxZmW|AL?p)#SfqXzSeZy9E#os~FQRyA~3Y zb5qdO^}<){jV4%u2*_BaHj@e3biCq37g|%*^J#)Udp+xURXNiRjpae_`PX&Hl(XOi zX{u^;yQ4nC#!D6_%3^_<9e>TyL^ge8{c+4ni&kJhQepvUq>oSvChF*iL%|24d9jbu zqE*oO~9~?3M{8&}aNjk|Cb32~1eGTW&&lE977y$YaJ?Dg@5`C~N)L z~W#8>;p|+x_FJ>ST)FnfdOC8&GRL!XOndX zc5B5u&bjC>A`FJUQUl3yOu~mc)@90Pozmv%Y_ScFMM1_Yle5@$?%3Buj@-wU&z*R~ z0$4}15xyaGrl1(rZh}&843W^~fheO$ooWl&TSW|&Wz!lr&Ft%q-wm5GZi6nSJ{5^w zTz*QqeUq>?{EZ1l-jhI9uBk3jt5;3CecIR~-HB==_g`tVIwdxW#qI5ng$@}`y8@GkSq;9YCp>cU&PXuSJ|5|HP|FWvx zpKK71+RylR{fDVPL7(t{^VmL0)9N}g)rO>Wa(wBGTp!VuW#=B_USe~$$0qwbhvT5s zT3+EWwLeBG@Mo?#uOC$DX4+n(9d-8GxpV*Q9+Py7zd2VgZHl~Sp8xwF-CuWi--eZf zJSa)elEWJ=Fis}M!tu~2+iW~;OK>B!XIdw#C(aqqxM*BK^1z0WU0hnaexdVI&Okq-SBCu=5z}@=Kq376iGhZ@*5qI0;<^ z9RMe0_@nzEIaB5X1(Lu_1UJ!DXp8ee&7<-!xt{n?p#wRdn@G&u!!aboJ2HybxIBdN z4$_NM$;_yN8DR>6Wr)wbWe`j{g_$H5m=1@ziK{3mMnzBBPL*vYF`yTY0C%JwVNNLB z*Ht?-T@nvkxDHYn$)r}44tMKAVkoDVSqlaBql-i@1;T~_Y%D5qbeIimf-GRp7j!h> zs0JiE9Y!Qe28w)oQBgip)0H+h-a@nmgp9w$UIo)lYwMUHT3X)=vcpM~o!w}{bYRKj zF@Yg+1DW(vGBYp$D@;l~6{Vt5BQt1yyDKPxu=kSsNDA|aWY>V*?4hIqEf|O+!LEea z0|{JY^V(zkYEEQUfcq`kLKV;Vk^An#q_Im@g%5#~d-09E+wB%GVVbA_?iBu1Ya?Cb zpm=TW@T3BvNLlYeMs3CWYQl2#+rW_U0p(@Ia;NV1=f>awpg^@}z;p`6g8no^kkEFr zy}`Xq=Z0qoQ6^J|p@bGcsaxXDmk*^nTl%K#R*R3Ey7bJGKan**h^TMum8f`{bh|DE zxs|x7R=pUqsXum#ysZRlB~3JRH;zYnD$o6#D6Sq@7du#WYSJ&Nc3E>+YV1!uw@qS0N#s*dlS3A_ z30S(yDY)huUA*m`R<=()SZmOp*QT;H| zj!`>JcC1EUq(}{^s(J#YMOdht*iCA-^L(@p%m!)EfOJ|UHy?)PIlQ`p0xyqynf!_s z$HjWN=K%G0c2YZ5LS3H07oM=f=nsA+$10$XUlfAJdR4y{{d+{zKfDE)zAn$v9l_U&eA z;zXs5O2zg{&`4;^6j_L9d0i{UNYLA~&2DDlnfNsmW_%Ut)X8f8+HIy=p*(zus(vQ^CRtm1M2H+_&Qu{*`(d&Hvbr^=8w?j#3CgXr2qp(!)G}U{tBoC%CRgd*G)rpC_Rrhry zV@>p?p)qlGvu@1{BnjVeA=-CZnHk!=d|CPAIA50*OfsYB7NQW}{BnEwk6JPzPH*cZ z`}ahqK+?_#4lJx-7r6Fl#Ec&`VAuUBFbbzMdtS`Ix{XSOa0<~{;4>bFYT!rf`TFCP z-JkPEVa*H9DqZeuXxQlGw*`2^kLR+(Dgpa#bm~-GH@GXxr|nv5{uYUC_bx_aag z-ugoz$OY7Or{{!u*}mSqd$v80r5NLF8sglizcT;6XL|=@K6`(pkDX%u=pe+YFGxvD z+1-sG0j>|9rQl}?&e@)z%sAN(XWX)@)D$Nw;eLS0OAWqC(-W3|VmN>PK3W8mmSqsgI^jtvu$5tg_6>L`2+2JoHV>E){Uor$c3%2a>!v$fJYZ11T`Y4@m8zQq(tQogIa`(f3YC&=f z$F!E!LbBF3iR^+$*G5@=|1BovAPAZ`p5AU zv)|)6jcd>9>;}#ZOOHSIO;oj&a4R^|IhiAlndnN_M1Dm2Op1^XCrI*z*?de(PO+4J zioA6{{zV<)ym}92;0xw}`pu9F^AqVPf4ep&?zx$yovYh;@z4tGA^%EeEiG;=|4CTE zIJ@3L>UkENgr$T2P46dFBr~dV>`>xwTuz2B;{`f@8{c&6g>8m4_NZ3Q?;3lf(on0M zABg6e|GK#w*+i1SM4yUYKZlHdTU2`*pPKit>9|v!TY4$sdA)Q|8P?I$C%VbBV7-al z)x*2#OB5)W^&ukK3F|~>tv&!R^+G)`x0pjnJ{ni=jfB4Aiwyy!-bL1_oRk@Zxx0K5=)FzbAAvLZdfQF3W7>eoU-5Q_F}3 zF51=Q#1#VlQ$7C7DB=hbzA;VfvHQQ&#(V_YP*V% zm8Ebcj2+M)r3YbX)A9TTZ6b+Jf|Pry+D5bkQqtkhC1iRBGT) zWTi{O@bNSb1f&ejs7Ne$2eu)qC9`5G68b#%F(%D&ywQ-#)R9qRNHlX(WRCIb4n4?< z_CCl7Kp5g0OnQ)oF6~Yk$BCSn5WnX8UdpZSgSx&-@OUgDe7eCl(~%BQ@it5%_>6I# zVFrt4S)!4uJ|6gI7f9L?6B_n|V_!u#1_dj6v+ghBP=>9dYk;fzzhwMBQ4 zFdW#uR9Y8X`LBouMLKgLW1)w1rbDj-DfLIMr>@v;3n13sPD$#B`ap_9x@+mhndBp- z#Zv6K?N$n)2}cWz8e5TzKZPx=-3QC!wtDOM|6~{0TMxgc{&ak4#vNzvG>#g`$kY~s zY|*aO2`T$tH;zf|)~!D=p_v_hhmuI7VMToZO*I-GAuAA6A z=0iiIF&J-wK|(zm>H=H3`iNeXE+x(Vn8`mv=7u&YeAQ#@zv&@cLXm zud2}W5NPnrj2L5oH2qf!F}^UCU*V-(hME&Tx~l}CqfNTf6ZJRiJux4B$gEZ$z8~_x z=~`*3<=mRHhn&5-jlK`1iFZ0F#4_?A6zymQ65wiclVL>LlI{PUY&3K!bN&+T3huto zJ&-R@r=*ONd5x8X(y0u1-lK@vtN8o0^+x^?hnLQ{&{HZjzpYNgsKE*;W1;ZJsgb-x zm^NqrXFx9aL2|o*Eyw{HAc>Hl776<&cuq#Vj718s#Ewi8=!Q(dX&j{7j5uJI0tV@7 zsSV(E8#)d?$S)&Rpog8#Kmb$%TRvwsyv3`mv02hd#dh|@iZ|^c{LqNAG3Ek)A*dxn zHB=!?e(eAz?Omx@Be_sc?22&9=?94Qf^3t#?i1wdFS+q4R9_@GoZ za0tZR%!*8gU`C4WLV3#-n?e{GC0`+J3Pz8SUMIU<*;FzM64`t$Xr!x|e>vY)m0~i$ zs!@=~lLgk|m=-9^?_UR2ZrNJ>w}zzAtT z#YhX;H@OK)6sPi6E=FC%g*cy$hO=cgHA1S1=Nk*pvrAF>cW&f68RR*0^;%6$lrsI0!o zz+@Ir#O+R?Y?%EywaxV^4*R0&Ec(r~oApQ*0jn#bBo*m=Y!EZD%mL=ALtR2z@>fOF z#8f;iS%W*a8)&CelbT|6F0ns#gItNkJF>vUiWe*uv^0<5Fd%MBEHq|Ug*Xq4`S0@u z4`pQ+s6~mDd8W;9Q_uKX^?#%LPGHG_?f=WUz`N#qRj1ozl2Z!Z$ps0iM-CPyXKI2m ztRmD^4$F3bCJ?610<~O$R~vLgBCx$o7=qM*1$YJ07N^&sxVJ5gfcl=@A#rt>jo21$ z?oOy{Ka56rg;p(@bP;&X0>r-;STbS5kL{Pq^Ty%fG&BI#3nF2xVQKLKn6 zpTFFH;Cbe){kv}0E$P4+&mk+t`nzsIrze5k1f{;NHy_^7>y$@-pfJ9(N&n*NkUdMl zV`V*<@vi>hbdaDxs8RWmUYN7es^M^6k^p)hHceAEX{i|ZCu3qL-E`>OWm@~HV*WHC#>agp9zxjQ5>y_Tgsi22rNl3H* zIi2bKZg?tdHaH%ZR3OoOw5nho*;=Mu6nONDNznE_^8;>o?^Z-F5?tGyel*hYcpQ#0u zQSDb?oX1ebCi>&-nH?=0NU)g$4q_=Q zvvCarnRqG&>BKLR?>(l36p&=NZr>p+KDZ2=09PSW+l1iLs0%xscm?TE7C;!SosphY z{-8eS8{Qjx$9Z84v(ke0D`t`m(+~8m+uVPWn`fqPj}V+%cTtfHm7YDqR(YY3aE)}d zg6zc2NSPL}H{Y3wa1zfMjjF*eC$gAGom6N@$U!&U>oKAdgVru_O8n`(@ZzK_?}r)z z8Id)qg>7IsJqjkxE2CAnNJ4RI5gdg9ki1g zck-+I_`1yowzDyy(8nN{{KFm_&hXtHMH+aI>q(Ku7p2R!fm2N%W_ivtCJ@R_iuW6u zZZ}XBc%ekBx+D+Jgu}ny>W#x=@SEso(>+b*2zHUp5@j^ZTp6p&Zzo*ex&~j1+*3i= zHn5Z{Kw3l0!jj#Sq`b#nfH)r-*8`x2H7bZhniiX#X(xd`_UiP^DOVK9}}tT z8MP<2%;+vdTwr)w9_RVRg*89Nhfdmyzd~Xg?3S&`YmhmKqHg)|@>l1)WmqwTg6ph)Kx|j`SMv;pDA59j*?VXtXek%iS!3482Ima&AWlHsZDllsJ(?+ zMW!*O);0WhPvs!ylZw6~&Lm&Uh$UFFy0v5PL%=u;^d<=rZ8S2pKqT$wxYz1cl7&-e zrFH#D=4PMnLlx84V|J9IhJ6Bt3a_MOwP#XSJU8A!BQ4r~g_>hrp?e zNg@6&bc7kJD+}#K=vBl!9EYmnBk%l5sew2?YsvPldW{nCWv=i;z{%v@E zn1ltG0Su7x+0zJ&U7@%Dx=@mOxRkTvi2w{JIm8lWsyuIF{3CxRIyTE0?WjT|9D1{_ zvT{ZYw+IsVP?Mh^4qjWXjvqDvvLz=>W+W#r)N=koXTyCoofr2CpIz+c9j-@uRe5P| z2)7TG!BBsOQ2ouViEGXwui8v@hd@}f0{{RO9)KKXhcjkwQ3Gd$28MZn1e4^K2IS)+ zew@j1_+TM6Qv-Bo+wAIv3cA8nc$F`|kR~Ecw~Iu}5C?rd{f*glimcrs8I=#6J`>$2 zB<8rpN*B$HY6DM|X4FMgErhn!*14akQrAWe<$k;cp_`S%ajgTwk#gNy{U@*ay2%uZz`<7b18;#5+q48wT+ODT0~><;5zuX*tdd-u$J*zh zXN;2gmm=@Pv_K`B>ET1CHdD$lT;9vrM+v{bYZAAU|GEzc`nk)}Mj^lq*(R*r5ex~r zHik9L=W6|j(J6DWkK#uQA&5Cdyb-n}x*EP8#;ub@3{*Z09I1vKWh81JdfX*3?D>0wn-bbGIS?h&~hn;q4he++aM7QLa5np zAsCb&_58Nww);UM;Bk&_{*lG;C-bLIDVyxAWj#ydzp6|lG1-1K2#!W~Th$3=ttyhv zTtch^atdM_L`KL^ z7+_PTg&&2`qm0Bt9J_BcE?h|%C*5AgT$R4z|J*iU-&lZL#Kro%V%$PCV7cv7Y#y_(7kauDb6`I5u9et#-7uE&iU)s(D$KUzfPLAIrPh*R1MVn`0jjrMLM>)48 zpVIE>!aXJg|AuxsI+=LR8EkRNKO>L_tKrrJd2FApY_}%7yJo$Chc75{rD~||06nOm zP<*1)Y1We+TQ4rYy9se)P*@T}O7D16J>Ixzataq*DA3;^qkYQW4`O4KbEUZ8HRo0` zB1vqTOoL^C1tuzSPpFVwhd+K@n4zahilt0EJSJgYgkTf5w2jcG+qtH%*-Z>`U(rY({ zL-)o_{&JUe5>+b(BM)LMn~89-LaLgX+A!mYIe z4)8C?t-MJ6Qd|hGS%&kB&WIOs62(>&gHTRCN;}Z=vTW5|L$@!LK%}lXMHrbP;<%tO ze8YDyT?_q+`R%++0urnB?^x<74arIw@klaaO9}Q}`kmkwU{3pVcDTke9>_HT)ktQL zEXLXqeGK7{Ck#=#<(mqhlaN`lTL~XBoz(tIRyH?vMhmxcZ;@{w=&KHkqE^PL^SaSE z7g=w+fcIT-0r*15USz;1tc1&_uS$SC3kA_5fexM?|57e{Up8({TLQ2i5O zi?LJ-obH`{jWG7tdLZGyT$pH&J?gPGHpdYnGLHC|8__tW?611QV+T${ZJ+}xkJa?c zpT8jre(3>F)HwKdMA#XPP;OLow7;}oi<>E3RURf#$Wf7Sgi8mf26yI$$%VTlNVRXs zo^RX2E%cZw+C$VU;f6uYRs613hvVn22E=g1t51SeC;6yj*tshRyk&IzJ-4!HQe|ZQ z2XpH4`gq{ea$BH{CrK1WLYPuQcyM>1XM=tEGgM5nvE$eFhjSF}TDPGfB4v1^8*ALU zpk2CYAw8pDJmLnV3O-C^Zq=}8Bf}{un#{7Wl6L!_QVi7MT&>)Tuo_@S7-6pO9a@s+ znFr(TJtpiwJmHPzdK&H$ZNR8#7UFc4OXZTp=u0P209+_SW;9II(63P_f?1GQJmn^7 zJK4`wTg3f8fW7WwVk;#k!>pRo9IvB|mZxXi%~Lvc&;kF=$JBpN0yE=@ywS^&sU?Q1 zZADfXoookx%KyE1UnW7A)!T4W5CoA=DoKdGB;0kxwq6I&97p{@zuKUHIOrc_cj&dF z!+MP8N`XMA2V~HJ`&ZNyXh4C&rVigXfv5(UG*9ywCi67Ih2vMQ_-F^MShX2MugUSr z|KVW2ywtq5>+dRr4u;oFnWF6u?mtoUdZ3X)|EeK;LVF`MPABDNArqDwF@8HG5~r<; z<=ZlAcue%;FQdC+C&WOP{aUtQu@k}_2UCb!Ce@>^TPKGzGa;nfMjb+k%kvKJJ7r-m zPlOka;ze48M$JLnuL*8Cf#$;-V=z0WH?!-zBjtAff#k}m^igoQUxU(+^q-tUTaQ7p z8xnk1R`DoS;`99;^6ZBNe6(+`OhAKZBkju;-zF+m(7%lt=&rL#iUUIv{QDkHhUNp z+h_h2ak8VwcX27&sp}mVZTk%*(4x6i?w5vZdQ9K5D-Dit)YG;b9JQS^_ZRI4a4bt4 z-<2iX17IG{r${lJt!fxBYn2sAJr*x-szE8D8^Sa``M!`8IL5|Tu!Jjix8+7JEM%f@beZ&+fe(~ZwMTaxxq@tnT$*#;!} ze10e3TTi7=`jpcmR!8dPFBmWMGQ^TEsxPlY=K!RVS{kp$jxqMnk3R`B#hSR|Z8|X}%0dHJ`&x7X|myqm-z!eUw!RY2~kX zrLaq-hvOk`>Lul&++)F)m^MdnHha0flCPgT>*xlVh5Nm z@Fx(9MTJTUq?-+h*T`fxifZCllUAt+l2E$fNCR(c#05`+l}sZ$P_efk(P3Z!d6WJssT&tM6uaWwX+M`Ey6;{cyDN09;H;sE=f+nWNXt$tqD;Yvq2@)} zQRQnFZTQb(g+8;}_Xx_6Fv8VRPGkRs#!dCq6BQ3GWl_2}~qHaJuLS`$$|ENL#F;>4!*Mc|59C=7a-_udKCHG10QyUFvGz-+Mb( zPNa%FLL=dMQ(-XM#&{Y!351NK28_({_-9tUh+Lw4h_kwJs$IJV_2qYi z!hPNkuPRdyy#ZavgB?HAe4f?`&3SDBvi_KGO9SqLSpwQq+rI)0g-^bJNKOlG)iUb) z#bx2!*aco7+V__qQhC&Gaz$#Sd>>V+7D7#fjaRyJtsf94T|?#paFi0jfazZ}KYWA?ofVK6`fVkMP&T9hRAeA+_@5SX{-*vZPJhh?+%JB3#NMDPOoNDCrx!H)tsAJZKmcIZ_iA^%m+=+826{_ELvp4Ra6VcU; zsQ(bWBerIriptNUFE1|Z9F5%& z2#}r%n=YGNWeT3@ehdkg05uKGg~SCKGOF9Hu6=62=ldj-UtS#?d849q+&J|oKNxW~ zRJo2oxoQ!@>^@>7CQ&7Z$wCQHv*KBT;{o;+YzpF&EZ+B-)+f?k0F7&1Vj(a00Oq?V zQqRz&XzC-vCwzr-MSIzm96c}hG`tFS?mS1Tgi7SU?cBnOKI3ncK}2#NAEL+0Du0rG z)g0L&a-tl&XE^&1=a;;!8hXgCjI-Cl!M?8+DQ!<@oX0GQ!)Q!5`zAe$C^|>C~ zG$VnTE7O__$ne0U%jwO~C`ey?OXY#nR8oa|6r8CN+!&YC`jBNb?o9cM<3Qoiw4dp@vHaNJ^O z#*FI6qrnvbr2Ya}02l zH`?dij>f%g2RvSp1TS?3>tTpBKaPH1I`8)TBmR7mhKGRVmadilXmW}WN}=yR0L2$% z!P%G=V3(nS@h^Bckkh(@)n#|NxX;_@c^qo^9uf~?@?GidK3`Hg33O6%b2e5 zts!_SYhJ2_`S%i{$?=?q(+;%hoj%Ix;bnQVbL=YjW0xMQ%6IkU|2qQGfgkFSsT%@hMR9D9y5#4l&0AfCVs&rMS9tl)S6A&C<6px-94vtiHpB!q;J z+uCXIJ%pcYC(Op?lCfGdF%k^&@}V z=OcaL0b29mj=EC!$7e99gwH_NaV6Qn8c0Lo!G}b~UP7YziC_Lc?%-4feb-03Sm*tI z^r+zZqoYUQ&xWJ;-Osn1s?Tc;1=*~L_zp_Rw%%(>^tjS|MX8-|+Gr-B#J&4YUCTQF zaUNZ4j{c0UYS?O6d}Z(iExvp=Za|TS0vSeRK5>977;%tM2^AB|vuR(JdjT8d{}%3l z9R8#0uaTWDTpaWZWXMV1TqvQzb&^*XPLaKmx)E7*mZByUqrm6QXGrGsEL&wjgDkV& z7h1E|oKl|6;K(j%o8aQ|?e!8eRMc&(s9I4aaqF>WYYu3G#wEdrWVJl$wAGzMtLy6H zLkxuC&eg_7E){Y>YWHx$j#HEqojPVDgev(EhH}9SRb46$IkE?RGi&|w?z#E>+Ykik zDswLq_hv`c4OPKzN89$acKvn+e-fqE9tVZI>)Xe}ASNUYJ_Rck1F?mQ`ebfe|Ch%< zU}i}d%y?LIFRA_FAM@%WXqw0g%&K4CUjFT2UM47zYZ)dGkhXz8XG&93TO@1j!C;2T zCPr3j`S+;_H8&w_er`X&Uyb$BmQwY@D4r#3fZy)@g)hM*DxnA5S7+n())6)44kCY9 zG=0Gvz}3q-fpqr@GFSga>^npvNF}u}wUh?ItovX7^T&7+`f8|zDwcPlpkt8`wk8=` zLGHB;wJ*;=i(*H)q<;@72Eco;yYH*lH#f~lDiPV)kxQwHa;cq3);&ixp&@8rH@0)+ zfl8XN>#zk?vE%U!an*|tqB|E7)*>x8{if5zY>J{Ddj7lN?>D^~R=*VXUNJ@;AIm^s zI~m)}8JJnec*~u{b|Ls+)z{9g@>#_ivYynDB5@MPRdH&wMq?A=akmYZq;>C3G@4xJ zz1>o{Y<1I|Omf$3d_r5zQSXHerFE8LGxSFfbD!ZY3bkv^%BUe!vdr%<#hb7Ik^$6~ zQJ`T5Q`s`!3IXA=#(xUsFs{%T3{gbQIar`r0*?%#i}1=j(|txsnDV}Rqdjb%eP>lz zB&HDx#5&;SsOX&CWdMx=FA?s&ChCX3+ zllO?b!h|+~2cnXPLb?$BdDVyVb{zMLSRR&VBu&jHdGS8U>Y?Hu;q_mv_@f;FDYD>C zC@L|bw$u!F>=d;sttYCFVu_Uz5av=78>#a;tLGsrnY4u#_62%qrmhb!Ud!ErvNV3b zoQ?qyV7y7CN-k9@+Kh=~n zI)b>zBJd8bKnXZXnnaj(DBr$7Rx_1ie2_>5V>&#b72LI*Pj@uyWnVxg#O1oQAI~IS zCXp5p-k2KLSFQ^(^mkXWr@MMq0+O3d#~kq(Zc=h-1i*&g>*JEoe2xDuMflWTJdOU9 z{;WX;O{or>aPoEXH0LUCPAJI1G3x!r_kLYjBmlE{sF1A;(+Ll_CDON24KsH{#6}SW z{UX3~_yax7DVV5p?2$CxRGfW5c-=cCK#^|$A~ehe}3 zj=v#d!M(EUv4wStgVn@jlH+N9T|aLKtXqLP>u7`}x$v6SggG&R?SuP<^(@fo_h#L} zP-!84PW4A0=A-bdK)60P2Ax+$*8VcpQ1~<`7P)_>Gh{OfwlIqJ56>6E1kT`Z}?(?vZ(Sr2TXc9biDZt;XXTaKR1_O6*L(cvQ;h-uCMO^wNA3mb#ZY!Kw2*Gy)m34Xf~>skfB_9>3fR|coU;?^QSq4v`VXT z9CnO1a4C)ZSwyFY&_I`p{^0 zsOFVOTyYTv4$NFXF%z!#UvVU^aY#)%t5%Zdq^9DJzPju4a{hkN}KN0HbV0KOq);k`aS6vN(zc4 z?~gg-fJicQZt92+UPSm_8)wv9tH(XUeXq@sMg4!5*ACr`>c*sk^1_U2jdkj{Xm*aC zp%%UUiQI=`c?BTX&?)T3FZ8fFKdE%2%SV6W%fWVrZq zw+XMn(e#wy_wW7^pn3Cncg7{BRi1DuWKy`?;`qe!9yypFM;I~FUW0NRS#8mo*?L0o zq~*;A>;r!rAYaAnl@^Tf={M!1lpTD&`^s}{P-JLTXQI0R`%_3hJHqD%RwKi~OuQOO z6Tuu>^Dv!p(I4<`56L{sWW|PcwGMx58X5fePgx{hGB9#Hr{MW#_b}Iq^a&xwX-io~ zEu<(<>AOC@fTv%|VC8KgMR2zg%}xO(0bD1PL}_WM=0TM<`4jD3oTwffZKdz0xqPm- znQ(Dfv6BK>>mKud_&!Z(Ny$GaKEu~!T*U_rOZ@BYxZk(cJGm7KFwou3YFb}=DV}>t zHm~5vB$=t3X~&4hYCg7Fg#CeJY5=Ug3U26(LQlv6E(`o0YIxmvSV*gjJINL+c`4Jv zjvN()llP$>C#tYhC6;kYZL9f#L?Pq@H+QdhB^Yr~BUc)}m*VZ3f<|>6{uE@&S*)0x z5%p~rW;DOJdzFsT;EsCu9UGt_@xqHo#U>U=Z;-4_!^Horhr5uR7$2!{FQLoDOk{|Y z-(+l1CRG~*ogR{?=mQ5!yh6Ls0;J;0$<7WDf>l<+rdE=Ux$**4Y+XK2T^hUg!huy{ z+87DVZwkk@lY3s)dej&O5Q=Vg)J2EBeIbb@+RNtm?~6Gf3Dq^i8c?M5zJ}*hD6%g} z^$16z3+M4mA>C=o)=eFHzqb$Z0Gg+x?yHnx@*J3sM6C##AGkUcWTvd+VY9@*oe`(c z0pgM7-bP3U|MwzIgq>g*HTzJwgB{8pZoye=8#H!!x>+}@JAW;rzKHfRS`F2X&V-@j z^uBk2rsLFhiqIF1m*Zy#!{?;3*361(s0*c;d^@G^KeVm&9j_XjT6bz{72xt^K2Djh z=_Iou^`Jpu@@_wJcf$Jd25h*?6Czi7Ml}mte3q#%;4#-q<{8^I?O@h&z%7=CXnp8uJ!l%5l%ui3!9)``M2CJX@em;_WXveHGOe?spd)QO(S$A&3{~g@@q7KE z%wK@Wukm;WP^fjt+F^F5Q1$@qnf9dY4rABnZmTbvTwx#Pgz8 zH`}!jX~X4%-kav@F2^f%xMhN75*YIkEEMq9HX!UtoVxw3U*b7f{N*I?)qVz1t*RNA zZlS5p1~6|3CV71U2sQ1lw$XxPtd$Mrd#} zu=m@MG?JqMnnZ+zMOA09B0BE(V@RNSNJhy6bwg;4)YCD6z>pak8IvaH2ueHTa&lN+ z*u?kui#IHJu&UzCY@GicPToCibp)-@9Hr22I6|0c1zKk{cTEAU48A|%EP$1_!ID*<@u)l;QiPCwPzdzOR)T0)oz=ZY*fIq)FsHGrb9 z>A5(2A>nT*(nbn$qVXucZm~HE!ak(@*GeVO0K_{!ha~PJ))lAVST4*;DBkXe%}df+ zphF*SCEjiYdLAY-Q4T`zh}>q0I30rlgc{?f44;-vnS)f#q`Sa>86|ZNVCHpXZiTsQ zIsKMZ8Iq&DhmPHECy*qKw{DgkFV1qWA6d3|NxX$@uc%G5GfX~2NIi_a!B8EFxzBYO zqorZ-JROy=Lj44N!GMpa&YKEN2i_qk=uLp9>e(dk41Lz_fwj z--+MUcr8X8_=7Lc_jDC2tp!*-=;7mW8 z-niEd*!`G|YixbA>*%GHgPFk{u)vrvFH)ni-(C ze@=dmMjyQ5zWr^ZJ1JS~>s&?I^H}(1y<4!;{UJ3Kl^HiuD%{hU(2HT}3t-J+m<_|o zVWb4{*8v<3ygIx_F>m5L@^b z@qzIYL=UJrCSwV261)5@H4TIH80mN?OT68fA(QDLL%+lk6;Z@sy~v&?Y{i4!DU#&^ zNtK@<*|L|ET3_p=P9)y!EXiO(FN^ zc!S5JColYc^neJw8hg7;fFuEc1P+)u^3L6RH|K8ez>+#@iC4NJ@CmDQ(>@ZE*rReZI2CAe!Zylwc5Av-ejlJ%i+RD{WcC`i3U<7Gj^EizqP z4l?TZ<6hhPU4r}klVFu~SQakAZZ|K7ot(DjBBN<|9uBEj zoEfvlDu|XNtMoN+B{nvkvF`H9l5D)U=GXt(*&3;9`DHLmWIv|bnL2dt5BHXmudOLx z^sgq{n~l(4QEMkmWzMgmd2ismV$0NaMtAQ$O)Y#X8mW|UaiVB>yiK{?7fNXz`Gw>2 zn0n%s{Qq{1%4S@mkDA_{2FFA=TA?~5-DAj~84nBWpenzSpX301y!Fmx`ZdTjlp^SJ zI(8W!btt4%^QCIGr!^4osH+W=(zGk(bI;A-&@}#_mzWI1^TFQtc0i`@x;Y?t43b2I zW5B)-le0s+3u$#)fwq!PpS#f+Uw>X8Y?wjjr$iXsP7tRX4J5#$GTklEDoBuB;UCZq zovUq~=tiOIh0#5H=bFQ8u3|W+lwBjR9z)<;-3WD62|6Smn zcyL=quz_j8IP*B@8;cskM0ZlLUW(4Vj-zpu5gEGuM{<2=j%?WU@CN#Wyn9TZ!I{C1 zvxmUTQTb&?Cny#yltbKh?`_%|fPlqRY0MNy!2YSm+s<%HwudV5Ia$~f{LVV(F^22+ z%e_N7l`d%O*3$tZZE$sTs%>k|LM~vteD~t$4{NJq4=L z%8;gugAh_;WZqPV;a>$TFJ)6naw51C5yRH8gDDU;I_GPHn-|V*)kqEco zy*hK5sh&0Ay;VPF&`as|B6}w9x6w4I5%*=hV3!r1&;L_~n4v^)^3w8WZ?k@LH_M?< z>;BOIuz7nhaH6M1592JM4E}^CMkki6Q0R20$xm44tD^*A<5a8@=c-B50p^xEnys>c{~`fj&&vqiwE9==%2@@}5S@bkO&t&>IO(0y?{0z6a=0 zUWpU~9_PekjS`~IMR2fVhBPD%^k5yoV&9Png%PxlDvf~0d1e7o`NUss>%1W3o>7-T z*xc(QFX}?k}6;3TsBUpME-V z1eN(t=&RNEA6GDpEl{8~Feb;KqlU7gTOCeCO8RuaCY_S2vW^Exq03ifvjsr(2~!pp z3YI(=I@dardS7Lc*Yav~`cXXQJT3vL%8{j-Io0P#?xfbv}yRLaf+Q zm@O+I9-bM34hw;;+j`%J0f3u=@OY1z$VcpdP(#F(7F6w}gZScR6PH{cnSr}G2YRbm zIJW23Hs(5*;`bCsMqNP!Cgqa7(61Tx>rbCgGJIW5L`5T3n@_~*V&<0Htmf~w`GRRlpo3d4DL?lq zIWoN+Gc$9R2;6Kjs@;pJnofSGg-3@A3lvTb`K%{91(rn`8H=^3(WbLzs`rOwwZX^Y zg`T$Us?IBe`FBBm+88+fEA#J!u@_&yM~E+%H(b+Bm%D|bxAjd-_T)i+6*xFMRL9+) zvM3Xc*%xKS^s?z_tvPQn_4nIWFPbkja~DGeRbE5?FKVg3L_rzGZ;vI(`8zUbfhp=} zfrgK*NMBYkA=SVN$u?2c7rI#SUH-i~=dvA#%6&hU8!aFF4`1L=B+dH`pzO4Wk6x|? zb7rI-QqUUQP+szRc(pF_G$9(~kDyxuN@=(qUK25TjrR0d+>`6^9KeMlE$l_Y2_^UDm%$jL(so*Y)jaoA%} zw-W)k^eQychIe#SDP;45`7ndoo;#A zJpN53?T#Ng+&l1*pbO9-lFgb2I&D7v1I6gwhuM4%0{dL~>hlrFDj)*x0TGy_$TcO~ z%CA$W)G*0V3v1*#tWfC50nz_Jd$kZyv}Gb%;(Stdb)*@4{5-HQkx2u7F}80)WyLU7 zVdh_(*f72?4{(lys*}F4vR-hqsHxk=a+UhLxD_V`Y<;i}Z``3qy9+HMJm1#Klq*$O zF}NIcRaK;cva+#{!t?uR~ndpP$!>XEB#b zFIr6Jf4GmI((1;C<`k?KsHN`i!YT@BtNt$rMJfRHbSVnWnD8k#)R`@#J7dlUJY65g z7CNt;Yp{P2bYqa}pBa%}@22(umrh)(RMxkabMuMmfQB_ zlBXon3#?)AsLx?gLI@I-a!`xF+QyI?m6JPutE=%_g_l0e>s@|oWMnm$$QDf{F#xB5 zZn8e;Be^341n5Y)dVwDQ;>t?K>51!O2~^AwR>R(BR|321t3ryd%Z-}8@Uy>{Vx3`i z!m7yti&}a)iu|_i_Aw*za;TQlFj2&Sc{q;fdb(mG^%C;2Jh_Kis;5h2nx>FA0FV?E zj7A_}IMZN0U-G@gJBa!By%jm>j6?#FrkHT2PMILnG*F44nYeTv1wd+C#<)^~qA7qK zLwgpqR&hjLv(&s0y~1T$w`(NYCk&AV2CgGXf%*bx=+QtO2#d2=Ky1Cs%hfjwEZBKeNeREl(?%4y%lR)cRklQ7G!*=*KaR+VeIuTjNp6H>b=#K$A)nL&FFs~iB&`5f%nT49*^zXUUv z8v;SjIpFFSm%k?%SVYA{%WgB;IEhZ$|SIMH`%!27QBGyW>9`QvZo4o zw^8T`m5uM>@!!!C5%1Ac@XEZe+>d070HNow+_aw*v)bK(aB(RdP0e+AT~n4NIAU9B zD@RrE(uDj+ikW_YKFJD&QEB74e|=sC5qo<%WP2&y$|`09xakm=l(wa-U1Gb=TU2xJ zc5JsChP&U#mu=fWK5_7kE`!j6vh@lujI2vx9kwewE?$&1xcc?cAchpmj~hqF&4x*1 z54J4@P5(=>0;TfG92VR#xC>k>EBYMAiK=ePvg|`7;?OELiZ-;tloiMXwrw}t9xB+i z9#2(sH)HnUK=UMjk%xh`nJ`w?kZ35gRP)+Wk)>hsLJ>`?VXWVj)-rS0k*1i&W>-5u(~CV1OF{rYRcj z0evJpbBEiquze5$B)6E3d=5}$+P_=@8aKqvT+?m)ucx%)U1s}$4C7C&f27K|QH!6I zf=Z+)op{ScZe+a%i$gjrdK~ST#q0Ry zSgLTYH(7T&tuGC`aJl|{7`ghqG`%I780+eXL~{9$c5aM&%Q&}G!n2CT9g{iG1 zR_kawRsm280kX8zU+_%TN!3-rVz4=d*e_K3JG6@h*vdSrZ5UV=Jr>=MzP5c^Ebhok zts^EgDLj>M$PeSj6=fUx2P$+X4Pw(9=7SrK?LganX@`9>Qa!wS$MIy_EggOBS9QMu zMnwSHcc*fIrvK=9@f~J(&hU3lF!*d_ka^xd>uVx*3|wPAy)-!xKq@LPz+O65> zzN1FCdJlnhs-yqJyvJ&NOu_%np@Jboe>dGb6x|Q@rTHnv9`ye&=%-!3^fRVhN4-*> z^>z&W5m4ZL2UrD<<))^RYP=VNJ@F!^etLF#4O1y7(rfFzE&O|T2a;EWqrnQxa2OgjwyG-(x7Y6+YDre zQiJpx0RkFn8W5}c-UjiYJ$YuTGFyHjLvZLljTsF23~~Hf zkgBA^kVgc74!{y@%N4BF!hC)!z5|M8_$pyGla#FZnyd&F zw_jDT)HJg{&@Z$1@SBmkW-c^eSgE?Mo@hd3vlSl1a=h=~)fY?n$($MwJ~lSU@_i1> z(|_G{f5@hs<@-G*>zd0Lj~o7Ppt8#Yl@|=Z9m7yqSUa7|&e6F0-Q4XvrcSRx&TAH( z@WK!Nztu#XUs)#H(13P-S!Sl6JBKfGhS^6sv0qRy2jq+}x>>b1y^+xfR6)vPU7JA~ zVj!$n!i&xZe8iY$!Tv>PiY*SZOboL<`v^?HyZuL3H=dFjQz7|oG# zZ*R3@9{J&HOsM-sW8Z;R(=3c`d@xRJ83O8lHJQeMMSTy)1_I44L{v0phx1Oh^gm#} z=)bqdrl{#}?sQuSu8bl_h}OA$M@0GH<_5N8Ac9)%pO?~8nE@bHORoMKuDTG>n*p->EdrX`Mb*&)ZK=}YMhEY?XodFSPs?(g&ZbGqSK6phUGQ+xH`%R8^@O1e3ENoq2tSS@{4>EmfO5J zSJgYtor)5K!7FCc00w?F9rlSh_8Gkalkl z!aZWWXuLgls+WRzoS}lzKu}kU-~mz2p3mejO!k5FHbxs0d?<39xH5`{iZQPk<{lxB z(N;*MnVC!~DZ$fRa-85aQ!YJ4S!RlpfZ;*LV3K=~e3bn3PL%8Nh_nO@^>_w)5#c0= z>Bf@A-GU{02NKLz0eJ&(a?;)$Kjn-_kZ6LiIDCU7=&`(Eg@GZ;4mnyT2m4JrSIz(} zNy;}pNbJyl#b0XyDkG9A9vngFmWB}!a3X~0>h}V9@P?aywZtwXuywo%JIelZju2h= za1d?>7QY0{^uQ3BcXe8IVshP^ks$8(v+i)F@9G6W1v$B+$5|T9l7Gs2Qu1Nq> zI4DoVLxB~>l1iWjSCfbplE8Z&7_;s=JC~AvM2<@k!uMl5D%y8^2?O5>91;Pqa+Cuz zJO1wIyXeI)Cx$qtp-9$&O+_Uz=n5zo!F8VfJ!u)P;+-boiUBDf)BBgur)n%{dXDCb|riKK*Eh;z1T z`jbK%pu`=-mmngDcqh?lp4=``IqEF0=r9?o5$(Uaoq~J;*jf97MO!JHV*xaU)r;zp zC$_K9X`isvqERH6;&dg(Cv9_d^~7O+3AC7FX4%@e3AgDxQF0V@>t0GO1ywkO)+iw3 z5NrwT^IL+;^P$o8TWkaw%|d#_N0p$UM)m7DIY}gt({V(ggGQPeaA(ok6|U(M^p{E+ z;O0kCZ4TRs7)iir_xLe9I+Ju56&2fVCEb~ZMh_&p$LwKAWy=ys%$wU60xm}zWJ{UZ z+0h&KH$#(Wv zqhrrDdpp~57)8K8s459l%=W6Whhs5DTFd+wP2ws1g_PVGS?D@WZINSPS0?ZB;E#Ly z=ySg&k>tBfVXI^A>Sd~xh$*NbnK*P+I(Erk>4gqXLC>plG|$< zS85OVoHnN28YCQeM5!qEnxan!fDvcT-*Pr|pt@pcq}MXcF#6o0ZVOFc$3$q0%9+S{ z(NDH|`==)SiMi(?`5WcKG%^aw`>G=zot9-ED^)`Ard@(mm}Xx|pA7*fVpU$W4Kama`I_d6Oc2Yh2O~%6e?VZ)_w>;F`qbL>7uA z8O#Px6-w8k+sH;kG*+eR_1$9gG<-S7`4ziayTy5&5f*qzF+!r;-h4VBNT|An#M13f z3`XDJPs`1FOK;yGPfl$4{M6S&af%@-E0kyg(*IO?Jf2YJUza~UdD)8px2FGxF84@D z0LnfmnUEexG)@iKv)oQZR6noieQnaN>_3R8Ujz8_iJh4gt3uV&!vqzRw#!Bp#4BHQEJM1(BhL`K7-AJOI^NU+_a z{1`0D$#x10ysw|PXb=qmv-mZAJ|YAsc`1N{I4!YRuXVpcBBMq>c(K&w?MX&n$~;QM zFy^AZiW)GD`8@2Xke3K5E^&XQgV9VTFEN6~C-4i!w?dUSPh2+qipmLf0ZW-V1F?c2 z1DZZnCg6H32>Wn}V&r8Jns{Mj(+2&}>`x(uH1d);9cuvC!;ju@6?OrT07V@Xtb#N; z3ZB!4ti`<6xR7|xv2|FHKildw*#56Git{9-e?(1*Akl!JlB!iQHMR-yj9Fo%l9(uT z;$YVCe(wF?J>cKVYQQsGOR1l2EDHkGkX{5ekO1R|2RJ%#dg4LV))&L%nM0TW5hJoj z0WqMy+Mq057w)6vI+Ddf!6j@54+CiowCj1-V?22D3v7!8vV`xly%b28TH7vgR4nR? zz{Lv2@g$K3I2YS%@(T<=adZN~G&QEAPuZc1glEyUy>im^U9-{NAJeLYTXee)>`xoo4WqfDH_S{&J7)M8x zqTu|65kvkkq-lgnM#)4;sVWQrg@MDZpCld~xU&#tQI6?vRMn<6q0UJ<(2BCeQ4yaI z<^YpNobABKp!|XV1k8#A<8Ij^_C4u&oijS7ic=1Y1Xlph==vyP9F~@fydlL*j2o(7 z8v$3qOq)Jv&^bT__=p*U=nFEB4qGrlSi>lR!xGpy&i zaJj6%o^ypiXP?#2w{lT{kfe`2u<7169(LdxSjTuY#s>DzQeeN<+}DA$3FlFusQbtp zlf2#vvY}xW2}N=1Z%jom66C0F=AYr|nac7|G&e#f!VE#95<$cK?Ga`~K|QX|B;5Bd zhQ>5o)Apbh*YkRgFnv67(Qtyw&OKv%lTK9lZZkDJAt`$ zQtJ=gga|i37oyCBm@$~JT8x6d7FPE4+_W^n{a8W~s@&rdQY_(8rmTnKrWsKL8C~~t z#JA@N;uq>=cyVyfCbCvq{GUSO+r2oD=!Iw!8u6^R_&kJ2eXmSe?sT!SvFo>%^STJh zwCR43;NXw&&E&N_i*K#TOMP{VAC4dw6in zJq}5^+H#v7Kbv`BP$)%~h$7U5^ghi#8x>I)dMntOd%C#6#O#EI!s&RZR zO{1p5)D1^g&_|6-KB|UOr*-!tURngTnIQMp7M*>?eUvp$@A;83Uz#Qn*SS=QHu_8t zxBO7=Tm7#wy8_hcqDyhyWc(ka@XaY?k@Os!98wfqA&Hti7U39=6$R!FS8W;Yex)?) zIO-lvEbBTCTJ7(boywoZjhSV`D0D|?bD|S6lh2RF;_@kHJ<=h`|NOZL>X}<^b$yll zw!SzzzpVQP#S&$;ZvaKWBKT~OPMXP1G|qV2gBAo*qh=0@@bh6i)u!Wf>aD9`=h)_O z5^ahm1dwb><5r8_>jJ&$a1g*iDzOI5RVEldAPCk;i=`}34%Ap6?uUa8<{ed!p-g#$ z1o;dl8Zdne@Mg43YeBt#^L`v?kVo%^Pxj*)D7-+`zSaXO=<&!YH8aTpFFigT zms}Y5_{$KpzAe+4Ljdgu!|>xzJv|!jHgN6fQyr3}g2Q5uB9av%P!_+|Hra$o@Fchz zaOVt7-e|C|0@=IK^$%3WHsHE1M0v6MDR*d1GeP>@q#%|bBJMkrl2>u|5qJhk@Ft|@ z9x-|F0gZ5q7Qza}8Opjz{XNWSI5R*Akp!M7(<8{9WV+IV^5-R#&P6~W-5WD7zgTEd zmA8M;26$0`d#P#OLgL=(G4RhySF1xrh=8$*cmO}kBQ*=2ONIGU_xy{#dKtcB06u{)Y)1L?8?~cJAaRtW8>0 zc4`8O0+LqAXAhxi-2NBG6}fVq#_393x^E?wC3P zEZ7hOt%Mo#6aOI~S*{$6Ipb}e64a4kI+kYJ5UtvojEoGt`_)P+7jebU1U6@&2(HB2 z-LPXp0hD9&74LXkZX04m9U%)^vbf|lX+oA?>Uq%8c5cJwx@+DhEZY*kDkr_>N>pP) z-O;17&*;UJQuO8AUmTqj_lWV2h&mh=0y=ZK*?8t)?LBd2i{E3a7cS)P!t8lgoqzo% z;_Rqx$i2%w2O<4mv}t-c-|P->2M#6Y?mnSQgi?pDVMY*@^M$$gk*+q|v2A#YyBTZ! ze%@ZDrDha?-$)|KBjTl(01qFCjO^xt!6`!Tq$2=fz5OIer1LS~=459_26@DQRn!57 z(B#7`EJ}yPCkpN7O&ZB%NS{O7oMO2x^&G+0KbI#=VF4(yW&a|fL8Mn8bxR)PbD!j1ZdYKedd~g z1R;ygQT%3+1wCfP(bFL-6x#@&6+i`3Q74jiIGSS@aVX`^tgGC-9KmGRJ<1XU^vrV+ zA-WmZ81MI}5~S#p58X=+AxHR!wISc`%mDhs6^LTEn3+FLE~)xp_ZrLViYDA7<0mTp zTWMCv@d6Gzh+Ha1Raz_bSc^eR9#G=l4>X>gSCYvsN_ff&+d`OHX@&uY`xkc!3!`qgQn&P{O;EG5Jd2yx1Yf zajTDU(8~v1E+oXeqLdf+x+p&ip8KZD1=6{kDYAo_sB3*O5eUN>FUIdPer={c%)SQ( z2Ek^|GE?Ss!ixaj0;O-Q@dq*LZh>)-YtjiJ9ic|>u|n|}{|c{6Um$o-{>qpMI86Cz zoS@@@Iz4<_K*GgG=ee_!w8SA*3Q!+rQj?5f&HQsWkZvA`43CLEf!5~-xU#p2 ze|#?ZWe=up`z~c-y*5&zwF;MXrcAt=XI&K2 z>d9o#pVH`M@2EcT|D1iy_dv=Yhqb!Ir!hsvt9_FOLQNy9H*F{_M0JwVRyTCJWwKh${?uhQ=&S}Q zDHmm#K)8Eiao#12;zAiaxFT8WU{-BHv~*pc259tUPSL?Z=OiK%=f|<~ErrISkz*cs zBBssjCZyAXxMB*=p3t+f=m0!%*hf%vI=T~Ro*JP3FdgL`h(_oYS|Cz|*cLFl0BZ;| zay&`R89;BML+N}vpeTNbwUP(85o}J>GIyxOUOzQAO@2hjhbLNpEqETIzFPiUnjcLD zY%yl`&fsiU;Ie?dv?oAQ4^zd&21f>ZVyDn)W-zl1{gjm88i7BFqCFZjuw{hbU1r)5 zjHCwxP>ATFS`WdFl2`>8c1p|9qwb%hk2LVo503|ZwOr0rRIn4H;VBWVI;myVi(kc` z0OBr>8_Y!uK>*{&?YgpSCpE!GQNbGUv+MDEhLaA#AOUvX*8g_+h0y17h<-3RN(Q3f z_dn2F-y#Nv;}%p>nAg@wZBA6= zldmG216+vk^_l7Lf!w*!5A16GF3wmGrCh~VdCYwgFqU|FvN!Ew`m4^T9t6^j=Ij!1 z3lM6S`yj9jNE8M!ITCajL;6SwdtHdW8P(xNVPM!jc-SPz*a4P`lqXNyF%mV4C8qH4 zMh#J+Kz3kXralxi=r=tgqAD>XoS2PkK(qlf1Quxo&`Mv&XPja1oWyqp`^LbEu|H3K z!-wgWVxy{e-!sI63L%Y#o`a_L@1FoET$&KN*g9b{z<;)+5luE3Ziv8xa8)&yBs$!8 zEJyT>?)W*@EVq1x;ev85T0tQ$)Z_rWREvI-E0;nHYcDT|=*!E7 z1+zF9OkGwtB10=IdAiNlV?kVzV3Dka<}a!(F}SAWmk2F~UHjq9+AmYaSZWCAl;9%f zLM2WMT7b5t?nTdV{k?uCnmk##W)leM~X6r#b$tVrI`;+cM1Uvj0W)?X;m zZz2W0MOssQ{CPsnc&Wr&@%XulRTY=oevPZ%M}0rQ&$f1ha2{>h3-0R!>3pkYuNA5+ z1)`p}U?eUa8=_z8mBI5$EjIalE5e}zz%6~uI)zS z>HV)5&&`*ku196^TI4uRzF#mArMg!2J-paW5m%Bw(h(}Eq)Rkl0 zBi1noYdEO;ZtwWNqc)4gpC1+yYn=zg(!)msq&V>_1_B|Iz?8}?w&G&siL-07e!h7McMLtD6MF)B5#?>pX|SS z=x2#8+&J)u*HOVRY|@tUPp*a0I9~G>HTu&(HGjRen<^{-Af0Q??Q>)gA6?ddl%UW& z2u(FX8%$@9M20h+8N41STc7s;wP`Ryyx6Ts-~ruCTnI5nMX;WPJGM4$IcL*_KBFua zWfcc#qf^Mu4CNA_XI%E+yBXJ%J=W9WGv*2R5DZX~by#o>Tcb7smj!dO!N@QY0iY@l zIh4m@UpY^emULs%GT_wG^83l1eLC2hXrSxD1U)bE17z%@X1CavFRIEVYkvcRuNF}7 z4C@w3QR!Y@(3lo;*OG2xm2&akJPkwog%D}P7yyNcOT`ntn08-nd8g= z=W_`v`%N2^H6s;P^dSNcyBzRc9Vz6JI-q@=%*T1_`Ph!99N0eU5G5itsB+22-s|?Z zE!Z+F!RYk3Hj(|E9P2H|AVVBsk#0c29XJL`#K%Y()^#HN;E-Anf4C)2>C&d0JZWEj zPhp=2PO*qk78uDV{6JE@*TPuoB)w>;LO?7QFej4p8>S}PC^i({?Fdkh&hy+$-w_Lu zp!XmYk`Ux0R1btbQN_aijexO__^j``}N*ex}wMwxi3@diL4qvB)z$hQOdbJVM22phLJPgE9HlVioTCkkD73XmiK`}f^jj( za8Z{q?{3hY`IQ9gVT8>BKkx4o#g6HJqK725$b!aK+r^UkAbk|^^_0{eOevr6LT^S0 ze1b!k;x;Cr(0&Y}?L zJ9=_NnAcM^s8n)ua}V1CShkcZPoHwwz`b_g;f4eAw_IRi0$|pAEjUNRKJl7NOy4e4 zQbmhoxOXauc8dF}VDTMsrnAKZ8@{htr`7 zI6W-)$2d)aw9_!je(KGQj$EX;f3FoDfVVK{;4=ULO<|*cmQNnsEvwZtyY?MF!@e$Ph27TIL6UuO)Y zO^CG@0s|@^!y3}xL#Pc!w9hzp1JUb-7HW%dZ9QxI<-R&)0y6cZFr1P@k{li~3M2dE z@g;A<^LcN-Gx4XOaxiZ7O1X+Qz3DsXMPh8=u_kT|OTQkoeRdZ*soRa^FkPW~PEm(C zUY8G2Qf*@V^3FtgqpISK!3e{d@rlmgk=D-k_3AKh2q!<61#tDpir<<}kvpz>9-QV+Z)0RUn z!?D{I;~~$(#OB(*Gev3;|V1N?oK_7NS4E(VC3yqEK_OMc-4r za5wjsTZXke4k{v>i_#k`YaG|D*Sp{8**9I@Iw`ka%O4YI8&dNIj#y=zZ){Fo0+;mY z^dVp~-3JgcDE!TdJ%Np2V3}!5P+>>8oETZ8eqiKMM&gl?6e~V-Qs!8C4GH#FmC2HJ zlVBEG$^W<$AP<3^L~h0W8xS6?oe+l$w87ftFO5>b;0nBck0nncA?5Vu+)Q+So%I}45fx=EdU4Z<_h2pw1>IxKxppcRRW3w26X zIJ0!Ou<0905>QL?vr@v+W+F7UMd_L)B~F3wQ8WU!1h!v2L5f6(=4{5`qA^KZ>*>vo zT)`g_G+4P1c4YQk%jv!-P{sAWZ?jJAbotPDQEZ@wEN1gcGae^wrxDzLqeVwsiy&$! zV@hVOV~wM64jK9$Ir|rzNTqr-w5s=)>Wmw{Ba@pflHL%>9jl3oNyu=lM}pO%^=p3{ zw3jSXQ}_|7pJY4~SK@YR#9(q6VK0f%jXx?Z_n82u+Rd%OPQ}+N{pQO-&ENZrb=zp* z0`Rx%SzR#Zz}}o?ty<&73@WpydMC81L2AuFpZoh?tJX+bz*xbQ30sRO!irLEM_s+}S-V}LI?O!vU1vDnR`#>B!CpHFSoL9-EEjv|v@OILuFWD7YQ6J7<&#jr) zQOTZywt7g~a>fE9`r-$bI*uA6STvF_8+r#e$Novu8_&r#7ptyGbSTAA;mlm;D@@n8 z({jz$?ficD_7^#S3^!)%u@7q;GHkZR~gP6CSep!TCKbIAOg$7Zu>2*RIl|ncKE)< zgyO9!qh^ow-Uq4WQNr495R8QcmQMK$hGVliaXLe)8Wqe8m(a%Idpk|L+IkP%mG-vH zXf=o*5D@Zn+wl&cS#Ns0D~QQ8Ob9bikcltM*etGoDEZ9GG&`?jbMyi~>ES#TOHGoT z?7?3@BePj?ei$b|VXPsaTLbVR^TABq_oBD@{$OZ46Id9o0yE59G^y3pxBPruF_hyj z#Gk()>Ue=W!OH{6S#JAm!`3R*HV^>wA|2OcoZlAAZ*bX6MDMy}?cCu#8O%Bot_v*Z zG50=s&Acf(P9m_YvOc+Kgi-9vM;MMkMb~m@!k;*p(y=n^VPi&H{<~JQY3Jz+pBwmn zX&!;CY>41emnbXtesJptwmf{JIen@`K+;@2yivr_DqGKswv(4228QWV`bM}}&tQaN z5(-E-z>}Rm>#M)lK`=yYEVzVTY&#!5abxu)n!vxl)Dmk&SY-~)ZYnY`OXPgNo^;P& zy|vtkq7kukp}A|HyF9-$HQUlFnFb&9V353kwEk;$nUMDB?(Hf#Y|Z4YX+KJ!2uU7% z7d&sht!!6bv=`*?mG9vXzZyOFKy@UxFlN&%_xB7q{r2d=WE=+)TcYY#Pq(w?EAcyL z(Nz8YN^LCV3~r9gZgnqhHE+F5r_KFtjd_Zf zRmS8-Z}1O84rP5{EU?kz*) z=Yg$RC#NG7w$8VdO6>z5${U(5VGCSyT@FV{gl-bL0^hTX^7qTd=to|&;C@XOE7m3h~#7c(d; z-P_%n``byT&D>OvrxaRpYYwc9va4ZCV$&GaJ_n2C#6jAvwr)^6I5!egL=#SxKnGDS z;<<5I&eI^|W#vyC26`&BbE-PDevZfQ*_6y9|K&jjAZiW9=xVo3(2gyy`vuOUoGfEc z3_$@7cT2AtX6g5@kC)6{JLeQbb<}`CKrAr)@V4B`U*)riNIH+fiSv0U-qrX?a0_;q z&CxSHU&jOK)_`HdXE2vgDm2TfAoTe7_kc(p|7j7I7*lh6kni0@A-x&XUwhFx+_Ysl zAZ!@Mvo^t-qf-m9-EX+)+D;L815+^m+1TEGhx#~(59qFyt4tqb|D$Ol0jPjQr#cjY zRiC_`%>!#Q=hp+|V*yJYx(%i=rqd@;JTV`IlaGhutut~nKli16dB6Q!UmZs>DAHi$ z++LtQEV9}&_$>@O>z#K1o`JstY-(cJ#B5{4ka~0BypIT$JHK z2#n$M$_-8n)v!ldNFCxCrryM-SBy=xM{Wuir~5hKrs0rVP%H^Z@{sFMLl6J~P$l-y zWZ9dP!f>x2QLuT~av!CxFOoBUgR$g7b2YHxk^9A7syMp2qCH8z*pG5LY!YDZz~mi6 z8}p#-imj!dadapXFMgv+@a1PbSk-0ZN@RnmMgZquv~rCxUC_7MXn%oOn)5znjh=gV zTmA!U(5m{AWxn^o@aL%_PSSWsWOMAWaP;dRs3@~Wr4mrImI~~VZ%qZYUL?JE!+*W6 z|L=U4ltANzO+I5LoSZJf#=w^FRZ3JFyZJgD@3*w?p68>Vn@=`hN7KIk%vL#GUeh@{ zF_ks1J>zmeyX;JQm?jKxtq=E|b-T>p*gsL$GbJ{*^n(;U95Q4ru{tjXf}TFmyUyw( zbuu{{(!|q<>H*<4pO@Lqo7dcCoc$G>P~?NZ$$Ez8<9-b+nGw_fL_Q-oa33h5uz2_(Y(MQ3_aKkjr~Z@xbEu12%*R)GF6a{gf^?>fod zLRa3U`Ay*|$~pn>(|wv>3OB`jhRS#MH`j@QpeJN}R6()S4|i(7dAX|{RTf^60CSY{o4BC&*$=Y5%bvd1rb{*H4s{uh9vq6o>9I|ZHD8~; zpASEC`#Uk4#2i0xmc7DpfK-efKU`2c&nEGC?i!Kf9d}5~s?`M*Q)@Py?m(K~zcCTK?SEK<9ge@KL+RBY;-CPZT5Qa+%S3FwU22aH_KNn-S35V{F6&d*s1< zv2BaLO9%zKUyxM-iK4@7WoKIaGe_NhDva3ffr$noB>aF?y8@=W~hpV#C}Iwd3- z{LTb2>3cCXi|Oy%?RY<(Y#Lv8KnE*bKb2#<>Y^s~MInMPF> zINWO6zMMQ%w)=N5Q4q+(Hri!8FZ({p#ctkkPG5!9#c&%*k+YCT+jUH8uG zy3icW0W(}XWLhuJ>$dJ3irKvVlFkWZh&f^Kav&>UMWuTvP*%{L!NvaK*9m83&Wiy!NyG3|semOtiBR zN+M1zW3qGCIJN5umv0D9vqPa^zW+QzHj(qp=6St@e~Hy-j6iY7_le({$7C#GWpYxQ;@-=jTOgA=5} zrk+G-8TVG_Nub=s1eYG+8E7P~!h zm{FrY8n&0ZKVX~;gj2Ed z7~R?zskvB9r|YSj&vSw6*x1z196jda?)OK#?|UWp7ak-8LCUgmqkxzbOF`UAJp0G- zYH3b5HveA$7c41xYR%UhZTCku9}pf$-5yw}VHt1aCq7Cwd)KQNefPZ}mnoUUUO7CKKw0>~ejR*r+k4K3bpD%8oDpJqq@2<66^9c&)zHdACtk>DNVNe8}A2^0P z@`=Li*tr=!i}~nT|3nr_yyPt~y&l`{zSjL({(|z0UZe~u0#t-wSoA*29&Io%I#0Iy z1cO|aNs$DGSn-NyFJ-fT@^QPIYjz(VPY%n~X#6qRdoA=i$F(h*A^JmQ7&SRY(#OQC zH?zIZ?1d3tg3ieFoQBrK{+}E74@QzFZ5d)pV@b_ZVd@CWG!9W;eb0}b1QjPkEwDK9 z9C`XdzUWnB_hW=5AskLjlQX-p19{2Tq{RE6Dbe+Ovzt7@{N$piPSSn=6(>g!yNSC~RX?;!V zhIn+DpLG#|I;Sn7H*jQux_P`EZBF@m~2%uY&6<0rqF6OX}cV* z9^ue#PB?8BUDYZpq}AQ`1%uc8c(ZdeZ}F>SC^-t8_9R`{wbCHzL*Apa>q@(G$Vly3 z`Gx)8keWJDEt)EITm#$qCp|S%jU=#at^Po1fOlKcZpjB{i=Hf3-4V;u?wtXSJSAi+J@9Y38}Y@ zc}Qp_wh^D?!Yi^2%S2h(c8c;s{DZR^X_G+FpMr|wej~2qr50JqsMVzaT?H?XSjeL% zV;ClKL)#kH@JS4HV5-Th)T*bN1ZR4-ORJ5h_AXSO9(jX^c?KQkwj@xnG!UM?kdviP zc5)EIyTtF<_B#>dSOT8L4-6uvRu~hj7C&__^oO1Wn*WGc&?u=h1N$%W(O_u12?PhW zn}R~t#UplyxWYnd9^*oa7O_2|U`qN^AC@yh*NW-j76PGC>hkTV6(LG4b#buC;#NCy zFg0K!BBHbgjsdNRwRD)xrsG0R7jeWeey!`-wi4(--V*bGzi z^H({Hx)>#;vd{IIr%mtPz{V2Kk(IO-dwD1bUg(LMEkxqjXNUa}n+LAt z5bksHEZmY@^^0vujB_JIBTS66=$8O9G#420lZ=9f$!VmoFHVB&eJ^$cBePY)Y`vK} z9uLMTx|{E+C9VDu+yNlcM7q_25^yOxx|Z{*$lC~Ihl>9(ETUj3E@Bp>kHC!btB0x} ziU?CCvG5ZDN9MYMj4WAE1C*r6N$nDm5E69At8d8&Bu+|C-&YheX%S^kkLS9G2&a3y z*vKe?eDq9;D#k4?KFft5U6GBJHN`@Hlv-jvA%0{)NmurTfs{nAEm28E*0#U_Y1*(j z={2oD5*|eYHbhU|ZbU7}x#%P-4Jd(?^(OR$IK~=oL|Y0BoPL9EOjN9!cPhD zHH`@J!I7Q`8Q>uNkVK<}08sc2GDzg6B+#aWtXwkPG^B85K?zVc8nw!mF8kRHMOUaP zT`I>GiC$s14zL7JXdstxCHqCn#xIyAApseXS@%hy48d1|#nQ_V;LsjU2Rsbio`pzp z&hlUxATnZHAc%70hPPa|d5blENSy%~-}zSpciL}!v7Fje?z&*K9aa-SM3JBJI4e#0 zQ-t};O(ZA?Vy#+P_>PF-Vd1m!pr>6 zCUr=u66lFK$dRY$Rv6E55+4-GNHk1L%;!cH4GhX4AM_&m+pl-TN))ea$S4bfJe-pk zS@EhVH<^Mq2dg&yiE4bxPh{bP)(-J`6=1({=6P&%5i)2uQllSysm1h_D9`bKNQmPG$iH*Q@Mrq%8uzMvFk=9puTL}@5u zgSKfvjQhwr7FU>Uz=$)BRVM+49Z^&%5&*4*XCXn4&J!Q?gALFS$U{mbcL3%R=XWYI zSodT5l|pyj69a^IapOCv{#>MmMPj$`D>`{YptYeo}w6NlHrF&*fWAtku1mR3aR z9TRcK5^Q)TM(+n^tu&HYnIWWD8KEDeq*7@WE#8flC`byp5Bn_0X8=wp9A`x*kFp=# zl~;bj8t*hAI)zVUwx7$uFiP(TDsF_C3{gd(A`)dpGG$>GQ0-#@8q9wX83|Gn7j>tt zu&s{Kmvz3+C)t|=WabqmYiuJ7zmxqcub?E@>pzEd(bNQ*SlAgNOAJYEdGr#Nd% zDTM1wSjmK}72iYpK>CV4PL|jNc%-z7OUJ=Z3L1fE(#9f1A@UXysXkBm722Vy+pA(q z#S$2K)Ca^grmBcvN*XoABuL&012%?UnMM2@NKB7ep@zi&)JIK(6^4!JS_&0)R3%|Y zlKv8(ukDk_!~%TA3y=5jmd((PP;@VJABUi6VP#udh9;UKHK*~6x z>&RaZYY}^V#>G>XQsPbX`~)#nO67vNKvhHshocQ?g@H71vzR9UWk97fS5IyzkB?{O z!afNr;ma%$PBx^VmNAgz=Q>FYDW#;GcDSGiKN9pcW7;27@US)?;K(I0o9CP;XD4|7 zKP>=1aEJ?X#LT5?YR%a2*KlmS91H2M9V0?l8Cof(Q2@xVyW($+`D` z@1FnUzM_hP530ZJ-FvUKdUsFnq=#~IT-sc%vR_eX1Ca?KzW)h;_?9&r53AsaXI*31pgq)j?)26CjpimeeD}0a+$$m`e{on`T9e*nJ+a7qr8J==}_k2QP z^sd^={C#cr*JT?{vlKPij#rWWH^M{rh!6GD|o}c;JLmiqP;e*^^nHMG;XspS|o-557L&@vS z3ln=7M9>#3vK%K$-4GO8o;wYB1o!T-C%k$DX}&T;mJq5+d@1fHfo4twWrUw3m+^I# zBay)!gHM=0q$&=}eKdS)sgl6F*zdA>GijN5>T5f#Xm}4McyKr<7h<~KP7gI0HQyV)OU8k4oCYI(yAkIz$Id z{n;#V`(-jcWUw(r5-OZv#Ns22xmP~n6H<=zPwWGdK?yN}-1qU0(jkcI2h)$XRij4RyZjjg z2>T2Owi!4^<2D4Y(q?QmcU0ifmosaW@A6nxqtWb4&hk`xDD6i6!jC0g*??minOCnZ zTtp}OvWPwz7n;8P6pf)YLPE^J6?Z2sQSkEkQxrj9wPUICwp6~6-}N@O;GI>>ZnO*v zynrSTS@Yq;tgrLi9f0qbD0@ydvyb$SYfKNQDdJw=V*7vU3Q}v5E`H6Yp=-9_weCcv zW$I{mzkjoO*Q>+LY^EE2s{d=I!wAKhly%jVAYNCQXw6P+_4$)GNb+LoPpjhI(jzC+ zRrGFk^dX8y_|Vo%pA!jH`6sbGv3N>FBrmeOYAwn4@p;CaNIDT>bIMzm z7?EEeO{s)pXmi;T7@l=#k)l1vyh^xGq7}&g24Qqw;oKBdc&l0$gCsl4xc;Kb_*GqC zIJ>en*j(a;pb|$L{gbYlrdRB3`MBmZ%w0?fK<;qYw9e`wLFe(1bA4r@^V=akx_E2B z>!~z6$jF5B6z|!~d*Jl#-dc(xY&WI)DNQs`_+sDw!|>?luPQS_PLm2XL7)*wQC8g~MJ4Y*dvw(f}Bz zO^O?$Gcpwip{N=QQS|qWqKOgWGY3c)NdTW!)6!)GMWEW$tYD+8mMcpzGcit)-NbyG zl$Pw3Nj^!~v*|Lw59-)0>XO7Z3k0E2^;^Xe-SAU!BDppqJw)$N?m9b9Cd5J2NYiv! zS_v#O6#dewQs}9g)K2cVWX$BvWx^7G>$Ki*u2C9dhnFa%m?^^7{pNQ4rNV9ed5TvG^o zYA71huxVh}gKZ@Rrs6%{q(s-Gf-fP3O@d?LN0tVNi0#-E0(o%Y5X{6`OVt`1Hd4}+ zVKMQG5XErEpA+OFedX`LErJ&1M54i+68A+~XAoWL>zhz7#f9?sCy@hdgZlwaXzyY- z8nBBvXR1_VW?uF^lU-!!{qm{OH*MHFp%%rS0ovXE{b|-Oqg2z`=kEVLOw9bH{SJX@ zH8hgi!i}F8j}ZH``sisd&#@g(Y>RwVW)_*}74O@hfs0JI9XOUiCu_6=T7+wQWT022 z6+1VSO`uU9Ft4HFyx=o?th%iEVH4LoW9E2b-hJ`1w^POEGTNJ$0EeoSH+9Kugx=1X zn{gm%TX}Q)yMZl0aPKf3$f1kjj1gm=F!UoMuU~=Ll3%QD1*5&etdXk^M{Zf^EnqhQ<*?|A%Eq9nSt6fP15jr7K^7efig&Zn~> ze0V$f?T-O5;G{O7bN`)*%V!-41Hln+&D>9sj%5C~Ih+JblwmJD@C0-NDEZi5c_Gg1 z;j&pJX?sxggssRlA|NBxcjTctDISN9EUsg8yj2{hNSd;1>mM*~Cn^Vo#9YiaO#uLSJdb)A@$3+&}Km+eR{P`%WJ>x?cYL z=6qpr@#|v6)vmmyb#5lQl0`uI<2tqE56M_NaXe%O$}rTAPsN{7Bs~3$O7XN=w0Y_E zG9@wv9$t_r1)3ZNb=VI#pG(@>%MGE+dFK?4@p;$f%SREwYD%rL{w7jC%0j4K}pKI--S_vy4qqV zEbeLEUuET% z;GR+{)&LA$oPS<0uU!CKRhMoc`*7hVLi)#~=rC*t=^-PwC*Ho`xlw}~~$?cH?k zK5s7_SxDfp^Xsr;M`Fv!y#;EUxF?D5-Vsd9qi-Xe8w3#IRhc z--62qGErfl$v9YbUC!(!HJ)YGX8B%U{7L2a6!;Cik$IFn=BQ+c{<$av%@{N*k`SAt(o#TS79X8u?`5$+pEKELKSG^4YuhF zykCqxvo5dxpljriCS(c)?doPBS93>p-XQxrDULElG` zA=)Om%;-@+3w3x9qud)^ySNC{Qa}xvExVDBPhWVNDX78w(E--aG)8x(`YEv*L& zyDKvU(7O%kXgyl)F~M@p7hL?Q#;GQM8&%|AKjok*jG4i|qPq{yBD8y06Hx?ZYXyfe zXm5Va&|Xi)+J*rviv>7uXZs&$>-r8eHwiY%NtU_hm_GvL(@&3#w;T<%o?-erkW8Tv z&b0;&73$217cY9-eICr;Xap|eJJnu;wC66BAH063+~PDAiobBhT&DL^FG)8He4(vT z=J(Ai^bRDT2V?o_;iq&d~E4jb)d5y zC|b{`p3$3`WT}jUM+70;c#&M&`3)?Y=Zo=*J!K`v%Zd!ny^Exv3;E*o`H`}Hi(HE| z|BpLz%Qp^>_!G0K(jlGOx?eCrTBY&3vosQ3^d#Z*T`%XnY@P{Sat9wwJlj39S~xez zH1cOi^M3Q1X6;ykwHcvk=f+FJd(Wt#_4$}sOEYwl1luO8Yt~IP8`PI~QzPd%#cO`iVZ$NR}?uq%@m2ZBrL*B)}-IwFovuYJs zJBAv@%a;wO`m0T5#8xa1+S#~r7U#1U((rbdO-_vrNu5Y&}5d`-_!Bz#uu1 z2eeT?5$M2mA{Ce;=|CY*N;s*fwZ2L@$T@3NA|SkzIPp-z8qJzfJH0ZU-tyJ=%H%BN za`#m-E2zz7b%Q6f?LnaJR2RCdV07iwm{E!u?Xb`Y@sMUSWI8E$CDWrUsJV7*wOeNI z50UK*>FyWICE5&GUEau=9yU>W%gaBtI1)h0MftO;WGOsmsj_uryViZ-=H-G(M|nnY z8>_8V(%7UQm+;-w!Ow*H*#f(xT3h5PkQ61!-YoPJ8K&U42{CRE+2QtB0W6~=$+w@WvoS}%S$;b?Ub$OM0ml`bjkp=Fy#FOC+tshB^ zlgpn=1$^sFgYVO!zKP?0UJen1Yj)$lraL+ueEV#?mNAqzkU{uWKh~_U&w2&frema) z4JIwDzVw+pt(gqFL(Npkn-wxdE>EHTPE(uaW9=U(X|Lz|%gh`A1wL`!2bHOcu7!D2 zH-viR<-!FRlf$Nq*sLN7y+Y9wn7@d4<;_&5 z(w%P07`213!X=^lPgx0bm)oKO#;T0AqwNQ&RK&9BY5^H;Tpew>l&)|+uWlEjY!0mFZ1uT@De%}E!7m?DFnYML((&2VAD$-#%6qpsYU15<&0ot? znUy&#kdL+|(aF?JvBYcU*r<0bL zN9d(rY}@*x{B~c#VT*7#9q|U-KtyvL;x5;MV>7q#A`K6;)}mIB>qj>!TR&s}9s(^R zZX(fb`+3vi$UTKBpoGXr8i}VqRuW)p=)|Ay+m#Z&fBY7H&#@8j!&LslUH5d`!>^R_ z7@*)|W6xBy!_17Rp$ETxJpos&k%~&^^?B0O=(%lxkujRq?5z#Nnnt3y!|Pj&cgnrk z=k>8idfgy09swAnlDX%o;7}p|8q<1MB;q>BqBjwjf(Mg=dj+f4ex8?<*jAiUVOW5qph{ z05e?R76NRHP`{Q9T~dGsA|=j z^J$y9$R4QT@;y6i*x<(ou4Jg26a*mKomN&|0&o!X3qN)7|c#p5@Gtj9YF zM>RJ0zx99VwE0>DVYR|qmYhjShDKeG%V>anDa9{K2U~-_2D(nEpJ|Yb&+7H5<{qD! zP$bXwS5%?_U|vompZ&chi*^04tQSi|-g*S<~*tpiv2=ZfD*9|^#9!u228Mi*l@HW=G zX>d?<^(IDCrTUF{H^RA@sb!TOoqh4|=pbn=-$d-hBGY3Y#yC73^Q`q|Kv z-A>r2R(?C1N{g%HOxA4>I-b|;uE8*)W-=;j`t+tnGB7eVgTZ>`*ICEY$X-2?#?l~* zLZi%1jXL$>ni;Xq@xBm&Vw8KUWIyrv)O^g;hdd+i-CdTmAsaga0TXw}i!T$+Q=wjf z&uxxDTXy5Gxa=Bz>R4-W;9x+AT248*+iim9|9CzhKfN&&5!nJ$L|#fG-mko0rCwLJ z%4U=woU}mFR)2IgI5hUc9lx_lM3j-r8RdL{*}F3`4lq|6@o~CHhfc~8milsnN=8|{ z&NZ*Z*!;`eX1vdqWyTh$q>hv5AHvk}0_&ZiUdLSn@LGdD%VQBn4;{%elb_3d&q?Mc zYf;{VwDcvUx~Hb_O2&#m2>(Q%NA9zCOSDcE6)=b^iE2@p@*+V=-=a0LfiAp$DNqB^ z$zd@DNjMysTZj6tP$+gNZ#vM6}@ZrEqUk^gKV{oa9GvHE7utjW3nODhZhGP6o?SQ2K7Y@;oX}sm%eeeDIjaUCh&?H$ z{GrGdVl$7Z;C8=tuO~v$GPJP`v*i&P&)(NYY(maYXK4DkE8Q&zVbU4iRcJ3u-Y^QX zk0-zs{}?`lKlX5gqu!EvackBT^kmtUNTt*tT)m$J85F+0oT04@L_oY%%(}KW&*X~O z_-ly_C+k*%t%m~#a;!6JN*@VgVUM7g%y_byXXY&D&px|-6uTvz^U(qcs|0-m`GIr_ zFFa4b=P99kcWA8F-1UP)FH9qS)zf?^ZBCgguUBRi3ei~Rs}>@@_^UIzDA}HClTW}@ zX87I4lew!wL3hA9Ee&We!41C;(%k%zJ-Q`7uRsy6L#<^C~bOiJx9C~o(elF_(|cPrlGPM6P^9gJ~nS}dt9D4CvVf1 z$3~$Yfn5caGI9jZMaZGMVoWan5O4Hs4BgFB=`TWOEZ&5nf}r$v^Vl@QjKSYvhtEK4 zF>=8$NYVuUOlR&%J0w^6gDqoz^|TB|%G|t?04(SN)zwz0&B;}GxlVVdL5R*=Vzp%; z{!XcU&ZK{lU^Ik_qNcwCLu1-GO-WZ=p@|{8!#>^O&$9?HKqfx?pp7T%wD3I;6esAPvW=01(QIaRxzxb1 za9(p&r8$3@R0(2hi9|NmdQNPbnc2` zrp?m!Y{}ZeSM|qSe)80u3Ge%u-AB1R(7bA|<$i5=&1*`b<>7u!#Y64YX1(36PREHl zk*O0qN0((EpXIwQ#>TIkD)(I~{^hg}0})&6bE6!Ktv{!i;`o1Ie&--cPnQ%jGJV z37v1}Z2Qc%Ge?ZO+N}QlPC=q_qanJhnJp7^@f8>2koP@XU` z8c#e!E0$cXr#dTHr214q%TTLMdP7%>%P3j%b<>#rdX34Hd?tA(IZL0vGaA4q#vSHO ze8QNcOeHJWBL5LiyAZc(X3urn>E( zdV7BkxgRLeE+n656WH@RdQOf!`SKQn`&{Al03@R7E%2zC%6f1G$(gMLo2Gw@IJvs~ z2~?FyCADMp_FzxMdtrX=9+$QvpPC!ut?H^#QsR$bdlK58ml@1yp2OaEFvvB(< zhzYp$GLBWwP;DHt_HW_UZJdjrP#zaZi@&ed7l_7Z59;3}H>nM81e|iX@04?JhHzYW zju+a#zY(3kn+FX*tBaNT6^9D*Gb|ZtNFCL@=E%&gb{xiGUa%;yX_t)(ckY-N-;|;% zBFynA@`EJKM9Zr3S+u5nj%}UlP}{1*J*g%c5)H`4FjC#apki+YuX#LcMAc^Nh3wG z2K%RO;6h&+^fWStP3mg$EMJ(t4o&|ep7rh?bx?D>0(cM{RpC%ndtm<;wLNzMs{S%v zw(b7*`gCB(U#hC#@>D<0_k7WiR>ifQIYl!s%@}&!I46(cdL|SOooaT3rtD(+XG7il zIlZnl&+;zGVkqw=N9o84IV4BdnRw1-H(^j)?Q(Y*mSF(uEsEpb{Vk6X+1R> zoRxUskN~kwEmKDF=X_dHWTNEy)n-H~bz`;4s8kn3%%8VvRU{S+bs46fA1oZZb0Wr` zV6uBD9-S}C1}{^dgBEQim{e=NEPCHCs(2q z$zWYhFzI0WMO?PD4G7K4r{;EUoB}Ou&+e2HW;6@q69$T^DJ(afbEb{=djkmAW2qc3RgPpqcjA9m!vO}Qv5z3d_ZG!P4rVcjB) zVnRXK{PyPHk_pqvyGD(^o12@og`*#prn_`1ze_tt(2?^$Y?%_WeiH(DieKdenZ(=% zN!;pnQWf$%H8Uy-0`D#0ccnS)kYo^N)DmU}>vZN&2RH0M_(HlWX%%`&`fIrq-KO>D zDbx4!uG8<}3a^bB_05N)ryFgB#%lK`XY`GVIu39Ebg&mBYq=2S zPy*U&-UOD`9xbqyhQDr}UrlF#_>z1gemHcN+AkPUzP2gTRDj3gwENK8;kt>$KKnzyv8>CzIf;bD*2rw z1fXHp>p^H;|H=}O>OS#J`s#yWgU3&9mP$?bTOa<;B>pR%*h!hCstid%z^6XpL(q84 zsT3)YSudd*Jr2#;av>1%$_pO~QjWoYDZgJP+X#q5+wgSR)vDR&v2Ao*g(R1?xLh6> zf(+XH48=;Aw9D{T)#n@&Y)i)W=iE7kRxU2w{Pq$iH#+kdGF={{J_IdRdA=?-@bp3pGVkmD zjM=q4=JH|hha+;%&^Nw#i{ot$%c-fUXmSz1hn3G_<^C6Uv!TjAY=wE1SO8Y4jtqbT z56xFVv%yPT3Z30AdI{2g`lGU=?O*aSOADiv8Z2VyD#hWpyZ98)mbn)iGJ^6iTfT@8 zz5TvX;S%Qeckgz7Dp7Q%Eu(B9Dml+&50Bn1MJZ{L3QDG@f7wfgnuXLyqFz zBseUA_EJN$h@R`aap8A;;)|Nf%IsHh-NL%2c4`KUfSXFwo`j6V20EBLRWtMBanxI# zowdIBRB|;>eauo@<#V=48@5AzG8`NAm~0TIg} zpnVca1exrc%9@w%XusU(78{Tw zXA{%m!M(!ts=xy`Y>YW@LC^d|TxqF4M=9fFj{7p1npT^2cyFlxtYml|xB^gUjS8Jv zdV8sz+CR6%5hF9XKVh7xM?Mn6^Sa~&sV2%22|m;rEX1c}7oECUn15kps7(7K@bJhb zG~@3+74a0W3hi{`dSW}3s9X_L5W@y~BeRT~*`}PChzA`@@Gm?VNU(`i&8|wNQ5j{z z@|!jDA~>jQPmaPQ4Qd{4q&?S4BDFH^mIOx2SXql|OX8 zQ@@6F%EDfoNMy`1p2VsE`1*5N%`b?Xp>77}1`MecBNDCzDbCdTyZIv%A&z>QQ|R-7 z12SNIn|iP9!3vh)0#8JJ^Wdt4t%wjq-Gt(?`RWNaCi4caVpAduNxPG4Q+*r8M7rOK z8HwP{hw}gxgix8^uENWSc!861pWtNWB_;De!1gb63q~Zs9GhW4r;y?f>qkM6k~=Ua zr?Ey6z$`gg7KK6D)eF7(t#<^?wW#wV(eSlJrB0D)8zlfFlGfasjzz~hg8|f-9_kbF z@5p1mQ%k*$tn6l>&E1M**OTo1#EW)qWg%YbYJe%K-YfrvI#W%BPw-@%bD;nBCHoAW zK#c8V;mtS8XH_&m>UNfu7Bq_G&ufmUZZ)kk(eWe48yD+H)EQWqy=va+GcFsmLhFd~ zwJa-;63yAxh}yTSf+J}E7vcY31U-5BA|U{g4{ej4G6zb@T8M?f$VHLW z6=c^aK(1>>B5ar`rJ68nB0wfzsrqI98PjK{2yWdfKXL5dPk_e4#G(noKx4C5DF?i8 zc)6*nn?$wc0_?BWGk8sE$P`}Yw&v6CeAy);ingtklu3B;_3Q_&qz!^GZ8PgsZJF@& zK!fFk=(M`DgRxe|3?RMEW>44=qN{S0c(s~2H)21h-7oD|6dN?na|BA&^{JRqaUtE| z_~Cln2)>vsrzpH3=0|t()!#0bFrfrjIZ-2)`_rsA452xngh%AZX_Rk(giB9;M^O3i z2-;1_Hbi}Ml!_j-$nUBHvU=WjW!>pdVvxu6uvupIt@X<%F16Hf^TW27>5nb9r_(q)?_Qw|@;ckUQY&=aCub};5-y9Xu zK|Sxq{{%mZ+vby?5;@BZ?!nhdOvlOWtX$`kJkk4CsXTMNBONIvOg=W}!vY}?Ys&Tu zS*50gmoJ-kX|-i6jN1#DW;Jcro3@qy#Td_&;fxU|V9xmp?%R>=#XNHF-()?^vju-T zsP;abiTGQu_KcnctK*5KG--i1k+lhAa(rS!EUlm0HmOVilRp&Ln^^+;((#qWmAv~h zJ@9rCqgS{lF?YI)-9(a-VG5H^=s4ZgmOdz3iZ5P{gz%Zm{Z;`$7I?gsCo!M@=C6O~ z+7GXu?u5mL@(^@?6hG=jq)0rN!J`|5ZIo+d_c6N(biCy-#$6;Iv$*(GZO)sZS7qBb ze|f;npC-aeSmP|CLg$*&8-283c6p}7L(kRHlZkvP=NmiEX&W=J;CxUiUiDkvF<~!3 zt&zht;Rz>H7vx4BRnYh*ZLRmAwW&S88TYUd#(ny$XoI#^>fpaW(PO9jTlYgXMU4}0 zWWJ|1h=eb>_yfNiCtHm$B^cD`8+{q_lOcXl^}du3gT>$d&~6e&=r(*{^$|GVvF}C? zH7#vVYjT~gj^mF$;Z(3K3)%=96#lI^1Mk_|!;0_yqpc|ZX(#y0Z&ls|$&3Gj*c;-G z7IOYzRAYwd>Kj0aT%c45RaQ0vT({EC zZGormwQotM#`4<0lRq_Vnt~08BV2zR!G)T@SOQ3;Lg| z+~4QzLl(05g!!l!n`4^xmLJcnQ#4)BDsn2QBrn(_1U?-7r3E9pOLu>(*YM$1Xb0ZZ)%TU#?;E|H9B=hR zS$v^;Y2e63=g-CFSk9%qnWjJa>&Lc}w=Z{HQNw5NQb+#9-;<`CJhxlj02oyIWg9hq zy`cs9;i2Ns!DC1`{CAf7M6r~a?vCQ^mKSuuEMG1la7_)Q+e`Id81lZ8cLw4&eP1+v za49N#-J(CSTc}>Ga*+L8psHKVp_V`6_sl<&e?Q8!9k}x}>feV9{*yEQ<5j#@aBzL` zz?b{%AFlo%{?nd~{pN!!y)sq$e=aos(d3I(guvoY{x7zl{Lvx){e} zc-1PuD6!0&;a^Zxbda6YrCaD<8MW^T_3fa5Ig=_Er_XU)UtUxa{$qfDHJ1~Hqiulv z|1%fdS;PZyv~O29M^NEf1`V;8SQpwGenZ$7bf!ILxj}CnF#S=@t*I`d<&Vvf8a|OW zcs=i|GW-{ZX)ztaespWiS=dPYNk+dn`tAPO`JcM~E-C+4(=ys{U>y~@JR|+{u>NJL z8EwR0t2RdIFNi-OAcl&)`Sr`nQbmg^hV=ZfWW`F!fcU%f zu3R(Hf9mW1edh5VF|yw4{TV47=7FFNvZ~vPVJF0<;z2sPn?y3l=Vx9W8#%g zaaB>m7*O=Hq3k(V{4Hc^u7zsvkAbj+iZl_FP6POJ1G) zXYy9nv9agW@pc;UH9p$S=YKjeyoE8ubT&|O*{6^2p0o$zeLY3oie)6>7{}Bd{E1rd z1z)Wg$+N@gUV1uN36etF3E*&zQew^J6C2ZRb0qz(lKv1tZ{D7#!^EuKf_@(XJCG4& z^Ex%Ubjx!y)pnDOf}W(A26u88FWr9&p)%4+Ciz}W<&Tf>r(k=l@_thrdg>GlM?q9l zgFxgbh%}+vqM)CW+5N~;us|)2YpgO%7Jsg%smogEJHQXFEWJwXe}n)Th2Kudzj#Iv zNX3+;3!}w_4r+WQjDSHT?ReNt#Il%FX&epOig%n&Skl~_+-BFSQV08;|1i8KPouxV z(T2*sfBWrkn#A)R<1OMw!Ph3RK8gjLioXCl0mF**Y35{r0SarqMu-T1iVkKL`1Yj{D5ogWmm$1^41Xki^$Vz>%*!04YS+N;#w2y$ zv^|poHNO)4!-Xp zl>UyB1Ksvbo71n-zC299B;r95vCl{XCVwIo9DQ2B0^L)%^FZdiP!Q$fO0cHJ=!Z@X zh+WMl#KGm9Dl~5YE6V@<>j)}#y$!X8Btl>(+sMy{e9@0m;Eph+2os34%fqiuG+oS4 z38DFS<7t?!HV4*#p343YSm4eWX{<}eFOuu?O=;n-7P_^bo60Y{Vho$xoA!X$y`g_{ z+yL~WL}EfjxL>F}AvXbZ1F~fkfM{$$LAJ2c!6rvDT$Eb+c@Ry=Omd#TEyrG!F;Xof z`2tiS?{)@m!qt{;VPX@mUN4(CX@m7^SrR&xj1o484*2{ImSUjWRG0o=08%Yb$oJAq z612q)e>49R2pT;Ne16lZ_SE`GowzhU+ z8hdbTFdVjzfNgNdIB~gPKAvWQ}em*XyVcUfGCE80mpcMXcZxl4{`Pz zXB_->ffAHi)^j@JHXSqu5HM*IE`sMc09jKPMxq1<$BOaL&#ivO<(wOjO~8 zxz74NsR8V?HOr?X!r2w#lFNYMrz;=_-+_ke-Nx?5U{KTZyrI0)|HA6129B6;M!~^D z1(&SwPDzt!tA5?N6}buY6KwTeh3T zNesuQub6^qe4=@22#9UKd@C2;msX4SXjZ8rN~o`T@J!;AyvLf$S^mOJ$byH*X+OlKxwOe>-Td7Psssxoam0B=6izgEwnC{Kw2%=6%*n9-qJnxU4^T|AU$KJNo zzc6ss*}IXu%d<_F1MS(cMED)ylJ67Fv9%l-7{#|n zNf&k-&ZbAQM{sIZwwW;Qybol;vZ-;caHL1xiB)%N?IgPV3XwM0PCcsBa+6AMRHs(U z{!Djjg01a#jOXruLoNKsQa0Q-=+f zasHD3WT*3@O%zoH_ANm8=a7Iw$QQs@)QT}#!Tsv`sY+_Eo0x~U3$53Ehu2Hb7Vf|1 zr84)6YaqK(0p|BR%Vn_sI7{{JPg*@&+&Sdo^YnzUx2oi|{p7qX`}{WJO`pnRuUA8Z z@BK+SQ!-}=ZDkW4VAX~+5vH8O{&}@Te*9Qd&_soD|633Ti8o_&s3ON?t;(gUi;6A` z&64{<(*rBTq6n0EkiC^rRf9id$IM7-Z)Uz)swC!UYxN$4ZHPF2H9acW8ddox zQCBgKhn79lY^A#kT3rPbBgK-ZPqomba;^1(vpL&e z0|TA-f8=sFxbsJVmpvv9FQ=z&Cq3{m56jj&zi%B>8{PT%->$daPCq(rEUS=YPaZPt z;ID*0Xj3tsFeA68zM5H11Y7=bCYj;OWgAZcdAO3Lz#T!)L7CbblD2-?vX4pFdWhse zZU=lEB7A();B9I6u2>T%B3s(*yh?JGQR!8WQvtFI3%%RAaUQi=bJcTR>!tarDT~CI zTWcr|eoyf-q5mztC*9MNo}IvGQo~D5gUrOI~%AIni@Cs z*Y@Q9R@xPo@I|_J9wik#V%-9r?1Un@N*Kd4>g$ikMIONAB3!S4p35n0{^m8Vb&JFD zQPuL{jQ!x|Ee8JY@V;@@_pf|S-B-aP09Cmwf~FCs9X&_vmGyW_H0ZDl9e4jjq+r zIR6VQzXuPl#xn+ToJ#}*0ya5GG0oEaw$oAv2m6Hs%s6_66QA7VA+ZFO9k=H9x(ScK zaY9WIq00FdEm)t*{S$CxYF#gO)k#638$hj+ySKY@5UAIBG??#Tc%J$f?NS?cmJ}h#(62Z2SNQvFRel}k6%J$Rx7j!F+$ssZ=slpU0 za8dpFSX8`9ONyrMkGO3!56q_Q zZNc>6!1Da+yz2a{2^~vC$9#{H_+%Og>&N5zEz3`0wZbrBbG1Z#aFcfCefQ(-{`aG< zwhLVU2M&?D%5%llQi2y9r>1{x@&1$C0=0i{JoQVj!4)YLC7eAY^sLo1B^*&%gh1=& zXAX~@j}M8DXN!lqJyT@9t7!|D`7*n~KL(|BDzI_&GET9RT)Z4sfl3tK<*-mtsdXt>-;xiA1NJQ zYQsO7)POY3Pwa-e{6ad}1eumgDyyg1#lFZ(f==eLx(b^4W)Q6aH7Uaaz(yYROt(#2 zs*cUb>p0D5zVUAFz0c|=2md?r#}%Xbk+T?LaZ9NcAA#YisaGFd-?Ad<{V54m(FT{B z`-i6zyKs@mg|E6KUnf>2w@6#b1oCc|!0MzN9fqcLZML2Dz*~JNRyf`rhklp9%Je-+ zKio?9*@7%uXDJSZR!4FjnZ&vxv& z8t%?aqKT$ZQ*2^7SyxWDpu%2;d5h0XMBMSl)VR^r-MlTER$Y1KYe|t~Q;{Q$%m>+` zSeJGBffXCh=xRSrb77Z(9rHhtG(3=Rap37{5w8UJ0sc-JRqJSx;FJ;#(#^eJw~_ZO`fsoyoM#@-YffuIwJ1EDz|l~jtLiB zDp}I|34mBeH+*g49u1vA;`LP-k9nd? zDtH#$?~ES4?-i;bibKOfFhuXG8ZC7BRts#*)V=NO@zZXV0s9&eIV0oEzJPgYVR*`j&1zNB`pxP?;Iv(nO9o9KN=q!2613fIrBA?(`$v*(0~Zt2AiF zZG#vDcGOgJ+5bOV`r`Zl58C^GM(_WB)xPp!-aL_;Rd?*aUAXz#byiqr(W^5r4}R>8 zTW@Yrc;?x%;G28bu6tFkYAARxY zX1o2zz4HI}_3wZG_4(pG9ls(&#C}Px`)Zq#`q6to_pI<$42#a~?p@WrsBwKH*0vdF ztunCokheVrY&kv@;wY5wS86?*x8!6~_`WNFYP&CAEtAolv2e%avR8d!+oD}XzV510 z{&h!uf8ddqTcx+}`}O7W<4qO70}t27Ra;&$*c>NPFze*EE`M>>6)Ws7Ox?9}s{Ybx zA6MLeKi{_e*`Jy1^7UUHEq*Sq_vz!&&CB;cSAKs-!sYQ=4`Exakj&&Qe~wHqOA@`j z=Fz50UmCv}Xscdr)V_V2Kl1Rw2-NvZSkvS{5O64LTGC?s;*+`Ex|wMt_K!E0%5h{D!kV>O=xSkeq?YuVTE7zQKx0sPZ>T9a7krfM>o`Bv(p zN#6wAVwSS3<7sBaGNu45g@k~mP<)Mu8>keT5WU1{^ZwH5R(^>KHFOhiIow{gs4(tp zmDr9Cu~O-&vt6fDv$*^YoydDu^V6#NOT}bouDh}6_ojrl{b}Fl$%~)0@c7$U@aCUY zB$KD@gC(I0CxqraRPWv4wY6ebtBi1WNM=J=U`Xe??58iaG*a&KOm{90e8Et5eWUGs zEhFr0$pcKlWXzr50d(GC^#uV_TvcybOH`_yS|m72?wpCp(tsC>JMA>v7G}6FJOMmz zoKJdIaK!}gi@bMzR-SmYZ&Q_y^Y{Hh3NDUoSSR+n3NEjSF&)~P{w7dB!oL%saID1YoThzMoG8rHb_YE_Ul_Sk|pQV&c5<;<4F z68KeR0UI4_uT8#%*)m7YQ`1Bg3bv}q9L29HOdD9yY@K%nG*5-SsSJ|Xumm`7r20^U zfT}29?pXbGi5d1_hpn|$1v+1s)7=ois;i(j;#&2s*h^rTYZ3J%Oaz-J)`S5gJj-?^ zFLL-`giC@a(D~kn0(tPO+6s)Lu-Bfsc!CSq$M6O@U#Kq$zpB;1p`@*&No+KUq2z^x z(If_+njbC2;L8g}OEGvAHCj=^3yj7gUs3+44_+f>w0r&4A_gGvboFyt=akR{0Mt=D AhyVZp literal 0 HcmV?d00001 diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 05ea526c99e..73fd361a32d 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; import org.apache.rocketmq.common.ServiceThread; @@ -68,6 +69,7 @@ public class CommitLog { private final AppendMessageCallback appendMessageCallback; private final ThreadLocal putMessageThreadLocal; protected HashMap topicQueueTable = new HashMap(1024); + protected Map lmqTopicQueueTable = new ConcurrentHashMap<>(1024); protected volatile long confirmOffset = -1L; private volatile long beginTimeInLock = 0; @@ -76,6 +78,8 @@ public class CommitLog { private volatile Set fullStorePaths = Collections.emptySet(); + private final MultiDispatch multiDispatch; + public CommitLog(final DefaultMessageStore defaultMessageStore) { String storePath = defaultMessageStore.getMessageStoreConfig().getStorePathCommitLog(); if (storePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) { @@ -107,6 +111,8 @@ protected PutMessageThreadLocal initialValue() { }; this.putMessageLock = defaultMessageStore.getMessageStoreConfig().isUseReentrantLockWhenPutMessage() ? new PutMessageReentrantLock() : new PutMessageSpinLock(); + this.multiDispatch = new MultiDispatch(defaultMessageStore, this); + } public void setFullStorePaths(Set fullStorePaths) { @@ -117,6 +123,10 @@ public Set getFullStorePaths() { return fullStorePaths; } + public ThreadLocal getPutMessageThreadLocal() { + return putMessageThreadLocal; + } + public boolean load() { boolean result = this.mappedFileQueue.load(); log.info("load commit log " + (result ? "OK" : "Failed")); @@ -964,6 +974,7 @@ public void removeQueueFromTopicQueueTable(final String topic, final int queueId String key = topic + "-" + queueId; synchronized (this) { this.topicQueueTable.remove(key); + this.lmqTopicQueueTable.remove(key); } log.info("removeQueueFromTopicQueueTable OK Topic: {} QueueId: {}", topic, queueId); @@ -987,6 +998,10 @@ public long lockTimeMills() { return diff; } + public Map getLmqTopicQueueTable() { + return this.lmqTopicQueueTable; + } + abstract class FlushCommitLogService extends ServiceThread { protected static final int RETRY_TIMES_OVER = 10; } @@ -1298,6 +1313,11 @@ public AppendMessageResult doAppend(final long fileFromOffset, final ByteBuffer CommitLog.this.topicQueueTable.put(key, queueOffset); } + boolean multiDispatchWrapResult = CommitLog.this.multiDispatch.wrapMultiDispatch(msgInner); + if (!multiDispatchWrapResult) { + return new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR); + } + // Transaction messages that require special handling final int tranType = MessageSysFlag.getTransactionValue(msgInner.getSysFlag()); switch (tranType) { @@ -1361,6 +1381,7 @@ public AppendMessageResult doAppend(final long fileFromOffset, final ByteBuffer case MessageSysFlag.TRANSACTION_COMMIT_TYPE: // The next update ConsumeQueue information CommitLog.this.topicQueueTable.put(key, ++queueOffset); + CommitLog.this.multiDispatch.updateMultiQueueOffset(msgInner); break; default: break; @@ -1691,6 +1712,9 @@ private void resetByteBuffer(final ByteBuffer byteBuffer, final int limit) { byteBuffer.limit(limit); } + public ByteBuffer getEncoderBuffer() { + return encoderBuffer; + } } static class PutMessageThreadLocal { diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java index 87ff0a096cf..fdc725db7f5 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java @@ -19,7 +19,11 @@ import java.io.File; import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; + +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.store.config.BrokerRole; @@ -376,7 +380,7 @@ public long getMinOffsetInQueue() { return this.minLogicOffset / CQ_STORE_UNIT_SIZE; } - public void putMessagePositionInfoWrapper(DispatchRequest request) { + public void putMessagePositionInfoWrapper(DispatchRequest request, boolean multiQueue) { final int maxRetries = 30; boolean canWrite = this.defaultMessageStore.getRunningFlags().isCQWriteable(); for (int i = 0; i < maxRetries && canWrite; i++) { @@ -403,6 +407,9 @@ public void putMessagePositionInfoWrapper(DispatchRequest request) { this.defaultMessageStore.getStoreCheckpoint().setPhysicMsgTimestamp(request.getStoreTimestamp()); } this.defaultMessageStore.getStoreCheckpoint().setLogicsMsgTimestamp(request.getStoreTimestamp()); + if (multiQueue) { + multiDispatchLmqQueue(request, maxRetries); + } return; } else { // XXX: warn and notify me @@ -422,6 +429,52 @@ public void putMessagePositionInfoWrapper(DispatchRequest request) { this.defaultMessageStore.getRunningFlags().makeLogicsQueueError(); } + private void multiDispatchLmqQueue(DispatchRequest request, int maxRetries) { + Map prop = request.getPropertiesMap(); + String multiDispatchQueue = prop.get(MessageConst.PROPERTY_INNER_MULTI_DISPATCH); + String multiQueueOffset = prop.get(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET); + String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER); + String[] queueOffsets = multiQueueOffset.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER); + if (queues.length != queueOffsets.length) { + log.error("[bug] queues.length!=queueOffsets.length ", request.getTopic()); + return; + } + for (int i = 0; i < queues.length; i++) { + String queueName = queues[i]; + long queueOffset = Long.parseLong(queueOffsets[i]); + int queueId = request.getQueueId(); + if (this.defaultMessageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queueName)) { + queueId = 0; + } + doDispatchLmqQueue(request, maxRetries, queueName, queueOffset, queueId); + + } + return; + } + + private void doDispatchLmqQueue(DispatchRequest request, int maxRetries, String queueName, long queueOffset, + int queueId) { + ConsumeQueue cq = this.defaultMessageStore.findConsumeQueue(queueName, queueId); + boolean canWrite = this.defaultMessageStore.getRunningFlags().isCQWriteable(); + for (int i = 0; i < maxRetries && canWrite; i++) { + boolean result = cq.putMessagePositionInfo(request.getCommitLogOffset(), request.getMsgSize(), + request.getTagsCode(), + queueOffset); + if (result) { + break; + } else { + log.warn("[BUG]put commit log position info to " + queueName + ":" + queueId + " " + request.getCommitLogOffset() + + " failed, retry " + i + " times"); + + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + log.warn("", e); + } + } + } + } + private boolean putMessagePositionInfo(final long offset, final int size, final long tagsCode, final long cqOffset) { @@ -584,4 +637,5 @@ protected boolean isExtWriteEnable() { public boolean isExtAddr(long tagsCode) { return ConsumeQueueExt.isExtAddr(tagsCode); } + } diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java index 1f24dc7771b..bfc9fd282b1 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -39,7 +39,10 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; + +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.ServiceThread; @@ -47,6 +50,7 @@ import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExtBatch; @@ -109,6 +113,8 @@ public class DefaultMessageStore implements MessageStore { private AtomicLong printTimes = new AtomicLong(0); + private final AtomicInteger lmqConsumeQueueNum = new AtomicInteger(0); + private final LinkedList dispatcherList; private RandomAccessFile lockFile; @@ -419,6 +425,23 @@ private PutMessageStatus checkStoreStatus() { return PutMessageStatus.PUT_OK; } + private PutMessageStatus checkLmqMessage(MessageExtBrokerInner msg) { + if (msg.getProperties() != null + && StringUtils.isNotBlank(msg.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH)) + && this.isLmqConsumeQueueNumExceeded()) { + return PutMessageStatus.LMQ_CONSUME_QUEUE_NUM_EXCEEDED; + } + return PutMessageStatus.PUT_OK; + } + + private boolean isLmqConsumeQueueNumExceeded() { + if (this.getMessageStoreConfig().isEnableLmq() && this.getMessageStoreConfig().isEnableMultiDispatch() + && this.lmqConsumeQueueNum.get() > this.messageStoreConfig.getMaxLmqConsumeQueueNum()) { + return true; + } + return false; + } + @Override public CompletableFuture asyncPutMessage(MessageExtBrokerInner msg) { PutMessageStatus checkStoreStatus = this.checkStoreStatus(); @@ -431,6 +454,12 @@ public CompletableFuture asyncPutMessage(MessageExtBrokerInner return CompletableFuture.completedFuture(new PutMessageResult(msgCheckStatus, null)); } + PutMessageStatus lmqMsgCheckStatus = this.checkLmqMessage(msg); + if (msgCheckStatus == PutMessageStatus.LMQ_CONSUME_QUEUE_NUM_EXCEEDED) { + return CompletableFuture.completedFuture(new PutMessageResult(lmqMsgCheckStatus, null)); + } + + long beginTime = this.getSystemClock().now(); CompletableFuture putResultFuture = this.commitLog.asyncPutMessage(msg); @@ -448,7 +477,6 @@ public CompletableFuture asyncPutMessage(MessageExtBrokerInner return putResultFuture; } - public CompletableFuture asyncPutMessages(MessageExtBatch messageExtBatch) { PutMessageStatus checkStoreStatus = this.checkStoreStatus(); if (checkStoreStatus != PutMessageStatus.PUT_OK) { @@ -532,6 +560,11 @@ public GetMessageResult getMessage(final String group, final String topic, final return null; } + if (MixAll.isLmq(topic) && this.isLmqConsumeQueueNumExceeded()) { + log.warn("message store is not available, broker config enableLmq and enableMultiDispatch, lmq consumeQueue num exceed maxLmqConsumeQueueNum config num"); + return null; + } + long beginTime = this.getSystemClock().now(); GetMessageStatus status = GetMessageStatus.NO_MESSAGE_IN_QUEUE; @@ -1016,7 +1049,8 @@ public int cleanUnusedTopic(Set topics) { String topic = next.getKey(); if (!topics.contains(topic) && !topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC) - && !topic.equals(TopicValidator.RMQ_SYS_TRANS_OP_HALF_TOPIC)) { + && !topic.equals(TopicValidator.RMQ_SYS_TRANS_OP_HALF_TOPIC) + && !MixAll.isLmq(topic)) { ConcurrentMap queueTable = next.getValue(); for (ConsumeQueue cq : queueTable.values()) { cq.destroy(); @@ -1218,6 +1252,9 @@ public ConsumeQueue findConsumeQueue(String topic, int queueId) { if (oldLogic != null) { logic = oldLogic; } else { + if (MixAll.isLmq(topic)) { + lmqConsumeQueueNum.getAndIncrement(); + } logic = newLogic; } } @@ -1424,6 +1461,9 @@ private void putConsumeQueue(final String topic, final int queueId, final Consum map = new ConcurrentHashMap(); map.put(queueId, consumeQueue); this.consumeQueueTable.put(topic, map); + if (MixAll.isLmq(topic)) { + this.lmqConsumeQueueNum.getAndIncrement(); + } } else { map.put(queueId, consumeQueue); } @@ -1497,7 +1537,23 @@ public void doDispatch(DispatchRequest req) { public void putMessagePositionInfo(DispatchRequest dispatchRequest) { ConsumeQueue cq = this.findConsumeQueue(dispatchRequest.getTopic(), dispatchRequest.getQueueId()); - cq.putMessagePositionInfoWrapper(dispatchRequest); + cq.putMessagePositionInfoWrapper(dispatchRequest, checkMultiDispatchQueue(dispatchRequest)); + } + + private boolean checkMultiDispatchQueue(DispatchRequest dispatchRequest) { + if (!this.messageStoreConfig.isEnableMultiDispatch()) { + return false; + } + Map prop = dispatchRequest.getPropertiesMap(); + if (prop == null && prop.isEmpty()) { + return false; + } + String multiDispatchQueue = prop.get(MessageConst.PROPERTY_INNER_MULTI_DISPATCH); + String multiQueueOffset = prop.get(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET); + if (StringUtils.isBlank(multiDispatchQueue) || StringUtils.isBlank(multiQueueOffset)) { + return false; + } + return true; } @Override @@ -1517,6 +1573,29 @@ public void handleScheduleMessageService(final BrokerRole brokerRole) { } + @Override + public void cleanUnusedLmqTopic(String topic) { + if (this.consumeQueueTable.containsKey(topic)) { + ConcurrentMap map = this.consumeQueueTable.get(topic); + if (map != null) { + ConsumeQueue cq = map.get(0); + cq.destroy(); + log.info("cleanUnusedLmqTopic: {} {} ConsumeQueue cleaned", + cq.getTopic(), + cq.getQueueId() + ); + + this.commitLog.removeQueueFromTopicQueueTable(cq.getTopic(), cq.getQueueId()); + this.lmqConsumeQueueNum.getAndDecrement(); + } + this.consumeQueueTable.remove(topic); + if (this.brokerConfig.isAutoDeleteUnusedStats()) { + this.brokerStatsManager.onTopicDeleted(topic); + } + log.info("cleanUnusedLmqTopic: {},topic destroyed", topic); + } + } + public int remainTransientStoreBufferNumbs() { return this.transientStorePool.availableBufferNums(); } @@ -1974,6 +2053,7 @@ private void doReput() { dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1, dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(), dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap()); + notifyMessageArrive4MultiQueue(dispatchRequest); } this.reputFromOffset += size; @@ -2016,6 +2096,34 @@ private void doReput() { } } + private void notifyMessageArrive4MultiQueue(DispatchRequest dispatchRequest) { + Map prop = dispatchRequest.getPropertiesMap(); + if (prop == null) { + return; + } + String multiDispatchQueue = prop.get(MessageConst.PROPERTY_INNER_MULTI_DISPATCH); + String multiQueueOffset = prop.get(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET); + if (StringUtils.isBlank(multiDispatchQueue) || StringUtils.isBlank(multiQueueOffset)) { + return; + } + String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER); + String[] queueOffsets = multiQueueOffset.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER); + if (queues.length != queueOffsets.length) { + return; + } + for (int i = 0; i < queues.length; i++) { + String queueName = queues[i]; + long queueOffset = Long.parseLong(queueOffsets[i]); + int queueId = dispatchRequest.getQueueId(); + if (DefaultMessageStore.this.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queueName)) { + queueId = 0; + } + DefaultMessageStore.this.messageArrivingListener.arriving( + queueName, queueId, queueOffset + 1, dispatchRequest.getTagsCode(), + dispatchRequest.getStoreTimestamp(), dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap()); + } + } + @Override public void run() { DefaultMessageStore.log.info(this.getServiceName() + " service started"); diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java index a8c658bfe26..6771ede633a 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java @@ -397,4 +397,14 @@ QueryMessageResult queryMessage(final String topic, final String key, final int * @param brokerRole */ void handleScheduleMessageService(BrokerRole brokerRole); + + /** + * Clean unused lmq topic. + * When calling to clean up the lmq topic, + * the lmq topic cannot be used to write messages at the same time, + * otherwise the messages of the cleaning lmq topic may be lost, + * please call this method with caution + * @param topic lmq topic + */ + void cleanUnusedLmqTopic(String topic); } diff --git a/store/src/main/java/org/apache/rocketmq/store/MultiDispatch.java b/store/src/main/java/org/apache/rocketmq/store/MultiDispatch.java new file mode 100644 index 00000000000..679eed12344 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/MultiDispatch.java @@ -0,0 +1,157 @@ +/* + * 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; + +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.store.CommitLog.MessageExtEncoder; + +/** + * not-thread-safe + */ +public class MultiDispatch { + private static final InternalLogger LOGGER = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private final StringBuilder keyBuilder = new StringBuilder(); + private final DefaultMessageStore messageStore; + private final CommitLog commitLog; + + public MultiDispatch(DefaultMessageStore messageStore, CommitLog commitLog) { + this.messageStore = messageStore; + this.commitLog = commitLog; + } + + public String queueKey(String queueName, MessageExtBrokerInner msgInner) { + keyBuilder.setLength(0); + keyBuilder.append(queueName); + keyBuilder.append('-'); + int queueId = msgInner.getQueueId(); + if (messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queueName)) { + queueId = 0; + } + keyBuilder.append(queueId); + return keyBuilder.toString(); + } + + public boolean wrapMultiDispatch(final MessageExtBrokerInner msgInner) { + if (!messageStore.getMessageStoreConfig().isEnableMultiDispatch()) { + return true; + } + String multiDispatchQueue = msgInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH); + if (StringUtils.isBlank(multiDispatchQueue)) { + return true; + } + String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER); + Long[] queueOffsets = new Long[queues.length]; + for (int i = 0; i < queues.length; i++) { + String key = queueKey(queues[i], msgInner); + Long queueOffset; + try { + queueOffset = getTopicQueueOffset(key); + } catch (Exception e) { + return false; + } + if (null == queueOffset) { + queueOffset = 0L; + if (messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(key)) { + commitLog.getLmqTopicQueueTable().put(key, queueOffset); + } else { + commitLog.getTopicQueueTable().put(key, queueOffset); + } + } + queueOffsets[i] = queueOffset; + } + MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET, + StringUtils.join(queueOffsets, MixAll.MULTI_DISPATCH_QUEUE_SPLITTER)); + removeWaitStorePropertyString(msgInner); + return rebuildMsgInner(msgInner); + } + + private void removeWaitStorePropertyString(MessageExtBrokerInner msgInner) { + if (msgInner.getProperties().containsKey(MessageConst.PROPERTY_WAIT_STORE_MSG_OK)) { + // There is no need to store "WAIT=true", remove it from propertiesString to save 9 bytes for each message. + // It works for most case. In some cases msgInner.setPropertiesString invoked later and replace it. + String waitStoreMsgOKValue = msgInner.getProperties().remove(MessageConst.PROPERTY_WAIT_STORE_MSG_OK); + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); + // Reput to properties, since msgInner.isWaitStoreMsgOK() will be invoked later + msgInner.getProperties().put(MessageConst.PROPERTY_WAIT_STORE_MSG_OK, waitStoreMsgOKValue); + } else { + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); + } + } + + private boolean rebuildMsgInner(MessageExtBrokerInner msgInner) { + MessageExtEncoder encoder = this.commitLog.getPutMessageThreadLocal().get().getEncoder(); + PutMessageResult encodeResult = encoder.encode(msgInner); + if (encodeResult != null) { + LOGGER.error("rebuild msgInner for multiDispatch", encodeResult); + return false; + } + msgInner.setEncodedBuff(encoder.getEncoderBuffer()); + return true; + + } + + public void updateMultiQueueOffset(final MessageExtBrokerInner msgInner) { + if (!messageStore.getMessageStoreConfig().isEnableMultiDispatch()) { + return; + } + String multiDispatchQueue = msgInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH); + if (StringUtils.isBlank(multiDispatchQueue)) { + return; + } + String multiQueueOffset = msgInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET); + if (StringUtils.isBlank(multiQueueOffset)) { + LOGGER.error("[bug] no multiQueueOffset when updating {}", msgInner.getTopic()); + return; + } + String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER); + String[] queueOffsets = multiQueueOffset.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER); + if (queues.length != queueOffsets.length) { + LOGGER.error("[bug] num is not equal when updateMultiQueueOffset {}", msgInner.getTopic()); + return; + } + for (int i = 0; i < queues.length; i++) { + String key = queueKey(queues[i], msgInner); + long queueOffset = Long.parseLong(queueOffsets[i]); + if (messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(key)) { + commitLog.getLmqTopicQueueTable().put(key, ++queueOffset); + } else { + commitLog.getTopicQueueTable().put(key, ++queueOffset); + } + } + } + + private Long getTopicQueueOffset(String key) throws Exception { + Long offset = null; + if (messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(key)) { + Long queueNextOffset = commitLog.getLmqTopicQueueTable().get(key); + if (queueNextOffset != null) { + offset = queueNextOffset; + } + } else { + offset = commitLog.getTopicQueueTable().get(key); + } + return offset; + } + +} diff --git a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java b/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java index e1631d7d05c..29d0d95d9af 100644 --- a/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java +++ b/store/src/main/java/org/apache/rocketmq/store/PutMessageStatus.java @@ -27,4 +27,5 @@ public enum PutMessageStatus { PROPERTIES_SIZE_EXCEEDED, OS_PAGECACHE_BUSY, UNKNOWN_ERROR, + LMQ_CONSUME_QUEUE_NUM_EXCEEDED, } 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 e1439a0c9d8..1188f2191f7 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 @@ -159,6 +159,10 @@ public class MessageStoreConfig { private boolean enableScheduleMessageStats = true; + private boolean enableLmq = false; + private boolean enableMultiDispatch = false; + private int maxLmqConsumeQueueNum = 20000; + public boolean isDebugLockEnable() { return debugLockEnable; } @@ -744,4 +748,28 @@ public boolean isEnableScheduleMessageStats() { public void setEnableScheduleMessageStats(boolean enableScheduleMessageStats) { this.enableScheduleMessageStats = enableScheduleMessageStats; } + + public boolean isEnableLmq() { + return enableLmq; + } + + public void setEnableLmq(boolean enableLmq) { + this.enableLmq = enableLmq; + } + + public boolean isEnableMultiDispatch() { + return enableMultiDispatch; + } + + public void setEnableMultiDispatch(boolean enableMultiDispatch) { + this.enableMultiDispatch = enableMultiDispatch; + } + + public int getMaxLmqConsumeQueueNum() { + return maxLmqConsumeQueueNum; + } + + public void setMaxLmqConsumeQueueNum(int maxLmqConsumeQueueNum) { + this.maxLmqConsumeQueueNum = maxLmqConsumeQueueNum; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/stats/LmqBrokerStatsManager.java b/store/src/main/java/org/apache/rocketmq/store/stats/LmqBrokerStatsManager.java new file mode 100644 index 00000000000..f0e23fe6388 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/stats/LmqBrokerStatsManager.java @@ -0,0 +1,120 @@ +/* + * 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.stats; + +import org.apache.rocketmq.common.MixAll; + +public class LmqBrokerStatsManager extends BrokerStatsManager { + + public LmqBrokerStatsManager(String clusterName, boolean enableQueueStat) { + super(clusterName, enableQueueStat); + } + + @Override + public void incGroupGetNums(final String group, final String topic, final int incValue) { + String lmqGroup = group; + String lmqTopic = topic; + if (MixAll.isLmq(group)) { + lmqGroup = MixAll.LMQ_PREFIX; + } + if (MixAll.isLmq(topic)) { + lmqTopic = MixAll.LMQ_PREFIX; + } + super.incGroupGetNums(lmqGroup, lmqTopic, incValue); + } + + @Override + public void incGroupGetSize(final String group, final String topic, final int incValue) { + String lmqGroup = group; + String lmqTopic = topic; + if (MixAll.isLmq(group)) { + lmqGroup = MixAll.LMQ_PREFIX; + } + if (MixAll.isLmq(topic)) { + lmqTopic = MixAll.LMQ_PREFIX; + } + super.incGroupGetSize(lmqGroup, lmqTopic, incValue); + } + + @Override + public void incGroupGetLatency(final String group, final String topic, final int queueId, final int incValue) { + String lmqGroup = group; + String lmqTopic = topic; + if (MixAll.isLmq(group)) { + lmqGroup = MixAll.LMQ_PREFIX; + } + if (MixAll.isLmq(topic)) { + lmqTopic = MixAll.LMQ_PREFIX; + } + super.incGroupGetLatency(lmqGroup, lmqTopic, queueId, incValue); + } + + @Override + public void incSendBackNums(final String group, final String topic) { + String lmqGroup = group; + String lmqTopic = topic; + if (MixAll.isLmq(group)) { + lmqGroup = MixAll.LMQ_PREFIX; + } + if (MixAll.isLmq(topic)) { + lmqTopic = MixAll.LMQ_PREFIX; + } + super.incSendBackNums(lmqGroup, lmqTopic); + } + + @Override + public double tpsGroupGetNums(final String group, final String topic) { + String lmqGroup = group; + String lmqTopic = topic; + if (MixAll.isLmq(group)) { + lmqGroup = MixAll.LMQ_PREFIX; + } + if (MixAll.isLmq(topic)) { + lmqTopic = MixAll.LMQ_PREFIX; + } + return super.tpsGroupGetNums(lmqGroup, lmqTopic); + } + + @Override + public void recordDiskFallBehindTime(final String group, final String topic, final int queueId, + final long fallBehind) { + String lmqGroup = group; + String lmqTopic = topic; + if (MixAll.isLmq(group)) { + lmqGroup = MixAll.LMQ_PREFIX; + } + if (MixAll.isLmq(topic)) { + lmqTopic = MixAll.LMQ_PREFIX; + } + super.recordDiskFallBehindTime(lmqGroup, lmqTopic, queueId, fallBehind); + } + + @Override + public void recordDiskFallBehindSize(final String group, final String topic, final int queueId, + final long fallBehind) { + String lmqGroup = group; + String lmqTopic = topic; + if (MixAll.isLmq(group)) { + lmqGroup = MixAll.LMQ_PREFIX; + } + if (MixAll.isLmq(topic)) { + lmqTopic = MixAll.LMQ_PREFIX; + } + super.recordDiskFallBehindSize(lmqGroup, lmqTopic, queueId, fallBehind); + } + +} diff --git a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java index 7654e0a288d..668c069217d 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java @@ -27,6 +27,7 @@ import java.util.Map; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.stats.BrokerStatsManager; @@ -146,6 +147,34 @@ public void arriving(String topic, int queueId, long logicOffset, long tagsCode, return master; } + protected DefaultMessageStore genForMultiQueue() throws Exception { + MessageStoreConfig messageStoreConfig = buildStoreConfig( + commitLogFileSize, cqFileSize, true, cqExtFileSize + ); + + messageStoreConfig.setEnableLmq(true); + messageStoreConfig.setEnableMultiDispatch(true); + + BrokerConfig brokerConfig = new BrokerConfig(); + + DefaultMessageStore master = new DefaultMessageStore( + messageStoreConfig, + new BrokerStatsManager(brokerConfig.getBrokerClusterName(), brokerConfig.isEnableDetailStat()), + new MessageArrivingListener() { + @Override + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, + long msgStoreTime, byte[] filterBitMap, Map properties) { + } + } + , brokerConfig); + + assertThat(master.load()).isTrue(); + + master.start(); + + return master; + } + protected void putMsg(DefaultMessageStore master) throws Exception { long totalMsgs = 200; @@ -158,6 +187,33 @@ protected void putMsg(DefaultMessageStore master) throws Exception { } } + protected void putMsgMultiQueue(DefaultMessageStore master) throws Exception { + for (long i = 0; i < 1; i++) { + master.putMessage(buildMessageMultiQueue()); + } + } + + private MessageExtBrokerInner buildMessageMultiQueue() { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + msg.setTopic(topic); + msg.setTags("TAG1"); + msg.setKeys("Hello"); + msg.setBody(msgBody); + msg.setKeys(String.valueOf(System.currentTimeMillis())); + msg.setQueueId(queueId); + msg.setSysFlag(0); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setStoreHost(StoreHost); + msg.setBornHost(BornHost); + for (int i = 0; i < 1; i++) { + msg.putUserProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH, "%LMQ%123,%LMQ%456"); + msg.putUserProperty(String.valueOf(i), "imagoodperson" + i); + } + msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties())); + + return msg; + } + protected void deleteDirectory(String rootPath) { File file = new File(rootPath); deleteFile(file); @@ -217,6 +273,89 @@ public void testPutMessagePositionInfo_buildCQRepeatedly() throws Exception { } + @Test + public void testPutMessagePositionInfoWrapper_MultiQueue() throws Exception { + DefaultMessageStore messageStore = null; + try { + messageStore = genForMultiQueue(); + + + int totalMessages = 10; + + for (int i = 0; i < totalMessages; i++) { + putMsgMultiQueue(messageStore); + } + Thread.sleep(5); + + ConsumeQueue cq = messageStore.getConsumeQueueTable().get(topic).get(queueId); + Method method = cq.getClass().getDeclaredMethod("putMessagePositionInfoWrapper", DispatchRequest.class, boolean.class); + + assertThat(method).isNotNull(); + + method.setAccessible(true); + + SelectMappedBufferResult result = messageStore.getCommitLog().getData(0); + assertThat(result != null).isTrue(); + + DispatchRequest dispatchRequest = messageStore.getCommitLog().checkMessageAndReturnSize(result.getByteBuffer(), false, false); + + assertThat(cq).isNotNull(); + + Object dispatchResult = method.invoke(cq, dispatchRequest, true); + + ConsumeQueue lmqCq1 = messageStore.getConsumeQueueTable().get("%LMQ%123").get(0); + + ConsumeQueue lmqCq2 = messageStore.getConsumeQueueTable().get("%LMQ%456").get(0); + + assertThat(lmqCq1).isNotNull(); + + assertThat(lmqCq2).isNotNull(); + + } finally { + if (messageStore != null) { + messageStore.shutdown(); + messageStore.destroy(); + } + deleteDirectory(storePath); + } + + } + + @Test + public void testPutMessagePositionInfoMultiQueue() throws Exception { + DefaultMessageStore messageStore = null; + try { + + messageStore = genForMultiQueue(); + + int totalMessages = 10; + + for (int i = 0; i < totalMessages; i++) { + putMsgMultiQueue(messageStore); + } + Thread.sleep(5); + + ConsumeQueue cq = messageStore.getConsumeQueueTable().get(topic).get(queueId); + + ConsumeQueue lmqCq1 = messageStore.getConsumeQueueTable().get("%LMQ%123").get(0); + + ConsumeQueue lmqCq2 = messageStore.getConsumeQueueTable().get("%LMQ%456").get(0); + + assertThat(cq).isNotNull(); + + assertThat(lmqCq1).isNotNull(); + + assertThat(lmqCq2).isNotNull(); + + } finally { + if (messageStore != null) { + messageStore.shutdown(); + messageStore.destroy(); + } + deleteDirectory(storePath); + } + } + @Test public void testConsumeQueueWithExtendData() { DefaultMessageStore master = null; diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java index 96451e3c6bf..b565c5c6671 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.store.config.FlushDiskType; import org.apache.rocketmq.store.config.MessageStoreConfig; @@ -641,6 +642,21 @@ private void damageCommitlog(long offset) throws Exception { fileChannel.close(); } + @Test + public void testCleanUnusedLmqTopic() throws Exception { + String lmqTopic = "%LMQ%123"; + + MessageExtBrokerInner messageExtBrokerInner = buildMessage(); + messageExtBrokerInner.setTopic("test"); + messageExtBrokerInner.setQueueId(0); + messageExtBrokerInner.getProperties().put(MessageConst.PROPERTY_INNER_MULTI_DISPATCH, lmqTopic); + messageStore.putMessage(messageExtBrokerInner); + + Thread.sleep(3000); + messageStore.cleanUnusedLmqTopic(lmqTopic); + + } + private class MyMessageArrivingListener implements MessageArrivingListener { @Override public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime, diff --git a/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java b/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java new file mode 100644 index 00000000000..45e4d06652d --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java @@ -0,0 +1,98 @@ +/* + * 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; + +import java.io.File; +import java.net.InetSocketAddress; +import java.nio.charset.Charset; + +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class MultiDispatchTest { + + private CommitLog commitLog; + private MultiDispatch multiDispatch; + + @Before + public void init() throws Exception { + MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); + messageStoreConfig.setMappedFileSizeCommitLog(1024 * 8); + messageStoreConfig.setMappedFileSizeConsumeQueue(1024 * 4); + messageStoreConfig.setMaxHashSlotNum(100); + messageStoreConfig.setMaxIndexNum(100 * 10); + messageStoreConfig.setStorePathRootDir(System.getProperty("user.home") + File.separator + "unitteststore1"); + messageStoreConfig.setStorePathCommitLog( + System.getProperty("user.home") + File.separator + "unitteststore1" + File.separator + "commitlog"); + + messageStoreConfig.setEnableLmq(true); + messageStoreConfig.setEnableMultiDispatch(true); + //too much reference + DefaultMessageStore messageStore = new DefaultMessageStore(messageStoreConfig, null, null, null); + this.commitLog = new CommitLog(messageStore); + this.multiDispatch = new MultiDispatch(messageStore, commitLog); + } + + @After + public void destroy() { + UtilAll.deleteFile(new File(System.getProperty("user.home") + File.separator + "unitteststore1")); + } + + @Test + public void queueKey() { + MessageExtBrokerInner messageExtBrokerInner = mock(MessageExtBrokerInner.class); + when(messageExtBrokerInner.getQueueId()).thenReturn(2); + String ret = multiDispatch.queueKey("%LMQ%lmq123", messageExtBrokerInner); + assertEquals(ret, "%LMQ%lmq123-0"); + } + + @Test + public void wrapMultiDispatch() { + MessageExtBrokerInner messageExtBrokerInner = mock(MessageExtBrokerInner.class); + when(messageExtBrokerInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH)).thenReturn( + "%LMQ%123,%LMQ%456"); + when(messageExtBrokerInner.getTopic()).thenReturn("test"); + when(messageExtBrokerInner.getBody()).thenReturn("aaa".getBytes(Charset.forName("UTF-8"))); + when(messageExtBrokerInner.getBornHost()).thenReturn(new InetSocketAddress("127.0.0.1", 54270)); + when(messageExtBrokerInner.getStoreHost()).thenReturn(new InetSocketAddress("127.0.0.1", 10911)); + multiDispatch.wrapMultiDispatch(messageExtBrokerInner); + assertTrue(commitLog.getLmqTopicQueueTable().size() == 2); + assertTrue(commitLog.getLmqTopicQueueTable().get("%LMQ%123-0") == 0L); + assertTrue(commitLog.getLmqTopicQueueTable().get("%LMQ%456-0") == 0L); + } + + @Test + public void updateMultiQueueOffset() { + MessageExtBrokerInner messageExtBrokerInner = mock(MessageExtBrokerInner.class); + when(messageExtBrokerInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH)).thenReturn("%LMQ%123,%LMQ%456"); + when(messageExtBrokerInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET)).thenReturn("0,1"); + multiDispatch.updateMultiQueueOffset(messageExtBrokerInner); + assertTrue(commitLog.getLmqTopicQueueTable().size() == 2); + assertTrue(commitLog.getLmqTopicQueueTable().get("%LMQ%123-0") == 1L); + assertTrue(commitLog.getLmqTopicQueueTable().get("%LMQ%456-0") == 2L); + } +} \ No newline at end of file diff --git a/test/pom.xml b/test/pom.xml index 93ff590cb0b..9d8336e050d 100644 --- a/test/pom.xml +++ b/test/pom.xml @@ -45,6 +45,27 @@ truth 0.30 + + org.mockito + mockito-core + 2.28.2 + test + + + junit + junit + 4.12 + test + + + ${project.groupId} + rocketmq-client + + + ${project.groupId} + rocketmq-tools + + diff --git a/test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java b/test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java new file mode 100644 index 00000000000..392b306734e --- /dev/null +++ b/test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java @@ -0,0 +1,305 @@ +/* + * 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.test.lmq.benchmark; + +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer; +import org.apache.rocketmq.client.consumer.PullCallback; +import org.apache.rocketmq.client.consumer.PullResult; +import org.apache.rocketmq.client.consumer.PullStatus; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.header.QueryConsumerOffsetRequestHeader; +import org.apache.rocketmq.common.protocol.header.UpdateConsumerOffsetRequestHeader; +import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.test.util.StatUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +public class BenchLmqStore { + private static Logger logger = LoggerFactory.getLogger(BenchLmqStore.class); + private static String namesrv = System.getProperty("namesrv", "127.0.0.1:9876"); + private static String lmqTopic = System.getProperty("lmqTopic", "lmqTestTopic"); + private static boolean enableSub = Boolean.parseBoolean(System.getProperty("enableSub", "true")); + private static String queuePrefix = System.getProperty("queuePrefix", "lmqTest"); + private static int tps = Integer.parseInt(System.getProperty("tps", "1")); + private static int lmqNum = Integer.parseInt(System.getProperty("lmqNum", "1")); + private static int sendThreadNum = Integer.parseInt(System.getProperty("sendThreadNum", "64")); + private static int consumerThreadNum = Integer.parseInt(System.getProperty("consumerThreadNum", "64")); + private static String brokerName = System.getProperty("brokerName", "broker-a"); + private static int size = Integer.parseInt(System.getProperty("size", "128")); + private static int suspendTime = Integer.parseInt(System.getProperty("suspendTime", "2000")); + private static final boolean RETRY_NO_MATCHED_MSG = Boolean.parseBoolean(System.getProperty("retry_no_matched_msg", "false")); + private static boolean benchOffset = Boolean.parseBoolean(System.getProperty("benchOffset", "false")); + private static int benchOffsetNum = Integer.parseInt(System.getProperty("benchOffsetNum", "1")); + private static Map offsetMap = new ConcurrentHashMap<>(256); + private static Map pullStatus = new ConcurrentHashMap<>(256); + private static Map> pullEvent = new ConcurrentHashMap<>(256); + public static DefaultMQProducer defaultMQProducer; + private static int pullConsumerNum = Integer.parseInt(System.getProperty("pullConsumerNum", "8")); + public static DefaultMQPullConsumer[] defaultMQPullConsumers = new DefaultMQPullConsumer[pullConsumerNum]; + private static AtomicLong rid = new AtomicLong(); + private static final String LMQ_PREFIX = "%LMQ%"; + + public static void main(String[] args) throws InterruptedException, MQClientException, MQBrokerException, + RemotingException { + defaultMQProducer = new DefaultMQProducer(); + defaultMQProducer.setProducerGroup("PID_LMQ_TEST"); + defaultMQProducer.setVipChannelEnabled(false); + defaultMQProducer.setNamesrvAddr(namesrv); + defaultMQProducer.start(); + //defaultMQProducer.createTopic(lmqTopic, lmqTopic, 8); + for (int i = 0; i < pullConsumerNum; i++) { + DefaultMQPullConsumer defaultMQPullConsumer = new DefaultMQPullConsumer(); + defaultMQPullConsumers[i] = defaultMQPullConsumer; + defaultMQPullConsumer.setNamesrvAddr(namesrv); + defaultMQPullConsumer.setVipChannelEnabled(false); + defaultMQPullConsumer.setConsumerGroup("CID_RMQ_SYS_LMQ_TEST_" + i); + defaultMQPullConsumer.setInstanceName("CID_RMQ_SYS_LMQ_TEST_" + i); + defaultMQPullConsumer.setRegisterTopics(new HashSet<>(Arrays.asList(lmqTopic))); + defaultMQPullConsumer.setBrokerSuspendMaxTimeMillis(suspendTime); + defaultMQPullConsumer.setConsumerTimeoutMillisWhenSuspend(suspendTime + 1000); + defaultMQPullConsumer.start(); + } + Thread.sleep(3000L); + if (benchOffset) { + doBenchOffset(); + return; + } + ScheduledThreadPoolExecutor consumerPool = new ScheduledThreadPoolExecutor(consumerThreadNum, new ThreadFactoryImpl("test")); + for (int i = 0; i < consumerThreadNum; i++) { + final int idx = i; + consumerPool.scheduleWithFixedDelay(() -> { + try { + Map map = pullEvent.get(idx); + if (map == null) { + return; + } + for (Map.Entry entry : map.entrySet()) { + try { + Boolean status = pullStatus.get(entry.getKey()); + if (Boolean.TRUE.equals(status)) { + continue; + } + doPull(map, entry.getKey(), entry.getValue()); + } catch (Exception e) { + logger.error("pull broker msg error", e); + } + } + } catch (Exception e) { + logger.error("exec doPull task error", e); + } + }, 1, 1, TimeUnit.MILLISECONDS); + } + // init queue sub + if (enableSub && lmqNum > 0 && StringUtils.isNotBlank(brokerName)) { + for (int i = 0; i < lmqNum; i++) { + long idx = rid.incrementAndGet(); + String queue = LMQ_PREFIX + queuePrefix + idx % lmqNum; + MessageQueue mq = new MessageQueue(queue, brokerName, 0); + int queueHash = Math.abs(queue.hashCode()) % consumerThreadNum; + pullEvent.putIfAbsent(queueHash, new ConcurrentHashMap<>()); + pullEvent.get(queueHash).put(mq, idx); + } + } + Thread.sleep(5000L); + doSend(); + } + public static void doSend() { + StringBuilder sb = new StringBuilder(); + for (int j = 0; j < size; j += 10) { + sb.append("hello baby"); + } + byte[] body = sb.toString().getBytes(); + String pubKey = "pub"; + ExecutorService sendPool = Executors.newFixedThreadPool(sendThreadNum); + for (int i = 0; i < sendThreadNum; i++) { + sendPool.execute(() -> { + while (true) { + if (StatUtil.isOverFlow(pubKey, tps)) { + try { + Thread.sleep(100L); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + long start = System.currentTimeMillis(); + try { + long idx = rid.incrementAndGet(); + Message message = new Message(lmqTopic, body); + String queue = lmqTopic; + if (lmqNum > 0) { + queue = LMQ_PREFIX + queuePrefix + idx % lmqNum; + message.putUserProperty("INNER_MULTI_DISPATCH", queue); + } + SendResult sendResult = defaultMQProducer.send(message); + StatUtil.addInvoke(pubKey, System.currentTimeMillis() - start); + if (StatUtil.nowTps(pubKey) < 10) { + logger.warn("pub: {} ", sendResult.getMsgId()); + } + if (enableSub) { + MessageQueue mq = new MessageQueue(queue, sendResult.getMessageQueue().getBrokerName(), + lmqNum > 0 ? 0 : sendResult.getMessageQueue().getQueueId()); + int queueHash = Math.abs(queue.hashCode()) % consumerThreadNum; + pullEvent.putIfAbsent(queueHash, new ConcurrentHashMap<>()); + pullEvent.get(queueHash).put(mq, idx); + } + } catch (Exception e) { + logger.error("", e); + StatUtil.addInvoke(pubKey, System.currentTimeMillis() - start, false); + } + } + }); + } + } + public static void doPull(Map eventMap, MessageQueue mq, Long eventId) throws RemotingException, InterruptedException, MQClientException { + if (!enableSub) { + eventMap.remove(mq, eventId); + pullStatus.remove(mq); + return; + } + DefaultMQPullConsumer defaultMQPullConsumer = defaultMQPullConsumers[(int) (eventId % pullConsumerNum)]; + Long offset = offsetMap.get(mq); + if (offset == null) { + long start = System.currentTimeMillis(); + offset = defaultMQPullConsumer.maxOffset(mq); + StatUtil.addInvoke("maxOffset", System.currentTimeMillis() - start); + offsetMap.put(mq, offset); + } + long start = System.currentTimeMillis(); + if (null != pullStatus.putIfAbsent(mq, true)) { + return; + } + defaultMQPullConsumer.pullBlockIfNotFound( + mq, "*", offset, 32, + new PullCallback() { + @Override + public void onSuccess(PullResult pullResult) { + StatUtil.addInvoke(pullResult.getPullStatus().name(), System.currentTimeMillis() - start); + eventMap.remove(mq, eventId); + pullStatus.remove(mq); + offsetMap.put(mq, pullResult.getNextBeginOffset()); + StatUtil.addInvoke("doPull", System.currentTimeMillis() - start); + if (PullStatus.NO_MATCHED_MSG.equals(pullResult.getPullStatus()) && RETRY_NO_MATCHED_MSG) { + long idx = rid.incrementAndGet(); + eventMap.put(mq, idx); + } + List list = pullResult.getMsgFoundList(); + if (list == null || list.isEmpty()) { + StatUtil.addInvoke("NoMsg", System.currentTimeMillis() - start); + return; + } + for (MessageExt messageExt : list) { + StatUtil.addInvoke("sub", System.currentTimeMillis() - messageExt.getBornTimestamp()); + if (StatUtil.nowTps("sub") < 10) { + logger.warn("sub: {}", messageExt.getMsgId()); + } + } + } + @Override + public void onException(Throwable e) { + eventMap.remove(mq, eventId); + pullStatus.remove(mq); + logger.error("", e); + StatUtil.addInvoke("doPull", System.currentTimeMillis() - start, false); + } + }); + } + public static void doBenchOffset() throws RemotingException, InterruptedException, MQClientException { + ExecutorService sendPool = Executors.newFixedThreadPool(sendThreadNum); + Map offsetMap = new ConcurrentHashMap<>(); + String statKey = "benchOffset"; + TopicRouteData topicRouteData = defaultMQPullConsumers[0].getDefaultMQPullConsumerImpl(). + getRebalanceImpl().getMqClientFactory().getMQClientAPIImpl(). + getTopicRouteInfoFromNameServer(lmqTopic, 3000); + HashMap brokerMap = topicRouteData.getBrokerDatas().get(0).getBrokerAddrs(); + if (brokerMap == null || brokerMap.isEmpty()) { + return; + } + String brokerAddress = brokerMap.get(MixAll.MASTER_ID); + for (int i = 0; i < sendThreadNum; i++) { + final int flag = i; + sendPool.execute(new Runnable() { + @Override + public void run() { + while (true) { + try { + if (StatUtil.isOverFlow(statKey, tps)) { + Thread.sleep(100L); + } + long start = System.currentTimeMillis(); + DefaultMQPullConsumer defaultMQPullConsumer = defaultMQPullConsumers[(int) (rid.incrementAndGet() % pullConsumerNum)]; + long id = rid.incrementAndGet(); + String lmq = LMQ_PREFIX + queuePrefix + id % benchOffsetNum; + String lmqCid = LMQ_PREFIX + "GID_LMQ@@c" + flag + "-" + id % benchOffsetNum; + Long offset = offsetMap.get(lmq); + if (offset == null) { + offsetMap.put(lmq, 0L); + } + long newOffset1 = offsetMap.get(lmq) + 1; + UpdateConsumerOffsetRequestHeader updateHeader = new UpdateConsumerOffsetRequestHeader(); + updateHeader.setTopic(lmq); + updateHeader.setConsumerGroup(lmqCid); + updateHeader.setQueueId(0); + updateHeader.setCommitOffset(newOffset1); + defaultMQPullConsumer + .getDefaultMQPullConsumerImpl() + .getRebalanceImpl() + .getMqClientFactory() + .getMQClientAPIImpl().updateConsumerOffset(brokerAddress, updateHeader, 1000); + QueryConsumerOffsetRequestHeader queryHeader = new QueryConsumerOffsetRequestHeader(); + queryHeader.setTopic(lmq); + queryHeader.setConsumerGroup(lmqCid); + queryHeader.setQueueId(0); + long newOffset2 = defaultMQPullConsumer + .getDefaultMQPullConsumerImpl() + .getRebalanceImpl() + .getMqClientFactory() + .getMQClientAPIImpl() + .queryConsumerOffset(brokerAddress, queryHeader, 1000); + offsetMap.put(lmq, newOffset2); + if (newOffset1 != newOffset2) { + StatUtil.addInvoke("ErrorOffset", 1); + } + StatUtil.addInvoke(statKey, System.currentTimeMillis() - start); + } catch (Exception e) { + logger.error("", e); + } + } + } + }); + } + } +} \ No newline at end of file diff --git a/test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java b/test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java new file mode 100644 index 00000000000..5645d66dac3 --- /dev/null +++ b/test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java @@ -0,0 +1,478 @@ +/* + * 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.test.util; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import javax.annotation.Generated; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.math.BigDecimal.ROUND_HALF_UP; + +@Generated("StatUtil") +public class StatUtil { + private static Logger sysLogger = LoggerFactory.getLogger(StatUtil.class); + private static Logger logger = LoggerFactory.getLogger("StatLogger"); + private static final int MAX_KEY_NUM = Integer.parseInt(System.getProperty("stat.util.key.max.num", "10000")); + private static volatile ConcurrentMap invokeCache = new ConcurrentHashMap<>(64); + private static volatile ConcurrentMap> secondInvokeCache = new ConcurrentHashMap<>( + 64); + + private static final int STAT_WINDOW_SECONDS = Integer.parseInt(System.getProperty("stat.win.seconds", "60")); + private static final String SPLITTER = "|"; + private static ScheduledExecutorService daemon = Executors.newSingleThreadScheduledExecutor(); + + static class Invoke { + AtomicLong totalPv = new AtomicLong(); + AtomicLong failPv = new AtomicLong(); + AtomicLong sumRt = new AtomicLong(); + AtomicLong maxRt = new AtomicLong(); + AtomicLong minRt = new AtomicLong(); + AtomicInteger topSecondPv = new AtomicInteger(); + AtomicInteger secondPv = new AtomicInteger(); + AtomicLong second = new AtomicLong(System.currentTimeMillis() / 1000L); + } + + static class SecondInvoke implements Comparable { + AtomicLong total = new AtomicLong(); + AtomicLong fail = new AtomicLong(); + AtomicLong sumRt = new AtomicLong(); + AtomicLong maxRt = new AtomicLong(); + AtomicLong minRt = new AtomicLong(); + Long second = nowSecond(); + + @Override + public int compareTo(SecondInvoke o) { + return o.second.compareTo(second); + } + } + + static { + daemon.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + printInvokeStat(); + printSecondInvokeStat(); + } catch (Exception e) { + logger.error("", e); + } + } + }, STAT_WINDOW_SECONDS, STAT_WINDOW_SECONDS, TimeUnit.SECONDS); + } + + private static void printInvokeStat() { + Map tmp = invokeCache; + invokeCache = new ConcurrentHashMap<>(64); + + sysLogger.warn("printInvokeStat key count:{}", tmp.size()); + for (Map.Entry entry : tmp.entrySet()) { + String key = entry.getKey(); + Invoke invoke = entry.getValue(); + logger.warn("{}", + buildLog(key, invoke.topSecondPv.get(), invoke.totalPv.get(), invoke.failPv.get(), invoke.minRt.get(), + invoke.maxRt.get(), invoke.sumRt.get())); + } + } + + private static void printSecondInvokeStat() { + sysLogger.warn("printSecondInvokeStat key count:{}", secondInvokeCache.size()); + for (Map.Entry> entry : secondInvokeCache.entrySet()) { + String key = entry.getKey(); + Map secondInvokeMap = entry.getValue(); + long totalPv = 0L; + long failPv = 0L; + long topSecondPv = 0L; + long sumRt = 0L; + long maxRt = 0L; + long minRt = 0L; + + for (Map.Entry invokeEntry : secondInvokeMap.entrySet()) { + long second = invokeEntry.getKey(); + SecondInvoke secondInvoke = invokeEntry.getValue(); + if (nowSecond() - second >= STAT_WINDOW_SECONDS) { + secondInvokeMap.remove(second); + continue; + } + long secondPv = secondInvoke.total.get(); + totalPv += secondPv; + failPv += secondInvoke.fail.get(); + sumRt += secondInvoke.sumRt.get(); + if (maxRt < secondInvoke.maxRt.get()) { + maxRt = secondInvoke.maxRt.get(); + } + if (minRt > secondInvoke.minRt.get()) { + minRt = secondInvoke.minRt.get(); + } + if (topSecondPv < secondPv) { + topSecondPv = secondPv; + } + } + if (secondInvokeMap.isEmpty()) { + secondInvokeCache.remove(key); + continue; + } + logger.warn("{}", buildLog(key, topSecondPv, totalPv, failPv, minRt, maxRt, sumRt)); + } + } + + private static String buildLog(String key, long topSecondPv, long totalPv, long failPv, long minRt, long maxRt, + long sumRt) { + StringBuilder sb = new StringBuilder(); + sb.append(SPLITTER); + sb.append(key); + sb.append(SPLITTER); + sb.append(topSecondPv); + sb.append(SPLITTER); + int tps = new BigDecimal(totalPv).divide(new BigDecimal(STAT_WINDOW_SECONDS), + ROUND_HALF_UP).intValue(); + sb.append(tps); + sb.append(SPLITTER); + sb.append(totalPv); + sb.append(SPLITTER); + sb.append(failPv); + sb.append(SPLITTER); + sb.append(minRt); + sb.append(SPLITTER); + long avg = new BigDecimal(sumRt).divide(new BigDecimal(totalPv), + ROUND_HALF_UP).longValue(); + sb.append(avg); + sb.append(SPLITTER); + sb.append(maxRt); + return sb.toString(); + } + + public static String buildKey(String... keys) { + if (keys == null || keys.length <= 0) { + return null; + } + StringBuilder sb = new StringBuilder(); + for (String key : keys) { + sb.append(key); + sb.append(","); + } + sb.deleteCharAt(sb.length() - 1); + return sb.toString(); + } + + public static void addInvoke(String key, long rt) { + addInvoke(key, rt, true); + } + + private static Invoke getAndSetInvoke(String key) { + Invoke invoke = invokeCache.get(key); + if (invoke == null) { + invokeCache.putIfAbsent(key, new Invoke()); + } + return invokeCache.get(key); + } + + public static void addInvoke(String key, int num, long rt, boolean success) { + if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) { + return; + } + Invoke invoke = getAndSetInvoke(key); + if (invoke == null) { + return; + } + + invoke.totalPv.getAndAdd(num); + if (!success) { + invoke.failPv.getAndAdd(num); + } + long now = nowSecond(); + AtomicLong oldSecond = invoke.second; + if (oldSecond.get() == now) { + invoke.secondPv.getAndAdd(num); + } else { + if (oldSecond.compareAndSet(oldSecond.get(), now)) { + if (invoke.secondPv.get() > invoke.topSecondPv.get()) { + invoke.topSecondPv.set(invoke.secondPv.get()); + } + invoke.secondPv.set(num); + } else { + invoke.secondPv.getAndAdd(num); + } + } + + invoke.sumRt.addAndGet(rt); + if (invoke.maxRt.get() < rt) { + invoke.maxRt.set(rt); + } + if (invoke.minRt.get() > rt) { + invoke.minRt.set(rt); + } + } + + public static void addInvoke(String key, long rt, boolean success) { + if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) { + return; + } + Invoke invoke = getAndSetInvoke(key); + if (invoke == null) { + return; + } + + invoke.totalPv.getAndIncrement(); + if (!success) { + invoke.failPv.getAndIncrement(); + } + long now = nowSecond(); + AtomicLong oldSecond = invoke.second; + if (oldSecond.get() == now) { + invoke.secondPv.getAndIncrement(); + } else { + if (oldSecond.compareAndSet(oldSecond.get(), now)) { + if (invoke.secondPv.get() > invoke.topSecondPv.get()) { + invoke.topSecondPv.set(invoke.secondPv.get()); + } + invoke.secondPv.set(1); + } else { + invoke.secondPv.getAndIncrement(); + } + } + + invoke.sumRt.addAndGet(rt); + if (invoke.maxRt.get() < rt) { + invoke.maxRt.set(rt); + } + if (invoke.minRt.get() > rt) { + invoke.minRt.set(rt); + } + } + + public static SecondInvoke getAndSetSecondInvoke(String key) { + if (!secondInvokeCache.containsKey(key)) { + secondInvokeCache.putIfAbsent(key, new ConcurrentHashMap<>(STAT_WINDOW_SECONDS)); + } + Map secondInvokeMap = secondInvokeCache.get(key); + if (secondInvokeMap == null) { + return null; + } + long second = nowSecond(); + if (!secondInvokeMap.containsKey(second)) { + secondInvokeMap.putIfAbsent(second, new SecondInvoke()); + } + return secondInvokeMap.get(second); + } + + public static void addSecondInvoke(String key, long rt) { + addSecondInvoke(key, rt, true); + } + + public static void addSecondInvoke(String key, long rt, boolean success) { + if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) { + return; + } + SecondInvoke secondInvoke = getAndSetSecondInvoke(key); + if (secondInvoke == null) { + return; + } + secondInvoke.total.addAndGet(1); + if (!success) { + secondInvoke.fail.addAndGet(1); + } + secondInvoke.sumRt.addAndGet(rt); + if (secondInvoke.maxRt.get() < rt) { + secondInvoke.maxRt.set(rt); + } + if (secondInvoke.minRt.get() > rt) { + secondInvoke.minRt.set(rt); + } + } + + public static void addPv(String key, long totalPv) { + addPv(key, totalPv, true); + } + + public static void addPv(String key, long totalPv, boolean success) { + if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) { + return; + } + if (totalPv <= 0) { + return; + } + Invoke invoke = getAndSetInvoke(key); + if (invoke == null) { + return; + } + invoke.totalPv.addAndGet(totalPv); + if (!success) { + invoke.failPv.addAndGet(totalPv); + } + long now = nowSecond(); + AtomicLong oldSecond = invoke.second; + if (oldSecond.get() == now) { + invoke.secondPv.addAndGet((int)totalPv); + } else { + if (oldSecond.compareAndSet(oldSecond.get(), now)) { + if (invoke.secondPv.get() > invoke.topSecondPv.get()) { + invoke.topSecondPv.set(invoke.secondPv.get()); + } + invoke.secondPv.set((int)totalPv); + } else { + invoke.secondPv.addAndGet((int)totalPv); + } + } + } + + public static void addSecondPv(String key, long totalPv) { + addSecondPv(key, totalPv, true); + } + + public static void addSecondPv(String key, long totalPv, boolean success) { + if (invokeCache.size() > MAX_KEY_NUM || secondInvokeCache.size() > MAX_KEY_NUM) { + return; + } + if (totalPv <= 0) { + return; + } + SecondInvoke secondInvoke = getAndSetSecondInvoke(key); + if (secondInvoke == null) { + return; + } + secondInvoke.total.addAndGet(totalPv); + if (!success) { + secondInvoke.fail.addAndGet(totalPv); + } + } + + public static boolean isOverFlow(String key, int tps) { + return nowTps(key) >= tps; + } + + public static int nowTps(String key) { + Map secondInvokeMap = secondInvokeCache.get(key); + if (secondInvokeMap != null) { + SecondInvoke secondInvoke = secondInvokeMap.get(nowSecond()); + if (secondInvoke != null) { + return (int)secondInvoke.total.get(); + } + } + Invoke invoke = invokeCache.get(key); + if (invoke == null) { + return 0; + } + AtomicLong oldSecond = invoke.second; + if (oldSecond.get() == nowSecond()) { + return invoke.secondPv.get(); + } + return 0; + } + + public static int totalPvInWindow(String key, int windowSeconds) { + List list = secondInvokeList(key, windowSeconds); + long totalPv = 0; + for (int i = 0; i < windowSeconds && i < list.size(); i++) { + totalPv += list.get(i).total.get(); + } + return (int)totalPv; + } + + public static int failPvInWindow(String key, int windowSeconds) { + List list = secondInvokeList(key, windowSeconds); + long failPv = 0; + for (int i = 0; i < windowSeconds && i < list.size(); i++) { + failPv += list.get(i).fail.get(); + } + return (int)failPv; + } + + public static int topTpsInWindow(String key, int windowSeconds) { + List list = secondInvokeList(key, windowSeconds); + long topTps = 0; + for (int i = 0; i < windowSeconds && i < list.size(); i++) { + long secondPv = list.get(i).total.get(); + if (topTps < secondPv) { + topTps = secondPv; + } + } + return (int)topTps; + } + + public static int avgRtInWindow(String key, int windowSeconds) { + List list = secondInvokeList(key, windowSeconds); + long sumRt = 0; + long totalPv = 0; + for (int i = 0; i < windowSeconds && i < list.size(); i++) { + sumRt += list.get(i).sumRt.get(); + totalPv += list.get(i).total.get(); + } + if (totalPv <= 0) { + return 0; + } + long avg = new BigDecimal(sumRt).divide(new BigDecimal(totalPv), + ROUND_HALF_UP).longValue(); + return (int)avg; + } + + public static int maxRtInWindow(String key, int windowSeconds) { + List list = secondInvokeList(key, windowSeconds); + long maxRt = 0; + long totalPv = 0; + for (int i = 0; i < windowSeconds && i < list.size(); i++) { + if (maxRt < list.get(i).maxRt.get()) { + maxRt = list.get(i).maxRt.get(); + } + } + return (int)maxRt; + } + + public static int minRtInWindow(String key, int windowSeconds) { + List list = secondInvokeList(key, windowSeconds); + long minRt = 0; + long totalPv = 0; + for (int i = 0; i < windowSeconds && i < list.size(); i++) { + if (minRt < list.get(i).minRt.get()) { + minRt = list.get(i).minRt.get(); + } + } + return (int)minRt; + } + + private static List secondInvokeList(String key, int windowSeconds) { + if (windowSeconds > STAT_WINDOW_SECONDS || windowSeconds <= 0) { + throw new IllegalArgumentException("windowSeconds Must Not be great than " + STAT_WINDOW_SECONDS); + } + Map secondInvokeMap = secondInvokeCache.get(key); + if (secondInvokeMap == null || secondInvokeMap.isEmpty()) { + return new ArrayList<>(); + } + List list = new ArrayList<>(); + list.addAll(secondInvokeMap.values()); + Collections.sort(list); + return list; + } + + private static long nowSecond() { + return System.currentTimeMillis() / 1000L; + } + +} diff --git a/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java b/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java new file mode 100644 index 00000000000..98bb8bf1961 --- /dev/null +++ b/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java @@ -0,0 +1,100 @@ +/* + * 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.test.lmq; + +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer; +import org.apache.rocketmq.client.consumer.PullCallback; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.consumer.DefaultMQPullConsumerImpl; +import org.apache.rocketmq.client.impl.consumer.RebalanceImpl; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.header.QueryConsumerOffsetRequestHeader; +import org.apache.rocketmq.common.protocol.header.UpdateConsumerOffsetRequestHeader; +import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.common.protocol.route.TopicRouteData; +import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.test.lmq.benchmark.BenchLmqStore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class TestBenchLmqStore { + @Test + public void test() throws MQBrokerException, RemotingException, InterruptedException, MQClientException { + System.setProperty("sendThreadNum", "1"); + System.setProperty("pullConsumerNum", "1"); + System.setProperty("consumerThreadNum", "1"); + BenchLmqStore.defaultMQProducer = mock(DefaultMQProducer.class); + SendResult sendResult = new SendResult(); + when(BenchLmqStore.defaultMQProducer.send(any(Message.class))).thenReturn(sendResult); + BenchLmqStore.doSend(); + Thread.sleep(100L); + //verify(BenchLmqStore.defaultMQProducer, atLeastOnce()).send(any(Message.class)); + BenchLmqStore.defaultMQPullConsumers = new DefaultMQPullConsumer[1]; + BenchLmqStore.defaultMQPullConsumers[0] = mock(DefaultMQPullConsumer.class); + BenchLmqStore.doPull(new ConcurrentHashMap<>(), new MessageQueue(), 1L); + verify(BenchLmqStore.defaultMQPullConsumers[0], atLeastOnce()).pullBlockIfNotFound(any(MessageQueue.class), anyString(), anyLong(), anyInt(), any( + PullCallback.class)); + } + @Test + public void testOffset() throws RemotingException, InterruptedException, MQClientException, MQBrokerException, IllegalAccessException { + System.setProperty("sendThreadNum", "1"); + DefaultMQPullConsumer defaultMQPullConsumer = mock(DefaultMQPullConsumer.class); + BenchLmqStore.defaultMQPullConsumers = new DefaultMQPullConsumer[1]; + BenchLmqStore.defaultMQPullConsumers[0] = defaultMQPullConsumer; + DefaultMQPullConsumerImpl defaultMQPullConsumerImpl = mock(DefaultMQPullConsumerImpl.class); + when(defaultMQPullConsumer.getDefaultMQPullConsumerImpl()).thenReturn(defaultMQPullConsumerImpl); + RebalanceImpl rebalanceImpl = mock(RebalanceImpl.class); + when(defaultMQPullConsumerImpl.getRebalanceImpl()).thenReturn(rebalanceImpl); + MQClientInstance mqClientInstance = mock(MQClientInstance.class); + when(rebalanceImpl.getMqClientFactory()).thenReturn(mqClientInstance); + MQClientAPIImpl mqClientAPI = mock(MQClientAPIImpl.class); + when(mqClientInstance.getMQClientAPIImpl()).thenReturn(mqClientAPI); + TopicRouteData topicRouteData = new TopicRouteData(); + HashMap brokerAddrs = new HashMap<>(); + brokerAddrs.put(MixAll.MASTER_ID, "test"); + List brokerData = Arrays.asList(new BrokerData("test", "test", brokerAddrs)); + topicRouteData.setBrokerDatas(brokerData); + FieldUtils.writeStaticField(BenchLmqStore.class, "lmqTopic", "test", true); + when(mqClientAPI.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(topicRouteData); + BenchLmqStore.doBenchOffset(); + Thread.sleep(100L); + verify(mqClientAPI, atLeastOnce()).queryConsumerOffset(anyString(), any(QueryConsumerOffsetRequestHeader.class), anyLong()); + verify(mqClientAPI, atLeastOnce()).updateConsumerOffset(anyString(), any(UpdateConsumerOffsetRequestHeader.class), anyLong()); + } +} \ No newline at end of file From 2035e6fc5d970bd6f4369c55103855c1167df7bd Mon Sep 17 00:00:00 2001 From: dinglei Date: Fri, 14 Jan 2022 11:06:36 +0800 Subject: [PATCH 097/141] fix build error in RIP-28 at getmQclientFactory (#3753) --- .../apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java b/test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java index 392b306734e..d9335ebec53 100644 --- a/test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java +++ b/test/src/main/java/org/apache/rocketmq/test/lmq/benchmark/BenchLmqStore.java @@ -242,7 +242,7 @@ public static void doBenchOffset() throws RemotingException, InterruptedExceptio Map offsetMap = new ConcurrentHashMap<>(); String statKey = "benchOffset"; TopicRouteData topicRouteData = defaultMQPullConsumers[0].getDefaultMQPullConsumerImpl(). - getRebalanceImpl().getMqClientFactory().getMQClientAPIImpl(). + getRebalanceImpl().getmQClientFactory().getMQClientAPIImpl(). getTopicRouteInfoFromNameServer(lmqTopic, 3000); HashMap brokerMap = topicRouteData.getBrokerDatas().get(0).getBrokerAddrs(); if (brokerMap == null || brokerMap.isEmpty()) { @@ -277,7 +277,7 @@ public void run() { defaultMQPullConsumer .getDefaultMQPullConsumerImpl() .getRebalanceImpl() - .getMqClientFactory() + .getmQClientFactory() .getMQClientAPIImpl().updateConsumerOffset(brokerAddress, updateHeader, 1000); QueryConsumerOffsetRequestHeader queryHeader = new QueryConsumerOffsetRequestHeader(); queryHeader.setTopic(lmq); @@ -286,7 +286,7 @@ public void run() { long newOffset2 = defaultMQPullConsumer .getDefaultMQPullConsumerImpl() .getRebalanceImpl() - .getMqClientFactory() + .getmQClientFactory() .getMQClientAPIImpl() .queryConsumerOffset(brokerAddress, queryHeader, 1000); offsetMap.put(lmq, newOffset2); From bfaef39f081a287c133e02af8bb3914c3116481a Mon Sep 17 00:00:00 2001 From: dinglei Date: Fri, 14 Jan 2022 11:38:50 +0800 Subject: [PATCH 098/141] fix build error in RIP-28 at getmQClientFactory (#3754) * fix build error in RIP-28 at getmQclientFactory * fix build error in RIP-28 at getmQclientFactory --- .../java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java b/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java index 98bb8bf1961..3457d6106b4 100644 --- a/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java +++ b/test/src/test/java/org/apache/rocketmq/test/lmq/TestBenchLmqStore.java @@ -82,7 +82,7 @@ public void testOffset() throws RemotingException, InterruptedException, MQClien RebalanceImpl rebalanceImpl = mock(RebalanceImpl.class); when(defaultMQPullConsumerImpl.getRebalanceImpl()).thenReturn(rebalanceImpl); MQClientInstance mqClientInstance = mock(MQClientInstance.class); - when(rebalanceImpl.getMqClientFactory()).thenReturn(mqClientInstance); + when(rebalanceImpl.getmQClientFactory()).thenReturn(mqClientInstance); MQClientAPIImpl mqClientAPI = mock(MQClientAPIImpl.class); when(mqClientInstance.getMQClientAPIImpl()).thenReturn(mqClientAPI); TopicRouteData topicRouteData = new TopicRouteData(); From 80bcfbc9268c10c9878110590b35a1dc6eb47b19 Mon Sep 17 00:00:00 2001 From: sunhangda Date: Fri, 14 Jan 2022 13:54:00 +0800 Subject: [PATCH 099/141] [Issue #3556] Fix:When broker is down, rocketmq client can not retry under Async send model (#3555) --- .../rocketmq/client/impl/MQClientAPIImpl.java | 105 ++++++++++-------- .../client/impl/MQClientAPIImplTest.java | 36 +++--- 2 files changed, 78 insertions(+), 63 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index 89c998b7f4e..6d83f0033dd 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -523,65 +523,72 @@ private void sendMessageAsync( final DefaultMQProducerImpl producer ) throws InterruptedException, RemotingException { final long beginStartTime = System.currentTimeMillis(); - this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { - @Override - public void operationComplete(ResponseFuture responseFuture) { - long cost = System.currentTimeMillis() - beginStartTime; - RemotingCommand response = responseFuture.getResponseCommand(); - if (null == sendCallback && response != null) { - - try { - SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr); - if (context != null && sendResult != null) { - context.setSendResult(sendResult); - context.getProducer().executeSendMessageHookAfter(context); - } - } catch (Throwable e) { - } - - producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false); - return; - } - - if (response != null) { - try { - SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr); - assert sendResult != null; - if (context != null) { - context.setSendResult(sendResult); - context.getProducer().executeSendMessageHookAfter(context); - } + try { + this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { + @Override + public void operationComplete(ResponseFuture responseFuture) { + long cost = System.currentTimeMillis() - beginStartTime; + RemotingCommand response = responseFuture.getResponseCommand(); + if (null == sendCallback && response != null) { try { - sendCallback.onSuccess(sendResult); + SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr); + if (context != null && sendResult != null) { + context.setSendResult(sendResult); + context.getProducer().executeSendMessageHookAfter(context); + } } catch (Throwable e) { } producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false); - } catch (Exception e) { - producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, e, context, false, producer); + return; } - } else { - producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true); - if (!responseFuture.isSendRequestOK()) { - MQClientException ex = new MQClientException("send request failed", responseFuture.getCause()); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, ex, context, true, producer); - } else if (responseFuture.isTimeout()) { - MQClientException ex = new MQClientException("wait response timeout " + responseFuture.getTimeoutMillis() + "ms", - responseFuture.getCause()); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, ex, context, true, producer); + + if (response != null) { + try { + SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr); + assert sendResult != null; + if (context != null) { + context.setSendResult(sendResult); + context.getProducer().executeSendMessageHookAfter(context); + } + + try { + sendCallback.onSuccess(sendResult); + } catch (Throwable e) { + } + + producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false); + } catch (Exception e) { + producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, + retryTimesWhenSendFailed, times, e, context, false, producer); + } } else { - MQClientException ex = new MQClientException("unknow reseaon", responseFuture.getCause()); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, ex, context, true, producer); + producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true); + if (!responseFuture.isSendRequestOK()) { + MQClientException ex = new MQClientException("send request failed", responseFuture.getCause()); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, + retryTimesWhenSendFailed, times, ex, context, true, producer); + } else if (responseFuture.isTimeout()) { + MQClientException ex = new MQClientException("wait response timeout " + responseFuture.getTimeoutMillis() + "ms", + responseFuture.getCause()); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, + retryTimesWhenSendFailed, times, ex, context, true, producer); + } else { + MQClientException ex = new MQClientException("unknow reseaon", responseFuture.getCause()); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, + retryTimesWhenSendFailed, times, ex, context, true, producer); + } } } - } - }); + }); + } catch (Exception ex) { + long cost = System.currentTimeMillis() - beginStartTime; + producer.updateFaultItem(brokerName, cost, true); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, + retryTimesWhenSendFailed, times, ex, context, true, producer); + } } private void onExceptionImpl(final String brokerName, diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java index e1b3bed76fd..c8446bdf104 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java @@ -197,23 +197,31 @@ public void onException(Throwable e) { public void testSendMessageAsync_WithException() throws RemotingException, InterruptedException, MQBrokerException { doThrow(new RemotingTimeoutException("Remoting Exception in Test")).when(remotingClient) .invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class)); - try { - mqClientAPI.sendMessage(brokerAddr, brokerName, msg, new SendMessageRequestHeader(), - 3 * 1000, CommunicationMode.ASYNC, new SendMessageContext(), defaultMQProducerImpl); - failBecauseExceptionWasNotThrown(RemotingException.class); - } catch (RemotingException e) { - assertThat(e).hasMessage("Remoting Exception in Test"); - } + SendMessageContext sendMessageContext = new SendMessageContext(); + sendMessageContext.setProducer(new DefaultMQProducerImpl(new DefaultMQProducer())); + mqClientAPI.sendMessage(brokerAddr, brokerName, msg, new SendMessageRequestHeader(), 3 * 1000, CommunicationMode.ASYNC, + new SendCallback() { + @Override + public void onSuccess(SendResult sendResult) { + } + @Override + public void onException(Throwable e) { + assertThat(e).hasMessage("Remoting Exception in Test"); + } + }, null, null, 0, sendMessageContext, defaultMQProducerImpl); doThrow(new InterruptedException("Interrupted Exception in Test")).when(remotingClient) .invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class)); - try { - mqClientAPI.sendMessage(brokerAddr, brokerName, msg, new SendMessageRequestHeader(), - 3 * 1000, CommunicationMode.ASYNC, new SendMessageContext(), defaultMQProducerImpl); - failBecauseExceptionWasNotThrown(InterruptedException.class); - } catch (InterruptedException e) { - assertThat(e).hasMessage("Interrupted Exception in Test"); - } + mqClientAPI.sendMessage(brokerAddr, brokerName, msg, new SendMessageRequestHeader(), 3 * 1000, CommunicationMode.ASYNC, + new SendCallback() { + @Override + public void onSuccess(SendResult sendResult) { + } + @Override + public void onException(Throwable e) { + assertThat(e).hasMessage("Interrupted Exception in Test"); + } + }, null, null, 0, sendMessageContext, defaultMQProducerImpl); } @Test From c5da66d841b35d55dba01e1b240e536cf8ae6040 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B5=B5=E5=BB=B6?= Date: Fri, 14 Jan 2022 13:55:54 +0800 Subject: [PATCH 100/141] Check producerTable or consumerTable is empty when updateTopicRouteInfoFromNameServer (#2581) --- .../apache/rocketmq/client/impl/factory/MQClientInstance.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java index 6d35ed63028..a48db656f3e 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java @@ -639,7 +639,7 @@ public boolean updateTopicRouteInfoFromNameServer(final String topic, boolean is } // Update Pub info - { + if (!producerTable.isEmpty()) { TopicPublishInfo publishInfo = topicRouteData2TopicPublishInfo(topic, topicRouteData); publishInfo.setHaveTopicRouterInfo(true); Iterator> it = this.producerTable.entrySet().iterator(); @@ -653,7 +653,7 @@ public boolean updateTopicRouteInfoFromNameServer(final String topic, boolean is } // Update sub info - { + if (!consumerTable.isEmpty()) { Set subscribeInfo = topicRouteData2TopicSubscribeInfo(topic, topicRouteData); Iterator> it = this.consumerTable.entrySet().iterator(); while (it.hasNext()) { From 6dff04fcb6405fc53a90730cf36ed9a79335bef5 Mon Sep 17 00:00:00 2001 From: zhaohai <33314633+zhaohai1299002788@users.noreply.github.com> Date: Fri, 14 Jan 2022 13:58:49 +0800 Subject: [PATCH 101/141] [ISSUE #3642] Display GID and Topic with namespace (#3648) --- .../protocol/body/ConsumerRunningInfo.java | 8 ++++---- .../broker/BrokerConsumeStatsSubCommad.java | 6 +++--- .../consumer/ConsumerProgressSubCommand.java | 18 +++++++++--------- .../command/stats/StatsAllSubCommand.java | 12 ++++++------ .../command/topic/TopicListSubCommand.java | 6 +++--- 5 files changed, 25 insertions(+), 25 deletions(-) diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java index d7942eb4a0a..0c3df0d5a87 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ConsumerRunningInfo.java @@ -223,7 +223,7 @@ public String formatString() { { sb.append("\n\n#Consumer Offset#\n"); - sb.append(String.format("%-32s %-32s %-4s %-20s%n", + sb.append(String.format("%-64s %-32s %-4s %-20s%n", "#Topic", "#Broker Name", "#QID", @@ -245,7 +245,7 @@ public String formatString() { { sb.append("\n\n#Consumer MQ Detail#\n"); - sb.append(String.format("%-32s %-32s %-4s %-20s%n", + sb.append(String.format("%-64s %-32s %-4s %-20s%n", "#Topic", "#Broker Name", "#QID", @@ -255,7 +255,7 @@ public String formatString() { Iterator> it = this.mqTable.entrySet().iterator(); while (it.hasNext()) { Entry next = it.next(); - String item = String.format("%-32s %-32s %-4d %s%n", + String item = String.format("%-64s %-32s %-4d %s%n", next.getKey().getTopic(), next.getKey().getBrokerName(), next.getKey().getQueueId(), @@ -267,7 +267,7 @@ public String formatString() { { sb.append("\n\n#Consumer RT&TPS#\n"); - sb.append(String.format("%-32s %14s %14s %14s %14s %18s %25s%n", + sb.append(String.format("%-64s %14s %14s %14s %14s %18s %25s%n", "#Topic", "#Pull RT", "#Pull TPS", diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java index 0dee6dee278..f462b27818a 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java @@ -105,7 +105,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t } ConsumeStatsList consumeStatsList = defaultMQAdminExt.fetchConsumeStatsInBroker(brokerAddr, isOrder, timeoutMillis); - System.out.printf("%-32s %-32s %-32s %-4s %-20s %-20s %-20s %s%n", + System.out.printf("%-64s %-64s %-32s %-4s %-20s %-20s %-20s %s%n", "#Topic", "#Group", "#Broker Name", @@ -136,8 +136,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t } if (offsetWrapper.getLastTimestamp() > 0) - System.out.printf("%-32s %-32s %-32s %-4d %-20d %-20d %-20d %s%n", - UtilAll.frontStringAtLeast(mq.getTopic(), 32), + System.out.printf("%-64s %-64s %-32s %-4d %-20d %-20d %-20d %s%n", + UtilAll.frontStringAtLeast(mq.getTopic(), 64), group, UtilAll.frontStringAtLeast(mq.getBrokerName(), 32), mq.getQueueId(), diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java index 7985f9d5918..48e0c1be4d6 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java @@ -112,7 +112,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t messageQueueAllocationResult = getMessageQueueAllocationResult(defaultMQAdminExt, consumerGroup); } if (showClientIP) { - System.out.printf("%-32s %-32s %-4s %-20s %-20s %-20s %-20s %s%n", + System.out.printf("%-64s %-32s %-4s %-20s %-20s %-20s %-20s %s%n", "#Topic", "#Broker Name", "#QID", @@ -122,7 +122,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t "#Diff", "#LastTime"); } else { - System.out.printf("%-32s %-32s %-4s %-20s %-20s %-20s %s%n", + System.out.printf("%-64s %-32s %-4s %-20s %-20s %-20s %s%n", "#Topic", "#Broker Name", "#QID", @@ -151,8 +151,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t clientIP = messageQueueAllocationResult.get(mq); } if (showClientIP) { - System.out.printf("%-32s %-32s %-4d %-20d %-20d %-20s %-20d %s%n", - UtilAll.frontStringAtLeast(mq.getTopic(), 32), + System.out.printf("%-64s %-32s %-4d %-20d %-20d %-20s %-20d %s%n", + UtilAll.frontStringAtLeast(mq.getTopic(), 64), UtilAll.frontStringAtLeast(mq.getBrokerName(), 32), mq.getQueueId(), offsetWrapper.getBrokerOffset(), @@ -162,8 +162,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t lastTime ); } else { - System.out.printf("%-32s %-32s %-4d %-20d %-20d %-20d %s%n", - UtilAll.frontStringAtLeast(mq.getTopic(), 32), + System.out.printf("%-64s %-32s %-4d %-20d %-20d %-20d %s%n", + UtilAll.frontStringAtLeast(mq.getTopic(), 64), UtilAll.frontStringAtLeast(mq.getBrokerName(), 32), mq.getQueueId(), offsetWrapper.getBrokerOffset(), @@ -178,7 +178,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t System.out.printf("Consume TPS: %.2f%n", consumeStats.getConsumeTps()); System.out.printf("Diff Total: %d%n", diffTotal); } else { - System.out.printf("%-32s %-6s %-24s %-5s %-14s %-7s %s%n", + System.out.printf("%-64s %-6s %-24s %-5s %-14s %-7s %s%n", "#Group", "#Count", "#Version", @@ -221,8 +221,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t groupConsumeInfo.setVersion(cc.computeMinVersion()); } - System.out.printf("%-32s %-6d %-24s %-5s %-14s %-7d %d%n", - UtilAll.frontStringAtLeast(groupConsumeInfo.getGroup(), 32), + System.out.printf("%-64s %-6d %-24s %-5s %-14s %-7d %d%n", + UtilAll.frontStringAtLeast(groupConsumeInfo.getGroup(), 64), groupConsumeInfo.getCount(), groupConsumeInfo.getCount() > 0 ? groupConsumeInfo.versionDesc() : "OFFLINE", groupConsumeInfo.consumeTypeDesc(), diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java index 47070882e35..e41d57e30a8 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java @@ -94,9 +94,9 @@ public static void printTopicDetail(final DefaultMQAdminExt admin, final String if (!activeTopic || (inMsgCntToday > 0) || (outMsgCntToday > 0)) { - System.out.printf("%-32s %-32s %12d %11.2f %11.2f %14d %14d%n", - UtilAll.frontStringAtLeast(topic, 32), - UtilAll.frontStringAtLeast(group, 32), + System.out.printf("%-64s %-64s %12d %11.2f %11.2f %14d %14d%n", + UtilAll.frontStringAtLeast(topic, 64), + UtilAll.frontStringAtLeast(group, 64), accumulate, inTPS, outTPS, @@ -108,8 +108,8 @@ public static void printTopicDetail(final DefaultMQAdminExt admin, final String } else { if (!activeTopic || (inMsgCntToday > 0)) { - System.out.printf("%-32s %-32s %12d %11.2f %11s %14d %14s%n", - UtilAll.frontStringAtLeast(topic, 32), + System.out.printf("%-64s %-64s %12d %11.2f %11s %14d %14s%n", + UtilAll.frontStringAtLeast(topic, 64), "", 0, inTPS, @@ -171,7 +171,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t TopicList topicList = defaultMQAdminExt.fetchAllTopicList(); - System.out.printf("%-32s %-32s %12s %11s %11s %14s %14s%n", + System.out.printf("%-64s %-64s %12s %11s %11s %14s %14s%n", "#Topic", "#Consumer Group", "#Accumulation", diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java index 56aea2e8084..eefaa8f2785 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java @@ -96,10 +96,10 @@ public void execute(final CommandLine commandLine, final Options options, } for (String group : groupList.getGroupList()) { - System.out.printf("%-20s %-48s %-48s%n", + System.out.printf("%-20s %-64s %-64s%n", UtilAll.frontStringAtLeast(clusterName, 20), - UtilAll.frontStringAtLeast(topic, 48), - UtilAll.frontStringAtLeast(group, 48) + UtilAll.frontStringAtLeast(topic, 64), + UtilAll.frontStringAtLeast(group, 64) ); } } From c832523742b52306cfabcee03959a85da62e2e9d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Sat, 15 Jan 2022 17:25:44 +0800 Subject: [PATCH 102/141] [ISSUE #3757] docs: Polish the document 'Design_Query.md'. (#3758) --- docs/en/Design_Query.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/Design_Query.md b/docs/en/Design_Query.md index 983d48df9b0..887a073a9d9 100644 --- a/docs/en/Design_Query.md +++ b/docs/en/Design_Query.md @@ -12,6 +12,8 @@ The MessageId in RocketMQ has a total length of 16 bytes, including the broker a The IndexFile provides the user with the querying service by “Querying Messages by Message Key”. The IndexFile is stored in $HOME\store\index${fileName}, and the file name is named after the timestamp at the time of creation. The file size is fixed, which is 420,000,040 bytes (40+5million\*4+20million\*20). If the UNIQ_KEY is set in the properties of the message, then the "topic + ‘#’ + UNIQ_KEY" will be used as the index. Likewise, if the KEYS is set in the properties of the message (multiple KEYs should be separated by spaces), then the "topic + ‘#’ + KEY" will be used as the index. -The index data contains four fields, Key Hash, CommitLog offset, Timestamp and NextIndex offset, for a total of 20 Bytes. The NextIndex offset of the index data will point to the previous index data if the Key Hash of the index data is the same as that of the previous index data. If a hash conflict occurs, then the NextIndex offset can be used as the field to string all conflicting indexes in a linked list. What the Timestamp records is the time difference between two storeTimestamps, instead of a specific time. The structure of the entire IndexFile is shown in the graph. The Header is used to store some general statistics, which needs 40 bytes. The Slot Table of 4\*5million bytes does not save the real index data, but saves the header of the singly linked list corresponding to each slot. The Index Linked List of 20\*20million is the real index data, that is, an Index File can hold 20million indexes. +The header of IndexFile contains eight fields, `beginTimestamp`(8 bytes), `endTimestamp`(8 bytes), `beginPhyOffset`(8 bytes), `endPhyOffset`(8 bytes), `hashSlotCount`(4 bytes), and `indexCount`(4 bytes).`beginTimestamp` and `endTimestamp` represents the storeTimestamp of the message corresponding to the first and last index, `beginPhyOffset` and `endPhyOffset` represent the physical offset of the message corresponding to the first and last index. `hashSlotCount` represents the count of hash slot. `indexCount` represents the count of indexes. + +The index data contains four fields, `Key Hash`, `CommitLog offset`, `Timestamp` and `NextIndex offset`, for a total of 20 Bytes. The `NextIndex offset` of the index data will point to the previous index data if the `Key Hash` of the index data is the same as that of the previous index data. If a hash conflict occurs, then the `NextIndex offset` can be used as the field to string all conflicting indexes in a linked list. What the `Timestamp` records is the time difference between the `storeTimestamp` of the message associated with the current key and the `BeginTimestamp` in the `IndexHeader`, instead of a specific time. The structure of the entire IndexFile is shown in the graph. The Header is used to store some general statistics, which needs 40 bytes. The Slot Table of 4\*5million bytes does not save the real index data, but saves the header of the singly linked list corresponding to each slot. The Index Linked List of 20\*20million is the real index data, that is, an Index File can hold 20million indexes. The specific method of "Query Message by Message Key" is that the topic and message key are used to find the record in the IndexFile, and then read the message from the file of CommitLog according to the CommitLog offset in this record. \ No newline at end of file From 4f39218066523344a9cca65946c3ae1b0f1a6218 Mon Sep 17 00:00:00 2001 From: yuz10 <845238369@qq.com> Date: Sun, 16 Jan 2022 20:38:48 +0800 Subject: [PATCH 103/141] Avoid new lines in log (#3762) --- .../java/org/apache/rocketmq/logging/inner/LoggingBuilder.java | 2 +- .../java/org/apache/rocketmq/logging/inner/LoggingEvent.java | 3 +++ pom.xml | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingBuilder.java b/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingBuilder.java index 3ec440bc882..7468cd498f5 100644 --- a/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingBuilder.java +++ b/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingBuilder.java @@ -1209,7 +1209,7 @@ public String format(LoggingEvent event) { sb.append(" "); sb.append(event.getLoggerName()); sb.append(" - "); - sb.append(event.getMessage()); + sb.append(event.getRenderedMessage()); String[] throwableStr = event.getThrowableStr(); if (throwableStr != null) { sb.append("\r\n"); diff --git a/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingEvent.java b/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingEvent.java index 1b3e9553933..44554e2b7e9 100644 --- a/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingEvent.java +++ b/logging/src/main/java/org/apache/rocketmq/logging/inner/LoggingEvent.java @@ -68,6 +68,9 @@ public String getRenderedMessage() { } else { renderedMessage = message.toString(); } + if (renderedMessage != null) { + renderedMessage = renderedMessage.replace('\r', ' ').replace('\n', ' '); + } } return renderedMessage; } diff --git a/pom.xml b/pom.xml index 19abca03605..c2c8f8b3bdf 100644 --- a/pom.xml +++ b/pom.xml @@ -527,7 +527,7 @@ ch.qos.logback logback-classic - 1.0.13 + 1.2.10 commons-cli From 1b3b3a814628c730506db366fc9174ed33553f32 Mon Sep 17 00:00:00 2001 From: sunhangda Date: Wed, 19 Jan 2022 11:07:52 +0800 Subject: [PATCH 104/141] Adding exception message with broker addr when occuring broker connect timeout (#3772) * fix:when broker is down,async send model can not retry * fix Issue #3556 * fix Issue #3556 * fix Issue #3556 * fix Issue #3556 * async send model success retry when occurs Exception * test case testSendMessageAsync_WithException * modify test case testSendMessageAsync_WithException * adding exception message with broker addr when occuring broker connect timeout * repair code style --- .../apache/rocketmq/remoting/netty/NettyRemotingClient.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java index 569746554d6..cbd17fadb23 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java @@ -375,7 +375,7 @@ public RemotingCommand invokeSync(String addr, final RemotingCommand request, lo doBeforeRpcHooks(addr, request); long costTime = System.currentTimeMillis() - beginStartTime; if (timeoutMillis < costTime) { - throw new RemotingTimeoutException("invokeSync call timeout"); + throw new RemotingTimeoutException("invokeSync call the addr[" + addr + "] timeout"); } RemotingCommand response = this.invokeSyncImpl(channel, request, timeoutMillis - costTime); doAfterRpcHooks(RemotingHelper.parseChannelRemoteAddr(channel), request, response); @@ -525,7 +525,7 @@ public void invokeAsync(String addr, RemotingCommand request, long timeoutMillis doBeforeRpcHooks(addr, request); long costTime = System.currentTimeMillis() - beginStartTime; if (timeoutMillis < costTime) { - throw new RemotingTooMuchRequestException("invokeAsync call timeout"); + throw new RemotingTooMuchRequestException("invokeAsync call the addr[" + addr + "] timeout"); } this.invokeAsyncImpl(channel, request, timeoutMillis - costTime, invokeCallback); } catch (RemotingSendRequestException e) { From 41f58e4b86712a36c240735daa572916964d722a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Wed, 19 Jan 2022 20:14:34 +0800 Subject: [PATCH 105/141] [ISSUE #3774] docs: Polish the document 'Example_Transaction.md'. (#3775) --- docs/en/Example_Transaction.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/Example_Transaction.md b/docs/en/Example_Transaction.md index 5d1cff9320e..e2170738e9d 100644 --- a/docs/en/Example_Transaction.md +++ b/docs/en/Example_Transaction.md @@ -2,9 +2,9 @@ ## 1 Transaction message status There are three states for transaction message: -- TransactionStatus.CommitTransaction: commit transaction, it means that allow consumers to consume this message. -- TransactionStatus.RollbackTransaction: rollback transaction, it means that the message will be deleted and not allowed to consume. -- TransactionStatus.Unknown: intermediate state, it means that MQ is needed to check back to determine the status. +- LocalTransactionState.COMMIT_MESSAGE: commit transaction, it means that allow consumers to consume this message. +- LocalTransactionState.ROLLBACK_MESSAGE: rollback transaction, it means that the message will be deleted and not allowed to consume. +- LocalTransactionState.UNKNOW: intermediate state, it means that MQ is needed to check back to determine the status. ## 2 Send transactional message example @@ -91,6 +91,6 @@ public class TransactionListenerImpl implements TransactionListener { 2. In order to avoid a single message being checked too many times and lead to half queue message accumulation, we limited the number of checks for a single message to 15 times by default, but users can change this limit by change the ```transactionCheckMax``` parameter in the configuration of the broker, if one message has been checked over ```transactionCheckMax``` times, broker will discard this message and print an error log at the same time by default. Users can change this behavior by override the ```AbstractTransactionalMessageCheckListener``` class. 3. A transactional message will be checked after a certain period of time that determined by parameter ```transactionTimeout``` in the configuration of the broker. And users also can change this limit by set user property ```CHECK_IMMUNITY_TIME_IN_SECONDS``` when sending transactional message, this parameter takes precedence over the ```transactionTimeout``` parameter. 4. A transactional message maybe checked or consumed more than once. -5. Committed message reput to the user’s target topic may fail. Currently, it depends on the log record. High availability is ensured by the high availability mechanism of RocketMQ itself. If you want to ensure that the transactional message isn’t lost and the transaction integrity is guaranteed, it is recommended to use synchronous double write. mechanism. -6. Producer IDs of transactional messages cannot be shared with producer IDs of other types of messages. Unlike other types of message, transactional messages allow backward queries. MQ Server query clients by their Producer IDs. +5. Committed message reput to the user’s target topic may fail. Currently, it depends on the log record. High availability is ensured by the high availability mechanism of RocketMQ itself. If you want to ensure that the transactional message isn’t lost and the transaction integrity is guaranteed, it is recommended to use synchronous double write mechanism. +6. `producerGroup` for producers of transactional messages cannot be shared with `producerGroup` for producers of other types of messages. Unlike other types of message, transactional messages allow backward queries. MQ Server query clients by their `producerGroup` of producers. From 84cde5f5beddfc16621f6b5dba31883796d2c216 Mon Sep 17 00:00:00 2001 From: makabakaboom <402879834@qq.com> Date: Thu, 20 Jan 2022 13:37:01 +0800 Subject: [PATCH 106/141] #3015 when slaveAckOffset greater than local commitLog offset, break connection and return (#3016) when slaveAckOffset greater than local commitLog offset, break connection and return --- .../java/org/apache/rocketmq/store/ha/HAConnection.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java index 11af1e2feb0..dd68c73c37f 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java +++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java @@ -168,6 +168,12 @@ private boolean processReadEvent() { if (HAConnection.this.slaveRequestOffset < 0) { HAConnection.this.slaveRequestOffset = readOffset; log.info("slave[" + HAConnection.this.clientAddr + "] request offset " + readOffset); + } else if (HAConnection.this.slaveAckOffset > HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset()){ + log.warn("slave[{}] request offset={} greater than local commitLog offset={}. ", + HAConnection.this.clientAddr, + HAConnection.this.slaveAckOffset, + HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset()); + return false; } HAConnection.this.haService.notifyTransferSome(HAConnection.this.slaveAckOffset); From 97198fadf691a2f65c1f588b0466fa9ee7da1657 Mon Sep 17 00:00:00 2001 From: zhangyang Date: Thu, 20 Jan 2022 22:22:43 +0800 Subject: [PATCH 107/141] [ISSUE #3449] Delayed message supports asynchronous delivery (#3458) * [schedule] Delayed message repeated delivery optimization Signed-off-by: zhangyang21 * [schedule] Support asynchronous delivery Signed-off-by: zhangyang * use maxDelayLevel as deliverThreadPoolNums Signed-off-by: zhangyang21 * [HAConnection] fix code style Signed-off-by: zhangyang21 --- .../store/config/MessageStoreConfig.java | 28 + .../rocketmq/store/ha/HAConnection.java | 2 +- .../schedule/ScheduleMessageService.java | 621 ++++++++++++++---- .../store/ScheduleMessageServiceTest.java | 116 +++- 4 files changed, 625 insertions(+), 142 deletions(-) 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 1188f2191f7..bb1e01f247e 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 @@ -163,6 +163,10 @@ public class MessageStoreConfig { private boolean enableMultiDispatch = false; private int maxLmqConsumeQueueNum = 20000; + private boolean enableScheduleAsyncDeliver = false; + private int scheduleAsyncDeliverMaxPendingLimit = 2000; + private int scheduleAsyncDeliverMaxResendNum2Blocked = 3; + public boolean isDebugLockEnable() { return debugLockEnable; } @@ -772,4 +776,28 @@ public int getMaxLmqConsumeQueueNum() { public void setMaxLmqConsumeQueueNum(int maxLmqConsumeQueueNum) { this.maxLmqConsumeQueueNum = maxLmqConsumeQueueNum; } + + public boolean isEnableScheduleAsyncDeliver() { + return enableScheduleAsyncDeliver; + } + + public void setEnableScheduleAsyncDeliver(boolean enableScheduleAsyncDeliver) { + this.enableScheduleAsyncDeliver = enableScheduleAsyncDeliver; + } + + public int getScheduleAsyncDeliverMaxPendingLimit() { + return scheduleAsyncDeliverMaxPendingLimit; + } + + public void setScheduleAsyncDeliverMaxPendingLimit(int scheduleAsyncDeliverMaxPendingLimit) { + this.scheduleAsyncDeliverMaxPendingLimit = scheduleAsyncDeliverMaxPendingLimit; + } + + public int getScheduleAsyncDeliverMaxResendNum2Blocked() { + return scheduleAsyncDeliverMaxResendNum2Blocked; + } + + public void setScheduleAsyncDeliverMaxResendNum2Blocked(int scheduleAsyncDeliverMaxResendNum2Blocked) { + this.scheduleAsyncDeliverMaxResendNum2Blocked = scheduleAsyncDeliverMaxResendNum2Blocked; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java index dd68c73c37f..c08c515c0c9 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java +++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java @@ -168,7 +168,7 @@ private boolean processReadEvent() { if (HAConnection.this.slaveRequestOffset < 0) { HAConnection.this.slaveRequestOffset = readOffset; log.info("slave[" + HAConnection.this.clientAddr + "] request offset " + readOffset); - } else if (HAConnection.this.slaveAckOffset > HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset()){ + } else if (HAConnection.this.slaveAckOffset > HAConnection.this.haService.getDefaultMessageStore().getMaxPhyOffset()) { log.warn("slave[{}] request offset={} greater than local commitLog offset={}. ", HAConnection.this.clientAddr, HAConnection.this.slaveAckOffset, diff --git a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java index c45287ff659..d5b4e8d3909 100644 --- a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java +++ b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java @@ -19,8 +19,12 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -54,6 +58,8 @@ public class ScheduleMessageService extends ConfigManager { private static final long FIRST_DELAY_TIME = 1000L; private static final long DELAY_FOR_A_WHILE = 100L; private static final long DELAY_FOR_A_PERIOD = 10000L; + private static final long WAIT_FOR_SHUTDOWN = 5000L; + private static final long DELAY_FOR_A_SLEEP = 10L; private final ConcurrentMap delayLevelTable = new ConcurrentHashMap(32); @@ -63,13 +69,19 @@ public class ScheduleMessageService extends ConfigManager { private final DefaultMessageStore defaultMessageStore; private final AtomicBoolean started = new AtomicBoolean(false); private ScheduledExecutorService deliverExecutorService; - private int deliverThreadPoolNums = Runtime.getRuntime().availableProcessors(); private MessageStore writeMessageStore; private int maxDelayLevel; + private boolean enableAsyncDeliver = false; + private ScheduledExecutorService handleExecutorService; + private final Map> deliverPendingTable = + new ConcurrentHashMap<>(32); public ScheduleMessageService(final DefaultMessageStore defaultMessageStore) { this.defaultMessageStore = defaultMessageStore; this.writeMessageStore = defaultMessageStore; + if (defaultMessageStore != null) { + this.enableAsyncDeliver = defaultMessageStore.getMessageStoreConfig().isEnableScheduleAsyncDeliver(); + } } public static int queueId2DelayLevel(final int queueId) { @@ -116,7 +128,10 @@ public long computeDeliverTimestamp(final int delayLevel, final long storeTimest public void start() { if (started.compareAndSet(false, true)) { super.load(); - this.deliverExecutorService = new ScheduledThreadPoolExecutor(deliverThreadPoolNums, new ThreadFactoryImpl("ScheduleMessageTimerThread_")); + this.deliverExecutorService = new ScheduledThreadPoolExecutor(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageTimerThread_")); + if (this.enableAsyncDeliver) { + this.handleExecutorService = new ScheduledThreadPoolExecutor(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageExecutorHandleThread_")); + } for (Map.Entry entry : this.delayLevelTable.entrySet()) { Integer level = entry.getKey(); Long timeDelay = entry.getValue(); @@ -126,6 +141,9 @@ public void start() { } if (timeDelay != null) { + if (this.enableAsyncDeliver) { + this.handleExecutorService.schedule(new HandlePutResultTask(level), FIRST_DELAY_TIME, TimeUnit.MILLISECONDS); + } this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask(level, offset), FIRST_DELAY_TIME, TimeUnit.MILLISECONDS); } } @@ -148,7 +166,29 @@ public void run() { public void shutdown() { if (this.started.compareAndSet(true, false) && null != this.deliverExecutorService) { - this.deliverExecutorService.shutdownNow(); + this.deliverExecutorService.shutdown(); + try { + this.deliverExecutorService.awaitTermination(WAIT_FOR_SHUTDOWN, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + log.error("deliverExecutorService awaitTermination error", e); + } + + if (this.handleExecutorService != null) { + this.handleExecutorService.shutdown(); + try { + this.handleExecutorService.awaitTermination(WAIT_FOR_SHUTDOWN, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + log.error("handleExecutorService awaitTermination error", e); + } + } + + if (this.deliverPendingTable != null) { + for (int i = 1; i <= this.deliverPendingTable.size(); i++) { + log.warn("deliverPendingTable level: {}, size: {}", i, this.deliverPendingTable.get(i).size()); + } + } + + this.persist(); } } @@ -255,6 +295,9 @@ public boolean parseDelayLevel() { long num = Long.parseLong(value.substring(0, value.length() - 1)); long delayTimeMillis = tu * num; this.delayLevelTable.put(level, delayTimeMillis); + if (this.enableAsyncDeliver) { + this.deliverPendingTable.put(level, new LinkedBlockingQueue<>()); + } } } catch (Exception e) { log.error("parseDelayLevel exception", e); @@ -265,6 +308,36 @@ public boolean parseDelayLevel() { return true; } + private MessageExtBrokerInner messageTimeup(MessageExt msgExt) { + MessageExtBrokerInner msgInner = new MessageExtBrokerInner(); + msgInner.setBody(msgExt.getBody()); + msgInner.setFlag(msgExt.getFlag()); + MessageAccessor.setProperties(msgInner, msgExt.getProperties()); + + TopicFilterType topicFilterType = MessageExt.parseTopicFilterType(msgInner.getSysFlag()); + long tagsCodeValue = + MessageExtBrokerInner.tagsString2tagsCode(topicFilterType, msgInner.getTags()); + msgInner.setTagsCode(tagsCodeValue); + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties())); + + msgInner.setSysFlag(msgExt.getSysFlag()); + msgInner.setBornTimestamp(msgExt.getBornTimestamp()); + msgInner.setBornHost(msgExt.getBornHost()); + msgInner.setStoreHost(msgExt.getStoreHost()); + msgInner.setReconsumeTimes(msgExt.getReconsumeTimes()); + + msgInner.setWaitStoreMsgOK(false); + MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_DELAY_TIME_LEVEL); + + msgInner.setTopic(msgInner.getProperty(MessageConst.PROPERTY_REAL_TOPIC)); + + String queueIdStr = msgInner.getProperty(MessageConst.PROPERTY_REAL_QUEUE_ID); + int queueId = Integer.parseInt(queueIdStr); + msgInner.setQueueId(queueId); + + return msgInner; + } + class DeliverDelayedMessageTimerTask implements Runnable { private final int delayLevel; private final long offset; @@ -283,8 +356,7 @@ public void run() { } catch (Exception e) { // XXX: warn and notify me log.error("ScheduleMessageService, executeOnTimeup exception", e); - ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask( - this.delayLevel, this.offset), DELAY_FOR_A_PERIOD, TimeUnit.MILLISECONDS); + this.scheduleNextTimerTask(this.offset, DELAY_FOR_A_PERIOD); } } @@ -308,158 +380,431 @@ public void executeOnTimeup() { ScheduleMessageService.this.defaultMessageStore.findConsumeQueue(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, delayLevel2QueueId(delayLevel)); - long failScheduleOffset = offset; + if (cq == null) { + this.scheduleNextTimerTask(this.offset, DELAY_FOR_A_WHILE); + return; + } - if (cq != null) { - SelectMappedBufferResult bufferCQ = cq.getIndexBuffer(this.offset); - if (bufferCQ != null) { - try { - long nextOffset = offset; - int i = 0; - ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit(); - for (; i < bufferCQ.getSize(); i += ConsumeQueue.CQ_STORE_UNIT_SIZE) { - long offsetPy = bufferCQ.getByteBuffer().getLong(); - int sizePy = bufferCQ.getByteBuffer().getInt(); - long tagsCode = bufferCQ.getByteBuffer().getLong(); - - if (cq.isExtAddr(tagsCode)) { - if (cq.getExt(tagsCode, cqExtUnit)) { - tagsCode = cqExtUnit.getTagsCode(); - } else { - //can't find ext content.So re compute tags code. - log.error("[BUG] can't find consume queue extend file content!addr={}, offsetPy={}, sizePy={}", - tagsCode, offsetPy, sizePy); - long msgStoreTime = defaultMessageStore.getCommitLog().pickupStoreTimestamp(offsetPy, sizePy); - tagsCode = computeDeliverTimestamp(delayLevel, msgStoreTime); - } - } + SelectMappedBufferResult bufferCQ = cq.getIndexBuffer(this.offset); + if (bufferCQ == null) { + long resetOffset; + if ((resetOffset = cq.getMinOffsetInQueue()) > this.offset) { + log.error("schedule CQ offset invalid. offset={}, cqMinOffset={}, queueId={}", + this.offset, resetOffset, cq.getQueueId()); + } else if ((resetOffset = cq.getMaxOffsetInQueue()) < this.offset) { + log.error("schedule CQ offset invalid. offset={}, cqMaxOffset={}, queueId={}", + this.offset, resetOffset, cq.getQueueId()); + } else { + resetOffset = this.offset; + } - long now = System.currentTimeMillis(); - long deliverTimestamp = this.correctDeliverTimestamp(now, tagsCode); - - nextOffset = offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE); - - long countdown = deliverTimestamp - now; - - if (countdown <= 0) { - MessageExt msgExt = - ScheduleMessageService.this.defaultMessageStore.lookMessageByOffset( - offsetPy, sizePy); - - if (msgExt != null) { - try { - MessageExtBrokerInner msgInner = this.messageTimeup(msgExt); - if (TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC.equals(msgInner.getTopic())) { - log.error("[BUG] the real topic of schedule msg is {}, discard the msg. msg={}", - msgInner.getTopic(), msgInner); - continue; - } - PutMessageResult putMessageResult = - ScheduleMessageService.this.writeMessageStore - .putMessage(msgInner); - - if (putMessageResult != null - && putMessageResult.getPutMessageStatus() == PutMessageStatus.PUT_OK) { - if (ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig().isEnableScheduleMessageStats()) { - ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incQueueGetNums(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, delayLevel - 1, putMessageResult.getAppendMessageResult().getMsgNum()); - ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incQueueGetSize(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, delayLevel - 1, putMessageResult.getAppendMessageResult().getWroteBytes()); - ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incGroupGetNums(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, putMessageResult.getAppendMessageResult().getMsgNum()); - ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incGroupGetSize(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, putMessageResult.getAppendMessageResult().getWroteBytes()); - ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incTopicPutNums(msgInner.getTopic(), putMessageResult.getAppendMessageResult().getMsgNum(), 1); - ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incTopicPutSize(msgInner.getTopic(), - putMessageResult.getAppendMessageResult().getWroteBytes()); - ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incBrokerPutNums(putMessageResult.getAppendMessageResult().getMsgNum()); - } - continue; - } else { - // XXX: warn and notify me - log.error( - "ScheduleMessageService, a message time up, but reput it failed, topic: {} msgId {}", - msgExt.getTopic(), msgExt.getMsgId()); - ScheduleMessageService.this.deliverExecutorService.schedule( - new DeliverDelayedMessageTimerTask(this.delayLevel, - nextOffset), DELAY_FOR_A_PERIOD, TimeUnit.MILLISECONDS); - ScheduleMessageService.this.updateOffset(this.delayLevel, - nextOffset); - return; - } - } catch (Exception e) { - /* - * XXX: warn and notify me - */ - log.error( - "ScheduleMessageService, messageTimeup execute error, drop it. msgExt={}, nextOffset={}, offsetPy={}, sizePy={}", msgExt, nextOffset, offsetPy, sizePy, e); - } - } - } else { - ScheduleMessageService.this.deliverExecutorService.schedule( - new DeliverDelayedMessageTimerTask(this.delayLevel, nextOffset), - countdown, TimeUnit.MILLISECONDS); - ScheduleMessageService.this.updateOffset(this.delayLevel, nextOffset); - return; - } - } // end of for + this.scheduleNextTimerTask(resetOffset, DELAY_FOR_A_WHILE); + return; + } - nextOffset = offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE); - ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask( - this.delayLevel, nextOffset), DELAY_FOR_A_WHILE, TimeUnit.MILLISECONDS); - ScheduleMessageService.this.updateOffset(this.delayLevel, nextOffset); + long nextOffset = this.offset; + try { + int i = 0; + ConsumeQueueExt.CqExtUnit cqExtUnit = new ConsumeQueueExt.CqExtUnit(); + for (; i < bufferCQ.getSize() && isStarted(); i += ConsumeQueue.CQ_STORE_UNIT_SIZE) { + long offsetPy = bufferCQ.getByteBuffer().getLong(); + int sizePy = bufferCQ.getByteBuffer().getInt(); + long tagsCode = bufferCQ.getByteBuffer().getLong(); + + if (cq.isExtAddr(tagsCode)) { + if (cq.getExt(tagsCode, cqExtUnit)) { + tagsCode = cqExtUnit.getTagsCode(); + } else { + //can't find ext content.So re compute tags code. + log.error("[BUG] can't find consume queue extend file content!addr={}, offsetPy={}, sizePy={}", + tagsCode, offsetPy, sizePy); + long msgStoreTime = defaultMessageStore.getCommitLog().pickupStoreTimestamp(offsetPy, sizePy); + tagsCode = computeDeliverTimestamp(delayLevel, msgStoreTime); + } + } + + long now = System.currentTimeMillis(); + long deliverTimestamp = this.correctDeliverTimestamp(now, tagsCode); + nextOffset = offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE); + + long countdown = deliverTimestamp - now; + if (countdown > 0) { + this.scheduleNextTimerTask(nextOffset, DELAY_FOR_A_WHILE); return; - } finally { + } - bufferCQ.release(); + MessageExt msgExt = ScheduleMessageService.this.defaultMessageStore.lookMessageByOffset(offsetPy, sizePy); + if (msgExt == null) { + continue; } - } // end of if (bufferCQ != null) - else { - - long cqMinOffset = cq.getMinOffsetInQueue(); - long cqMaxOffset = cq.getMaxOffsetInQueue(); - if (offset < cqMinOffset) { - failScheduleOffset = cqMinOffset; - log.error("schedule CQ offset invalid. offset={}, cqMinOffset={}, cqMaxOffset={}, queueId={}", - offset, cqMinOffset, cqMaxOffset, cq.getQueueId()); + + MessageExtBrokerInner msgInner = ScheduleMessageService.this.messageTimeup(msgExt); + if (TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC.equals(msgInner.getTopic())) { + log.error("[BUG] the real topic of schedule msg is {}, discard the msg. msg={}", + msgInner.getTopic(), msgInner); + continue; + } + + boolean deliverSuc; + if (ScheduleMessageService.this.enableAsyncDeliver) { + deliverSuc = this.asyncDeliver(msgInner, msgExt.getMsgId(), offset, offsetPy, sizePy); + } else { + deliverSuc = this.syncDeliver(msgInner, msgExt.getMsgId(), offset, offsetPy, sizePy); } - if (offset > cqMaxOffset) { - failScheduleOffset = cqMaxOffset; - log.error("schedule CQ offset invalid. offset={}, cqMinOffset={}, cqMaxOffset={}, queueId={}", - offset, cqMinOffset, cqMaxOffset, cq.getQueueId()); + if (!deliverSuc) { + this.scheduleNextTimerTask(nextOffset, DELAY_FOR_A_WHILE); + return; } } - } // end of if (cq != null) - ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask(this.delayLevel, - failScheduleOffset), DELAY_FOR_A_WHILE, TimeUnit.MILLISECONDS); + nextOffset = this.offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE); + } catch (Exception e) { + log.error("ScheduleMessageService, messageTimeup execute error, offset = {}", nextOffset, e); + } finally { + bufferCQ.release(); + } + + this.scheduleNextTimerTask(nextOffset, DELAY_FOR_A_WHILE); } - private MessageExtBrokerInner messageTimeup(MessageExt msgExt) { - MessageExtBrokerInner msgInner = new MessageExtBrokerInner(); - msgInner.setBody(msgExt.getBody()); - msgInner.setFlag(msgExt.getFlag()); - MessageAccessor.setProperties(msgInner, msgExt.getProperties()); + public void scheduleNextTimerTask(long offset, long delay) { + ScheduleMessageService.this.deliverExecutorService.schedule(new DeliverDelayedMessageTimerTask( + this.delayLevel, offset), delay, TimeUnit.MILLISECONDS); + } - TopicFilterType topicFilterType = MessageExt.parseTopicFilterType(msgInner.getSysFlag()); - long tagsCodeValue = - MessageExtBrokerInner.tagsString2tagsCode(topicFilterType, msgInner.getTags()); - msgInner.setTagsCode(tagsCodeValue); - msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties())); + private boolean syncDeliver(MessageExtBrokerInner msgInner, String msgId, long offset, long offsetPy, + int sizePy) { + PutResultProcess resultProcess = deliverMessage(msgInner, msgId, offset, offsetPy, sizePy, false); + PutMessageResult result = resultProcess.get(); + boolean sendStatus = result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK; + if (sendStatus) { + ScheduleMessageService.this.updateOffset(this.delayLevel, resultProcess.getNextOffset()); + } + return sendStatus; + } - msgInner.setSysFlag(msgExt.getSysFlag()); - msgInner.setBornTimestamp(msgExt.getBornTimestamp()); - msgInner.setBornHost(msgExt.getBornHost()); - msgInner.setStoreHost(msgExt.getStoreHost()); - msgInner.setReconsumeTimes(msgExt.getReconsumeTimes()); + private boolean asyncDeliver(MessageExtBrokerInner msgInner, String msgId, long offset, long offsetPy, + int sizePy) { + Queue processesQueue = ScheduleMessageService.this.deliverPendingTable.get(this.delayLevel); + + //Flow Control + int currentPendingNum = processesQueue.size(); + int maxPendingLimit = ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig() + .getScheduleAsyncDeliverMaxPendingLimit(); + if (currentPendingNum > maxPendingLimit) { + log.warn("Asynchronous deliver triggers flow control, " + + "currentPendingNum={}, maxPendingLimit={}", currentPendingNum, maxPendingLimit); + return false; + } - msgInner.setWaitStoreMsgOK(false); - MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_DELAY_TIME_LEVEL); + //Blocked + PutResultProcess firstProcess = processesQueue.peek(); + if (firstProcess != null && firstProcess.need2Blocked()) { + log.warn("Asynchronous deliver block. info={}", firstProcess.toString()); + return false; + } - msgInner.setTopic(msgInner.getProperty(MessageConst.PROPERTY_REAL_TOPIC)); + PutResultProcess resultProcess = deliverMessage(msgInner, msgId, offset, offsetPy, sizePy, true); + processesQueue.add(resultProcess); + return true; + } - String queueIdStr = msgInner.getProperty(MessageConst.PROPERTY_REAL_QUEUE_ID); - int queueId = Integer.parseInt(queueIdStr); - msgInner.setQueueId(queueId); + private PutResultProcess deliverMessage(MessageExtBrokerInner msgInner, String msgId, long offset, + long offsetPy, int sizePy, boolean autoResend) { + CompletableFuture future = + ScheduleMessageService.this.writeMessageStore.asyncPutMessage(msgInner); + return new PutResultProcess() + .setTopic(msgInner.getTopic()) + .setDelayLevel(this.delayLevel) + .setOffset(offset) + .setPhysicOffset(offsetPy) + .setPhysicSize(sizePy) + .setMsgId(msgId) + .setAutoResend(autoResend) + .setFuture(future) + .thenProcess(); + } + } + + public class HandlePutResultTask implements Runnable { + private final int delayLevel; - return msgInner; + public HandlePutResultTask(int delayLevel) { + this.delayLevel = delayLevel; } + + @Override + public void run() { + LinkedBlockingQueue pendingQueue = + ScheduleMessageService.this.deliverPendingTable.get(this.delayLevel); + + PutResultProcess putResultProcess; + while ((putResultProcess = pendingQueue.peek()) != null) { + try { + switch (putResultProcess.getStatus()) { + case SUCCESS: + ScheduleMessageService.this.updateOffset(this.delayLevel, putResultProcess.getNextOffset()); + pendingQueue.remove(); + break; + case RUNNING: + break; + case EXCEPTION: + if (!isStarted()) { + log.warn("HandlePutResultTask shutdown, info={}", putResultProcess.toString()); + return; + } + log.warn("putResultProcess error, info={}", putResultProcess.toString()); + putResultProcess.onException(); + break; + case SKIP: + log.warn("putResultProcess skip, info={}", putResultProcess.toString()); + pendingQueue.remove(); + break; + } + } catch (Exception e) { + log.error("HandlePutResultTask exception. info={}", putResultProcess.toString(), e); + putResultProcess.onException(); + } + } + + if (isStarted()) { + ScheduleMessageService.this.handleExecutorService + .schedule(new HandlePutResultTask(this.delayLevel), DELAY_FOR_A_SLEEP, TimeUnit.MILLISECONDS); + } + } + } + + public class PutResultProcess { + private String topic; + private long offset; + private long physicOffset; + private int physicSize; + private int delayLevel; + private String msgId; + private boolean autoResend = false; + private CompletableFuture future; + + private volatile int resendCount = 0; + private volatile ProcessStatus status = ProcessStatus.RUNNING; + + public PutResultProcess setTopic(String topic) { + this.topic = topic; + return this; + } + + public PutResultProcess setOffset(long offset) { + this.offset = offset; + return this; + } + + public PutResultProcess setPhysicOffset(long physicOffset) { + this.physicOffset = physicOffset; + return this; + } + + public PutResultProcess setPhysicSize(int physicSize) { + this.physicSize = physicSize; + return this; + } + + public PutResultProcess setDelayLevel(int delayLevel) { + this.delayLevel = delayLevel; + return this; + } + + public PutResultProcess setMsgId(String msgId) { + this.msgId = msgId; + return this; + } + + public PutResultProcess setAutoResend(boolean autoResend) { + this.autoResend = autoResend; + return this; + } + + public PutResultProcess setFuture(CompletableFuture future) { + this.future = future; + return this; + } + + public String getTopic() { + return topic; + } + + public long getOffset() { + return offset; + } + + public long getNextOffset() { + return offset + 1; + } + + public long getPhysicOffset() { + return physicOffset; + } + + public int getPhysicSize() { + return physicSize; + } + + public Integer getDelayLevel() { + return delayLevel; + } + + public String getMsgId() { + return msgId; + } + + public boolean isAutoResend() { + return autoResend; + } + + public CompletableFuture getFuture() { + return future; + } + + public int getResendCount() { + return resendCount; + } + + public PutResultProcess thenProcess() { + this.future.thenAccept(result -> { + this.handleResult(result); + }); + + this.future.exceptionally(e -> { + log.error("ScheduleMessageService put message exceptionally, info: {}", + PutResultProcess.this.toString(), e); + + onException(); + return null; + }); + return this; + } + + private void handleResult(PutMessageResult result) { + if (result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK) { + onSuccess(result); + } else { + log.warn("ScheduleMessageService put message failed. info: {}.", result); + onException(); + } + } + + public void onSuccess(PutMessageResult result) { + this.status = ProcessStatus.SUCCESS; + if (ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig().isEnableScheduleMessageStats()) { + ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incQueueGetNums(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, delayLevel - 1, result.getAppendMessageResult().getMsgNum()); + ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incQueueGetSize(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, delayLevel - 1, result.getAppendMessageResult().getWroteBytes()); + ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incGroupGetNums(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, result.getAppendMessageResult().getMsgNum()); + ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incGroupGetSize(MixAll.SCHEDULE_CONSUMER_GROUP, TopicValidator.RMQ_SYS_SCHEDULE_TOPIC, result.getAppendMessageResult().getWroteBytes()); + ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incTopicPutNums(this.topic, result.getAppendMessageResult().getMsgNum(), 1); + ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incTopicPutSize(this.topic, result.getAppendMessageResult().getWroteBytes()); + ScheduleMessageService.this.defaultMessageStore.getBrokerStatsManager().incBrokerPutNums(result.getAppendMessageResult().getMsgNum()); + } + } + + public void onException() { + log.warn("ScheduleMessageService onException, info: {}", this.toString()); + if (this.autoResend) { + this.resend(); + } else { + this.status = ProcessStatus.SKIP; + } + } + + public ProcessStatus getStatus() { + return this.status; + } + + public PutMessageResult get() { + try { + return this.future.get(); + } catch (InterruptedException | ExecutionException e) { + return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, null); + } + } + + private void resend() { + log.info("Resend message, info: {}", this.toString()); + + // Gradually increase the resend interval. + try { + Thread.sleep(Math.min(this.resendCount++ * 100, 60 * 1000)); + } catch (InterruptedException e) { + e.printStackTrace(); + } + + try { + MessageExt msgExt = ScheduleMessageService.this.defaultMessageStore.lookMessageByOffset(this.physicOffset, this.physicSize); + if (msgExt == null) { + log.warn("ScheduleMessageService resend not found message. info: {}", this.toString()); + this.status = need2Skip() ? ProcessStatus.SKIP : ProcessStatus.EXCEPTION; + return; + } + + MessageExtBrokerInner msgInner = ScheduleMessageService.this.messageTimeup(msgExt); + PutMessageResult result = ScheduleMessageService.this.writeMessageStore.putMessage(msgInner); + this.handleResult(result); + if (result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK) { + log.info("Resend message success, info: {}", this.toString()); + } + } catch (Exception e) { + this.status = ProcessStatus.EXCEPTION; + log.error("Resend message error, info: {}", this.toString(), e); + } + } + + public boolean need2Blocked() { + int maxResendNum2Blocked = ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig() + .getScheduleAsyncDeliverMaxResendNum2Blocked(); + return this.resendCount > maxResendNum2Blocked; + } + + public boolean need2Skip() { + int maxResendNum2Blocked = ScheduleMessageService.this.defaultMessageStore.getMessageStoreConfig() + .getScheduleAsyncDeliverMaxResendNum2Blocked(); + return this.resendCount > maxResendNum2Blocked * 2; + } + + @Override + public String toString() { + return "PutResultProcess{" + + "topic='" + topic + '\'' + + ", offset=" + offset + + ", physicOffset=" + physicOffset + + ", physicSize=" + physicSize + + ", delayLevel=" + delayLevel + + ", msgId='" + msgId + '\'' + + ", autoResend=" + autoResend + + ", resendCount=" + resendCount + + ", status=" + status + + '}'; + } + } + + public enum ProcessStatus { + /** + * In process, the processing result has not yet been returned. + * */ + RUNNING, + + /** + * Put message success. + * */ + SUCCESS, + + /** + * Put message exception. + * When autoResend is true, the message will be resend. + * */ + EXCEPTION, + + /** + * Skip put message. + * When the message cannot be looked, the message will be skipped. + * */ + SKIP, } } diff --git a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java index bfcebd7eb85..1c0451ccc9d 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ScheduleMessageServiceTest.java @@ -18,18 +18,36 @@ package org.apache.rocketmq.store; import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Random; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.store.config.FlushDiskType; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.schedule.ScheduleMessageService; import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.Assert; import org.junit.Test; import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class ScheduleMessageServiceTest { @@ -40,7 +58,9 @@ public void testCorrectDelayOffset_whenInit() throws Exception { ConcurrentMap offsetTable = null; - ScheduleMessageService scheduleMessageService = new ScheduleMessageService((DefaultMessageStore) buildMessageStore()); + DefaultMessageStore defaultMessageStore = new DefaultMessageStore(buildMessageStoreConfig(), + new BrokerStatsManager("simpleTest", true), null, new BrokerConfig()); + ScheduleMessageService scheduleMessageService = new ScheduleMessageService(defaultMessageStore); scheduleMessageService.parseDelayLevel(); ConcurrentMap offsetTable1 = new ConcurrentHashMap<>(); @@ -71,7 +91,7 @@ public void testCorrectDelayOffset_whenInit() throws Exception { } - private MessageStore buildMessageStore() throws Exception { + private MessageStoreConfig buildMessageStoreConfig() throws Exception { MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); messageStoreConfig.setMappedFileSizeCommitLog(1024 * 1024 * 10); messageStoreConfig.setMappedFileSizeConsumeQueue(1024 * 1024 * 10); @@ -79,6 +99,96 @@ private MessageStore buildMessageStore() throws Exception { messageStoreConfig.setMaxIndexNum(100 * 100); messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); messageStoreConfig.setFlushIntervalConsumeQueue(1); - return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), null, new BrokerConfig()); + return messageStoreConfig; + } + + @Test + public void testHandlePutResultTask() throws Exception { + DefaultMessageStore messageStore = mock(DefaultMessageStore.class); + MessageStoreConfig config = buildMessageStoreConfig(); + config.setEnableScheduleMessageStats(false); + config.setEnableScheduleAsyncDeliver(true); + when(messageStore.getMessageStoreConfig()).thenReturn(config); + ScheduleMessageService scheduleMessageService = new ScheduleMessageService(messageStore); + scheduleMessageService.parseDelayLevel(); + + Field field = scheduleMessageService.getClass().getDeclaredField("deliverPendingTable"); + field.setAccessible(true); + Map> deliverPendingTable = + (Map>) field.get(scheduleMessageService); + + field = scheduleMessageService.getClass().getDeclaredField("offsetTable"); + field.setAccessible(true); + ConcurrentMap offsetTable = + (ConcurrentMap) field.get(scheduleMessageService); + for (int i = 1; i <= scheduleMessageService.getMaxDelayLevel(); i++) { + offsetTable.put(i, 0L); + } + + int deliverThreadPoolNums = Runtime.getRuntime().availableProcessors(); + ScheduledExecutorService handleExecutorService = new ScheduledThreadPoolExecutor(deliverThreadPoolNums, + new ThreadFactoryImpl("ScheduleMessageExecutorHandleThread_")); + field = scheduleMessageService.getClass().getDeclaredField("handleExecutorService"); + field.setAccessible(true); + field.set(scheduleMessageService, handleExecutorService); + + field = scheduleMessageService.getClass().getDeclaredField("started"); + field.setAccessible(true); + AtomicBoolean started = (AtomicBoolean) field.get(scheduleMessageService); + started.set(true); + + for (int level = 1; level <= scheduleMessageService.getMaxDelayLevel(); level++) { + ScheduleMessageService.HandlePutResultTask handlePutResultTask = scheduleMessageService.new HandlePutResultTask(level); + handleExecutorService.schedule(handlePutResultTask, 10L, TimeUnit.MILLISECONDS); + } + + MessageExt messageExt = new MessageExt(); + messageExt.putUserProperty("init", "test"); + messageExt.getProperties().put(MessageConst.PROPERTY_REAL_QUEUE_ID, "0"); + when(messageStore.lookMessageByOffset(anyLong(), anyInt())).thenReturn(messageExt); + when(messageStore.putMessage(any())).thenReturn(new PutMessageResult(PutMessageStatus.PUT_OK, null)); + + int msgNum = 100; + int totalMsgNum = msgNum * scheduleMessageService.getMaxDelayLevel(); + List> putMsgFutrueList = new ArrayList<>(totalMsgNum); + for (int level = 1; level <= scheduleMessageService.getMaxDelayLevel(); level++) { + for (int num = 0; num < msgNum; num++) { + CompletableFuture future = new CompletableFuture<>(); + ScheduleMessageService.PutResultProcess putResultProcess = scheduleMessageService.new PutResultProcess(); + putResultProcess = putResultProcess + .setOffset(num) + .setAutoResend(true) + .setFuture(future) + .thenProcess(); + deliverPendingTable.get(level).add(putResultProcess); + putMsgFutrueList.add(future); + } + } + + Collections.shuffle(putMsgFutrueList); + Random random = new Random(); + for (CompletableFuture future : putMsgFutrueList) { + PutMessageStatus status; + if (random.nextInt(1000) % 2 == 0) { + status = PutMessageStatus.PUT_OK; + } else { + status = PutMessageStatus.OS_PAGECACHE_BUSY; + } + + if (random.nextInt(1000) % 2 == 0) { + PutMessageResult result = new PutMessageResult(status, null); + future.complete(result); + } else { + future.completeExceptionally(new Throwable("complete exceptionally")); + } + } + + Thread.sleep(1000); + for (int level = 1; level <= scheduleMessageService.getMaxDelayLevel(); level++) { + Assert.assertEquals(0, deliverPendingTable.get(level).size()); + Assert.assertEquals(msgNum, offsetTable.get(level).longValue()); + } + + scheduleMessageService.shutdown(); } } From b46496d11b51f0454652ade3171dabcbc2b0921b Mon Sep 17 00:00:00 2001 From: Colin678 <1020084984@qq.com> Date: Fri, 21 Jan 2022 09:50:13 +0800 Subject: [PATCH 108/141] Remove useless code (#3695) * [Simple packaging method]packaging method * [Remove useless code]Maybe dever forgot to delete it * [Remove useless notes]Wrong commit --- .../java/org/apache/rocketmq/common/sysflag/TopicSysFlag.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/common/src/main/java/org/apache/rocketmq/common/sysflag/TopicSysFlag.java b/common/src/main/java/org/apache/rocketmq/common/sysflag/TopicSysFlag.java index cc23ff1a3f3..a2bb5084051 100644 --- a/common/src/main/java/org/apache/rocketmq/common/sysflag/TopicSysFlag.java +++ b/common/src/main/java/org/apache/rocketmq/common/sysflag/TopicSysFlag.java @@ -59,7 +59,4 @@ public static int clearUnitSubFlag(final int sysFlag) { public static boolean hasUnitSubFlag(final int sysFlag) { return (sysFlag & FLAG_UNIT_SUB) == FLAG_UNIT_SUB; } - - public static void main(String[] args) { - } } From e50bf54c63e4c2d2804ac24e4dc0d50cda545564 Mon Sep 17 00:00:00 2001 From: Heng Du Date: Fri, 21 Jan 2022 10:22:29 +0800 Subject: [PATCH 109/141] Update README.md --- README.md | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 79b316c2987..a4293d5c606 100644 --- a/README.md +++ b/README.md @@ -31,9 +31,23 @@ It offers a variety of features: * Feature-rich administrative dashboard for configuration, metrics and monitoring * Authentication and authorization * Free open source connectors, for both sources and sinks - +* Lightweight real-time computing ---------- +## Apache RocketMQ Community +* [RocketMQ Streams](https://github.com/apache/rocketmq-streams) +* [RocketMQ-Flink](https://github.com/apache/rocketmq-flink) +* [RocketMQ Client CPP](https://github.com/apache/rocketmq-client-cpp) +* [RocketMQ Client Go](https://github.com/apache/rocketmq-client-go) +* [RocketMQ Client Python](https://github.com/apache/rocketmq-client-python) +* [RocketMQ Client Nodejs](https://github.com/apache/rocketmq-client-nodejs) +* [RocketMQ Spring](https://github.com/apache/rocketmq-spring) +* [RocketMQ-Exporter](https://github.com/apache/rocketmq-exporter) +* [RocketMQ Operator](https://github.com/apache/rocketmq-operator) +* [RocketMQ-Docker](https://github.com/apache/rocketmq-docker) +* [RocketMQ Incubating Community Projects](https://github.com/apache/rocketmq-externals) + +---------- ## Learn it & Contact us * Mailing Lists: * Home: @@ -46,9 +60,7 @@ It offers a variety of features: ---------- -## Apache RocketMQ Community -* [RocketMQ Community Projects](https://github.com/apache/rocketmq-externals) ----------- + ## Contributing We always welcome new contributions, whether for trivial cleanups, [big new features](https://github.com/apache/rocketmq/wiki/RocketMQ-Improvement-Proposal) or other material rewards, more details see [here](http://rocketmq.apache.org/docs/how-to-contribute/). From ad5fd7fe634cd59a7eb0ceef93a4d9388ecec43f Mon Sep 17 00:00:00 2001 From: Heng Du Date: Fri, 21 Jan 2022 10:23:01 +0800 Subject: [PATCH 110/141] Update README.md --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index a4293d5c606..0aca8a7fa01 100644 --- a/README.md +++ b/README.md @@ -36,13 +36,13 @@ It offers a variety of features: ## Apache RocketMQ Community * [RocketMQ Streams](https://github.com/apache/rocketmq-streams) -* [RocketMQ-Flink](https://github.com/apache/rocketmq-flink) +* [RocketMQ Flink](https://github.com/apache/rocketmq-flink) * [RocketMQ Client CPP](https://github.com/apache/rocketmq-client-cpp) * [RocketMQ Client Go](https://github.com/apache/rocketmq-client-go) * [RocketMQ Client Python](https://github.com/apache/rocketmq-client-python) * [RocketMQ Client Nodejs](https://github.com/apache/rocketmq-client-nodejs) * [RocketMQ Spring](https://github.com/apache/rocketmq-spring) -* [RocketMQ-Exporter](https://github.com/apache/rocketmq-exporter) +* [RocketMQ Exporter](https://github.com/apache/rocketmq-exporter) * [RocketMQ Operator](https://github.com/apache/rocketmq-operator) * [RocketMQ-Docker](https://github.com/apache/rocketmq-docker) * [RocketMQ Incubating Community Projects](https://github.com/apache/rocketmq-externals) From e8cae430d4677db3d52af6f92e8d2aba2f8e5e0d Mon Sep 17 00:00:00 2001 From: Heng Du Date: Fri, 21 Jan 2022 10:24:01 +0800 Subject: [PATCH 111/141] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 0aca8a7fa01..41392af2eae 100644 --- a/README.md +++ b/README.md @@ -44,7 +44,7 @@ It offers a variety of features: * [RocketMQ Spring](https://github.com/apache/rocketmq-spring) * [RocketMQ Exporter](https://github.com/apache/rocketmq-exporter) * [RocketMQ Operator](https://github.com/apache/rocketmq-operator) -* [RocketMQ-Docker](https://github.com/apache/rocketmq-docker) +* [RocketMQ Docker](https://github.com/apache/rocketmq-docker) * [RocketMQ Incubating Community Projects](https://github.com/apache/rocketmq-externals) ---------- From bda0ae0305863cc5fa2e0578439dfa4f4d9908b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Fri, 21 Jan 2022 15:30:01 +0800 Subject: [PATCH 112/141] [ISSUE #3724]: Polish the unit test of class ConsumeMessageConcurrentlyService. (#3725) --- ...ConsumeMessageConcurrentlyServiceTest.java | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java index 5d69aa2251e..7badc3b1433 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java @@ -19,12 +19,16 @@ import java.io.ByteArrayOutputStream; import java.lang.reflect.Field; import java.net.InetSocketAddress; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; + +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.consumer.PullCallback; import org.apache.rocketmq.client.consumer.PullResult; @@ -36,6 +40,7 @@ import org.apache.rocketmq.client.impl.CommunicationMode; import org.apache.rocketmq.client.impl.FindBrokerResult; import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.MQClientManager; import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.client.stat.ConsumerStatsManager; import org.apache.rocketmq.common.message.MessageClientExt; @@ -45,10 +50,10 @@ import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader; import org.apache.rocketmq.common.stats.StatsItem; import org.apache.rocketmq.common.stats.StatsItemSet; +import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.exception.RemotingException; import org.junit.After; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -81,6 +86,13 @@ public class ConsumeMessageConcurrentlyServiceTest { @Before public void init() throws Exception { + ConcurrentMap factoryTable = (ConcurrentMap) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true); + Collection instances = factoryTable.values(); + for (MQClientInstance instance : instances) { + instance.shutdown(); + } + factoryTable.clear(); + consumerGroup = "FooBarGroup" + System.currentTimeMillis(); pushConsumer = new DefaultMQPushConsumer(consumerGroup); pushConsumer.setNamesrvAddr("127.0.0.1:9876"); @@ -100,12 +112,15 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, field.setAccessible(true); field.set(pushConsumerImpl, rebalancePushImpl); pushConsumer.subscribe(topic, "*"); - pushConsumer.start(); - mQClientFactory = spy(pushConsumerImpl.getmQClientFactory()); + // suppress updateTopicRouteInfoFromNameServer + pushConsumer.changeInstanceNameToPID(); + mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(pushConsumer, (RPCHook) FieldUtils.readDeclaredField(pushConsumerImpl, "rpcHook", true)); + mQClientFactory = spy(mQClientFactory); field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory"); field.setAccessible(true); field.set(pushConsumerImpl, mQClientFactory); + factoryTable.put(pushConsumer.buildMQClientId(), mQClientFactory); field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); field.setAccessible(true); @@ -117,7 +132,6 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, field.set(pushConsumerImpl, pullAPIWrapper); pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory); - mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl); when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), nullable(PullCallback.class))) @@ -140,12 +154,13 @@ public PullResult answer(InvocationOnMock mock) throws Throwable { }); doReturn(new FindBrokerResult("127.0.0.1:10912", false)).when(mQClientFactory).findBrokerAddressInSubscribe(anyString(), anyLong(), anyBoolean()); + doReturn(false).when(mQClientFactory).updateTopicRouteInfoFromNameServer(anyString()); Set messageQueueSet = new HashSet(); messageQueueSet.add(createPullRequest().getMessageQueue()); pushConsumer.getDefaultMQPushConsumerImpl().updateTopicSubscribeInfo(topic, messageQueueSet); + pushConsumer.start(); } - @Ignore @Test public void testPullMessage_ConsumeSuccess() throws InterruptedException, RemotingException, MQBrokerException, NoSuchFieldException,Exception { final CountDownLatch countDownLatch = new CountDownLatch(1); From 35334e266984c4a3ee6179ffd44bb37372788696 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Sat, 22 Jan 2022 21:06:25 +0800 Subject: [PATCH 113/141] [ISSUE #3782] docs: Fix the default value of parameter 'sendMsgTimeout'. (#3783) Co-authored-by: penglong --- docs/cn/best_practice.md | 2 +- docs/en/Configuration_Client.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/cn/best_practice.md b/docs/cn/best_practice.md index 6f9e844dca8..caf0e0ac208 100755 --- a/docs/cn/best_practice.md +++ b/docs/cn/best_practice.md @@ -264,7 +264,7 @@ DefaultMQProducer、TransactionMQProducer、DefaultMQPushConsumer、DefaultMQPul | producerGroup | DEFAULT_PRODUCER | Producer组名,多个Producer如果属于一个应用,发送同样的消息,则应该将它们归为同一组 | | createTopicKey | TBW102 | 在发送消息时,自动创建服务器不存在的topic,需要指定Key,该Key可用于配置发送消息所在topic的默认路由。 | | defaultTopicQueueNums | 4 | 在发送消息,自动创建服务器不存在的topic时,默认创建的队列数 | -| sendMsgTimeout | 10000 | 发送消息超时时间,单位毫秒 | +| sendMsgTimeout | 3000 | 发送消息超时时间,单位毫秒 | | compressMsgBodyOverHowmuch | 4096 | 消息Body超过多大开始压缩(Consumer收到消息会自动解压缩),单位字节 | | retryAnotherBrokerWhenNotStoreOK | FALSE | 如果发送消息返回sendResult,但是sendStatus!=SEND_OK,是否重试发送 | | retryTimesWhenSendFailed | 2 | 如果消息发送失败,最大重试次数,该参数只对同步发送模式起作用 | diff --git a/docs/en/Configuration_Client.md b/docs/en/Configuration_Client.md index 0fcd95fcffd..7aabaa862a5 100644 --- a/docs/en/Configuration_Client.md +++ b/docs/en/Configuration_Client.md @@ -59,7 +59,7 @@ HTTP static server addressing is recommended, because it is simple client deploy | producerGroup | DEFAULT_PRODUCER | The name of the Producer group. If multiple producers belong to one application and send the same message, they should be grouped into the same group | | createTopicKey | TBW102 | When a message is sent, topics that do not exist on the server are automatically created and a Key is specified that can be used to configure the default route to the topic where the message is sent.| | defaultTopicQueueNums | 4 | The number of default queue when sending messages and auto created topic which not exists the server| -| sendMsgTimeout | 10000 | Timeout time of sending message in milliseconds | +| sendMsgTimeout | 3000 | Timeout time of sending message in milliseconds | | compressMsgBodyOverHowmuch | 4096 | The message Body begins to compress beyond the size(the Consumer gets the message automatically unzipped.), unit of byte| | retryAnotherBrokerWhenNotStoreOK | FALSE | If send message and return sendResult but sendStatus!=SEND_OK, Whether to resend | | retryTimesWhenSendFailed | 2 | If send message failed, maximum number of retries, this parameter only works for synchronous send mode| From 8fdfb42ac8ceea24c7c1b6352df59ff8b585acbb Mon Sep 17 00:00:00 2001 From: zplovekq <39880124+zplovekq@users.noreply.github.com> Date: Mon, 24 Jan 2022 13:54:08 +0800 Subject: [PATCH 114/141] [ISSUES #3048]add example of OnewayProducer and ScheduledMessage (#3053) * fix the issue #2994 add the example for OnewayProducer and ScheduledMessageConsumer. remove a unused line and add notes. * rollback * change code style * modify the code style Co-authored-by: zoupeinie --- .../schedule/ScheduledMessageConsumer.java | 51 +++++++++++++++++++ .../schedule/ScheduledMessageProducer.java | 41 +++++++++++++++ .../example/simple/OnewayProducer.java | 45 ++++++++++++++++ 3 files changed, 137 insertions(+) create mode 100644 example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageConsumer.java create mode 100644 example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageProducer.java create mode 100644 example/src/main/java/org/apache/rocketmq/example/simple/OnewayProducer.java diff --git a/example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageConsumer.java b/example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageConsumer.java new file mode 100644 index 00000000000..fdb4c86eb1f --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageConsumer.java @@ -0,0 +1,51 @@ +/* + * 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.example.schedule; + +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.common.message.MessageExt; + +import java.util.List; + +public class ScheduledMessageConsumer { + + public static void main(String[] args) throws Exception { + // Instantiate message consumer + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("ExampleConsumer"); + // Subscribe topics + consumer.subscribe("TestTopic", "*"); + // Register message listener + consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List messages, ConsumeConcurrentlyContext context) { + for (MessageExt message : messages) { + // Print approximate delay time period + System.out.printf("Receive message[msgId=%s %d ms later]\n", message.getMsgId(), + System.currentTimeMillis() - message.getStoreTimestamp()); + } + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + // Launch consumer + consumer.start(); + //info:to see the time effect, run the consumer first , it will wait for the msg + //then start the producer + } +} \ No newline at end of file diff --git a/example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageProducer.java b/example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageProducer.java new file mode 100644 index 00000000000..0f6b722d530 --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/schedule/ScheduledMessageProducer.java @@ -0,0 +1,41 @@ +/* + * 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.example.schedule; + +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.common.message.Message; + +public class ScheduledMessageProducer { + public static void main(String[] args) throws Exception { + // Instantiate a producer to send scheduled messages + DefaultMQProducer producer = new DefaultMQProducer("ExampleProducerGroup"); + // Launch producer + producer.start(); + int totalMessagesToSend = 100; + for (int i = 0; i < totalMessagesToSend; i++) { + Message message = new Message("TestTopic", ("Hello scheduled message " + i).getBytes()); + // This message will be delivered to consumer 10 seconds later. + message.setDelayTimeLevel(3); + // Send the message + producer.send(message); + } + + // Shutdown producer after use. + producer.shutdown(); + } + +} \ No newline at end of file diff --git a/example/src/main/java/org/apache/rocketmq/example/simple/OnewayProducer.java b/example/src/main/java/org/apache/rocketmq/example/simple/OnewayProducer.java new file mode 100644 index 00000000000..6932f669fc1 --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/simple/OnewayProducer.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.example.simple; + +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.remoting.common.RemotingHelper; + +public class OnewayProducer { + public static void main(String[] args) throws Exception { + //Instantiate with a producer group name. + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + // Specify name server addresses. + producer.setNamesrvAddr("localhost:9876"); + //Launch the instance. + producer.start(); + for (int i = 0; i < 100; i++) { + //Create a message instance, specifying topic, tag and message body. + Message msg = new Message("TopicTest" /* Topic */, + "TagA" /* Tag */, + ("Hello RocketMQ " + + i).getBytes(RemotingHelper.DEFAULT_CHARSET) /* Message body */ + ); + //Call send message to deliver message to one of brokers. + producer.sendOneway(msg); + } + //Wait for sending to complete + Thread.sleep(5000); + producer.shutdown(); + } +} \ No newline at end of file From 493c91a38be3be5f101565aac99b7e170d659fdf Mon Sep 17 00:00:00 2001 From: Kvicii <42023367+Kvicii@users.noreply.github.com> Date: Wed, 26 Jan 2022 10:42:31 +0800 Subject: [PATCH 115/141] deprecated DefaultMQPullConsumer (#3796) If there is no explanation, you need to explicitly specify the replacement class Co-authored-by: Kvicii --- .../apache/rocketmq/client/consumer/DefaultMQPullConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java index 0876a94e4c5..1c3f3da4d6e 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java @@ -35,6 +35,7 @@ import org.apache.rocketmq.remoting.exception.RemotingException; /** + * @deprecated * Default pulling consumer. * This class will be removed in 2022, and a better implementation {@link DefaultLitePullConsumer} is recommend to use * in the scenario of actively pulling messages. From 8ce2cdb004a6b968ad51a985fa262b84fa97d041 Mon Sep 17 00:00:00 2001 From: Xinda Date: Wed, 26 Jan 2022 10:44:01 +0800 Subject: [PATCH 116/141] perf: avoid multiple expansion when the number of elements in the `MessageConst` class is determined (#3721) `HashSet`, threshold = initialCapacity * 0.75 --- .../java/org/apache/rocketmq/common/message/MessageConst.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java index ba9b7443cbb..81b7823f3b5 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageConst.java @@ -57,7 +57,7 @@ public class MessageConst { public static final String KEY_SEPARATOR = " "; - public static final HashSet STRING_HASH_SET = new HashSet(); + public static final HashSet STRING_HASH_SET = new HashSet<>(64); static { STRING_HASH_SET.add(PROPERTY_TRACE_SWITCH); From 30d1c58a0c8909797a010220f67e41a15b13abf4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Thu, 27 Jan 2022 14:29:32 +0800 Subject: [PATCH 117/141] [ISSUE #3801] Polish the document 'CLITools.md'. (#3802) --- docs/en/CLITools.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/CLITools.md b/docs/en/CLITools.md index c53b69bf8c0..596e84e7703 100644 --- a/docs/en/CLITools.md +++ b/docs/en/CLITools.md @@ -35,7 +35,7 @@ Before introducing the mqadmin management tool, the following points need to be The -c option declares the name of the cluster, which represents the cluster in which the current topic is located. (clusters are available through clusterList query) - -h- + -h Print help information @@ -815,7 +815,7 @@ Before introducing the mqadmin management tool, the following points need to be resetOffsetByTime Reset both offset,broker and consumer by timestamp + border-top:none;width:65pt'>Reset consumer offset by timestamp(without client restart). -h Print help information @@ -833,7 +833,7 @@ Before introducing the mqadmin management tool, the following points need to be -s - Resets the offset corresponding to this timestamp + Resets the offset corresponding to this timestamp in a format see -h, if you want to reset to maxOffset, the value is 'now'. -f From d5be91fa00136d8c2df37d83ea94dab37d630939 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Thu, 27 Jan 2022 14:58:09 +0800 Subject: [PATCH 118/141] [ISSUE #3789] optimize: Tag the name of consuming thread whith consumeGroup. (#3795) * It is useful for debug. --- .../ConsumeMessageConcurrentlyService.java | 8 +- .../ConsumeMessageOrderlyService.java | 8 +- ...ConsumeMessageConcurrentlyServiceTest.java | 32 ++++ .../ConsumeMessageOrderlyServiceTest.java | 177 ++++++++++++++++++ 4 files changed, 223 insertions(+), 2 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java index 537dbee832d..384f3f12bee 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java @@ -70,13 +70,19 @@ public ConsumeMessageConcurrentlyService(DefaultMQPushConsumerImpl defaultMQPush this.consumerGroup = this.defaultMQPushConsumer.getConsumerGroup(); this.consumeRequestQueue = new LinkedBlockingQueue(); + String consumeThreadPrefix = null; + if (consumerGroup.length() > 100) { + consumeThreadPrefix = new StringBuilder("ConsumeMessageThread_").append(consumerGroup.substring(0, 100)).append("_").toString(); + } else { + consumeThreadPrefix = new StringBuilder("ConsumeMessageThread_").append(consumerGroup).append("_").toString(); + } this.consumeExecutor = new ThreadPoolExecutor( this.defaultMQPushConsumer.getConsumeThreadMin(), this.defaultMQPushConsumer.getConsumeThreadMax(), 1000 * 60, TimeUnit.MILLISECONDS, this.consumeRequestQueue, - new ThreadFactoryImpl("ConsumeMessageThread_")); + new ThreadFactoryImpl(consumeThreadPrefix)); this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("ConsumeMessageScheduledThread_")); this.cleanExpireMsgExecutors = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("CleanExpireMsgScheduledThread_")); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java index 8d92b5718a6..812e8ab87dd 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java @@ -75,13 +75,19 @@ public ConsumeMessageOrderlyService(DefaultMQPushConsumerImpl defaultMQPushConsu this.consumerGroup = this.defaultMQPushConsumer.getConsumerGroup(); this.consumeRequestQueue = new LinkedBlockingQueue(); + String consumeThreadPrefix = null; + if (consumerGroup.length() > 100) { + consumeThreadPrefix = new StringBuilder("ConsumeMessageThread_").append(consumerGroup.substring(0, 100)).append("_").toString(); + } else { + consumeThreadPrefix = new StringBuilder("ConsumeMessageThread_").append(consumerGroup).append("_").toString(); + } this.consumeExecutor = new ThreadPoolExecutor( this.defaultMQPushConsumer.getConsumeThreadMin(), this.defaultMQPushConsumer.getConsumeThreadMax(), 1000 * 60, TimeUnit.MILLISECONDS, this.consumeRequestQueue, - new ThreadFactoryImpl("ConsumeMessageThread_")); + new ThreadFactoryImpl(consumeThreadPrefix)); this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("ConsumeMessageScheduledThread_")); } diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java index 7badc3b1433..c12f2fc9e05 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java @@ -231,4 +231,36 @@ private PullResultExt createPullResult(PullMessageRequestHeader requestHeader, P } return new PullResultExt(pullStatus, requestHeader.getQueueOffset() + messageExtList.size(), 123, 2048, messageExtList, 0, outputStream.toByteArray()); } + + @Test + public void testConsumeThreadName() throws Exception { + final CountDownLatch countDownLatch = new CountDownLatch(1); + final AtomicReference consumeThreadName = new AtomicReference(); + + StringBuilder consumeGroup2 = new StringBuilder(); + for (int i = 0; i < 101; i++) { + consumeGroup2.append(i).append("#"); + } + pushConsumer.setConsumerGroup(consumeGroup2.toString()); + ConsumeMessageConcurrentlyService normalServie = new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, + ConsumeConcurrentlyContext context) { + consumeThreadName.set(Thread.currentThread().getName()); + countDownLatch.countDown(); + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(normalServie); + + PullMessageService pullMessageService = mQClientFactory.getPullMessageService(); + pullMessageService.executePullRequestImmediately(createPullRequest()); + countDownLatch.await(); + System.out.println(consumeThreadName.get()); + if (consumeGroup2.length() <= 100) { + assertThat(consumeThreadName.get()).startsWith("ConsumeMessageThread_" + consumeGroup2 + "_"); + } else { + assertThat(consumeThreadName.get()).startsWith("ConsumeMessageThread_" + consumeGroup2.substring(0, 100) + "_"); + } + } } diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyServiceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyServiceTest.java index 4cfa011434c..8ea1727a455 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyServiceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyServiceTest.java @@ -16,30 +16,146 @@ */ package org.apache.rocketmq.client.impl.consumer; +import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.PullCallback; +import org.apache.rocketmq.client.consumer.PullResult; +import org.apache.rocketmq.client.consumer.PullStatus; import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyContext; import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; +import org.apache.rocketmq.client.impl.CommunicationMode; +import org.apache.rocketmq.client.impl.FindBrokerResult; +import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.MQClientManager; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.message.MessageClientExt; +import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.body.CMResult; +import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader; +import org.apache.rocketmq.remoting.RPCHook; import org.junit.Before; import org.junit.Test; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; public class ConsumeMessageOrderlyServiceTest { private String consumerGroup; private String topic = "FooBar"; private String brokerName = "BrokerA"; private DefaultMQPushConsumer pushConsumer; + private MQClientInstance mQClientFactory; + @Mock + private MQClientAPIImpl mQClientAPIImpl; + private PullAPIWrapper pullAPIWrapper; + private RebalancePushImpl rebalancePushImpl; @Before public void init() throws Exception { + ConcurrentMap factoryTable = (ConcurrentMap) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true); + Collection instances = factoryTable.values(); + for (MQClientInstance instance : instances) { + instance.shutdown(); + } + factoryTable.clear(); consumerGroup = "FooBarGroup" + System.currentTimeMillis(); pushConsumer = new DefaultMQPushConsumer(consumerGroup); + + pushConsumer.setNamesrvAddr("127.0.0.1:9876"); + pushConsumer.setPullInterval(60 * 1000); + + pushConsumer.registerMessageListener(new MessageListenerOrderly() { + + @Override + public ConsumeOrderlyStatus consumeMessage(List msgs, ConsumeOrderlyContext context) { + return ConsumeOrderlyStatus.SUCCESS; + } + }); + + + DefaultMQPushConsumerImpl pushConsumerImpl = pushConsumer.getDefaultMQPushConsumerImpl(); + rebalancePushImpl = spy(new RebalancePushImpl(pushConsumer.getDefaultMQPushConsumerImpl())); + Field field = DefaultMQPushConsumerImpl.class.getDeclaredField("rebalanceImpl"); + field.setAccessible(true); + field.set(pushConsumerImpl, rebalancePushImpl); + pushConsumer.subscribe(topic, "*"); + + // suppress updateTopicRouteInfoFromNameServer + pushConsumer.changeInstanceNameToPID(); + mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(pushConsumer, (RPCHook) FieldUtils.readDeclaredField(pushConsumerImpl, "rpcHook", true)); + mQClientFactory = spy(mQClientFactory); + field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory"); + field.setAccessible(true); + field.set(pushConsumerImpl, mQClientFactory); + factoryTable.put(pushConsumer.buildMQClientId(), mQClientFactory); + + mQClientAPIImpl = mock(MQClientAPIImpl.class); + field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); + field.setAccessible(true); + field.set(mQClientFactory, mQClientAPIImpl); + + pullAPIWrapper = spy(new PullAPIWrapper(mQClientFactory, consumerGroup, false)); + field = DefaultMQPushConsumerImpl.class.getDeclaredField("pullAPIWrapper"); + field.setAccessible(true); + field.set(pushConsumerImpl, pullAPIWrapper); + + pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory); + + when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), + anyLong(), any(CommunicationMode.class), nullable(PullCallback.class))) + .thenAnswer(new Answer() { + @Override + public PullResult answer(InvocationOnMock mock) throws Throwable { + PullMessageRequestHeader requestHeader = mock.getArgument(1); + MessageClientExt messageClientExt = new MessageClientExt(); + messageClientExt.setTopic(topic); + messageClientExt.setQueueId(0); + messageClientExt.setMsgId("123"); + messageClientExt.setBody(new byte[] {'a'}); + messageClientExt.setOffsetMsgId("234"); + messageClientExt.setBornHost(new InetSocketAddress(8080)); + messageClientExt.setStoreHost(new InetSocketAddress(8080)); + PullResult pullResult = createPullResult(requestHeader, PullStatus.FOUND, Collections.singletonList(messageClientExt)); + ((PullCallback) mock.getArgument(4)).onSuccess(pullResult); + return pullResult; + } + }); + + doReturn(new FindBrokerResult("127.0.0.1:10912", false)).when(mQClientFactory).findBrokerAddressInSubscribe(anyString(), anyLong(), anyBoolean()); + doReturn(false).when(mQClientFactory).updateTopicRouteInfoFromNameServer(anyString()); + Set messageQueueSet = new HashSet(); + messageQueueSet.add(createPullRequest().getMessageQueue()); + pushConsumer.getDefaultMQPushConsumerImpl().updateTopicSubscribeInfo(topic, messageQueueSet); + pushConsumer.start(); } @Test @@ -83,4 +199,65 @@ public ConsumeOrderlyStatus consumeMessage(List msgs, ConsumeOrderly assertTrue(consumeMessageOrderlyService.consumeMessageDirectly(msg, brokerName).getConsumeResult().equals(CMResult.CR_THROW_EXCEPTION)); } + @Test + public void testConsumeThreadName() throws Exception { + final CountDownLatch countDownLatch = new CountDownLatch(1); + final AtomicReference consumeThreadName = new AtomicReference(); + + StringBuilder consumeGroup2 = new StringBuilder(); + for (int i = 0; i < 101; i++) { + consumeGroup2.append(i).append("#"); + } + pushConsumer.setConsumerGroup(consumeGroup2.toString()); + + MessageListenerOrderly listenerOrderly = new MessageListenerOrderly() { + @Override + public ConsumeOrderlyStatus consumeMessage(List msgs, ConsumeOrderlyContext context) { + consumeThreadName.set(Thread.currentThread().getName()); + countDownLatch.countDown(); + return ConsumeOrderlyStatus.SUCCESS; + } + }; + ConsumeMessageOrderlyService consumeMessageOrderlyService = new ConsumeMessageOrderlyService(pushConsumer.getDefaultMQPushConsumerImpl(), listenerOrderly); + pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(consumeMessageOrderlyService); + + + PullMessageService pullMessageService = mQClientFactory.getPullMessageService(); + pullMessageService.executePullRequestImmediately(createPullRequest()); + countDownLatch.await(); + System.out.println(consumeThreadName.get()); + if (consumeGroup2.length() <= 100) { + assertThat(consumeThreadName.get()).startsWith("ConsumeMessageThread_" + consumeGroup2 + "_"); + } else { + assertThat(consumeThreadName.get()).startsWith("ConsumeMessageThread_" + consumeGroup2.substring(0, 100) + "_"); + } + } + + private PullRequest createPullRequest() { + PullRequest pullRequest = new PullRequest(); + pullRequest.setConsumerGroup(consumerGroup); + pullRequest.setNextOffset(1024); + + MessageQueue messageQueue = new MessageQueue(); + messageQueue.setBrokerName(brokerName); + messageQueue.setQueueId(0); + messageQueue.setTopic(topic); + pullRequest.setMessageQueue(messageQueue); + ProcessQueue processQueue = new ProcessQueue(); + processQueue.setLocked(true); + processQueue.setLastLockTimestamp(System.currentTimeMillis()); + pullRequest.setProcessQueue(processQueue); + + return pullRequest; + } + + private PullResultExt createPullResult(PullMessageRequestHeader requestHeader, PullStatus pullStatus, + List messageExtList) throws Exception { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + for (MessageExt messageExt : messageExtList) { + outputStream.write(MessageDecoder.encode(messageExt, false)); + } + return new PullResultExt(pullStatus, requestHeader.getQueueOffset() + messageExtList.size(), 123, 2048, messageExtList, 0, outputStream.toByteArray()); + } + } From f21917085166237d5cf9331aad82c1a8c6f0f46c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E5=B0=8F=E6=BC=AA?= <644120242@qq.com> Date: Thu, 3 Feb 2022 17:54:46 +0800 Subject: [PATCH 119/141] [ISSUE #3786] fix: The length of properties value should be checked before converting them to short. (#3788) --- .../org/apache/rocketmq/common/message/MessageDecoder.java | 5 ++++- .../src/main/java/org/apache/rocketmq/store/CommitLog.java | 7 ++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java index c94700e5feb..929912772e5 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java @@ -478,7 +478,10 @@ public static byte[] encodeMessage(Message message) { String properties = messageProperties2String(message.getProperties()); byte[] propertiesBytes = properties.getBytes(CHARSET_UTF8); //note properties length must not more than Short.MAX - short propertiesLength = (short) propertiesBytes.length; + int propsLen = propertiesBytes.length; + if (propsLen > Short.MAX_VALUE) + throw new RuntimeException(String.format("Properties size of message exceeded, properties size: {}, maxSize: {}.", propsLen, Short.MAX_VALUE)); + short propertiesLength = (short) propsLen; int sysFlag = message.getFlag(); int storeSize = 4 // 1 TOTALSIZE + 4 // 2 MAGICCOD diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 73fd361a32d..c10ffc32708 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -1606,7 +1606,12 @@ protected ByteBuffer encode(final MessageExtBatch messageExtBatch, PutMessageCon // properties from MessageExtBatch String batchPropStr = MessageDecoder.messageProperties2String(messageExtBatch.getProperties()); final byte[] batchPropData = batchPropStr.getBytes(MessageDecoder.CHARSET_UTF8); - final short batchPropLen = (short) batchPropData.length; + int batchPropDataLen = batchPropData.length; + if (batchPropDataLen > Short.MAX_VALUE) { + CommitLog.log.warn("Properties size of messageExtBatch exceeded, properties size: {}, maxSize: {}.", batchPropDataLen, Short.MAX_VALUE); + throw new RuntimeException("Properties size of messageExtBatch exceeded!"); + } + final short batchPropLen = (short) batchPropDataLen; int batchSize = 0; while (messagesByteBuff.hasRemaining()) { From 7496081da676f4d2cb8f9c2a256a3aa692cdf2bf Mon Sep 17 00:00:00 2001 From: Kvicii <42023367+Kvicii@users.noreply.github.com> Date: Sat, 5 Feb 2022 16:56:14 +0800 Subject: [PATCH 120/141] fix ISSUE 2812 (#3813) --- .../rocketmq/client/impl/producer/DefaultMQProducerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java index b6e99ba0e33..9afbe9bcb70 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java @@ -732,7 +732,7 @@ private SendResult sendKernelImpl(final Message msg, } final String tranMsg = msg.getProperty(MessageConst.PROPERTY_TRANSACTION_PREPARED); - if (tranMsg != null && Boolean.parseBoolean(tranMsg)) { + if (Boolean.parseBoolean(tranMsg)) { sysFlag |= MessageSysFlag.TRANSACTION_PREPARED_TYPE; } From 32dd2d82c10e7761a80b494dfb691c08a2dcbb0a Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Sat, 5 Feb 2022 16:44:21 +0100 Subject: [PATCH 121/141] [Issue-3816] upgrade log4jv2 to 2.17.1 (#3817) https://github.com/apache/rocketmq/issues/3816 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index c2c8f8b3bdf..cb362d87594 100644 --- a/pom.xml +++ b/pom.xml @@ -594,12 +594,12 @@ org.apache.logging.log4j log4j-core - 2.15.0 + 2.17.1 org.apache.logging.log4j log4j-slf4j-impl - 2.15.0 + 2.17.1 commons-validator From 32511145ae88d8c975f092a52ccc75dce49248cb Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Sun, 6 Feb 2022 13:41:19 +0100 Subject: [PATCH 122/141] [Issue-3814] use snakeyaml 1.30 (#3815) due to a CVE --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index cb362d87594..6411015b93c 100644 --- a/pom.xml +++ b/pom.xml @@ -584,7 +584,7 @@ org.yaml snakeyaml - 1.19 + 1.30 commons-codec From dbbc88a6fe3449f13fdf8584f74e98ac69613349 Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Mon, 7 Feb 2022 02:44:30 +0100 Subject: [PATCH 123/141] guava 31.0.1-jre (#3821) https://github.com/apache/rocketmq/issues/3818 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6411015b93c..1671bd1f29b 100644 --- a/pom.xml +++ b/pom.xml @@ -569,7 +569,7 @@ com.google.guava guava - 19.0 + 31.0.1-jre io.openmessaging From d5c88001252c64dc273fea0e07932ef007513d22 Mon Sep 17 00:00:00 2001 From: yuz10 <845238369@qq.com> Date: Mon, 7 Feb 2022 23:51:02 +0800 Subject: [PATCH 124/141] add tests for AllocateMessageQueueAveragely (#3811) --- .../AllocateMessageQueueAveragelyTest.java | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyTest.java diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyTest.java new file mode 100644 index 00000000000..b486106783f --- /dev/null +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyTest.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.client.consumer.rebalance; + +import junit.framework.TestCase; +import org.apache.rocketmq.common.message.MessageQueue; +import org.junit.Assert; + +import java.util.ArrayList; +import java.util.List; + +public class AllocateMessageQueueAveragelyTest extends TestCase { + + public void testAllocateMessageQueueAveragely() { + List consumerIdList = createConsumerIdList(4); + List messageQueueList = createMessageQueueList(10); + int[] results = new int[consumerIdList.size()]; + for (int i = 0; i < consumerIdList.size(); i++) { + List result = new AllocateMessageQueueAveragely().allocate("", consumerIdList.get(i), messageQueueList, consumerIdList); + results[i] = result.size(); + } + Assert.assertArrayEquals(new int[]{3, 3, 2, 2}, results); + } + + private List createConsumerIdList(int size) { + List consumerIdList = new ArrayList(size); + for (int i = 0; i < size; i++) { + consumerIdList.add("CID_PREFIX" + i); + } + return consumerIdList; + } + + private List createMessageQueueList(int size) { + List messageQueueList = new ArrayList(size); + for (int i = 0; i < size; i++) { + MessageQueue mq = new MessageQueue("topic", "brokerName", i); + messageQueueList.add(mq); + } + return messageQueueList; + } + + +} From 810b75766b7cd6692f871208aaec43f6132abfe1 Mon Sep 17 00:00:00 2001 From: Kvicii <42023367+Kvicii@users.noreply.github.com> Date: Tue, 8 Feb 2022 17:54:07 +0800 Subject: [PATCH 125/141] [ISSUE #3797] Improve DefaultMQProducerImpl code (#3809) * improve DefaultMQProducerImpl code * revert change Co-authored-by: Kvicii --- .../client/impl/producer/DefaultMQProducerImpl.java | 10 +++++----- .../client/producer/LocalTransactionExecuter.java | 2 +- .../client/producer/TransactionCheckListener.java | 3 ++- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java index 9afbe9bcb70..f3f9caf43a9 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java @@ -283,9 +283,7 @@ public boolean isPublishTopicNeedUpdate(String topic) { } /** - * This method will be removed in the version 5.0.0 and getCheckListener is recommended. - * - * @return + * @deprecated This method will be removed in the version 5.0.0 and {@link DefaultMQProducerImpl#getCheckListener} is recommended. */ @Override @Deprecated @@ -485,6 +483,7 @@ public void send(Message msg, } /** + * @deprecated * It will be removed at 4.4.0 cause for exception handling and the wrong Semantics of timeout. A new one will be * provided in next version * @@ -1035,6 +1034,7 @@ public void send(Message msg, MessageQueue mq, SendCallback sendCallback) } /** + * @deprecated * It will be removed at 4.4.0 cause for exception handling and the wrong Semantics of timeout. A new one will be * provided in next version * @@ -1193,7 +1193,7 @@ public void run() { sendSelectImpl(msg, selector, arg, CommunicationMode.ASYNC, sendCallback, timeout - costTime); } catch (MQBrokerException e) { - throw new MQClientException("unknownn exception", e); + throw new MQClientException("unknown exception", e); } } catch (Exception e) { sendCallback.onException(e); @@ -1205,7 +1205,7 @@ public void run() { }); } catch (RejectedExecutionException e) { - throw new MQClientException("exector rejected ", e); + throw new MQClientException("executor rejected ", e); } } diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/LocalTransactionExecuter.java b/client/src/main/java/org/apache/rocketmq/client/producer/LocalTransactionExecuter.java index 28789b91d54..267ba10bd91 100644 --- a/client/src/main/java/org/apache/rocketmq/client/producer/LocalTransactionExecuter.java +++ b/client/src/main/java/org/apache/rocketmq/client/producer/LocalTransactionExecuter.java @@ -19,7 +19,7 @@ import org.apache.rocketmq.common.message.Message; /** - * This interface will be removed in the version 5.0.0, interface {@link TransactionListener} is recommended. + * @deprecated This interface will be removed in the version 5.0.0, interface {@link TransactionListener} is recommended. */ @Deprecated public interface LocalTransactionExecuter { diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/TransactionCheckListener.java b/client/src/main/java/org/apache/rocketmq/client/producer/TransactionCheckListener.java index 2d7cf5819f6..5f595065755 100644 --- a/client/src/main/java/org/apache/rocketmq/client/producer/TransactionCheckListener.java +++ b/client/src/main/java/org/apache/rocketmq/client/producer/TransactionCheckListener.java @@ -17,8 +17,9 @@ package org.apache.rocketmq.client.producer; import org.apache.rocketmq.common.message.MessageExt; + /** - * This interface will be removed in the version 5.0.0, interface {@link TransactionListener} is recommended. + * @deprecated This interface will be removed in the version 5.0.0, interface {@link TransactionListener} is recommended. */ @Deprecated public interface TransactionCheckListener { From ca92d367fda6030adde4ce87ee09b335047857ae Mon Sep 17 00:00:00 2001 From: caomao95 <528637588@qq.com> Date: Wed, 9 Feb 2022 19:08:43 +0800 Subject: [PATCH 126/141] Update design.md (#3835) * Update .travis.yml Depracated MODERATE noisy mail for dev@rocketmq.apache.org. Still Failing: apache/rocketmq#5670 (5.0.0-alpha-static-topic - 5ebc327) @dongeforever * Updated Notice file to 2022 (#3735) * Update design.md delete redundant text Co-authored-by: von gosling Co-authored-by: ZhangJian He --- NOTICE | 2 +- docs/cn/design.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/NOTICE b/NOTICE index 086ee9fa175..a347efb1756 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache RocketMQ -Copyright 2016-2021 The Apache Software Foundation +Copyright 2016-2022 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/docs/cn/design.md b/docs/cn/design.md index 169bc4aaeff..e2623c9c5b9 100644 --- a/docs/cn/design.md +++ b/docs/cn/design.md @@ -109,7 +109,7 @@ RocketMQ中的负载均衡都在Client端完成,具体来说的话,主要可 #### 4.1 Producer的负载均衡 Producer端在发送消息的时候,会先根据Topic找到指定的TopicPublishInfo,在获取了TopicPublishInfo路由信息后,RocketMQ的客户端在默认方式下selectOneMessageQueue()方法会从TopicPublishInfo中的messageQueueList中选择一个队列(MessageQueue)进行发送消息。具体的容错策略均在MQFaultStrategy这个类中定义。这里有一个sendLatencyFaultEnable开关变量,如果开启,在随机递增取模的基础上,再过滤掉not available的Broker代理。所谓的"latencyFaultTolerance",是指对之前失败的,按一定的时间做退避。例如,如果上次请求的latency超过550Lms,就退避3000Lms;超过1000L,就退避60000L;如果关闭,采用随机递增取模的方式选择一个队列(MessageQueue)来发送消息,latencyFaultTolerance机制是实现消息发送高可用的核心关键所在。 #### 4.2 Consumer的负载均衡 -在RocketMQ中,Consumer端的两种消费模式(Push/Pull)都是基于拉模式来获取消息的,而在Push模式只是对pull模式的一种封装,其本质实现为消息拉取线程在从服务器拉取到一批消息后,然后提交到消息消费线程池后,又“马不停蹄”的继续向服务器再次尝试拉取消息。如果未拉取到消息,则延迟一下又继续拉取。在两种基于拉模式的消费方式(Push/Pull)中,均需要Consumer端在知道从Broker端的哪一个消息队列—队列中去获取消息。因此,有必要在Consumer端来做负载均衡,即Broker端中多个MessageQueue分配给同一个ConsumerGroup中的哪些Consumer消费。 +在RocketMQ中,Consumer端的两种消费模式(Push/Pull)都是基于拉模式来获取消息的,而在Push模式只是对pull模式的一种封装,其本质实现为消息拉取线程在从服务器拉取到一批消息后,然后提交到消息消费线程池后,又“马不停蹄”的继续向服务器再次尝试拉取消息。如果未拉取到消息,则延迟一下又继续拉取。在两种基于拉模式的消费方式(Push/Pull)中,均需要Consumer端知道从Broker端的哪一个消息队列中去获取消息。因此,有必要在Consumer端来做负载均衡,即Broker端中多个MessageQueue分配给同一个ConsumerGroup中的哪些Consumer消费。 1、Consumer端的心跳包发送 From 8fa9ca592ce526ffe191ec4e9c6889997e5bfc93 Mon Sep 17 00:00:00 2001 From: huangli Date: Thu, 10 Feb 2022 11:15:04 +0800 Subject: [PATCH 127/141] [ISSUE 3825] Use default SO_SNDBUF/SO_RCVBUF/WRITE_BUFFER_WATER_MARK value. (#3826) This commit may greatly increase thoughtput when network lantency is big or message size is big, especially for consumer. In one broker (M/S sync master async flush), one producer send test. 250 byte body: 0.93x old version 4000 byte body: 3.85x of old version 16000 byte body: 6.07x of old version 100000 byte body: 7.93x of old version The old version is 4.9.3-SNAPSHOT in this test. In this test the network latency from producer to broker is 1.61ms. --- .../apache/rocketmq/broker/BrokerStartup.java | 9 --------- .../rocketmq/remoting/common/RemotingUtil.java | 9 +++++++-- .../remoting/netty/NettyRemotingClient.java | 18 ++++++++++++++---- .../remoting/netty/NettyRemotingServer.java | 18 ++++++++++++++---- .../remoting/netty/NettySystemConfig.java | 8 ++++---- .../apache/rocketmq/store/ha/HAConnection.java | 9 +++++++-- 6 files changed, 46 insertions(+), 25 deletions(-) diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java index bb2eb9135a7..19e618ba2da 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerStartup.java @@ -32,7 +32,6 @@ import org.apache.rocketmq.remoting.common.TlsMode; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyServerConfig; -import org.apache.rocketmq.remoting.netty.NettySystemConfig; import org.apache.rocketmq.remoting.netty.TlsSystemConfig; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.srvutil.ServerUtil; @@ -90,14 +89,6 @@ public static void shutdown(final BrokerController controller) { public static BrokerController createBrokerController(String[] args) { System.setProperty(RemotingCommand.REMOTING_VERSION_KEY, Integer.toString(MQVersion.CURRENT_VERSION)); - if (null == System.getProperty(NettySystemConfig.COM_ROCKETMQ_REMOTING_SOCKET_SNDBUF_SIZE)) { - NettySystemConfig.socketSndbufSize = 131072; - } - - if (null == System.getProperty(NettySystemConfig.COM_ROCKETMQ_REMOTING_SOCKET_RCVBUF_SIZE)) { - NettySystemConfig.socketRcvbufSize = 131072; - } - try { //PackageConflictDetect.detectFastjson(); Options options = ServerUtil.buildCommandlineOptions(new Options()); diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java index 3914314f0e2..d5ce20b0f12 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/common/RemotingUtil.java @@ -34,6 +34,7 @@ import java.util.Enumeration; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.remoting.netty.NettySystemConfig; public class RemotingUtil { public static final String OS_NAME = System.getProperty("os.name"); @@ -193,8 +194,12 @@ public static SocketChannel connect(SocketAddress remote, final int timeoutMilli sc.configureBlocking(true); sc.socket().setSoLinger(false, -1); sc.socket().setTcpNoDelay(true); - sc.socket().setReceiveBufferSize(1024 * 64); - sc.socket().setSendBufferSize(1024 * 64); + if (NettySystemConfig.socketSndbufSize > 0) { + sc.socket().setReceiveBufferSize(NettySystemConfig.socketSndbufSize); + } + if (NettySystemConfig.socketRcvbufSize > 0) { + sc.socket().setSendBufferSize(NettySystemConfig.socketRcvbufSize); + } sc.socket().connect(remote, timeoutMillis); sc.configureBlocking(false); return sc; diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java index cbd17fadb23..5ced3b7eb4d 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java @@ -166,10 +166,6 @@ public Thread newThread(Runnable r) { .option(ChannelOption.TCP_NODELAY, true) .option(ChannelOption.SO_KEEPALIVE, false) .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, nettyClientConfig.getConnectTimeoutMillis()) - .option(ChannelOption.SO_SNDBUF, nettyClientConfig.getClientSocketSndBufSize()) - .option(ChannelOption.SO_RCVBUF, nettyClientConfig.getClientSocketRcvBufSize()) - .option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(nettyClientConfig.getWriteBufferLowWaterMark(), - nettyClientConfig.getWriteBufferHighWaterMark())) .handler(new ChannelInitializer() { @Override public void initChannel(SocketChannel ch) throws Exception { @@ -191,6 +187,20 @@ public void initChannel(SocketChannel ch) throws Exception { new NettyClientHandler()); } }); + if (nettyClientConfig.getClientSocketSndBufSize() > 0) { + log.info("client set SO_SNDBUF to {}", nettyClientConfig.getClientSocketSndBufSize()); + handler.option(ChannelOption.SO_SNDBUF, nettyClientConfig.getClientSocketSndBufSize()); + } + if (nettyClientConfig.getClientSocketRcvBufSize() > 0) { + log.info("client set SO_RCVBUF to {}", nettyClientConfig.getClientSocketRcvBufSize()); + handler.option(ChannelOption.SO_RCVBUF, nettyClientConfig.getClientSocketRcvBufSize()); + } + if (nettyClientConfig.getWriteBufferLowWaterMark() > 0 && nettyClientConfig.getWriteBufferHighWaterMark() > 0) { + log.info("client set netty WRITE_BUFFER_WATER_MARK to {},{}", + nettyClientConfig.getWriteBufferLowWaterMark(), nettyClientConfig.getWriteBufferHighWaterMark()); + handler.option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark( + nettyClientConfig.getWriteBufferLowWaterMark(), nettyClientConfig.getWriteBufferHighWaterMark())); + } this.timer.scheduleAtFixedRate(new TimerTask() { @Override diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java index 2b7413f7a90..22440af943f 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java @@ -203,10 +203,6 @@ public Thread newThread(Runnable r) { .option(ChannelOption.SO_REUSEADDR, true) .option(ChannelOption.SO_KEEPALIVE, false) .childOption(ChannelOption.TCP_NODELAY, true) - .childOption(ChannelOption.SO_SNDBUF, nettyServerConfig.getServerSocketSndBufSize()) - .childOption(ChannelOption.SO_RCVBUF, nettyServerConfig.getServerSocketRcvBufSize()) - .childOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark( - nettyServerConfig.getWriteBufferLowWaterMark(), nettyServerConfig.getWriteBufferHighWaterMark())) .localAddress(new InetSocketAddress(this.nettyServerConfig.getListenPort())) .childHandler(new ChannelInitializer() { @Override @@ -222,6 +218,20 @@ public void initChannel(SocketChannel ch) throws Exception { ); } }); + if (nettyServerConfig.getServerSocketSndBufSize() > 0) { + log.info("server set SO_SNDBUF to {}", nettyServerConfig.getServerSocketSndBufSize()); + childHandler.childOption(ChannelOption.SO_SNDBUF, nettyServerConfig.getServerSocketSndBufSize()); + } + if (nettyServerConfig.getServerSocketRcvBufSize() > 0) { + log.info("server set SO_RCVBUF to {}", nettyServerConfig.getServerSocketRcvBufSize()); + childHandler.childOption(ChannelOption.SO_RCVBUF, nettyServerConfig.getServerSocketRcvBufSize()); + } + if (nettyServerConfig.getWriteBufferLowWaterMark() > 0 && nettyServerConfig.getWriteBufferHighWaterMark() > 0) { + log.info("server set netty WRITE_BUFFER_WATER_MARK to {},{}", + nettyServerConfig.getWriteBufferLowWaterMark(), nettyServerConfig.getWriteBufferHighWaterMark()); + childHandler.childOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark( + nettyServerConfig.getWriteBufferLowWaterMark(), nettyServerConfig.getWriteBufferHighWaterMark())); + } if (nettyServerConfig.isServerPooledByteBufAllocatorEnable()) { childHandler.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT); diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java index 4290baea7a9..edad8444ed8 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java @@ -50,9 +50,9 @@ public class NettySystemConfig { public static final int CLIENT_ONEWAY_SEMAPHORE_VALUE = Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_CLIENT_ONEWAY_SEMAPHORE_VALUE, "65535")); public static int socketSndbufSize = - Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_SNDBUF_SIZE, "65535")); + Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_SNDBUF_SIZE, "0")); public static int socketRcvbufSize = - Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_RCVBUF_SIZE, "65535")); + Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_RCVBUF_SIZE, "0")); public static int socketBacklog = Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_SOCKET_BACKLOG, "1024")); public static int clientWorkerSize = @@ -64,8 +64,8 @@ public class NettySystemConfig { public static boolean clientCloseSocketIfTimeout = Boolean.parseBoolean(System.getProperty(COM_ROCKETMQ_REMOTING_CLIENT_CLOSE_SOCKET_IF_TIMEOUT, "true")); public static int writeBufferHighWaterMark = - Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_WRITE_BUFFER_HIGH_WATER_MARK_VALUE, "4194304"));//4M + Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_WRITE_BUFFER_HIGH_WATER_MARK_VALUE, "0")); public static int writeBufferLowWaterMark = - Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_WRITE_BUFFER_LOW_WATER_MARK, "1048576")); //1MB + Integer.parseInt(System.getProperty(COM_ROCKETMQ_REMOTING_WRITE_BUFFER_LOW_WATER_MARK, "0")); } diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java index c08c515c0c9..4c26971c078 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java +++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java @@ -26,6 +26,7 @@ import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.remoting.common.RemotingUtil; +import org.apache.rocketmq.remoting.netty.NettySystemConfig; import org.apache.rocketmq.store.SelectMappedBufferResult; public class HAConnection { @@ -46,8 +47,12 @@ public HAConnection(final HAService haService, final SocketChannel socketChannel this.socketChannel.configureBlocking(false); this.socketChannel.socket().setSoLinger(false, -1); this.socketChannel.socket().setTcpNoDelay(true); - this.socketChannel.socket().setReceiveBufferSize(1024 * 64); - this.socketChannel.socket().setSendBufferSize(1024 * 64); + if (NettySystemConfig.socketSndbufSize > 0) { + this.socketChannel.socket().setReceiveBufferSize(NettySystemConfig.socketSndbufSize); + } + if (NettySystemConfig.socketRcvbufSize > 0) { + this.socketChannel.socket().setSendBufferSize(NettySystemConfig.socketRcvbufSize); + } this.writeSocketService = new WriteSocketService(this.socketChannel); this.readSocketService = new ReadSocketService(this.socketChannel); this.haService.getConnectionCount().incrementAndGet(); From 4992f4ee9c09222ed1aaf76f3f4b54f4fedc6ffd Mon Sep 17 00:00:00 2001 From: Aaron-TangCode <36765247+Aaron-TangCode@users.noreply.github.com> Date: Thu, 10 Feb 2022 11:43:43 +0800 Subject: [PATCH 128/141] [ISSUE ##3832] in order to improve the performance by change the variable location of queueId (#3833) * change variable queueId location * [fix]the wildcard Co-authored-by: tanghailin --- .../org/apache/rocketmq/store/CommitLog.java | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index c10ffc32708..1af5c587622 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -16,22 +16,6 @@ */ package org.apache.rocketmq.store; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Supplier; - import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; @@ -50,6 +34,22 @@ import org.apache.rocketmq.store.ha.HAService; import org.apache.rocketmq.store.schedule.ScheduleMessageService; +import java.net.Inet4Address; +import java.net.Inet6Address; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + /** * Store all metadata downtime for recovery, data protection reliability */ @@ -608,8 +608,7 @@ public CompletableFuture asyncPutMessage(final MessageExtBroke StoreStatsService storeStatsService = this.defaultMessageStore.getStoreStatsService(); String topic = msg.getTopic(); - int queueId = msg.getQueueId(); - +// int queueId msg.getQueueId(); final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag()); if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) { @@ -620,7 +619,7 @@ public CompletableFuture asyncPutMessage(final MessageExtBroke } topic = TopicValidator.RMQ_SYS_SCHEDULE_TOPIC; - queueId = ScheduleMessageService.delayLevel2QueueId(msg.getDelayTimeLevel()); + int queueId = ScheduleMessageService.delayLevel2QueueId(msg.getDelayTimeLevel()); // Backup real topic, queueId MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_TOPIC, msg.getTopic()); From 8b78f6114f92cc3e482eab602e4b5cdff9d79f0c Mon Sep 17 00:00:00 2001 From: zhangjidi2016 <1017543663@qq.com> Date: Fri, 11 Feb 2022 11:30:07 +0800 Subject: [PATCH 129/141] [ISSUE #3674]Add unit test for AllocateMessageQueueAveragelyByCircle (#3839) Co-authored-by: zhangjidi --- ...cateMessageQueueAveragelyByCircleTest.java | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyByCircleTest.java diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyByCircleTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyByCircleTest.java new file mode 100644 index 00000000000..497766d9a6b --- /dev/null +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/rebalance/AllocateMessageQueueAveragelyByCircleTest.java @@ -0,0 +1,68 @@ +/* + * 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.client.consumer.rebalance; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import junit.framework.TestCase; +import org.apache.rocketmq.common.message.MessageQueue; +import org.junit.Assert; + +public class AllocateMessageQueueAveragelyByCircleTest extends TestCase { + + public void testAllocateMessageQueueAveragelyByCircle() { + List consumerIdList = createConsumerIdList(4); + List messageQueueList = createMessageQueueList(10); + // the consumerId not in cidAll + List allocateQueues = new AllocateMessageQueueAveragelyByCircle().allocate("", "CID_PREFIX", messageQueueList, consumerIdList); + Assert.assertEquals(0, allocateQueues.size()); + + Map consumerAllocateQueue = new HashMap(consumerIdList.size()); + for (String consumerId : consumerIdList) { + List queues = new AllocateMessageQueueAveragelyByCircle().allocate("", consumerId, messageQueueList, consumerIdList); + int[] queueIds = new int[queues.size()]; + for (int i = 0; i < queues.size(); i++) { + queueIds[i] = queues.get(i).getQueueId(); + } + consumerAllocateQueue.put(consumerId, queueIds); + } + Assert.assertArrayEquals(new int[] {0, 4, 8}, consumerAllocateQueue.get("CID_PREFIX0")); + Assert.assertArrayEquals(new int[] {1, 5, 9}, consumerAllocateQueue.get("CID_PREFIX1")); + Assert.assertArrayEquals(new int[] {2, 6}, consumerAllocateQueue.get("CID_PREFIX2")); + Assert.assertArrayEquals(new int[] {3, 7}, consumerAllocateQueue.get("CID_PREFIX3")); + } + + private List createConsumerIdList(int size) { + List consumerIdList = new ArrayList(size); + for (int i = 0; i < size; i++) { + consumerIdList.add("CID_PREFIX" + i); + } + return consumerIdList; + } + + private List createMessageQueueList(int size) { + List messageQueueList = new ArrayList(size); + for (int i = 0; i < size; i++) { + MessageQueue mq = new MessageQueue("topic", "brokerName", i); + messageQueueList.add(mq); + } + return messageQueueList; + } +} + From 48e8c1955b660ccffb65e9b0e7dfc602551d0b38 Mon Sep 17 00:00:00 2001 From: yuz10 <845238369@qq.com> Date: Sun, 13 Feb 2022 10:02:40 +0800 Subject: [PATCH 130/141] add tests for computePullFromWhereWithException (#3844) --- .../consumer/RebalanceLitePullImplTest.java | 100 ++++++++++++++++++ .../impl/consumer/RebalancePushImplTest.java | 74 ++++++++++++- 2 files changed, 171 insertions(+), 3 deletions(-) create mode 100644 client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalanceLitePullImplTest.java diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalanceLitePullImplTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalanceLitePullImplTest.java new file mode 100644 index 00000000000..ad244ebfceb --- /dev/null +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalanceLitePullImplTest.java @@ -0,0 +1,100 @@ +/* + * 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.client.impl.consumer; + +import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; +import org.apache.rocketmq.client.consumer.store.OffsetStore; +import org.apache.rocketmq.client.consumer.store.ReadOffsetType; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.impl.MQAdminImpl; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.message.MessageQueue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class RebalanceLitePullImplTest { + private MessageQueue mq = new MessageQueue("topic1", "broker1", 0); + private MessageQueue retryMq = new MessageQueue(MixAll.RETRY_GROUP_TOPIC_PREFIX + "group", "broker1", 0); + private DefaultLitePullConsumerImpl consumerImpl = mock(DefaultLitePullConsumerImpl.class); + private RebalanceLitePullImpl rebalanceImpl = new RebalanceLitePullImpl(consumerImpl); + private OffsetStore offsetStore = mock(OffsetStore.class); + private DefaultLitePullConsumer consumer = new DefaultLitePullConsumer(); + private MQClientInstance client = mock(MQClientInstance.class); + private MQAdminImpl admin = mock(MQAdminImpl.class); + + public RebalanceLitePullImplTest() { + when(consumerImpl.getDefaultLitePullConsumer()).thenReturn(consumer); + when(consumerImpl.getOffsetStore()).thenReturn(offsetStore); + rebalanceImpl.setmQClientFactory(client); + when(client.getMQAdminImpl()).thenReturn(admin); + } + + @Test + public void testComputePullFromWhereWithException_ne_minus1() throws MQClientException { + for (ConsumeFromWhere where : new ConsumeFromWhere[]{ + ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET, + ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET, + ConsumeFromWhere.CONSUME_FROM_TIMESTAMP}) { + consumer.setConsumeFromWhere(where); + + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(0L); + assertEquals(0, rebalanceImpl.computePullFromWhereWithException(mq)); + + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-2L); + assertEquals(-1, rebalanceImpl.computePullFromWhereWithException(mq)); + } + } + + @Test + public void testComputePullFromWhereWithException_eq_minus1_last() throws MQClientException { + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-1L); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); + when(admin.maxOffset(any(MessageQueue.class))).thenReturn(12345L); + + assertEquals(12345L, rebalanceImpl.computePullFromWhereWithException(mq)); + + assertEquals(0L, rebalanceImpl.computePullFromWhereWithException(retryMq)); + } + + @Test + public void testComputePullFromWhereWithException_eq_minus1_first() throws MQClientException { + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-1L); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + assertEquals(0, rebalanceImpl.computePullFromWhereWithException(mq)); + } + + @Test + public void testComputePullFromWhereWithException_eq_minus1_timestamp() throws MQClientException { + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-1L); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP); + when(admin.searchOffset(any(MessageQueue.class), anyLong())).thenReturn(12345L); + when(admin.maxOffset(any(MessageQueue.class))).thenReturn(23456L); + + assertEquals(12345L, rebalanceImpl.computePullFromWhereWithException(mq)); + + assertEquals(23456L, rebalanceImpl.computePullFromWhereWithException(retryMq)); + } + + +} diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImplTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImplTest.java index 796a3943087..a60d88ea019 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImplTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/RebalancePushImplTest.java @@ -19,11 +19,16 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; + import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely; import org.apache.rocketmq.client.consumer.store.OffsetStore; +import org.apache.rocketmq.client.consumer.store.ReadOffsetType; import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.impl.MQAdminImpl; import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; @@ -36,9 +41,12 @@ import org.mockito.stubbing.Answer; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) @@ -47,10 +55,23 @@ public class RebalancePushImplTest { private DefaultMQPushConsumerImpl defaultMQPushConsumer = new DefaultMQPushConsumerImpl(new DefaultMQPushConsumer("RebalancePushImplTest"), null); @Mock private MQClientInstance mqClientInstance; - @Mock - private OffsetStore offsetStore; + private OffsetStore offsetStore = mock(OffsetStore.class); private String consumerGroup = "CID_RebalancePushImplTest"; private String topic = "TopicA"; + private MessageQueue mq = new MessageQueue("topic1", "broker1", 0); + private MessageQueue retryMq = new MessageQueue(MixAll.RETRY_GROUP_TOPIC_PREFIX + "group", "broker1", 0); + private DefaultMQPushConsumerImpl consumerImpl = mock(DefaultMQPushConsumerImpl.class); + private RebalancePushImpl rebalanceImpl = new RebalancePushImpl(consumerImpl); + private DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(); + private MQClientInstance client = mock(MQClientInstance.class); + private MQAdminImpl admin = mock(MQAdminImpl.class); + + public RebalancePushImplTest() { + when(consumerImpl.getDefaultMQPushConsumer()).thenReturn(consumer); + when(consumerImpl.getOffsetStore()).thenReturn(offsetStore); + rebalanceImpl.setmQClientFactory(client); + when(client.getMQAdminImpl()).thenReturn(admin); + } @Test public void testMessageQueueChanged_CountThreshold() { @@ -160,4 +181,51 @@ public void testMessageQueueChanged_ConsumerRuntimeInfo() throws MQClientExcepti assertThat(defaultMQPushConsumer.consumerRunningInfo().getProperties().get("pullThresholdSizeForTopic")).isEqualTo("1024"); assertThat(defaultMQPushConsumer.consumerRunningInfo().getProperties().get("pullThresholdForTopic")).isEqualTo("1024"); } -} \ No newline at end of file + + @Test + public void testComputePullFromWhereWithException_ne_minus1() throws MQClientException { + for (ConsumeFromWhere where : new ConsumeFromWhere[]{ + ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET, + ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET, + ConsumeFromWhere.CONSUME_FROM_TIMESTAMP}) { + consumer.setConsumeFromWhere(where); + + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(0L); + assertEquals(0, rebalanceImpl.computePullFromWhereWithException(mq)); + + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-2L); + assertEquals(-1, rebalanceImpl.computePullFromWhereWithException(mq)); + } + } + + @Test + public void testComputePullFromWhereWithException_eq_minus1_last() throws MQClientException { + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-1L); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); + when(admin.maxOffset(any(MessageQueue.class))).thenReturn(12345L); + + assertEquals(12345L, rebalanceImpl.computePullFromWhereWithException(mq)); + + assertEquals(0L, rebalanceImpl.computePullFromWhereWithException(retryMq)); + } + + @Test + public void testComputePullFromWhereWithException_eq_minus1_first() throws MQClientException { + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-1L); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + assertEquals(0, rebalanceImpl.computePullFromWhereWithException(mq)); + } + + @Test + public void testComputePullFromWhereWithException_eq_minus1_timestamp() throws MQClientException { + when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(-1L); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP); + when(admin.searchOffset(any(MessageQueue.class), anyLong())).thenReturn(12345L); + when(admin.maxOffset(any(MessageQueue.class))).thenReturn(23456L); + + assertEquals(12345L, rebalanceImpl.computePullFromWhereWithException(mq)); + + assertEquals(23456L, rebalanceImpl.computePullFromWhereWithException(retryMq)); + } + +} From 459b9d0be86e9584b2633f84be047beea8410a41 Mon Sep 17 00:00:00 2001 From: Martin Grigorov Date: Wed, 16 Feb 2022 02:52:13 +0200 Subject: [PATCH 131/141] Re-enable testing on ARM64 (#3850) * Re-enable testing on ARM64 But this time on: 1) AWS Graviton2 machines - they are more reliable 2) Ubuntu 20.04 - it is better maintained by Travis team Signed-off-by: Martin Tzvetanov Grigorov * Run `mvn verify` in batch mode and do not log network transfer progress Signed-off-by: Martin Tzvetanov Grigorov * Do not use -ntp (--no-transfer-progress) Because it is not available in Maven 3.6.0 (latest available in Ubuntu 18.04) Signed-off-by: Martin Tzvetanov Grigorov --- .travis.yml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index c55d9291e92..5f08c8500b8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -18,8 +18,11 @@ matrix: # On Linux we install latest OpenJDK 1.8 from Ubuntu repositories - name: Linux x86_64 arch: amd64 -# - name: Linux aarch64 -# arch: arm64 + - name: Linux aarch64 + dist: focal + arch: arm64-graviton2 + group: edge + virt: vm cache: directories: @@ -45,7 +48,7 @@ before_script: - ulimit -c unlimited script: - - mvn verify -DskipTests + - mvn -B verify -DskipTests - travis_retry mvn -B clean apache-rat:check - travis_retry mvn -B install jacoco:report coveralls:report - travis_retry mvn -B clean install -pl test -Pit-test From a45c7bc60eee5885841d3eb0cb93d16c88f32506 Mon Sep 17 00:00:00 2001 From: Heng Du Date: Wed, 16 Feb 2022 10:45:38 +0800 Subject: [PATCH 132/141] [ISSUE #3852]Move the Logappender module to external (#3851) * feat(logappender) move logappender to external(https://lists.apache.org/thread/938sjby3hnnzcttpp20p17ywhc0rgr8s) * remove logappender module --- logappender/pom.xml | 80 ------- .../logappender/common/ProducerInstance.java | 97 -------- .../log4j/RocketmqLog4jAppender.java | 189 --------------- .../log4j2/RocketmqLog4j2Appender.java | 226 ------------------ .../logback/RocketmqLogbackAppender.java | 179 -------------- .../logappender/AbstractTestCase.java | 72 ------ .../logappender/Log4jPropertiesTest.java | 32 --- .../rocketmq/logappender/Log4jTest.java | 42 ---- .../rocketmq/logappender/Log4jXmlTest.java | 32 --- .../rocketmq/logappender/LogbackTest.java | 52 ---- .../rocketmq/logappender/log4j2Test.java | 44 ---- .../test/resources/log4j-example.properties | 33 --- .../src/test/resources/log4j-example.xml | 56 ----- .../src/test/resources/log4j2-example.xml | 41 ---- .../src/test/resources/logback-example.xml | 81 ------- pom.xml | 1 - 16 files changed, 1257 deletions(-) delete mode 100644 logappender/pom.xml delete mode 100644 logappender/src/main/java/org/apache/rocketmq/logappender/common/ProducerInstance.java delete mode 100644 logappender/src/main/java/org/apache/rocketmq/logappender/log4j/RocketmqLog4jAppender.java delete mode 100644 logappender/src/main/java/org/apache/rocketmq/logappender/log4j2/RocketmqLog4j2Appender.java delete mode 100644 logappender/src/main/java/org/apache/rocketmq/logappender/logback/RocketmqLogbackAppender.java delete mode 100644 logappender/src/test/java/org/apache/rocketmq/logappender/AbstractTestCase.java delete mode 100644 logappender/src/test/java/org/apache/rocketmq/logappender/Log4jPropertiesTest.java delete mode 100644 logappender/src/test/java/org/apache/rocketmq/logappender/Log4jTest.java delete mode 100644 logappender/src/test/java/org/apache/rocketmq/logappender/Log4jXmlTest.java delete mode 100644 logappender/src/test/java/org/apache/rocketmq/logappender/LogbackTest.java delete mode 100644 logappender/src/test/java/org/apache/rocketmq/logappender/log4j2Test.java delete mode 100644 logappender/src/test/resources/log4j-example.properties delete mode 100644 logappender/src/test/resources/log4j-example.xml delete mode 100644 logappender/src/test/resources/log4j2-example.xml delete mode 100644 logappender/src/test/resources/logback-example.xml diff --git a/logappender/pom.xml b/logappender/pom.xml deleted file mode 100644 index 306edc7a1ae..00000000000 --- a/logappender/pom.xml +++ /dev/null @@ -1,80 +0,0 @@ - - - - - org.apache.rocketmq - rocketmq-all - 4.9.3-SNAPSHOT - - 4.0.0 - rocketmq-logappender - jar - rocketmq-logappender ${project.version} - - - - org.slf4j - slf4j-api - true - - - log4j - log4j - true - - - org.apache.logging.log4j - log4j-core - true - - - ch.qos.logback - logback-classic - true - - - ${project.groupId} - rocketmq-client - - - ${project.groupId} - rocketmq-namesrv - test - - - ch.qos.logback - logback-classic - - - - - ${project.groupId} - rocketmq-broker - test - - - ch.qos.logback - logback-classic - - - - - - - - diff --git a/logappender/src/main/java/org/apache/rocketmq/logappender/common/ProducerInstance.java b/logappender/src/main/java/org/apache/rocketmq/logappender/common/ProducerInstance.java deleted file mode 100644 index d2adac53dbc..00000000000 --- a/logappender/src/main/java/org/apache/rocketmq/logappender/common/ProducerInstance.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender.common; - -import org.apache.commons.lang3.StringUtils; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.producer.DefaultMQProducer; -import org.apache.rocketmq.client.producer.MQProducer; - -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -/** - * Common Producer component - */ -public class ProducerInstance { - - public static final String APPENDER_TYPE = "APPENDER_TYPE"; - - public static final String LOG4J_APPENDER = "LOG4J_APPENDER"; - - public static final String LOG4J2_APPENDER = "LOG4J2_APPENDER"; - - public static final String LOGBACK_APPENDER = "LOGBACK_APPENDER"; - - public static final String DEFAULT_GROUP = "rocketmq_appender"; - - private ConcurrentHashMap producerMap = new ConcurrentHashMap(); - - private static ProducerInstance instance = new ProducerInstance(); - - public static ProducerInstance getProducerInstance() { - return instance; - } - - private String genKey(String nameServerAddress, String group) { - return nameServerAddress + "_" + group; - } - - public MQProducer getInstance(String nameServerAddress, String group) throws MQClientException { - if (StringUtils.isBlank(group)) { - group = DEFAULT_GROUP; - } - - String genKey = genKey(nameServerAddress, group); - MQProducer p = getProducerInstance().producerMap.get(genKey); - if (p != null) { - return p; - } - - DefaultMQProducer defaultMQProducer = new DefaultMQProducer(group); - defaultMQProducer.setNamesrvAddr(nameServerAddress); - MQProducer beforeProducer = null; - beforeProducer = getProducerInstance().producerMap.putIfAbsent(genKey, defaultMQProducer); - if (beforeProducer != null) { - return beforeProducer; - } - defaultMQProducer.start(); - return defaultMQProducer; - } - - public void removeAndClose(String nameServerAddress, String group) { - if (group == null) { - group = DEFAULT_GROUP; - } - String genKey = genKey(nameServerAddress, group); - MQProducer producer = getProducerInstance().producerMap.remove(genKey); - - if (producer != null) { - producer.shutdown(); - } - } - - public void closeAll() { - Set> entries = getProducerInstance().producerMap.entrySet(); - for (Map.Entry entry : entries) { - getProducerInstance().producerMap.remove(entry.getKey()); - entry.getValue().shutdown(); - } - } - -} diff --git a/logappender/src/main/java/org/apache/rocketmq/logappender/log4j/RocketmqLog4jAppender.java b/logappender/src/main/java/org/apache/rocketmq/logappender/log4j/RocketmqLog4jAppender.java deleted file mode 100644 index 646e924828a..00000000000 --- a/logappender/src/main/java/org/apache/rocketmq/logappender/log4j/RocketmqLog4jAppender.java +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender.log4j; - -import org.apache.rocketmq.common.message.Message; -import org.apache.rocketmq.logappender.common.ProducerInstance; -import org.apache.log4j.AppenderSkeleton; -import org.apache.log4j.helpers.LogLog; -import org.apache.log4j.spi.ErrorCode; -import org.apache.log4j.spi.LoggingEvent; -import org.apache.rocketmq.client.producer.MQProducer; - -/** - * Log4j Appender Component - */ -public class RocketmqLog4jAppender extends AppenderSkeleton { - - /** - * Appended message tag define - */ - private String tag; - - /** - * Whitch topic to send log messages - */ - private String topic; - - private boolean locationInfo; - - /** - * Log producer send instance - */ - private MQProducer producer; - - /** - * RocketMQ nameserver address - */ - private String nameServerAddress; - - /** - * Log producer group - */ - private String producerGroup; - - public RocketmqLog4jAppender() { - } - - public void activateOptions() { - LogLog.debug("Getting initial context."); - if (!checkEntryConditions()) { - return; - } - try { - producer = ProducerInstance.getProducerInstance().getInstance(nameServerAddress, producerGroup); - } catch (Exception e) { - LogLog.error("activateOptions nameserver:" + nameServerAddress + " group:" + producerGroup + " " + e.getMessage()); - } - } - - /** - * Info,error,warn,callback method implementation - */ - public void append(LoggingEvent event) { - if (null == producer) { - return; - } - if (locationInfo) { - event.getLocationInformation(); - } - byte[] data = this.layout.format(event).getBytes(); - try { - Message msg = new Message(topic, tag, data); - msg.getProperties().put(ProducerInstance.APPENDER_TYPE, ProducerInstance.LOG4J_APPENDER); - - //Send message and do not wait for the ack from the message broker. - producer.sendOneway(msg); - } catch (Exception e) { - String msg = new String(data); - errorHandler.error("Could not send message in RocketmqLog4jAppender [" + name + "].Message is :" + msg, e, - ErrorCode.GENERIC_FAILURE); - } - } - - protected boolean checkEntryConditions() { - String fail = null; - - if (this.topic == null) { - fail = "No topic"; - } else if (this.tag == null) { - fail = "No tag"; - } - - if (fail != null) { - errorHandler.error(fail + " for RocketmqLog4jAppender named [" + name + "]."); - return false; - } else { - return true; - } - } - - /** - * When system exit,this method will be called to close resources - */ - public synchronized void close() { - // The synchronized modifier avoids concurrent append and close operations - - if (this.closed) - return; - - LogLog.debug("Closing RocketmqLog4jAppender [" + name + "]."); - this.closed = true; - - try { - ProducerInstance.getProducerInstance().removeAndClose(this.nameServerAddress, this.producerGroup); - } catch (Exception e) { - LogLog.error("Closing RocketmqLog4jAppender [" + name + "] nameServerAddress:" + nameServerAddress + " group:" + producerGroup + " " + e.getMessage()); - } - // Help garbage collection - producer = null; - } - - public boolean requiresLayout() { - return true; - } - - public String getTopic() { - return topic; - } - - public void setTopic(String topic) { - this.topic = topic; - } - - public String getTag() { - return tag; - } - - public void setTag(String tag) { - this.tag = tag; - } - - /** - * Returns value of the LocationInfo property which - * determines whether location (stack) info is sent to the remote - * subscriber. - */ - public boolean isLocationInfo() { - return locationInfo; - } - - /** - * If true, the information sent to the remote subscriber will - * include caller's location information. By default no location - * information is sent to the subscriber. - */ - public void setLocationInfo(boolean locationInfo) { - this.locationInfo = locationInfo; - } - - /** - * Returns the message producer,Only valid after - * activateOptions() method has been invoked. - */ - protected MQProducer getProducer() { - return producer; - } - - public void setNameServerAddress(String nameServerAddress) { - this.nameServerAddress = nameServerAddress; - } - - public void setProducerGroup(String producerGroup) { - this.producerGroup = producerGroup; - } -} diff --git a/logappender/src/main/java/org/apache/rocketmq/logappender/log4j2/RocketmqLog4j2Appender.java b/logappender/src/main/java/org/apache/rocketmq/logappender/log4j2/RocketmqLog4j2Appender.java deleted file mode 100644 index 9543f1c2917..00000000000 --- a/logappender/src/main/java/org/apache/rocketmq/logappender/log4j2/RocketmqLog4j2Appender.java +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender.log4j2; - -import org.apache.logging.log4j.core.Appender; -import org.apache.logging.log4j.core.Filter; -import org.apache.logging.log4j.core.ErrorHandler; -import org.apache.logging.log4j.core.Layout; -import org.apache.logging.log4j.core.LogEvent; -import org.apache.logging.log4j.core.config.Node; -import org.apache.logging.log4j.core.config.plugins.Plugin; -import org.apache.logging.log4j.core.config.plugins.PluginBuilderAttribute; -import org.apache.logging.log4j.core.config.plugins.PluginBuilderFactory; -import org.apache.logging.log4j.core.config.plugins.PluginElement; -import org.apache.rocketmq.common.message.Message; -import org.apache.rocketmq.logappender.common.ProducerInstance; -import org.apache.logging.log4j.core.appender.AbstractAppender; -import org.apache.logging.log4j.core.config.plugins.validation.constraints.Required; -import org.apache.logging.log4j.core.layout.SerializedLayout; -import org.apache.rocketmq.client.producer.MQProducer; - -import java.io.Serializable; -import java.util.concurrent.TimeUnit; - -/** - * Log4j2 Appender Component - */ -@Plugin(name = "RocketMQ", - category = Node.CATEGORY, - elementType = Appender.ELEMENT_TYPE, - printObject = true) -public class RocketmqLog4j2Appender extends AbstractAppender { - - /** - * RocketMQ nameserver address - */ - private String nameServerAddress; - - /** - * Log producer group - */ - private String producerGroup; - - /** - * Log producer send instance - */ - private MQProducer producer; - - /** - * Appended message tag define - */ - private String tag; - - /** - * Whitch topic to send log messages - */ - private String topic; - - protected RocketmqLog4j2Appender(String name, Filter filter, Layout layout, - boolean ignoreExceptions, String nameServerAddress, String producerGroup, - String topic, String tag) { - super(name, filter, layout, ignoreExceptions); - this.producer = producer; - this.topic = topic; - this.tag = tag; - this.nameServerAddress = nameServerAddress; - this.producerGroup = producerGroup; - try { - this.producer = ProducerInstance.getProducerInstance().getInstance(this.nameServerAddress, this.producerGroup); - } catch (Exception e) { - ErrorHandler handler = this.getHandler(); - if (handler != null) { - handler.error("Starting RocketmqLog4j2Appender [" + this.getName() - + "] nameServerAddress:" + nameServerAddress + " group:" + producerGroup + " " + e.getMessage()); - } - } - } - - /** - * Info,error,warn,callback method implementation - */ - public void append(LogEvent event) { - if (null == producer) { - return; - } - byte[] data = this.getLayout().toByteArray(event); - try { - Message msg = new Message(topic, tag, data); - msg.getProperties().put(ProducerInstance.APPENDER_TYPE, ProducerInstance.LOG4J2_APPENDER); - - //Send message and do not wait for the ack from the message broker. - producer.sendOneway(msg); - } catch (Exception e) { - ErrorHandler handler = this.getHandler(); - if (handler != null) { - String msg = new String(data); - handler.error("Could not send message in RocketmqLog4j2Appender [" + this.getName() + "].Message is : " + msg, e); - } - - } - } - - /** - * When system exit,this method will be called to close resources - */ - public boolean stop(long timeout, TimeUnit timeUnit) { - this.setStopping(); - try { - ProducerInstance.getProducerInstance().removeAndClose(this.nameServerAddress, this.producerGroup); - } catch (Exception e) { - ErrorHandler handler = this.getHandler(); - if (handler != null) { - handler.error("Closeing RocketmqLog4j2Appender [" + this.getName() - + "] nameServerAddress:" + nameServerAddress + " group:" + producerGroup + " " + e.getMessage()); - } - } - - boolean stopped = super.stop(timeout, timeUnit, false); - this.setStopped(); - return stopped; - } - - /** - * Log4j2 builder creator - */ - @PluginBuilderFactory - public static RocketmqLog4j2Appender.Builder newBuilder() { - return new RocketmqLog4j2Appender.Builder(); - } - - /** - * Log4j2 xml builder define - */ - public static class Builder implements org.apache.logging.log4j.core.util.Builder { - - @PluginBuilderAttribute - @Required(message = "A name for the RocketmqLog4j2Appender must be specified") - private String name; - - @PluginElement("Layout") - private Layout layout; - - @PluginElement("Filter") - private Filter filter; - - @PluginBuilderAttribute - private boolean ignoreExceptions; - - @PluginBuilderAttribute - private String tag; - - @PluginBuilderAttribute - private String nameServerAddress; - - @PluginBuilderAttribute - private String producerGroup; - - @PluginBuilderAttribute - @Required(message = "A topic name must be specified") - private String topic; - - private Builder() { - this.layout = SerializedLayout.createLayout(); - this.ignoreExceptions = true; - } - - public RocketmqLog4j2Appender.Builder setName(String name) { - this.name = name; - return this; - } - - public RocketmqLog4j2Appender.Builder setLayout(Layout layout) { - this.layout = layout; - return this; - } - - public RocketmqLog4j2Appender.Builder setFilter(Filter filter) { - this.filter = filter; - return this; - } - - public RocketmqLog4j2Appender.Builder setIgnoreExceptions(boolean ignoreExceptions) { - this.ignoreExceptions = ignoreExceptions; - return this; - } - - public RocketmqLog4j2Appender.Builder setTag(final String tag) { - this.tag = tag; - return this; - } - - public RocketmqLog4j2Appender.Builder setTopic(final String topic) { - this.topic = topic; - return this; - } - - public RocketmqLog4j2Appender.Builder setNameServerAddress(String nameServerAddress) { - this.nameServerAddress = nameServerAddress; - return this; - } - - public RocketmqLog4j2Appender.Builder setProducerGroup(String producerGroup) { - this.producerGroup = producerGroup; - return this; - } - - public RocketmqLog4j2Appender build() { - return new RocketmqLog4j2Appender(name, filter, layout, ignoreExceptions, - nameServerAddress, producerGroup, topic, tag); - } - } -} diff --git a/logappender/src/main/java/org/apache/rocketmq/logappender/logback/RocketmqLogbackAppender.java b/logappender/src/main/java/org/apache/rocketmq/logappender/logback/RocketmqLogbackAppender.java deleted file mode 100644 index 4018cd44a17..00000000000 --- a/logappender/src/main/java/org/apache/rocketmq/logappender/logback/RocketmqLogbackAppender.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender.logback; - -import ch.qos.logback.classic.net.LoggingEventPreSerializationTransformer; -import ch.qos.logback.classic.spi.ILoggingEvent; -import ch.qos.logback.core.AppenderBase; -import ch.qos.logback.core.Layout; -import ch.qos.logback.core.spi.PreSerializationTransformer; -import ch.qos.logback.core.status.ErrorStatus; -import org.apache.rocketmq.common.message.Message; -import org.apache.rocketmq.logappender.common.ProducerInstance; -import org.apache.rocketmq.client.producer.MQProducer; - -/** - * Logback Appender Component - */ -public class RocketmqLogbackAppender extends AppenderBase { - - /** - * Message tag define - */ - private String tag; - - /** - * Whitch topic to send log messages - */ - private String topic; - - /** - * RocketMQ nameserver address - */ - private String nameServerAddress; - - /** - * Log producer group - */ - private String producerGroup; - - /** - * Log producer send instance - */ - private MQProducer producer; - - private Layout layout; - - private PreSerializationTransformer pst = new LoggingEventPreSerializationTransformer(); - - /** - * Info,error,warn,callback method implementation - */ - @Override - protected void append(ILoggingEvent event) { - if (!isStarted()) { - return; - } - String logStr = this.layout.doLayout(event); - try { - Message msg = new Message(topic, tag, logStr.getBytes()); - msg.getProperties().put(ProducerInstance.APPENDER_TYPE, ProducerInstance.LOGBACK_APPENDER); - - //Send message and do not wait for the ack from the message broker. - producer.sendOneway(msg); - } catch (Exception e) { - addError("Could not send message in RocketmqLogbackAppender [" + name + "]. Message is : " + logStr, e); - } - } - - /** - * Options are activated and become effective only after calling this method. - */ - public void start() { - int errors = 0; - - if (this.layout == null) { - addStatus(new ErrorStatus("No layout set for the RocketmqLogbackAppender named \"" + name + "\".", this)); - errors++; - } - - if (errors > 0 || !checkEntryConditions()) { - return; - } - try { - producer = ProducerInstance.getProducerInstance().getInstance(nameServerAddress, producerGroup); - } catch (Exception e) { - addError("Starting RocketmqLogbackAppender [" + this.getName() - + "] nameServerAddress:" + nameServerAddress + " group:" + producerGroup + " " + e.getMessage()); - } - if (producer != null) { - super.start(); - } - } - - /** - * When system exit,this method will be called to close resources - */ - public synchronized void stop() { - // The synchronized modifier avoids concurrent append and close operations - if (!this.started) { - return; - } - - this.started = false; - - try { - ProducerInstance.getProducerInstance().removeAndClose(this.nameServerAddress, this.producerGroup); - } catch (Exception e) { - addError("Closeing RocketmqLogbackAppender [" + this.getName() - + "] nameServerAddress:" + nameServerAddress + " group:" + producerGroup + " " + e.getMessage()); - } - - // Help garbage collection - producer = null; - } - - protected boolean checkEntryConditions() { - String fail = null; - - if (this.topic == null) { - fail = "No topic"; - } - - if (fail != null) { - addError(fail + " for RocketmqLogbackAppender named [" + name + "]."); - return false; - } else { - return true; - } - } - - public Layout getLayout() { - return this.layout; - } - - /** - * Set the pattern layout to format the log. - */ - public void setLayout(Layout layout) { - this.layout = layout; - } - - public String getTag() { - return tag; - } - - public void setTag(String tag) { - this.tag = tag; - } - - public String getTopic() { - return topic; - } - - public void setTopic(String topic) { - this.topic = topic; - } - - public void setNameServerAddress(String nameServerAddress) { - this.nameServerAddress = nameServerAddress; - } - - public void setProducerGroup(String producerGroup) { - this.producerGroup = producerGroup; - } -} diff --git a/logappender/src/test/java/org/apache/rocketmq/logappender/AbstractTestCase.java b/logappender/src/test/java/org/apache/rocketmq/logappender/AbstractTestCase.java deleted file mode 100644 index 38904c0bfc7..00000000000 --- a/logappender/src/test/java/org/apache/rocketmq/logappender/AbstractTestCase.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender; - -import org.apache.rocketmq.client.producer.DefaultMQProducer; - -import org.apache.rocketmq.common.message.*; -import org.apache.rocketmq.logappender.common.ProducerInstance; -import org.junit.Before; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import static org.mockito.Mockito.*; - -import java.lang.reflect.Field; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Basic test rocketmq broker and name server init - */ -public class AbstractTestCase { - - private static CopyOnWriteArrayList messages = new CopyOnWriteArrayList<>(); - - @Before - public void mockLoggerAppender() throws Exception { - DefaultMQProducer defaultMQProducer = spy(new DefaultMQProducer("loggerAppender")); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - Message message = (Message) invocationOnMock.getArgument(0); - messages.add(message); - return null; - } - }).when(defaultMQProducer).sendOneway(any(Message.class)); - ProducerInstance spy = mock(ProducerInstance.class); - Field instance = ProducerInstance.class.getDeclaredField("instance"); - instance.setAccessible(true); - instance.set(ProducerInstance.class, spy); - doReturn(defaultMQProducer).when(spy).getInstance(anyString(), anyString()); - } - - public void clear() { - - } - - protected int consumeMessages(int count, final String key, int timeout) { - final AtomicInteger cc = new AtomicInteger(0); - for (Message message : messages) { - String body = new String(message.getBody()); - if (body.contains(key)) { - cc.incrementAndGet(); - } - } - return cc.get(); - } -} diff --git a/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jPropertiesTest.java b/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jPropertiesTest.java deleted file mode 100644 index 86752301a19..00000000000 --- a/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jPropertiesTest.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender; - -import org.apache.log4j.PropertyConfigurator; - -public class Log4jPropertiesTest extends Log4jTest { - - @Override - public void init() { - PropertyConfigurator.configure("src/test/resources/log4j-example.properties"); - } - - @Override - public String getType() { - return "properties"; - } -} diff --git a/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jTest.java b/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jTest.java deleted file mode 100644 index cf99c6937f4..00000000000 --- a/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jTest.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender; - -import org.apache.log4j.Logger; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public abstract class Log4jTest extends AbstractTestCase { - - @Before - public abstract void init(); - - public abstract String getType(); - - @Test - public void testLog4j() { - clear(); - Logger logger = Logger.getLogger("testLogger"); - for (int i = 0; i < 10; i++) { - logger.info("log4j " + this.getType() + " simple test message " + i); - } - int received = consumeMessages(10, "log4j", 10); - Assert.assertTrue(received > 5); - } - -} diff --git a/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jXmlTest.java b/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jXmlTest.java deleted file mode 100644 index 6743f7c1668..00000000000 --- a/logappender/src/test/java/org/apache/rocketmq/logappender/Log4jXmlTest.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender; - -import org.apache.log4j.xml.DOMConfigurator; - -public class Log4jXmlTest extends Log4jTest { - - @Override - public void init() { - DOMConfigurator.configure("src/test/resources/log4j-example.xml"); - } - - @Override - public String getType() { - return "xml"; - } -} diff --git a/logappender/src/test/java/org/apache/rocketmq/logappender/LogbackTest.java b/logappender/src/test/java/org/apache/rocketmq/logappender/LogbackTest.java deleted file mode 100644 index 3dc18f0dccf..00000000000 --- a/logappender/src/test/java/org/apache/rocketmq/logappender/LogbackTest.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender; - -import ch.qos.logback.classic.LoggerContext; -import ch.qos.logback.classic.joran.JoranConfigurator; -import ch.qos.logback.core.joran.spi.JoranException; -import ch.qos.logback.core.util.StatusPrinter; -import java.io.File; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class LogbackTest extends AbstractTestCase { - - @Before - public void init() throws JoranException { - LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory(); - JoranConfigurator configurator = new JoranConfigurator(); - configurator.setContext(lc); - lc.reset(); - configurator.doConfigure(new File("src/test/resources/logback-example.xml")); - StatusPrinter.printInCaseOfErrorsOrWarnings(lc); - } - - @Test - public void testLogback() { - clear(); - Logger logger = LoggerFactory.getLogger("testLogger"); - for (int i = 0; i < 10; i++) { - logger.info("logback test message " + i); - } - int received = consumeMessages(10, "logback", 10); - Assert.assertTrue(received >= 5); - } -} diff --git a/logappender/src/test/java/org/apache/rocketmq/logappender/log4j2Test.java b/logappender/src/test/java/org/apache/rocketmq/logappender/log4j2Test.java deleted file mode 100644 index 6f6af60876e..00000000000 --- a/logappender/src/test/java/org/apache/rocketmq/logappender/log4j2Test.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.rocketmq.logappender; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.core.config.Configurator; -import org.apache.rocketmq.client.exception.MQClientException; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class log4j2Test extends AbstractTestCase { - - @Before - public void init() { - Configurator.initialize("log4j2", "src/test/resources/log4j2-example.xml"); - } - - @Test - public void testLog4j2() throws InterruptedException, MQClientException { - clear(); - Logger logger = LogManager.getLogger("test"); - for (int i = 0; i < 10; i++) { - logger.info("log4j2 log message " + i); - } - int received = consumeMessages(10, "log4j2", 10); - Assert.assertTrue(received > 5); - } -} diff --git a/logappender/src/test/resources/log4j-example.properties b/logappender/src/test/resources/log4j-example.properties deleted file mode 100644 index 63b2a988c49..00000000000 --- a/logappender/src/test/resources/log4j-example.properties +++ /dev/null @@ -1,33 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -log4j.rootLogger=INFO,stdout -log4j.logger.testLogger=INFO,mq -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.Target=System.out -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%d %-4r [%t] (%F:%L) %-5p - %m%n -log4j.appender.store=org.apache.log4j.DailyRollingFileAppender -log4j.appender.store.File=${user.home}/logs/rocketmqlogs/appender.log -log4j.appender.store.Append=true -log4j.appender.store.DatePattern='_'yyyy-MM-dd'.log' -log4j.appender.store.layout=org.apache.log4j.PatternLayout -log4j.appender.store.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-4r [%t] (%F:%L) %-5p - %m%n -log4j.appender.mq=org.apache.rocketmq.logappender.log4j.RocketmqLog4jAppender -log4j.appender.mq.Tag=log -log4j.appender.mq.Topic=TopicTest -log4j.appender.mq.ProducerGroup=loggerAppender -log4j.appender.mq.NameServerAddress=127.0.0.1:9876 -log4j.appender.mq.layout=org.apache.log4j.PatternLayout -log4j.appender.mq.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-4r [%t] (%F:%L) %-5p - %m%n \ No newline at end of file diff --git a/logappender/src/test/resources/log4j-example.xml b/logappender/src/test/resources/log4j-example.xml deleted file mode 100644 index 6bddde9825c..00000000000 --- a/logappender/src/test/resources/log4j-example.xml +++ /dev/null @@ -1,56 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/logappender/src/test/resources/log4j2-example.xml b/logappender/src/test/resources/log4j2-example.xml deleted file mode 100644 index c310855498b..00000000000 --- a/logappender/src/test/resources/log4j2-example.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/logappender/src/test/resources/logback-example.xml b/logappender/src/test/resources/logback-example.xml deleted file mode 100644 index 3786137baf4..00000000000 --- a/logappender/src/test/resources/logback-example.xml +++ /dev/null @@ -1,81 +0,0 @@ - - - - - - - ${user.home}/logs/simple/system.log - true - - ${user.home}/logs/simple/system.%i.log - - 1 - 30 - - - 100MB - - - %date %p %t - %m%n - UTF-8 - - - - - System.out - - %date %p %t - %m%n - UTF-8 - - - - - ${user.home}/logs/simple/daily.log - - ${user.home}/logs/simple/daily.log.%d{yyyy-MM-dd_HH} - 30 - - - %date %p %t - %m%n - - - - - log1 - TopicTest - loggerAppender - 127.0.0.1:9876 - - %date %p %t - %m%n - - - - - - - - - - - - - - - - - diff --git a/pom.xml b/pom.xml index 1671bd1f29b..7932472039f 100644 --- a/pom.xml +++ b/pom.xml @@ -113,7 +113,6 @@ store namesrv remoting - logappender srvutil filter test From fc014049a2fbec2e1b87b09bfe16ec7b3e15029b Mon Sep 17 00:00:00 2001 From: Ni Ze Date: Thu, 17 Feb 2022 13:27:09 +0800 Subject: [PATCH 133/141] fix(log) add log for interruptedException (#3854) --- .../client/impl/consumer/DefaultLitePullConsumerImpl.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java index bb3536ba4cd..d5157501b2c 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java @@ -849,6 +849,8 @@ public void run() { break; } updatePullOffset(messageQueue, pullResult.getNextBeginOffset(), processQueue); + } catch (InterruptedException interruptedException) { + log.warn("Polling thread was interrupted.", interruptedException); } catch (Throwable e) { pullDelayTimeMills = pullTimeDelayMillsWhenException; log.error("An error occurred in pull message process.", e); From 7428d6520b9829051370badd00a7c4b1e8bf1e4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=99=E4=BB=A3=E7=A0=81=E7=9A=84=E7=8C=AB?= <33629004+shuangchengsun@users.noreply.github.com> Date: Thu, 17 Feb 2022 14:00:21 +0800 Subject: [PATCH 134/141] [ISSUE #3827]Improve Performance of transactional message and schedule message (#3828) * [ISSUE #3827] Part 1. Add Slave Timeout config * [ISSUE #3827] Part 2. Improve Performance of Transactional Message and Delay Message Scheduling Co-authored-by: huangli --- .../org/apache/rocketmq/store/CommitLog.java | 29 ++++--- .../rocketmq/store/DefaultMessageStore.java | 23 +++-- .../rocketmq/store/FlushDiskWatcher.java | 78 +++++++++++++++++ .../store/config/MessageStoreConfig.java | 9 ++ .../apache/rocketmq/store/ha/HAService.java | 9 +- .../rocketmq/store/FlushDiskWatcherTest.java | 84 +++++++++++++++++++ 6 files changed, 203 insertions(+), 29 deletions(-) create mode 100644 store/src/main/java/org/apache/rocketmq/store/FlushDiskWatcher.java create mode 100644 store/src/test/java/org/apache/rocketmq/store/FlushDiskWatcherTest.java diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 1af5c587622..bf66af500aa 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -79,6 +79,7 @@ public class CommitLog { private volatile Set fullStorePaths = Collections.emptySet(); private final MultiDispatch multiDispatch; + private final FlushDiskWatcher flushDiskWatcher; public CommitLog(final DefaultMessageStore defaultMessageStore) { String storePath = defaultMessageStore.getMessageStoreConfig().getStorePathCommitLog(); @@ -113,6 +114,7 @@ protected PutMessageThreadLocal initialValue() { this.multiDispatch = new MultiDispatch(defaultMessageStore, this); + flushDiskWatcher = new FlushDiskWatcher(); } public void setFullStorePaths(Set fullStorePaths) { @@ -136,6 +138,10 @@ public boolean load() { public void start() { this.flushCommitLogService.start(); + flushDiskWatcher.setDaemon(true); + flushDiskWatcher.start(); + + if (defaultMessageStore.getMessageStoreConfig().isTransientStorePoolEnable()) { this.commitLogService.start(); } @@ -147,6 +153,8 @@ public void shutdown() { } this.flushCommitLogService.shutdown(); + + flushDiskWatcher.shutdown(true); } public long flush() { @@ -722,10 +730,6 @@ public CompletableFuture asyncPutMessage(final MessageExtBroke } if (replicaStatus != PutMessageStatus.PUT_OK) { putMessageResult.setPutMessageStatus(replicaStatus); - if (replicaStatus == PutMessageStatus.FLUSH_SLAVE_TIMEOUT) { - log.error("do sync transfer other node, wait return, but failed, topic: {} tags: {} client address: {}", - msg.getTopic(), msg.getTags(), msg.getBornHostNameString()); - } } return putMessageResult; }); @@ -835,10 +839,6 @@ public CompletableFuture asyncPutMessages(final MessageExtBatc } if (replicaStatus != PutMessageStatus.PUT_OK) { putMessageResult.setPutMessageStatus(replicaStatus); - if (replicaStatus == PutMessageStatus.FLUSH_SLAVE_TIMEOUT) { - log.error("do sync transfer other node, wait return, but failed, topic: {} client address: {}", - messageExtBatch.getTopic(), messageExtBatch.getBornHostNameString()); - } } return putMessageResult; }); @@ -852,6 +852,7 @@ public CompletableFuture submitFlushRequest(AppendMessageResul if (messageExt.isWaitStoreMsgOK()) { GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(), this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout()); + flushDiskWatcher.add(request); service.putRequest(request); return request.future(); } else { @@ -876,7 +877,7 @@ public CompletableFuture submitReplicaRequest(AppendMessageRes if (messageExt.isWaitStoreMsgOK()) { if (service.isSlaveOK(result.getWroteBytes() + result.getWroteOffset())) { GroupCommitRequest request = new GroupCommitRequest(result.getWroteOffset() + result.getWroteBytes(), - this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout()); + this.defaultMessageStore.getMessageStoreConfig().getSlaveTimeout()); service.putRequest(request); service.getWaitNotifyObject().wakeupAll(); return request.future(); @@ -1142,19 +1143,17 @@ public long getJointime() { public static class GroupCommitRequest { private final long nextOffset; private CompletableFuture flushOKFuture = new CompletableFuture<>(); - private final long startTimestamp = System.currentTimeMillis(); - private long timeoutMillis = Long.MAX_VALUE; + private final long deadLine; public GroupCommitRequest(long nextOffset, long timeoutMillis) { this.nextOffset = nextOffset; - this.timeoutMillis = timeoutMillis; + this.deadLine = System.nanoTime() + (timeoutMillis * 1_000_000); } - public GroupCommitRequest(long nextOffset) { - this.nextOffset = nextOffset; + public long getDeadLine() { + return deadLine; } - public long getNextOffset() { return nextOffset; } diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java index bfc9fd282b1..f11d5f314bd 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -39,6 +39,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -509,18 +510,26 @@ public CompletableFuture asyncPutMessages(MessageExtBatch mess @Override public PutMessageResult putMessage(MessageExtBrokerInner msg) { - try { - return asyncPutMessage(msg).get(); - } catch (InterruptedException | ExecutionException e) { - return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, null); - } + return waitForPutResult(asyncPutMessage(msg)); } @Override public PutMessageResult putMessages(MessageExtBatch messageExtBatch) { + return waitForPutResult(asyncPutMessages(messageExtBatch)); + } + + private PutMessageResult waitForPutResult(CompletableFuture putMessageResultFuture) { try { - return asyncPutMessages(messageExtBatch).get(); - } catch (InterruptedException | ExecutionException e) { + int putMessageTimeout = + Math.max(this.messageStoreConfig.getSyncFlushTimeout(), + this.messageStoreConfig.getSlaveTimeout()) + 5000; + return putMessageResultFuture.get(putMessageTimeout, TimeUnit.MILLISECONDS); + } catch (ExecutionException | InterruptedException e) { + return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, null); + } catch (TimeoutException e) { + log.error("usually it will never timeout, putMessageTimeout is much bigger than slaveTimeout and " + + "flushTimeout so the result can be got anyway, but in some situations timeout will happen like full gc " + + "process hangs or other unexpected situations."); return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, null); } } diff --git a/store/src/main/java/org/apache/rocketmq/store/FlushDiskWatcher.java b/store/src/main/java/org/apache/rocketmq/store/FlushDiskWatcher.java new file mode 100644 index 00000000000..980a496c55a --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/FlushDiskWatcher.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store; + + +import java.util.concurrent.LinkedBlockingQueue; + +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.store.CommitLog.GroupCommitRequest; + +public class FlushDiskWatcher extends ServiceThread { + private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private final LinkedBlockingQueue commitRequests = new LinkedBlockingQueue<>(); + + @Override + public String getServiceName() { + return FlushDiskWatcher.class.getSimpleName(); + } + + @Override + public void run() { + while (!isStopped()) { + GroupCommitRequest request = null; + try { + request = commitRequests.take(); + } catch (InterruptedException e) { + log.warn("take flush disk commit request, but interrupted, this may caused by shutdown"); + continue; + } + while (!request.future().isDone()) { + long now = System.nanoTime(); + if (now - request.getDeadLine() >= 0) { + request.wakeupCustomer(PutMessageStatus.FLUSH_DISK_TIMEOUT); + break; + } + // To avoid frequent thread switching, replace future.get with sleep here, + long sleepTime = (request.getDeadLine() - now) / 1_000_000; + sleepTime = Math.min(10, sleepTime); + if (sleepTime == 0) { + request.wakeupCustomer(PutMessageStatus.FLUSH_DISK_TIMEOUT); + break; + } + try { + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + log.warn( + "An exception occurred while waiting for flushing disk to complete. this may caused by shutdown"); + break; + } + } + } + } + + public void add(GroupCommitRequest request) { + commitRequests.add(request); + } + + public int queueSize() { + return commitRequests.size(); + } +} 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 bb1e01f247e..45293e637b5 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 @@ -131,6 +131,7 @@ public class MessageStoreConfig { @ImportantField private FlushDiskType flushDiskType = FlushDiskType.ASYNC_FLUSH; private int syncFlushTimeout = 1000 * 5; + private int slaveTimeout = 3000; private String messageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h"; private long flushDelayOffsetInterval = 1000 * 10; @ImportantField @@ -547,6 +548,14 @@ public void setSyncFlushTimeout(int syncFlushTimeout) { this.syncFlushTimeout = syncFlushTimeout; } + public int getSlaveTimeout() { + return slaveTimeout; + } + + public void setSlaveTimeout(int slaveTimeout) { + this.slaveTimeout = slaveTimeout; + } + public String getHaMasterAddress() { return haMasterAddress; } diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java index 845935bb9dd..c9c05b308a7 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java +++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java @@ -287,17 +287,12 @@ private void doWaitTransfer() { if (!this.requestsRead.isEmpty()) { for (CommitLog.GroupCommitRequest req : this.requestsRead) { boolean transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset(); - long waitUntilWhen = HAService.this.defaultMessageStore.getSystemClock().now() - + HAService.this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout(); - while (!transferOK && HAService.this.defaultMessageStore.getSystemClock().now() < waitUntilWhen) { + long deadLine = req.getDeadLine(); + while (!transferOK && deadLine - System.nanoTime() > 0) { this.notifyTransferObject.waitForRunning(1000); transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset(); } - if (!transferOK) { - log.warn("transfer messsage to slave timeout, " + req.getNextOffset()); - } - req.wakeupCustomer(transferOK ? PutMessageStatus.PUT_OK : PutMessageStatus.FLUSH_SLAVE_TIMEOUT); } diff --git a/store/src/test/java/org/apache/rocketmq/store/FlushDiskWatcherTest.java b/store/src/test/java/org/apache/rocketmq/store/FlushDiskWatcherTest.java new file mode 100644 index 00000000000..60f392d4b63 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/FlushDiskWatcherTest.java @@ -0,0 +1,84 @@ +/** + * 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; + +import java.util.LinkedList; +import java.util.List; + +import org.apache.rocketmq.store.CommitLog.GroupCommitRequest; +import org.junit.Assert; +import org.junit.Test; + +public class FlushDiskWatcherTest { + + private final long timeoutMill = 5000; + + @Test + public void testTimeout() throws Exception { + FlushDiskWatcher flushDiskWatcher = new FlushDiskWatcher(); + flushDiskWatcher.setDaemon(true); + flushDiskWatcher.start(); + + int count = 100; + List requestList = new LinkedList<>(); + for (int i = 0; i < count; i++) { + GroupCommitRequest groupCommitRequest = + new GroupCommitRequest(0, timeoutMill); + requestList.add(groupCommitRequest); + flushDiskWatcher.add(groupCommitRequest); + } + Thread.sleep(2 * timeoutMill); + + for (GroupCommitRequest request : requestList) { + request.wakeupCustomer(PutMessageStatus.PUT_OK); + } + + for (GroupCommitRequest request : requestList) { + Assert.assertTrue(request.future().isDone()); + Assert.assertEquals(request.future().get(), PutMessageStatus.FLUSH_DISK_TIMEOUT); + } + Assert.assertEquals(flushDiskWatcher.queueSize(), 0); + flushDiskWatcher.shutdown(); + } + + @Test + public void testWatcher() throws Exception { + FlushDiskWatcher flushDiskWatcher = new FlushDiskWatcher(); + flushDiskWatcher.setDaemon(true); + flushDiskWatcher.start(); + + int count = 100; + List requestList = new LinkedList<>(); + for (int i = 0; i < count; i++) { + GroupCommitRequest groupCommitRequest = + new GroupCommitRequest(0, timeoutMill); + requestList.add(groupCommitRequest); + flushDiskWatcher.add(groupCommitRequest); + groupCommitRequest.wakeupCustomer(PutMessageStatus.PUT_OK); + } + Thread.sleep((timeoutMill << 20) / 1000000); + for (GroupCommitRequest request : requestList) { + Assert.assertTrue(request.future().isDone()); + Assert.assertEquals(request.future().get(), PutMessageStatus.PUT_OK); + } + Assert.assertEquals(flushDiskWatcher.queueSize(), 0); + flushDiskWatcher.shutdown(); + } + + +} From cc478bd425953175bcc59e2c98a90a08ea48464f Mon Sep 17 00:00:00 2001 From: HuiTong Date: Fri, 18 Feb 2022 23:26:12 +0800 Subject: [PATCH 135/141] fix scheduleAtFixedRate bug (#3861) --- .../impl/consumer/ConsumeMessageConcurrentlyService.java | 6 +++++- .../client/impl/consumer/ConsumeMessageOrderlyService.java | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java index 384f3f12bee..f8b74a02f9c 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyService.java @@ -93,7 +93,11 @@ public void start() { @Override public void run() { - cleanExpireMsg(); + try { + cleanExpireMsg(); + } catch (Throwable e) { + log.error("scheduleAtFixedRate cleanExpireMsg exception", e); + } } }, this.defaultMQPushConsumer.getConsumeTimeout(), this.defaultMQPushConsumer.getConsumeTimeout(), TimeUnit.MINUTES); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java index 812e8ab87dd..cc3aee4ef59 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageOrderlyService.java @@ -97,7 +97,11 @@ public void start() { this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override public void run() { - ConsumeMessageOrderlyService.this.lockMQPeriodically(); + try { + ConsumeMessageOrderlyService.this.lockMQPeriodically(); + } catch (Throwable e) { + log.error("scheduleAtFixedRate lockMQPeriodically exception", e); + } } }, 1000 * 1, ProcessQueue.REBALANCE_LOCK_INTERVAL, TimeUnit.MILLISECONDS); } From fef8670662a55e071bbaa7f96b3440ea6b938ee9 Mon Sep 17 00:00:00 2001 From: llIlll <10194588+llIlll@users.noreply.github.com> Date: Sat, 19 Feb 2022 09:48:15 +0800 Subject: [PATCH 136/141] [ISSUE #3863] Updated Notice file to 2022 (#3864) * Update .travis.yml Depracated MODERATE noisy mail for dev@rocketmq.apache.org. Still Failing: apache/rocketmq#5670 (5.0.0-alpha-static-topic - 5ebc327) @dongeforever * Updated Notice file to 2022 (#3735) * Updated Notice file to 2022 Co-authored-by: von gosling Co-authored-by: ZhangJian He --- distribution/NOTICE-BIN | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/NOTICE-BIN b/distribution/NOTICE-BIN index b7bba8daab0..145520b5449 100644 --- a/distribution/NOTICE-BIN +++ b/distribution/NOTICE-BIN @@ -1,5 +1,5 @@ Apache RocketMQ -Copyright 2016-2021 The Apache Software Foundation +Copyright 2016-2022 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From c1bb6aa27b10e1d55995ee9cef22b1905f5ee843 Mon Sep 17 00:00:00 2001 From: sunxi92 Date: Sat, 19 Feb 2022 09:52:31 +0800 Subject: [PATCH 137/141] [ISSUE #2986] Support for multiple ACL files in a fixed directory (#3761) * acl temp * acl * fix test case * fix code style issues * add considerations on compatibility to the original one ACL config file and scalability of supporting multiple config files in different directories. * fix test case testWatch * 1.fix some issues 2.add a detailed design document * Add warn log when the accesskey is repeated in multiple ACL files. * 1.Change the folder of acl configuration to conf/acl 2.Add the logic to check if path is a directory in the method of getAllAclFiles(String path) * Add a parameter in AclFileWatchService constructor. * Add logic to determine if path exists in the getAllAclFiles(String path) method in PlainPermissionManager.java and AclFileWatchService.java * Fix the serialization problem of allAclFileVersion field in clusterAclConfigVersion command * 1.Fix the serialization problem of allAclFileVersion field in clusterAclConfigVersion command 2.Improve the logic of updateAccessConfig method --- .../apache/rocketmq/acl/AccessValidator.java | 13 + .../acl/plain/PlainAccessValidator.java | 19 +- .../acl/plain/PlainPermissionManager.java | 427 +++++++++++---- .../rocketmq/acl/common/AclUtilsTest.java | 13 +- .../acl/plain/PlainAccessValidatorTest.java | 494 ++++++++++++------ .../acl/plain/PlainPermissionManagerTest.java | 59 +-- .../plain_acl.yml} | 32 +- .../test/resources/conf/plain_acl_null.yml | 18 - .../processor/AdminBrokerProcessor.java | 1 + .../rocketmq/client/impl/MQClientAPIImpl.java | 7 + .../protocol/body/ClusterAclVersionInfo.java | 15 +- .../GetBrokerAclConfigResponseHeader.java | 10 + distribution/conf/{ => acl}/plain_acl.yml | 44 +- ...ple_ACL_Files_\350\256\276\350\256\241.md" | 137 +++++ .../rocketmq/srvutil/AclFileWatchService.java | 162 ++++++ ...ClusterAclConfigVersionListSubCommand.java | 32 +- 16 files changed, 1107 insertions(+), 376 deletions(-) rename acl/src/test/resources/conf/{plain_acl_format_error.yml => acl/plain_acl.yml} (59%) delete mode 100644 acl/src/test/resources/conf/plain_acl_null.yml rename distribution/conf/{ => acl}/plain_acl.yml (61%) create mode 100644 "docs/cn/acl/RocketMQ_Multiple_ACL_Files_\350\256\276\350\256\241.md" create mode 100644 srvutil/src/main/java/org/apache/rocketmq/srvutil/AclFileWatchService.java diff --git a/acl/src/main/java/org/apache/rocketmq/acl/AccessValidator.java b/acl/src/main/java/org/apache/rocketmq/acl/AccessValidator.java index da53e982e07..167fa26e886 100644 --- a/acl/src/main/java/org/apache/rocketmq/acl/AccessValidator.java +++ b/acl/src/main/java/org/apache/rocketmq/acl/AccessValidator.java @@ -18,7 +18,10 @@ package org.apache.rocketmq.acl; import java.util.List; +import java.util.Map; + import org.apache.rocketmq.common.AclConfig; +import org.apache.rocketmq.common.DataVersion; import org.apache.rocketmq.common.PlainAccessConfig; import org.apache.rocketmq.remoting.protocol.RemotingCommand; @@ -60,17 +63,27 @@ public interface AccessValidator { * * @return */ + @Deprecated String getAclConfigVersion(); /** * Update globalWhiteRemoteAddresses in acl yaml config file + * * @return */ boolean updateGlobalWhiteAddrsConfig(List globalWhiteAddrsList); /** * get broker cluster acl config information + * * @return */ AclConfig getAllAclConfig(); + + /** + * get all access resource config version information + * + * @return + */ + Map getAllAclConfigVersion(); } diff --git a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java index c76816cb703..98858cfc965 100644 --- a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java +++ b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java @@ -27,6 +27,7 @@ import org.apache.rocketmq.acl.common.Permission; import org.apache.rocketmq.acl.common.SessionCredentials; import org.apache.rocketmq.common.AclConfig; +import org.apache.rocketmq.common.DataVersion; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.PlainAccessConfig; import org.apache.rocketmq.common.protocol.RequestCode; @@ -127,7 +128,7 @@ public AccessResource parse(RemotingCommand request, String remoteAddr) { SortedMap map = new TreeMap(); for (Map.Entry entry : request.getExtFields().entrySet()) { if (!SessionCredentials.SIGNATURE.equals(entry.getKey()) - && !MixAll.UNIQUE_MSG_QUERY_FLAG.equals(entry.getKey())) { + && !MixAll.UNIQUE_MSG_QUERY_FLAG.equals(entry.getKey())) { map.put(entry.getKey(), entry.getValue()); } } @@ -150,7 +151,7 @@ public boolean deleteAccessConfig(String accesskey) { return aclPlugEngine.deleteAccessConfig(accesskey); } - @Override public String getAclConfigVersion() { + @Override public String getAclConfigVersion() { return aclPlugEngine.getAclConfigDataVersion(); } @@ -161,4 +162,18 @@ public boolean deleteAccessConfig(String accesskey) { @Override public AclConfig getAllAclConfig() { return aclPlugEngine.getAllAclConfig(); } + + public Map createAclAccessConfigMap(Map existedAccountMap, + PlainAccessConfig plainAccessConfig) { + return aclPlugEngine.createAclAccessConfigMap(existedAccountMap, plainAccessConfig); + } + + public Map updateAclConfigFileVersion(Map updateAclConfigMap) { + return aclPlugEngine.updateAclConfigFileVersion(updateAclConfigMap); + } + + @Override + public Map getAllAclConfigVersion() { + return aclPlugEngine.getDataVersionMap(); + } } diff --git a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionManager.java b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionManager.java index f7af586df49..7f2936a0dad 100644 --- a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionManager.java +++ b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionManager.java @@ -18,6 +18,19 @@ import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.acl.common.AclConstants; import org.apache.rocketmq.acl.common.AclException; @@ -30,51 +43,149 @@ import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; -import org.apache.rocketmq.srvutil.FileWatchService; - -import java.io.File; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; +import org.apache.rocketmq.srvutil.AclFileWatchService; public class PlainPermissionManager { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME); - private static final String DEFAULT_PLAIN_ACL_FILE = "/conf/plain_acl.yml"; - private String fileHome = System.getProperty(MixAll.ROCKETMQ_HOME_PROPERTY, System.getenv(MixAll.ROCKETMQ_HOME_ENV)); - private String fileName = System.getProperty("rocketmq.acl.plain.file", DEFAULT_PLAIN_ACL_FILE); + private String defaultAclDir = fileHome + File.separator + "conf" + File.separator + "acl"; + + private String defaultAclFile = fileHome + File.separator + System.getProperty("rocketmq.acl.plain.file", "conf/plain_acl.yml"); - private Map plainAccessResourceMap = new HashMap<>(); + private Map> aclPlainAccessResourceMap = new HashMap<>(); + + private Map accessKeyTable = new HashMap<>(); private List globalWhiteRemoteAddressStrategy = new ArrayList<>(); private RemoteAddressStrategyFactory remoteAddressStrategyFactory = new RemoteAddressStrategyFactory(); + private Map> globalWhiteRemoteAddressStrategyMap = new HashMap<>(); + private boolean isWatchStart; + private Map dataVersionMap = new HashMap<>(); + + @Deprecated private final DataVersion dataVersion = new DataVersion(); + private List fileList = new ArrayList<>(); + public PlainPermissionManager() { load(); watch(); } + public List getAllAclFiles(String path) { + if (!new File(path).exists()) { + log.info("The default acl dir {} is not exist", path); + return new ArrayList<>(); + } + List allAclFileFullPath = new ArrayList<>(); + File file = new File(path); + File[] files = file.listFiles(); + for (int i = 0; i < files.length; i++) { + String fileName = files[i].getAbsolutePath(); + File f = new File(fileName); + if (fileName.equals(fileHome + MixAll.ACL_CONF_TOOLS_FILE)) { + continue; + } else if (fileName.endsWith(".yml") || fileName.endsWith(".yaml")) { + allAclFileFullPath.add(fileName); + } else if (f.isDirectory()) { + allAclFileFullPath.addAll(getAllAclFiles(fileName)); + } + } + return allAclFileFullPath; + } + public void load() { + if (fileHome == null || fileHome.isEmpty()) { + return; + } + + Map> aclPlainAccessResourceMap = new HashMap<>(); + Map accessKeyTable = new HashMap<>(); + List globalWhiteRemoteAddressStrategy = new ArrayList<>(); + Map> globalWhiteRemoteAddressStrategyMap = new HashMap<>(); + Map dataVersionMap = new HashMap<>(); + fileList = getAllAclFiles(defaultAclDir); + if (new File(defaultAclFile).exists() && !fileList.contains(defaultAclFile)) { + fileList.add(defaultAclFile); + } + + for (int i = 0; i < fileList.size(); i++) { + JSONObject plainAclConfData = AclUtils.getYamlDataObject(fileList.get(i), + JSONObject.class); + if (plainAclConfData == null || plainAclConfData.isEmpty()) { + throw new AclException(String.format("%s file is not data", fileList.get(i))); + } + log.info("Broker plain acl conf data is : ", plainAclConfData.toString()); + + List globalWhiteRemoteAddressStrategyList = new ArrayList<>(); + JSONArray globalWhiteRemoteAddressesList = plainAclConfData.getJSONArray("globalWhiteRemoteAddresses"); + if (globalWhiteRemoteAddressesList != null && !globalWhiteRemoteAddressesList.isEmpty()) { + for (int j = 0; j < globalWhiteRemoteAddressesList.size(); j++) { + globalWhiteRemoteAddressStrategyList.add(remoteAddressStrategyFactory. + getRemoteAddressStrategy(globalWhiteRemoteAddressesList.getString(j))); + } + } + if (globalWhiteRemoteAddressStrategyList.size() > 0) { + globalWhiteRemoteAddressStrategyMap.put(fileList.get(i), globalWhiteRemoteAddressStrategyList); + globalWhiteRemoteAddressStrategy.addAll(globalWhiteRemoteAddressStrategyList); + } + + JSONArray accounts = plainAclConfData.getJSONArray(AclConstants.CONFIG_ACCOUNTS); + Map plainAccessResourceMap = new HashMap<>(); + if (accounts != null && !accounts.isEmpty()) { + List plainAccessConfigList = accounts.toJavaList(PlainAccessConfig.class); + for (PlainAccessConfig plainAccessConfig : plainAccessConfigList) { + PlainAccessResource plainAccessResource = buildPlainAccessResource(plainAccessConfig); + //AccessKey can not be defined in multiple ACL files + if (accessKeyTable.get(plainAccessResource.getAccessKey()) == null) { + plainAccessResourceMap.put(plainAccessResource.getAccessKey(), plainAccessResource); + accessKeyTable.put(plainAccessResource.getAccessKey(), fileList.get(i)); + } else { + log.warn("The accesssKey {} is repeated in multiple ACL files", plainAccessResource.getAccessKey()); + } + } + } + if (plainAccessResourceMap.size() > 0) { + aclPlainAccessResourceMap.put(fileList.get(i), plainAccessResourceMap); + } + + JSONArray tempDataVersion = plainAclConfData.getJSONArray(AclConstants.CONFIG_DATA_VERSION); + DataVersion dataVersion = new DataVersion(); + if (tempDataVersion != null && !tempDataVersion.isEmpty()) { + List dataVersions = tempDataVersion.toJavaList(DataVersion.class); + DataVersion firstElement = dataVersions.get(0); + dataVersion.assignNewOne(firstElement); + } + dataVersionMap.put(fileList.get(i), dataVersion); + } + + if (dataVersionMap.containsKey(defaultAclFile)) { + this.dataVersion.assignNewOne(dataVersionMap.get(defaultAclFile)); + } + this.dataVersionMap = dataVersionMap; + this.globalWhiteRemoteAddressStrategyMap = globalWhiteRemoteAddressStrategyMap; + this.globalWhiteRemoteAddressStrategy = globalWhiteRemoteAddressStrategy; + this.aclPlainAccessResourceMap = aclPlainAccessResourceMap; + this.accessKeyTable = accessKeyTable; + } + + public void load(String aclFilePath) { Map plainAccessResourceMap = new HashMap<>(); List globalWhiteRemoteAddressStrategy = new ArrayList<>(); - JSONObject plainAclConfData = AclUtils.getYamlDataObject(fileHome + File.separator + fileName, + JSONObject plainAclConfData = AclUtils.getYamlDataObject(aclFilePath, JSONObject.class); if (plainAclConfData == null || plainAclConfData.isEmpty()) { - throw new AclException(String.format("%s file is not data", fileHome + File.separator + fileName)); + throw new AclException(String.format("%s file is not data", aclFilePath)); } log.info("Broker plain acl conf data is : ", plainAclConfData.toString()); JSONArray globalWhiteRemoteAddressesList = plainAclConfData.getJSONArray("globalWhiteRemoteAddresses"); @@ -85,43 +196,79 @@ public void load() { } } + this.globalWhiteRemoteAddressStrategy.addAll(globalWhiteRemoteAddressStrategy); + if (this.globalWhiteRemoteAddressStrategyMap.get(aclFilePath) != null) { + List remoteAddressStrategyList = this.globalWhiteRemoteAddressStrategyMap.get(aclFilePath); + for (int i = 0; i < remoteAddressStrategyList.size(); i++) { + this.globalWhiteRemoteAddressStrategy.remove(remoteAddressStrategyList.get(i)); + } + this.globalWhiteRemoteAddressStrategyMap.put(aclFilePath, globalWhiteRemoteAddressStrategy); + } + + JSONArray accounts = plainAclConfData.getJSONArray(AclConstants.CONFIG_ACCOUNTS); if (accounts != null && !accounts.isEmpty()) { List plainAccessConfigList = accounts.toJavaList(PlainAccessConfig.class); for (PlainAccessConfig plainAccessConfig : plainAccessConfigList) { PlainAccessResource plainAccessResource = buildPlainAccessResource(plainAccessConfig); - plainAccessResourceMap.put(plainAccessResource.getAccessKey(), plainAccessResource); + //AccessKey can not be defined in multiple ACL files + if (this.accessKeyTable.get(plainAccessResource.getAccessKey()) == null) { + plainAccessResourceMap.put(plainAccessResource.getAccessKey(), plainAccessResource); + this.accessKeyTable.put(plainAccessResource.getAccessKey(), aclFilePath); + } } } // For loading dataversion part just JSONArray tempDataVersion = plainAclConfData.getJSONArray(AclConstants.CONFIG_DATA_VERSION); + DataVersion dataVersion = new DataVersion(); if (tempDataVersion != null && !tempDataVersion.isEmpty()) { - List dataVersion = tempDataVersion.toJavaList(DataVersion.class); - DataVersion firstElement = dataVersion.get(0); - this.dataVersion.assignNewOne(firstElement); + List dataVersions = tempDataVersion.toJavaList(DataVersion.class); + DataVersion firstElement = dataVersions.get(0); + dataVersion.assignNewOne(firstElement); } - this.globalWhiteRemoteAddressStrategy = globalWhiteRemoteAddressStrategy; - this.plainAccessResourceMap = plainAccessResourceMap; + this.aclPlainAccessResourceMap.put(aclFilePath, plainAccessResourceMap); + this.dataVersionMap.put(aclFilePath, dataVersion); + if (aclFilePath.equals(defaultAclFile)) { + this.dataVersion.assignNewOne(dataVersion); + } } + + @Deprecated public String getAclConfigDataVersion() { return this.dataVersion.toJson(); } - private Map updateAclConfigFileVersion(Map updateAclConfigMap) { + public Map getDataVersionMap() { + return this.dataVersionMap; + } + + public Map updateAclConfigFileVersion(Map updateAclConfigMap) { + Object dataVersions = updateAclConfigMap.get(AclConstants.CONFIG_DATA_VERSION); + DataVersion dataVersion = new DataVersion(); + if (dataVersions != null) { + List> dataVersionList = (List>) dataVersions; + if (dataVersionList.size() > 0) { + dataVersion.setTimestamp((long) dataVersionList.get(0).get("timestamp")); + dataVersion.setCounter(new AtomicLong(Long.parseLong(dataVersionList.get(0).get("counter").toString()))); + } + } dataVersion.nextVersion(); List> versionElement = new ArrayList>(); - Map accountsMap = new LinkedHashMap() { - { - put(AclConstants.CONFIG_COUNTER, dataVersion.getCounter().longValue()); - put(AclConstants.CONFIG_TIME_STAMP, dataVersion.getTimestamp()); - } - }; + Map accountsMap = new LinkedHashMap(); + accountsMap.put(AclConstants.CONFIG_COUNTER, dataVersion.getCounter().longValue()); + accountsMap.put(AclConstants.CONFIG_TIME_STAMP, dataVersion.getTimestamp()); + versionElement.add(accountsMap); updateAclConfigMap.put(AclConstants.CONFIG_DATA_VERSION, versionElement); + + List> accounts = (List>) updateAclConfigMap.get(AclConstants.CONFIG_ACCOUNTS); + String accessKey = (String) accounts.get(0).get(AclConstants.CONFIG_ACCESS_KEY); + String aclFileName = accessKeyTable.get(accessKey); + dataVersionMap.put(aclFileName, dataVersion); return updateAclConfigMap; } @@ -136,14 +283,11 @@ public boolean updateAccessConfig(PlainAccessConfig plainAccessConfig) { Permission.checkResourcePerms(plainAccessConfig.getTopicPerms()); Permission.checkResourcePerms(plainAccessConfig.getGroupPerms()); - Map aclAccessConfigMap = AclUtils.getYamlDataObject(fileHome + File.separator + fileName, - Map.class); - if (aclAccessConfigMap == null || aclAccessConfigMap.isEmpty()) { - throw new AclException(String.format("the %s file is not found or empty", fileHome + File.separator + fileName)); - } - List> accounts = (List>) aclAccessConfigMap.get(AclConstants.CONFIG_ACCOUNTS); - Map updateAccountMap = null; - if (accounts != null) { + if (accessKeyTable.containsKey(plainAccessConfig.getAccessKey())) { + Map updateAccountMap = null; + String aclFileName = accessKeyTable.get(plainAccessConfig.getAccessKey()); + Map aclAccessConfigMap = AclUtils.getYamlDataObject(aclFileName, Map.class); + List> accounts = (List>) aclAccessConfigMap.get(AclConstants.CONFIG_ACCOUNTS); for (Map account : accounts) { if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(plainAccessConfig.getAccessKey())) { // Update acl access config elements @@ -151,27 +295,62 @@ public boolean updateAccessConfig(PlainAccessConfig plainAccessConfig) { updateAccountMap = createAclAccessConfigMap(account, plainAccessConfig); accounts.add(updateAccountMap); aclAccessConfigMap.put(AclConstants.CONFIG_ACCOUNTS, accounts); - - if (AclUtils.writeDataObject(fileHome + File.separator + fileName, updateAclConfigFileVersion(aclAccessConfigMap))) { - return true; + break; + } + } + Map accountMap = aclPlainAccessResourceMap.get(aclFileName); + if (accountMap == null) { + accountMap = new HashMap(1); + accountMap.put(plainAccessConfig.getAccessKey(), buildPlainAccessResource(plainAccessConfig)); + } else if (accountMap.size() == 0) { + accountMap.put(plainAccessConfig.getAccessKey(), buildPlainAccessResource(plainAccessConfig)); + } else { + for (Map.Entry entry : accountMap.entrySet()) { + if (entry.getValue().equals(plainAccessConfig.getAccessKey())) { + PlainAccessResource plainAccessResource = buildPlainAccessResource(plainAccessConfig); + accountMap.put(entry.getKey(), plainAccessResource); + break; + } + } + } + aclPlainAccessResourceMap.put(aclFileName, accountMap); + return AclUtils.writeDataObject(aclFileName, updateAclConfigFileVersion(aclAccessConfigMap)); + } else { + String fileName = defaultAclFile; + //Create acl access config elements on the default acl file + if (aclPlainAccessResourceMap.get(defaultAclFile) == null || aclPlainAccessResourceMap.get(defaultAclFile).size() == 0) { + try { + File defaultAclFile = new File(fileName); + if (!defaultAclFile.exists()) { + defaultAclFile.createNewFile(); } - return false; + } catch (IOException e) { + log.warn("create default acl file has exception when update accessConfig. ", e); } } - // Create acl access config elements + Map aclAccessConfigMap = AclUtils.getYamlDataObject(defaultAclFile, Map.class); + if (aclAccessConfigMap == null) { + aclAccessConfigMap = new HashMap<>(); + aclAccessConfigMap.put(AclConstants.CONFIG_ACCOUNTS, new ArrayList<>()); + } + List> accounts = (List>) aclAccessConfigMap.get(AclConstants.CONFIG_ACCOUNTS); accounts.add(createAclAccessConfigMap(null, plainAccessConfig)); aclAccessConfigMap.put(AclConstants.CONFIG_ACCOUNTS, accounts); - if (AclUtils.writeDataObject(fileHome + File.separator + fileName, updateAclConfigFileVersion(aclAccessConfigMap))) { - return true; + accessKeyTable.put(plainAccessConfig.getAccessKey(), fileName); + if (aclPlainAccessResourceMap.get(fileName) == null) { + Map plainAccessResourceMap = new HashMap<>(1); + plainAccessResourceMap.put(plainAccessConfig.getAccessKey(), buildPlainAccessResource(plainAccessConfig)); + aclPlainAccessResourceMap.put(fileName, plainAccessResourceMap); + } else { + Map plainAccessResourceMap = aclPlainAccessResourceMap.get(fileName); + plainAccessResourceMap.put(plainAccessConfig.getAccessKey(), buildPlainAccessResource(plainAccessConfig)); + aclPlainAccessResourceMap.put(fileName, plainAccessResourceMap); } - return false; + return AclUtils.writeDataObject(defaultAclFile, updateAclConfigFileVersion(aclAccessConfigMap)); } - - log.error("Users must ensure that the acl yaml config file has accounts node element"); - return false; } - private Map createAclAccessConfigMap(Map existedAccountMap, + public Map createAclAccessConfigMap(Map existedAccountMap, PlainAccessConfig plainAccessConfig) { Map newAccountsMap = null; @@ -225,38 +404,37 @@ public boolean deleteAccessConfig(String accesskey) { return false; } - Map aclAccessConfigMap = AclUtils.getYamlDataObject(fileHome + File.separator + fileName, - Map.class); - if (aclAccessConfigMap == null || aclAccessConfigMap.isEmpty()) { - throw new AclException(String.format("the %s file is not found or empty", fileHome + File.separator + fileName)); - } - List> accounts = (List>) aclAccessConfigMap.get("accounts"); - if (accounts != null) { + if (accessKeyTable.containsKey(accesskey)) { + String aclFileName = accessKeyTable.get(accesskey); + Map aclAccessConfigMap = AclUtils.getYamlDataObject(aclFileName, + Map.class); + if (aclAccessConfigMap == null || aclAccessConfigMap.isEmpty()) { + throw new AclException(String.format("the %s file is not found or empty", aclFileName)); + } + List> accounts = (List>) aclAccessConfigMap.get("accounts"); Iterator> itemIterator = accounts.iterator(); while (itemIterator.hasNext()) { - if (itemIterator.next().get(AclConstants.CONFIG_ACCESS_KEY).equals(accesskey)) { // Delete the related acl config element itemIterator.remove(); aclAccessConfigMap.put(AclConstants.CONFIG_ACCOUNTS, accounts); - - if (AclUtils.writeDataObject(fileHome + File.separator + fileName, updateAclConfigFileVersion(aclAccessConfigMap))) { - return true; - } - return false; + return AclUtils.writeDataObject(aclFileName, updateAclConfigFileVersion(aclAccessConfigMap)); } } } - log.error("Users must ensure that the acl yaml config file has related acl config elements"); - return false; } public boolean updateGlobalWhiteAddrsConfig(List globalWhiteAddrsList) { - Map aclAccessConfigMap = AclUtils.getYamlDataObject(fileHome + File.separator + fileName, - Map.class); + + if (globalWhiteAddrsList == null) { + log.error("Parameter value globalWhiteAddrsList is null,Please check your parameter"); + return false; + } + + Map aclAccessConfigMap = AclUtils.getYamlDataObject(defaultAclFile, Map.class); if (aclAccessConfigMap == null || aclAccessConfigMap.isEmpty()) { - throw new AclException(String.format("the %s file is not found or empty", fileHome + File.separator + fileName)); + throw new AclException(String.format("the %s file is not found or empty", defaultAclFile)); } List globalWhiteRemoteAddrList = (List) aclAccessConfigMap.get(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS); @@ -267,13 +445,41 @@ public boolean updateGlobalWhiteAddrsConfig(List globalWhiteAddrsList) { } // Update globalWhiteRemoteAddr element in memory map firstly aclAccessConfigMap.put(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS, globalWhiteRemoteAddrList); - if (AclUtils.writeDataObject(fileHome + File.separator + fileName, updateAclConfigFileVersion(aclAccessConfigMap))) { - return true; - } + return AclUtils.writeDataObject(defaultAclFile, updateAclConfigFileVersion(aclAccessConfigMap)); + } + + log.error("Users must ensure that the acl yaml config file has globalWhiteRemoteAddresses flag in the {} firstly", defaultAclFile); + return false; + } + + public boolean updateGlobalWhiteAddrsConfig(List globalWhiteAddrsList, String fileName) { + if (globalWhiteAddrsList == null) { + log.error("Parameter value globalWhiteAddrsList is null,Please check your parameter"); + return false; + } + + File file = new File(fileName); + if (!file.exists() || file.isDirectory()) { + log.error("Parameter value fileName is not exist or is a directory,Please check your parameter"); return false; } - log.error("Users must ensure that the acl yaml config file has globalWhiteRemoteAddresses flag firstly"); + Map aclAccessConfigMap = AclUtils.getYamlDataObject(fileName, Map.class); + if (aclAccessConfigMap == null || aclAccessConfigMap.isEmpty()) { + throw new AclException(String.format("the %s file is not found or empty", fileName)); + } + List globalWhiteRemoteAddrList = (List) aclAccessConfigMap.get(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS); + if (globalWhiteRemoteAddrList != null) { + globalWhiteRemoteAddrList.clear(); + if (globalWhiteAddrsList != null) { + globalWhiteRemoteAddrList.addAll(globalWhiteAddrsList); + } + // Update globalWhiteRemoteAddr element in memory map firstly + aclAccessConfigMap.put(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS, globalWhiteRemoteAddrList); + return AclUtils.writeDataObject(fileName, updateAclConfigFileVersion(aclAccessConfigMap)); + } + + log.error("Users must ensure that the acl yaml config file has globalWhiteRemoteAddresses flag in the {} firstly", fileName); return false; } @@ -281,40 +487,65 @@ public AclConfig getAllAclConfig() { AclConfig aclConfig = new AclConfig(); List configs = new ArrayList<>(); List whiteAddrs = new ArrayList<>(); - JSONObject plainAclConfData = AclUtils.getYamlDataObject(fileHome + File.separator + fileName, - JSONObject.class); - if (plainAclConfData == null || plainAclConfData.isEmpty()) { - throw new AclException(String.format("%s file is not data", fileHome + File.separator + fileName)); - } - JSONArray globalWhiteAddrs = plainAclConfData.getJSONArray(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS); - if (globalWhiteAddrs != null && !globalWhiteAddrs.isEmpty()) { - whiteAddrs = globalWhiteAddrs.toJavaList(String.class); - } - JSONArray accounts = plainAclConfData.getJSONArray(AclConstants.CONFIG_ACCOUNTS); - if (accounts != null && !accounts.isEmpty()) { - configs = accounts.toJavaList(PlainAccessConfig.class); + Set accessKeySets = new HashSet<>(); + + for (int i = 0; i < fileList.size(); i++) { + String path = fileList.get(i); + JSONObject plainAclConfData = AclUtils.getYamlDataObject(path, + JSONObject.class); + if (plainAclConfData == null || plainAclConfData.isEmpty()) { + throw new AclException(String.format("%s file is not data", path)); + } + JSONArray globalWhiteAddrs = plainAclConfData.getJSONArray(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS); + if (globalWhiteAddrs != null && !globalWhiteAddrs.isEmpty()) { + whiteAddrs.addAll(globalWhiteAddrs.toJavaList(String.class)); + } + + JSONArray accounts = plainAclConfData.getJSONArray(AclConstants.CONFIG_ACCOUNTS); + if (accounts != null && !accounts.isEmpty()) { + List plainAccessConfigs = accounts.toJavaList(PlainAccessConfig.class); + for (int j = 0; j < plainAccessConfigs.size(); j++) { + if (!accessKeySets.contains(plainAccessConfigs.get(j).getAccessKey())) { + accessKeySets.add(plainAccessConfigs.get(j).getAccessKey()); + PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); + plainAccessConfig.setGroupPerms(plainAccessConfigs.get(j).getGroupPerms()); + plainAccessConfig.setDefaultTopicPerm(plainAccessConfigs.get(j).getDefaultTopicPerm()); + plainAccessConfig.setDefaultGroupPerm(plainAccessConfigs.get(j).getDefaultGroupPerm()); + plainAccessConfig.setAccessKey(plainAccessConfigs.get(j).getAccessKey()); + plainAccessConfig.setSecretKey(plainAccessConfigs.get(j).getSecretKey()); + plainAccessConfig.setAdmin(plainAccessConfigs.get(j).isAdmin()); + plainAccessConfig.setTopicPerms(plainAccessConfigs.get(j).getTopicPerms()); + plainAccessConfig.setWhiteRemoteAddress(plainAccessConfigs.get(j).getWhiteRemoteAddress()); + configs.add(plainAccessConfig); + } + } + } } - aclConfig.setGlobalWhiteAddrs(whiteAddrs); aclConfig.setPlainAccessConfigs(configs); + aclConfig.setGlobalWhiteAddrs(whiteAddrs); return aclConfig; } private void watch() { try { - String watchFilePath = fileHome + fileName; - FileWatchService fileWatchService = new FileWatchService(new String[] {watchFilePath}, new FileWatchService.Listener() { + AclFileWatchService aclFileWatchService = new AclFileWatchService(defaultAclDir, defaultAclFile, new AclFileWatchService.Listener() { @Override - public void onChanged(String path) { - log.info("The plain acl yml changed, reload the context"); + public void onFileChanged(String aclFileName) { + load(aclFileName); + } + + @Override + public void onFileNumChanged(String path) { load(); } }); - fileWatchService.start(); - log.info("Succeed to start AclWatcherService"); + aclFileWatchService.start(); + log.info("Succeed to start AclFileWatchService"); this.isWatchStart = true; } catch (Exception e) { log.error("Failed to start AclWatcherService", e); } + } void checkPerm(PlainAccessResource needCheckedAccess, PlainAccessResource ownedAccess) { @@ -355,18 +586,19 @@ void checkPerm(PlainAccessResource needCheckedAccess, PlainAccessResource ownedA } void clearPermissionInfo() { - this.plainAccessResourceMap.clear(); + this.aclPlainAccessResourceMap.clear(); + this.accessKeyTable.clear(); this.globalWhiteRemoteAddressStrategy.clear(); } public void checkPlainAccessConfig(PlainAccessConfig plainAccessConfig) throws AclException { if (plainAccessConfig.getAccessKey() == null - || plainAccessConfig.getSecretKey() == null - || plainAccessConfig.getAccessKey().length() <= AclConstants.ACCESS_KEY_MIN_LENGTH - || plainAccessConfig.getSecretKey().length() <= AclConstants.SECRET_KEY_MIN_LENGTH) { + || plainAccessConfig.getSecretKey() == null + || plainAccessConfig.getAccessKey().length() <= AclConstants.ACCESS_KEY_MIN_LENGTH + || plainAccessConfig.getSecretKey().length() <= AclConstants.SECRET_KEY_MIN_LENGTH) { throw new AclException(String.format( - "The accessKey=%s and secretKey=%s cannot be null and length should longer than 6", - plainAccessConfig.getAccessKey(), plainAccessConfig.getSecretKey())); + "The accessKey=%s and secretKey=%s cannot be null and length should longer than 6", + plainAccessConfig.getAccessKey(), plainAccessConfig.getSecretKey())); } } @@ -404,12 +636,13 @@ public void validate(PlainAccessResource plainAccessResource) { throw new AclException(String.format("No accessKey is configured")); } - if (!plainAccessResourceMap.containsKey(plainAccessResource.getAccessKey())) { + if (!accessKeyTable.containsKey(plainAccessResource.getAccessKey())) { throw new AclException(String.format("No acl config for %s", plainAccessResource.getAccessKey())); } // Check the white addr for accesskey - PlainAccessResource ownedAccess = plainAccessResourceMap.get(plainAccessResource.getAccessKey()); + String aclFileName = accessKeyTable.get(plainAccessResource.getAccessKey()); + PlainAccessResource ownedAccess = aclPlainAccessResourceMap.get(aclFileName).get(plainAccessResource.getAccessKey()); if (ownedAccess.getRemoteAddressStrategy().match(plainAccessResource)) { return; } diff --git a/acl/src/test/java/org/apache/rocketmq/acl/common/AclUtilsTest.java b/acl/src/test/java/org/apache/rocketmq/acl/common/AclUtilsTest.java index e2a212ada8e..e79cd908c0c 100644 --- a/acl/src/test/java/org/apache/rocketmq/acl/common/AclUtilsTest.java +++ b/acl/src/test/java/org/apache/rocketmq/acl/common/AclUtilsTest.java @@ -294,23 +294,18 @@ public void getYamlDataIgnoreFileNotFoundExceptionTest() { Assert.assertTrue(yamlDataObject == null); } - @Test(expected = Exception.class) - public void getYamlDataExceptionTest() { - - AclUtils.getYamlDataObject("src/test/resources/conf/plain_acl_format_error.yml", Map.class); - } @Test public void getAclRPCHookTest() { - RPCHook errorContRPCHook = AclUtils.getAclRPCHook("src/test/resources/conf/plain_acl_format_error.yml"); - Assert.assertNull(errorContRPCHook); + //RPCHook errorContRPCHook = AclUtils.getAclRPCHook("src/test/resources/conf/plain_acl_format_error.yml"); + //Assert.assertNull(errorContRPCHook); RPCHook noFileRPCHook = AclUtils.getAclRPCHook("src/test/resources/plain_acl_format_error1.yml"); Assert.assertNull(noFileRPCHook); - RPCHook emptyContRPCHook = AclUtils.getAclRPCHook("src/test/resources/conf/plain_acl_null.yml"); - Assert.assertNull(emptyContRPCHook); + //RPCHook emptyContRPCHook = AclUtils.getAclRPCHook("src/test/resources/conf/plain_acl_null.yml"); + //Assert.assertNull(emptyContRPCHook); RPCHook incompleteContRPCHook = AclUtils.getAclRPCHook("src/test/resources/conf/plain_acl_incomplete.yml"); Assert.assertNull(incompleteContRPCHook); 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 fb8eba1e920..62d98572053 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 @@ -16,9 +16,13 @@ */ package org.apache.rocketmq.acl.plain; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -29,6 +33,7 @@ import org.apache.rocketmq.acl.common.AclUtils; import org.apache.rocketmq.acl.common.SessionCredentials; import org.apache.rocketmq.common.AclConfig; +import org.apache.rocketmq.common.DataVersion; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.PlainAccessConfig; import org.apache.rocketmq.common.protocol.RequestCode; @@ -56,10 +61,11 @@ public class PlainAccessValidatorTest { private AclClientRPCHook aclClient; private SessionCredentials sessionCredentials; + @Before public void init() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl.yml"); + File file = new File("src/test/resources"); + System.setProperty("rocketmq.home.dir", file.getAbsolutePath()); plainAccessValidator = new PlainAccessValidator(); sessionCredentials = new SessionCredentials(); sessionCredentials.setAccessKey("RocketMQ"); @@ -420,16 +426,14 @@ public void validateGetAllTopicConfigTest() { } @Test - public void updateAccessAclYamlConfigNormalTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_update_create.yml"); - - String targetFileName = "src/test/resources/conf/plain_acl_update_create.yml"; + public void addAccessAclYamlConfigTest() throws InterruptedException { + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); - plainAccessConfig.setAccessKey("RocketMQ"); + plainAccessConfig.setAccessKey("rocketmq3"); plainAccessConfig.setSecretKey("1234567890"); + plainAccessConfig.setWhiteRemoteAddress("192.168.0.*"); plainAccessConfig.setDefaultGroupPerm("PUB"); plainAccessConfig.setDefaultTopicPerm("SUB"); List topicPerms = new ArrayList(); @@ -442,16 +446,21 @@ public void updateAccessAclYamlConfigNormalTest() { plainAccessConfig.setGroupPerms(groupPerms); PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - // Update acl access yaml config file plainAccessValidator.updateAccessConfig(plainAccessConfig); + Thread.sleep(10000); - Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>) readableMap.get("accounts"); - Map verifyMap = null; - for (Map account : accounts) { - if (account.get("accessKey").equals(plainAccessConfig.getAccessKey())) { - verifyMap = account; - break; + Map verifyMap = new HashMap<>(); + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + for (PlainAccessConfig plainAccessConfig1 : plainAccessConfigs) { + if (plainAccessConfig1.getAccessKey().equals(plainAccessConfig.getAccessKey())) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig1.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_TOPIC_PERM, plainAccessConfig1.getDefaultTopicPerm()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_GROUP_PERM, plainAccessConfig1.getDefaultGroupPerm()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig1.isAdmin()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig1.getWhiteRemoteAddress()); + verifyMap.put(AclConstants.CONFIG_TOPIC_PERMS, plainAccessConfig1.getTopicPerms()); + verifyMap.put(AclConstants.CONFIG_GROUP_PERMS, plainAccessConfig1.getGroupPerms()); } } @@ -463,58 +472,125 @@ public void updateAccessAclYamlConfigNormalTest() { Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).size(), 2); Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).size(), 2); - // Verify the dateversion element is correct or not - List> dataVersions = (List>) readableMap.get("dataVersion"); - Assert.assertEquals(1, dataVersions.get(0).get("counter")); + String aclFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; + Map readableMap = AclUtils.getYamlDataObject(aclFileName, Map.class); + List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); + Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); - // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } @Test - public void updateAccessAclYamlConfigTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_update_create.yml"); + public void getAccessAclYamlConfigTest() { + String accessKey = "rocketmq2"; + PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + Map verifyMap = new HashMap<>(); + for (PlainAccessConfig plainAccessConfig : plainAccessConfigs) { + if (plainAccessConfig.getAccessKey().equals(accessKey)) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig.isAdmin()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig.getWhiteRemoteAddress()); + } + } + + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "12345678"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_ADMIN_ROLE), true); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_WHITE_ADDR), "192.168.1.*"); + + String aclFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/acl/plain_acl.yml"; + Map dataVersionMap = plainAccessValidator.getAllAclConfigVersion(); + DataVersion dataVersion = dataVersionMap.get(aclFileName); + Assert.assertEquals(0, dataVersion.getCounter().get()); + } - String targetFileName = "src/test/resources/conf/plain_acl_update_create.yml"; + @Test + public void updateAccessAclYamlConfigTest() throws InterruptedException{ + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); - plainAccessConfig.setAccessKey("RocketMQ"); - plainAccessConfig.setSecretKey("123456789111"); + plainAccessConfig.setAccessKey("rocketmq3"); + plainAccessConfig.setSecretKey("1234567890"); + plainAccessConfig.setWhiteRemoteAddress("192.168.0.*"); + plainAccessConfig.setDefaultGroupPerm("PUB"); + plainAccessConfig.setDefaultTopicPerm("SUB"); + List topicPerms = new ArrayList(); + topicPerms.add("topicC=PUB|SUB"); + topicPerms.add("topicB=PUB"); + plainAccessConfig.setTopicPerms(topicPerms); + List groupPerms = new ArrayList(); + groupPerms.add("groupB=PUB|SUB"); + groupPerms.add("groupC=DENY"); + plainAccessConfig.setGroupPerms(groupPerms); PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - // Update element in the acl access yaml config file plainAccessValidator.updateAccessConfig(plainAccessConfig); - Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>) readableMap.get(AclConstants.CONFIG_ACCOUNTS); - Map verifyMap = null; - for (Map account : accounts) { - if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(plainAccessConfig.getAccessKey())) { - verifyMap = account; - break; + Thread.sleep(10000); + + PlainAccessConfig plainAccessConfig1 = new PlainAccessConfig(); + plainAccessConfig1.setAccessKey("rocketmq3"); + plainAccessConfig1.setSecretKey("1234567891"); + plainAccessConfig1.setWhiteRemoteAddress("192.168.0.*"); + plainAccessConfig1.setDefaultGroupPerm("PUB"); + plainAccessConfig1.setDefaultTopicPerm("SUB"); + List topicPerms1 = new ArrayList(); + topicPerms1.add("topicC=PUB|SUB"); + topicPerms1.add("topicB=PUB"); + plainAccessConfig1.setTopicPerms(topicPerms1); + List groupPerms1 = new ArrayList(); + groupPerms1.add("groupB=PUB|SUB"); + groupPerms1.add("groupC=DENY"); + plainAccessConfig1.setGroupPerms(groupPerms1); + + plainAccessValidator.updateAccessConfig(plainAccessConfig1); + + Thread.sleep(10000); + + Map verifyMap = new HashMap<>(); + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + for (PlainAccessConfig plainAccessConfig2 : plainAccessConfigs) { + if (plainAccessConfig2.getAccessKey().equals(plainAccessConfig1.getAccessKey())) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig2.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_TOPIC_PERM, plainAccessConfig2.getDefaultTopicPerm()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_GROUP_PERM, plainAccessConfig2.getDefaultGroupPerm()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig2.isAdmin()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig2.getWhiteRemoteAddress()); + verifyMap.put(AclConstants.CONFIG_TOPIC_PERMS, plainAccessConfig2.getTopicPerms()); + verifyMap.put(AclConstants.CONFIG_GROUP_PERMS, plainAccessConfig2.getGroupPerms()); } } - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "123456789111"); - // Restore the backup file and flush to yaml file + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "1234567891"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_TOPIC_PERM), "SUB"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_GROUP_PERM), "PUB"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_ADMIN_ROLE), false); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_WHITE_ADDR), "192.168.0.*"); + Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).size(), 2); + Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).size(), 2); + + String aclFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; + Map readableMap = AclUtils.getYamlDataObject(aclFileName, Map.class); + List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); + Assert.assertEquals(2, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); + AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } @Test - public void createAndUpdateAccessAclYamlConfigNormalTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_update_create.yml"); - - String targetFileName = "src/test/resources/conf/plain_acl_update_create.yml"; + public void deleteAccessAclYamlConfigTest() throws InterruptedException { + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); - plainAccessConfig.setAccessKey("RocketMQ33"); - plainAccessConfig.setSecretKey("123456789111"); + plainAccessConfig.setAccessKey("rocketmq3"); + plainAccessConfig.setSecretKey("1234567890"); + plainAccessConfig.setWhiteRemoteAddress("192.168.0.*"); plainAccessConfig.setDefaultGroupPerm("PUB"); - plainAccessConfig.setDefaultTopicPerm("DENY"); + plainAccessConfig.setDefaultTopicPerm("SUB"); List topicPerms = new ArrayList(); topicPerms.add("topicC=PUB|SUB"); topicPerms.add("topicB=PUB"); @@ -525,194 +601,257 @@ public void createAndUpdateAccessAclYamlConfigNormalTest() { plainAccessConfig.setGroupPerms(groupPerms); PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - // Create element in the acl access yaml config file plainAccessValidator.updateAccessConfig(plainAccessConfig); - Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>) readableMap.get(AclConstants.CONFIG_ACCOUNTS); - Map verifyMap = null; - for (Map account : accounts) { - if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(plainAccessConfig.getAccessKey())) { - verifyMap = account; - break; - } - } - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "123456789111"); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_TOPIC_PERM), "DENY"); - Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_DEFAULT_GROUP_PERM), "PUB"); - Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).size(), 2); - Assert.assertEquals(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).size(), 2); - Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).contains("topicC=PUB|SUB")); - Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_TOPIC_PERMS)).contains("topicB=PUB")); - Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).contains("groupB=PUB|SUB")); - Assert.assertTrue(((List) verifyMap.get(AclConstants.CONFIG_GROUP_PERMS)).contains("groupC=DENY")); - - // Verify the dateversion element is correct or not - List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); - Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); + String accessKey = "rocketmq3"; + plainAccessValidator.deleteAccessConfig(accessKey); + Thread.sleep(10000); - // Update element in the acl config yaml file - PlainAccessConfig plainAccessConfig2 = new PlainAccessConfig(); - plainAccessConfig2.setAccessKey("rocketmq2"); - plainAccessConfig2.setSecretKey("1234567890123"); - - // Update acl access yaml config file secondly - plainAccessValidator.updateAccessConfig(plainAccessConfig2); - - Map readableMap2 = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts2 = (List>) readableMap2.get(AclConstants.CONFIG_ACCOUNTS); - Map verifyMap2 = null; - for (Map account : accounts2) { - if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(plainAccessConfig2.getAccessKey())) { - verifyMap2 = account; - break; + Map verifyMap = new HashMap<>(); + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + for (PlainAccessConfig plainAccessConfig1 : plainAccessConfigs) { + if (plainAccessConfig1.getAccessKey().equals(accessKey)) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig1.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_TOPIC_PERM, plainAccessConfig1.getDefaultTopicPerm()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_GROUP_PERM, plainAccessConfig1.getDefaultGroupPerm()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig1.isAdmin()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig1.getWhiteRemoteAddress()); + verifyMap.put(AclConstants.CONFIG_TOPIC_PERMS, plainAccessConfig1.getTopicPerms()); + verifyMap.put(AclConstants.CONFIG_GROUP_PERMS, plainAccessConfig1.getGroupPerms()); } } - // Verify the dateversion element after updating is correct or not - List> dataVersions2 = (List>) readableMap2.get(AclConstants.CONFIG_DATA_VERSION); - Assert.assertEquals(2, dataVersions2.get(0).get(AclConstants.CONFIG_COUNTER)); - Assert.assertEquals(verifyMap2.get(AclConstants.CONFIG_SECRET_KEY), "1234567890123"); + Assert.assertEquals(verifyMap.size(), 0); - // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } - @Test(expected = AclException.class) - public void updateAccessAclYamlConfigExceptionTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_update_create.yml"); + @Test + public void updateGlobalWhiteRemoteAddressesTest() throws InterruptedException { + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; + Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); - plainAccessConfig.setAccessKey("RocketMQ"); - plainAccessConfig.setSecretKey("12345"); + List globalWhiteAddrsList = new ArrayList<>(); + globalWhiteAddrsList.add("192.168.1.*"); PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - // Update acl access yaml config file - plainAccessValidator.updateAccessConfig(plainAccessConfig); + Assert.assertEquals(plainAccessValidator.updateGlobalWhiteAddrsConfig(globalWhiteAddrsList), true); + + String aclFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; + Map readableMap = AclUtils.getYamlDataObject(aclFileName, Map.class); + List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); + Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); + AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } @Test - public void deleteAccessAclYamlConfigNormalTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_delete.yml"); + public void addYamlConfigTest() throws IOException, InterruptedException { + String fileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/acl/plain_acl_test.yml"; + File transport = new File(fileName); + transport.delete(); + transport.createNewFile(); + FileWriter writer = new FileWriter(transport); + writer.write("accounts:\r\n"); + writer.write("- accessKey: watchrocketmqx\r\n"); + writer.write(" secretKey: 12345678\r\n"); + writer.write(" whiteRemoteAddress: 127.0.0.1\r\n"); + writer.write(" admin: true\r\n"); + writer.flush(); + writer.close(); + + Thread.sleep(1000); - String targetFileName = "src/test/resources/conf/plain_acl_delete.yml"; - Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - - String accessKey = "rocketmq2"; PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - plainAccessValidator.deleteAccessConfig(accessKey); - - Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - List> accounts = (List>) readableMap.get(AclConstants.CONFIG_ACCOUNTS); - Map verifyMap = null; - for (Map account : accounts) { - if (account.get(AclConstants.CONFIG_ACCESS_KEY).equals(accessKey)) { - verifyMap = account; - break; + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + Map verifyMap = new HashMap<>(); + for (PlainAccessConfig plainAccessConfig : plainAccessConfigs) { + if (plainAccessConfig.getAccessKey().equals("watchrocketmqx")) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig.getWhiteRemoteAddress()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig.isAdmin()); } } - // Verify the specified element is removed or not - Assert.assertEquals(verifyMap, null); - // Verify the dateversion element is correct or not - List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); - Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "12345678"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_WHITE_ADDR), "127.0.0.1"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_ADMIN_ROLE), true); - // Restore the backup file and flush to yaml file - AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); + Map dataVersionMap = plainAccessValidator.getAllAclConfigVersion(); + DataVersion dataVersion = dataVersionMap.get(fileName); + Assert.assertEquals(0, dataVersion.getCounter().get()); + + transport.delete(); } @Test - public void updateAccessAclYamlConfigWithNoAccoutsExceptionTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_with_no_accouts.yml"); + public void updateAccessAnotherAclYamlConfigTest() throws IOException, InterruptedException { + String fileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/acl/plain_acl_test.yml"; + File transport = new File(fileName); + transport.delete(); + transport.createNewFile(); + FileWriter writer = new FileWriter(transport); + writer.write("accounts:\r\n"); + writer.write("- accessKey: watchrocketmqy\r\n"); + writer.write(" secretKey: 12345678\r\n"); + writer.write(" whiteRemoteAddress: 127.0.0.1\r\n"); + writer.write(" admin: true\r\n"); + writer.write("- accessKey: watchrocketmqx\r\n"); + writer.write(" secretKey: 123456781\r\n"); + writer.write(" whiteRemoteAddress: 127.0.0.1\r\n"); + writer.write(" admin: true\r\n"); + writer.flush(); + writer.close(); + + Thread.sleep(1000); - String targetFileName = "src/test/resources/conf/plain_acl_with_no_accouts.yml"; - Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); + PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); - plainAccessConfig.setAccessKey("RocketMQ"); + plainAccessConfig.setAccessKey("watchrocketmqy"); plainAccessConfig.setSecretKey("1234567890"); + plainAccessConfig.setWhiteRemoteAddress("127.0.0.1"); + plainAccessConfig.setAdmin(false); - PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - // Update acl access yaml config file and verify the return value is true - Assert.assertEquals(plainAccessValidator.updateAccessConfig(plainAccessConfig), false); - } + plainAccessValidator.updateAccessConfig(plainAccessConfig); - @Test(expected = AclException.class) - public void createAndUpdateAccessAclYamlConfigExceptionTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_update_create.yml"); + Thread.sleep(1000); - PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); - plainAccessConfig.setAccessKey("RocketMQ33"); - plainAccessConfig.setSecretKey("123456789111"); - List topicPerms = new ArrayList(); - topicPerms.add("topicB=PUB"); - plainAccessConfig.setTopicPerms(topicPerms); - List groupPerms = new ArrayList(); - groupPerms.add("groupC=DENY1"); - plainAccessConfig.setGroupPerms(groupPerms); + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + Map verifyMap = new HashMap<>(); + for (PlainAccessConfig plainAccessConfig1 : plainAccessConfigs) { + if (plainAccessConfig1.getAccessKey().equals("watchrocketmqy")) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig1.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig1.getWhiteRemoteAddress()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig1.isAdmin()); + } + } + + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "1234567890"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_WHITE_ADDR), "127.0.0.1"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_ADMIN_ROLE), false); + + Map dataVersionMap = plainAccessValidator.getAllAclConfigVersion(); + DataVersion dataVersion = dataVersionMap.get(fileName); + Assert.assertEquals(1, dataVersion.getCounter().get()); + + transport.delete(); - PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - // Create element in the acl access yaml config file - plainAccessValidator.updateAccessConfig(plainAccessConfig); } @Test(expected = AclException.class) public void createAndUpdateAccessAclNullSkExceptionTest() { + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/acl/plain_acl.yml"; + Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); + PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); plainAccessConfig.setAccessKey("RocketMQ33"); // secret key is null PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); plainAccessValidator.updateAccessConfig(plainAccessConfig); + + AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } @Test - public void updateGlobalWhiteAddrsNormalTest() { - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_global_white_addrs.yml"); - - String targetFileName = "src/test/resources/conf/plain_acl_global_white_addrs.yml"; + public void addAccessDefaultAclYamlConfigTest() throws InterruptedException { + PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); - PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); - // Update global white remote addr value list in the acl access yaml config file + PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); + plainAccessConfig.setAccessKey("watchrocketmqh"); + plainAccessConfig.setSecretKey("1234567890"); + plainAccessConfig.setWhiteRemoteAddress("127.0.0.1"); + plainAccessConfig.setAdmin(false); - List globalWhiteAddrsList = new ArrayList(); - globalWhiteAddrsList.add("10.10.154.1"); - globalWhiteAddrsList.add("10.10.154.2"); - globalWhiteAddrsList.add("10.10.154.3"); - plainAccessValidator.updateGlobalWhiteAddrsConfig(globalWhiteAddrsList); + plainAccessValidator.updateAccessConfig(plainAccessConfig); - Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); + Thread.sleep(10000); + + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + Map verifyMap = new HashMap<>(); + for (PlainAccessConfig plainAccessConfig1 : plainAccessConfigs) { + if (plainAccessConfig1.getAccessKey().equals("watchrocketmqh")) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig1.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig1.getWhiteRemoteAddress()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig1.isAdmin()); + } + } - List globalWhiteAddrList = (List) readableMap.get(AclConstants.CONFIG_GLOBAL_WHITE_ADDRS); - Assert.assertTrue(globalWhiteAddrList.contains("10.10.154.1")); - Assert.assertTrue(globalWhiteAddrList.contains("10.10.154.2")); - Assert.assertTrue(globalWhiteAddrList.contains("10.10.154.3")); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_SECRET_KEY), "1234567890"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_WHITE_ADDR), "127.0.0.1"); + Assert.assertEquals(verifyMap.get(AclConstants.CONFIG_ADMIN_ROLE), false); - // Verify the dateversion element is correct or not + Map readableMap = AclUtils.getYamlDataObject(targetFileName, Map.class); List> dataVersions = (List>) readableMap.get(AclConstants.CONFIG_DATA_VERSION); Assert.assertEquals(1, dataVersions.get(0).get(AclConstants.CONFIG_COUNTER)); - // Restore the backup file and flush to yaml file AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } + @Test + public void deleteAccessAnotherAclYamlConfigTest() throws IOException, InterruptedException { + String fileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/acl/plain_acl_test.yml"; + File transport = new File(fileName); + transport.delete(); + transport.createNewFile(); + FileWriter writer = new FileWriter(transport); + writer.write("accounts:\r\n"); + writer.write("- accessKey: watchrocketmqx\r\n"); + writer.write(" secretKey: 12345678\r\n"); + writer.write(" whiteRemoteAddress: 127.0.0.1\r\n"); + writer.write(" admin: true\r\n"); + writer.write("- accessKey: watchrocketmqy\r\n"); + writer.write(" secretKey: 1234567890\r\n"); + writer.write(" whiteRemoteAddress: 127.0.0.1\r\n"); + writer.write(" admin: false\r\n"); + writer.flush(); + writer.close(); + + Thread.sleep(1000); + + PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); + plainAccessValidator.deleteAccessConfig("watchrocketmqx"); + Thread.sleep(10000); + + Map verifyMap = new HashMap<>(); + AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); + List plainAccessConfigs = aclConfig.getPlainAccessConfigs(); + for (PlainAccessConfig plainAccessConfig : plainAccessConfigs) { + if (plainAccessConfig.getAccessKey().equals("watchrocketmqx")) { + verifyMap.put(AclConstants.CONFIG_SECRET_KEY, plainAccessConfig.getSecretKey()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_TOPIC_PERM, plainAccessConfig.getDefaultTopicPerm()); + verifyMap.put(AclConstants.CONFIG_DEFAULT_GROUP_PERM, plainAccessConfig.getDefaultGroupPerm()); + verifyMap.put(AclConstants.CONFIG_ADMIN_ROLE, plainAccessConfig.isAdmin()); + verifyMap.put(AclConstants.CONFIG_WHITE_ADDR, plainAccessConfig.getWhiteRemoteAddress()); + verifyMap.put(AclConstants.CONFIG_TOPIC_PERMS, plainAccessConfig.getTopicPerms()); + verifyMap.put(AclConstants.CONFIG_GROUP_PERMS, plainAccessConfig.getGroupPerms()); + } + } + + Assert.assertEquals(verifyMap.size(), 0); + + transport.delete(); + } + @Test public void getAllAclConfigTest() { PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); AclConfig aclConfig = plainAccessValidator.getAllAclConfig(); - Assert.assertEquals(aclConfig.getGlobalWhiteAddrs().size(), 2); + Assert.assertEquals(aclConfig.getGlobalWhiteAddrs().size(), 4); Assert.assertEquals(aclConfig.getPlainAccessConfigs().size(), 2); } @Test public void updateAccessConfigEmptyPermListTest() { + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; + Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); + PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); String accessKey = "updateAccessConfigEmptyPerm"; @@ -724,15 +863,23 @@ public void updateAccessConfigEmptyPermListTest() { plainAccessConfig.setTopicPerms(new ArrayList<>()); plainAccessValidator.updateAccessConfig(plainAccessConfig); - PlainAccessConfig result = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs() - .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); - Assert.assertEquals(0, result.getTopicPerms().size()); + List plainAccessConfigs = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs(); + for (int i = 0; i < plainAccessConfigs.size(); i++) { + PlainAccessConfig plainAccessConfig1 = plainAccessConfigs.get(i); + if (plainAccessConfig1.getAccessKey() == accessKey) { + Assert.assertEquals(0, plainAccessConfig1.getTopicPerms().size()); + } + } plainAccessValidator.deleteAccessConfig(accessKey); + AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } @Test public void updateAccessConfigEmptyWhiteRemoteAddressTest() { + String targetFileName = System.getProperty("rocketmq.home.dir") + File.separator + "conf/plain_acl.yml"; + Map backUpAclConfigMap = AclUtils.getYamlDataObject(targetFileName, Map.class); + PlainAccessValidator plainAccessValidator = new PlainAccessValidator(); PlainAccessConfig plainAccessConfig = new PlainAccessConfig(); String accessKey = "updateAccessConfigEmptyWhiteRemoteAddress"; @@ -744,10 +891,15 @@ public void updateAccessConfigEmptyWhiteRemoteAddressTest() { plainAccessConfig.setWhiteRemoteAddress(""); plainAccessValidator.updateAccessConfig(plainAccessConfig); - PlainAccessConfig result = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs() - .stream().filter(c -> c.getAccessKey().equals(accessKey)).findFirst().orElse(null); - Assert.assertEquals("", result.getWhiteRemoteAddress()); + List plainAccessConfigs = plainAccessValidator.getAllAclConfig().getPlainAccessConfigs(); + for (int i = 0; i < plainAccessConfigs.size(); i++) { + PlainAccessConfig plainAccessConfig1 = plainAccessConfigs.get(i); + if (plainAccessConfig1.getAccessKey() == accessKey) { + Assert.assertEquals("", plainAccessConfig1.getWhiteRemoteAddress()); + } + } plainAccessValidator.deleteAccessConfig(accessKey); + AclUtils.writeDataObject(targetFileName, backUpAclConfigMap); } } 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 d5ffb0c1d15..d2dabc0b2e7 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 @@ -62,9 +62,9 @@ public void init() throws NoSuchFieldException, SecurityException, IOException { ANYPlainAccessResource = clonePlainAccessResource(Permission.ANY); DENYPlainAccessResource = clonePlainAccessResource(Permission.DENY); - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl.yml"); - + File file = new File("src/test/resources"); + System.setProperty("rocketmq.home.dir", file.getAbsolutePath()); + plainPermissionManager = new PlainPermissionManager(); } @@ -117,7 +117,7 @@ public void buildPlainAccessResourceTest() { Assert.assertEquals(resourcePermMap.size(), 3); Assert.assertEquals(resourcePermMap.get(PlainAccessResource.getRetryTopic("groupA")).byteValue(), Permission.DENY); - Assert.assertEquals(resourcePermMap.get(PlainAccessResource.getRetryTopic("groupB")).byteValue(), Permission.PUB|Permission.SUB); + Assert.assertEquals(resourcePermMap.get(PlainAccessResource.getRetryTopic("groupB")).byteValue(), Permission.PUB | Permission.SUB); Assert.assertEquals(resourcePermMap.get(PlainAccessResource.getRetryTopic("groupC")).byteValue(), Permission.PUB); List topics = new ArrayList(); @@ -130,7 +130,7 @@ public void buildPlainAccessResourceTest() { Assert.assertEquals(resourcePermMap.size(), 6); Assert.assertEquals(resourcePermMap.get("topicA").byteValue(), Permission.DENY); - Assert.assertEquals(resourcePermMap.get("topicB").byteValue(), Permission.PUB|Permission.SUB); + Assert.assertEquals(resourcePermMap.get("topicB").byteValue(), Permission.PUB | Permission.SUB); Assert.assertEquals(resourcePermMap.get("topicC").byteValue(), Permission.PUB); } @@ -157,6 +157,7 @@ public void checkPerm() { plainPermissionManager.checkPerm(plainAccessResource, ANYPlainAccessResource); } + @Test(expected = AclException.class) public void checkErrorPermDefaultValueNotMatch() { @@ -164,6 +165,7 @@ public void checkErrorPermDefaultValueNotMatch() { plainAccessResource.addResourceAndPerm("topicF", Permission.PUB); plainPermissionManager.checkPerm(plainAccessResource, SUBPlainAccessResource); } + @Test(expected = AclException.class) public void accountNullTest() { plainAccessConfig.setAccessKey(null); @@ -184,25 +186,20 @@ public void passWordtNullTest() { @Test(expected = AclException.class) public void passWordThanTest() { - plainAccessConfig.setAccessKey("123"); + plainAccessConfig.setSecretKey("123"); plainPermissionManager.buildPlainAccessResource(plainAccessConfig); } - @Test(expected = AclException.class) - public void testPlainAclPlugEngineInit() { - System.setProperty("rocketmq.home.dir", ""); - new PlainPermissionManager().load(); - } @SuppressWarnings("unchecked") @Test public void cleanAuthenticationInfoTest() throws IllegalAccessException { // PlainPermissionManager.addPlainAccessResource(plainAccessResource); - Map> plainAccessResourceMap = (Map>) FieldUtils.readDeclaredField(plainPermissionManager, "plainAccessResourceMap", true); + Map> plainAccessResourceMap = (Map>) FieldUtils.readDeclaredField(plainPermissionManager, "aclPlainAccessResourceMap", true); Assert.assertFalse(plainAccessResourceMap.isEmpty()); plainPermissionManager.clearPermissionInfo(); - plainAccessResourceMap = (Map>) FieldUtils.readDeclaredField(plainPermissionManager, "plainAccessResourceMap", true); + plainAccessResourceMap = (Map>) FieldUtils.readDeclaredField(plainPermissionManager, "aclPlainAccessResourceMap", true); Assert.assertTrue(plainAccessResourceMap.isEmpty()); // RemoveDataVersionFromYamlFile("src/test/resources/conf/plain_acl.yml"); } @@ -213,34 +210,36 @@ public void isWatchStartTest() { PlainPermissionManager plainPermissionManager = new PlainPermissionManager(); Assert.assertTrue(plainPermissionManager.isWatchStart()); // RemoveDataVersionFromYamlFile("src/test/resources/conf/plain_acl.yml"); - } - @Test - public void testWatch() throws IOException, IllegalAccessException ,InterruptedException{ - System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl-test.yml"); - String fileName =System.getProperty("rocketmq.home.dir", "src/test/resources")+System.getProperty("rocketmq.acl.plain.file", "/conf/plain_acl.yml"); + public void testWatch() throws IOException, IllegalAccessException, InterruptedException { + File file = new File("src/test/resources"); + System.setProperty("rocketmq.home.dir", file.getAbsolutePath()); + + String fileName = System.getProperty("rocketmq.home.dir") + File.separator + "/conf/acl/plain_acl_test.yml"; File transport = new File(fileName); transport.delete(); transport.createNewFile(); FileWriter writer = new FileWriter(transport); writer.write("accounts:\r\n"); - writer.write("- accessKey: watchrocketmq\r\n"); + writer.write("- accessKey: watchrocketmqx\r\n"); writer.write(" secretKey: 12345678\r\n"); writer.write(" whiteRemoteAddress: 127.0.0.1\r\n"); writer.write(" admin: true\r\n"); writer.flush(); writer.close(); + Thread.sleep(1000); PlainPermissionManager plainPermissionManager = new PlainPermissionManager(); Assert.assertTrue(plainPermissionManager.isWatchStart()); + Map accessKeyTable = (Map) FieldUtils.readDeclaredField(plainPermissionManager, "accessKeyTable", true); + String aclFileName = accessKeyTable.get("watchrocketmqx"); { - Map plainAccessResourceMap = (Map) FieldUtils.readDeclaredField(plainPermissionManager, "plainAccessResourceMap", true); - PlainAccessResource accessResource = plainAccessResourceMap.get("watchrocketmq"); + Map> plainAccessResourceMap = (Map>) FieldUtils.readDeclaredField(plainPermissionManager, "aclPlainAccessResourceMap", true); + PlainAccessResource accessResource = plainAccessResourceMap.get(aclFileName).get("watchrocketmqx"); Assert.assertNotNull(accessResource); Assert.assertEquals(accessResource.getSecretKey(), "12345678"); Assert.assertTrue(accessResource.isAdmin()); @@ -251,16 +250,16 @@ public void testWatch() throws IOException, IllegalAccessException ,InterruptedE List> accounts = (List>) updatedMap.get("accounts"); accounts.get(0).remove("accessKey"); accounts.get(0).remove("secretKey"); - accounts.get(0).put("accessKey", "watchrocketmq1"); + accounts.get(0).put("accessKey", "watchrocketmq1y"); accounts.get(0).put("secretKey", "88888888"); accounts.get(0).put("admin", "false"); // Update file and flush to yaml file AclUtils.writeDataObject(fileName, updatedMap); - Thread.sleep(1000); + Thread.sleep(10000); { - Map plainAccessResourceMap = (Map) FieldUtils.readDeclaredField(plainPermissionManager, "plainAccessResourceMap", true); - PlainAccessResource accessResource = plainAccessResourceMap.get("watchrocketmq1"); + Map> plainAccessResourceMap = (Map>) FieldUtils.readDeclaredField(plainPermissionManager, "aclPlainAccessResourceMap", true); + PlainAccessResource accessResource = plainAccessResourceMap.get(aclFileName).get("watchrocketmq1y"); Assert.assertNotNull(accessResource); Assert.assertEquals(accessResource.getSecretKey(), "88888888"); Assert.assertFalse(accessResource.isAdmin()); @@ -268,13 +267,5 @@ public void testWatch() throws IOException, IllegalAccessException ,InterruptedE } transport.delete(); System.setProperty("rocketmq.home.dir", "src/test/resources"); - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl.yml"); - } - - @Test(expected = AclException.class) - public void initializeTest() { - System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl_null.yml"); - new PlainPermissionManager(); - } } diff --git a/acl/src/test/resources/conf/plain_acl_format_error.yml b/acl/src/test/resources/conf/acl/plain_acl.yml similarity index 59% rename from acl/src/test/resources/conf/plain_acl_format_error.yml rename to acl/src/test/resources/conf/acl/plain_acl.yml index 46782c5655e..5641a94bfa2 100644 --- a/acl/src/test/resources/conf/plain_acl_format_error.yml +++ b/acl/src/test/resources/conf/acl/plain_acl.yml @@ -15,12 +15,30 @@ ## suggested format -date 2015-02-01 -accounts: - - name: Jai +globalWhiteRemoteAddresses: + - 10.10.103.* + - 192.168.0.* + accounts: -- accessKey: RocketMQ - secretKey: 12345678 - whiteRemoteAddress: 192.168.0.* - admin: false + - accessKey: RocketMQ + secretKey: 12345678 + whiteRemoteAddress: 192.168.0.* + admin: false + defaultTopicPerm: DENY + defaultGroupPerm: SUB + topicPerms: + - topicA=DENY + - topicB=PUB|SUB + - topicC=SUB + groupPerms: + # the group should convert to retry topic + - groupA=DENY + - groupB=SUB + - groupC=SUB + + - accessKey: rocketmq2 + secretKey: 12345678 + whiteRemoteAddress: 192.168.1.* + # if it is admin, it could access all resources + admin: true diff --git a/acl/src/test/resources/conf/plain_acl_null.yml b/acl/src/test/resources/conf/plain_acl_null.yml deleted file mode 100644 index bc30380c888..00000000000 --- a/acl/src/test/resources/conf/plain_acl_null.yml +++ /dev/null @@ -1,18 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -## suggested format - - diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java index d7d7b63bbf4..9d188ab52c9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java @@ -429,6 +429,7 @@ private RemotingCommand getBrokerAclConfigVersion(ChannelHandlerContext ctx, Rem try { AccessValidator accessValidator = this.brokerController.getAccessValidatorMap().get(PlainAccessValidator.class); + responseHeader.setAllAclFileVersion(JSON.toJSONString(accessValidator.getAllAclConfigVersion())); responseHeader.setVersion(accessValidator.getAclConfigVersion()); responseHeader.setBrokerAddr(this.brokerController.getBrokerAddr()); responseHeader.setBrokerName(this.brokerController.getBrokerConfig().getBrokerName()); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index 6289bf92dbd..8506432e08f 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -166,6 +166,7 @@ import org.apache.rocketmq.remoting.protocol.LanguageCode; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import com.alibaba.fastjson.JSON; public class MQClientAPIImpl { @@ -387,6 +388,12 @@ public ClusterAclVersionInfo getBrokerClusterAclInfo(final String addr, clusterAclVersionInfo.setBrokerName(responseHeader.getBrokerName()); clusterAclVersionInfo.setBrokerAddr(responseHeader.getBrokerAddr()); clusterAclVersionInfo.setAclConfigDataVersion(DataVersion.fromJson(responseHeader.getVersion(), DataVersion.class)); + HashMap dataVersionMap = JSON.parseObject(responseHeader.getAllAclFileVersion(), HashMap.class); + Map allAclConfigDataVersion = new HashMap(); + for (Map.Entry entry : dataVersionMap.entrySet()) { + allAclConfigDataVersion.put(entry.getKey(),DataVersion.fromJson(JSON.toJSONString(entry.getValue()), DataVersion.class)); + } + clusterAclVersionInfo.setAllAclConfigDataVersion(allAclConfigDataVersion); return clusterAclVersionInfo; } default: diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterAclVersionInfo.java b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterAclVersionInfo.java index aeae9d59a96..27c55de3f45 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterAclVersionInfo.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/body/ClusterAclVersionInfo.java @@ -19,14 +19,19 @@ import org.apache.rocketmq.common.DataVersion; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import java.util.Map; + public class ClusterAclVersionInfo extends RemotingSerializable { private String brokerName; private String brokerAddr; + @Deprecated private DataVersion aclConfigDataVersion; + private Map allAclConfigDataVersion; + private String clusterName; public String getBrokerName() { @@ -45,7 +50,6 @@ public void setBrokerAddr(String brokerAddr) { this.brokerAddr = brokerAddr; } - public String getClusterName() { return clusterName; } @@ -61,4 +65,13 @@ public DataVersion getAclConfigDataVersion() { public void setAclConfigDataVersion(DataVersion aclConfigDataVersion) { this.aclConfigDataVersion = aclConfigDataVersion; } + + public Map getAllAclConfigDataVersion() { + return allAclConfigDataVersion; + } + + public void setAllAclConfigDataVersion( + Map allAclConfigDataVersion) { + this.allAclConfigDataVersion = allAclConfigDataVersion; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetBrokerAclConfigResponseHeader.java b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetBrokerAclConfigResponseHeader.java index 43fbe47ae7f..70c6d5e2020 100644 --- a/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetBrokerAclConfigResponseHeader.java +++ b/common/src/main/java/org/apache/rocketmq/common/protocol/header/GetBrokerAclConfigResponseHeader.java @@ -25,6 +25,8 @@ public class GetBrokerAclConfigResponseHeader implements CommandCustomHeader { @CFNotNull private String version; + private String allAclFileVersion; + @CFNotNull private String brokerName; @@ -68,4 +70,12 @@ public String getClusterName() { public void setClusterName(String clusterName) { this.clusterName = clusterName; } + + public String getAllAclFileVersion() { + return allAclFileVersion; + } + + public void setAllAclFileVersion(String allAclFileVersion) { + this.allAclFileVersion = allAclFileVersion; + } } diff --git a/distribution/conf/plain_acl.yml b/distribution/conf/acl/plain_acl.yml similarity index 61% rename from distribution/conf/plain_acl.yml rename to distribution/conf/acl/plain_acl.yml index 5a44fbe3138..2435380d856 100644 --- a/distribution/conf/plain_acl.yml +++ b/distribution/conf/acl/plain_acl.yml @@ -14,29 +14,29 @@ # limitations under the License. globalWhiteRemoteAddresses: -- 10.10.103.* -- 192.168.0.* + - 10.10.103.* + - 192.168.0.* accounts: -- accessKey: RocketMQ - secretKey: 12345678 - whiteRemoteAddress: - admin: false - defaultTopicPerm: DENY - defaultGroupPerm: SUB - topicPerms: - - topicA=DENY - - topicB=PUB|SUB - - topicC=SUB - groupPerms: - # the group should convert to retry topic - - groupA=DENY - - groupB=PUB|SUB - - groupC=SUB + - accessKey: RocketMQ + secretKey: 12345678 + whiteRemoteAddress: + admin: false + defaultTopicPerm: DENY + defaultGroupPerm: SUB + topicPerms: + - topicA=DENY + - topicB=PUB|SUB + - topicC=SUB + groupPerms: + # the group should convert to retry topic + - groupA=DENY + - groupB=PUB|SUB + - groupC=SUB -- accessKey: rocketmq2 - secretKey: 12345678 - whiteRemoteAddress: 192.168.1.* - # if it is admin, it could access all resources - admin: true + - accessKey: rocketmq2 + secretKey: 12345678 + whiteRemoteAddress: 192.168.1.* + # if it is admin, it could access all resources + admin: true diff --git "a/docs/cn/acl/RocketMQ_Multiple_ACL_Files_\350\256\276\350\256\241.md" "b/docs/cn/acl/RocketMQ_Multiple_ACL_Files_\350\256\276\350\256\241.md" new file mode 100644 index 00000000000..9e11be554c7 --- /dev/null +++ "b/docs/cn/acl/RocketMQ_Multiple_ACL_Files_\350\256\276\350\256\241.md" @@ -0,0 +1,137 @@ +# Version记录 +| 时间 | 主要内容 | 作者 | +| --- | --- | --- | +| 2022-01-27 | 初版,包括需求背景、兼容性影响、重要业务逻辑和后续扩展性考虑 | sunxi92 | + +中文文档在描述特定专业术语时,仍然使用英文。 +# 需求背景 +RocketMQ ACL特性目前只支持单个ACL配置文件,当存在很多用户时该配置文件会非常大,因此提出支持多ACL配置文件的想法。 +如果支持该特性那么也方便对RocketMQ用户进行分类。 + +# 兼容性影响 +当前在支持多ACL配置文件特性的设计上是向前兼容的。 + +# 重要业务逻辑 +## 1. ACL配置文件存储路径 +ACL配置文件夹是在RocketMQ安装目录下的conf/acl目录中,也可以在该路径新建子目录并在子目录中新建ACL配置文件,同时也保留了之前默认的配置文件conf/plain_acl.yml。 +注意:目前用户还不能自定义配置文件目录。 +## 2. ACL配置文件更新 +热感知:当检测到ACL配置文件改动会自动刷新数据,判断ACL配置文件是否发生变化的依据是文件的修改时间是否发生变化 +## 3. RocketMQ Broker缓存ACL配置信息数据结构设计 +- aclPlainAccessResourceMap + +aclPlainAccessResourceMap是个Map类型,用来缓存所有ACL配置文件的权限数据,其中key表示ACL配置文件的绝对路径, +value表示相应配置文件中的权限数据,需要注意的是value也是一个Map类型,其中key是String类型表示AccessKey,value是PlainAccessResource类型。 +- accessKeyTable + +accessKeyTable是个Map类型,用来缓存AccessKey和ACL配置文件的映射关系,其中key表示AccessKey,value表示ACL配置文件的绝对路径。 +- globalWhiteRemoteAddressStrategy + +globalWhiteRemoteAddressStrategy用来缓存所有ACL配置文件的全局白名单。 +- globalWhiteRemoteAddressStrategyMap + +globalWhiteRemoteAddressStrategyMap是个Map类型,用来缓存ACL配置文件和全局白名单的映射关系 +- dataVersionMap + +dataVersionMap是个Map类型,用来缓存所有ACL配置文件的DataVersion,其中key表示ACL配置文件的绝对路径,value表示该配置文件对应的DataVersion。 +## 4.加载和监控ACL配置文件 +### 4.1 加载ACL配置文件 +- load() + +load()方法会获取"RocketMQ安装目录/conf"目录(包括该目录的子目录)和"rocketmq.acl.plain.file"下所有ACL配置文件,然后遍历这些文件读取权限数据和全局白名单。 +- load(String aclFilePath) + +load(String aclFilePath)方法完成加载指定ACL配置文件内容的功能,将配置文件中的全局白名单globalWhiteRemoteAddresses和用户权限accounts加载到缓存中, +这里需要注意以下几点: + +(1)判断缓存中该配置文件的全局白名单globalWhiteRemoteAddresses和用户权限accounts数据是否为空,如果不为空则需要注意删除文件原有数据 + +(2)相同的accessKey只允许存在在一个ACL配置文件中 +### 4.2 监控ACL配置文件 +watch()方法用来监控"RocketMQ安装目录/conf"目录下所有ACL配置文件和"rocketmq.acl.plain.file"是否发生变化,变化考虑两种情况:一种是ACL配置文件的数量发生变化, +此时会调用load()方法重新加载所有配置文件的数据;一种是配置文件的内容发生变化;具体完成监控ACL配置文件变化的是AclFileWatchService服务, +该服务是一个线程,当启动该服务后它会以WATCH_INTERVAL(该参数目前设置为5秒,目前还不能在Broker配置文件中设置)的时间间隔来执行其核心逻辑。在该服务中会记录其监控的ACL配置文件目录aclPath、 +ACL配置文件的数量aclFilesNum、所有ACL配置文件绝对路径fileList以及每个ACL配置文件最近一次修改的时间fileLastModifiedTime +(Map类型,key为ACL配置文件的绝对路径,value为其最近一次修改时间)。 +该服务的核心逻辑如下: +获取ACL配置文件数量并和aclFilesNum进行比较是否相等,如果不相等则更新aclFilesNum和fileList并调用load()方法重新加载所有配置文件; +如果相等则遍历每个ACL配置文件,获取其最近一次修改的时间,并将该时间与fileLastModifiedTime中记录的时间进行比较,如果不相等则表示该文件发生过修改, +此时调用load(String aclFilePath)方法重新加载该配置文件。 + +## 5. 权限数据相关操作修改 +(1) updateAclConfigFileVersion(Map updateAclConfigMap) + +添加对缓存dataVersionMap的修改 + +(2)updateAccessConfig(PlainAccessConfig plainAccessConfig) + +将该方法原有的逻辑修改为:首先判断accessKeyTable中是否包含待修改的accessKey,如果包含则根据accessKey来获取其对应的ACL配置文件绝对路径, +再根据该路径更新aclPlainAccessResourceMap中缓存的数据,最后将该ACL配置文件中的数据写回原文件;如果不包含则会将数据写到"rocketmq.acl.plain.file"配置文件中, +然后更新accessKeyTable和aclPlainAccessResourceMap,最后最后将该ACL配置文件中的数据写回原文件。 + +(3)deleteAccessConfig(String accesskey) + +将该方法原有的逻辑修改为:判断accessKeyTable中是否存在accesskey,如果不存在则返回false,否则将其删除并将修改后的数据写回原文件。 + +(4)getAllAclConfig() + +fileList中存储了所有ACL配置文件的绝对路径,遍历fileList分别从各ACL配置文件中读取数据并组装返回 + +(5)updateGlobalWhiteAddrsConfig(List globalWhiteAddrsList, String fileName) + +该方法是新增的,完成功能是修改指定ACL配置文件的全局白名单,为后续添加相关运维命令做准备 +## 6. ACL相关运维命令修改 +(1)ClusterAclConfigVersionListSubCommand + +将printClusterBaseInfo(final DefaultMQAdminExt defaultMQAdminExt, final String addr)方法原有的逻辑修改为: +获取全部的ACL配置文件的DataVersion并输出。注意:获取的全部ACL配置文件的DataVersion集合可能为空,这里需要添加判断 + +(2)GetBrokerAclConfigResponseHeader + +在GetBrokerAclConfigResponseHeader中新增allAclFileVersion字段,它是个Map类型,其key表示ACL配置文件的绝对路径,value表示对应ACL配置文件的DataVersion + +(3)ClusterAclVersionInfo + +在ClusterAclVersionInfo中废弃了aclConfigDataVersion属性,增加了allAclConfigDataVersion属性,该属性是个Map类型,用来存储所有ACL配置文件的版本数据, +其中key表示ACL配置文件的绝对路径,value表示对应ACL配置文件的DataVersion + +## 7. 关于ACL配置文件DataVersion存储修改 + +在原来版本中ACL权限数据存储在一个配置文件中,所以只记录了该配置文件的DataVersion,而现在需要支持多个配置文件特性,每个配置文件都有自己的DataVersion, +为了能够准确记录所有配置文件的DataVersion,需要调整相关类型的属性、接口及方法。 + +(1)PlainPermissionManager + +对PlainPermissionManager属性的修改具体如下: + +- 废弃dataVersion属性,该属性在历史版本中是用来存来存储默认ACL配置文件的DataVersion + +- 新增dataVersionMap属性用来缓存所有ACL配置文件的DataVersion,它是一个Map类型,其key表示ACL配置文件的绝对路径,value表示对应配置文件的DataVersion + +(2)AccessValidator + +对AccessValidator的修改如下: + +- 废弃String getAclConfigVersion();,该接口原来是获取ACL配置文件文件的版本数据 + +- 新增Map getAllAclConfigVersion();该接口是用来获取所有ACL配置文件的版本数据,接口会返回一个Map类型数据, +key表示各ACL配置文件的绝对路径,value表示对应配置文件的版本数据 + +(3)PlainAccessValidator + +由于PlainAccessValidator实现了AccessValidator接口,所以相应地增加了getAllAclConfigVersion()方法 + +# 后续扩展性考虑 +1.目前的修改只支持ACL配置文件存储在"RocketMQ安装目录/conf"目录下,后续可以考虑支持多目录; + +2.目前ACL配置文件路径是不支持让用户指定,后续可以考虑让用户指定指定ACL配置文件的存储路径 + +3.当前updateGlobalWhiteAddrsConfig命令只支持修改"rocketmq.acl.plain.file"文件中全局白名单, +后续可以扩展为修改指定ACL配置文件的全局白名单(如果参数中没有传ACL配置文件则会修改"rocketmq.acl.plain.file"文件) + +4.目前ACL数据中的secretKey是以明文形式存储在文件中,在一些对此类信息敏感的行业是不允许以明文落地,后续可以考虑安全性问题 + +5.目前ACL数据存储只支持文件形式存储,后续可以考虑增加数据库存储 + + + diff --git a/srvutil/src/main/java/org/apache/rocketmq/srvutil/AclFileWatchService.java b/srvutil/src/main/java/org/apache/rocketmq/srvutil/AclFileWatchService.java new file mode 100644 index 00000000000..e6fe5b3588f --- /dev/null +++ b/srvutil/src/main/java/org/apache/rocketmq/srvutil/AclFileWatchService.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.srvutil; + +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.security.MessageDigest; +import java.util.HashMap; +import java.util.Map; + +public class AclFileWatchService extends ServiceThread { + private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME); + + private final String aclPath; + private int aclFilesNum; + @Deprecated + private final Map fileCurrentHash; + private Map fileLastModifiedTime; + private List fileList = new ArrayList<>(); + private final AclFileWatchService.Listener listener; + private static final int WATCH_INTERVAL = 5000; + private MessageDigest md = MessageDigest.getInstance("MD5"); + private String defaultAclFile; + + public AclFileWatchService(String path, String defaultAclFile, final AclFileWatchService.Listener listener) throws Exception { + this.aclPath = path; + this.defaultAclFile = defaultAclFile; + this.fileCurrentHash = new HashMap<>(); + this.fileLastModifiedTime = new HashMap<>(); + this.listener = listener; + + getAllAclFiles(path); + if (new File(this.defaultAclFile).exists() && !fileList.contains(this.defaultAclFile)) { + fileList.add(this.defaultAclFile); + } + this.aclFilesNum = fileList.size(); + for (int i = 0; i < aclFilesNum; i++) { + String fileAbsolutePath = fileList.get(i); + this.fileLastModifiedTime.put(fileAbsolutePath, new File(fileAbsolutePath).lastModified()); + } + + } + + public void getAllAclFiles(String path) { + File file = new File(path); + if (!file.exists()) { + log.info("The default acl dir {} is not exist", path); + return; + } + File[] files = file.listFiles(); + for (int i = 0; i < files.length; i++) { + String fileName = files[i].getAbsolutePath(); + File f = new File(fileName); + if (fileName.equals(aclPath + File.separator + "tools.yml")) { + continue; + } else if (fileName.endsWith(".yml") || fileName.endsWith(".yaml")) { + fileList.add(fileName); + } else if (f.isDirectory()) { + getAllAclFiles(fileName); + } + } + } + + @Override + public String getServiceName() { + return "AclFileWatchService"; + } + + @Override + public void run() { + log.info(this.getServiceName() + " service started"); + + while (!this.isStopped()) { + try { + this.waitForRunning(WATCH_INTERVAL); + + if (fileList.size() > 0) { + fileList.clear(); + } + getAllAclFiles(aclPath); + if (new File(defaultAclFile).exists() && !fileList.contains(defaultAclFile)) { + fileList.add(defaultAclFile); + } + int realAclFilesNum = fileList.size(); + + if (aclFilesNum != realAclFilesNum) { + log.info("aclFilesNum: " + aclFilesNum + " realAclFilesNum: " + realAclFilesNum); + aclFilesNum = realAclFilesNum; + log.info("aclFilesNum: " + aclFilesNum + " realAclFilesNum: " + realAclFilesNum); + Map fileLastModifiedTime = new HashMap<>(realAclFilesNum); + for (int i = 0; i < realAclFilesNum; i++) { + String fileAbsolutePath = fileList.get(i); + fileLastModifiedTime.put(fileAbsolutePath, new File(fileAbsolutePath).lastModified()); + } + this.fileLastModifiedTime = fileLastModifiedTime; + listener.onFileNumChanged(aclPath); + } else { + for (int i = 0; i < aclFilesNum; i++) { + String fileName = fileList.get(i); + Long newLastModifiedTime = new File(fileName).lastModified(); + if (!newLastModifiedTime.equals(fileLastModifiedTime.get(fileName))) { + fileLastModifiedTime.put(fileName, newLastModifiedTime); + listener.onFileChanged(fileName); + } + } + } + } catch (Exception e) { + log.warn(this.getServiceName() + " service has exception. ", e); + } + } + log.info(this.getServiceName() + " service end"); + } + + @Deprecated + private String hash(String filePath) throws IOException { + Path path = Paths.get(filePath); + md.update(Files.readAllBytes(path)); + byte[] hash = md.digest(); + return UtilAll.bytes2string(hash); + } + + public interface Listener { + /** + * Will be called when the target file is changed + * + * @param aclFileName the changed file absolute path + */ + void onFileChanged(String aclFileName); + + /** + * Will be called when the number of the acl file is changed + * + * @param path the path of the acl dir + */ + void onFileNumChanged(String path); + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java index c1e86fbd1ad..f474e5ccc54 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java @@ -19,6 +19,7 @@ import java.sql.Timestamp; import java.text.DateFormat; import java.text.SimpleDateFormat; +import java.util.Map; import java.util.Set; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; @@ -58,7 +59,7 @@ public class ClusterAclConfigVersionListSubCommand implements SubCommand { optionGroup.setRequired(true); options.addOptionGroup(optionGroup); - + return options; } @@ -85,10 +86,11 @@ public class ClusterAclConfigVersionListSubCommand implements SubCommand { Set masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); - System.out.printf("%-16s %-22s %-22s %-20s %-22s%n", + System.out.printf("%-16s %-22s %-22s %-20s %-22s %-22s%n", "#Cluster Name", "#Broker Name", "#Broker Addr", + "#AclFilePath", "#AclConfigVersionNum", "#AclLastUpdateTime" ); @@ -112,20 +114,20 @@ private void printClusterBaseInfo( final DefaultMQAdminExt defaultMQAdminExt, final String addr) throws InterruptedException, MQBrokerException, RemotingException, MQClientException { - ClusterAclVersionInfo clusterAclVersionInfo = defaultMQAdminExt.examineBrokerClusterAclVersionInfo(addr); - DataVersion aclDataVersion = clusterAclVersionInfo.getAclConfigDataVersion(); - String versionNum = String.valueOf(aclDataVersion.getCounter()); - + Map aclDataVersion = clusterAclVersionInfo.getAllAclConfigDataVersion(); DateFormat sdf = new SimpleDateFormat(UtilAll.YYYY_MM_DD_HH_MM_SS); - String timeStampStr = sdf.format(new Timestamp(aclDataVersion.getTimestamp())); - - System.out.printf("%-16s %-22s %-22s %-20s %-22s%n", - clusterAclVersionInfo.getClusterName(), - clusterAclVersionInfo.getBrokerName(), - clusterAclVersionInfo.getBrokerAddr(), - versionNum, - timeStampStr - ); + if (aclDataVersion.size() > 0) { + for (Map.Entry entry : aclDataVersion.entrySet()) { + System.out.printf("%-16s %-22s %-22s %-20s %-22s %-22s%n", + clusterAclVersionInfo.getClusterName(), + clusterAclVersionInfo.getBrokerName(), + clusterAclVersionInfo.getBrokerAddr(), + entry.getKey(), + String.valueOf(entry.getValue().getCounter()), + sdf.format(new Timestamp(entry.getValue().getTimestamp())) + ); + } + } } } From 2369a9152d1accf34adc30f729a92e12a1429c61 Mon Sep 17 00:00:00 2001 From: Rishi Kumar Ray <87641376+RishiKumarRay@users.noreply.github.com> Date: Sun, 20 Feb 2022 13:02:20 +0530 Subject: [PATCH 138/141] depreciated unused class (#3846) * depreciated MQhelper * Update MQHelper.java --- client/src/main/java/org/apache/rocketmq/client/MQHelper.java | 1 + 1 file changed, 1 insertion(+) diff --git a/client/src/main/java/org/apache/rocketmq/client/MQHelper.java b/client/src/main/java/org/apache/rocketmq/client/MQHelper.java index c3378e4b309..cfd0b7ee1cb 100644 --- a/client/src/main/java/org/apache/rocketmq/client/MQHelper.java +++ b/client/src/main/java/org/apache/rocketmq/client/MQHelper.java @@ -25,6 +25,7 @@ import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; public class MQHelper { + @Deprecated public static void resetOffsetByTimestamp( final MessageModel messageModel, final String consumerGroup, From 35c95eb7a67820f25de8471e50e9988c85a8dc04 Mon Sep 17 00:00:00 2001 From: tigerlee Date: Mon, 21 Feb 2022 22:11:04 +0800 Subject: [PATCH 139/141] prepare to release rocketmq 4.9.3 version --- common/src/main/java/org/apache/rocketmq/common/MQVersion.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/java/org/apache/rocketmq/common/MQVersion.java b/common/src/main/java/org/apache/rocketmq/common/MQVersion.java index bb30e9a5d44..3cf24e33a3b 100644 --- a/common/src/main/java/org/apache/rocketmq/common/MQVersion.java +++ b/common/src/main/java/org/apache/rocketmq/common/MQVersion.java @@ -18,7 +18,7 @@ public class MQVersion { - public static final int CURRENT_VERSION = Version.V4_9_2.ordinal(); + public static final int CURRENT_VERSION = Version.V4_9_3.ordinal(); public static String getVersionDesc(int value) { int length = Version.values().length; From c7989f85744e65e24b43955c88987288487b4707 Mon Sep 17 00:00:00 2001 From: tigerlee Date: Tue, 22 Feb 2022 00:25:37 +0800 Subject: [PATCH 140/141] [maven-release-plugin] prepare release rocketmq-all-4.9.3 --- acl/pom.xml | 2 +- broker/pom.xml | 2 +- client/pom.xml | 2 +- common/pom.xml | 2 +- distribution/pom.xml | 2 +- example/pom.xml | 2 +- filter/pom.xml | 2 +- logging/pom.xml | 2 +- namesrv/pom.xml | 2 +- openmessaging/pom.xml | 2 +- pom.xml | 4 ++-- remoting/pom.xml | 2 +- srvutil/pom.xml | 2 +- store/pom.xml | 2 +- test/pom.xml | 2 +- tools/pom.xml | 2 +- 16 files changed, 17 insertions(+), 17 deletions(-) diff --git a/acl/pom.xml b/acl/pom.xml index bed3c46340d..d78cfa993f3 100644 --- a/acl/pom.xml +++ b/acl/pom.xml @@ -13,7 +13,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 rocketmq-acl rocketmq-acl ${project.version} diff --git a/broker/pom.xml b/broker/pom.xml index d36ad8b73ef..7bb49560092 100644 --- a/broker/pom.xml +++ b/broker/pom.xml @@ -13,7 +13,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/client/pom.xml b/client/pom.xml index 4f6f1959346..576bb3eec8a 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/common/pom.xml b/common/pom.xml index d8274d55b61..cbbedab252b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/distribution/pom.xml b/distribution/pom.xml index df5ddc4647a..b2038c632e1 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -20,7 +20,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 rocketmq-distribution rocketmq-distribution ${project.version} diff --git a/example/pom.xml b/example/pom.xml index 5c254828504..727b0d76b18 100644 --- a/example/pom.xml +++ b/example/pom.xml @@ -19,7 +19,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/filter/pom.xml b/filter/pom.xml index 3f7ad3da99a..147955c25af 100644 --- a/filter/pom.xml +++ b/filter/pom.xml @@ -20,7 +20,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/logging/pom.xml b/logging/pom.xml index 268e0173511..fdd004def6c 100644 --- a/logging/pom.xml +++ b/logging/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/namesrv/pom.xml b/namesrv/pom.xml index 8f650c8da29..b854050a54c 100644 --- a/namesrv/pom.xml +++ b/namesrv/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/openmessaging/pom.xml b/openmessaging/pom.xml index 0882c7f5c49..242c92a25f4 100644 --- a/openmessaging/pom.xml +++ b/openmessaging/pom.xml @@ -20,7 +20,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/pom.xml b/pom.xml index 7932472039f..4033f806f60 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ 2012 org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 pom Apache RocketMQ ${project.version} http://rocketmq.apache.org/ @@ -38,7 +38,7 @@ git@github.com:apache/rocketmq.git scm:git:git@github.com:apache/rocketmq.git scm:git:git@github.com:apache/rocketmq.git - HEAD + rocketmq-all-4.9.3 diff --git a/remoting/pom.xml b/remoting/pom.xml index a0e00ec0a4a..598b87d92f4 100644 --- a/remoting/pom.xml +++ b/remoting/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/srvutil/pom.xml b/srvutil/pom.xml index e9e755b0700..96dd92dd3ca 100644 --- a/srvutil/pom.xml +++ b/srvutil/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/store/pom.xml b/store/pom.xml index 46aee036231..506ae0431a6 100644 --- a/store/pom.xml +++ b/store/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/test/pom.xml b/test/pom.xml index 9d8336e050d..a9d9aa84ec1 100644 --- a/test/pom.xml +++ b/test/pom.xml @@ -20,7 +20,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 diff --git a/tools/pom.xml b/tools/pom.xml index 2bd2a8b5ac4..e1df8b2ad32 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3-SNAPSHOT + 4.9.3 4.0.0 From b606d9c638c2b9bf018d9a9968fdc9e5d6acf07f Mon Sep 17 00:00:00 2001 From: tigerlee Date: Tue, 22 Feb 2022 00:25:55 +0800 Subject: [PATCH 141/141] [maven-release-plugin] prepare for next development iteration --- acl/pom.xml | 2 +- broker/pom.xml | 2 +- client/pom.xml | 2 +- common/pom.xml | 2 +- distribution/pom.xml | 2 +- example/pom.xml | 2 +- filter/pom.xml | 2 +- logging/pom.xml | 2 +- namesrv/pom.xml | 2 +- openmessaging/pom.xml | 2 +- pom.xml | 4 ++-- remoting/pom.xml | 2 +- srvutil/pom.xml | 2 +- store/pom.xml | 2 +- test/pom.xml | 2 +- tools/pom.xml | 2 +- 16 files changed, 17 insertions(+), 17 deletions(-) diff --git a/acl/pom.xml b/acl/pom.xml index d78cfa993f3..2c5cac2d9e0 100644 --- a/acl/pom.xml +++ b/acl/pom.xml @@ -13,7 +13,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT rocketmq-acl rocketmq-acl ${project.version} diff --git a/broker/pom.xml b/broker/pom.xml index 7bb49560092..fee6d9ea2c8 100644 --- a/broker/pom.xml +++ b/broker/pom.xml @@ -13,7 +13,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/client/pom.xml b/client/pom.xml index 576bb3eec8a..51e53194f93 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/common/pom.xml b/common/pom.xml index cbbedab252b..610be62347a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/distribution/pom.xml b/distribution/pom.xml index b2038c632e1..da6e83ca5df 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -20,7 +20,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT rocketmq-distribution rocketmq-distribution ${project.version} diff --git a/example/pom.xml b/example/pom.xml index 727b0d76b18..2de90626914 100644 --- a/example/pom.xml +++ b/example/pom.xml @@ -19,7 +19,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/filter/pom.xml b/filter/pom.xml index 147955c25af..80c59e89542 100644 --- a/filter/pom.xml +++ b/filter/pom.xml @@ -20,7 +20,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/logging/pom.xml b/logging/pom.xml index fdd004def6c..fbc4b1e002b 100644 --- a/logging/pom.xml +++ b/logging/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/namesrv/pom.xml b/namesrv/pom.xml index b854050a54c..7ecbcd3af6a 100644 --- a/namesrv/pom.xml +++ b/namesrv/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/openmessaging/pom.xml b/openmessaging/pom.xml index 242c92a25f4..23d28713df7 100644 --- a/openmessaging/pom.xml +++ b/openmessaging/pom.xml @@ -20,7 +20,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index 4033f806f60..ec051303dd2 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ 2012 org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT pom Apache RocketMQ ${project.version} http://rocketmq.apache.org/ @@ -38,7 +38,7 @@ git@github.com:apache/rocketmq.git scm:git:git@github.com:apache/rocketmq.git scm:git:git@github.com:apache/rocketmq.git - rocketmq-all-4.9.3 + HEAD diff --git a/remoting/pom.xml b/remoting/pom.xml index 598b87d92f4..32320e6d050 100644 --- a/remoting/pom.xml +++ b/remoting/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/srvutil/pom.xml b/srvutil/pom.xml index 96dd92dd3ca..f49cd6fd131 100644 --- a/srvutil/pom.xml +++ b/srvutil/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/store/pom.xml b/store/pom.xml index 506ae0431a6..c583057fd6e 100644 --- a/store/pom.xml +++ b/store/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/test/pom.xml b/test/pom.xml index a9d9aa84ec1..505e6c65034 100644 --- a/test/pom.xml +++ b/test/pom.xml @@ -20,7 +20,7 @@ rocketmq-all org.apache.rocketmq - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0 diff --git a/tools/pom.xml b/tools/pom.xml index e1df8b2ad32..f5164c0e15f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -19,7 +19,7 @@ org.apache.rocketmq rocketmq-all - 4.9.3 + 4.9.4-SNAPSHOT 4.0.0