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 87c01a5b3055470be8d1772457afde58d9e55a95..c3e4efa252c1c8fb4e3d9b2d0214e2b5b965d736 100644 --- a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java +++ b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java @@ -16,7 +16,9 @@ */ package org.apache.rocketmq.client; +import java.util.Collection; import java.util.HashSet; +import java.util.Iterator; import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.UtilAll; @@ -95,7 +97,6 @@ public class ClientConfig { } } - public String withNamespace(String resource) { return NamespaceUtil.wrapNamespace(this.getNamespace(), resource); } @@ -124,9 +125,21 @@ public class ClientConfig { if (StringUtils.isEmpty(this.getNamespace())) { return queue; } - return new MessageQueue(withNamespace(queue.getTopic()), queue.getBrokerName(), queue.getQueueId()); } + + public Collection queuesWithNamespace(Collection queues) { + if (StringUtils.isEmpty(this.getNamespace())) { + return queues; + } + Iterator iter = queues.iterator(); + while (iter.hasNext()) { + MessageQueue queue = iter.next(); + queue.setTopic(withNamespace(queue.getTopic())); + } + return queues; + } + public void resetClientConfig(final ClientConfig cc) { this.namesrvAddr = cc.namesrvAddr; this.clientIP = cc.clientIP; @@ -170,6 +183,7 @@ public class ClientConfig { /** * Domain name mode access way does not support the delimiter(;), and only one domain name can be set. + * * @param namesrvAddr name server address */ public void setNamesrvAddr(String namesrvAddr) { 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 5d6acc0f872bf645ecc28cc1665ab94c8c34725d..1b96cd058ff129dc01d1d18aff44c0f0a6e8da1c 100644 --- a/client/src/main/java/org/apache/rocketmq/client/Validators.java +++ b/client/src/main/java/org/apache/rocketmq/client/Validators.java @@ -53,14 +53,17 @@ public class Validators { if (UtilAll.isBlank(group)) { throw new MQClientException("the specified group is blank", null); } + + if (group.length() > CHARACTER_MAX_LENGTH) { + 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); } - if (group.length() > CHARACTER_MAX_LENGTH) { - throw new MQClientException("the specified group is longer than group max length 255.", null); - } + } /** diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumer.java new file mode 100644 index 0000000000000000000000000000000000000000..99976d55d3ad90bd5514008b201482ca726458fd --- /dev/null +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumer.java @@ -0,0 +1,414 @@ +/* + * 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; + +import java.util.Collection; +import java.util.List; + +import org.apache.rocketmq.client.ClientConfig; +import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely; +import org.apache.rocketmq.client.consumer.store.OffsetStore; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.impl.consumer.DefaultLitePullConsumerImpl; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; +import org.apache.rocketmq.remoting.RPCHook; + +public class DefaultLitePullConsumer extends ClientConfig implements LitePullConsumer { + + private final DefaultLitePullConsumerImpl defaultLitePullConsumerImpl; + + /** + * Consumers belonging to the same consumer group share a group id. The consumers in a group then + * divides the topic as fairly amongst themselves as possible by establishing that each queue is only + * consumed by a single consumer from the group. If all consumers are from the same group, it functions + * as a traditional message queue. Each message would be consumed by one consumer of the group only. + * When multiple consumer groups exist, the flow of the data consumption model aligns with the traditional + * publish-subscribe model. The messages are broadcast to all consumer groups. + */ + private String consumerGroup; + + /** + * Long polling mode, the Consumer connection max suspend time, it is not recommended to modify + */ + private long brokerSuspendMaxTimeMillis = 1000 * 20; + + /** + * Long polling mode, the Consumer connection timeout(must greater than brokerSuspendMaxTimeMillis), it is not + * recommended to modify + */ + private long consumerTimeoutMillisWhenSuspend = 1000 * 30; + + /** + * The socket timeout in milliseconds + */ + private long consumerPullTimeoutMillis = 1000 * 10; + + /** + * Consumption pattern,default is clustering + */ + private MessageModel messageModel = MessageModel.CLUSTERING; + /** + * Message queue listener + */ + private MessageQueueListener messageQueueListener; + /** + * Offset Storage + */ + private OffsetStore offsetStore; + + /** + * Queue allocation algorithm + */ + private AllocateMessageQueueStrategy allocateMessageQueueStrategy = new AllocateMessageQueueAveragely(); + /** + * Whether the unit of subscription group + */ + private boolean unitMode = false; + + /** + * The flag for auto commit offset + */ + private boolean autoCommit = true; + + /** + * Pull thread number + */ + private int pullThreadNums = 20; + + /** + * Maximum commit offset interval time in milliseconds. + */ + private long autoCommitIntervalMillis = 5 * 1000; + + /** + * Maximum number of messages pulled each time. + */ + private int pullBatchSize = 10; + + /** + * Flow control threshold for consume request, each consumer will cache at most 10000 consume requests by default. + * Consider the {@code pullBatchSize}, the instantaneous value may exceed the limit + */ + private long pullThresholdForAll = 10000; + + /** + * Consume max span offset. + */ + private int consumeMaxSpan = 2000; + + /** + * Flow control threshold on queue level, each message queue will cache at most 1000 messages by default, Consider + * the {@code pullBatchSize}, the instantaneous value may exceed the limit + */ + private int pullThresholdForQueue = 1000; + + /** + * Limit the cached message size on queue level, each message queue will cache at most 100 MiB messages by default, + * Consider the {@code pullBatchSize}, the instantaneous value may exceed the limit + * + *

+ * The size of a message only measured by message body, so it's not accurate + */ + private int pullThresholdSizeForQueue = 100; + + /** + * The poll timeout in milliseconds + */ + private long pollTimeoutMillis = 1000 * 5; + + /** + * Interval time in in milliseconds for checking changes in topic metadata. + */ + private long topicMetadataCheckIntervalMillis = 30 * 1000; + + /** + * Default constructor. + */ + public DefaultLitePullConsumer() { + this(null, MixAll.DEFAULT_CONSUMER_GROUP, null); + } + + /** + * Constructor specifying consumer group. + * + * @param consumerGroup Consumer group. + */ + public DefaultLitePullConsumer(final String consumerGroup) { + this(null, consumerGroup, null); + } + + /** + * Constructor specifying RPC hook. + * + * @param rpcHook RPC hook to execute before each remoting command. + */ + public DefaultLitePullConsumer(RPCHook rpcHook) { + this(null, MixAll.DEFAULT_CONSUMER_GROUP, rpcHook); + } + + /** + * Constructor specifying consumer group, RPC hook + * + * @param consumerGroup Consumer group. + * @param rpcHook RPC hook to execute before each remoting command. + */ + public DefaultLitePullConsumer(final String consumerGroup, RPCHook rpcHook) { + this(null, consumerGroup, rpcHook); + } + + /** + * Constructor specifying namespace, consumer group and RPC hook. + * + * @param consumerGroup Consumer group. + * @param rpcHook RPC hook to execute before each remoting command. + */ + public DefaultLitePullConsumer(final String namespace, final String consumerGroup, RPCHook rpcHook) { + this.namespace = namespace; + this.consumerGroup = consumerGroup; + defaultLitePullConsumerImpl = new DefaultLitePullConsumerImpl(this, rpcHook); + } + + @Override + public void start() throws MQClientException { + this.defaultLitePullConsumerImpl.start(); + } + + @Override + public void shutdown() { + this.defaultLitePullConsumerImpl.shutdown(); + } + + @Override + public void subscribe(String topic, String subExpression) throws MQClientException { + this.defaultLitePullConsumerImpl.subscribe(withNamespace(topic), subExpression); + } + + @Override + public void subscribe(String topic, MessageSelector messageSelector) throws MQClientException { + this.defaultLitePullConsumerImpl.subscribe(withNamespace(topic), messageSelector); + } + + @Override + public void unsubscribe(String topic) { + this.defaultLitePullConsumerImpl.unsubscribe(withNamespace(topic)); + } + + @Override + public void assign(Collection messageQueues) { + defaultLitePullConsumerImpl.assign(queuesWithNamespace(messageQueues)); + } + + @Override + public List poll() { + return defaultLitePullConsumerImpl.poll(this.getPollTimeoutMillis()); + } + + @Override + public List poll(long timeout) { + return defaultLitePullConsumerImpl.poll(timeout); + } + + @Override + public void seek(MessageQueue messageQueue, long offset) throws MQClientException { + this.defaultLitePullConsumerImpl.seek(queueWithNamespace(messageQueue), offset); + } + + @Override + public void pause(Collection messageQueues) { + this.defaultLitePullConsumerImpl.pause(queuesWithNamespace(messageQueues)); + } + + @Override + public void resume(Collection messageQueues) { + this.defaultLitePullConsumerImpl.resume(queuesWithNamespace(messageQueues)); + } + + @Override + public Collection fetchMessageQueues(String topic) throws MQClientException { + return this.defaultLitePullConsumerImpl.fetchMessageQueues(withNamespace(topic)); + } + + @Override + public Long offsetForTimestamp(MessageQueue messageQueue, Long timestamp) throws MQClientException { + return this.defaultLitePullConsumerImpl.searchOffset(queueWithNamespace(messageQueue), timestamp); + } + + @Override + public void registerTopicMessageQueueChangeListener(String topic, + TopicMessageQueueChangeListener topicMessageQueueChangeListener) throws MQClientException { + this.defaultLitePullConsumerImpl.registerTopicMessageQueueChangeListener(withNamespace(topic), topicMessageQueueChangeListener); + } + + @Override + public void commitSync() { + this.defaultLitePullConsumerImpl.commitSync(); + } + + @Override + public Long committed(MessageQueue messageQueue) throws MQClientException { + return this.defaultLitePullConsumerImpl.committed(messageQueue); + } + + @Override + public boolean isAutoCommit() { + return autoCommit; + } + + @Override + public void setAutoCommit(boolean autoCommit) { + this.autoCommit = autoCommit; + } + + public int getPullThreadNums() { + return pullThreadNums; + } + + public void setPullThreadNums(int pullThreadNums) { + this.pullThreadNums = pullThreadNums; + } + + public long getAutoCommitIntervalMillis() { + return autoCommitIntervalMillis; + } + + public void setAutoCommitIntervalMillis(long autoCommitIntervalMillis) { + this.autoCommitIntervalMillis = autoCommitIntervalMillis; + } + + public int getPullBatchSize() { + return pullBatchSize; + } + + public void setPullBatchSize(int pullBatchSize) { + this.pullBatchSize = pullBatchSize; + } + + public long getPullThresholdForAll() { + return pullThresholdForAll; + } + + public void setPullThresholdForAll(long pullThresholdForAll) { + this.pullThresholdForAll = pullThresholdForAll; + } + + public int getConsumeMaxSpan() { + return consumeMaxSpan; + } + + public void setConsumeMaxSpan(int consumeMaxSpan) { + this.consumeMaxSpan = consumeMaxSpan; + } + + public int getPullThresholdForQueue() { + return pullThresholdForQueue; + } + + public void setPullThresholdForQueue(int pullThresholdForQueue) { + this.pullThresholdForQueue = pullThresholdForQueue; + } + + public int getPullThresholdSizeForQueue() { + return pullThresholdSizeForQueue; + } + + public void setPullThresholdSizeForQueue(int pullThresholdSizeForQueue) { + this.pullThresholdSizeForQueue = pullThresholdSizeForQueue; + } + + public AllocateMessageQueueStrategy getAllocateMessageQueueStrategy() { + return allocateMessageQueueStrategy; + } + + public void setAllocateMessageQueueStrategy(AllocateMessageQueueStrategy allocateMessageQueueStrategy) { + this.allocateMessageQueueStrategy = allocateMessageQueueStrategy; + } + + public long getBrokerSuspendMaxTimeMillis() { + return brokerSuspendMaxTimeMillis; + } + + public long getPollTimeoutMillis() { + return pollTimeoutMillis; + } + + public void setPollTimeoutMillis(long pollTimeoutMillis) { + this.pollTimeoutMillis = pollTimeoutMillis; + } + + public OffsetStore getOffsetStore() { + return offsetStore; + } + + public void setOffsetStore(OffsetStore offsetStore) { + this.offsetStore = offsetStore; + } + + public boolean isUnitMode() { + return unitMode; + } + + public void setUnitMode(boolean isUnitMode) { + this.unitMode = isUnitMode; + } + + public MessageModel getMessageModel() { + return messageModel; + } + + public void setMessageModel(MessageModel messageModel) { + this.messageModel = messageModel; + } + + public String getConsumerGroup() { + return consumerGroup; + } + + public MessageQueueListener getMessageQueueListener() { + return messageQueueListener; + } + + public void setMessageQueueListener(MessageQueueListener messageQueueListener) { + this.messageQueueListener = messageQueueListener; + } + + public long getConsumerPullTimeoutMillis() { + return consumerPullTimeoutMillis; + } + + public void setConsumerPullTimeoutMillis(long consumerPullTimeoutMillis) { + this.consumerPullTimeoutMillis = consumerPullTimeoutMillis; + } + + public long getConsumerTimeoutMillisWhenSuspend() { + return consumerTimeoutMillisWhenSuspend; + } + + public void setConsumerTimeoutMillisWhenSuspend(long consumerTimeoutMillisWhenSuspend) { + this.consumerTimeoutMillisWhenSuspend = consumerTimeoutMillisWhenSuspend; + } + + public long getTopicMetadataCheckIntervalMillis() { + return topicMetadataCheckIntervalMillis; + } + + public void setTopicMetadataCheckIntervalMillis(long topicMetadataCheckIntervalMillis) { + this.topicMetadataCheckIntervalMillis = topicMetadataCheckIntervalMillis; + } +} 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 17e60080e0c9a8548db716679a6c90e8eb179b55..9e291b563858e4af4217d9e3a26467bb67fc3b68 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 @@ -38,9 +38,13 @@ import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.exception.RemotingException; /** - * Default pulling consumer + * 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. */ +@Deprecated public class DefaultMQPullConsumer extends ClientConfig implements MQPullConsumer { + protected final transient DefaultMQPullConsumerImpl defaultMQPullConsumerImpl; /** diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/LitePullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/LitePullConsumer.java new file mode 100644 index 0000000000000000000000000000000000000000..d6e657ff18cf491b5893fae0da127fe90f19aa75 --- /dev/null +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/LitePullConsumer.java @@ -0,0 +1,175 @@ +/* + * 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; + +import java.util.Collection; +import java.util.List; + +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; + +public interface LitePullConsumer { + + /** + * Start the consumer + */ + void start() throws MQClientException; + + /** + * Shutdown the consumer + */ + void shutdown(); + + /** + * Subscribe some topic with subExpression + * + * @param subExpression subscription expression.it only support or operation such as "tag1 || tag2 || tag3"
if + * null or * expression,meaning subscribe all + * @throws MQClientException if there is any client error. + */ + void subscribe(final String topic, final String subExpression) throws MQClientException; + + /** + * Subscribe some topic with selector. + * + * @param selector message selector({@link MessageSelector}), can be null. + * @throws MQClientException if there is any client error. + */ + void subscribe(final String topic, final MessageSelector selector) throws MQClientException; + + /** + * Unsubscribe consumption some topic + * + * @param topic Message topic that needs to be unsubscribe. + */ + void unsubscribe(final String topic); + + /** + * Manually assign a list of message queues to this consumer. This interface does not allow for incremental + * assignment and will replace the previous assignment (if there is one). + * + * @param messageQueues Message queues that needs to be assigned. + */ + void assign(Collection messageQueues); + + /** + * Fetch data for the topics or partitions specified using assign API + * + * @return list of message, can be null. + */ + List poll(); + + /** + * Fetch data for the topics or partitions specified using assign API + * + * @param timeout The amount time, in milliseconds, spent waiting in poll if data is not available. Must not be + * negative + * @return list of message, can be null. + */ + List poll(long timeout); + + /** + * Overrides the fetch offsets that the consumer will use on the next poll. If this API is invoked for the same + * message queue more than once, the latest offset will be used on the next poll(). Note that you may lose data if + * this API is arbitrarily used in the middle of consumption. + * + * @param messageQueue + * @param offset + */ + void seek(MessageQueue messageQueue, long offset) throws MQClientException; + + /** + * Suspend pulling from the requested message queues. + * + * Because of the implementation of pre-pull, fetch data in {@link #poll()} will not stop immediately until the + * messages of the requested message queues drain. + * + * Note that this method does not affect message queue subscription. In particular, it does not cause a group + * rebalance. + * + * @param messageQueues Message queues that needs to be paused. + */ + void pause(Collection messageQueues); + + /** + * Resume specified message queues which have been paused with {@link #pause(Collection)}. + * + * @param messageQueues Message queues that needs to be resumed. + */ + void resume(Collection messageQueues); + + /** + * Whether to enable auto-commit consume offset. + * + * @return true if enable auto-commit, false if disable auto-commit. + */ + boolean isAutoCommit(); + + /** + * Set whether to enable auto-commit consume offset. + * + * @param autoCommit Whether to enable auto-commit. + */ + void setAutoCommit(boolean autoCommit); + + /** + * Get metadata about the message queues for a given topic. + * + * @param topic The topic that need to get metadata. + * @return collection of message queues + * @throws MQClientException if there is any client error. + */ + Collection fetchMessageQueues(String topic) throws MQClientException; + + /** + * Look up the offsets for the given message queue by timestamp. The returned offset for each message queue is the + * earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding message + * queue. + * + * @param messageQueue Message queues that needs to get offset by timestamp. + * @param timestamp + * @return offset + * @throws MQClientException if there is any client error. + */ + Long offsetForTimestamp(MessageQueue messageQueue, Long timestamp) throws MQClientException; + + /** + * Manually commit consume offset. + */ + void commitSync(); + + /** + * Get the last committed offset for the given message queue. + * + * @param messageQueue + * @return offset, if offset equals -1 means no offset in broker. + * @throws MQClientException if there is any client error. + */ + Long committed(MessageQueue messageQueue) throws MQClientException; + + /** + * Register a callback for sensing topic metadata changes. + * + * @param topic The topic that need to monitor. + * @param topicMessageQueueChangeListener Callback when topic metadata changes, refer {@link + * TopicMessageQueueChangeListener} + * @throws MQClientException if there is any client error. + */ + void registerTopicMessageQueueChangeListener(String topic, + TopicMessageQueueChangeListener topicMessageQueueChangeListener) throws MQClientException; +} diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java index 28b807c2ed89ac460d9cfd0e61a35dc144c42242..a8e96283f9d06c11a29197d1f62f0c494abb5d56 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java @@ -169,4 +169,5 @@ public interface MQPullConsumer extends MQConsumer { */ void sendMessageBack(MessageExt msg, int delayLevel, String brokerName, String consumerGroup) throws RemotingException, MQBrokerException, InterruptedException, MQClientException; + } diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumerScheduleService.java b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumerScheduleService.java index 5436688052cc52634782d5f4f344933470159973..4d57313f8fa215b6b0fba0b3239c13b01d7425b3 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumerScheduleService.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumerScheduleService.java @@ -32,7 +32,9 @@ import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.remoting.RPCHook; /** - * Schedule service for pull consumer + * Schedule service for pull consumer. + * This Consumer will be removed in 2022, and a better implementation {@link + * DefaultLitePullConsumer} is recommend to use in the scenario of actively pulling messages. */ public class MQPullConsumerScheduleService { private final InternalLogger log = ClientLogger.getLog(); @@ -157,7 +159,7 @@ public class MQPullConsumerScheduleService { } } - class PullTaskImpl implements Runnable { + public class PullTaskImpl implements Runnable { private final MessageQueue messageQueue; private volatile boolean cancelled = false; diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/TopicMessageQueueChangeListener.java b/client/src/main/java/org/apache/rocketmq/client/consumer/TopicMessageQueueChangeListener.java new file mode 100644 index 0000000000000000000000000000000000000000..fa6fd134e230118654eac42d68ee5296305c0e06 --- /dev/null +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/TopicMessageQueueChangeListener.java @@ -0,0 +1,30 @@ +/* + * 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; + +import java.util.Set; +import org.apache.rocketmq.common.message.MessageQueue; + +public interface TopicMessageQueueChangeListener { + /** + * This method will be invoked in the condition of queue numbers changed, These scenarios occur when the topic is + * expanded or shrunk. + * + * @param messageQueues + */ + void onChanged(String topic, Set messageQueues); +} 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 c1524e10777981e25ef03326ef0abbc8727041e8..6b762383717bd01889b355334540f0fca061d6ec 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 @@ -117,25 +117,24 @@ public class RemoteBrokerOffsetStore implements OffsetStore { return; final HashSet unusedMQ = new HashSet(); - if (!mqs.isEmpty()) { - for (Map.Entry entry : this.offsetTable.entrySet()) { - MessageQueue mq = entry.getKey(); - AtomicLong offset = entry.getValue(); - if (offset != null) { - if (mqs.contains(mq)) { - try { - this.updateConsumeOffsetToBroker(mq, offset.get()); - log.info("[persistAll] Group: {} ClientId: {} updateConsumeOffsetToBroker {} {}", - this.groupName, - this.mQClientFactory.getClientId(), - mq, - offset.get()); - } catch (Exception e) { - log.error("updateConsumeOffsetToBroker exception, " + mq.toString(), e); - } - } else { - unusedMQ.add(mq); + + for (Map.Entry entry : this.offsetTable.entrySet()) { + MessageQueue mq = entry.getKey(); + AtomicLong offset = entry.getValue(); + if (offset != null) { + if (mqs.contains(mq)) { + try { + this.updateConsumeOffsetToBroker(mq, offset.get()); + log.info("[persistAll] Group: {} ClientId: {} updateConsumeOffsetToBroker {} {}", + this.groupName, + this.mQClientFactory.getClientId(), + mq, + offset.get()); + } catch (Exception e) { + log.error("updateConsumeOffsetToBroker exception, " + mq.toString(), e); } + } else { + unusedMQ.add(mq); } } } @@ -187,8 +186,7 @@ public class RemoteBrokerOffsetStore implements OffsetStore { } /** - * Update the Consumer Offset in one way, once the Master is off, updated to Slave, - * here need to be optimized. + * Update the Consumer Offset in one way, once the Master is off, updated to Slave, here need to be optimized. */ private void updateConsumeOffsetToBroker(MessageQueue mq, long offset) throws RemotingException, MQBrokerException, InterruptedException, MQClientException { @@ -196,15 +194,13 @@ public class RemoteBrokerOffsetStore implements OffsetStore { } /** - * Update the Consumer Offset synchronously, once the Master is off, updated to Slave, - * here need to be optimized. + * Update the Consumer Offset synchronously, once the Master is off, updated to Slave, here need to be optimized. */ @Override public void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean isOneway) throws RemotingException, MQBrokerException, InterruptedException, MQClientException { FindBrokerResult findBrokerResult = this.mQClientFactory.findBrokerAddressInAdmin(mq.getBrokerName()); if (null == findBrokerResult) { - this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic()); findBrokerResult = this.mQClientFactory.findBrokerAddressInAdmin(mq.getBrokerName()); } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientManager.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientManager.java index 31553a67ae5fd28cfc2605ecc633f7aaf5edbeb4..053c049c9cd703832afe9044b92fd8e82fa2b719 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientManager.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientManager.java @@ -40,11 +40,11 @@ public class MQClientManager { return instance; } - public MQClientInstance getAndCreateMQClientInstance(final ClientConfig clientConfig) { - return getAndCreateMQClientInstance(clientConfig, null); + public MQClientInstance getOrCreateMQClientInstance(final ClientConfig clientConfig) { + return getOrCreateMQClientInstance(clientConfig, null); } - public MQClientInstance getAndCreateMQClientInstance(final ClientConfig clientConfig, RPCHook rpcHook) { + public MQClientInstance getOrCreateMQClientInstance(final ClientConfig clientConfig, RPCHook rpcHook) { String clientId = clientConfig.buildMQClientId(); MQClientInstance instance = this.factoryTable.get(clientId); if (null == instance) { diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/AssignedMessageQueue.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/AssignedMessageQueue.java new file mode 100644 index 0000000000000000000000000000000000000000..c0c6f6030c2599457aef833f389e62a22fcee296 --- /dev/null +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/AssignedMessageQueue.java @@ -0,0 +1,241 @@ +/* + * 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 java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.rocketmq.common.message.MessageQueue; + +public class AssignedMessageQueue { + + private final ConcurrentHashMap assignedMessageQueueState; + + private RebalanceImpl rebalanceImpl; + + public AssignedMessageQueue() { + assignedMessageQueueState = new ConcurrentHashMap(); + } + + public void setRebalanceImpl(RebalanceImpl rebalanceImpl) { + this.rebalanceImpl = rebalanceImpl; + } + + public Set messageQueues() { + return assignedMessageQueueState.keySet(); + } + + public boolean isPaused(MessageQueue messageQueue) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + return messageQueueState.isPaused(); + } + return true; + } + + public void pause(Collection messageQueues) { + for (MessageQueue messageQueue : messageQueues) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (assignedMessageQueueState.get(messageQueue) != null) { + messageQueueState.setPaused(true); + } + } + } + + public void resume(Collection messageQueueCollection) { + for (MessageQueue messageQueue : messageQueueCollection) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (assignedMessageQueueState.get(messageQueue) != null) { + messageQueueState.setPaused(false); + } + } + } + + public ProcessQueue getProcessQueue(MessageQueue messageQueue) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + return messageQueueState.getProcessQueue(); + } + return null; + } + + public long getPullOffset(MessageQueue messageQueue) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + return messageQueueState.getPullOffset(); + } + return -1; + } + + public void updatePullOffset(MessageQueue messageQueue, long offset) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + messageQueueState.setPullOffset(offset); + } + } + + public long getConusmerOffset(MessageQueue messageQueue) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + return messageQueueState.getConsumeOffset(); + } + return -1; + } + + public void updateConsumeOffset(MessageQueue messageQueue, long offset) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + messageQueueState.setConsumeOffset(offset); + } + } + + public void setSeekOffset(MessageQueue messageQueue, long offset) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + messageQueueState.setSeekOffset(offset); + } + } + + public long getSeekOffset(MessageQueue messageQueue) { + MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue); + if (messageQueueState != null) { + return messageQueueState.getSeekOffset(); + } + return -1; + } + + public void updateAssignedMessageQueue(String topic, Collection assigned) { + synchronized (this.assignedMessageQueueState) { + Iterator> it = this.assignedMessageQueueState.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry next = it.next(); + if (next.getKey().getTopic().equals(topic)) { + if (!assigned.contains(next.getKey())) { + next.getValue().getProcessQueue().setDropped(true); + it.remove(); + } + } + } + addAssignedMessageQueue(assigned); + } + } + + public void updateAssignedMessageQueue(Collection assigned) { + synchronized (this.assignedMessageQueueState) { + Iterator> it = this.assignedMessageQueueState.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry next = it.next(); + if (!assigned.contains(next.getKey())) { + next.getValue().getProcessQueue().setDropped(true); + it.remove(); + } + } + addAssignedMessageQueue(assigned); + } + } + + private void addAssignedMessageQueue(Collection assigned) { + for (MessageQueue messageQueue : assigned) { + if (!this.assignedMessageQueueState.containsKey(messageQueue)) { + MessageQueueState messageQueueState; + if (rebalanceImpl != null && rebalanceImpl.getProcessQueueTable().get(messageQueue) != null) { + messageQueueState = new MessageQueueState(messageQueue, rebalanceImpl.getProcessQueueTable().get(messageQueue)); + } else { + ProcessQueue processQueue = new ProcessQueue(); + messageQueueState = new MessageQueueState(messageQueue, processQueue); + } + this.assignedMessageQueueState.put(messageQueue, messageQueueState); + } + } + } + + public void removeAssignedMessageQueue(String topic) { + synchronized (this.assignedMessageQueueState) { + Iterator> it = this.assignedMessageQueueState.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry next = it.next(); + if (next.getKey().getTopic().equals(topic)) { + it.remove(); + } + } + } + } + + private class MessageQueueState { + private MessageQueue messageQueue; + private ProcessQueue processQueue; + private volatile boolean paused = false; + private volatile long pullOffset = -1; + private volatile long consumeOffset = -1; + private volatile long seekOffset = -1; + + private MessageQueueState(MessageQueue messageQueue, ProcessQueue processQueue) { + this.messageQueue = messageQueue; + this.processQueue = processQueue; + } + + public MessageQueue getMessageQueue() { + return messageQueue; + } + + public void setMessageQueue(MessageQueue messageQueue) { + this.messageQueue = messageQueue; + } + + public boolean isPaused() { + return paused; + } + + public void setPaused(boolean paused) { + this.paused = paused; + } + + public long getPullOffset() { + return pullOffset; + } + + public void setPullOffset(long pullOffset) { + this.pullOffset = pullOffset; + } + + public ProcessQueue getProcessQueue() { + return processQueue; + } + + public void setProcessQueue(ProcessQueue processQueue) { + this.processQueue = processQueue; + } + + public long getConsumeOffset() { + return consumeOffset; + } + + public void setConsumeOffset(long consumeOffset) { + this.consumeOffset = consumeOffset; + } + + public long getSeekOffset() { + return seekOffset; + } + + public void setSeekOffset(long seekOffset) { + this.seekOffset = seekOffset; + } + } +} 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 new file mode 100644 index 0000000000000000000000000000000000000000..a37c3a0149047a7313e9cc2f3a92e3b4bc0a07c4 --- /dev/null +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java @@ -0,0 +1,1073 @@ +/* + * 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 java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.Collection; +import java.util.HashSet; +import java.util.Properties; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; + +import org.apache.rocketmq.client.Validators; +import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; +import org.apache.rocketmq.client.consumer.TopicMessageQueueChangeListener; +import org.apache.rocketmq.client.consumer.MessageSelector; +import org.apache.rocketmq.client.consumer.MessageQueueListener; +import org.apache.rocketmq.client.consumer.PullResult; +import org.apache.rocketmq.client.consumer.store.LocalFileOffsetStore; +import org.apache.rocketmq.client.consumer.store.OffsetStore; +import org.apache.rocketmq.client.consumer.store.ReadOffsetType; +import org.apache.rocketmq.client.consumer.store.RemoteBrokerOffsetStore; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.hook.FilterMessageHook; +import org.apache.rocketmq.client.impl.CommunicationMode; +import org.apache.rocketmq.client.impl.MQClientManager; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.client.log.ClientLogger; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.ServiceState; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.filter.ExpressionType; +import org.apache.rocketmq.common.filter.FilterAPI; +import org.apache.rocketmq.common.help.FAQUrl; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.NamespaceUtil; +import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo; +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.sysflag.PullSysFlag; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.exception.RemotingException; + +public class DefaultLitePullConsumerImpl implements MQConsumerInner { + + private final InternalLogger log = ClientLogger.getLog(); + + private final long consumerStartTimestamp = System.currentTimeMillis(); + + private final RPCHook rpcHook; + + private final ArrayList filterMessageHookList = new ArrayList(); + + private volatile ServiceState serviceState = ServiceState.CREATE_JUST; + + protected MQClientInstance mQClientFactory; + + private PullAPIWrapper pullAPIWrapper; + + private OffsetStore offsetStore; + + private RebalanceImpl rebalanceImpl = new RebalanceLitePullImpl(this); + + private enum SubscriptionType { + NONE, SUBSCRIBE, ASSIGN + } + + private static final String NOT_RUNNING_EXCEPTION_MESSAGE = "The consumer not running, please start it first."; + + private static final String SUBSCRIPTION_CONFILCT_EXCEPTION_MESSAGE = "Subscribe and assign are mutually exclusive."; + /** + * the type of subscription + */ + private SubscriptionType subscriptionType = SubscriptionType.NONE; + /** + * Delay some time when exception occur + */ + private static final long PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION = 1000; + /** + * Flow control interval + */ + private static final long PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL = 50; + /** + * Delay some time when suspend pull service + */ + private static final long PULL_TIME_DELAY_MILLS_WHEN_PAUSE = 1000; + + private DefaultLitePullConsumer defaultLitePullConsumer; + + private final ConcurrentMap taskTable = + new ConcurrentHashMap(); + + private AssignedMessageQueue assignedMessageQueue = new AssignedMessageQueue(); + + private final BlockingQueue consumeRequestCache = new LinkedBlockingQueue(); + + private ScheduledThreadPoolExecutor scheduledThreadPoolExecutor; + + private final ScheduledExecutorService scheduledExecutorService; + + private Map topicMessageQueueChangeListenerMap = new HashMap(); + + private Map> messageQueuesForTopic = new HashMap>(); + + private long consumeRequestFlowControlTimes = 0L; + + private long queueFlowControlTimes = 0L; + + private long queueMaxSpanFlowControlTimes = 0L; + + private long nextAutoCommitDeadline = -1L; + + private final MessageQueueLock messageQueueLock = new MessageQueueLock(); + + public DefaultLitePullConsumerImpl(final DefaultLitePullConsumer defaultLitePullConsumer, final RPCHook rpcHook) { + this.defaultLitePullConsumer = defaultLitePullConsumer; + this.rpcHook = rpcHook; + this.scheduledThreadPoolExecutor = new ScheduledThreadPoolExecutor( + this.defaultLitePullConsumer.getPullThreadNums(), + new ThreadFactoryImpl("PullMsgThread-" + this.defaultLitePullConsumer.getConsumerGroup()) + ); + this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + return new Thread(r, "MonitorMessageQueueChangeThread"); + } + }); + } + + private void checkServiceState() { + if (this.serviceState != ServiceState.RUNNING) + throw new IllegalStateException(NOT_RUNNING_EXCEPTION_MESSAGE); + } + + private synchronized void setSubscriptionType(SubscriptionType type) { + if (this.subscriptionType == SubscriptionType.NONE) + this.subscriptionType = type; + else if (this.subscriptionType != type) + throw new IllegalStateException(SUBSCRIPTION_CONFILCT_EXCEPTION_MESSAGE); + } + + private void updateAssignedMessageQueue(String topic, Set assignedMessageQueue) { + this.assignedMessageQueue.updateAssignedMessageQueue(topic, assignedMessageQueue); + } + + private void updatePullTask(String topic, Set mqNewSet) { + Iterator> it = this.taskTable.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry next = it.next(); + if (next.getKey().getTopic().equals(topic)) { + if (!mqNewSet.contains(next.getKey())) { + next.getValue().setCancelled(true); + it.remove(); + } + } + } + startPullTask(mqNewSet); + } + + class MessageQueueListenerImpl implements MessageQueueListener { + @Override + public void messageQueueChanged(String topic, Set mqAll, Set mqDivided) { + MessageModel messageModel = defaultLitePullConsumer.getMessageModel(); + switch (messageModel) { + case BROADCASTING: + updateAssignedMessageQueue(topic, mqAll); + updatePullTask(topic, mqAll); + break; + case CLUSTERING: + updateAssignedMessageQueue(topic, mqDivided); + updatePullTask(topic, mqDivided); + break; + default: + break; + } + } + } + + private int nextPullBatchSize() { + return Math.min(this.defaultLitePullConsumer.getPullBatchSize(), consumeRequestCache.remainingCapacity()); + } + + public synchronized void shutdown() { + switch (this.serviceState) { + case CREATE_JUST: + break; + case RUNNING: + persistConsumerOffset(); + this.mQClientFactory.unregisterConsumer(this.defaultLitePullConsumer.getConsumerGroup()); + scheduledThreadPoolExecutor.shutdown(); + scheduledExecutorService.shutdown(); + this.mQClientFactory.shutdown(); + this.serviceState = ServiceState.SHUTDOWN_ALREADY; + log.info("the consumer [{}] shutdown OK", this.defaultLitePullConsumer.getConsumerGroup()); + break; + default: + break; + } + } + + public synchronized void start() throws MQClientException { + switch (this.serviceState) { + case CREATE_JUST: + this.serviceState = ServiceState.START_FAILED; + + this.checkConfig(); + + if (this.defaultLitePullConsumer.getMessageModel() == MessageModel.CLUSTERING) { + this.defaultLitePullConsumer.changeInstanceNameToPID(); + } + + initMQClientFactory(); + + initRebalanceImpl(); + + initPullAPIWrapper(); + + initOffsetStore(); + + mQClientFactory.start(); + + startScheduleTask(); + + this.serviceState = ServiceState.RUNNING; + + log.info("the consumer [{}] start OK", this.defaultLitePullConsumer.getConsumerGroup()); + + operateAfterRunning(); + + break; + case RUNNING: + case START_FAILED: + case SHUTDOWN_ALREADY: + throw new MQClientException("The PullConsumer service state not OK, maybe started once, " + + this.serviceState + + FAQUrl.suggestTodo(FAQUrl.CLIENT_SERVICE_NOT_OK), + null); + default: + break; + } + } + + private void initMQClientFactory() throws MQClientException { + this.mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(this.defaultLitePullConsumer, this.rpcHook); + boolean registerOK = mQClientFactory.registerConsumer(this.defaultLitePullConsumer.getConsumerGroup(), this); + if (!registerOK) { + this.serviceState = ServiceState.CREATE_JUST; + + throw new MQClientException("The consumer group[" + this.defaultLitePullConsumer.getConsumerGroup() + + "] has been created before, specify another name please." + FAQUrl.suggestTodo(FAQUrl.GROUP_NAME_DUPLICATE_URL), + null); + } + } + + 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); + } + + private void initPullAPIWrapper() { + this.pullAPIWrapper = new PullAPIWrapper( + mQClientFactory, + this.defaultLitePullConsumer.getConsumerGroup(), isUnitMode()); + this.pullAPIWrapper.registerFilterMessageHook(filterMessageHookList); + } + + private void initOffsetStore() throws MQClientException { + if (this.defaultLitePullConsumer.getOffsetStore() != null) { + this.offsetStore = this.defaultLitePullConsumer.getOffsetStore(); + } else { + switch (this.defaultLitePullConsumer.getMessageModel()) { + case BROADCASTING: + this.offsetStore = new LocalFileOffsetStore(this.mQClientFactory, this.defaultLitePullConsumer.getConsumerGroup()); + break; + case CLUSTERING: + this.offsetStore = new RemoteBrokerOffsetStore(this.mQClientFactory, this.defaultLitePullConsumer.getConsumerGroup()); + break; + default: + break; + } + this.defaultLitePullConsumer.setOffsetStore(this.offsetStore); + } + this.offsetStore.load(); + } + + private void startScheduleTask() { + scheduledExecutorService.scheduleAtFixedRate( + new Runnable() { + @Override + public void run() { + try { + fetchTopicMessageQueuesAndCompare(); + } catch (Exception e) { + log.error("ScheduledTask fetchMessageQueuesAndCompare exception", e); + } + } + }, 1000 * 10, this.getDefaultLitePullConsumer().getTopicMetadataCheckIntervalMillis(), TimeUnit.MILLISECONDS); + } + + private void operateAfterRunning() throws MQClientException { + // If subscribe function invoke before start function, then update topic subscribe info after initialization. + if (subscriptionType == SubscriptionType.SUBSCRIBE) { + updateTopicSubscribeInfoWhenSubscriptionChanged(); + } + // If assign function invoke before start function, then update pull task after initialization. + if (subscriptionType == SubscriptionType.ASSIGN) { + updateAssignPullTask(assignedMessageQueue.messageQueues()); + } + + for (String topic : topicMessageQueueChangeListenerMap.keySet()) { + Set messageQueues = fetchMessageQueues(topic); + messageQueuesForTopic.put(topic, messageQueues); + } + this.mQClientFactory.checkClientInBroker(); + } + + private void checkConfig() throws MQClientException { + // Check consumerGroup + Validators.checkGroup(this.defaultLitePullConsumer.getConsumerGroup()); + + // Check consumerGroup name is not equal default consumer group name. + if (this.defaultLitePullConsumer.getConsumerGroup().equals(MixAll.DEFAULT_CONSUMER_GROUP)) { + throw new MQClientException( + "consumerGroup can not equal " + + MixAll.DEFAULT_CONSUMER_GROUP + + ", please specify another one." + + FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), + null); + } + + // Check messageModel is not null. + if (null == this.defaultLitePullConsumer.getMessageModel()) { + throw new MQClientException( + "messageModel is null" + + FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), + null); + } + + // Check allocateMessageQueueStrategy is not null + if (null == this.defaultLitePullConsumer.getAllocateMessageQueueStrategy()) { + throw new MQClientException( + "allocateMessageQueueStrategy is null" + + FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), + null); + } + + if (this.defaultLitePullConsumer.getConsumerTimeoutMillisWhenSuspend() < this.defaultLitePullConsumer.getBrokerSuspendMaxTimeMillis()) { + throw new MQClientException( + "Long polling mode, the consumer consumerTimeoutMillisWhenSuspend must greater than brokerSuspendMaxTimeMillis" + + FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), + null); + } + } + + public PullAPIWrapper getPullAPIWrapper() { + return pullAPIWrapper; + } + + private void startPullTask(Collection mqSet) { + for (MessageQueue messageQueue : mqSet) { + if (!this.taskTable.containsKey(messageQueue)) { + PullTaskImpl pullTask = new PullTaskImpl(messageQueue); + this.taskTable.put(messageQueue, pullTask); + this.scheduledThreadPoolExecutor.schedule(pullTask, 0, TimeUnit.MILLISECONDS); + } + } + } + + private void updateAssignPullTask(Collection mqNewSet) { + Iterator> it = this.taskTable.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry next = it.next(); + if (!mqNewSet.contains(next.getKey())) { + next.getValue().setCancelled(true); + it.remove(); + } + } + + startPullTask(mqNewSet); + } + + private void updateTopicSubscribeInfoWhenSubscriptionChanged() { + Map subTable = rebalanceImpl.getSubscriptionInner(); + if (subTable != null) { + for (final Map.Entry entry : subTable.entrySet()) { + final String topic = entry.getKey(); + this.mQClientFactory.updateTopicRouteInfoFromNameServer(topic); + } + } + } + + public synchronized void subscribe(String topic, String subExpression) throws MQClientException { + try { + if (topic == null || topic.equals("")) { + throw new IllegalArgumentException("Topic can not be null or empty."); + } + setSubscriptionType(SubscriptionType.SUBSCRIBE); + SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(defaultLitePullConsumer.getConsumerGroup(), + topic, subExpression); + this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData); + this.defaultLitePullConsumer.setMessageQueueListener(new MessageQueueListenerImpl()); + assignedMessageQueue.setRebalanceImpl(this.rebalanceImpl); + if (serviceState == ServiceState.RUNNING) { + this.mQClientFactory.sendHeartbeatToAllBrokerWithLock(); + updateTopicSubscribeInfoWhenSubscriptionChanged(); + } + } catch (Exception e) { + throw new MQClientException("subscribe exception", e); + } + } + + public synchronized void subscribe(String topic, MessageSelector messageSelector) throws MQClientException { + try { + if (topic == null || topic.equals("")) { + throw new IllegalArgumentException("Topic can not be null or empty."); + } + setSubscriptionType(SubscriptionType.SUBSCRIBE); + if (messageSelector == null) { + subscribe(topic, SubscriptionData.SUB_ALL); + return; + } + SubscriptionData subscriptionData = FilterAPI.build(topic, + messageSelector.getExpression(), messageSelector.getExpressionType()); + this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData); + this.defaultLitePullConsumer.setMessageQueueListener(new MessageQueueListenerImpl()); + assignedMessageQueue.setRebalanceImpl(this.rebalanceImpl); + if (serviceState == ServiceState.RUNNING) { + this.mQClientFactory.sendHeartbeatToAllBrokerWithLock(); + updateTopicSubscribeInfoWhenSubscriptionChanged(); + } + } catch (Exception e) { + throw new MQClientException("subscribe exception", e); + } + } + + public synchronized void unsubscribe(final String topic) { + this.rebalanceImpl.getSubscriptionInner().remove(topic); + removePullTaskCallback(topic); + assignedMessageQueue.removeAssignedMessageQueue(topic); + } + + public synchronized void assign(Collection messageQueues) { + if (messageQueues == null || messageQueues.isEmpty()) { + throw new IllegalArgumentException("Message queues can not be null or empty."); + } + setSubscriptionType(SubscriptionType.ASSIGN); + assignedMessageQueue.updateAssignedMessageQueue(messageQueues); + if (serviceState == ServiceState.RUNNING) { + updateAssignPullTask(messageQueues); + } + } + + private void maybeAutoCommit() { + long now = System.currentTimeMillis(); + if (now >= nextAutoCommitDeadline) { + commitAll(); + nextAutoCommitDeadline = now + defaultLitePullConsumer.getAutoCommitIntervalMillis(); + } + } + + public synchronized List poll(long timeout) { + try { + checkServiceState(); + if (timeout < 0) + throw new IllegalArgumentException("Timeout must not be negative"); + + if (defaultLitePullConsumer.isAutoCommit()) { + maybeAutoCommit(); + } + long endTime = System.currentTimeMillis() + timeout; + + ConsumeRequest consumeRequest = consumeRequestCache.poll(endTime - System.currentTimeMillis(), TimeUnit.MILLISECONDS); + + if (endTime - System.currentTimeMillis() > 0) { + while (consumeRequest != null && consumeRequest.getProcessQueue().isDropped()) { + consumeRequest = consumeRequestCache.poll(endTime - System.currentTimeMillis(), TimeUnit.MILLISECONDS); + if (endTime - System.currentTimeMillis() <= 0) + break; + } + } + + if (consumeRequest != null && !consumeRequest.getProcessQueue().isDropped()) { + List messages = consumeRequest.getMessageExts(); + long offset = consumeRequest.getProcessQueue().removeMessage(messages); + assignedMessageQueue.updateConsumeOffset(consumeRequest.getMessageQueue(), offset); + //If namespace not null , reset Topic without namespace. + this.resetTopic(messages); + return messages; + } + } catch (InterruptedException ignore) { + + } + + return Collections.emptyList(); + } + + public void pause(Collection messageQueues) { + assignedMessageQueue.pause(messageQueues); + } + + public void resume(Collection messageQueues) { + assignedMessageQueue.resume(messageQueues); + } + + public synchronized void seek(MessageQueue messageQueue, long offset) throws MQClientException { + if (!assignedMessageQueue.messageQueues().contains(messageQueue)) { + if (subscriptionType == SubscriptionType.SUBSCRIBE) { + throw new MQClientException("The message queue is not in assigned list, may be rebalancing, message queue: " + messageQueue, null); + } else { + throw new MQClientException("The message queue is not in assigned list, message queue: " + messageQueue, null); + } + } + long minOffset = minOffset(messageQueue); + long maxOffset = maxOffset(messageQueue); + if (offset < minOffset || offset > maxOffset) { + throw new MQClientException("Seek offset illegal, seek offset = " + offset + ", min offset = " + minOffset + ", max offset = " + maxOffset, null); + } + final Object objLock = messageQueueLock.fetchLockObject(messageQueue); + synchronized (objLock) { + assignedMessageQueue.setSeekOffset(messageQueue, offset); + clearMessageQueueInCache(messageQueue); + } + } + + private long maxOffset(MessageQueue messageQueue) throws MQClientException { + checkServiceState(); + return this.mQClientFactory.getMQAdminImpl().maxOffset(messageQueue); + } + + private long minOffset(MessageQueue messageQueue) throws MQClientException { + checkServiceState(); + return this.mQClientFactory.getMQAdminImpl().minOffset(messageQueue); + } + + private void removePullTaskCallback(final String topic) { + removePullTask(topic); + } + + private void removePullTask(final String topic) { + Iterator> it = this.taskTable.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry next = it.next(); + if (next.getKey().getTopic().equals(topic)) { + next.getValue().setCancelled(true); + it.remove(); + } + } + } + + public synchronized void commitSync() { + try { + for (MessageQueue messageQueue : assignedMessageQueue.messageQueues()) { + long consumerOffset = assignedMessageQueue.getConusmerOffset(messageQueue); + if (consumerOffset != -1) { + ProcessQueue processQueue = assignedMessageQueue.getProcessQueue(messageQueue); + long preConsumerOffset = this.getOffsetStore().readOffset(messageQueue, ReadOffsetType.READ_FROM_MEMORY); + if (processQueue != null && !processQueue.isDropped() && consumerOffset != preConsumerOffset) { + updateConsumeOffset(messageQueue, consumerOffset); + updateConsumeOffsetToBroker(messageQueue, consumerOffset, false); + } + } + } + if (defaultLitePullConsumer.getMessageModel() == MessageModel.BROADCASTING) { + offsetStore.persistAll(assignedMessageQueue.messageQueues()); + } + } catch (Exception e) { + log.error("An error occurred when update consume offset synchronously.", e); + } + } + + private synchronized void commitAll() { + try { + for (MessageQueue messageQueue : assignedMessageQueue.messageQueues()) { + long consumerOffset = assignedMessageQueue.getConusmerOffset(messageQueue); + if (consumerOffset != -1) { + ProcessQueue processQueue = assignedMessageQueue.getProcessQueue(messageQueue); + long preConsumerOffset = this.getOffsetStore().readOffset(messageQueue, ReadOffsetType.READ_FROM_MEMORY); + if (processQueue != null && !processQueue.isDropped() && consumerOffset != preConsumerOffset) { + updateConsumeOffset(messageQueue, consumerOffset); + updateConsumeOffsetToBroker(messageQueue, consumerOffset, true); + } + } + } + if (defaultLitePullConsumer.getMessageModel() == MessageModel.BROADCASTING) { + offsetStore.persistAll(assignedMessageQueue.messageQueues()); + } + } catch (Exception e) { + log.error("An error occurred when update consume offset Automatically."); + } + } + + private void updatePullOffset(MessageQueue messageQueue, long nextPullOffset) { + if (assignedMessageQueue.getSeekOffset(messageQueue) == -1) { + assignedMessageQueue.updatePullOffset(messageQueue, nextPullOffset); + } + } + + private void submitConsumeRequest(ConsumeRequest consumeRequest) { + try { + consumeRequestCache.put(consumeRequest); + } catch (InterruptedException e) { + log.error("Submit consumeRequest error", e); + } + } + + private long fetchConsumeOffset(MessageQueue messageQueue, boolean fromStore) { + checkServiceState(); + return this.offsetStore.readOffset(messageQueue, fromStore ? ReadOffsetType.READ_FROM_STORE : ReadOffsetType.MEMORY_FIRST_THEN_STORE); + } + + public long committed(MessageQueue messageQueue) throws MQClientException { + checkServiceState(); + long offset = this.offsetStore.readOffset(messageQueue, ReadOffsetType.READ_FROM_STORE); + if (offset == -2) + throw new MQClientException("Fetch consume offset from broker exception", null); + return offset; + } + + private void clearMessageQueueInCache(MessageQueue messageQueue) { + ProcessQueue processQueue = assignedMessageQueue.getProcessQueue(messageQueue); + if (processQueue != null) { + processQueue.clear(); + } + Iterator iter = consumeRequestCache.iterator(); + while (iter.hasNext()) { + if (iter.next().getMessageQueue().equals(messageQueue)) + iter.remove(); + } + } + + private long nextPullOffset(MessageQueue messageQueue) { + long offset = -1; + long seekOffset = assignedMessageQueue.getSeekOffset(messageQueue); + if (seekOffset != -1) { + offset = seekOffset; + assignedMessageQueue.updateConsumeOffset(messageQueue, offset); + assignedMessageQueue.setSeekOffset(messageQueue, -1); + } else { + offset = assignedMessageQueue.getPullOffset(messageQueue); + if (offset == -1) { + offset = fetchConsumeOffset(messageQueue, false); + if (offset == -1 && defaultLitePullConsumer.getMessageModel() == MessageModel.BROADCASTING) { + offset = 0; + } + } + } + return offset; + } + + public long searchOffset(MessageQueue mq, long timestamp) throws MQClientException { + checkServiceState(); + return this.mQClientFactory.getMQAdminImpl().searchOffset(mq, timestamp); + } + + public class PullTaskImpl implements Runnable { + private final MessageQueue messageQueue; + private volatile boolean cancelled = false; + + public PullTaskImpl(final MessageQueue messageQueue) { + this.messageQueue = messageQueue; + } + + @Override + public void run() { + + if (!this.isCancelled()) { + + if (assignedMessageQueue.isPaused(messageQueue)) { + scheduledThreadPoolExecutor.schedule(this, PULL_TIME_DELAY_MILLS_WHEN_PAUSE, TimeUnit.MILLISECONDS); + log.debug("Message Queue: {} has been paused!", messageQueue); + return; + } + + ProcessQueue processQueue = assignedMessageQueue.getProcessQueue(messageQueue); + + if (processQueue == null && processQueue.isDropped()) { + log.info("The message queue not be able to poll, because it's dropped. group={}, messageQueue={}", defaultLitePullConsumer.getConsumerGroup(), this.messageQueue); + return; + } + + if (consumeRequestCache.size() * defaultLitePullConsumer.getPullBatchSize() > defaultLitePullConsumer.getPullThresholdForAll()) { + scheduledThreadPoolExecutor.schedule(this, PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL, TimeUnit.MILLISECONDS); + if ((consumeRequestFlowControlTimes++ % 1000) == 0) + log.warn("The consume request count exceeds threshold {}, so do flow control, consume request count={}, flowControlTimes={}", consumeRequestCache.size(), consumeRequestFlowControlTimes); + return; + } + + long cachedMessageCount = processQueue.getMsgCount().get(); + long cachedMessageSizeInMiB = processQueue.getMsgSize().get() / (1024 * 1024); + + if (cachedMessageCount > defaultLitePullConsumer.getPullThresholdForQueue()) { + scheduledThreadPoolExecutor.schedule(this, PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL, TimeUnit.MILLISECONDS); + if ((queueFlowControlTimes++ % 1000) == 0) { + log.warn( + "The cached message count exceeds the threshold {}, so do flow control, minOffset={}, maxOffset={}, count={}, size={} MiB, flowControlTimes={}", + defaultLitePullConsumer.getPullThresholdForQueue(), processQueue.getMsgTreeMap().firstKey(), processQueue.getMsgTreeMap().lastKey(), cachedMessageCount, cachedMessageSizeInMiB, queueFlowControlTimes); + } + return; + } + + if (cachedMessageSizeInMiB > defaultLitePullConsumer.getPullThresholdSizeForQueue()) { + scheduledThreadPoolExecutor.schedule(this, PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL, TimeUnit.MILLISECONDS); + if ((queueFlowControlTimes++ % 1000) == 0) { + log.warn( + "The cached message size exceeds the threshold {} MiB, so do flow control, minOffset={}, maxOffset={}, count={}, size={} MiB, flowControlTimes={}", + defaultLitePullConsumer.getPullThresholdSizeForQueue(), processQueue.getMsgTreeMap().firstKey(), processQueue.getMsgTreeMap().lastKey(), cachedMessageCount, cachedMessageSizeInMiB, queueFlowControlTimes); + } + return; + } + + if (processQueue.getMaxSpan() > defaultLitePullConsumer.getConsumeMaxSpan()) { + scheduledThreadPoolExecutor.schedule(this, PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL, TimeUnit.MILLISECONDS); + if ((queueMaxSpanFlowControlTimes++ % 1000) == 0) { + log.warn( + "The queue's messages, span too long, so do flow control, minOffset={}, maxOffset={}, maxSpan={}, flowControlTimes={}", + processQueue.getMsgTreeMap().firstKey(), processQueue.getMsgTreeMap().lastKey(), processQueue.getMaxSpan(), queueMaxSpanFlowControlTimes); + } + return; + } + + long offset = nextPullOffset(messageQueue); + long pullDelayTimeMills = 0; + try { + SubscriptionData subscriptionData; + if (subscriptionType == SubscriptionType.SUBSCRIBE) { + String topic = this.messageQueue.getTopic(); + subscriptionData = rebalanceImpl.getSubscriptionInner().get(topic); + } else { + String topic = this.messageQueue.getTopic(); + subscriptionData = FilterAPI.buildSubscriptionData(defaultLitePullConsumer.getConsumerGroup(), + topic, SubscriptionData.SUB_ALL); + } + + PullResult pullResult = pull(messageQueue, subscriptionData, offset, nextPullBatchSize()); + switch (pullResult.getPullStatus()) { + case FOUND: + final Object objLock = messageQueueLock.fetchLockObject(messageQueue); + synchronized (objLock) { + if (pullResult.getMsgFoundList() != null && !pullResult.getMsgFoundList().isEmpty() && assignedMessageQueue.getSeekOffset(messageQueue) == -1) { + processQueue.putMessage(pullResult.getMsgFoundList()); + submitConsumeRequest(new ConsumeRequest(pullResult.getMsgFoundList(), messageQueue, processQueue)); + } + } + break; + case OFFSET_ILLEGAL: + log.warn("The pull request offset illegal, {}", pullResult.toString()); + break; + default: + break; + } + updatePullOffset(messageQueue, pullResult.getNextBeginOffset()); + } catch (Throwable e) { + pullDelayTimeMills = PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION; + log.error("An error occurred in pull message process.", e); + } + + if (!this.isCancelled()) { + scheduledThreadPoolExecutor.schedule(this, pullDelayTimeMills, TimeUnit.MILLISECONDS); + } else { + log.warn("The Pull Task is cancelled after doPullTask, {}", messageQueue); + } + } + } + + public boolean isCancelled() { + return cancelled; + } + + public void setCancelled(boolean cancelled) { + this.cancelled = cancelled; + } + + public MessageQueue getMessageQueue() { + return messageQueue; + } + } + + private PullResult pull(MessageQueue mq, SubscriptionData subscriptionData, long offset, int maxNums) + throws MQClientException, RemotingException, MQBrokerException, InterruptedException { + return pull(mq, subscriptionData, offset, maxNums, this.defaultLitePullConsumer.getConsumerPullTimeoutMillis()); + } + + private PullResult pull(MessageQueue mq, SubscriptionData subscriptionData, long offset, int maxNums, long timeout) + throws MQClientException, RemotingException, MQBrokerException, InterruptedException { + return this.pullSyncImpl(mq, subscriptionData, offset, maxNums, true, timeout); + } + + private PullResult pullSyncImpl(MessageQueue mq, SubscriptionData subscriptionData, long offset, int maxNums, + boolean block, + long timeout) + throws MQClientException, RemotingException, MQBrokerException, InterruptedException { + + if (null == mq) { + throw new MQClientException("mq is null", null); + } + + if (offset < 0) { + throw new MQClientException("offset < 0", null); + } + + if (maxNums <= 0) { + throw new MQClientException("maxNums <= 0", null); + } + + int sysFlag = PullSysFlag.buildSysFlag(false, block, true, false); + + long timeoutMillis = block ? this.defaultLitePullConsumer.getConsumerTimeoutMillisWhenSuspend() : timeout; + + boolean isTagType = ExpressionType.isTagType(subscriptionData.getExpressionType()); + PullResult pullResult = this.pullAPIWrapper.pullKernelImpl( + mq, + subscriptionData.getSubString(), + subscriptionData.getExpressionType(), + isTagType ? 0L : subscriptionData.getSubVersion(), + offset, + maxNums, + sysFlag, + 0, + this.defaultLitePullConsumer.getBrokerSuspendMaxTimeMillis(), + timeoutMillis, + CommunicationMode.SYNC, + null + ); + this.pullAPIWrapper.processPullResult(mq, pullResult, subscriptionData); + return pullResult; + } + + private void resetTopic(List msgList) { + if (null == msgList || msgList.size() == 0) { + return; + } + + //If namespace not null , reset Topic without namespace. + for (MessageExt messageExt : msgList) { + if (null != this.defaultLitePullConsumer.getNamespace()) { + messageExt.setTopic(NamespaceUtil.withoutNamespace(messageExt.getTopic(), this.defaultLitePullConsumer.getNamespace())); + } + } + + } + + public void updateConsumeOffset(MessageQueue mq, long offset) { + checkServiceState(); + this.offsetStore.updateOffset(mq, offset, false); + } + + @Override + public String groupName() { + return this.defaultLitePullConsumer.getConsumerGroup(); + } + + @Override + public MessageModel messageModel() { + return this.defaultLitePullConsumer.getMessageModel(); + } + + @Override + public ConsumeType consumeType() { + return ConsumeType.CONSUME_ACTIVELY; + } + + @Override + public ConsumeFromWhere consumeFromWhere() { + return ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET; + } + + @Override + public Set subscriptions() { + Set subSet = new HashSet(); + + subSet.addAll(this.rebalanceImpl.getSubscriptionInner().values()); + + return subSet; + } + + @Override + public void doRebalance() { + if (this.rebalanceImpl != null) { + this.rebalanceImpl.doRebalance(false); + } + } + + @Override + public void persistConsumerOffset() { + try { + checkServiceState(); + Set mqs = new HashSet(); + Set allocateMq = this.rebalanceImpl.getProcessQueueTable().keySet(); + mqs.addAll(allocateMq); + this.offsetStore.persistAll(mqs); + } catch (Exception e) { + log.error("group: " + this.defaultLitePullConsumer.getConsumerGroup() + " persistConsumerOffset exception", e); + } + } + + @Override + public void updateTopicSubscribeInfo(String topic, Set info) { + Map subTable = this.rebalanceImpl.getSubscriptionInner(); + if (subTable != null) { + if (subTable.containsKey(topic)) { + this.rebalanceImpl.getTopicSubscribeInfoTable().put(topic, info); + } + } + } + + @Override + public boolean isSubscribeTopicNeedUpdate(String topic) { + Map subTable = this.rebalanceImpl.getSubscriptionInner(); + if (subTable != null) { + if (subTable.containsKey(topic)) { + return !this.rebalanceImpl.topicSubscribeInfoTable.containsKey(topic); + } + } + + return false; + } + + @Override + public boolean isUnitMode() { + return this.defaultLitePullConsumer.isUnitMode(); + } + + @Override + public ConsumerRunningInfo consumerRunningInfo() { + ConsumerRunningInfo info = new ConsumerRunningInfo(); + + Properties prop = MixAll.object2Properties(this.defaultLitePullConsumer); + prop.put(ConsumerRunningInfo.PROP_CONSUMER_START_TIMESTAMP, String.valueOf(this.consumerStartTimestamp)); + info.setProperties(prop); + + info.getSubscriptionSet().addAll(this.subscriptions()); + return info; + } + + private void updateConsumeOffsetToBroker(MessageQueue mq, long offset, boolean isOneway) throws RemotingException, + MQBrokerException, InterruptedException, MQClientException { + this.offsetStore.updateConsumeOffsetToBroker(mq, offset, isOneway); + } + + public OffsetStore getOffsetStore() { + return offsetStore; + } + + public DefaultLitePullConsumer getDefaultLitePullConsumer() { + return defaultLitePullConsumer; + } + + public Set fetchMessageQueues(String topic) throws MQClientException { + checkServiceState(); + Set result = this.mQClientFactory.getMQAdminImpl().fetchSubscribeMessageQueues(topic); + return parseMessageQueues(result); + } + + private synchronized void fetchTopicMessageQueuesAndCompare() throws MQClientException { + for (Map.Entry entry : topicMessageQueueChangeListenerMap.entrySet()) { + String topic = entry.getKey(); + TopicMessageQueueChangeListener topicMessageQueueChangeListener = entry.getValue(); + Set oldMessageQueues = messageQueuesForTopic.get(topic); + Set newMessageQueues = fetchMessageQueues(topic); + boolean isChanged = !isSetEqual(newMessageQueues, oldMessageQueues); + if (isChanged) { + messageQueuesForTopic.put(topic, newMessageQueues); + if (topicMessageQueueChangeListener != null) { + topicMessageQueueChangeListener.onChanged(topic, newMessageQueues); + } + } + } + } + + private boolean isSetEqual(Set set1, Set set2) { + if (set1 == null && set2 == null) { + return true; + } + + if (set1 == null || set2 == null || set1.size() != set2.size() + || set1.size() == 0 || set2.size() == 0) { + return false; + } + + Iterator iter = set2.iterator(); + boolean isEqual = true; + while (iter.hasNext()) { + if (!set1.contains(iter.next())) { + isEqual = false; + } + } + return isEqual; + } + + public synchronized void registerTopicMessageQueueChangeListener(String topic, + TopicMessageQueueChangeListener listener) throws MQClientException { + if (topic == null || listener == null) { + throw new MQClientException("Topic or listener is null", null); + } + if (topicMessageQueueChangeListenerMap.containsKey(topic)) { + log.warn("Topic {} had been registered, new listener will overwrite the old one", topic); + } + topicMessageQueueChangeListenerMap.put(topic, listener); + if (this.serviceState == ServiceState.RUNNING) { + Set messageQueues = fetchMessageQueues(topic); + messageQueuesForTopic.put(topic, messageQueues); + } + } + + private Set parseMessageQueues(Set queueSet) { + Set resultQueues = new HashSet(); + for (MessageQueue messageQueue : queueSet) { + String userTopic = NamespaceUtil.withoutNamespace(messageQueue.getTopic(), + this.defaultLitePullConsumer.getNamespace()); + resultQueues.add(new MessageQueue(userTopic, messageQueue.getBrokerName(), messageQueue.getQueueId())); + } + return resultQueues; + } + + public class ConsumeRequest { + private final List messageExts; + private final MessageQueue messageQueue; + private final ProcessQueue processQueue; + + public ConsumeRequest(final List messageExts, final MessageQueue messageQueue, + final ProcessQueue processQueue) { + this.messageExts = messageExts; + this.messageQueue = messageQueue; + this.processQueue = processQueue; + } + + public List getMessageExts() { + return messageExts; + } + + public MessageQueue getMessageQueue() { + return messageQueue; + } + + public ProcessQueue getProcessQueue() { + return processQueue; + } + + } +} 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 9975cb835e57119ac3f9adc773b590bac6434954..6c63ff494bf93b541aa0a2f5ad099bbed36cfa86 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 @@ -69,6 +69,11 @@ import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.exception.RemotingException; +/** + * This class will be removed in 2022, and a better implementation {@link DefaultLitePullConsumerImpl} is recommend to use + * in the scenario of actively pulling messages. + */ +@Deprecated public class DefaultMQPullConsumerImpl implements MQConsumerInner { private final InternalLogger log = ClientLogger.getLog(); private final DefaultMQPullConsumer defaultMQPullConsumer; @@ -77,7 +82,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner { private final ArrayList consumeMessageHookList = new ArrayList(); private final ArrayList filterMessageHookList = new ArrayList(); private volatile ServiceState serviceState = ServiceState.CREATE_JUST; - private MQClientInstance mQClientFactory; + protected MQClientInstance mQClientFactory; private PullAPIWrapper pullAPIWrapper; private OffsetStore offsetStore; private RebalanceImpl rebalanceImpl = new RebalancePullImpl(this); @@ -632,7 +637,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner { this.defaultMQPullConsumer.changeInstanceNameToPID(); } - this.mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(this.defaultMQPullConsumer, this.rpcHook); + this.mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(this.defaultMQPullConsumer, this.rpcHook); this.rebalanceImpl.setConsumerGroup(this.defaultMQPullConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultMQPullConsumer.getMessageModel()); 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 8615746ee2812e80c67723b3b929915604bb80f6..ac92dc9ff527eaba63f8c8a219119cb192fb1e74 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 @@ -581,7 +581,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { this.defaultMQPushConsumer.changeInstanceNameToPID(); } - this.mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(this.defaultMQPushConsumer, this.rpcHook); + this.mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(this.defaultMQPushConsumer, this.rpcHook); this.rebalanceImpl.setConsumerGroup(this.defaultMQPushConsumer.getConsumerGroup()); this.rebalanceImpl.setMessageModel(this.defaultMQPushConsumer.getMessageModel()); 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 0a52817ced7d9350c9a38dfc3fe358c193307364..092da9aa33e1997639ed311e822397bec7c1d053 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 @@ -26,6 +26,7 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; + import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.log.ClientLogger; import org.apache.rocketmq.logging.InternalLogger; @@ -431,4 +432,5 @@ public class ProcessQueue { public void setLastConsumeTimestamp(long lastConsumeTimestamp) { this.lastConsumeTimestamp = lastConsumeTimestamp; } + } 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 9ad07c7e4433548d64693c294e2b1bf58ea0e530..146fce6e1e3b3c7c311074efdca9d718a5a261f3 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 @@ -41,8 +41,10 @@ import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; /** - * Base class for rebalance algorithm + * This class will be removed in 2022, and a better implementation {@link RebalanceLitePullImpl} is recommend to use + * in the scenario of actively pulling messages. */ +@Deprecated public abstract class RebalanceImpl { protected static final InternalLogger log = ClientLogger.getLog(); protected final ConcurrentMap processQueueTable = new ConcurrentHashMap(64); diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceLitePullImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceLitePullImpl.java new file mode 100644 index 0000000000000000000000000000000000000000..0b8ec67778f2d86034c3573a4b69de417f89c539 --- /dev/null +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/RebalanceLitePullImpl.java @@ -0,0 +1,82 @@ +/* + * 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.AllocateMessageQueueStrategy; +import org.apache.rocketmq.client.consumer.MessageQueueListener; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType; +import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; + +import java.util.List; +import java.util.Set; + +public class RebalanceLitePullImpl extends RebalanceImpl { + + private final DefaultLitePullConsumerImpl litePullConsumerImpl; + + public RebalanceLitePullImpl(DefaultLitePullConsumerImpl litePullConsumerImpl) { + this(null, null, null, null, litePullConsumerImpl); + } + + public RebalanceLitePullImpl(String consumerGroup, MessageModel messageModel, + AllocateMessageQueueStrategy allocateMessageQueueStrategy, + MQClientInstance mQClientFactory, DefaultLitePullConsumerImpl litePullConsumerImpl) { + super(consumerGroup, messageModel, allocateMessageQueueStrategy, mQClientFactory); + this.litePullConsumerImpl = litePullConsumerImpl; + } + + @Override + public void messageQueueChanged(String topic, Set mqAll, Set mqDivided) { + MessageQueueListener messageQueueListener = this.litePullConsumerImpl.getDefaultLitePullConsumer().getMessageQueueListener(); + if (messageQueueListener != null) { + try { + messageQueueListener.messageQueueChanged(topic, mqAll, mqDivided); + } catch (Throwable e) { + log.error("messageQueueChanged exception", e); + } + } + } + + @Override + public boolean removeUnnecessaryMessageQueue(MessageQueue mq, ProcessQueue pq) { + this.litePullConsumerImpl.getOffsetStore().persist(mq); + this.litePullConsumerImpl.getOffsetStore().removeOffset(mq); + return true; + } + + @Override + public ConsumeType consumeType() { + return ConsumeType.CONSUME_ACTIVELY; + } + + @Override + public void removeDirtyOffset(final MessageQueue mq) { + this.litePullConsumerImpl.getOffsetStore().removeOffset(mq); + } + + @Override + public long computePullFromWhere(MessageQueue mq) { + return 0; + } + + @Override + public void dispatchPullRequest(List pullRequestList) { + } + +} 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 2a4fb7dfa6cedc44079425b5c455b85db56e9e1c..bbd2eecb1c3785f7520ade1d830b8f2ce19cb1e4 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 @@ -246,10 +246,6 @@ public class MQClientInstance { log.info("the client factory [{}] start OK", this.clientId); this.serviceState = ServiceState.RUNNING; break; - case RUNNING: - break; - case SHUTDOWN_ALREADY: - break; case START_FAILED: throw new MQClientException("The Factory object[" + this.getClientId() + "] has been created before, and failed.", null); default: @@ -366,7 +362,6 @@ public class MQClientInstance { } /** - * * @param offsetTable * @param namespace * @return newOffsetTable @@ -385,6 +380,7 @@ public class MQClientInstance { return newOffsetTable; } + /** * Remove offline broker */ @@ -676,10 +672,13 @@ public class MQClientInstance { } else { log.warn("updateTopicRouteInfoFromNameServer, getTopicRouteInfoFromNameServer return null, Topic: {}", topic); } - } catch (Exception e) { + } catch (MQClientException e) { if (!topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX) && !topic.equals(MixAll.AUTO_CREATE_TOPIC_KEY_TOPIC)) { log.warn("updateTopicRouteInfoFromNameServer Exception", e); } + } catch (RemotingException e) { + log.error("updateTopicRouteInfoFromNameServer Exception", e); + throw new IllegalStateException(e); } finally { this.lockNamesrv.unlock(); } @@ -743,9 +742,10 @@ public class MQClientInstance { return false; } + /** - * This method will be removed in the version 5.0.0,because filterServer was removed,and method subscribe(final String topic, final MessageSelector messageSelector) - * is recommended. + * This method will be removed in the version 5.0.0,because filterServer was removed,and method + * subscribe(final String topic, final MessageSelector messageSelector) is recommended. */ @Deprecated private void uploadFilterClassToAllFilterServer(final String consumerGroup, final String fullClassName, 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 180ce1974e5e6368538efda0c3b5e946a39d6d64..e740c834b44562cb2e5a6111912cd57315ccd8ff 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 @@ -187,7 +187,7 @@ public class DefaultMQProducerImpl implements MQProducerInner { this.defaultMQProducer.changeInstanceNameToPID(); } - this.mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(this.defaultMQProducer, rpcHook); + this.mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(this.defaultMQProducer, rpcHook); boolean registerOK = mQClientFactory.registerProducer(this.defaultMQProducer.getProducerGroup(), this); if (!registerOK) { @@ -533,6 +533,15 @@ public class DefaultMQProducerImpl implements MQProducerInner { this.mqFaultStrategy.updateFaultItem(brokerName, currentLatency, isolation); } + private void validateNameServerSetting() throws MQClientException { + 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); + } + + } + private SendResult sendDefaultImpl( Message msg, final CommunicationMode communicationMode, @@ -541,7 +550,6 @@ public class DefaultMQProducerImpl implements MQProducerInner { ) throws MQClientException, RemotingException, MQBrokerException, InterruptedException { this.makeSureStateOK(); Validators.checkMessage(msg, this.defaultMQProducer); - final long invokeID = random.nextLong(); long beginTimestampFirst = System.currentTimeMillis(); long beginTimestampPrev = beginTimestampFirst; @@ -672,13 +680,9 @@ public class DefaultMQProducerImpl implements MQProducerInner { throw mqClientException; } - 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); - } + validateNameServerSetting(); - throw new MQClientException("No route info of this topic, " + msg.getTopic() + FAQUrl.suggestTodo(FAQUrl.NO_TOPIC_ROUTE_INFO), + throw new MQClientException("No route info of this topic: " + msg.getTopic() + FAQUrl.suggestTodo(FAQUrl.NO_TOPIC_ROUTE_INFO), null).setResponseCode(ClientErrorCode.NOT_FOUND_TOPIC_EXCEPTION); } @@ -1125,6 +1129,7 @@ public class DefaultMQProducerImpl implements MQProducerInner { } } + validateNameServerSetting(); throw new MQClientException("No route info for this topic, " + msg.getTopic(), null); } 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 new file mode 100644 index 0000000000000000000000000000000000000000..b55f8a0a1c89867753e2a2b1b0b404966ad04b1e --- /dev/null +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java @@ -0,0 +1,500 @@ +/* + * 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; + +import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.rocketmq.client.ClientConfig; +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.CommunicationMode; +import org.apache.rocketmq.client.impl.FindBrokerResult; +import org.apache.rocketmq.client.impl.MQAdminImpl; +import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.MQClientManager; +import org.apache.rocketmq.client.impl.consumer.AssignedMessageQueue; +import org.apache.rocketmq.client.impl.consumer.DefaultLitePullConsumerImpl; +import org.apache.rocketmq.client.impl.consumer.PullAPIWrapper; +import org.apache.rocketmq.client.impl.consumer.PullResultExt; +import org.apache.rocketmq.client.impl.consumer.RebalanceImpl; +import org.apache.rocketmq.client.impl.consumer.RebalanceService; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.MixAll; +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.header.PullMessageRequestHeader; +import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Spy; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown; +import static org.junit.Assert.assertEquals; +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.spy; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(DefaultLitePullConsumerImpl.class) +public class DefaultLitePullConsumerTest { + @Spy + private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); + + @Mock + private MQClientAPIImpl mQClientAPIImpl; + @Mock + private MQAdminImpl mQAdminImpl; + + private RebalanceImpl rebalanceImpl; + private OffsetStore offsetStore; + private DefaultLitePullConsumerImpl litePullConsumerImpl; + private String consumerGroup = "LitePullConsumerGroup"; + private String topic = "LitePullConsumerTest"; + private String brokerName = "BrokerA"; + private boolean flag = false; + + @Before + public void init() throws Exception { + PowerMockito.suppress(PowerMockito.method(DefaultLitePullConsumerImpl.class, "updateTopicSubscribeInfoWhenSubscriptionChanged")); + Field field = MQClientInstance.class.getDeclaredField("rebalanceService"); + field.setAccessible(true); + RebalanceService rebalanceService = (RebalanceService) field.get(mQClientFactory); + field = RebalanceService.class.getDeclaredField("waitInterval"); + field.setAccessible(true); + field.set(rebalanceService, 100); + } + + @Test + public void testAssign_PollMessageSuccess() throws Exception { + DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); + try { + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + List result = litePullConsumer.poll(); + assertThat(result.get(0).getTopic()).isEqualTo(topic); + assertThat(result.get(0).getBody()).isEqualTo(new byte[] {'a'}); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testSubscribe_PollMessageSuccess() throws Exception { + DefaultLitePullConsumer litePullConsumer = createSubscribeLitePullConsumer(); + try { + Set messageQueueSet = new HashSet(); + messageQueueSet.add(createMessageQueue()); + litePullConsumerImpl.updateTopicSubscribeInfo(topic, messageQueueSet); + litePullConsumer.setPollTimeoutMillis(20 * 1000); + List result = litePullConsumer.poll(); + assertThat(result.get(0).getTopic()).isEqualTo(topic); + assertThat(result.get(0).getBody()).isEqualTo(new byte[] {'a'}); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testSubscribe_BroadcastPollMessageSuccess() throws Exception { + DefaultLitePullConsumer litePullConsumer = createBroadcastLitePullConsumer(); + try { + Set messageQueueSet = new HashSet(); + messageQueueSet.add(createMessageQueue()); + litePullConsumerImpl.updateTopicSubscribeInfo(topic, messageQueueSet); + litePullConsumer.setPollTimeoutMillis(20 * 1000); + List result = litePullConsumer.poll(); + assertThat(result.get(0).getTopic()).isEqualTo(topic); + assertThat(result.get(0).getBody()).isEqualTo(new byte[] {'a'}); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testSubscriptionType_AssignAndSubscribeExclusive() throws Exception { + DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); + try { + litePullConsumer.subscribe(topic, "*"); + litePullConsumer.assign(Collections.singletonList(createMessageQueue())); + failBecauseExceptionWasNotThrown(IllegalStateException.class); + } catch (IllegalStateException e) { + assertThat(e).hasMessageContaining("Subscribe and assign are mutually exclusive."); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testFetchMessageQueues_FetchMessageQueuesBeforeStart() throws Exception { + DefaultLitePullConsumer litePullConsumer = createNotStartLitePullConsumer(); + try { + litePullConsumer.fetchMessageQueues(topic); + failBecauseExceptionWasNotThrown(IllegalStateException.class); + } catch (IllegalStateException e) { + assertThat(e).hasMessageContaining("The consumer not running, please start it first."); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testSeek_SeekOffsetSuccess() throws Exception { + DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); + when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L); + when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(500L); + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + long offset = litePullConsumer.committed(messageQueue); + litePullConsumer.seek(messageQueue, offset); + Field field = DefaultLitePullConsumerImpl.class.getDeclaredField("assignedMessageQueue"); + field.setAccessible(true); + AssignedMessageQueue assignedMessageQueue = (AssignedMessageQueue) field.get(litePullConsumerImpl); + assertEquals(assignedMessageQueue.getSeekOffset(messageQueue), offset); + litePullConsumer.shutdown(); + } + + @Test + public void testSeek_SeekOffsetIllegal() throws Exception { + DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); + when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L); + when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(100L); + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + try { + litePullConsumer.seek(messageQueue, -1); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("min offset = 0"); + } + + try { + litePullConsumer.seek(messageQueue, 1000); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("max offset = 100"); + } + litePullConsumer.shutdown(); + } + + @Test + public void testSeek_MessageQueueNotInAssignList() throws Exception { + DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); + try { + litePullConsumer.seek(createMessageQueue(), 0); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("The message queue is not in assigned list"); + } finally { + litePullConsumer.shutdown(); + } + + litePullConsumer = createSubscribeLitePullConsumer(); + try { + litePullConsumer.seek(createMessageQueue(), 0); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("The message queue is not in assigned list, may be rebalancing"); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testOffsetForTimestamp_FailedAndSuccess() throws Exception { + MessageQueue messageQueue = createMessageQueue(); + DefaultLitePullConsumer litePullConsumer = createNotStartLitePullConsumer(); + try { + litePullConsumer.offsetForTimestamp(messageQueue, 123456L); + failBecauseExceptionWasNotThrown(IllegalStateException.class); + } catch (IllegalStateException e) { + assertThat(e).hasMessageContaining("The consumer not running, please start it first."); + } finally { + litePullConsumer.shutdown(); + } + doReturn(123L).when(mQAdminImpl).searchOffset(any(MessageQueue.class), anyLong()); + litePullConsumer = createStartLitePullConsumer(); + long offset = litePullConsumer.offsetForTimestamp(messageQueue, 123456L); + assertThat(offset).isEqualTo(123L); + } + + @Test + public void testPauseAndResume_Success() throws Exception { + DefaultLitePullConsumer litePullConsumer = createNotStartLitePullConsumer(); + try { + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + litePullConsumer.pause(Collections.singletonList(messageQueue)); + litePullConsumer.start(); + initDefaultLitePullConsumer(litePullConsumer); + List result = litePullConsumer.poll(); + assertThat(result.isEmpty()).isTrue(); + litePullConsumer.resume(Collections.singletonList(messageQueue)); + result = litePullConsumer.poll(); + assertThat(result.get(0).getTopic()).isEqualTo(topic); + assertThat(result.get(0).getBody()).isEqualTo(new byte[] {'a'}); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testRegisterTopicMessageQueueChangeListener_Success() throws Exception { + flag = false; + DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); + doReturn(Collections.emptySet()).when(mQAdminImpl).fetchSubscribeMessageQueues(anyString()); + litePullConsumer.setTopicMetadataCheckIntervalMillis(10); + litePullConsumer.registerTopicMessageQueueChangeListener(topic, new TopicMessageQueueChangeListener() { + @Override public void onChanged(String topic, Set messageQueues) { + flag = true; + } + }); + Set set = new HashSet(); + set.add(createMessageQueue()); + doReturn(set).when(mQAdminImpl).fetchSubscribeMessageQueues(anyString()); + Thread.sleep(11 * 1000); + assertThat(flag).isTrue(); + } + + @Test + public void testFlowControl_Success() throws Exception { + DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); + try { + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.setPullThresholdForAll(-1); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + litePullConsumer.setPollTimeoutMillis(500); + List result = litePullConsumer.poll(); + assertThat(result).isEmpty(); + } finally { + litePullConsumer.shutdown(); + } + + litePullConsumer = createStartLitePullConsumer(); + try { + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.setPullThresholdForQueue(-1); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + litePullConsumer.setPollTimeoutMillis(500); + List result = litePullConsumer.poll(); + assertThat(result).isEmpty(); + } finally { + litePullConsumer.shutdown(); + } + + litePullConsumer = createStartLitePullConsumer(); + try { + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.setPullThresholdSizeForQueue(-1); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + litePullConsumer.setPollTimeoutMillis(500); + List result = litePullConsumer.poll(); + assertThat(result).isEmpty(); + } finally { + litePullConsumer.shutdown(); + } + + litePullConsumer = createStartLitePullConsumer(); + try { + MessageQueue messageQueue = createMessageQueue(); + litePullConsumer.setConsumeMaxSpan(-1); + litePullConsumer.assign(Collections.singletonList(messageQueue)); + litePullConsumer.setPollTimeoutMillis(500); + List result = litePullConsumer.poll(); + assertThat(result).isEmpty(); + } finally { + litePullConsumer.shutdown(); + } + } + + @Test + public void testCheckConfig_Exception() { + DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer(MixAll.DEFAULT_CONSUMER_GROUP); + try { + litePullConsumer.start(); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("consumerGroup can not equal"); + } finally { + litePullConsumer.shutdown(); + } + + litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis()); + litePullConsumer.setMessageModel(null); + try { + litePullConsumer.start(); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("messageModel is null"); + } finally { + litePullConsumer.shutdown(); + } + + litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis()); + litePullConsumer.setAllocateMessageQueueStrategy(null); + try { + litePullConsumer.start(); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("allocateMessageQueueStrategy is null"); + } finally { + litePullConsumer.shutdown(); + } + + litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis()); + litePullConsumer.setConsumerTimeoutMillisWhenSuspend(1); + try { + litePullConsumer.start(); + failBecauseExceptionWasNotThrown(MQClientException.class); + } catch (MQClientException e) { + assertThat(e).hasMessageContaining("Long polling mode, the consumer consumerTimeoutMillisWhenSuspend must greater than brokerSuspendMaxTimeMillis"); + } finally { + litePullConsumer.shutdown(); + } + + } + + private void initDefaultLitePullConsumer(DefaultLitePullConsumer litePullConsumer) throws Exception { + + Field field = DefaultLitePullConsumer.class.getDeclaredField("defaultLitePullConsumerImpl"); + field.setAccessible(true); + litePullConsumerImpl = (DefaultLitePullConsumerImpl) field.get(litePullConsumer); + field = DefaultLitePullConsumerImpl.class.getDeclaredField("mQClientFactory"); + field.setAccessible(true); + field.set(litePullConsumerImpl, mQClientFactory); + + PullAPIWrapper pullAPIWrapper = litePullConsumerImpl.getPullAPIWrapper(); + field = PullAPIWrapper.class.getDeclaredField("mQClientFactory"); + field.setAccessible(true); + field.set(pullAPIWrapper, mQClientFactory); + + field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); + field.setAccessible(true); + field.set(mQClientFactory, mQClientAPIImpl); + + field = MQClientInstance.class.getDeclaredField("mQAdminImpl"); + field.setAccessible(true); + field.set(mQClientFactory, mQAdminImpl); + + field = DefaultLitePullConsumerImpl.class.getDeclaredField("rebalanceImpl"); + field.setAccessible(true); + rebalanceImpl = (RebalanceImpl) field.get(litePullConsumerImpl); + field = RebalanceImpl.class.getDeclaredField("mQClientFactory"); + field.setAccessible(true); + field.set(rebalanceImpl, mQClientFactory); + + offsetStore = spy(litePullConsumerImpl.getOffsetStore()); + field = DefaultLitePullConsumerImpl.class.getDeclaredField("offsetStore"); + field.setAccessible(true); + field.set(litePullConsumerImpl, offsetStore); + + when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), + anyLong(), any(CommunicationMode.class), nullable(PullCallback.class))) + .thenAnswer(new Answer() { + @Override + public Object 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)); + return pullResult; + } + }); + + when(mQClientFactory.findBrokerAddressInSubscribe(anyString(), anyLong(), anyBoolean())).thenReturn(new FindBrokerResult("127.0.0.1:10911", false)); + + doReturn(Collections.singletonList(mQClientFactory.getClientId())).when(mQClientFactory).findConsumerIdList(anyString(), anyString()); + + doReturn(123L).when(offsetStore).readOffset(any(MessageQueue.class), any(ReadOffsetType.class)); + } + + private DefaultLitePullConsumer createSubscribeLitePullConsumer() throws Exception { + DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis()); + litePullConsumer.setNamesrvAddr("127.0.0.1:9876"); + litePullConsumer.subscribe(topic, "*"); + litePullConsumer.start(); + initDefaultLitePullConsumer(litePullConsumer); + return litePullConsumer; + } + + private DefaultLitePullConsumer createStartLitePullConsumer() throws Exception { + DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis()); + litePullConsumer.setNamesrvAddr("127.0.0.1:9876"); + litePullConsumer.start(); + initDefaultLitePullConsumer(litePullConsumer); + return litePullConsumer; + } + + private DefaultLitePullConsumer createNotStartLitePullConsumer() { + DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis()); + return litePullConsumer; + } + + private DefaultLitePullConsumer createBroadcastLitePullConsumer() throws Exception { + DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis()); + litePullConsumer.setNamesrvAddr("127.0.0.1:9876"); + litePullConsumer.setMessageModel(MessageModel.BROADCASTING); + litePullConsumer.subscribe(topic, "*"); + litePullConsumer.start(); + initDefaultLitePullConsumer(litePullConsumer); + return litePullConsumer; + } + + private MessageQueue createMessageQueue() { + MessageQueue messageQueue = new MessageQueue(); + messageQueue.setBrokerName(brokerName); + messageQueue.setQueueId(0); + messageQueue.setTopic(topic); + return messageQueue; + } + + 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()); + } +} diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumerTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumerTest.java index 569055dea2566068b0ca165d893b710dcd08b751..7afaf2bea05738779d6379effaa0d358c3c4a16a 100644 --- a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumerTest.java @@ -54,7 +54,7 @@ import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) public class DefaultMQPullConsumerTest { @Spy - private MQClientInstance mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); @Mock private MQClientAPIImpl mQClientAPIImpl; private DefaultMQPullConsumer pullConsumer; 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 ff2fb78bbfb9df89596abc2ad7b35fcc33a2df3e..e6f0e866882ec460ba4505d007fca8e0ebfd995c 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 @@ -59,8 +59,10 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.invocation.InvocationOnMock; -import org.mockito.junit.MockitoJUnitRunner; import org.mockito.stubbing.Answer; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown; @@ -73,7 +75,8 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; -@RunWith(MockitoJUnitRunner.class) +@RunWith(PowerMockRunner.class) +@PrepareForTest(DefaultMQPushConsumerImpl.class) public class DefaultMQPushConsumerTest { private String consumerGroup; private String topic = "FooBar"; @@ -102,10 +105,12 @@ public class DefaultMQPushConsumerTest { }); DefaultMQPushConsumerImpl pushConsumerImpl = pushConsumer.getDefaultMQPushConsumerImpl(); + PowerMockito.suppress(PowerMockito.method(DefaultMQPushConsumerImpl.class, "updateTopicSubscribeInfoWhenSubscriptionChanged")); rebalancePushImpl = spy(new RebalancePushImpl(pushConsumer.getDefaultMQPushConsumerImpl())); Field field = DefaultMQPushConsumerImpl.class.getDeclaredField("rebalanceImpl"); field.setAccessible(true); field.set(pushConsumerImpl, rebalancePushImpl); + pushConsumer.subscribe(topic, "*"); pushConsumer.start(); diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/factory/MQClientInstanceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/factory/MQClientInstanceTest.java index 171a95a8b68f520c1c9e833ad85fc66880b2cd35..bb2132111c179e8fb13dc283aa4634f2313e6040 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/factory/MQClientInstanceTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/factory/MQClientInstanceTest.java @@ -39,7 +39,7 @@ import static org.mockito.Mockito.mock; @RunWith(MockitoJUnitRunner.class) public class MQClientInstanceTest { - private MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private String topic = "FooBar"; private String group = "FooBarGroup"; 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 6f2f959632b7d71990307ddcffb8add1ef1f4896..941ee73f42183b5beb2eca4e62be0484d56fcf0d 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 @@ -70,7 +70,7 @@ import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) public class DefaultMQProducerTest { @Spy - private MQClientInstance mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); @Mock private MQClientAPIImpl mQClientAPIImpl; @Mock 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 0d00c9bdb448841226c093e1cf0d00d532c62104..496c5143e02387f5a39289f8a77ce61021fbd927 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 @@ -70,8 +70,10 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.invocation.InvocationOnMock; -import org.mockito.junit.MockitoJUnitRunner; import org.mockito.stubbing.Answer; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; @@ -83,7 +85,8 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; -@RunWith(MockitoJUnitRunner.class) +@RunWith(PowerMockRunner.class) +@PrepareForTest(DefaultMQPushConsumerImpl.class) public class DefaultMQConsumerWithTraceTest { private String consumerGroup; private String consumerGroupNormal; @@ -101,7 +104,6 @@ public class DefaultMQConsumerWithTraceTest { private DefaultMQPushConsumer normalPushConsumer; private DefaultMQPushConsumer customTraceTopicpushConsumer; - private AsyncTraceDispatcher asyncTraceDispatcher; private MQClientInstance mQClientTraceFactory; @Mock @@ -112,17 +114,16 @@ public class DefaultMQConsumerWithTraceTest { @Before public void init() throws Exception { consumerGroup = "FooBarGroup" + System.currentTimeMillis(); - pushConsumer = new DefaultMQPushConsumer(consumerGroup,true,""); + pushConsumer = new DefaultMQPushConsumer(consumerGroup, true, ""); consumerGroupNormal = "FooBarGroup" + System.currentTimeMillis(); - normalPushConsumer = new DefaultMQPushConsumer(consumerGroupNormal,false,""); - customTraceTopicpushConsumer = new DefaultMQPushConsumer(consumerGroup,true,customerTraceTopic); + normalPushConsumer = new DefaultMQPushConsumer(consumerGroupNormal, false, ""); + customTraceTopicpushConsumer = new DefaultMQPushConsumer(consumerGroup, true, customerTraceTopic); pushConsumer.setNamesrvAddr("127.0.0.1:9876"); pushConsumer.setPullInterval(60 * 1000); - asyncTraceDispatcher = (AsyncTraceDispatcher)pushConsumer.getTraceDispatcher(); + asyncTraceDispatcher = (AsyncTraceDispatcher) pushConsumer.getTraceDispatcher(); traceProducer = asyncTraceDispatcher.getTraceProducer(); - pushConsumer.registerMessageListener(new MessageListenerConcurrently() { @Override public ConsumeConcurrentlyStatus consumeMessage(List msgs, @@ -131,12 +132,14 @@ public class DefaultMQConsumerWithTraceTest { } }); + PowerMockito.suppress(PowerMockito.method(DefaultMQPushConsumerImpl.class, "updateTopicSubscribeInfoWhenSubscriptionChanged")); 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, "*"); + pushConsumer.start(); mQClientFactory = spy(pushConsumerImpl.getmQClientFactory()); 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 903be01cddbf13031368db73daa9df8ab6198ebb..3759acba139f9d7eb3d343855acbc27e338512b7 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 @@ -60,7 +60,7 @@ import static org.mockito.Mockito.when; public class DefaultMQProducerWithTraceTest { @Spy - private MQClientInstance mQClientFactory = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); @Mock private MQClientAPIImpl mQClientAPIImpl; @@ -87,7 +87,7 @@ public class DefaultMQProducerWithTraceTest { producer.setNamesrvAddr("127.0.0.1:9876"); normalProducer.setNamesrvAddr("127.0.0.1:9877"); customTraceTopicproducer.setNamesrvAddr("127.0.0.1:9878"); - message = new Message(topic, new byte[]{'a', 'b', 'c'}); + message = new Message(topic, new byte[] {'a', 'b', 'c'}); asyncTraceDispatcher = (AsyncTraceDispatcher) producer.getTraceDispatcher(); asyncTraceDispatcher.setTraceTopicName(customerTraceTopic); asyncTraceDispatcher.getHostProducer(); @@ -108,14 +108,13 @@ public class DefaultMQProducerWithTraceTest { field.setAccessible(true); field.set(mQClientFactory, mQClientAPIImpl); - 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(); + nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))).thenCallRealMethod(); when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class), - nullable(SendCallback.class), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))) - .thenReturn(createSendResult(SendStatus.SEND_OK)); + nullable(SendCallback.class), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class))) + .thenReturn(createSendResult(SendStatus.SEND_OK)); } diff --git a/docs/cn/README.md b/docs/cn/README.md index c8bdc8dc0d3f1ac460b551e213caf8cdb77a948d..3247985e536236592a822ee4c8f28bc44e705480 100644 --- a/docs/cn/README.md +++ b/docs/cn/README.md @@ -19,7 +19,7 @@ ### 3. 样例 -- [样例(Example)](RocketMQ_Example.md) :介绍RocketMQ的常见用法,包括基本样例、顺序消息样例、延时消息样例、批量消息样例、过滤消息样例、事物消息样例等。 +- [样例(Example)](RocketMQ_Example.md) :介绍RocketMQ的常见用法,包括基本样例、顺序消息样例、延时消息样例、批量消息样例、过滤消息样例、事务消息样例等。 ### 4. 最佳实践 diff --git a/docs/cn/concept.md b/docs/cn/concept.md index b486b38195d68a48a32ded7afe061c1db0216eea..8ad0be6da808a843b8c5fcdf3a03b00c1bc8032f 100644 --- a/docs/cn/concept.md +++ b/docs/cn/concept.md @@ -26,7 +26,7 @@ RocketMQ主要由 Producer、Broker、Consumer 三部分组成,其中Producer Consumer消费的一种类型,该模式下Broker收到数据后会主动推送给消费端,该消费模式一般实时性较高。 ## 9 生产者组(Producer Group) - 同一类Producer的集合,这类Producer发送同一类消息且发送逻辑一致。如果发送的是事物消息且原始生产者在发送之后崩溃,则Broker服务器会联系同一生产者组的其他生产者实例以提交或回溯消费。 + 同一类Producer的集合,这类Producer发送同一类消息且发送逻辑一致。如果发送的是事务消息且原始生产者在发送之后崩溃,则Broker服务器会联系同一生产者组的其他生产者实例以提交或回溯消费。 ## 10 消费者组(Consumer Group) 同一类Consumer的集合,这类Consumer通常消费同一类消息且消费逻辑一致。消费者组使得在消息消费方面,实现负载均衡和容错的目标变得非常容易。要注意的是,消费者组的消费者实例必须订阅完全相同的Topic。RocketMQ 支持两种消息模式:集群消费(Clustering)和广播消费(Broadcasting)。 diff --git a/docs/cn/operation.md b/docs/cn/operation.md index ae85a30b2320d5e0387ab31202d657b28cf966bc..e35bd6812dbf117c268382873ae2d9cbfe3e11be 100644 --- a/docs/cn/operation.md +++ b/docs/cn/operation.md @@ -1033,30 +1033,6 @@ $ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker -s 是否执行jstack - - getConsumerStatus - 获取 Consumer 消费进度 - -g - 消费者所属组名 - - - -t - 查询主题 - - - -i - Consumer 客户端 ip - - - -n - NameServer 服务地址,格式 ip:port - - - -h - 打印帮助 - updateSubGroup diff --git a/docs/en/Configuration_Client.md b/docs/en/Configuration_Client.md index 4b6d2fe39a536c0057c925e634961b66b613777e..dedb4240b7cf2ca614d665d351c3c7e7ea9aa42c 100644 --- a/docs/en/Configuration_Client.md +++ b/docs/en/Configuration_Client.md @@ -25,7 +25,7 @@ export NAMESRV_ADDR=192.168.0.1:9876;192.168.0.2:9876 ``` - HTTP static server addressing(default) -After client started, it will access a http static server address, as: , this URL return the following contents: +After client started, it will access the http static server address, as: , this URL return the following contents: ```text 192.168.0.1:9876;192.168.0.2:9876 diff --git a/example/src/main/java/org/apache/rocketmq/example/broadcast/PushConsumer.java b/example/src/main/java/org/apache/rocketmq/example/broadcast/PushConsumer.java index fb1f9bbde7110fe0b9a500e771cf70a8fe4f0c95..28e02341c69b6ea96a16d08cb22b0b06480c8a82 100644 --- a/example/src/main/java/org/apache/rocketmq/example/broadcast/PushConsumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/broadcast/PushConsumer.java @@ -50,4 +50,4 @@ public class PushConsumer { consumer.start(); System.out.printf("Broadcast Consumer Started.%n"); } -} +} \ No newline at end of file diff --git a/example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerAssign.java b/example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerAssign.java new file mode 100644 index 0000000000000000000000000000000000000000..e638de1c960ba9fd37a9f16515161d2aa3afe712 --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerAssign.java @@ -0,0 +1,53 @@ +/* + * 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 java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; + +public class LitePullConsumerAssign { + + public static volatile boolean running = true; + + public static void main(String[] args) throws Exception { + DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer("please_rename_unique_group_name"); + litePullConsumer.setAutoCommit(false); + litePullConsumer.start(); + Collection mqSet = litePullConsumer.fetchMessageQueues("TopicTest"); + List list = new ArrayList<>(mqSet); + List assignList = new ArrayList<>(); + for (int i = 0; i < list.size() / 2; i++) { + assignList.add(list.get(i)); + } + litePullConsumer.assign(assignList); + litePullConsumer.seek(assignList.get(0), 10); + try { + while (running) { + List messageExts = litePullConsumer.poll(); + System.out.printf("%s %n", messageExts); + litePullConsumer.commitSync(); + } + } finally { + litePullConsumer.shutdown(); + } + + } +} diff --git a/example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerSubscribe.java b/example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerSubscribe.java new file mode 100644 index 0000000000000000000000000000000000000000..1bfe49d7365e0460e3ef4bc3b290596a1f66867b --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerSubscribe.java @@ -0,0 +1,40 @@ +/* + * 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 java.util.List; +import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; +import org.apache.rocketmq.common.message.MessageExt; + +public class LitePullConsumerSubscribe { + + public static volatile boolean running = true; + + public static void main(String[] args) throws Exception { + DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer("please_rename_unique_group_name"); + litePullConsumer.subscribe("TopicTest", "*"); + litePullConsumer.start(); + try { + while (running) { + List messageExts = litePullConsumer.poll(); + System.out.printf("%s%n", messageExts); + } + } finally { + litePullConsumer.shutdown(); + } + } +} diff --git a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessorTest.java b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessorTest.java index a0e8137524afe2488663fa33b8ac36f667ce4db6..c8bf057606d6507c1c0ef434d37a02a332488a1b 100644 --- a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessorTest.java +++ b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/ClusterTestRequestProcessorTest.java @@ -52,7 +52,7 @@ import static org.mockito.Mockito.when; public class ClusterTestRequestProcessorTest { private ClusterTestRequestProcessor clusterTestProcessor; private DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private MQClientAPIImpl mQClientAPIImpl; private ChannelHandlerContext ctx; diff --git a/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/LocalMessageCache.java b/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/LocalMessageCache.java index 93e60a738e94a6e041dd8f6f0c98749b3c5a5159..d2fc6781654a5ef7207a533de1835cb4de252f1b 100644 --- a/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/LocalMessageCache.java +++ b/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/LocalMessageCache.java @@ -73,7 +73,7 @@ class LocalMessageCache implements ServiceLifecycle { pullOffsetTable.putIfAbsent(remoteQueue, rocketmqPullConsumer.fetchConsumeOffset(remoteQueue, false)); } catch (MQClientException e) { - log.error("A error occurred in fetch consume offset process.", e); + log.error("An error occurred in fetch consume offset process.", e); } } return pullOffsetTable.get(remoteQueue); @@ -124,7 +124,7 @@ class LocalMessageCache implements ServiceLifecycle { try { rocketmqPullConsumer.updateConsumeOffset(consumeRequest.getMessageQueue(), offset); } catch (MQClientException e) { - log.error("A error occurred in update consume offset process.", e); + log.error("An error occurred in update consume offset process.", e); } } } @@ -135,7 +135,7 @@ class LocalMessageCache implements ServiceLifecycle { try { rocketmqPullConsumer.updateConsumeOffset(messageQueue, offset); } catch (MQClientException e) { - log.error("A error occurred in update consume offset process.", e); + log.error("An error occurred in update consume offset process.", e); } } diff --git a/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/PullConsumerImpl.java b/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/PullConsumerImpl.java index d6735100ce2cb4a6496daff3d4ff55c8939657f8..945ecfac8551c670293598a6a53b04710a2472ed 100644 --- a/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/PullConsumerImpl.java +++ b/openmessaging/src/main/java/io/openmessaging/rocketmq/consumer/PullConsumerImpl.java @@ -167,7 +167,7 @@ public class PullConsumerImpl implements PullConsumer { } localMessageCache.updatePullOffset(mq, pullResult.getNextBeginOffset()); } catch (Exception e) { - log.error("A error occurred in pull message process.", e); + log.error("An error occurred in pull message process.", e); } } }); diff --git a/pom.xml b/pom.xml index bdc44062ec1221044bbf74ef6fc7a6584b252138..46b88ceb04c88ca4f506b1a7b5a5972f4a79a26b 100644 --- a/pom.xml +++ b/pom.xml @@ -106,6 +106,7 @@ ${project.basedir}/../test/target/jacoco-it.exec file:**/generated-sources/**,**/test/** + 2.0.2 @@ -458,6 +459,18 @@ 2.23.0 test + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-mockito2 + ${powermock.version} + test + diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/exception/RemotingConnectException.java b/remoting/src/main/java/org/apache/rocketmq/remoting/exception/RemotingConnectException.java index 8286177c2efb15b6341cd22771f5a31339650b93..40e3b5aeefbe43db9421baf2dc143809dd9bc053 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/exception/RemotingConnectException.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/exception/RemotingConnectException.java @@ -24,6 +24,6 @@ public class RemotingConnectException extends RemotingException { } public RemotingConnectException(String addr, Throwable cause) { - super("connect to <" + addr + "> failed", cause); + super("connect to " + addr + " failed", cause); } } 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 fc9df37c652dcaef17b95a77bf2bffdaf60c9ba7..415d4956b25c5da5897bb9f25e644f0867078919 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 @@ -358,8 +358,6 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti } } - - @Override public RemotingCommand invokeSync(String addr, final RemotingCommand request, long timeoutMillis) throws InterruptedException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException { @@ -393,7 +391,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti } } - private Channel getAndCreateChannel(final String addr) throws InterruptedException { + private Channel getAndCreateChannel(final String addr) throws RemotingConnectException, InterruptedException { if (null == addr) { return getAndCreateNameserverChannel(); } @@ -406,7 +404,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti return this.createChannel(addr); } - private Channel getAndCreateNameserverChannel() throws InterruptedException { + private Channel getAndCreateNameserverChannel() throws RemotingConnectException, InterruptedException { String addr = this.namesrvAddrChoosed.get(); if (addr != null) { ChannelWrapper cw = this.channelTables.get(addr); @@ -440,9 +438,8 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti return channelNew; } } + throw new RemotingConnectException(addrList.toString()); } - } catch (Exception e) { - log.error("getAndCreateNameserverChannel: create name server channel exception", e); } finally { this.lockNamesrvChannel.unlock(); } @@ -456,8 +453,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti private Channel createChannel(final String addr) throws InterruptedException { ChannelWrapper cw = this.channelTables.get(addr); if (cw != null && cw.isOK()) { - cw.getChannel().close(); - channelTables.remove(addr); + return cw.getChannel(); } if (this.lockChannelTables.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) { @@ -467,9 +463,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti if (cw != null) { if (cw.isOK()) { - cw.getChannel().close(); - this.channelTables.remove(addr); - createNewConnection = true; + return cw.getChannel(); } else if (!cw.getChannelFuture().isDone()) { createNewConnection = false; } else { @@ -587,7 +581,6 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti return channelEventListener; } - @Override public ExecutorService getCallbackExecutor() { return callbackExecutor != null ? callbackExecutor : publicExecutor; 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 32d169b2656016ebdf565891d3d10c16b1890c40..f94d5ba9ca59c4b4bda96aad560ed26b77412a0c 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 @@ -373,7 +373,7 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti switch (tlsMode) { case DISABLED: ctx.close(); - log.warn("Clients intend to establish a SSL connection while this server is running in SSL disabled mode"); + log.warn("Clients intend to establish an SSL connection while this server is running in SSL disabled mode"); break; case PERMISSIVE: case ENFORCING: @@ -384,7 +384,7 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti log.info("Handlers prepended to channel pipeline to establish SSL connection"); } else { ctx.close(); - log.error("Trying to establish a SSL connection but sslContext is null"); + log.error("Trying to establish an SSL connection but sslContext is null"); } break; 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 a63d3746b30663e562778201f6c2ee4dd1365b17..87ff0a096cf57a7f0d1ea7a82a4f3e94a262e374 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.store.config.BrokerRole; import org.apache.rocketmq.store.config.StorePathConfigHelper; public class ConsumeQueue { @@ -397,6 +398,10 @@ public class ConsumeQueue { boolean result = this.putMessagePositionInfo(request.getCommitLogOffset(), request.getMsgSize(), tagsCode, request.getConsumeQueueOffset()); if (result) { + if (this.defaultMessageStore.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE || + this.defaultMessageStore.getMessageStoreConfig().isEnableDLegerCommitLog()) { + this.defaultMessageStore.getStoreCheckpoint().setPhysicMsgTimestamp(request.getStoreTimestamp()); + } this.defaultMessageStore.getStoreCheckpoint().setLogicsMsgTimestamp(request.getStoreTimestamp()); return; } else { 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 210d5a997d4d5d4c7171d71a8e1af91b47b12e99..0051ceb52bd7faec0a8d852455ca73864aea5186 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 @@ -116,7 +116,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner { this.defaultMQAdminExt.changeInstanceNameToPID(); - this.mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(this.defaultMQAdminExt, rpcHook); + this.mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(this.defaultMQAdminExt, rpcHook); boolean registerOK = mqClientInstance.registerAdminExt(this.defaultMQAdminExt.getAdminExtGroup(), this); if (!registerOK) { diff --git a/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java index 78659800219742a7b2cf19cf188679d2584b13b8..3146b1781154792a2d7928e0306e9cf172b5ddac 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtTest.java @@ -85,7 +85,7 @@ import static org.mockito.Mockito.when; public class DefaultMQAdminExtTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; private static Properties properties = new Properties(); private static TopicList topicList = new TopicList(); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java index 33b449768d105b6d223c84d6b46ae5d22beba916..b556e5c1975bf5f2ccee6cccfa9d0a3548c9060b 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/CommandUtilTest.java @@ -48,7 +48,7 @@ import static org.mockito.Mockito.when; public class CommandUtilTest { private DefaultMQAdminExt defaultMQAdminExt; private DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private MQClientAPIImpl mQClientAPIImpl; @Before diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommadTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommadTest.java index 1089c1b70cd7475503662c9a8c71a78a7e7dfd6c..1abd8575b0d2817f045fe3bb563e05a8c36c5a24 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommadTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommadTest.java @@ -39,7 +39,6 @@ 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.ArgumentMatchers.anyBoolean; @@ -54,7 +53,7 @@ public class BrokerConsumeStatsSubCommadTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 b1d92d79a3fc8ae24f7c3912cc8bbf7ad5e75f24..c850d71d881534d4006645c494fd4731814b47e4 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 @@ -48,7 +48,7 @@ import static org.mockito.Mockito.when; public class BrokerStatusSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 a5ba24f6503187a81a0230c510e9e37ae9dccd42..241ae8829abfb28349e9d2fc537bf29dc4c6fe8b 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 @@ -46,7 +46,7 @@ import static org.mockito.Mockito.when; public class CleanExpiredCQSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 95373800727c5a3f3209ecfb0978c66da41a2fcf..759f783e49522b9ac5a721986fd2d66e506a936c 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 @@ -46,7 +46,7 @@ import static org.mockito.Mockito.when; public class CleanUnusedTopicCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 978a2fd59616c19347c3d6186f7ea58fcf9f519a..8bb40793790312e7a3bc36feb1dd4a4a11175fe6 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 @@ -48,7 +48,7 @@ import static org.mockito.Mockito.when; public class GetBrokerConfigCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/SendMsgStatusCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/SendMsgStatusCommandTest.java index c0f7639d93dfdcc1a5e2a6ccb96a30b399956d47..9e9bc789d93eb0c95169bf44b01bdaaac40cfddc 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/SendMsgStatusCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/SendMsgStatusCommandTest.java @@ -41,7 +41,7 @@ import static org.mockito.Mockito.mock; public class SendMsgStatusCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 46c6eb3e74596f0d48a2469b7acd8de404a6e6b5..c74107edf921be37b0af4b818a3e0b307d74a2bd 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 @@ -44,7 +44,7 @@ import static org.mockito.Mockito.mock; public class UpdateBrokerConfigSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 3d64ce2422cee9f6342576cbaad91fb397432dbe..584943ce442ad29d105a2dd9a682cec2b07ddeed 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 @@ -54,7 +54,7 @@ import static org.mockito.Mockito.when; public class ConsumerConnectionSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 76c8dc4b3ccc50a5363e59e15fabd7640332f240..060ba9383d5fc4fc3caef2f2d08275b0cf952d0e 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 @@ -49,7 +49,7 @@ import static org.mockito.Mockito.when; public class ProducerConnectionSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 6d69c10b5abbe752f2cfedb43177e6ca3de226d3..19d903cea6f8151313e68e566f977d7d65d66315 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 @@ -53,7 +53,7 @@ import static org.mockito.Mockito.when; public class ConsumerProgressSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 aa6f27de09f0881486338025453e63d72b541a52..7f44af8225ce6968623d639c7ad9f983ee96502b 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 @@ -65,7 +65,7 @@ import static org.mockito.Mockito.when; public class ConsumerStatusSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 e757608280ff8f912f95563e72a6734422712060..504b46567808c98bb77de104e13b5c5f37aeb9c4 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 @@ -61,7 +61,7 @@ public class QueryMsgByUniqueKeySubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; private static MQAdminImpl mQAdminImpl; 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 8163fd15bc939c1aa1f7a978df5d59d048ad5b60..dde80eb3e672165338cc6a50928bc19255c2975c 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 @@ -41,7 +41,6 @@ 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 org.mockito.ArgumentMatchers; @@ -52,7 +51,7 @@ import static org.mockito.Mockito.when; public class GetNamesrvConfigCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 af4deace0a9cd563944a336e8548f97f5d992797..c4edcafe6f2dc52c49a840b059f742989401b145 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 @@ -39,7 +39,7 @@ import static org.mockito.Mockito.mock; public class UpdateKvConfigCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 11711d08866e80f2f185fbfc6cef15b325f0203e..9befdf8948ab016ee920c43d9f307eae3de26818 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 @@ -49,7 +49,7 @@ import static org.mockito.Mockito.when; public class WipeWritePermSubCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 86454008c220cdcc5f4d88bc78a5cabfc5971a40..a01bf8167ae65c36046a97fa495686020a691002 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 @@ -47,7 +47,7 @@ import static org.mockito.Mockito.when; public class GetConsumerStatusCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass 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 b7af8c870b56d62a58738d16ee687d5cdebc1687..d73a996b3fbcd60a73f998cf2475340b5562bbb3 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 @@ -53,7 +53,7 @@ import static org.mockito.Mockito.when; public class ResetOffsetByTimeCommandTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; @BeforeClass diff --git a/tools/src/test/java/org/apache/rocketmq/tools/monitor/MonitorServiceTest.java b/tools/src/test/java/org/apache/rocketmq/tools/monitor/MonitorServiceTest.java index 4989a9b55ab37920a97415fbae24de81a7334300..57278b9b222fd61b0227c87ab4d2445f8da73408 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/monitor/MonitorServiceTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/monitor/MonitorServiceTest.java @@ -69,7 +69,7 @@ import static org.mockito.Mockito.when; public class MonitorServiceTest { private static DefaultMQAdminExt defaultMQAdminExt; private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; - private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()); private static MQClientAPIImpl mQClientAPIImpl; private static MonitorConfig monitorConfig; private static MonitorListener monitorListener;