未验证 提交 7848895e 编写于 作者: H Heng Du 提交者: GitHub

Merge branch 'develop' into enhanced_acl

dist: trusty
dist: bionic
notifications:
email:
......@@ -9,22 +9,38 @@ notifications:
language: java
jdk:
- oraclejdk8
matrix:
include:
# On OSX, run with default JDK only.
# - os: osx
# On Linux, run with specific JDKs only.
- os: linux
env: CUSTOM_JDK="oraclejdk8"
# On Linux we install latest OpenJDK 1.8 from Ubuntu repositories
- name: Linux x86_64
arch: amd64
- name: Linux aarch64
arch: arm64
cache:
directories:
- $HOME/.m2/repository
before_install:
- lscpu
- echo 'MAVEN_OPTS="$MAVEN_OPTS -Xmx1024m -XX:MaxPermSize=512m -XX:+BytecodeVerificationLocal"' >> ~/.mavenrc
- cat ~/.mavenrc
- if [[ "$TRAVIS_OS_NAME" == "osx" ]]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
- if [[ "$TRAVIS_OS_NAME" == "linux" ]]; then jdk_switcher use "$CUSTOM_JDK"; fi
install: |
if [[ "$TRAVIS_OS_NAME" == "linux" ]]; then
sudo apt update
sudo apt install -y openjdk-8-jdk maven
export JAVA_HOME="/usr/lib/jvm/java-8-openjdk-${TRAVIS_CPU_ARCH}/"
export PATH="$JAVA_HOME/bin:/usr/share/maven/bin:$PATH"
fi
before_script:
- java -version
- mvn -version
- ulimit -c unlimited
script:
- travis_retry mvn -B clean apache-rat:check
......
......@@ -15,7 +15,7 @@
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (properties) the power, direct or indirect, to cause the
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
......
Apache RocketMQ
Copyright 2016-2020 The Apache Software Foundation
Copyright 2016-2021 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
......@@ -6,7 +6,7 @@
[![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html)
[![Average time to resolve an issue](http://isitmaintained.com/badge/resolution/apache/rocketmq.svg)](http://isitmaintained.com/project/apache/rocketmq "Average time to resolve an issue")
[![Percentage of issues still open](http://isitmaintained.com/badge/open/apache/rocketmq.svg)](http://isitmaintained.com/project/apache/rocketmq "Percentage of issues still open")
![Twitter Follow](https://img.shields.io/twitter/follow/ApacheRocketMQ?style=social)
[![Twitter Follow](https://img.shields.io/twitter/follow/ApacheRocketMQ?style=social)](https://twitter.com/intent/follow?screen_name=ApacheRocketMQ)
**[Apache RocketMQ](https://rocketmq.apache.org) is a distributed messaging and streaming platform with low latency, high performance and reliability, trillion-level capacity and flexible scalability.**
......
......@@ -13,7 +13,7 @@
<parent>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId>
<version>4.8.0-SNAPSHOT</version>
<version>4.8.1-SNAPSHOT</version>
</parent>
<artifactId>rocketmq-acl</artifactId>
<name>rocketmq-acl ${project.version}</name>
......
......@@ -193,7 +193,7 @@ public class RemoteAddressStrategyFactory {
throw new AclException(String.format("RangeRemoteAddressStrategy netaddress examine scope Exception start is %s , end is %s", start, end));
}
}
return this.end > 0 ? true : false;
return this.end > 0;
}
private boolean ipv6Analysis(String[] strArray, int index) {
......
......@@ -13,7 +13,7 @@
<parent>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId>
<version>4.8.0-SNAPSHOT</version>
<version>4.8.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
......
......@@ -17,9 +17,9 @@
package org.apache.rocketmq.broker.mqtrace;
public interface SendMessageHook {
public String hookName();
String hookName();
public void sendMessageBefore(final SendMessageContext context);
void sendMessageBefore(final SendMessageContext context);
public void sendMessageAfter(final SendMessageContext context);
void sendMessageAfter(final SendMessageContext context);
}
......@@ -232,4 +232,20 @@ public class ConsumerOffsetManager extends ConfigManager {
}
}
public void removeOffset(final String group) {
Iterator<Entry<String, ConcurrentMap<Integer, Long>>> it = this.offsetTable.entrySet().iterator();
while (it.hasNext()) {
Entry<String, ConcurrentMap<Integer, Long>> next = it.next();
String topicAtGroup = next.getKey();
if (topicAtGroup.contains(group)) {
String[] arrays = topicAtGroup.split(TOPIC_GROUP_SEPARATOR);
if (arrays.length == 2 && group.equals(arrays[1])) {
it.remove();
log.warn("clean group offset {}", topicAtGroup);
}
}
}
}
}
......@@ -19,7 +19,6 @@ package org.apache.rocketmq.broker.out;
import java.io.UnsupportedEncodingException;
import java.util.ArrayList;
import java.util.List;
import java.util.Vector;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
......@@ -123,7 +122,7 @@ public class BrokerOuterAPI {
final int timeoutMills,
final boolean compressed) {
final List<RegisterBrokerResult> registerBrokerResultList = new Vector<>();
final List<RegisterBrokerResult> registerBrokerResultList = new CopyOnWriteArrayList<>();
List<String> nameServerAddressList = this.remotingClient.getNameServerAddressList();
if (nameServerAddressList != null && nameServerAddressList.size() > 0) {
......@@ -147,7 +146,7 @@ public class BrokerOuterAPI {
@Override
public void run() {
try {
RegisterBrokerResult result = registerBroker(namesrvAddr,oneway, timeoutMills,requestHeader,body);
RegisterBrokerResult result = registerBroker(namesrvAddr, oneway, timeoutMills, requestHeader, body);
if (result != null) {
registerBrokerResultList.add(result);
}
......
......@@ -714,6 +714,10 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
this.brokerController.getSubscriptionGroupManager().deleteSubscriptionGroupConfig(requestHeader.getGroupName());
if (requestHeader.isRemoveOffset()) {
this.brokerController.getConsumerOffsetManager().removeOffset(requestHeader.getGroupName());
}
if (this.brokerController.getBrokerConfig().isAutoDeleteUnusedStats()) {
this.brokerController.getBrokerStatsManager().onGroupDeleted(requestHeader.getGroupName());
}
......
......@@ -584,7 +584,7 @@ public class PullMessageProcessor extends AsyncNettyRequestProcessor implements
this.brokerController.getPullMessageExecutor().submit(new RequestTask(run, channel, request));
}
public void registerConsumeMessageHook(List<ConsumeMessageHook> sendMessageHookList) {
this.consumeMessageHookList = sendMessageHookList;
public void registerConsumeMessageHook(List<ConsumeMessageHook> consumeMessageHookList) {
this.consumeMessageHookList = consumeMessageHookList;
}
}
......@@ -251,28 +251,29 @@ public class ReplyMessageProcessor extends AbstractSendMessageProcessor implemen
// Failed
case CREATE_MAPEDFILE_FAILED:
log.info("create mapped file failed, server is busy or broken.");
log.warn("create mapped file failed, server is busy or broken.");
break;
case MESSAGE_ILLEGAL:
log.info(
log.warn(
"the message is illegal, maybe msg properties length limit 32k.");
break;
case PROPERTIES_SIZE_EXCEEDED:
log.info(
log.warn(
"the message is illegal, maybe msg body or properties length not matched. msg body length limit 128k.");
break;
case SERVICE_NOT_AVAILABLE:
log.info(
"service not available now, maybe disk full, maybe your broker machine memory too small.");
log.warn(
"service not available now. It may be caused by one of the following reasons: " +
"the broker's disk is full, messages are put to the slave, message store has been shut down, etc.");
break;
case OS_PAGECACHE_BUSY:
log.info("[PC_SYNCHRONIZED]broker busy, start flow control for a while");
log.warn("[PC_SYNCHRONIZED]broker busy, start flow control for a while");
break;
case UNKNOWN_ERROR:
log.info("UNKNOWN_ERROR");
log.warn("UNKNOWN_ERROR");
break;
default:
log.info("UNKNOWN_ERROR DEFAULT");
log.warn("UNKNOWN_ERROR DEFAULT");
break;
}
......
......@@ -218,6 +218,8 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
String originMsgId = MessageAccessor.getOriginMessageId(msgExt);
MessageAccessor.setOriginMessageId(msgInner, UtilAll.isBlank(originMsgId) ? msgExt.getMsgId() : originMsgId);
msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties()));
CompletableFuture<PutMessageResult> putMessageResult = this.brokerController.getMessageStore().asyncPutMessage(msgInner);
return putMessageResult.thenApply((r) -> {
if (r != null) {
......@@ -480,7 +482,8 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
case SERVICE_NOT_AVAILABLE:
response.setCode(ResponseCode.SERVICE_NOT_AVAILABLE);
response.setRemark(
"service not available now, maybe disk full, " + diskUtil() + ", maybe your broker machine memory too small.");
"service not available now. It may be caused by one of the following reasons: " +
"the broker's disk is full [" + diskUtil() + "], messages are put to the slave, message store has been shut down, etc.");
break;
case OS_PAGECACHE_BUSY:
response.setCode(ResponseCode.SYSTEM_ERROR);
......
......@@ -45,7 +45,7 @@ public class TopicConfigManager extends ConfigManager {
private static final long LOCK_TIMEOUT_MILLIS = 3000;
private static final int SCHEDULE_TOPIC_QUEUE_NUM = 18;
private transient final Lock lockTopicConfigTable = new ReentrantLock();
private transient final Lock topicConfigTableLock = new ReentrantLock();
private final ConcurrentMap<String, TopicConfig> topicConfigTable =
new ConcurrentHashMap<String, TopicConfig>(1024);
......@@ -159,7 +159,7 @@ public class TopicConfigManager extends ConfigManager {
boolean createNew = false;
try {
if (this.lockTopicConfigTable.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
if (this.topicConfigTableLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
try {
topicConfig = this.topicConfigTable.get(topic);
if (topicConfig != null)
......@@ -176,9 +176,7 @@ public class TopicConfigManager extends ConfigManager {
if (PermName.isInherited(defaultTopicConfig.getPerm())) {
topicConfig = new TopicConfig(topic);
int queueNums =
clientDefaultTopicQueueNums > defaultTopicConfig.getWriteQueueNums() ? defaultTopicConfig
.getWriteQueueNums() : clientDefaultTopicQueueNums;
int queueNums = Math.min(clientDefaultTopicQueueNums, defaultTopicConfig.getWriteQueueNums());
if (queueNums < 0) {
queueNums = 0;
......@@ -213,7 +211,7 @@ public class TopicConfigManager extends ConfigManager {
this.persist();
}
} finally {
this.lockTopicConfigTable.unlock();
this.topicConfigTableLock.unlock();
}
}
} catch (InterruptedException e) {
......@@ -239,7 +237,7 @@ public class TopicConfigManager extends ConfigManager {
boolean createNew = false;
try {
if (this.lockTopicConfigTable.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
if (this.topicConfigTableLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
try {
topicConfig = this.topicConfigTable.get(topic);
if (topicConfig != null)
......@@ -257,7 +255,7 @@ public class TopicConfigManager extends ConfigManager {
this.dataVersion.nextVersion();
this.persist();
} finally {
this.lockTopicConfigTable.unlock();
this.topicConfigTableLock.unlock();
}
}
} catch (InterruptedException e) {
......@@ -279,7 +277,7 @@ public class TopicConfigManager extends ConfigManager {
boolean createNew = false;
try {
if (this.lockTopicConfigTable.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
if (this.topicConfigTableLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
try {
topicConfig = this.topicConfigTable.get(TopicValidator.RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC);
if (topicConfig != null)
......@@ -297,7 +295,7 @@ public class TopicConfigManager extends ConfigManager {
this.dataVersion.nextVersion();
this.persist();
} finally {
this.lockTopicConfigTable.unlock();
this.topicConfigTableLock.unlock();
}
}
} catch (InterruptedException e) {
......
......@@ -164,7 +164,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
break;
}
if (removeMap.containsKey(i)) {
log.info("Half offset {} has been committed/rolled back", i);
log.debug("Half offset {} has been committed/rolled back", i);
Long removedOpOffset = removeMap.remove(i);
doneOpOffset.add(removedOpOffset);
} else {
......
......@@ -124,7 +124,7 @@ public class BrokerOuterAPITest {
boolean success = Iterables.any(booleanList,
new Predicate<Boolean>() {
public boolean apply(Boolean input) {
return input ? true : false;
return input;
}
});
......
......@@ -19,7 +19,7 @@
<parent>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId>
<version>4.8.0-SNAPSHOT</version>
<version>4.8.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
......@@ -47,6 +47,18 @@
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>io.opentracing</groupId>
<artifactId>opentracing-api</artifactId>
<version>0.33.0</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>io.opentracing</groupId>
<artifactId>opentracing-mock</artifactId>
<version>0.33.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-core</artifactId>
......
......@@ -94,7 +94,7 @@ public class ClientConfig {
public void changeInstanceNameToPID() {
if (this.instanceName.equals("DEFAULT")) {
this.instanceName = String.valueOf(UtilAll.getPid());
this.instanceName = UtilAll.getPid() + "#" + System.nanoTime();
}
}
......@@ -178,8 +178,8 @@ public class ClientConfig {
}
public String getNamesrvAddr() {
if (StringUtils.isNotEmpty(namesrvAddr) && NameServerAddressUtils.NAMESRV_ENDPOINT_PATTERN.matcher(namesrvAddr.trim()).matches()) {
return namesrvAddr.substring(NameServerAddressUtils.ENDPOINT_PREFIX.length());
if (StringUtils.isNotEmpty(namesrvAddr) && NameServerAddressUtils.validateInstanceEndpoint(namesrvAddr.trim())) {
return NameServerAddressUtils.getNameSrvAddrFromNamesrvEndpoint(namesrvAddr);
}
return namesrvAddr;
}
......
......@@ -23,19 +23,14 @@ public class ThreadLocalIndex {
private final ThreadLocal<Integer> threadLocalIndex = new ThreadLocal<Integer>();
private final Random random = new Random();
public int getAndIncrement() {
public int incrementAndGet() {
Integer index = this.threadLocalIndex.get();
if (null == index) {
index = Math.abs(random.nextInt());
if (index < 0)
index = 0;
this.threadLocalIndex.set(index);
}
index = Math.abs(index + 1);
if (index < 0)
index = 0;
this.threadLocalIndex.set(index);
return index;
}
......
......@@ -49,7 +49,7 @@ public class AllocateMessageQueueByMachineRoom implements AllocateMessageQueueSt
int startIndex = mod * currentIndex;
int endIndex = startIndex + mod;
for (int i = startIndex; i < endIndex; i++) {
result.add(mqAll.get(i));
result.add(premqAll.get(i));
}
if (rem > currentIndex) {
result.add(premqAll.get(currentIndex + mod * cidAll.size()));
......
/*
* 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.hook;
import org.apache.rocketmq.client.producer.LocalTransactionState;
import org.apache.rocketmq.common.message.Message;
public class EndTransactionContext {
private String producerGroup;
private Message message;
private String brokerAddr;
private String msgId;
private String transactionId;
private LocalTransactionState transactionState;
private boolean fromTransactionCheck;
public String getProducerGroup() {
return producerGroup;
}
public void setProducerGroup(String producerGroup) {
this.producerGroup = producerGroup;
}
public Message getMessage() {
return message;
}
public void setMessage(Message message) {
this.message = message;
}
public String getBrokerAddr() {
return brokerAddr;
}
public void setBrokerAddr(String brokerAddr) {
this.brokerAddr = brokerAddr;
}
public String getMsgId() {
return msgId;
}
public void setMsgId(String msgId) {
this.msgId = msgId;
}
public String getTransactionId() {
return transactionId;
}
public void setTransactionId(String transactionId) {
this.transactionId = transactionId;
}
public LocalTransactionState getTransactionState() {
return transactionState;
}
public void setTransactionState(LocalTransactionState transactionState) {
this.transactionState = transactionState;
}
public boolean isFromTransactionCheck() {
return fromTransactionCheck;
}
public void setFromTransactionCheck(boolean fromTransactionCheck) {
this.fromTransactionCheck = fromTransactionCheck;
}
}
/*
* 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.hook;
public interface EndTransactionHook {
String hookName();
void endTransaction(final EndTransactionContext context);
}
......@@ -282,8 +282,6 @@ public class ClientRemotingProcessor extends AsyncNettyRequestProcessor implemen
if (requestResponseFuture.getRequestCallback() != null) {
requestResponseFuture.getRequestCallback().onSuccess(replyMsg);
} else {
requestResponseFuture.putResponseMessage(replyMsg);
}
} else {
String bornHost = replyMsg.getBornHostString();
......
......@@ -1467,10 +1467,11 @@ public class MQClientAPIImpl {
throw new MQClientException(response.getCode(), response.getRemark());
}
public void deleteSubscriptionGroup(final String addr, final String groupName, final long timeoutMillis)
public void deleteSubscriptionGroup(final String addr, final String groupName, final boolean removeOffset, final long timeoutMillis)
throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
DeleteSubscriptionGroupRequestHeader requestHeader = new DeleteSubscriptionGroupRequestHeader();
requestHeader.setGroupName(groupName);
requestHeader.setRemoveOffset(removeOffset);
RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_SUBSCRIPTIONGROUP, requestHeader);
RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr),
......
......@@ -83,9 +83,12 @@ public class AssignedMessageQueue {
return -1;
}
public void updatePullOffset(MessageQueue messageQueue, long offset) {
public void updatePullOffset(MessageQueue messageQueue, long offset, ProcessQueue processQueue) {
MessageQueueState messageQueueState = assignedMessageQueueState.get(messageQueue);
if (messageQueueState != null) {
if (messageQueueState.getProcessQueue() != processQueue) {
return;
}
messageQueueState.setPullOffset(offset);
}
}
......
......@@ -478,7 +478,7 @@ public class ConsumeMessageOrderlyService implements ConsumeMessageService {
ConsumeReturnType returnType = ConsumeReturnType.SUCCESS;
boolean hasException = false;
try {
this.processQueue.getLockConsume().lock();
this.processQueue.getConsumeLock().lock();
if (this.processQueue.isDropped()) {
log.warn("consumeMessage, the message queue not be able to consume, because it's dropped. {}",
this.messageQueue);
......@@ -494,7 +494,7 @@ public class ConsumeMessageOrderlyService implements ConsumeMessageService {
messageQueue);
hasException = true;
} finally {
this.processQueue.getLockConsume().unlock();
this.processQueue.getConsumeLock().unlock();
}
if (null == status
......
......@@ -432,8 +432,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
throw new IllegalArgumentException("Topic can not be null or empty.");
}
setSubscriptionType(SubscriptionType.SUBSCRIBE);
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(defaultLitePullConsumer.getConsumerGroup(),
topic, subExpression);
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(topic, subExpression);
this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData);
this.defaultLitePullConsumer.setMessageQueueListener(new MessageQueueListenerImpl());
assignedMessageQueue.setRebalanceImpl(this.rebalanceImpl);
......@@ -613,9 +612,9 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
}
}
private void updatePullOffset(MessageQueue messageQueue, long nextPullOffset) {
private void updatePullOffset(MessageQueue messageQueue, long nextPullOffset, ProcessQueue processQueue) {
if (assignedMessageQueue.getSeekOffset(messageQueue) == -1) {
assignedMessageQueue.updatePullOffset(messageQueue, nextPullOffset);
assignedMessageQueue.updatePullOffset(messageQueue, nextPullOffset, processQueue);
}
}
......@@ -741,6 +740,9 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
}
long offset = nextPullOffset(messageQueue);
if (this.isCancelled() || processQueue.isDropped()) {
return;
}
long pullDelayTimeMills = 0;
try {
SubscriptionData subscriptionData;
......@@ -749,12 +751,13 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
subscriptionData = rebalanceImpl.getSubscriptionInner().get(topic);
} else {
String topic = this.messageQueue.getTopic();
subscriptionData = FilterAPI.buildSubscriptionData(defaultLitePullConsumer.getConsumerGroup(),
topic, SubscriptionData.SUB_ALL);
subscriptionData = FilterAPI.buildSubscriptionData(topic, SubscriptionData.SUB_ALL);
}
PullResult pullResult = pull(messageQueue, subscriptionData, offset, defaultLitePullConsumer.getPullBatchSize());
if (this.isCancelled() || processQueue.isDropped()) {
return;
}
switch (pullResult.getPullStatus()) {
case FOUND:
final Object objLock = messageQueueLock.fetchLockObject(messageQueue);
......@@ -771,7 +774,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
default:
break;
}
updatePullOffset(messageQueue, pullResult.getNextBeginOffset());
updatePullOffset(messageQueue, pullResult.getNextBeginOffset(), processQueue);
} catch (Throwable e) {
pullDelayTimeMills = pullTimeDelayMillsWhenException;
log.error("An error occurred in pull message process.", e);
......
......@@ -205,8 +205,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
}
try {
return FilterAPI.buildSubscriptionData(this.defaultMQPullConsumer.getConsumerGroup(),
mq.getTopic(), subExpression);
return FilterAPI.buildSubscriptionData(mq.getTopic(), subExpression);
} catch (Exception e) {
throw new MQClientException("parse subscription error", e);
}
......@@ -301,8 +300,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
public void subscriptionAutomatically(final String topic) {
if (!this.rebalanceImpl.getSubscriptionInner().containsKey(topic)) {
try {
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(this.defaultMQPullConsumer.getConsumerGroup(),
topic, SubscriptionData.SUB_ALL);
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(topic, SubscriptionData.SUB_ALL);
this.rebalanceImpl.subscriptionInner.putIfAbsent(topic, subscriptionData);
} catch (Exception ignore) {
}
......@@ -365,7 +363,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
for (String t : topics) {
SubscriptionData ms = null;
try {
ms = FilterAPI.buildSubscriptionData(this.groupName(), t, SubscriptionData.SUB_ALL);
ms = FilterAPI.buildSubscriptionData(t, SubscriptionData.SUB_ALL);
} catch (Exception e) {
log.error("parse subscription error", e);
}
......@@ -742,8 +740,7 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
Set<String> registerTopics = this.defaultMQPullConsumer.getRegisterTopics();
if (registerTopics != null) {
for (final String topic : registerTopics) {
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(this.defaultMQPullConsumer.getConsumerGroup(),
topic, SubscriptionData.SUB_ALL);
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(topic, SubscriptionData.SUB_ALL);
this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData);
}
}
......
......@@ -827,8 +827,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
for (final Map.Entry<String, String> entry : sub.entrySet()) {
final String topic = entry.getKey();
final String subString = entry.getValue();
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(this.defaultMQPushConsumer.getConsumerGroup(),
topic, subString);
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(topic, subString);
this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData);
}
}
......@@ -842,8 +841,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
break;
case CLUSTERING:
final String retryTopic = MixAll.getRetryTopic(this.defaultMQPushConsumer.getConsumerGroup());
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(this.defaultMQPushConsumer.getConsumerGroup(),
retryTopic, SubscriptionData.SUB_ALL);
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(retryTopic, SubscriptionData.SUB_ALL);
this.rebalanceImpl.getSubscriptionInner().put(retryTopic, subscriptionData);
break;
default:
......@@ -874,8 +872,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
public void subscribe(String topic, String subExpression) throws MQClientException {
try {
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(this.defaultMQPushConsumer.getConsumerGroup(),
topic, subExpression);
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(topic, subExpression);
this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData);
if (this.mQClientFactory != null) {
this.mQClientFactory.sendHeartbeatToAllBrokerWithLock();
......@@ -887,8 +884,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
public void subscribe(String topic, String fullClassName, String filterClassSource) throws MQClientException {
try {
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(this.defaultMQPushConsumer.getConsumerGroup(),
topic, "*");
SubscriptionData subscriptionData = FilterAPI.buildSubscriptionData(topic, "*");
subscriptionData.setSubString(fullClassName);
subscriptionData.setClassFilterMode(true);
subscriptionData.setFilterClassSource(filterClassSource);
......
......@@ -44,11 +44,11 @@ public class ProcessQueue {
public final static long REBALANCE_LOCK_INTERVAL = Long.parseLong(System.getProperty("rocketmq.client.rebalance.lockInterval", "20000"));
private final static long PULL_MAX_IDLE_TIME = Long.parseLong(System.getProperty("rocketmq.client.pull.pullMaxIdleTime", "120000"));
private final InternalLogger log = ClientLogger.getLog();
private final ReadWriteLock lockTreeMap = new ReentrantReadWriteLock();
private final ReadWriteLock treeMapLock = new ReentrantReadWriteLock();
private final TreeMap<Long, MessageExt> msgTreeMap = new TreeMap<Long, MessageExt>();
private final AtomicLong msgCount = new AtomicLong();
private final AtomicLong msgSize = new AtomicLong();
private final Lock lockConsume = new ReentrantLock();
private final Lock consumeLock = new ReentrantLock();
/**
* A subset of msgTreeMap, will only be used when orderly consume
*/
......@@ -83,7 +83,7 @@ public class ProcessQueue {
for (int i = 0; i < loop; i++) {
MessageExt msg = null;
try {
this.lockTreeMap.readLock().lockInterruptibly();
this.treeMapLock.readLock().lockInterruptibly();
try {
if (!msgTreeMap.isEmpty() && System.currentTimeMillis() - Long.parseLong(MessageAccessor.getConsumeStartTimeStamp(msgTreeMap.firstEntry().getValue())) > pushConsumer.getConsumeTimeout() * 60 * 1000) {
msg = msgTreeMap.firstEntry().getValue();
......@@ -92,7 +92,7 @@ public class ProcessQueue {
break;
}
} finally {
this.lockTreeMap.readLock().unlock();
this.treeMapLock.readLock().unlock();
}
} catch (InterruptedException e) {
log.error("getExpiredMsg exception", e);
......@@ -103,7 +103,7 @@ public class ProcessQueue {
pushConsumer.sendMessageBack(msg, 3);
log.info("send expire msg back. topic={}, msgId={}, storeHost={}, queueId={}, queueOffset={}", msg.getTopic(), msg.getMsgId(), msg.getStoreHost(), msg.getQueueId(), msg.getQueueOffset());
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
try {
if (!msgTreeMap.isEmpty() && msg.getQueueOffset() == msgTreeMap.firstKey()) {
try {
......@@ -113,7 +113,7 @@ public class ProcessQueue {
}
}
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (InterruptedException e) {
log.error("getExpiredMsg exception", e);
......@@ -127,7 +127,7 @@ public class ProcessQueue {
public boolean putMessage(final List<MessageExt> msgs) {
boolean dispatchToConsume = false;
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
try {
int validMsgCnt = 0;
for (MessageExt msg : msgs) {
......@@ -156,7 +156,7 @@ public class ProcessQueue {
}
}
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (InterruptedException e) {
log.error("putMessage exception", e);
......@@ -167,13 +167,13 @@ public class ProcessQueue {
public long getMaxSpan() {
try {
this.lockTreeMap.readLock().lockInterruptibly();
this.treeMapLock.readLock().lockInterruptibly();
try {
if (!this.msgTreeMap.isEmpty()) {
return this.msgTreeMap.lastKey() - this.msgTreeMap.firstKey();
}
} finally {
this.lockTreeMap.readLock().unlock();
this.treeMapLock.readLock().unlock();
}
} catch (InterruptedException e) {
log.error("getMaxSpan exception", e);
......@@ -186,7 +186,7 @@ public class ProcessQueue {
long result = -1;
final long now = System.currentTimeMillis();
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
this.lastConsumeTimestamp = now;
try {
if (!msgTreeMap.isEmpty()) {
......@@ -206,7 +206,7 @@ public class ProcessQueue {
}
}
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (Throwable t) {
log.error("removeMessage exception", t);
......@@ -245,12 +245,12 @@ public class ProcessQueue {
public void rollback() {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
try {
this.msgTreeMap.putAll(this.consumingMsgOrderlyTreeMap);
this.consumingMsgOrderlyTreeMap.clear();
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (InterruptedException e) {
log.error("rollback exception", e);
......@@ -259,7 +259,7 @@ public class ProcessQueue {
public long commit() {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
try {
Long offset = this.consumingMsgOrderlyTreeMap.lastKey();
msgCount.addAndGet(0 - this.consumingMsgOrderlyTreeMap.size());
......@@ -271,7 +271,7 @@ public class ProcessQueue {
return offset + 1;
}
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (InterruptedException e) {
log.error("commit exception", e);
......@@ -282,14 +282,14 @@ public class ProcessQueue {
public void makeMessageToConsumeAgain(List<MessageExt> msgs) {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
try {
for (MessageExt msg : msgs) {
this.consumingMsgOrderlyTreeMap.remove(msg.getQueueOffset());
this.msgTreeMap.put(msg.getQueueOffset(), msg);
}
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (InterruptedException e) {
log.error("makeMessageToCosumeAgain exception", e);
......@@ -300,7 +300,7 @@ public class ProcessQueue {
List<MessageExt> result = new ArrayList<MessageExt>(batchSize);
final long now = System.currentTimeMillis();
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
this.lastConsumeTimestamp = now;
try {
if (!this.msgTreeMap.isEmpty()) {
......@@ -319,7 +319,7 @@ public class ProcessQueue {
consuming = false;
}
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (InterruptedException e) {
log.error("take Messages exception", e);
......@@ -330,11 +330,11 @@ public class ProcessQueue {
public boolean hasTempMessage() {
try {
this.lockTreeMap.readLock().lockInterruptibly();
this.treeMapLock.readLock().lockInterruptibly();
try {
return !this.msgTreeMap.isEmpty();
} finally {
this.lockTreeMap.readLock().unlock();
this.treeMapLock.readLock().unlock();
}
} catch (InterruptedException e) {
}
......@@ -344,7 +344,7 @@ public class ProcessQueue {
public void clear() {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
this.treeMapLock.writeLock().lockInterruptibly();
try {
this.msgTreeMap.clear();
this.consumingMsgOrderlyTreeMap.clear();
......@@ -352,7 +352,7 @@ public class ProcessQueue {
this.msgSize.set(0);
this.queueOffsetMax = 0L;
} finally {
this.lockTreeMap.writeLock().unlock();
this.treeMapLock.writeLock().unlock();
}
} catch (InterruptedException e) {
log.error("rollback exception", e);
......@@ -367,8 +367,8 @@ public class ProcessQueue {
this.lastLockTimestamp = lastLockTimestamp;
}
public Lock getLockConsume() {
return lockConsume;
public Lock getConsumeLock() {
return consumeLock;
}
public long getLastPullTimestamp() {
......@@ -397,7 +397,7 @@ public class ProcessQueue {
public void fillProcessQueueInfo(final ProcessQueueInfo info) {
try {
this.lockTreeMap.readLock().lockInterruptibly();
this.treeMapLock.readLock().lockInterruptibly();
if (!this.msgTreeMap.isEmpty()) {
info.setCachedMsgMinOffset(this.msgTreeMap.firstKey());
......@@ -421,7 +421,7 @@ public class ProcessQueue {
info.setLastConsumeTimestamp(this.lastConsumeTimestamp);
} catch (Exception e) {
} finally {
this.lockTreeMap.readLock().unlock();
this.treeMapLock.readLock().unlock();
}
}
......
......@@ -88,11 +88,11 @@ public class RebalancePushImpl extends RebalanceImpl {
if (this.defaultMQPushConsumerImpl.isConsumeOrderly()
&& MessageModel.CLUSTERING.equals(this.defaultMQPushConsumerImpl.messageModel())) {
try {
if (pq.getLockConsume().tryLock(1000, TimeUnit.MILLISECONDS)) {
if (pq.getConsumeLock().tryLock(1000, TimeUnit.MILLISECONDS)) {
try {
return this.unlockDelay(mq, pq);
} finally {
pq.getLockConsume().unlock();
pq.getConsumeLock().unlock();
}
} else {
log.warn("[WRONG]mq is consuming, so can not unlock it, {}. maybe hanged for a while, {}",
......
......@@ -1043,6 +1043,11 @@ public class MQClientInstance {
slave = brokerId != MixAll.MASTER_ID;
found = brokerAddr != null;
if (!found && slave) {
brokerAddr = map.get(brokerId + 1);
found = brokerAddr != null;
}
if (!found && !onlyThisBroker) {
Entry<Long, String> entry = map.entrySet().iterator().next();
brokerAddr = entry.getValue();
......@@ -1100,7 +1105,7 @@ public class MQClientInstance {
return null;
}
public void resetOffset(String topic, String group, Map<MessageQueue, Long> offsetTable) {
public synchronized void resetOffset(String topic, String group, Map<MessageQueue, Long> offsetTable) {
DefaultMQPushConsumerImpl consumer = null;
try {
MQConsumerInner impl = this.consumerTable.get(group);
......@@ -1209,6 +1214,9 @@ public class MQClientInstance {
public ConsumerRunningInfo consumerRunningInfo(final String consumerGroup) {
MQConsumerInner mqConsumerInner = this.consumerTable.get(consumerGroup);
if (mqConsumerInner == null) {
return null;
}
ConsumerRunningInfo consumerRunningInfo = mqConsumerInner.consumerRunningInfo();
......
......@@ -44,6 +44,8 @@ import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.exception.RequestTimeoutException;
import org.apache.rocketmq.client.hook.CheckForbiddenContext;
import org.apache.rocketmq.client.hook.CheckForbiddenHook;
import org.apache.rocketmq.client.hook.EndTransactionContext;
import org.apache.rocketmq.client.hook.EndTransactionHook;
import org.apache.rocketmq.client.hook.SendMessageContext;
import org.apache.rocketmq.client.hook.SendMessageHook;
import org.apache.rocketmq.client.impl.CommunicationMode;
......@@ -101,6 +103,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
private final ConcurrentMap<String/* topic */, TopicPublishInfo> topicPublishInfoTable =
new ConcurrentHashMap<String, TopicPublishInfo>();
private final ArrayList<SendMessageHook> sendMessageHookList = new ArrayList<SendMessageHook>();
private final ArrayList<EndTransactionHook> endTransactionHookList = new ArrayList<EndTransactionHook>();
private final RPCHook rpcHook;
private final BlockingQueue<Runnable> asyncSenderThreadPoolQueue;
private final ExecutorService defaultAsyncSenderExecutor;
......@@ -171,6 +174,11 @@ public class DefaultMQProducerImpl implements MQProducerInner {
log.info("register sendMessage Hook, {}", hook.hookName());
}
public void registerEndTransactionHook(final EndTransactionHook hook) {
this.endTransactionHookList.add(hook);
log.info("register endTransaction Hook, {}", hook.hookName());
}
public void start() throws MQClientException {
this.start(true);
}
......@@ -386,6 +394,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
if (exception != null) {
remark = "checkLocalTransactionState Exception: " + RemotingHelper.exceptionSimpleDesc(exception);
}
doExecuteEndTransactionHook(msg, uniqueKey, brokerAddr, localTransactionState, true);
try {
DefaultMQProducerImpl.this.mQClientFactory.getMQClientAPIImpl().endTransactionOneway(brokerAddr, thisHeader, remark,
......@@ -967,6 +976,36 @@ public class DefaultMQProducerImpl implements MQProducerInner {
}
}
public boolean hasEndTransactionHook() {
return !this.endTransactionHookList.isEmpty();
}
public void executeEndTransactionHook(final EndTransactionContext context) {
if (!this.endTransactionHookList.isEmpty()) {
for (EndTransactionHook hook : this.endTransactionHookList) {
try {
hook.endTransaction(context);
} catch (Throwable e) {
log.warn("failed to executeEndTransactionHook", e);
}
}
}
}
public void doExecuteEndTransactionHook(Message msg, String msgId, String brokerAddr, LocalTransactionState state,
boolean fromTransactionCheck) {
if (hasEndTransactionHook()) {
EndTransactionContext context = new EndTransactionContext();
context.setProducerGroup(defaultMQProducer.getProducerGroup());
context.setBrokerAddr(brokerAddr);
context.setMessage(msg);
context.setMsgId(msgId);
context.setTransactionId(msg.getTransactionId());
context.setTransactionState(state);
context.setFromTransactionCheck(fromTransactionCheck);
executeEndTransactionHook(context);
}
}
/**
* DEFAULT ONEWAY -------------------------------------------------------
*/
......@@ -1266,7 +1305,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
}
try {
this.endTransaction(sendResult, localTransactionState, localException);
this.endTransaction(msg, sendResult, localTransactionState, localException);
} catch (Exception e) {
log.warn("local transaction execute " + localTransactionState + ", but end broker transaction failed", e);
}
......@@ -1290,6 +1329,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
}
public void endTransaction(
final Message msg,
final SendResult sendResult,
final LocalTransactionState localTransactionState,
final Throwable localException) throws RemotingException, MQBrokerException, InterruptedException, UnknownHostException {
......@@ -1318,6 +1358,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
break;
}
doExecuteEndTransactionHook(msg, sendResult.getMsgId(), brokerAddr, localTransactionState, false);
requestHeader.setProducerGroup(this.defaultMQProducer.getProducerGroup());
requestHeader.setTranStateTableOffset(sendResult.getQueueOffset());
requestHeader.setMsgId(sendResult.getMsgId());
......
......@@ -70,9 +70,9 @@ public class TopicPublishInfo {
if (lastBrokerName == null) {
return selectOneMessageQueue();
} else {
int index = this.sendWhichQueue.getAndIncrement();
for (int i = 0; i < this.messageQueueList.size(); i++) {
int pos = Math.abs(index++) % this.messageQueueList.size();
int index = this.sendWhichQueue.incrementAndGet();
int pos = Math.abs(index) % this.messageQueueList.size();
if (pos < 0)
pos = 0;
MessageQueue mq = this.messageQueueList.get(pos);
......@@ -85,7 +85,7 @@ public class TopicPublishInfo {
}
public MessageQueue selectOneMessageQueue() {
int index = this.sendWhichQueue.getAndIncrement();
int index = this.sendWhichQueue.incrementAndGet();
int pos = Math.abs(index) % this.messageQueueList.size();
if (pos < 0)
pos = 0;
......
......@@ -80,7 +80,7 @@ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String>
if (half <= 0) {
return tmpList.get(0).getName();
} else {
final int i = this.whichItemWorst.getAndIncrement() % half;
final int i = this.whichItemWorst.incrementAndGet() % half;
return tmpList.get(i).getName();
}
}
......
......@@ -58,7 +58,7 @@ public class MQFaultStrategy {
public MessageQueue selectOneMessageQueue(final TopicPublishInfo tpInfo, final String lastBrokerName) {
if (this.sendLatencyFaultEnable) {
try {
int index = tpInfo.getSendWhichQueue().getAndIncrement();
int index = tpInfo.getSendWhichQueue().incrementAndGet();
for (int i = 0; i < tpInfo.getMessageQueueList().size(); i++) {
int pos = Math.abs(index++) % tpInfo.getMessageQueueList().size();
if (pos < 0)
......@@ -74,7 +74,7 @@ public class MQFaultStrategy {
final MessageQueue mq = tpInfo.selectOneMessageQueue();
if (notBestBroker != null) {
mq.setBrokerName(notBestBroker);
mq.setQueueId(tpInfo.getSendWhichQueue().getAndIncrement() % writeQueueNums);
mq.setQueueId(tpInfo.getSendWhichQueue().incrementAndGet() % writeQueueNums);
}
return mq;
} else {
......
......@@ -29,6 +29,7 @@ import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl;
import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.client.trace.AsyncTraceDispatcher;
import org.apache.rocketmq.client.trace.TraceDispatcher;
import org.apache.rocketmq.client.trace.hook.EndTransactionTraceHookImpl;
import org.apache.rocketmq.client.trace.hook.SendMessageTraceHookImpl;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.Message;
......@@ -167,6 +168,8 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
traceDispatcher = dispatcher;
this.defaultMQProducerImpl.registerSendMessageHook(
new SendMessageTraceHookImpl(traceDispatcher));
this.defaultMQProducerImpl.registerEndTransactionHook(
new EndTransactionTraceHookImpl(traceDispatcher));
} catch (Throwable e) {
log.error("system mqtrace hook init failed ,maybe can't send msg trace data");
}
......@@ -252,6 +255,8 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
traceDispatcher = dispatcher;
this.getDefaultMQProducerImpl().registerSendMessageHook(
new SendMessageTraceHookImpl(traceDispatcher));
this.defaultMQProducerImpl.registerEndTransactionHook(
new EndTransactionTraceHookImpl(traceDispatcher));
} catch (Throwable e) {
log.error("system mqtrace hook init failed ,maybe can't send msg trace data");
}
......@@ -916,24 +921,24 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
long timeout) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
return this.defaultMQProducerImpl.send(batch(msgs), messageQueue, timeout);
}
@Override
public void send(Collection<Message> msgs, SendCallback sendCallback) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
this.defaultMQProducerImpl.send(batch(msgs), sendCallback);
}
@Override
public void send(Collection<Message> msgs, SendCallback sendCallback,
long timeout) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
this.defaultMQProducerImpl.send(batch(msgs), sendCallback, timeout);
}
@Override
public void send(Collection<Message> msgs, MessageQueue mq,
SendCallback sendCallback) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
this.defaultMQProducerImpl.send(batch(msgs), queueWithNamespace(mq), sendCallback);
}
@Override
public void send(Collection<Message> msgs, MessageQueue mq,
SendCallback sendCallback, long timeout) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
......
......@@ -51,6 +51,10 @@ public class TransactionMQProducer extends DefaultMQProducer {
super(namespace, producerGroup, rpcHook);
}
public TransactionMQProducer(final String namespace, final String producerGroup, RPCHook rpcHook, boolean enableMsgTrace, final String customizedTraceTopic) {
super(namespace, producerGroup, rpcHook, enableMsgTrace, customizedTraceTopic);
}
@Override
public void start() throws MQClientException {
this.defaultMQProducerImpl.initTransactionEnv();
......
......@@ -25,12 +25,10 @@ public class SelectMessageQueueByHash implements MessageQueueSelector {
@Override
public MessageQueue select(List<MessageQueue> mqs, Message msg, Object arg) {
int value = arg.hashCode();
int value = arg.hashCode() % mqs.size();
if (value < 0) {
value = Math.abs(value);
}
value = value % mqs.size();
return mqs.get(value);
}
}
......@@ -370,7 +370,7 @@ public class AsyncTraceDispatcher implements TraceDispatcher {
@Override
public void onException(Throwable e) {
log.info("send trace data ,the traceData is " + data);
log.error("send trace data failed, the traceData is {}", data, e);
}
};
if (traceBrokerSet.isEmpty()) {
......@@ -387,7 +387,7 @@ public class AsyncTraceDispatcher implements TraceDispatcher {
filterMqs.add(queue);
}
}
int index = sendWhichQueue.getAndIncrement();
int index = sendWhichQueue.incrementAndGet();
int pos = Math.abs(index) % filterMqs.size();
if (pos < 0) {
pos = 0;
......@@ -398,7 +398,7 @@ public class AsyncTraceDispatcher implements TraceDispatcher {
}
} catch (Exception e) {
log.info("send trace data,the traceData is" + data);
log.error("send trace data failed, the traceData is {}", data, e);
}
}
......
......@@ -16,6 +16,7 @@
*/
package org.apache.rocketmq.client.trace;
import org.apache.rocketmq.client.producer.LocalTransactionState;
import org.apache.rocketmq.common.UtilAll;
import org.apache.rocketmq.common.message.MessageType;
......@@ -32,7 +33,9 @@ public class TraceBean {
private int retryTimes;
private int bodyLength;
private MessageType msgType;
private LocalTransactionState transactionState;
private String transactionId;
private boolean fromTransactionCheck;
public MessageType getMsgType() {
return msgType;
......@@ -141,4 +144,28 @@ public class TraceBean {
public void setBodyLength(int bodyLength) {
this.bodyLength = bodyLength;
}
public LocalTransactionState getTransactionState() {
return transactionState;
}
public void setTransactionState(LocalTransactionState transactionState) {
this.transactionState = transactionState;
}
public String getTransactionId() {
return transactionId;
}
public void setTransactionId(String transactionId) {
this.transactionId = transactionId;
}
public boolean isFromTransactionCheck() {
return fromTransactionCheck;
}
public void setFromTransactionCheck(boolean fromTransactionCheck) {
this.fromTransactionCheck = fromTransactionCheck;
}
}
......@@ -25,4 +25,20 @@ public class TraceConstants {
public static final char FIELD_SPLITOR = (char) 2;
public static final String TRACE_INSTANCE_NAME = "PID_CLIENT_INNER_TRACE_PRODUCER";
public static final String TRACE_TOPIC_PREFIX = TopicValidator.SYSTEM_TOPIC_PREFIX + "TRACE_DATA_";
public static final String TO_PREFIX = "To_";
public static final String FROM_PREFIX = "From_";
public static final String END_TRANSACTION = "EndTransaction";
public static final String ROCKETMQ_SERVICE = "rocketmq";
public static final String ROCKETMQ_SUCCESS = "rocketmq.success";
public static final String ROCKETMQ_TAGS = "rocketmq.tags";
public static final String ROCKETMQ_KEYS = "rocketmq.keys";
public static final String ROCKETMQ_SOTRE_HOST = "rocketmq.store_host";
public static final String ROCKETMQ_BODY_LENGTH = "rocketmq.body_length";
public static final String ROCKETMQ_MSG_ID = "rocketmq.mgs_id";
public static final String ROCKETMQ_MSG_TYPE = "rocketmq.mgs_type";
public static final String ROCKETMQ_REGION_ID = "rocketmq.region_id";
public static final String ROCKETMQ_TRANSACTION_ID = "rocketmq.transaction_id";
public static final String ROCKETMQ_TRANSACTION_STATE = "rocketmq.transaction_state";
public static final String ROCKETMQ_IS_FROM_TRANSACTION_CHECK = "rocketmq.is_from_transaction_check";
public static final String ROCKETMQ_RETRY_TIMERS = "rocketmq.retry_times";
}
......@@ -16,6 +16,7 @@
*/
package org.apache.rocketmq.client.trace;
import org.apache.rocketmq.client.producer.LocalTransactionState;
import org.apache.rocketmq.common.message.MessageType;
import java.util.ArrayList;
......@@ -109,6 +110,27 @@ public class TraceDataEncoder {
subAfterContext.setGroupName(line[8]);
}
resList.add(subAfterContext);
} else if (line[0].equals(TraceType.EndTransaction.name())) {
TraceContext endTransactionContext = new TraceContext();
endTransactionContext.setTraceType(TraceType.EndTransaction);
endTransactionContext.setTimeStamp(Long.parseLong(line[1]));
endTransactionContext.setRegionId(line[2]);
endTransactionContext.setGroupName(line[3]);
TraceBean bean = new TraceBean();
bean.setTopic(line[4]);
bean.setMsgId(line[5]);
bean.setTags(line[6]);
bean.setKeys(line[7]);
bean.setStoreHost(line[8]);
bean.setMsgType(MessageType.values()[Integer.parseInt(line[9])]);
bean.setClientHost(line[10]);
bean.setTransactionId(line[11]);
bean.setTransactionState(LocalTransactionState.valueOf(line[12]));
bean.setFromTransactionCheck(Boolean.parseBoolean(line[13]));
endTransactionContext.setTraceBeans(new ArrayList<TraceBean>(1));
endTransactionContext.getTraceBeans().add(bean);
resList.add(endTransactionContext);
}
}
return resList;
......@@ -173,10 +195,28 @@ public class TraceDataEncoder {
.append(ctx.getContextCode()).append(TraceConstants.CONTENT_SPLITOR)
.append(ctx.getTimeStamp()).append(TraceConstants.CONTENT_SPLITOR)
.append(ctx.getGroupName()).append(TraceConstants.FIELD_SPLITOR);
}
}
break;
case EndTransaction: {
TraceBean bean = ctx.getTraceBeans().get(0);
sb.append(ctx.getTraceType()).append(TraceConstants.CONTENT_SPLITOR)//
.append(ctx.getTimeStamp()).append(TraceConstants.CONTENT_SPLITOR)//
.append(ctx.getRegionId()).append(TraceConstants.CONTENT_SPLITOR)//
.append(ctx.getGroupName()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getTopic()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getMsgId()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getTags()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getKeys()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getStoreHost()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getMsgType().ordinal()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getClientHost()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getTransactionId()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.getTransactionState().name()).append(TraceConstants.CONTENT_SPLITOR)//
.append(bean.isFromTransactionCheck()).append(TraceConstants.FIELD_SPLITOR);
}
break;
default:
}
transferBean.setTransData(sb.toString());
......
......@@ -20,4 +20,5 @@ public enum TraceType {
Pub,
SubBefore,
SubAfter,
EndTransaction,
}
/*
* 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.trace.hook;
import io.opentracing.Span;
import io.opentracing.SpanContext;
import io.opentracing.Tracer;
import io.opentracing.propagation.Format;
import io.opentracing.propagation.TextMapAdapter;
import io.opentracing.tag.Tags;
import org.apache.rocketmq.client.hook.ConsumeMessageContext;
import org.apache.rocketmq.client.hook.ConsumeMessageHook;
import org.apache.rocketmq.client.trace.TraceConstants;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.protocol.NamespaceUtil;
import java.util.ArrayList;
import java.util.List;
public class ConsumeMessageOpenTracingHookImpl implements ConsumeMessageHook {
private Tracer tracer;
public ConsumeMessageOpenTracingHookImpl(Tracer tracer) {
this.tracer = tracer;
}
@Override
public String hookName() {
return "ConsumeMessageOpenTracingHook";
}
@Override
public void consumeMessageBefore(ConsumeMessageContext context) {
if (context == null || context.getMsgList() == null || context.getMsgList().isEmpty()) {
return;
}
List<Span> spanList = new ArrayList<>();
for (MessageExt msg : context.getMsgList()) {
if (msg == null) {
continue;
}
Tracer.SpanBuilder spanBuilder = tracer
.buildSpan(TraceConstants.FROM_PREFIX + msg.getTopic())
.withTag(Tags.SPAN_KIND, Tags.SPAN_KIND_CONSUMER);
SpanContext spanContext = tracer.extract(Format.Builtin.TEXT_MAP, new TextMapAdapter(msg.getProperties()));
if (spanContext != null) {
spanBuilder.asChildOf(spanContext);
}
Span span = spanBuilder.start();
span.setTag(Tags.PEER_SERVICE, TraceConstants.ROCKETMQ_SERVICE);
span.setTag(Tags.MESSAGE_BUS_DESTINATION, NamespaceUtil.withoutNamespace(msg.getTopic()));
span.setTag(TraceConstants.ROCKETMQ_MSG_ID, msg.getMsgId());
span.setTag(TraceConstants.ROCKETMQ_TAGS, msg.getTags());
span.setTag(TraceConstants.ROCKETMQ_KEYS, msg.getKeys());
span.setTag(TraceConstants.ROCKETMQ_BODY_LENGTH, msg.getStoreSize());
span.setTag(TraceConstants.ROCKETMQ_RETRY_TIMERS, msg.getReconsumeTimes());
span.setTag(TraceConstants.ROCKETMQ_REGION_ID, msg.getProperty(MessageConst.PROPERTY_MSG_REGION));
spanList.add(span);
}
context.setMqTraceContext(spanList);
}
@Override
public void consumeMessageAfter(ConsumeMessageContext context) {
if (context == null || context.getMsgList() == null || context.getMsgList().isEmpty()) {
return;
}
List<Span> spanList = (List<Span>) context.getMqTraceContext();
if (spanList == null) {
return;
}
for (Span span : spanList) {
span.setTag(TraceConstants.ROCKETMQ_SUCCESS, context.isSuccess());
span.finish();
}
}
}
/*
* 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.trace.hook;
import io.opentracing.Span;
import io.opentracing.SpanContext;
import io.opentracing.Tracer;
import io.opentracing.propagation.Format;
import io.opentracing.propagation.TextMapAdapter;
import io.opentracing.tag.Tags;
import org.apache.rocketmq.client.hook.EndTransactionContext;
import org.apache.rocketmq.client.hook.EndTransactionHook;
import org.apache.rocketmq.client.trace.TraceConstants;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageType;
public class EndTransactionOpenTracingHookImpl implements EndTransactionHook {
private Tracer tracer;
public EndTransactionOpenTracingHookImpl(Tracer tracer) {
this.tracer = tracer;
}
@Override
public String hookName() {
return "EndTransactionOpenTracingHook";
}
@Override
public void endTransaction(EndTransactionContext context) {
if (context == null) {
return;
}
Message msg = context.getMessage();
Tracer.SpanBuilder spanBuilder = tracer
.buildSpan(TraceConstants.END_TRANSACTION)
.withTag(Tags.SPAN_KIND, Tags.SPAN_KIND_PRODUCER);
SpanContext spanContext = tracer.extract(Format.Builtin.TEXT_MAP, new TextMapAdapter(msg.getProperties()));
if (spanContext != null) {
spanBuilder.asChildOf(spanContext);
}
Span span = spanBuilder.start();
span.setTag(Tags.PEER_SERVICE, TraceConstants.ROCKETMQ_SERVICE);
span.setTag(Tags.MESSAGE_BUS_DESTINATION, msg.getTopic());
span.setTag(TraceConstants.ROCKETMQ_TAGS, msg.getTags());
span.setTag(TraceConstants.ROCKETMQ_KEYS, msg.getKeys());
span.setTag(TraceConstants.ROCKETMQ_SOTRE_HOST, context.getBrokerAddr());
span.setTag(TraceConstants.ROCKETMQ_MSG_ID, context.getMsgId());
span.setTag(TraceConstants.ROCKETMQ_MSG_TYPE, MessageType.Trans_msg_Commit.name());
span.setTag(TraceConstants.ROCKETMQ_TRANSACTION_ID, context.getTransactionId());
span.setTag(TraceConstants.ROCKETMQ_TRANSACTION_STATE, context.getTransactionState().name());
span.setTag(TraceConstants.ROCKETMQ_IS_FROM_TRANSACTION_CHECK, context.isFromTransactionCheck());
span.finish();
}
}
/*
* 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.trace.hook;
import org.apache.rocketmq.client.hook.EndTransactionContext;
import org.apache.rocketmq.client.hook.EndTransactionHook;
import org.apache.rocketmq.client.trace.AsyncTraceDispatcher;
import org.apache.rocketmq.client.trace.TraceBean;
import org.apache.rocketmq.client.trace.TraceContext;
import org.apache.rocketmq.client.trace.TraceDispatcher;
import org.apache.rocketmq.client.trace.TraceType;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageType;
import org.apache.rocketmq.common.protocol.NamespaceUtil;
import java.util.ArrayList;
public class EndTransactionTraceHookImpl implements EndTransactionHook {
private TraceDispatcher localDispatcher;
public EndTransactionTraceHookImpl(TraceDispatcher localDispatcher) {
this.localDispatcher = localDispatcher;
}
@Override
public String hookName() {
return "EndTransactionTraceHook";
}
@Override
public void endTransaction(EndTransactionContext context) {
//if it is message trace data,then it doesn't recorded
if (context == null || context.getMessage().getTopic().startsWith(((AsyncTraceDispatcher) localDispatcher).getTraceTopicName())) {
return;
}
Message msg = context.getMessage();
//build the context content of TuxeTraceContext
TraceContext tuxeContext = new TraceContext();
tuxeContext.setTraceBeans(new ArrayList<TraceBean>(1));
tuxeContext.setTraceType(TraceType.EndTransaction);
tuxeContext.setGroupName(NamespaceUtil.withoutNamespace(context.getProducerGroup()));
//build the data bean object of message trace
TraceBean traceBean = new TraceBean();
traceBean.setTopic(NamespaceUtil.withoutNamespace(context.getMessage().getTopic()));
traceBean.setTags(context.getMessage().getTags());
traceBean.setKeys(context.getMessage().getKeys());
traceBean.setStoreHost(context.getBrokerAddr());
traceBean.setMsgType(MessageType.Trans_msg_Commit);
traceBean.setClientHost(((AsyncTraceDispatcher)localDispatcher).getHostProducer().getmQClientFactory().getClientId());
traceBean.setMsgId(context.getMsgId());
traceBean.setTransactionState(context.getTransactionState());
traceBean.setTransactionId(context.getTransactionId());
traceBean.setFromTransactionCheck(context.isFromTransactionCheck());
String regionId = msg.getProperty(MessageConst.PROPERTY_MSG_REGION);
if (regionId == null || regionId.isEmpty()) {
regionId = MixAll.DEFAULT_TRACE_REGION_ID;
}
tuxeContext.setRegionId(regionId);
tuxeContext.getTraceBeans().add(traceBean);
tuxeContext.setTimeStamp(System.currentTimeMillis());
localDispatcher.append(tuxeContext);
}
}
/*
* 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.trace.hook;
import io.opentracing.Span;
import io.opentracing.SpanContext;
import io.opentracing.Tracer;
import io.opentracing.propagation.Format;
import io.opentracing.propagation.TextMapAdapter;
import io.opentracing.tag.Tags;
import org.apache.rocketmq.client.hook.SendMessageContext;
import org.apache.rocketmq.client.hook.SendMessageHook;
import org.apache.rocketmq.client.producer.SendStatus;
import org.apache.rocketmq.client.trace.TraceConstants;
import org.apache.rocketmq.common.message.Message;
public class SendMessageOpenTracingHookImpl implements SendMessageHook {
private Tracer tracer;
public SendMessageOpenTracingHookImpl(Tracer tracer) {
this.tracer = tracer;
}
@Override
public String hookName() {
return "SendMessageOpenTracingHook";
}
@Override
public void sendMessageBefore(SendMessageContext context) {
if (context == null) {
return;
}
Message msg = context.getMessage();
Tracer.SpanBuilder spanBuilder = tracer
.buildSpan(TraceConstants.TO_PREFIX + msg.getTopic())
.withTag(Tags.SPAN_KIND, Tags.SPAN_KIND_PRODUCER);
SpanContext spanContext = tracer.extract(Format.Builtin.TEXT_MAP, new TextMapAdapter(msg.getProperties()));
if (spanContext != null) {
spanBuilder.asChildOf(spanContext);
}
Span span = spanBuilder.start();
tracer.inject(span.context(), Format.Builtin.TEXT_MAP, new TextMapAdapter(msg.getProperties()));
span.setTag(Tags.PEER_SERVICE, TraceConstants.ROCKETMQ_SERVICE);
span.setTag(Tags.MESSAGE_BUS_DESTINATION, msg.getTopic());
span.setTag(TraceConstants.ROCKETMQ_TAGS, msg.getTags());
span.setTag(TraceConstants.ROCKETMQ_KEYS, msg.getKeys());
span.setTag(TraceConstants.ROCKETMQ_SOTRE_HOST, context.getBrokerAddr());
span.setTag(TraceConstants.ROCKETMQ_MSG_TYPE, context.getMsgType().name());
span.setTag(TraceConstants.ROCKETMQ_BODY_LENGTH, msg.getBody().length);
context.setMqTraceContext(span);
}
@Override
public void sendMessageAfter(SendMessageContext context) {
if (context == null || context.getMqTraceContext() == null) {
return;
}
if (context.getSendResult() == null) {
return;
}
if (context.getSendResult().getRegionId() == null) {
return;
}
Span span = (Span) context.getMqTraceContext();
span.setTag(TraceConstants.ROCKETMQ_SUCCESS, context.getSendResult().getSendStatus().equals(SendStatus.SEND_OK));
span.setTag(TraceConstants.ROCKETMQ_MSG_ID, context.getSendResult().getMsgId());
span.setTag(TraceConstants.ROCKETMQ_REGION_ID, context.getSendResult().getRegionId());
span.finish();
}
}
......@@ -22,11 +22,18 @@ import static org.assertj.core.api.Assertions.assertThat;
public class ThreadLocalIndexTest {
@Test
public void testGetAndIncrement() throws Exception {
public void testIncrementAndGet() throws Exception {
ThreadLocalIndex localIndex = new ThreadLocalIndex();
int initialVal = localIndex.getAndIncrement();
int initialVal = localIndex.incrementAndGet();
assertThat(localIndex.getAndIncrement()).isEqualTo(initialVal + 1);
assertThat(localIndex.incrementAndGet()).isEqualTo(initialVal + 1);
}
@Test
public void testIncrementAndGet2() throws Exception {
ThreadLocalIndex localIndex = new ThreadLocalIndex();
int initialVal = localIndex.incrementAndGet();
assertThat(initialVal >= 0);
}
}
\ No newline at end of file
......@@ -20,8 +20,12 @@ package org.apache.rocketmq.client.consumer;
import java.io.ByteArrayOutputStream;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.*;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.consumer.store.OffsetStore;
import org.apache.rocketmq.client.consumer.store.ReadOffsetType;
......@@ -46,16 +50,15 @@ 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.apache.rocketmq.remoting.RPCHook;
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.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;
......@@ -69,8 +72,7 @@ import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
@RunWith(PowerMockRunner.class)
@PrepareForTest(DefaultLitePullConsumerImpl.class)
@RunWith(MockitoJUnitRunner.class)
public class DefaultLitePullConsumerTest {
@Spy
private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig());
......@@ -92,7 +94,10 @@ public class DefaultLitePullConsumerTest {
@Before
public void init() throws Exception {
PowerMockito.suppress(PowerMockito.method(DefaultLitePullConsumerImpl.class, "updateTopicSubscribeInfoWhenSubscriptionChanged"));
ConcurrentMap<String, MQClientInstance> factoryTable = (ConcurrentMap<String, MQClientInstance>) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true);
factoryTable.forEach((s, instance) -> instance.shutdown());
factoryTable.clear();
Field field = MQClientInstance.class.getDeclaredField("rebalanceService");
field.setAccessible(true);
RebalanceService rebalanceService = (RebalanceService) field.get(mQClientFactory);
......@@ -180,7 +185,9 @@ public class DefaultLitePullConsumerTest {
when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L);
when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(500L);
MessageQueue messageQueue = createMessageQueue();
litePullConsumer.assign(Collections.singletonList(messageQueue));
List<MessageQueue> messageQueues = Collections.singletonList(messageQueue);
litePullConsumer.assign(messageQueues);
litePullConsumer.pause(messageQueues);
long offset = litePullConsumer.committed(messageQueue);
litePullConsumer.seek(messageQueue, offset);
Field field = DefaultLitePullConsumerImpl.class.getDeclaredField("assignedMessageQueue");
......@@ -196,7 +203,9 @@ public class DefaultLitePullConsumerTest {
when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L);
when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(500L);
MessageQueue messageQueue = createMessageQueue();
litePullConsumer.assign(Collections.singletonList(messageQueue));
List<MessageQueue> messageQueues = Collections.singletonList(messageQueue);
litePullConsumer.assign(messageQueues);
litePullConsumer.pause(messageQueues);
litePullConsumer.seekToBegin(messageQueue);
Field field = DefaultLitePullConsumerImpl.class.getDeclaredField("assignedMessageQueue");
field.setAccessible(true);
......@@ -211,7 +220,9 @@ public class DefaultLitePullConsumerTest {
when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L);
when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(500L);
MessageQueue messageQueue = createMessageQueue();
litePullConsumer.assign(Collections.singletonList(messageQueue));
List<MessageQueue> messageQueues = Collections.singletonList(messageQueue);
litePullConsumer.assign(messageQueues);
litePullConsumer.pause(messageQueues);
litePullConsumer.seekToEnd(messageQueue);
Field field = DefaultLitePullConsumerImpl.class.getDeclaredField("assignedMessageQueue");
field.setAccessible(true);
......@@ -226,7 +237,9 @@ public class DefaultLitePullConsumerTest {
when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L);
when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(100L);
MessageQueue messageQueue = createMessageQueue();
litePullConsumer.assign(Collections.singletonList(messageQueue));
List<MessageQueue> messageQueues = Collections.singletonList(messageQueue);
litePullConsumer.assign(messageQueues);
litePullConsumer.pause(messageQueues);
try {
litePullConsumer.seek(messageQueue, -1);
failBecauseExceptionWasNotThrown(MQClientException.class);
......@@ -515,9 +528,6 @@ public class DefaultLitePullConsumerTest {
public void testConsumerAfterShutdown() throws Exception {
DefaultLitePullConsumer defaultLitePullConsumer = createSubscribeLitePullConsumer();
DefaultLitePullConsumer mockConsumer = spy(defaultLitePullConsumer);
when(mockConsumer.poll(anyLong())).thenReturn(new ArrayList<>());
new AsyncConsumer().executeAsync(defaultLitePullConsumer);
Thread.sleep(100);
......@@ -574,9 +584,9 @@ public class DefaultLitePullConsumerTest {
when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class),
anyLong(), any(CommunicationMode.class), nullable(PullCallback.class)))
.thenAnswer(new Answer<Object>() {
.thenAnswer(new Answer<PullResult>() {
@Override
public Object answer(InvocationOnMock mock) throws Throwable {
public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
MessageClientExt messageClientExt = new MessageClientExt();
messageClientExt.setTopic(topic);
......@@ -602,6 +612,7 @@ public class DefaultLitePullConsumerTest {
DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis());
litePullConsumer.setNamesrvAddr("127.0.0.1:9876");
litePullConsumer.subscribe(topic, "*");
suppressUpdateTopicRouteInfoFromNameServer(litePullConsumer);
litePullConsumer.start();
initDefaultLitePullConsumer(litePullConsumer);
return litePullConsumer;
......@@ -610,6 +621,7 @@ public class DefaultLitePullConsumerTest {
private DefaultLitePullConsumer createStartLitePullConsumer() throws Exception {
DefaultLitePullConsumer litePullConsumer = new DefaultLitePullConsumer(consumerGroup + System.currentTimeMillis());
litePullConsumer.setNamesrvAddr("127.0.0.1:9876");
suppressUpdateTopicRouteInfoFromNameServer(litePullConsumer);
litePullConsumer.start();
initDefaultLitePullConsumer(litePullConsumer);
return litePullConsumer;
......@@ -625,6 +637,7 @@ public class DefaultLitePullConsumerTest {
litePullConsumer.setNamesrvAddr("127.0.0.1:9876");
litePullConsumer.setMessageModel(MessageModel.BROADCASTING);
litePullConsumer.subscribe(topic, "*");
suppressUpdateTopicRouteInfoFromNameServer(litePullConsumer);
litePullConsumer.start();
initDefaultLitePullConsumer(litePullConsumer);
return litePullConsumer;
......@@ -646,4 +659,15 @@ public class DefaultLitePullConsumerTest {
}
return new PullResultExt(pullStatus, requestHeader.getQueueOffset() + messageExtList.size(), 123, 2048, messageExtList, 0, outputStream.toByteArray());
}
private static void suppressUpdateTopicRouteInfoFromNameServer(DefaultLitePullConsumer litePullConsumer) throws IllegalAccessException {
DefaultLitePullConsumerImpl defaultLitePullConsumerImpl = (DefaultLitePullConsumerImpl) FieldUtils.readDeclaredField(litePullConsumer, "defaultLitePullConsumerImpl", true);
if (litePullConsumer.getMessageModel() == MessageModel.CLUSTERING) {
litePullConsumer.changeInstanceNameToPID();
}
MQClientInstance mQClientFactory = spy(MQClientManager.getInstance().getOrCreateMQClientInstance(litePullConsumer, (RPCHook) FieldUtils.readDeclaredField(defaultLitePullConsumerImpl, "rpcHook", true)));
ConcurrentMap<String, MQClientInstance> factoryTable = (ConcurrentMap<String, MQClientInstance>) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true);
factoryTable.put(litePullConsumer.buildMQClientId(), mQClientFactory);
doReturn(false).when(mQClientFactory).updateTopicRouteInfoFromNameServer(anyString());
}
}
......@@ -17,15 +17,17 @@
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 java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyContext;
......@@ -37,21 +39,22 @@ 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.MQClientAPIImpl;
import org.apache.rocketmq.client.impl.MQClientManager;
import org.apache.rocketmq.client.impl.consumer.ConsumeMessageConcurrentlyService;
import org.apache.rocketmq.client.impl.consumer.ConsumeMessageOrderlyService;
import org.apache.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl;
import org.apache.rocketmq.client.impl.consumer.ProcessQueue;
import org.apache.rocketmq.client.impl.consumer.PullAPIWrapper;
import org.apache.rocketmq.client.impl.consumer.PullMessageService;
import org.apache.rocketmq.client.impl.consumer.PullRequest;
import org.apache.rocketmq.client.impl.consumer.PullResultExt;
import org.apache.rocketmq.client.impl.consumer.RebalancePushImpl;
import org.apache.rocketmq.client.impl.consumer.RebalanceImpl;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.common.message.MessageClientExt;
import org.apache.rocketmq.common.message.MessageDecoder;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader;
import org.apache.rocketmq.remoting.RPCHook;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.junit.After;
import org.junit.Assert;
......@@ -60,10 +63,8 @@ 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;
......@@ -76,8 +77,7 @@ import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
@RunWith(PowerMockRunner.class)
@PrepareForTest(DefaultMQPushConsumerImpl.class)
@RunWith(MockitoJUnitRunner.class)
public class DefaultMQPushConsumerTest {
private String consumerGroup;
private String topic = "FooBar";
......@@ -86,57 +86,20 @@ public class DefaultMQPushConsumerTest {
@Mock
private MQClientAPIImpl mQClientAPIImpl;
private PullAPIWrapper pullAPIWrapper;
private RebalancePushImpl rebalancePushImpl;
private RebalanceImpl rebalanceImpl;
private DefaultMQPushConsumer pushConsumer;
@Before
public void init() throws Exception {
consumerGroup = "FooBarGroup" + System.currentTimeMillis();
pushConsumer = new DefaultMQPushConsumer(consumerGroup);
pushConsumer.setNamesrvAddr("127.0.0.1:9876");
pushConsumer.setPullInterval(60 * 1000);
pushConsumer.registerMessageListener(new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
return null;
}
});
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();
ConcurrentMap<String, MQClientInstance> factoryTable = (ConcurrentMap<String, MQClientInstance>) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true);
factoryTable.forEach((s, instance) -> instance.shutdown());
factoryTable.clear();
mQClientFactory = spy(pushConsumerImpl.getmQClientFactory());
field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory");
field.setAccessible(true);
field.set(pushConsumerImpl, mQClientFactory);
field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl");
field.setAccessible(true);
field.set(mQClientFactory, mQClientAPIImpl);
pullAPIWrapper = spy(new PullAPIWrapper(mQClientFactory, consumerGroup, false));
field = DefaultMQPushConsumerImpl.class.getDeclaredField("pullAPIWrapper");
field.setAccessible(true);
field.set(pushConsumerImpl, pullAPIWrapper);
pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory);
mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl);
when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class),
when(mQClientAPIImpl.pullMessage(anyString(), any(PullMessageRequestHeader.class),
anyLong(), any(CommunicationMode.class), nullable(PullCallback.class)))
.thenAnswer(new Answer<Object>() {
.thenAnswer(new Answer<PullResult>() {
@Override
public Object answer(InvocationOnMock mock) throws Throwable {
public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
MessageClientExt messageClientExt = new MessageClientExt();
messageClientExt.setTopic(topic);
......@@ -152,12 +115,42 @@ public class DefaultMQPushConsumerTest {
}
});
consumerGroup = "FooBarGroup" + System.currentTimeMillis();
pushConsumer = new DefaultMQPushConsumer(consumerGroup);
pushConsumer.setNamesrvAddr("127.0.0.1:9876");
pushConsumer.setPullInterval(60 * 1000);
pushConsumer.registerMessageListener(new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
return null;
}
});
DefaultMQPushConsumerImpl pushConsumerImpl = pushConsumer.getDefaultMQPushConsumerImpl();
// suppress updateTopicRouteInfoFromNameServer
pushConsumer.changeInstanceNameToPID();
mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(pushConsumer, (RPCHook) FieldUtils.readDeclaredField(pushConsumerImpl, "rpcHook", true));
FieldUtils.writeDeclaredField(mQClientFactory, "mQClientAPIImpl", mQClientAPIImpl, true);
mQClientFactory = spy(mQClientFactory);
factoryTable.put(pushConsumer.buildMQClientId(), mQClientFactory);
doReturn(false).when(mQClientFactory).updateTopicRouteInfoFromNameServer(anyString());
doReturn(new FindBrokerResult("127.0.0.1:10911", false)).when(mQClientFactory).findBrokerAddressInSubscribe(anyString(), anyLong(), anyBoolean());
doReturn(Collections.singletonList(mQClientFactory.getClientId())).when(mQClientFactory).findConsumerIdList(anyString(), anyString());
rebalanceImpl = spy(pushConsumerImpl.getRebalanceImpl());
doReturn(123L).when(rebalanceImpl).computePullFromWhere(any(MessageQueue.class));
FieldUtils.writeDeclaredField(pushConsumerImpl, "rebalanceImpl", rebalanceImpl, true);
Set<MessageQueue> messageQueueSet = new HashSet<MessageQueue>();
messageQueueSet.add(createPullRequest().getMessageQueue());
pushConsumer.getDefaultMQPushConsumerImpl().updateTopicSubscribeInfo(topic, messageQueueSet);
doReturn(123L).when(rebalancePushImpl).computePullFromWhere(any(MessageQueue.class));
pushConsumerImpl.updateTopicSubscribeInfo(topic, messageQueueSet);
pushConsumer.subscribe(topic, "*");
pushConsumer.start();
}
@After
......@@ -173,12 +166,12 @@ public class DefaultMQPushConsumerTest {
@Test
public void testPullMessage_Success() throws InterruptedException, RemotingException, MQBrokerException {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final MessageExt[] messageExts = new MessageExt[1];
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
messageExts[0] = msgs.get(0);
messageAtomic.set(msgs.get(0));
countDownLatch.countDown();
return null;
}
......@@ -186,20 +179,22 @@ public class DefaultMQPushConsumerTest {
PullMessageService pullMessageService = mQClientFactory.getPullMessageService();
pullMessageService.executePullRequestImmediately(createPullRequest());
countDownLatch.await();
assertThat(messageExts[0].getTopic()).isEqualTo(topic);
assertThat(messageExts[0].getBody()).isEqualTo(new byte[] {'a'});
countDownLatch.await(10, TimeUnit.SECONDS);
MessageExt msg = messageAtomic.get();
assertThat(msg).isNotNull();
assertThat(msg.getTopic()).isEqualTo(topic);
assertThat(msg.getBody()).isEqualTo(new byte[] {'a'});
}
@Test
public void testPullMessage_SuccessWithOrderlyService() throws Exception {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final MessageExt[] messageExts = new MessageExt[1];
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
MessageListenerOrderly listenerOrderly = new MessageListenerOrderly() {
@Override
public ConsumeOrderlyStatus consumeMessage(List<MessageExt> msgs, ConsumeOrderlyContext context) {
messageExts[0] = msgs.get(0);
messageAtomic.set(msgs.get(0));
countDownLatch.countDown();
return null;
}
......@@ -212,8 +207,10 @@ public class DefaultMQPushConsumerTest {
pullMessageService.executePullRequestLater(createPullRequest(), 100);
countDownLatch.await(10, TimeUnit.SECONDS);
assertThat(messageExts[0].getTopic()).isEqualTo(topic);
assertThat(messageExts[0].getBody()).isEqualTo(new byte[] {'a'});
MessageExt msg = messageAtomic.get();
assertThat(msg).isNotNull();
assertThat(msg.getTopic()).isEqualTo(topic);
assertThat(msg.getBody()).isEqualTo(new byte[] {'a'});
}
@Test
......@@ -279,7 +276,7 @@ public class DefaultMQPushConsumerTest {
PullMessageService pullMessageService = mQClientFactory.getPullMessageService();
pullMessageService.executePullRequestImmediately(createPullRequest());
countDownLatch.await();
assertThat(countDownLatch.await(10, TimeUnit.SECONDS)).isTrue();
pushConsumer.shutdown();
assertThat(messageConsumedFlag.get()).isTrue();
......
......@@ -16,7 +16,19 @@
*/
package org.apache.rocketmq.client.impl.consumer;
import 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 java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.consumer.PullCallback;
import org.apache.rocketmq.client.consumer.PullResult;
import org.apache.rocketmq.client.consumer.PullStatus;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
......@@ -40,23 +52,15 @@ import org.junit.Ignore;
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.junit.MockitoJUnitRunner;
import org.mockito.stubbing.Answer;
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 java.util.concurrent.CountDownLatch;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.*;
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;
......@@ -117,9 +121,9 @@ public class ConsumeMessageConcurrentlyServiceTest {
when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class),
anyLong(), any(CommunicationMode.class), nullable(PullCallback.class)))
.thenAnswer(new Answer<Object>() {
.thenAnswer(new Answer<PullResult>() {
@Override
public Object answer(InvocationOnMock mock) throws Throwable {
public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
MessageClientExt messageClientExt = new MessageClientExt();
messageClientExt.setTopic(topic);
......@@ -145,13 +149,13 @@ public class ConsumeMessageConcurrentlyServiceTest {
@Test
public void testPullMessage_ConsumeSuccess() throws InterruptedException, RemotingException, MQBrokerException, NoSuchFieldException,Exception {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final MessageExt[] messageExts = new MessageExt[1];
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
ConsumeMessageConcurrentlyService normalServie = new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
messageExts[0] = msgs.get(0);
messageAtomic.set(msgs.get(0));
countDownLatch.countDown();
return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
}
......@@ -175,8 +179,10 @@ public class ConsumeMessageConcurrentlyServiceTest {
StatsItem item = itemSet.getAndCreateStatsItem(topic + "@" + pushConsumer.getDefaultMQPushConsumerImpl().groupName());
assertThat(item.getValue().get()).isGreaterThan(0L);
assertThat(messageExts[0].getTopic()).isEqualTo(topic);
assertThat(messageExts[0].getBody()).isEqualTo(new byte[] {'a'});
MessageExt msg = messageAtomic.get();
assertThat(msg).isNotNull();
assertThat(msg.getTopic()).isEqualTo(topic);
assertThat(msg.getBody()).isEqualTo(new byte[] {'a'});
}
@After
......
......@@ -19,29 +19,44 @@ package org.apache.rocketmq.client.impl.factory;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.admin.MQAdminExtInner;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.impl.FindBrokerResult;
import org.apache.rocketmq.client.impl.MQClientManager;
import org.apache.rocketmq.client.impl.consumer.MQConsumerInner;
import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl;
import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
import org.apache.rocketmq.common.protocol.body.ConsumerRunningInfo;
import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType;
import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.junit.MockitoJUnitRunner;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public class MQClientInstanceTest {
private MQClientInstance mqClientInstance = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig());
private String topic = "FooBar";
private String group = "FooBarGroup";
private ConcurrentMap<String, HashMap<Long, String>> brokerAddrTable = new ConcurrentHashMap<String, HashMap<Long, String>>();
@Before
public void init() throws Exception {
FieldUtils.writeDeclaredField(mqClientInstance, "brokerAddrTable", brokerAddrTable, true);
}
@Test
public void testTopicRouteData2TopicPublishInfo() {
......@@ -64,7 +79,7 @@ public class MQClientInstanceTest {
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSynFlag(0);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
......@@ -74,6 +89,34 @@ public class MQClientInstanceTest {
assertThat(topicPublishInfo.getMessageQueueList().size()).isEqualTo(4);
}
@Test
public void testFindBrokerAddressInSubscribe() {
// dledger normal case
String brokerName = "BrokerA";
HashMap<Long, String> addrMap = new HashMap<Long, String>();
addrMap.put(0L, "127.0.0.1:10911");
addrMap.put(1L, "127.0.0.1:10912");
addrMap.put(2L, "127.0.0.1:10913");
brokerAddrTable.put(brokerName, addrMap);
long brokerId = 1;
FindBrokerResult brokerResult = mqClientInstance.findBrokerAddressInSubscribe(brokerName, brokerId, false);
assertThat(brokerResult).isNotNull();
assertThat(brokerResult.getBrokerAddr()).isEqualTo("127.0.0.1:10912");
assertThat(brokerResult.isSlave()).isTrue();
// dledger case, when node n0 was voted as the leader
brokerName = "BrokerB";
HashMap<Long, String> addrMapNew = new HashMap<Long, String>();
addrMapNew.put(0L, "127.0.0.1:10911");
addrMapNew.put(2L, "127.0.0.1:10912");
addrMapNew.put(3L, "127.0.0.1:10913");
brokerAddrTable.put(brokerName, addrMapNew);
brokerResult = mqClientInstance.findBrokerAddressInSubscribe(brokerName, brokerId, false);
assertThat(brokerResult).isNotNull();
assertThat(brokerResult.getBrokerAddr()).isEqualTo("127.0.0.1:10912");
assertThat(brokerResult.isSlave()).isTrue();
}
@Test
public void testRegisterProducer() {
boolean flag = mqClientInstance.registerProducer(group, mock(DefaultMQProducerImpl.class));
......@@ -100,6 +143,31 @@ public class MQClientInstanceTest {
assertThat(flag).isTrue();
}
@Test
public void testConsumerRunningInfoWhenConsumersIsEmptyOrNot() throws RemotingException, InterruptedException, MQBrokerException {
MQConsumerInner mockConsumerInner = mock(MQConsumerInner.class);
ConsumerRunningInfo mockConsumerRunningInfo = mock(ConsumerRunningInfo.class);
when(mockConsumerInner.consumerRunningInfo()).thenReturn(mockConsumerRunningInfo);
when(mockConsumerInner.consumeType()).thenReturn(ConsumeType.CONSUME_PASSIVELY);
Properties properties = new Properties();
when(mockConsumerRunningInfo.getProperties()).thenReturn(properties);
mqClientInstance.unregisterConsumer(group);
ConsumerRunningInfo runningInfo = mqClientInstance.consumerRunningInfo(group);
assertThat(runningInfo).isNull();
boolean flag = mqClientInstance.registerConsumer(group, mockConsumerInner);
assertThat(flag).isTrue();
runningInfo = mqClientInstance.consumerRunningInfo(group);
assertThat(runningInfo).isNotNull();
assertThat(mockConsumerInner.consumerRunningInfo().getProperties().get(ConsumerRunningInfo.PROP_CONSUME_TYPE));
mqClientInstance.unregisterConsumer(group);
flag = mqClientInstance.registerConsumer(group, mock(MQConsumerInner.class));
assertThat(flag).isTrue();
}
@Test
public void testRegisterAdminExt() {
boolean flag = mqClientInstance.registerAdminExt(group, mock(MQAdminExtInner.class));
......
......@@ -481,7 +481,7 @@ public class DefaultMQProducerTest {
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSynFlag(0);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
......
......@@ -44,6 +44,14 @@ public class SelectMessageQueueByHashTest {
String anotherOrderId = "234";
MessageQueue selected = selector.select(messageQueues, message, orderId);
assertThat(selector.select(messageQueues, message, anotherOrderId)).isNotEqualTo(selected);
//No exception is thrown while order Id hashcode is Integer.MIN
anotherOrderId = "polygenelubricants";
selector.select(messageQueues, message, anotherOrderId);
anotherOrderId = "GydZG_";
selector.select(messageQueues, message, anotherOrderId);
anotherOrderId = "DESIGNING WORKHOUSES";
selector.select(messageQueues, message, anotherOrderId);
}
}
\ No newline at end of file
/*
* 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.producer.selector;
import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageQueue;
import org.junit.Test;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static org.assertj.core.api.Assertions.assertThat;
public class SelectMessageQueueRetryTest {
private String topic = "TEST";
@Test
public void testSelect() throws Exception {
TopicPublishInfo topicPublishInfo = new TopicPublishInfo();
List<MessageQueue> messageQueueList = new ArrayList();
for (int i = 0; i < 3; i++) {
MessageQueue mq = new MessageQueue();
mq.setBrokerName("broker-" + i);
mq.setQueueId(0);
mq.setTopic(topic);
messageQueueList.add(mq);
}
topicPublishInfo.setMessageQueueList(messageQueueList);
Set<String> retryBrokerNameSet = retryBroker(topicPublishInfo);
//always in Set (broker-0,broker-1,broker-2)
assertThat(retryBroker(topicPublishInfo)).isEqualTo(retryBrokerNameSet);
}
private Set<String> retryBroker(TopicPublishInfo topicPublishInfo) {
MessageQueue mqTmp = null;
Set<String> retryBrokerNameSet = new HashSet();
for (int times = 0; times < 3; times++) {
String lastBrokerName = null == mqTmp ? null : mqTmp.getBrokerName();
mqTmp = topicPublishInfo.selectOneMessageQueue(lastBrokerName);
retryBrokerNameSet.add(mqTmp.getBrokerName());
}
return retryBrokerNameSet;
}
}
\ No newline at end of file
/*
* 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.trace;
import io.opentracing.mock.MockSpan;
import io.opentracing.mock.MockTracer;
import io.opentracing.tag.Tags;
import java.io.ByteArrayOutputStream;
import java.net.InetSocketAddress;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.consumer.PullCallback;
import org.apache.rocketmq.client.consumer.PullResult;
import org.apache.rocketmq.client.consumer.PullStatus;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
import org.apache.rocketmq.client.consumer.store.OffsetStore;
import org.apache.rocketmq.client.consumer.store.ReadOffsetType;
import org.apache.rocketmq.client.exception.MQBrokerException;
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.MQClientAPIImpl;
import org.apache.rocketmq.client.impl.MQClientManager;
import org.apache.rocketmq.client.impl.consumer.ConsumeMessageConcurrentlyService;
import org.apache.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl;
import org.apache.rocketmq.client.impl.consumer.ProcessQueue;
import org.apache.rocketmq.client.impl.consumer.PullAPIWrapper;
import org.apache.rocketmq.client.impl.consumer.PullMessageService;
import org.apache.rocketmq.client.impl.consumer.PullRequest;
import org.apache.rocketmq.client.impl.consumer.PullResultExt;
import org.apache.rocketmq.client.impl.consumer.RebalancePushImpl;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.client.trace.hook.ConsumeMessageOpenTracingHookImpl;
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.remoting.RPCHook;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.junit.MockitoJUnitRunner;
import org.mockito.stubbing.Answer;
import static org.assertj.core.api.Assertions.assertThat;
import static org.awaitility.Awaitility.waitAtMost;
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(MockitoJUnitRunner.class)
public class DefaultMQConsumerWithOpenTracingTest {
private String consumerGroup;
private String topic = "FooBar";
private String brokerName = "BrokerA";
private MQClientInstance mQClientFactory;
@Mock
private MQClientAPIImpl mQClientAPIImpl;
private PullAPIWrapper pullAPIWrapper;
private RebalancePushImpl rebalancePushImpl;
private DefaultMQPushConsumer pushConsumer;
private MockTracer tracer = new MockTracer();
@Before
public void init() throws Exception {
ConcurrentMap<String, MQClientInstance> factoryTable = (ConcurrentMap<String, MQClientInstance>) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true);
factoryTable.forEach((s, instance) -> instance.shutdown());
factoryTable.clear();
when(mQClientAPIImpl.pullMessage(anyString(), any(PullMessageRequestHeader.class),
anyLong(), any(CommunicationMode.class), nullable(PullCallback.class)))
.thenAnswer(new Answer<PullResult>() {
@Override
public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
MessageClientExt messageClientExt = new MessageClientExt();
messageClientExt.setTopic(topic);
messageClientExt.setQueueId(0);
messageClientExt.setMsgId("123");
messageClientExt.setBody(new byte[]{'a'});
messageClientExt.setOffsetMsgId("234");
messageClientExt.setBornHost(new InetSocketAddress(8080));
messageClientExt.setStoreHost(new InetSocketAddress(8080));
PullResult pullResult = createPullResult(requestHeader, PullStatus.FOUND, Collections.<MessageExt>singletonList(messageClientExt));
((PullCallback) mock.getArgument(4)).onSuccess(pullResult);
return pullResult;
}
});
consumerGroup = "FooBarGroup" + System.currentTimeMillis();
pushConsumer = new DefaultMQPushConsumer(consumerGroup);
pushConsumer.getDefaultMQPushConsumerImpl().registerConsumeMessageHook(
new ConsumeMessageOpenTracingHookImpl(tracer));
pushConsumer.setNamesrvAddr("127.0.0.1:9876");
pushConsumer.setPullInterval(60 * 1000);
OffsetStore offsetStore = Mockito.mock(OffsetStore.class);
Mockito.when(offsetStore.readOffset(any(MessageQueue.class), any(ReadOffsetType.class))).thenReturn(0L);
pushConsumer.setOffsetStore(offsetStore);
pushConsumer.registerMessageListener(new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
return null;
}
});
DefaultMQPushConsumerImpl pushConsumerImpl = pushConsumer.getDefaultMQPushConsumerImpl();
// suppress updateTopicRouteInfoFromNameServer
pushConsumer.changeInstanceNameToPID();
mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(pushConsumer, (RPCHook) FieldUtils.readDeclaredField(pushConsumerImpl, "rpcHook", true));
FieldUtils.writeDeclaredField(mQClientFactory, "mQClientAPIImpl", mQClientAPIImpl, true);
mQClientFactory = spy(mQClientFactory);
factoryTable.put(pushConsumer.buildMQClientId(), mQClientFactory);
doReturn(false).when(mQClientFactory).updateTopicRouteInfoFromNameServer(anyString());
doReturn(new FindBrokerResult("127.0.0.1:10911", false)).when(mQClientFactory).findBrokerAddressInSubscribe(anyString(), anyLong(), anyBoolean());
Set<MessageQueue> messageQueueSet = new HashSet<MessageQueue>();
messageQueueSet.add(createPullRequest().getMessageQueue());
pushConsumerImpl.updateTopicSubscribeInfo(topic, messageQueueSet);
pushConsumer.subscribe(topic, "*");
pushConsumer.start();
}
@After
public void terminate() {
pushConsumer.shutdown();
}
@Test
public void testPullMessage_WithTrace_Success() throws InterruptedException, RemotingException, MQBrokerException, MQClientException {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
messageAtomic.set(msgs.get(0));
countDownLatch.countDown();
return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
}
}));
PullMessageService pullMessageService = mQClientFactory.getPullMessageService();
pullMessageService.executePullRequestImmediately(createPullRequest());
countDownLatch.await(30, TimeUnit.SECONDS);
MessageExt msg = messageAtomic.get();
assertThat(msg).isNotNull();
assertThat(msg.getTopic()).isEqualTo(topic);
assertThat(msg.getBody()).isEqualTo(new byte[]{'a'});
// wait until consumeMessageAfter hook of tracer is done surely.
waitAtMost(1, TimeUnit.SECONDS).until(() -> tracer.finishedSpans().size() == 1);
MockSpan span = tracer.finishedSpans().get(0);
assertThat(span.tags().get(Tags.MESSAGE_BUS_DESTINATION.getKey())).isEqualTo(topic);
assertThat(span.tags().get(Tags.SPAN_KIND.getKey())).isEqualTo(Tags.SPAN_KIND_CONSUMER);
assertThat(span.tags().get(TraceConstants.ROCKETMQ_SUCCESS)).isEqualTo(true);
}
private PullRequest createPullRequest() {
PullRequest pullRequest = new PullRequest();
pullRequest.setConsumerGroup(consumerGroup);
pullRequest.setNextOffset(1024);
MessageQueue messageQueue = new MessageQueue();
messageQueue.setBrokerName(brokerName);
messageQueue.setQueueId(0);
messageQueue.setTopic(topic);
pullRequest.setMessageQueue(messageQueue);
ProcessQueue processQueue = new ProcessQueue();
processQueue.setLocked(true);
processQueue.setLastLockTimestamp(System.currentTimeMillis());
pullRequest.setProcessQueue(processQueue);
return pullRequest;
}
private PullResultExt createPullResult(PullMessageRequestHeader requestHeader, PullStatus pullStatus,
List<MessageExt> 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());
}
}
......@@ -26,8 +26,11 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.consumer.PullCallback;
import org.apache.rocketmq.client.consumer.PullResult;
......@@ -40,6 +43,7 @@ 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.MQClientAPIImpl;
import org.apache.rocketmq.client.impl.MQClientManager;
import org.apache.rocketmq.client.impl.consumer.ConsumeMessageConcurrentlyService;
import org.apache.rocketmq.client.impl.consumer.DefaultMQPushConsumerImpl;
import org.apache.rocketmq.client.impl.consumer.ProcessQueue;
......@@ -53,17 +57,16 @@ import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl;
import org.apache.rocketmq.client.producer.DefaultMQProducer;
import org.apache.rocketmq.client.producer.SendResult;
import org.apache.rocketmq.client.producer.SendStatus;
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.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.common.topic.TopicValidator;
import org.apache.rocketmq.remoting.RPCHook;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.junit.After;
import org.junit.Before;
......@@ -71,10 +74,8 @@ 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;
......@@ -86,8 +87,7 @@ import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
@RunWith(PowerMockRunner.class)
@PrepareForTest(DefaultMQPushConsumerImpl.class)
@RunWith(MockitoJUnitRunner.class)
public class DefaultMQConsumerWithTraceTest {
private String consumerGroup;
private String consumerGroupNormal;
......@@ -114,6 +114,10 @@ public class DefaultMQConsumerWithTraceTest {
@Before
public void init() throws Exception {
ConcurrentMap<String, MQClientInstance> factoryTable = (ConcurrentMap<String, MQClientInstance>) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true);
factoryTable.forEach((s, instance) -> instance.shutdown());
factoryTable.clear();
consumerGroup = "FooBarGroup" + System.currentTimeMillis();
pushConsumer = new DefaultMQPushConsumer(consumerGroup, true, "");
consumerGroupNormal = "FooBarGroup" + System.currentTimeMillis();
......@@ -133,8 +137,14 @@ public class DefaultMQConsumerWithTraceTest {
}
});
PowerMockito.suppress(PowerMockito.method(DefaultMQPushConsumerImpl.class, "updateTopicSubscribeInfoWhenSubscriptionChanged"));
DefaultMQPushConsumerImpl pushConsumerImpl = pushConsumer.getDefaultMQPushConsumerImpl();
// suppress updateTopicRouteInfoFromNameServer
pushConsumer.changeInstanceNameToPID();
mQClientFactory = spy(MQClientManager.getInstance().getOrCreateMQClientInstance(pushConsumer, (RPCHook) FieldUtils.readDeclaredField(pushConsumerImpl, "rpcHook", true)));
factoryTable.put(pushConsumer.buildMQClientId(), mQClientFactory);
doReturn(false).when(mQClientFactory).updateTopicRouteInfoFromNameServer(anyString());
rebalancePushImpl = spy(new RebalancePushImpl(pushConsumer.getDefaultMQPushConsumerImpl()));
Field field = DefaultMQPushConsumerImpl.class.getDeclaredField("rebalanceImpl");
field.setAccessible(true);
......@@ -172,9 +182,9 @@ public class DefaultMQConsumerWithTraceTest {
when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class),
anyLong(), any(CommunicationMode.class), nullable(PullCallback.class)))
.thenAnswer(new Answer<Object>() {
.thenAnswer(new Answer<PullResult>() {
@Override
public Object answer(InvocationOnMock mock) throws Throwable {
public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
MessageClientExt messageClientExt = new MessageClientExt();
messageClientExt.setTopic(topic);
......@@ -206,12 +216,12 @@ public class DefaultMQConsumerWithTraceTest {
traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl());
final CountDownLatch countDownLatch = new CountDownLatch(1);
final MessageExt[] messageExts = new MessageExt[1];
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
messageExts[0] = msgs.get(0);
messageAtomic.set(msgs.get(0));
countDownLatch.countDown();
return null;
}
......@@ -219,9 +229,11 @@ public class DefaultMQConsumerWithTraceTest {
PullMessageService pullMessageService = mQClientFactory.getPullMessageService();
pullMessageService.executePullRequestImmediately(createPullRequest());
countDownLatch.await(3000L, TimeUnit.MILLISECONDS);
assertThat(messageExts[0].getTopic()).isEqualTo(topic);
assertThat(messageExts[0].getBody()).isEqualTo(new byte[] {'a'});
countDownLatch.await(30, TimeUnit.SECONDS);
MessageExt msg = messageAtomic.get();
assertThat(msg).isNotNull();
assertThat(msg.getTopic()).isEqualTo(topic);
assertThat(msg.getBody()).isEqualTo(new byte[] {'a'});
}
private PullRequest createPullRequest() {
......@@ -271,7 +283,7 @@ public class DefaultMQConsumerWithTraceTest {
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSynFlag(0);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
......@@ -307,7 +319,7 @@ public class DefaultMQConsumerWithTraceTest {
queueData.setPerm(6);
queueData.setReadQueueNums(1);
queueData.setWriteQueueNums(1);
queueData.setTopicSynFlag(1);
queueData.setTopicSysFlag(1);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
......
/*
* 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.trace;
import io.opentracing.mock.MockSpan;
import io.opentracing.mock.MockTracer;
import io.opentracing.tag.Tags;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.hook.SendMessageContext;
import org.apache.rocketmq.client.impl.CommunicationMode;
import org.apache.rocketmq.client.impl.MQClientAPIImpl;
import org.apache.rocketmq.client.impl.MQClientManager;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl;
import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
import org.apache.rocketmq.client.producer.DefaultMQProducer;
import org.apache.rocketmq.client.producer.SendCallback;
import org.apache.rocketmq.client.producer.SendResult;
import org.apache.rocketmq.client.producer.SendStatus;
import org.apache.rocketmq.client.trace.hook.SendMessageOpenTracingHookImpl;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageType;
import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.common.topic.TopicValidator;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Spy;
import org.mockito.junit.MockitoJUnitRunner;
import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.nullable;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public class DefaultMQProducerWithOpenTracingTest {
@Spy
private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig());
@Mock
private MQClientAPIImpl mQClientAPIImpl;
private DefaultMQProducer producer;
private Message message;
private String topic = "FooBar";
private String producerGroupPrefix = "FooBar_PID";
private String producerGroupTemp = producerGroupPrefix + System.currentTimeMillis();
private String producerGroupTraceTemp = TopicValidator.RMQ_SYS_TRACE_TOPIC + System.currentTimeMillis();
private MockTracer tracer = new MockTracer();
@Before
public void init() throws Exception {
producer = new DefaultMQProducer(producerGroupTemp);
producer.getDefaultMQProducerImpl().registerSendMessageHook(
new SendMessageOpenTracingHookImpl(tracer));
producer.setNamesrvAddr("127.0.0.1:9876");
message = new Message(topic, new byte[] {'a', 'b', 'c'});
producer.start();
Field field = DefaultMQProducerImpl.class.getDeclaredField("mQClientFactory");
field.setAccessible(true);
field.set(producer.getDefaultMQProducerImpl(), mQClientFactory);
field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl");
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();
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));
}
@Test
public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
producer.send(message);
assertThat(tracer.finishedSpans().size()).isEqualTo(1);
MockSpan span = tracer.finishedSpans().get(0);
assertThat(span.tags().get(Tags.MESSAGE_BUS_DESTINATION.getKey())).isEqualTo(topic);
assertThat(span.tags().get(Tags.SPAN_KIND.getKey())).isEqualTo(Tags.SPAN_KIND_PRODUCER);
assertThat(span.tags().get(TraceConstants.ROCKETMQ_MSG_ID)).isEqualTo("123");
assertThat(span.tags().get(TraceConstants.ROCKETMQ_BODY_LENGTH)).isEqualTo(3);
assertThat(span.tags().get(TraceConstants.ROCKETMQ_REGION_ID)).isEqualTo("HZ");
assertThat(span.tags().get(TraceConstants.ROCKETMQ_MSG_TYPE)).isEqualTo(MessageType.Normal_Msg.name());
assertThat(span.tags().get(TraceConstants.ROCKETMQ_SOTRE_HOST)).isEqualTo("127.0.0.1:10911");
}
@After
public void terminate() {
producer.shutdown();
}
public static TopicRouteData createTopicRoute() {
TopicRouteData topicRouteData = new TopicRouteData();
topicRouteData.setFilterServerTable(new HashMap<String, List<String>>());
List<BrokerData> brokerDataList = new ArrayList<BrokerData>();
BrokerData brokerData = new BrokerData();
brokerData.setBrokerName("BrokerA");
brokerData.setCluster("DefaultCluster");
HashMap<Long, String> brokerAddrs = new HashMap<Long, String>();
brokerAddrs.put(0L, "127.0.0.1:10911");
brokerData.setBrokerAddrs(brokerAddrs);
brokerDataList.add(brokerData);
topicRouteData.setBrokerDatas(brokerDataList);
List<QueueData> queueDataList = new ArrayList<QueueData>();
QueueData queueData = new QueueData();
queueData.setBrokerName("BrokerA");
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
}
private SendResult createSendResult(SendStatus sendStatus) {
SendResult sendResult = new SendResult();
sendResult.setMsgId("123");
sendResult.setOffsetMsgId("123");
sendResult.setQueueOffset(456);
sendResult.setSendStatus(sendStatus);
sendResult.setRegionId("HZ");
return sendResult;
}
}
......@@ -168,7 +168,7 @@ public class DefaultMQProducerWithTraceTest {
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSynFlag(0);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
......@@ -204,7 +204,7 @@ public class DefaultMQProducerWithTraceTest {
queueData.setPerm(6);
queueData.setReadQueueNums(1);
queueData.setWriteQueueNums(1);
queueData.setTopicSynFlag(1);
queueData.setTopicSysFlag(1);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
......
......@@ -17,6 +17,7 @@
package org.apache.rocketmq.client.trace;
import org.apache.rocketmq.client.producer.LocalTransactionState;
import org.apache.rocketmq.common.UtilAll;
import org.apache.rocketmq.common.message.MessageType;
import org.junit.Assert;
......@@ -90,4 +91,46 @@ public class TraceDataEncoderTest {
Assert.assertEquals(traceTransferBean.getTransKey().size(), 2);
}
}
\ No newline at end of file
@Test
public void testEncoderFromContextBean_EndTransaction() {
TraceContext context = new TraceContext();
context.setTraceType(TraceType.EndTransaction);
context.setGroupName("PID-test");
context.setRegionId("DefaultRegion");
context.setTimeStamp(time);
TraceBean traceBean = new TraceBean();
traceBean.setTopic("topic-test");
traceBean.setKeys("Keys");
traceBean.setTags("Tags");
traceBean.setMsgId("AC1415116D1418B4AAC217FE1B4E0000");
traceBean.setStoreHost("127.0.0.1:10911");
traceBean.setClientHost("127.0.0.1@41700");
traceBean.setMsgType(MessageType.Trans_msg_Commit);
traceBean.setTransactionId("transactionId");
traceBean.setTransactionState(LocalTransactionState.COMMIT_MESSAGE);
traceBean.setFromTransactionCheck(false);
List<TraceBean> traceBeans = new ArrayList<TraceBean>();
traceBeans.add(traceBean);
context.setTraceBeans(traceBeans);
TraceTransferBean traceTransferBean = TraceDataEncoder.encoderFromContextBean(context);
Assert.assertEquals(traceTransferBean.getTransKey().size(), 2);
String traceData = traceTransferBean.getTransData();
TraceContext contextAfter = TraceDataEncoder.decoderFromTraceDataString(traceData).get(0);
Assert.assertEquals(context.getTraceType(), contextAfter.getTraceType());
Assert.assertEquals(context.getTimeStamp(), contextAfter.getTimeStamp());
Assert.assertEquals(context.getGroupName(), contextAfter.getGroupName());
TraceBean before = context.getTraceBeans().get(0);
TraceBean after = contextAfter.getTraceBeans().get(0);
Assert.assertEquals(before.getTopic(), after.getTopic());
Assert.assertEquals(before.getMsgId(), after.getMsgId());
Assert.assertEquals(before.getTags(), after.getTags());
Assert.assertEquals(before.getKeys(), after.getKeys());
Assert.assertEquals(before.getStoreHost(), after.getStoreHost());
Assert.assertEquals(before.getMsgType(), after.getMsgType());
Assert.assertEquals(before.getClientHost(), after.getClientHost());
Assert.assertEquals(before.getTransactionId(), after.getTransactionId());
Assert.assertEquals(before.getTransactionState(), after.getTransactionState());
Assert.assertEquals(before.isFromTransactionCheck(), after.isFromTransactionCheck());
}
}
/*
* 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.trace;
import io.opentracing.mock.MockSpan;
import io.opentracing.mock.MockTracer;
import io.opentracing.tag.Tags;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.hook.SendMessageContext;
import org.apache.rocketmq.client.impl.CommunicationMode;
import org.apache.rocketmq.client.impl.MQClientAPIImpl;
import org.apache.rocketmq.client.impl.MQClientManager;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl;
import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
import org.apache.rocketmq.client.producer.LocalTransactionState;
import org.apache.rocketmq.client.producer.SendCallback;
import org.apache.rocketmq.client.producer.SendResult;
import org.apache.rocketmq.client.producer.SendStatus;
import org.apache.rocketmq.client.producer.TransactionListener;
import org.apache.rocketmq.client.producer.TransactionMQProducer;
import org.apache.rocketmq.client.trace.hook.EndTransactionOpenTracingHookImpl;
import org.apache.rocketmq.client.trace.hook.SendMessageOpenTracingHookImpl;
import org.apache.rocketmq.common.message.Message;
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.message.MessageType;
import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.common.topic.TopicValidator;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Spy;
import org.mockito.junit.MockitoJUnitRunner;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.nullable;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public class TransactionMQProducerWithOpenTracingTest {
@Spy
private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig());
@Mock
private MQClientAPIImpl mQClientAPIImpl;
private TransactionMQProducer producer;
private Message message;
private String topic = "FooBar";
private String producerGroupPrefix = "FooBar_PID";
private String producerGroupTemp = producerGroupPrefix + System.currentTimeMillis();
private String producerGroupTraceTemp = TopicValidator.RMQ_SYS_TRACE_TOPIC + System.currentTimeMillis();
private MockTracer tracer = new MockTracer();
@Before
public void init() throws Exception {
TransactionListener transactionListener = new TransactionListener() {
@Override
public LocalTransactionState executeLocalTransaction(Message msg, Object arg) {
return LocalTransactionState.COMMIT_MESSAGE;
}
@Override
public LocalTransactionState checkLocalTransaction(MessageExt msg) {
return LocalTransactionState.COMMIT_MESSAGE;
}
};
producer = new TransactionMQProducer(producerGroupTemp);
producer.getDefaultMQProducerImpl().registerSendMessageHook(new SendMessageOpenTracingHookImpl(tracer));
producer.getDefaultMQProducerImpl().registerEndTransactionHook(new EndTransactionOpenTracingHookImpl(tracer));
producer.setTransactionListener(transactionListener);
producer.setNamesrvAddr("127.0.0.1:9876");
message = new Message(topic, new byte[] {'a', 'b', 'c'});
producer.start();
Field field = DefaultMQProducerImpl.class.getDeclaredField("mQClientFactory");
field.setAccessible(true);
field.set(producer.getDefaultMQProducerImpl(), mQClientFactory);
field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl");
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();
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));
}
@Test
public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, producer.getDefaultMQProducerImpl());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
producer.sendMessageInTransaction(message, null);
assertThat(tracer.finishedSpans().size()).isEqualTo(2);
MockSpan span = tracer.finishedSpans().get(1);
assertThat(span.tags().get(Tags.MESSAGE_BUS_DESTINATION.getKey())).isEqualTo(topic);
assertThat(span.tags().get(Tags.SPAN_KIND.getKey())).isEqualTo(Tags.SPAN_KIND_PRODUCER);
assertThat(span.tags().get(TraceConstants.ROCKETMQ_MSG_ID)).isEqualTo("123");
assertThat(span.tags().get(TraceConstants.ROCKETMQ_MSG_TYPE)).isEqualTo(MessageType.Trans_msg_Commit.name());
assertThat(span.tags().get(TraceConstants.ROCKETMQ_TRANSACTION_STATE)).isEqualTo(LocalTransactionState.COMMIT_MESSAGE.name());
assertThat(span.tags().get(TraceConstants.ROCKETMQ_IS_FROM_TRANSACTION_CHECK)).isEqualTo(false);
}
@After
public void terminate() {
producer.shutdown();
}
public static TopicRouteData createTopicRoute() {
TopicRouteData topicRouteData = new TopicRouteData();
topicRouteData.setFilterServerTable(new HashMap<String, List<String>>());
List<BrokerData> brokerDataList = new ArrayList<BrokerData>();
BrokerData brokerData = new BrokerData();
brokerData.setBrokerName("BrokerA");
brokerData.setCluster("DefaultCluster");
HashMap<Long, String> brokerAddrs = new HashMap<Long, String>();
brokerAddrs.put(0L, "127.0.0.1:10911");
brokerData.setBrokerAddrs(brokerAddrs);
brokerDataList.add(brokerData);
topicRouteData.setBrokerDatas(brokerDataList);
List<QueueData> queueDataList = new ArrayList<QueueData>();
QueueData queueData = new QueueData();
queueData.setBrokerName("BrokerA");
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
}
private SendResult createSendResult(SendStatus sendStatus) {
SendResult sendResult = new SendResult();
sendResult.setMsgId("123");
sendResult.setOffsetMsgId(MessageDecoder.createMessageId(new InetSocketAddress("127.0.0.1", 12), 1));
sendResult.setQueueOffset(456);
sendResult.setSendStatus(sendStatus);
sendResult.setRegionId("HZ");
sendResult.setMessageQueue(new MessageQueue(topic, "broker-trace", 0));
return sendResult;
}
}
/*
* 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.trace;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.hook.EndTransactionContext;
import org.apache.rocketmq.client.hook.EndTransactionHook;
import org.apache.rocketmq.client.hook.SendMessageContext;
import org.apache.rocketmq.client.impl.CommunicationMode;
import org.apache.rocketmq.client.impl.MQClientAPIImpl;
import org.apache.rocketmq.client.impl.MQClientManager;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl;
import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
import org.apache.rocketmq.client.producer.DefaultMQProducer;
import org.apache.rocketmq.client.producer.LocalTransactionState;
import org.apache.rocketmq.client.producer.SendCallback;
import org.apache.rocketmq.client.producer.SendResult;
import org.apache.rocketmq.client.producer.SendStatus;
import org.apache.rocketmq.client.producer.TransactionListener;
import org.apache.rocketmq.client.producer.TransactionMQProducer;
import org.apache.rocketmq.common.message.Message;
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.SendMessageRequestHeader;
import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.common.topic.TopicValidator;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Spy;
import org.mockito.junit.MockitoJUnitRunner;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.*;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public class TransactionMQProducerWithTraceTest {
@Spy
private MQClientInstance mQClientFactory = MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig());
@Mock
private MQClientAPIImpl mQClientAPIImpl;
@Mock
private EndTransactionHook endTransactionHook;
private AsyncTraceDispatcher asyncTraceDispatcher;
private TransactionMQProducer producer;
private DefaultMQProducer traceProducer;
private Message message;
private String topic = "FooBar";
private String producerGroupPrefix = "FooBar_PID";
private String producerGroupTemp = producerGroupPrefix + System.currentTimeMillis();
private String producerGroupTraceTemp = TopicValidator.RMQ_SYS_TRACE_TOPIC + System.currentTimeMillis();
private String customerTraceTopic = "rmq_trace_topic_12345";
@Before
public void init() throws Exception {
TransactionListener transactionListener = new TransactionListener() {
@Override
public LocalTransactionState executeLocalTransaction(Message msg, Object arg) {
return LocalTransactionState.COMMIT_MESSAGE;
}
@Override
public LocalTransactionState checkLocalTransaction(MessageExt msg) {
return LocalTransactionState.COMMIT_MESSAGE;
}
};
producer = new TransactionMQProducer(null, producerGroupTemp, null, true, null);
producer.setTransactionListener(transactionListener);
producer.setNamesrvAddr("127.0.0.1:9876");
message = new Message(topic, new byte[] {'a', 'b', 'c'});
asyncTraceDispatcher = (AsyncTraceDispatcher) producer.getTraceDispatcher();
traceProducer = asyncTraceDispatcher.getTraceProducer();
producer.start();
Field field = DefaultMQProducerImpl.class.getDeclaredField("mQClientFactory");
field.setAccessible(true);
field.set(producer.getDefaultMQProducerImpl(), mQClientFactory);
Field fieldTrace = DefaultMQProducerImpl.class.getDeclaredField("mQClientFactory");
fieldTrace.setAccessible(true);
fieldTrace.set(traceProducer.getDefaultMQProducerImpl(), mQClientFactory);
field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl");
field.setAccessible(true);
field.set(mQClientFactory, mQClientAPIImpl);
producer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl());
Field fieldHooks = DefaultMQProducerImpl.class.getDeclaredField("endTransactionHookList");
fieldHooks.setAccessible(true);
List<EndTransactionHook>hooks = new ArrayList<>();
hooks.add(endTransactionHook);
fieldHooks.set(producer.getDefaultMQProducerImpl(), hooks);
when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class),
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));
}
@Test
public void testSendMessageSync_WithTrace_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
AtomicReference<EndTransactionContext> context = new AtomicReference<>();
doAnswer(mock -> {
context.set(mock.getArgument(0));
return null;
}).when(endTransactionHook).endTransaction(any());
producer.sendMessageInTransaction(message, null);
EndTransactionContext ctx = context.get();
assertThat(ctx.getProducerGroup()).isEqualTo(producerGroupTemp);
assertThat(ctx.getMsgId()).isEqualTo("123");
assertThat(ctx.isFromTransactionCheck()).isFalse();
assertThat(new String(ctx.getMessage().getBody())).isEqualTo(new String(message.getBody()));
assertThat(ctx.getMessage().getTopic()).isEqualTo(topic);
}
@After
public void terminate() {
producer.shutdown();
}
public static TopicRouteData createTopicRoute() {
TopicRouteData topicRouteData = new TopicRouteData();
topicRouteData.setFilterServerTable(new HashMap<String, List<String>>());
List<BrokerData> brokerDataList = new ArrayList<BrokerData>();
BrokerData brokerData = new BrokerData();
brokerData.setBrokerName("BrokerA");
brokerData.setCluster("DefaultCluster");
HashMap<Long, String> brokerAddrs = new HashMap<Long, String>();
brokerAddrs.put(0L, "127.0.0.1:10911");
brokerData.setBrokerAddrs(brokerAddrs);
brokerDataList.add(brokerData);
topicRouteData.setBrokerDatas(brokerDataList);
List<QueueData> queueDataList = new ArrayList<QueueData>();
QueueData queueData = new QueueData();
queueData.setBrokerName("BrokerA");
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
return topicRouteData;
}
private SendResult createSendResult(SendStatus sendStatus) {
SendResult sendResult = new SendResult();
sendResult.setMsgId("123");
sendResult.setOffsetMsgId(MessageDecoder.createMessageId(new InetSocketAddress("127.0.0.1", 12), 1));
sendResult.setQueueOffset(456);
sendResult.setSendStatus(sendStatus);
sendResult.setRegionId("HZ");
sendResult.setMessageQueue(new MessageQueue(topic, "broker-trace", 0));
return sendResult;
}
}
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<Configuration status="WARN">
<Appenders>
<Console name="Console" target="SYSTEM_OUT">
<PatternLayout pattern="%d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n"/>
</Console>
</Appenders>
<Loggers>
<Root level="ERROR">
<AppenderRef ref="Console"/>
</Root>
</Loggers>
</Configuration>
\ No newline at end of file
......@@ -19,7 +19,7 @@
<parent>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId>
<version>4.8.0-SNAPSHOT</version>
<version>4.8.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
......
......@@ -18,7 +18,7 @@ package org.apache.rocketmq.common;
public class MQVersion {
public static final int CURRENT_VERSION = Version.V4_7_1.ordinal();
public static final int CURRENT_VERSION = Version.V4_8_0.ordinal();
public static String getVersionDesc(int value) {
int length = Version.values().length;
......
......@@ -461,33 +461,9 @@ public class UtilAll {
if (ip.length != 4) {
throw new RuntimeException("illegal ipv4 bytes");
}
if (ip[0] >= (byte) 1 && ip[0] <= (byte) 126) {
if (ip[1] == (byte) 1 && ip[2] == (byte) 1 && ip[3] == (byte) 1) {
return false;
}
if (ip[1] == (byte) 0 && ip[2] == (byte) 0 && ip[3] == (byte) 0) {
return false;
}
return true;
} else if (ip[0] >= (byte) 128 && ip[0] <= (byte) 191) {
if (ip[2] == (byte) 1 && ip[3] == (byte) 1) {
return false;
}
if (ip[2] == (byte) 0 && ip[3] == (byte) 0) {
return false;
}
return true;
} else if (ip[0] >= (byte) 192 && ip[0] <= (byte) 223) {
if (ip[3] == (byte) 1) {
return false;
}
if (ip[3] == (byte) 0) {
return false;
}
return true;
}
return false;
InetAddressValidator validator = InetAddressValidator.getInstance();
return validator.isValidInet4Address(ipToIPv4Str(ip));
}
private static boolean ipV6Check(byte[] ip) {
......
......@@ -20,10 +20,10 @@ public class PermName {
public static final int PERM_PRIORITY = 0x1 << 3;
public static final int PERM_READ = 0x1 << 2;
public static final int PERM_WRITE = 0x1 << 1;
public static final int PERM_INHERIT = 0x1 << 0;
public static final int PERM_INHERIT = 0x1;
public static String perm2String(final int perm) {
final StringBuffer sb = new StringBuffer("---");
final StringBuilder sb = new StringBuilder("---");
if (isReadable(perm)) {
sb.replace(0, 1, "R");
}
......
......@@ -36,8 +36,7 @@ public class FilterAPI {
return simple;
}
public static SubscriptionData buildSubscriptionData(final String consumerGroup, String topic,
String subString) throws Exception {
public static SubscriptionData buildSubscriptionData(String topic, String subString) throws Exception {
SubscriptionData subscriptionData = new SubscriptionData();
subscriptionData.setTopic(topic);
subscriptionData.setSubString(subString);
......@@ -67,7 +66,7 @@ public class FilterAPI {
public static SubscriptionData build(final String topic, final String subString,
final String type) throws Exception {
if (ExpressionType.TAG.equals(type) || type == null) {
return buildSubscriptionData(null, topic, subString);
return buildSubscriptionData(topic, subString);
}
if (subString == null || subString.length() < 1) {
......
......@@ -20,7 +20,7 @@ package org.apache.rocketmq.common.hook;
import java.nio.ByteBuffer;
public interface FilterCheckHook {
public String hookName();
String hookName();
public boolean isFilterMatched(final boolean isUnitMode, final ByteBuffer byteBuffer);
boolean isFilterMatched(final boolean isUnitMode, final ByteBuffer byteBuffer);
}
......@@ -42,6 +42,7 @@ public class MessageDecoder {
public static final char NAME_VALUE_SEPARATOR = 1;
public static final char PROPERTY_SEPARATOR = 2;
public static final int PHY_POS_POSITION = 4 + 4 + 4 + 4 + 4 + 8;
public static final int QUEUE_OFFSET_POSITION = 4 + 4 + 4 + 4 + 4;
public static final int SYSFLAG_POSITION = 4 + 4 + 4 + 4 + 4 + 8 + 8;
// public static final int BODY_SIZE_POSITION = 4 // 1 TOTALSIZE
// + 4 // 2 MAGICCODE
......@@ -104,7 +105,7 @@ public class MessageDecoder {
*
* @param byteBuffer msg commit log buffer.
*/
public static Map<String, String> decodeProperties(java.nio.ByteBuffer byteBuffer) {
public static Map<String, String> decodeProperties(ByteBuffer byteBuffer) {
int sysFlag = byteBuffer.getInt(SYSFLAG_POSITION);
int bornhostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20;
int storehostAddressLength = (sysFlag & MessageSysFlag.STOREHOSTADDRESS_V6_FLAG) == 0 ? 8 : 20;
......@@ -140,15 +141,15 @@ public class MessageDecoder {
return null;
}
public static MessageExt decode(java.nio.ByteBuffer byteBuffer) {
public static MessageExt decode(ByteBuffer byteBuffer) {
return decode(byteBuffer, true, true, false);
}
public static MessageExt clientDecode(java.nio.ByteBuffer byteBuffer, final boolean readBody) {
public static MessageExt clientDecode(ByteBuffer byteBuffer, final boolean readBody) {
return decode(byteBuffer, readBody, true, true);
}
public static MessageExt decode(java.nio.ByteBuffer byteBuffer, final boolean readBody) {
public static MessageExt decode(ByteBuffer byteBuffer, final boolean readBody) {
return decode(byteBuffer, readBody, true, false);
}
......@@ -263,12 +264,12 @@ public class MessageDecoder {
}
public static MessageExt decode(
java.nio.ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody) {
ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody) {
return decode(byteBuffer, readBody, deCompressBody, false);
}
public static MessageExt decode(
java.nio.ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody, final boolean isClient) {
ByteBuffer byteBuffer, final boolean readBody, final boolean deCompressBody, final boolean isClient) {
try {
MessageExt msgExt;
......@@ -390,11 +391,11 @@ public class MessageDecoder {
return null;
}
public static List<MessageExt> decodes(java.nio.ByteBuffer byteBuffer) {
public static List<MessageExt> decodes(ByteBuffer byteBuffer) {
return decodes(byteBuffer, true);
}
public static List<MessageExt> decodes(java.nio.ByteBuffer byteBuffer, final boolean readBody) {
public static List<MessageExt> decodes(ByteBuffer byteBuffer, final boolean readBody) {
List<MessageExt> msgExts = new ArrayList<MessageExt>();
while (byteBuffer.hasRemaining()) {
MessageExt msgExt = clientDecode(byteBuffer, readBody);
......
......@@ -25,6 +25,8 @@ public class DeleteSubscriptionGroupRequestHeader implements CommandCustomHeader
@CFNotNull
private String groupName;
private boolean removeOffset;
@Override
public void checkFields() throws RemotingCommandException {
}
......@@ -36,4 +38,12 @@ public class DeleteSubscriptionGroupRequestHeader implements CommandCustomHeader
public void setGroupName(String groupName) {
this.groupName = groupName;
}
public boolean isRemoveOffset() {
return removeOffset;
}
public void setRemoveOffset(boolean removeOffset) {
this.removeOffset = removeOffset;
}
}
......@@ -25,7 +25,7 @@ public class QueueData implements Comparable<QueueData> {
private int readQueueNums;
private int writeQueueNums;
private int perm;
private int topicSynFlag;
private int topicSysFlag;
public int getReadQueueNums() {
return readQueueNums;
......@@ -51,12 +51,12 @@ public class QueueData implements Comparable<QueueData> {
this.perm = perm;
}
public int getTopicSynFlag() {
return topicSynFlag;
public int getTopicSysFlag() {
return topicSysFlag;
}
public void setTopicSynFlag(int topicSynFlag) {
this.topicSynFlag = topicSynFlag;
public void setTopicSysFlag(int topicSysFlag) {
this.topicSysFlag = topicSysFlag;
}
@Override
......@@ -67,7 +67,7 @@ public class QueueData implements Comparable<QueueData> {
result = prime * result + perm;
result = prime * result + readQueueNums;
result = prime * result + writeQueueNums;
result = prime * result + topicSynFlag;
result = prime * result + topicSysFlag;
return result;
}
......@@ -91,7 +91,7 @@ public class QueueData implements Comparable<QueueData> {
return false;
if (writeQueueNums != other.writeQueueNums)
return false;
if (topicSynFlag != other.topicSynFlag)
if (topicSysFlag != other.topicSysFlag)
return false;
return true;
}
......@@ -99,7 +99,7 @@ public class QueueData implements Comparable<QueueData> {
@Override
public String toString() {
return "QueueData [brokerName=" + brokerName + ", readQueueNums=" + readQueueNums
+ ", writeQueueNums=" + writeQueueNums + ", perm=" + perm + ", topicSynFlag=" + topicSynFlag
+ ", writeQueueNums=" + writeQueueNums + ", perm=" + perm + ", topicSysFlag=" + topicSysFlag
+ "]";
}
......
......@@ -19,8 +19,7 @@ import org.apache.rocketmq.common.MixAll;
public class NameServerAddressUtils {
public static final String INSTANCE_PREFIX = "MQ_INST_";
public static final String INSTANCE_REGEX = INSTANCE_PREFIX + "\\w+_\\w+";
public static final String ENDPOINT_PREFIX = "http://";
public static final Pattern NAMESRV_ENDPOINT_PATTERN = Pattern.compile("^" + ENDPOINT_PREFIX + ".*");
public static final String ENDPOINT_PREFIX = "(\\w+://|)";
public static final Pattern INST_ENDPOINT_PATTERN = Pattern.compile("^" + ENDPOINT_PREFIX + INSTANCE_REGEX + "\\..*");
public static String getNameServerAddresses() {
......@@ -35,6 +34,13 @@ public class NameServerAddressUtils {
if (StringUtils.isEmpty(endpoint)) {
return null;
}
return endpoint.substring(ENDPOINT_PREFIX.length(), endpoint.indexOf('.'));
return endpoint.substring(endpoint.lastIndexOf("/") + 1, endpoint.indexOf('.'));
}
public static String getNameSrvAddrFromNamesrvEndpoint(String nameSrvEndpoint) {
if (StringUtils.isEmpty(nameSrvEndpoint)) {
return null;
}
return nameSrvEndpoint.substring(nameSrvEndpoint.lastIndexOf('/') + 1);
}
}
......@@ -33,7 +33,7 @@ public class FilterAPITest {
@Test
public void testBuildSubscriptionData() throws Exception {
SubscriptionData subscriptionData =
FilterAPI.buildSubscriptionData(group, topic, subString);
FilterAPI.buildSubscriptionData(topic, subString);
assertThat(subscriptionData.getTopic()).isEqualTo(topic);
assertThat(subscriptionData.getSubString()).isEqualTo(subString);
String[] tags = subString.split("\\|\\|");
......
......@@ -35,12 +35,14 @@ public class MessageClientIDSetterTest {
assertThat(ipStr).isEqualTo(ipStrFromID);
}
@Test
public void testGetPidFromID() {
int pid = UtilAll.getPid();
// Temporary fix on MacOS
short pid = (short) UtilAll.getPid();
String uniqID = MessageClientIDSetter.createUniqID();
int pidFromID = MessageClientIDSetter.getPidFromID(uniqID);
short pidFromID = (short) MessageClientIDSetter.getPidFromID(uniqID);
assertThat(pid).isEqualTo(pidFromID);
}
......
......@@ -43,7 +43,7 @@ public class TopicRouteDataTest {
queueData.setPerm(6);
queueData.setReadQueueNums(8);
queueData.setWriteQueueNums(8);
queueData.setTopicSynFlag(0);
queueData.setTopicSysFlag(0);
List<QueueData> queueDataList = new ArrayList<QueueData>();
queueDataList.add(queueData);
......@@ -78,7 +78,7 @@ public class TopicRouteDataTest {
queueData.setPerm(6);
queueData.setReadQueueNums(8);
queueData.setWriteQueueNums(8);
queueData.setTopicSynFlag(0);
queueData.setTopicSysFlag(0);
List<QueueData> queueDataList = new ArrayList<QueueData>();
queueDataList.add(queueData);
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.common.utils;
import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
public class NameServerAddressUtilsTest {
private static String endpoint1 = "http://127.0.0.1:9876";
private static String endpoint2 = "127.0.0.1:9876";
private static String endpoint3
= "http://MQ_INST_123456789_BXXUzaee.xxx:80";
private static String endpoint4 = "MQ_INST_123456789_BXXUzaee.xxx:80";
@Test
public void testValidateInstanceEndpoint() {
assertThat(NameServerAddressUtils.validateInstanceEndpoint(endpoint1)).isEqualTo(false);
assertThat(NameServerAddressUtils.validateInstanceEndpoint(endpoint2)).isEqualTo(false);
assertThat(NameServerAddressUtils.validateInstanceEndpoint(endpoint3)).isEqualTo(true);
assertThat(NameServerAddressUtils.validateInstanceEndpoint(endpoint4)).isEqualTo(true);
}
@Test
public void testParseInstanceIdFromEndpoint() {
assertThat(NameServerAddressUtils.parseInstanceIdFromEndpoint(endpoint3)).isEqualTo(
"MQ_INST_123456789_BXXUzaee");
assertThat(NameServerAddressUtils.parseInstanceIdFromEndpoint(endpoint4)).isEqualTo(
"MQ_INST_123456789_BXXUzaee");
}
@Test
public void testGetNameSrvAddrFromNamesrvEndpoint() {
assertThat(NameServerAddressUtils.getNameSrvAddrFromNamesrvEndpoint(endpoint1))
.isEqualTo("127.0.0.1:9876");
assertThat(NameServerAddressUtils.getNameSrvAddrFromNamesrvEndpoint(endpoint2))
.isEqualTo("127.0.0.1:9876");
assertThat(NameServerAddressUtils.getNameSrvAddrFromNamesrvEndpoint(endpoint3))
.isEqualTo("MQ_INST_123456789_BXXUzaee.xxx:80");
assertThat(NameServerAddressUtils.getNameSrvAddrFromNamesrvEndpoint(endpoint4))
.isEqualTo("MQ_INST_123456789_BXXUzaee.xxx:80");
}
}
......@@ -15,7 +15,7 @@
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (properties) the power, direct or indirect, to cause the
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
......
Apache RocketMQ
Copyright 2016-2019 The Apache Software Foundation
Copyright 2016-2021 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
......
#!/bin/sh
# 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.
case $1 in
producer)
pid=`ps ax | grep -i 'org.apache.rocketmq.example.benchmark.Producer' |grep java | grep -v grep | awk '{print $1}'`
if [ -z "$pid" ] ; then
echo "No benchmark producer running."
exit -1;
fi
echo "The benchmkar producer(${pid}) is running..."
kill ${pid}
echo "Send shutdown request to benchmark producer(${pid}) OK"
;;
consumer)
pid=`ps ax | grep -i 'org.apache.rocketmq.example.benchmark.Consumer' |grep java | grep -v grep | awk '{print $1}'`
if [ -z "$pid" ] ; then
echo "No benchmark consumer running."
exit -1;
fi
echo "The benchmark consumer(${pid}) is running..."
kill ${pid}
echo "Send shutdown request to benchmark consumer(${pid}) OK"
;;
tproducer)
pid=`ps ax | grep -i 'org.apache.rocketmq.example.benchmark.TransactionProducer' |grep java | grep -v grep | awk '{print $1}'`
if [ -z "$pid" ] ; then
echo "No benchmark transaction producer running."
exit -1;
fi
echo "The benchmkar transaction producer(${pid}) is running..."
kill ${pid}
echo "Send shutdown request to benchmark transaction producer(${pid}) OK"
;;
*)
echo "Useage: shutdown producer | consumer | tproducer"
esac
......@@ -15,4 +15,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
sh ./runclass.sh org.apache.rocketmq.example.benchmark.TransactionProducer $@
sh ./runclass.sh org.apache.rocketmq.example.benchmark.TransactionProducer $@ &
......@@ -36,7 +36,7 @@ set "JAVA_OPT=%JAVA_OPT% -XX:-OmitStackTraceInFastThrow"
set "JAVA_OPT=%JAVA_OPT% -XX:+AlwaysPreTouch"
set "JAVA_OPT=%JAVA_OPT% -XX:MaxDirectMemorySize=15g"
set "JAVA_OPT=%JAVA_OPT% -XX:-UseLargePages -XX:-UseBiasedLocking"
set "JAVA_OPT=%JAVA_OPT% -Djava.ext.dirs=%BASE_DIR%lib"
set "JAVA_OPT=%JAVA_OPT% -Djava.ext.dirs=%BASE_DIR%lib;%JAVA_HOME%\jre\lib\ext"
set "JAVA_OPT=%JAVA_OPT% -cp %CLASSPATH%"
"%JAVA%" %JAVA_OPT% %*
\ No newline at end of file
......@@ -31,7 +31,7 @@ set "JAVA_OPT=%JAVA_OPT% -XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollect
set "JAVA_OPT=%JAVA_OPT% -verbose:gc -Xloggc:"%USERPROFILE%\rmq_srv_gc.log" -XX:+PrintGCDetails"
set "JAVA_OPT=%JAVA_OPT% -XX:-OmitStackTraceInFastThrow"
set "JAVA_OPT=%JAVA_OPT% -XX:-UseLargePages"
set "JAVA_OPT=%JAVA_OPT% -Djava.ext.dirs=%BASE_DIR%lib"
set "JAVA_OPT=%JAVA_OPT% -Djava.ext.dirs=%BASE_DIR%lib;%JAVA_HOME%\jre\lib\ext"
set "JAVA_OPT=%JAVA_OPT% -cp "%CLASSPATH%""
"%JAVA%" %JAVA_OPT% %*
\ No newline at end of file
......@@ -66,8 +66,8 @@ choose_gc_options()
{
# Example of JAVA_MAJOR_VERSION value : '1', '9', '10', '11', ...
# '1' means releases befor Java 9
JAVA_MAJOR_VERSION=$("$JAVA" -version 2>&1 | sed -E -n 's/.* version "([0-9]*).*$/\1/p')
if [[ "$JAVA_MAJOR_VERSION" -lt "9" ]] ; then
JAVA_MAJOR_VERSION=$("$JAVA" -version 2>&1 | sed -r -n 's/.* version "([0-9]*).*$/\1/p')
if [ -z "$JAVA_MAJOR_VERSION" ] || [ "$JAVA_MAJOR_VERSION" -lt "9" ] ; then
JAVA_OPT="${JAVA_OPT} -XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC"
JAVA_OPT="${JAVA_OPT} -verbose:gc -Xloggc:${GC_LOG_DIR}/rmq_srv_gc_%p_%t.log -XX:+PrintGCDetails"
JAVA_OPT="${JAVA_OPT} -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
......
......@@ -20,7 +20,7 @@
<parent>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId>
<version>4.8.0-SNAPSHOT</version>
<version>4.8.1-SNAPSHOT</version>
</parent>
<artifactId>rocketmq-distribution</artifactId>
<name>rocketmq-distribution ${project.version}</name>
......
# 样例
-----
* [目录](#样例)
* [1 基本样例](#1-基本样例)
* [1.1 加入依赖:](#11-加入依赖)
* [1.2 消息发送](#12-消息发送)
* [1、Producer端发送同步消息](#1producer端发送同步消息)
* [2、发送异步消息](#2发送异步消息)
* [3、单向发送消息](#3单向发送消息)
* [1.3 消费消息](#13-消费消息)
* [2 顺序消息样例](#2-顺序消息样例)
* [2.1 顺序消息生产](#21-顺序消息生产)
* [2.2 顺序消费消息](#22-顺序消费消息)
* [3 延时消息样例](#3-延时消息样例)
* [3.1 启动消费者等待传入订阅消息](#31-启动消费者等待传入订阅消息)
* [3.2 发送延时消息](#32-发送延时消息)
* [3.3 验证](#33-验证)
* [3.4 延时消息的使用场景](#34-延时消息的使用场景)
* [3.5 延时消息的使用限制](#35-延时消息的使用限制)
* [4 批量消息样例](#4-批量消息样例)
* [4.1 发送批量消息](#41-发送批量消息)
* [4.2 消息列表分割](#42-消息列表分割)
* [5 过滤消息样例](#5-过滤消息样例)
* [5.1 基本语法](#51-基本语法)
* [5.2 使用样例](#52-使用样例)
* [1、生产者样例](#1生产者样例)
* [2、消费者样例](#2消费者样例)
* [6 消息事务样例](#6-消息事务样例)
* [6.1 发送事务消息样例](#61-发送事务消息样例)
* [1、创建事务性生产者](#1创建事务性生产者)
* [2、实现事务的监听接口](#2实现事务的监听接口)
* [6.2 事务消息使用上的限制](#62-事务消息使用上的限制)
* [7 Logappender样例](#7-logappender样例)
* [7.1 log4j样例](#71-log4j样例)
* [7.2 log4j2样例](#72-log4j2样例)
* [7.3 logback样例](#73-logback样例)
* [8 OpenMessaging样例](#8-openmessaging样例)
* [8.1 OMSProducer样例](#81-omsproducer样例)
* [8.2 OMSPullConsumer](#82-omspullconsumer)
* [8.3 OMSPushConsumer](#83-omspushconsumer)
-----
## 1 基本样例
......@@ -407,7 +446,7 @@ public class ScheduledMessageConsumer {
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> messages, ConsumeConcurrentlyContext context) {
for (MessageExt message : messages) {
// Print approximate delay time period
System.out.println("Receive message[msgId=" + message.getMsgId() + "] " + (System.currentTimeMillis() - message.getStoreTimestamp()) + "ms later");
System.out.println("Receive message[msgId=" + message.getMsgId() + "] " + (System.currentTimeMillis() - message.getBornTimestamp()) + "ms later");
}
return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
}
......
......@@ -12,11 +12,11 @@ RocketMQ架构上主要分为四部分,如上图所示:
- NameServer:NameServer是一个非常简单的Topic路由注册中心,其角色类似Dubbo中的zookeeper,支持Broker的动态注册与发现。主要包括两个功能:Broker管理,NameServer接受Broker集群的注册信息并且保存下来作为路由信息的基本数据。然后提供心跳检测机制,检查Broker是否还存活;路由信息管理,每个NameServer将保存关于Broker集群的整个路由信息和用于客户端查询的队列信息。然后Producer和Conumser通过NameServer就可以知道整个Broker集群的路由信息,从而进行消息的投递和消费。NameServer通常也是集群的方式部署,各实例间相互不进行信息通讯。Broker是向每一台NameServer注册自己的路由信息,所以每一个NameServer实例上面都保存一份完整的路由信息。当某个NameServer因某种原因下线了,Broker仍然可以向其它NameServer同步其路由信息,Producer,Consumer仍然可以动态感知Broker的路由的信息。
- BrokerServer:Broker主要负责消息的存储、投递和查询以及服务高可用保证,为了实现这些功能,Broker包含了以下几个重要子模块。
1. Remoting Module:整个Broker的实体,负责处理来自clients端的请求。
2. Client Manager:负责管理客户端(Producer/Consumer)和维护Consumer的Topic订阅信息
3. Store Service:提供方便简单的API接口处理消息存储到物理硬盘和查询功能。
4. HA Service:高可用服务,提供Master Broker 和 Slave Broker之间的数据同步功能。
5. Index Service:根据特定的Message key对投递到Broker的消息进行索引服务,以提供消息的快速查询。
1. Remoting Module:整个Broker的实体,负责处理来自clients端的请求。
2. Client Manager:负责管理客户端(Producer/Consumer)和维护Consumer的Topic订阅信息
3. Store Service:提供方便简单的API接口处理消息存储到物理硬盘和查询功能。
4. HA Service:高可用服务,提供Master Broker 和 Slave Broker之间的数据同步功能。
5. Index Service:根据特定的Message key对投递到Broker的消息进行索引服务,以提供消息的快速查询。
![](image/rocketmq_architecture_2.png)
......
......@@ -45,8 +45,8 @@
Producer的send方法本身支持内部重试,重试逻辑如下:
- 至多重试2次(同步发送为2次,异步发送为0次)
- 如果发送失败,则轮转到下一个Broker。这个方法的总耗时时间不超过sendMsgTimeout设置的值,默认10s。
- 至多重试2次。
- 如果同步模式发送失败,则轮转到下一个Broker,如果异步模式发送失败,则只会在当前Broker进行重试。这个方法的总耗时时间不超过sendMsgTimeout设置的值,默认10s。
- 如果本身向broker发送消息产生超时异常,就不会再重试。
以上策略也是在一定程度上保证了消息可以发送成功。如果业务对消息可靠性要求比较高,建议应用增加相应的重试逻辑:比如调用send同步方法发送失败时,则尝试将消息存储到db,然后由后台线程定时重试,确保消息一定到达Broker。
......
......@@ -38,7 +38,7 @@ RocketMQ主要由 Producer、Broker、Consumer 三部分组成,其中Producer
广播消费模式下,相同Consumer Group的每个Consumer实例都接收全量的消息。
## 13 普通顺序消息(Normal Ordered Message)
普通顺序消费模式下,消费者通过同一个消费队列收到的消息是有顺序的,不同消息队列收到的消息则可能是无顺序的。
普通顺序消费模式下,消费者通过同一个消息队列( Topic 分区,称作 Message Queue) 收到的消息是有顺序的,不同消息队列收到的消息则可能是无顺序的。
## 14 严格顺序消息(Strictly Ordered Message)
严格顺序消息模式下,消费者收到的所有消息均是有顺序的。
......
......@@ -58,7 +58,7 @@ code |int | 请求操作码,应答方根据不同的请求码进行不同的
language | LanguageCode | 请求方实现的语言 | 应答方实现的语言
version | int | 请求方程序的版本 | 应答方程序的版本
opaque | int |相当于requestId,在同一个连接上的不同请求标识码,与响应消息中的相对应 | 应答不做修改直接返回
flag | int | 区分是普通RPC还是onewayRPC得标志 | 区分是普通RPC还是onewayRPC得标志
flag | int | 区分是普通RPC还是onewayRPC的标志 | 区分是普通RPC还是onewayRPC的标志
remark | String | 传输自定义文本信息 | 传输自定义文本信息
extFields | HashMap<String, String> | 请求自定义扩展信息 | 响应自定义扩展信息
......
......@@ -4,7 +4,7 @@
## 1 使用场景
随着服务规模的扩大,单机服务无法满足性能和容量的要求,此时需要将服务拆分为更小粒度的服务或者部署多个服务实例构成集群来提供服务。在分布式场景下,RPC是最常用的联机调用的方式。
在构建分布式应用时,有些领域,例如金融服务领域,常常使用消息队列来构建服务总线,实现联机调用的目的。消息队列的主要场景是解耦、削峰填谷,在联机调用的场景下,需要将服务的调用抽象成基于消息的交互,并增强同步调用的这种交互逻辑。为了更好地支持消息队列在联机调用场景下的应用,rocketmq-4.7.0推出了“Request-Reply”特性来支持RPC调用。
在构建分布式应用时,有些领域,例如金融服务领域,常常使用消息队列来构建服务总线,实现联机调用的目的。消息队列的主要场景是解耦、削峰填谷,在联机调用的场景下,需要将服务的调用抽象成基于消息的交互,并增强同步调用的这种交互逻辑。为了更好地支持消息队列在联机调用场景下的应用,rocketmq-4.6.0推出了“Request-Reply”特性来支持RPC调用。
## 2 设计思路
在rocketmq中,整个同步调用主要包括两个过程:
......
## Load Balancing
Load balancing in RocketMQ is accomplished on Client side. Specifically, it can be divided into load balancing at Producer side when sending messages and load balancing at Constumer side when subscribing messages.
### 1 Producer Load Balancing
### Producer Load Balancing
When the Producer sends a message, it will first find the specified TopicPublishInfo according to Topic. After getting the routing information of TopicPublishInfo, the RocketMQ client will select a queue (MessageQueue) from the messageQueue List in TopicPublishInfo to send the message by default.Specific fault-tolerant strategies are defined in the MQFaultStrategy class.
Here is a sendLatencyFaultEnable switch variable, which, if turned on, filters out the Broker agent of not available on the basis of randomly gradually increasing modular arithmetic selection. The so-called "latencyFault Tolerance" refers to a certain period of time to avoid previous failures. For example, if the latency of the last request exceeds 550 Lms, it will evade 3000 Lms; if it exceeds 1000L, it will evade 60000 L; if it is closed, it will choose a queue (MessageQueue) to send messages by randomly gradually increasing modular arithmetic, and the latencyFault Tolerance mechanism is the key to achieve high availability of message sending.
### 2 Consumer Load Balancing
### Consumer Load Balancing
In RocketMQ, the two consumption modes (Push/Pull) on the Consumer side are both based on the pull mode to get the message, while in the Push mode it is only a kind of encapsulation of the pull mode, which is essentially implemented as the message pulling thread after pulling a batch of messages from the server. After submitting to the message consuming thread pool, it continues to try again to pull the message to the server. If the message is not pulled, the pull is delayed and continues. In both pull mode based consumption patterns (Push/Pull), the Consumer needs to know which message queue - queue from the Broker side to get the message. Therefore, it is necessary to do load balancing on the Consumer side, that is, which Consumer consumption is allocated to the same ConsumerGroup by more than one MessageQueue on the Broker side.
1, Heartbeat Packet Sending on Consumer side
#### 1 Heartbeat Packet Sending on Consumer side
After Consumer is started, it continuously sends heartbeat packets to all Broker instances in the RocketMQ cluster via timing task (which contains the message consumption group name, subscription relationship collection,Message communication mode and the value of the client id,etc). After receiving the heartbeat message from Consumer, Broker side maintains it in Consumer Manager's local caching variable—consumerTable, At the same time, the encapsulated client network channel information is stored in the local caching variable—channelInfoTable, which can provide metadata information for the later load balancing of Consumer.
2,Core Class for Load Balancing on Consumer side—RebalanceImpl
#### 2 Core Class for Load Balancing on Consumer side—RebalanceImpl
Starting the MQClientInstance instance in the startup process of the Consumer instance will complete the start of the load balancing service thread-RebalanceService (executed every 20 s). By looking at the source code, we can find that the run () method of the RebalanceService thread calls the rebalanceByTopic () method of the RebalanceImpl class, which is the core of the Consumer end load balancing. Here, rebalanceByTopic () method will do different logical processing depending on whether the consumer communication type is "broadcast mode" or "cluster mode". Here we mainly look at the main processing flow in cluster mode:
(1) Get the message consumption queue set (mqSet) under the Topic from the local cache variable—topicSubscribeInfoTable of the rebalanceImpl instance.
(2) Call mQClientFactory. findConsumerIdList () method to send a RPC communication request to Broker side to obtain the consumer Id list under the consumer group based on the parameters of topic and consumer group (consumer table constructed by Broker side based on the heartbeat data reported by the front consumer side responds and returns, business request code: GET_CONSUMER_LIST_BY_GROUP);
(3) First, the message consumption queue and the consumer Id under Topic are sorted, then the message queue to be pulled is calculated by using the message queue allocation strategy algorithm (default: the average allocation algorithm of the message queue). The average allocation algorithm here is similar to the paging algorithm. It ranks all MessageQueues like records. It ranks all consumers like pages. It calculates the average size of each page and the range of each page record. Finally, it traverses the whole range and calculates the records that the current consumer should allocate to (MessageQueue here).
##### 1) Get the message consumption queue set (mqSet) under the Topic from the local cache variable—topicSubscribeInfoTable of the rebalanceImpl instance.
##### 2) Call mQClientFactory. findConsumerIdList () method to send a RPC communication request to Broker side to obtain the consumer Id list under the consumer group based on the parameters of topic and consumer group (consumer table constructed by Broker side based on the heartbeat data reported by the front consumer side responds and returns, business request code: GET_CONSUMER_LIST_BY_GROUP);
##### 3) First, the message consumption queue and the consumer Id under Topic are sorted, then the message queue to be pulled is calculated by using the message queue allocation strategy algorithm (default: the average allocation algorithm of the message queue). The average allocation algorithm here is similar to the paging algorithm. It ranks all MessageQueues like records. It ranks all consumers like pages. It calculates the average size of each page and the range of each page record. Finally, it traverses the whole range and calculates the records that the current consumer should allocate to (MessageQueue here).
![Image text](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_8.png)
(4) Then, the updateProcessQueueTableInRebalance () method is invoked, which first compares the allocated message queue set (mqSet) with processQueueTable for filtering.
##### 4) Then, the updateProcessQueueTableInRebalance () method is invoked, which first compares the allocated message queue set (mqSet) with processQueueTable for filtering.
![Image text](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_9.png)
- The red part of the processQueueTable annotation in the figure above
......@@ -37,6 +39,6 @@ Starting the MQClientInstance instance in the startup process of the Consumer in
removeUnnecessaryMessageQueue () method to try to remove Entry as
above;
Finally, a ProcessQueue object is created for each MessageQueue in the filtered message queue set (mqSet) and stored in the processQueueTable queue of RebalanceImpl (where the computePullFromWhere (MessageQueue mq) method of the RebalanceImpl instance is invoked to obtain the next progress consumption value offset of the MessageQueue object, which is then populated into the attribute of pullRequest object to be created next time.), and create pull request object—pullRequest to add to pull list—pullRequestList, and finally execute dispatchPullRequest () method. PullRequest object of Pull message is put into the blocking queue pullRequestQueue of PullMessageService service thread in turn, and the request of Pull message is sent to Broker end after the service thread takes out. Among them, we can focus on the contrast, RebalancePushImpl and RebalancePullImpl two implementation classes dispatchPullRequest () method is different, the method in RebalancePullImpl class is empty, thus answering the last question in the previous article.
Finally, a ProcessQueue object is created for each MessageQueue in the filtered message queue set (mqSet) and stored in the processQueueTable queue of RebalanceImpl (where the computePullFromWhere (MessageQueue mq) method of the RebalanceImpl instance is invoked to obtain the next progress consumption value offset of the MessageQueue object, which is then populated into the attribute of pullRequest object to be created next time.), and create pull request object—pullRequest to add to pull list—pullRequestList, and finally execute dispatchPullRequest () method. PullRequest object of Pull message is put into the blocking queue pullRequestQueue of PullMessageService service thread in turn, and the request of Pull message is sent to Broker end after the service thread takes out.
The core design idea of message consumption queue is that a message consumption queue can only be consumed by one consumer in the same consumer group at the same time, and a message consumer can consume multiple message queues at the same time.
......@@ -19,7 +19,7 @@
<parent>
<artifactId>rocketmq-all</artifactId>
<groupId>org.apache.rocketmq</groupId>
<version>4.8.0-SNAPSHOT</version>
<version>4.8.1-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
......@@ -52,5 +52,15 @@
<groupId>org.javassist</groupId>
<artifactId>javassist</artifactId>
</dependency>
<dependency>
<groupId>io.jaegertracing</groupId>
<artifactId>jaeger-core</artifactId>
<version>1.6.0</version>
</dependency>
<dependency>
<groupId>io.jaegertracing</groupId>
<artifactId>jaeger-client</artifactId>
<version>1.6.0</version>
</dependency>
</dependencies>
</project>
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册