提交 b4c099cb 编写于 作者: A ayanamist

[RIP-21] submodule common & client & remoting

上级 efe598db
......@@ -27,6 +27,19 @@
<artifactId>rocketmq-client</artifactId>
<name>rocketmq-client ${project.version}</name>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>6</source>
<target>6</target>
</configuration>
</plugin>
</plugins>
</build>
<dependencies>
<dependency>
<groupId>${project.groupId}</groupId>
......@@ -73,5 +86,9 @@
<artifactId>log4j-slf4j-impl</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
</dependencies>
</project>
......@@ -25,6 +25,12 @@ public class MQBrokerException extends Exception {
private final String errorMessage;
private final String brokerAddr;
MQBrokerException() {
this.responseCode = 0;
this.errorMessage = null;
this.brokerAddr = null;
}
public MQBrokerException(int responseCode, String errorMessage) {
super(FAQUrl.attachDefaultURL("CODE: " + UtilAll.responseCode2String(responseCode) + " DESC: "
+ errorMessage));
......
/*
* 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.exception;
public class MQRedirectException extends MQBrokerException {
private static final StackTraceElement[] UNASSIGNED_STACK = new StackTraceElement[0];
private final byte[] body;
public MQRedirectException(byte[] responseBody) {
this.body = responseBody;
}
// This exception class is used as a flow control item, so stack trace is useless and performance killer.
@Override public synchronized Throwable fillInStackTrace() {
this.setStackTrace(UNASSIGNED_STACK);
return this;
}
public byte[] getBody() {
return body;
}
}
......@@ -16,40 +16,34 @@
*/
package org.apache.rocketmq.client.impl;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import com.google.common.base.Objects;
import com.google.common.collect.Lists;
import org.apache.rocketmq.client.QueryResult;
import org.apache.rocketmq.client.Validators;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.exception.MQRedirectException;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.TopicConfig;
import org.apache.rocketmq.common.help.FAQUrl;
import org.apache.rocketmq.common.protocol.NamespaceUtil;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.common.message.MessageClientIDSetter;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageDecoder;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.message.MessageId;
import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.common.protocol.NamespaceUtil;
import org.apache.rocketmq.common.protocol.ResponseCode;
import org.apache.rocketmq.common.protocol.header.QueryMessageRequestHeader;
import org.apache.rocketmq.common.protocol.header.QueryMessageResponseHeader;
import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.remoting.InvokeCallback;
import org.apache.rocketmq.remoting.common.RemotingUtil;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
......@@ -57,6 +51,17 @@ import org.apache.rocketmq.remoting.exception.RemotingException;
import org.apache.rocketmq.remoting.netty.ResponseFuture;
import org.apache.rocketmq.remoting.protocol.RemotingCommand;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
public class MQAdminImpl {
private final InternalLogger log = ClientLogger.getLog();
......@@ -182,6 +187,10 @@ public class MQAdminImpl {
}
public long searchOffset(MessageQueue mq, long timestamp) throws MQClientException {
LogicalQueueRouteData logicalQueueRouteData = searchLogicalQueueRouteByTimestamp(mq, timestamp);
if (logicalQueueRouteData != null) {
mq = logicalQueueRouteData.getMessageQueue();
}
String brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
if (null == brokerAddr) {
this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic());
......@@ -190,8 +199,9 @@ public class MQAdminImpl {
if (brokerAddr != null) {
try {
return this.mQClientFactory.getMQClientAPIImpl().searchOffset(brokerAddr, mq.getTopic(), mq.getQueueId(), timestamp,
long offset = this.mQClientFactory.getMQClientAPIImpl().searchOffset(brokerAddr, mq.getTopic(), mq.getQueueId(), timestamp,
timeoutMillis);
return correctLogicalQueueOffset(offset, logicalQueueRouteData);
} catch (Exception e) {
throw new MQClientException("Invoke Broker[" + brokerAddr + "] exception", e);
}
......@@ -201,24 +211,50 @@ public class MQAdminImpl {
}
public long maxOffset(MessageQueue mq) throws MQClientException {
String brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
if (null == brokerAddr) {
this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic());
brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
}
return this.maxOffset(mq, true);
}
if (brokerAddr != null) {
try {
return this.mQClientFactory.getMQClientAPIImpl().getMaxOffset(brokerAddr, mq.getTopic(), mq.getQueueId(), timeoutMillis);
} catch (Exception e) {
throw new MQClientException("Invoke Broker[" + brokerAddr + "] exception", e);
public long maxOffset(MessageQueue mq, boolean committed) throws MQClientException {
final MessageQueue origMq = mq;
String topic = mq.getTopic();
LogicalQueueRouteData previousQueueRouteData = null;
for (int i = 0; i < 5; i++) {
LogicalQueueRouteData maxQueueRouteData = this.searchLogicalQueueRouteByOffset(origMq, Long.MAX_VALUE);
if (maxQueueRouteData != null) {
if (previousQueueRouteData != null && Objects.equal(previousQueueRouteData.getMessageQueue(), maxQueueRouteData.getMessageQueue())) {
throw new MQClientException("Topic route info not latest", null);
}
previousQueueRouteData = maxQueueRouteData;
mq = maxQueueRouteData.getMessageQueue();
}
String brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
if (null == brokerAddr) {
this.mQClientFactory.updateTopicRouteInfoFromNameServer(topic);
brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
}
}
throw new MQClientException("The broker[" + mq.getBrokerName() + "] not exist", null);
if (brokerAddr != null) {
try {
long offset = this.mQClientFactory.getMQClientAPIImpl().getMaxOffset(brokerAddr, topic, mq.getQueueId(), committed, maxQueueRouteData != null, timeoutMillis);
return correctLogicalQueueOffset(offset, maxQueueRouteData);
} catch (MQRedirectException e) {
this.mQClientFactory.updateTopicRouteInfoFromNameServer(topic, false, null, Collections.singleton(mq.getQueueId()));
continue;
} catch (Exception e) {
throw new MQClientException("Invoke Broker[" + brokerAddr + "] exception", e);
}
}
throw new MQClientException("The broker[" + mq.getBrokerName() + "] not exist", null);
}
throw new MQClientException("Redirect exceed max times", null);
}
public long minOffset(MessageQueue mq) throws MQClientException {
LogicalQueueRouteData minQueueRouteData = searchLogicalQueueRouteByOffset(mq, 0L);
if (minQueueRouteData != null) {
mq = minQueueRouteData.getMessageQueue();
}
String brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
if (null == brokerAddr) {
this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic());
......@@ -227,7 +263,8 @@ public class MQAdminImpl {
if (brokerAddr != null) {
try {
return this.mQClientFactory.getMQClientAPIImpl().getMinOffset(brokerAddr, mq.getTopic(), mq.getQueueId(), timeoutMillis);
long offset = this.mQClientFactory.getMQClientAPIImpl().getMinOffset(brokerAddr, mq.getTopic(), mq.getQueueId(), timeoutMillis);
return correctLogicalQueueOffset(offset, minQueueRouteData);
} catch (Exception e) {
throw new MQClientException("Invoke Broker[" + brokerAddr + "] exception", e);
}
......@@ -236,7 +273,29 @@ public class MQAdminImpl {
throw new MQClientException("The broker[" + mq.getBrokerName() + "] not exist", null);
}
private List<LogicalQueueRouteData> queryLogicalQueueRouteData(MessageQueue mq) {
if (MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME.equals(mq.getBrokerName())) {
TopicRouteData topicRouteData = this.mQClientFactory.queryTopicRouteData(mq.getTopic());
if (topicRouteData == null) {
this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic());
topicRouteData = this.mQClientFactory.queryTopicRouteData(mq.getTopic());
}
if (topicRouteData != null) {
LogicalQueuesInfo logicalQueuesInfo = topicRouteData.getLogicalQueuesInfo();
if (logicalQueuesInfo != null) {
return logicalQueuesInfo.get(mq.getQueueId());
}
}
}
return null;
}
public long earliestMsgStoreTime(MessageQueue mq) throws MQClientException {
LogicalQueueRouteData minQueueRouteData = searchLogicalQueueRouteByOffset(mq, 0L);
if (minQueueRouteData != null) {
mq = minQueueRouteData.getMessageQueue();
}
String brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
if (null == brokerAddr) {
this.mQClientFactory.updateTopicRouteInfoFromNameServer(mq.getTopic());
......@@ -445,4 +504,71 @@ public class MQAdminImpl {
throw new MQClientException(ResponseCode.TOPIC_NOT_EXIST, "The topic[" + topic + "] not matched route info");
}
private static long correctLogicalQueueOffset(long offset, LogicalQueueRouteData logicalQueueRouteData) {
if (logicalQueueRouteData == null) {
return offset;
}
return logicalQueueRouteData.toLogicalQueueOffset(offset);
}
private LogicalQueueRouteData searchLogicalQueueRouteByTimestamp(MessageQueue mq, long timestamp) {
List<LogicalQueueRouteData> queueRouteDataList = this.queryLogicalQueueRouteData(mq);
if (queueRouteDataList == null) {
return null;
}
LogicalQueueRouteData logicalQueueRouteData = null;
for (LogicalQueueRouteData el : queueRouteDataList) {
if (!el.isReadable()) {
continue;
}
if (logicalQueueRouteData == null && el.getFirstMsgTimeMillis() < 0) {
logicalQueueRouteData = el;
} else if (el.getFirstMsgTimeMillis() >= 0) {
if (el.getFirstMsgTimeMillis() <= timestamp && el.getLastMsgTimeMillis() >= timestamp) {
logicalQueueRouteData = el;
break;
}
}
}
if (logicalQueueRouteData == null) {
logicalQueueRouteData = queueRouteDataList.get(queueRouteDataList.size() - 1);
}
return logicalQueueRouteData;
}
private LogicalQueueRouteData searchLogicalQueueRouteByOffset(MessageQueue mq, long offset) {
List<LogicalQueueRouteData> queueRouteDataList = this.queryLogicalQueueRouteData(mq);
if (queueRouteDataList == null) {
return null;
}
{
List<LogicalQueueRouteData> list = Lists.newArrayListWithCapacity(queueRouteDataList.size());
for (LogicalQueueRouteData queueRouteData : queueRouteDataList) {
if (LogicalQueueRouteData.READABLE_PREDICT.apply(queueRouteData)) {
list.add(queueRouteData);
}
}
queueRouteDataList = list;
}
if (queueRouteDataList.isEmpty()) {
return null;
}
if (offset <= 0) {
// min
return Collections.min(queueRouteDataList);
} else if (offset == Long.MAX_VALUE) {
// max
return Collections.max(queueRouteDataList);
}
Collections.sort(queueRouteDataList);
LogicalQueueRouteData searchKey = new LogicalQueueRouteData();
searchKey.setLogicalQueueDelta(offset);
int idx = Collections.binarySearch(queueRouteDataList, searchKey);
if (idx < 0) {
idx = -idx - 1;
idx -= 1;
}
return queueRouteDataList.get(idx);
}
}
......@@ -147,7 +147,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
private final MessageQueueLock messageQueueLock = new MessageQueueLock();
private final ArrayList<ConsumeMessageHook> consumeMessageHookList = new ArrayList<>();
private final ArrayList<ConsumeMessageHook> consumeMessageHookList = new ArrayList<ConsumeMessageHook>();
// only for test purpose, will be modified by reflection in unit test.
@SuppressWarnings("FieldMayBeFinal") private static boolean doNotUpdateTopicSubscribeInfoWhenSubscriptionChanged = false;
......@@ -898,6 +898,9 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
null
);
this.pullAPIWrapper.processPullResult(mq, pullResult, subscriptionData);
if (pullResult instanceof PullResultWithLogicalQueues) {
pullResult = ((PullResultWithLogicalQueues) pullResult).getOrigPullResultExt();
}
if (!this.consumeMessageHookList.isEmpty()) {
ConsumeMessageContext consumeMessageContext = new ConsumeMessageContext();
consumeMessageContext.setNamespace(defaultLitePullConsumer.getNamespace());
......
......@@ -265,6 +265,9 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
null
);
this.pullAPIWrapper.processPullResult(mq, pullResult, subscriptionData);
if (pullResult instanceof PullResultWithLogicalQueues) {
pullResult = ((PullResultWithLogicalQueues) pullResult).getOrigPullResultExt();
}
//If namespace is not null , reset Topic without namespace.
this.resetTopic(pullResult.getMsgFoundList());
if (!this.consumeMessageHookList.isEmpty()) {
......
......@@ -859,9 +859,11 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultMQPushConsumer.getAllocateMessageQueueStrategy());
this.rebalanceImpl.setmQClientFactory(this.mQClientFactory);
this.pullAPIWrapper = new PullAPIWrapper(
mQClientFactory,
this.defaultMQPushConsumer.getConsumerGroup(), isUnitMode());
if (this.pullAPIWrapper == null) {
this.pullAPIWrapper = new PullAPIWrapper(
mQClientFactory,
this.defaultMQPushConsumer.getConsumerGroup(), isUnitMode());
}
this.pullAPIWrapper.registerFilterMessageHook(filterMessageHookList);
if (this.defaultMQPushConsumer.getOffsetStore() != null) {
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.client.impl.consumer;
import java.util.List;
import org.apache.rocketmq.client.consumer.PullResult;
import org.apache.rocketmq.client.consumer.PullStatus;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
public class PullResultWithLogicalQueues extends PullResultExt {
private PullResultExt origPullResultExt;
private final LogicalQueueRouteData queueRouteData;
public PullResultWithLogicalQueues(PullResult pullResult, LogicalQueueRouteData floorQueueRouteData) {
super(pullResult.getPullStatus(), pullResult.getNextBeginOffset(), pullResult.getMinOffset(), pullResult.getMaxOffset(), pullResult.getMsgFoundList(),
pullResult instanceof PullResultExt ? ((PullResultExt) pullResult).getSuggestWhichBrokerId() : MixAll.MASTER_ID,
pullResult instanceof PullResultExt ? ((PullResultExt) pullResult).getMessageBinary() : null);
if (pullResult instanceof PullResultExt) {
this.origPullResultExt = (PullResultExt) pullResult;
} else {
this.origPullResultExt = new PullResultExt(pullResult.getPullStatus(), pullResult.getNextBeginOffset(), pullResult.getMinOffset(), pullResult.getMaxOffset(), pullResult.getMsgFoundList(), MixAll.MASTER_ID, null);
}
this.queueRouteData = floorQueueRouteData;
}
public PullResult getOrigPullResultExt() {
return origPullResultExt;
}
public LogicalQueueRouteData getQueueRouteData() {
return queueRouteData;
}
public void setOrigPullResultExt(PullResultExt pullResultExt) {
this.origPullResultExt = pullResultExt;
}
@Override public PullStatus getPullStatus() {
return origPullResultExt.getPullStatus();
}
@Override public long getNextBeginOffset() {
return origPullResultExt.getNextBeginOffset();
}
@Override public long getMinOffset() {
return origPullResultExt.getMinOffset();
}
@Override public long getMaxOffset() {
return origPullResultExt.getMaxOffset();
}
@Override public List<MessageExt> getMsgFoundList() {
return origPullResultExt.getMsgFoundList();
}
@Override public void setMsgFoundList(List<MessageExt> msgFoundList) {
origPullResultExt.setMsgFoundList(msgFoundList);
}
@Override public byte[] getMessageBinary() {
return origPullResultExt.getMessageBinary();
}
@Override public void setMessageBinary(byte[] messageBinary) {
origPullResultExt.setMessageBinary(messageBinary);
}
@Override public long getSuggestWhichBrokerId() {
return origPullResultExt.getSuggestWhichBrokerId();
}
@Override public String toString() {
return "PullResultWithLogicalQueues{" +
"origPullResultExt=" + origPullResultExt +
", queueRouteData=" + queueRouteData +
'}';
}
}
......@@ -18,6 +18,7 @@ package org.apache.rocketmq.client.impl.factory;
import java.io.UnsupportedEncodingException;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
......@@ -76,6 +77,8 @@ import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
import org.apache.rocketmq.common.protocol.heartbeat.ProducerData;
import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData;
import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.logging.InternalLogger;
......@@ -174,6 +177,32 @@ public class MQClientInstance {
}
info.setOrderTopic(true);
} else if (route.getOrderTopicConf() == null && route.getLogicalQueuesInfo() != null) {
info.setOrderTopic(false);
List<MessageQueue> messageQueueList = info.getMessageQueueList();
LogicalQueuesInfo logicalQueueInfo = route.getLogicalQueuesInfo();
for (Map.Entry<Integer, List<LogicalQueueRouteData>> entry : logicalQueueInfo.entrySet()) {
boolean someWritable = false;
for (LogicalQueueRouteData logicalQueueRouteData : entry.getValue()) {
if (logicalQueueRouteData.isWritable()) {
someWritable = true;
break;
}
}
if (!someWritable) {
continue;
}
MessageQueue mq = new MessageQueue();
mq.setQueueId(entry.getKey());
mq.setBrokerName(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME);
mq.setTopic(topic);
messageQueueList.add(mq);
}
Collections.sort(messageQueueList, new Comparator<MessageQueue>() {
@Override public int compare(MessageQueue o1, MessageQueue o2) {
return MixAll.compareInteger(o1.getQueueId(), o2.getQueueId());
}
});
} else {
List<QueueData> qds = route.getQueueDatas();
Collections.sort(qds);
......@@ -210,6 +239,27 @@ public class MQClientInstance {
public static Set<MessageQueue> topicRouteData2TopicSubscribeInfo(final String topic, final TopicRouteData route) {
Set<MessageQueue> mqList = new HashSet<MessageQueue>();
if (route.getLogicalQueuesInfo() != null) {
LogicalQueuesInfo logicalQueueInfo = route.getLogicalQueuesInfo();
for (Map.Entry<Integer, List<LogicalQueueRouteData>> entry : logicalQueueInfo.entrySet()) {
boolean someReadable = false;
for (LogicalQueueRouteData logicalQueueRouteData : entry.getValue()) {
if (logicalQueueRouteData.isReadable()) {
someReadable = true;
break;
}
}
if (!someReadable) {
continue;
}
MessageQueue mq = new MessageQueue();
mq.setQueueId(entry.getKey());
mq.setBrokerName(MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME);
mq.setTopic(topic);
mqList.add(mq);
}
return mqList;
}
List<QueueData> qds = route.getQueueDatas();
for (QueueData qd : qds) {
if (PermName.isReadable(qd.getPerm())) {
......@@ -606,6 +656,11 @@ public class MQClientInstance {
public boolean updateTopicRouteInfoFromNameServer(final String topic, boolean isDefault,
DefaultMQProducer defaultMQProducer) {
return this.updateTopicRouteInfoFromNameServer(topic, isDefault, defaultMQProducer, null);
}
public boolean updateTopicRouteInfoFromNameServer(final String topic, boolean isDefault,
DefaultMQProducer defaultMQProducer, Set<Integer> logicalQueueIdsFilter) {
try {
if (this.lockNamesrv.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
try {
......@@ -621,7 +676,7 @@ public class MQClientInstance {
}
}
} else {
topicRouteData = this.mQClientAPIImpl.getTopicRouteInfoFromNameServer(topic, 1000 * 3);
topicRouteData = this.mQClientAPIImpl.getTopicRouteInfoFromNameServer(topic, 1000 * 3, true, logicalQueueIdsFilter);
}
if (topicRouteData != null) {
TopicRouteData old = this.topicRouteTable.get(topic);
......@@ -633,7 +688,26 @@ public class MQClientInstance {
}
if (changed) {
TopicRouteData cloneTopicRouteData = topicRouteData.cloneTopicRouteData();
TopicRouteData cloneTopicRouteData = new TopicRouteData(topicRouteData);
if (logicalQueueIdsFilter != null && cloneTopicRouteData.getLogicalQueuesInfo() != null) {
TopicRouteData curTopicRouteData = this.topicRouteTable.get(topic);
if (curTopicRouteData != null) {
LogicalQueuesInfo curLogicalQueuesInfo = curTopicRouteData.getLogicalQueuesInfo();
if (curLogicalQueuesInfo != null) {
LogicalQueuesInfo cloneLogicalQueuesInfo = cloneTopicRouteData.getLogicalQueuesInfo();
curLogicalQueuesInfo.readLock().lock();
try {
for (Entry<Integer, List<LogicalQueueRouteData>> entry : curLogicalQueuesInfo.entrySet()) {
if (!cloneLogicalQueuesInfo.containsKey(entry.getKey())) {
cloneLogicalQueuesInfo.put(entry.getKey(), entry.getValue());
}
}
} finally {
curLogicalQueuesInfo.readLock().unlock();
}
}
}
}
for (BrokerData bd : topicRouteData.getBrokerDatas()) {
this.brokerAddrTable.put(bd.getBrokerName(), bd.getBrokerAddrs());
......@@ -791,8 +865,15 @@ public class MQClientInstance {
private boolean topicRouteDataIsChange(TopicRouteData olddata, TopicRouteData nowdata) {
if (olddata == null || nowdata == null)
return true;
TopicRouteData old = olddata.cloneTopicRouteData();
TopicRouteData now = nowdata.cloneTopicRouteData();
LogicalQueuesInfo oldLogicalQueuesInfo = olddata.getLogicalQueuesInfo();
LogicalQueuesInfo newLogicalQueuesInfo = nowdata.getLogicalQueuesInfo();
if (oldLogicalQueuesInfo != null && newLogicalQueuesInfo != null) {
return oldLogicalQueuesInfo.keySet().equals(newLogicalQueuesInfo.keySet());
} else if (oldLogicalQueuesInfo != null || newLogicalQueuesInfo != null) {
return true;
}
TopicRouteData old = new TopicRouteData(olddata);
TopicRouteData now = new TopicRouteData(nowdata);
Collections.sort(old.getQueueDatas());
Collections.sort(old.getBrokerDatas());
Collections.sort(now.getQueueDatas());
......@@ -814,6 +895,10 @@ public class MQClientInstance {
}
}
if (result) {
return true;
}
{
Iterator<Entry<String, MQConsumerInner>> it = this.consumerTable.entrySet().iterator();
while (it.hasNext() && !result) {
......@@ -1280,4 +1365,13 @@ public class MQClientInstance {
public ClientConfig getClientConfig() {
return clientConfig;
}
public TopicRouteData queryTopicRouteData(String topic) {
TopicRouteData data = this.getAnExistTopicRouteData(topic);
if (data == null) {
this.updateTopicRouteInfoFromNameServer(topic);
data = this.getAnExistTopicRouteData(topic);
}
return data;
}
}
......@@ -16,12 +16,16 @@
*/
package org.apache.rocketmq.client.impl.producer;
import com.alibaba.fastjson.JSON;
import com.google.common.base.Objects;
import java.io.IOException;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Random;
import java.util.Set;
import java.util.Timer;
......@@ -41,6 +45,7 @@ import org.apache.rocketmq.client.Validators;
import org.apache.rocketmq.client.common.ClientErrorCode;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.exception.MQRedirectException;
import org.apache.rocketmq.client.exception.RequestTimeoutException;
import org.apache.rocketmq.client.hook.CheckForbiddenContext;
import org.apache.rocketmq.client.hook.CheckForbiddenHook;
......@@ -62,6 +67,7 @@ import org.apache.rocketmq.client.producer.RequestFutureTable;
import org.apache.rocketmq.client.producer.RequestResponseFuture;
import org.apache.rocketmq.client.producer.SendCallback;
import org.apache.rocketmq.client.producer.SendResult;
import org.apache.rocketmq.client.producer.SendResultForLogicalQueue;
import org.apache.rocketmq.client.producer.SendStatus;
import org.apache.rocketmq.client.producer.TransactionCheckListener;
import org.apache.rocketmq.client.producer.TransactionListener;
......@@ -86,6 +92,9 @@ import org.apache.rocketmq.common.protocol.ResponseCode;
import org.apache.rocketmq.common.protocol.header.CheckTransactionStateRequestHeader;
import org.apache.rocketmq.common.protocol.header.EndTransactionRequestHeader;
import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.common.sysflag.MessageSysFlag;
import org.apache.rocketmq.common.utils.CorrelationIdUtil;
import org.apache.rocketmq.logging.InternalLogger;
......@@ -96,6 +105,8 @@ import org.apache.rocketmq.remoting.exception.RemotingException;
import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
;
public class DefaultMQProducerImpl implements MQProducerInner {
private final InternalLogger log = ClientLogger.getLog();
private final Random random = new Random();
......@@ -502,7 +513,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
*
* @param msg
* @param sendCallback
* @param timeout the <code>sendCallback</code> will be invoked at most time
* @param timeout the <code>sendCallback</code> will be invoked at most time
* @throws RejectedExecutionException
*/
@Deprecated
......@@ -718,6 +729,38 @@ public class DefaultMQProducerImpl implements MQProducerInner {
final SendCallback sendCallback,
final TopicPublishInfo topicPublishInfo,
final long timeout) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
if (MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME.equals(mq.getBrokerName())) {
LogicalQueueSendContext logicalQueueContext = new LogicalQueueSendContext(msg, mq, communicationMode, sendCallback, topicPublishInfo, timeout);
while (true) {
try {
SendResult sendResult = this.sendKernelImplWithoutRetry(msg,
logicalQueueContext.getModifiedMessageQueue(),
communicationMode,
logicalQueueContext.wrapSendCallback(),
topicPublishInfo,
timeout);
return logicalQueueContext.wrapSendResult(sendResult);
} catch (MQRedirectException e) {
if (!logicalQueueContext.shouldRetry(e)) {
throw new MQBrokerException(ResponseCode.SYSTEM_ERROR, "redirect");
}
} catch (RemotingException e) {
if (!logicalQueueContext.shouldRetry(e)) {
throw e;
}
}
}
} else {
return sendKernelImplWithoutRetry(msg, mq, communicationMode, sendCallback, topicPublishInfo, timeout);
}
}
private SendResult sendKernelImplWithoutRetry(final Message msg,
final MessageQueue mq,
final CommunicationMode communicationMode,
SendCallback sendCallback,
final TopicPublishInfo topicPublishInfo,
final long timeout) throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
long beginStartTime = System.currentTimeMillis();
String brokerAddr = this.mQClientFactory.findBrokerAddressInPublish(mq.getBrokerName());
if (null == brokerAddr) {
......@@ -754,6 +797,10 @@ public class DefaultMQProducerImpl implements MQProducerInner {
sysFlag |= MessageSysFlag.TRANSACTION_PREPARED_TYPE;
}
if (!CommunicationMode.ONEWAY.equals(communicationMode)) {
sysFlag |= MessageSysFlag.LOGICAL_QUEUE_FLAG;
}
if (hasCheckForbiddenHook()) {
CheckForbiddenContext checkForbiddenContext = new CheckForbiddenContext();
checkForbiddenContext.setNameSrvAddr(this.defaultMQProducer.getNamesrvAddr());
......@@ -1006,6 +1053,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
executeEndTransactionHook(context);
}
}
/**
* DEFAULT ONEWAY -------------------------------------------------------
*/
......@@ -1058,7 +1106,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
* @param msg
* @param mq
* @param sendCallback
* @param timeout the <code>sendCallback</code> will be invoked at most time
* @param timeout the <code>sendCallback</code> will be invoked at most time
* @throws MQClientException
* @throws RemotingException
* @throws InterruptedException
......@@ -1188,7 +1236,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
* @param selector
* @param arg
* @param sendCallback
* @param timeout the <code>sendCallback</code> will be invoked at most time
* @param timeout the <code>sendCallback</code> will be invoked at most time
* @throws MQClientException
* @throws RemotingException
* @throws InterruptedException
......@@ -1528,7 +1576,8 @@ public class DefaultMQProducerImpl implements MQProducerInner {
}
}
private Message waitResponse(Message msg, long timeout, RequestResponseFuture requestResponseFuture, long cost) throws InterruptedException, RequestTimeoutException, MQClientException {
private Message waitResponse(Message msg, long timeout, RequestResponseFuture requestResponseFuture,
long cost) throws InterruptedException, RequestTimeoutException, MQClientException {
Message responseMessage = requestResponseFuture.waitResponseMessage(timeout - cost);
if (responseMessage == null) {
if (requestResponseFuture.isSendRequestOk()) {
......@@ -1644,4 +1693,178 @@ public class DefaultMQProducerImpl implements MQProducerInner {
public DefaultMQProducer getDefaultMQProducer() {
return defaultMQProducer;
}
private class LogicalQueueSendContext implements SendCallback {
private final Message msg;
private final MessageQueue mq;
private final CommunicationMode communicationMode;
private final SendCallback sendCallback;
private final TopicPublishInfo topicPublishInfo;
private final long timeout;
private volatile LogicalQueuesInfo logicalQueuesInfo;
private volatile LogicalQueueRouteData writableQueueRouteData;
private final AtomicInteger retry = new AtomicInteger();
public LogicalQueueSendContext(Message msg, MessageQueue mq,
CommunicationMode communicationMode, SendCallback sendCallback,
TopicPublishInfo topicPublishInfo, long timeout) {
this.msg = msg;
this.mq = mq;
this.communicationMode = communicationMode;
this.sendCallback = sendCallback;
this.topicPublishInfo = topicPublishInfo;
this.timeout = timeout;
if (topicPublishInfo == null) {
topicPublishInfo = DefaultMQProducerImpl.this.tryToFindTopicPublishInfo(mq.getTopic());
}
if (topicPublishInfo != null) {
this.logicalQueuesInfo = topicPublishInfo.getTopicRouteData().getLogicalQueuesInfo();
} else {
this.logicalQueuesInfo = null;
}
}
private boolean notUsingLogicalQueue() {
return !Objects.equal(mq.getBrokerName(), MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME) || this.logicalQueuesInfo == null;
}
public MessageQueue getModifiedMessageQueue() throws MQClientException {
if (this.notUsingLogicalQueue()) {
return this.mq;
}
this.writableQueueRouteData = getWritableQueueRouteData();
MessageQueue mq = new MessageQueue(this.mq);
mq.setBrokerName(writableQueueRouteData.getBrokerName());
mq.setQueueId(writableQueueRouteData.getQueueId());
return mq;
}
private LogicalQueueRouteData getWritableQueueRouteData() throws MQClientException {
this.logicalQueuesInfo.readLock().lock();
try {
List<LogicalQueueRouteData> queueRouteDataList = logicalQueuesInfo.get(mq.getQueueId());
if (queueRouteDataList == null || queueRouteDataList.size() == 0) {
throw new MQClientException(String.format(Locale.ENGLISH, "send to a logical queue %d but no queue route data found", mq.getQueueId()), null);
}
// usually writable queue is placed in the last position, or second last when queue migrating
for (int i = queueRouteDataList.size() - 1; i >= 0; i--) {
LogicalQueueRouteData queueRouteData = queueRouteDataList.get(i);
if (queueRouteData.isWritable()) {
return queueRouteData;
}
}
throw new MQClientException(String.format(Locale.ENGLISH, "send to a logical queue %d but no writable queue route data found", mq.getQueueId()), null);
} finally {
this.logicalQueuesInfo.readLock().unlock();
}
}
@Override public void onSuccess(SendResult sendResult) {
this.sendCallback.onSuccess(this.wrapSendResult(sendResult));
}
@Override public void onException(Throwable t) {
if (this.shouldRetry(t)) {
try {
DefaultMQProducerImpl.this.sendKernelImplWithoutRetry(msg, this.getModifiedMessageQueue(), communicationMode, this, topicPublishInfo, timeout);
return;
} catch (Exception e) {
t = e;
}
}
if (t instanceof MQRedirectException) {
t = new MQBrokerException(ResponseCode.SYSTEM_ERROR, "redirect");
}
this.sendCallback.onException(t);
}
private void handleRedirectException(MQRedirectException re) {
byte[] responseBody = re.getBody();
log.info("LogicalQueueContext.processResponseBody got redirect {}: {}", this.writableQueueRouteData, responseBody != null ? new String(responseBody, MessageDecoder.CHARSET_UTF8) : null);
try {
List<LogicalQueueRouteData> newQueueRouteDataList = JSON.parseObject(responseBody, MixAll.TYPE_LIST_LOGICAL_QUEUE_ROUTE_DATA);
this.logicalQueuesInfo.updateLogicalQueueRouteDataList(this.mq.getQueueId(), newQueueRouteDataList);
} catch (Exception e) {
log.warn("LogicalQueueContext.processResponseBody {} update exception, fallback to updateTopicRouteInfoFromNameServer", this.writableQueueRouteData, e);
DefaultMQProducerImpl.this.mQClientFactory.updateTopicRouteInfoFromNameServer(this.mq.getTopic(), false, null, Collections.singleton(mq.getQueueId()));
TopicRouteData topicRouteData = DefaultMQProducerImpl.this.mQClientFactory.getAnExistTopicRouteData(mq.getTopic());
if (topicRouteData != null) {
this.logicalQueuesInfo = topicRouteData.getLogicalQueuesInfo();
} else {
this.logicalQueuesInfo = null;
}
}
}
public SendCallback wrapSendCallback() {
if (this.notUsingLogicalQueue()) {
return this.sendCallback;
}
if (!CommunicationMode.ASYNC.equals(this.communicationMode)) {
return this.sendCallback;
}
return this;
}
public boolean shouldRetry(Throwable t) {
this.incrRetry();
if (this.exceedMaxRetry()) {
log.warn("retry {} too many times: {}", this.retry.get(), this.writableQueueRouteData);
return false;
}
if (!this.writableQueueRouteData.isWritable()) {
log.warn("no writable queue: {}", this.writableQueueRouteData);
return false;
}
if (t instanceof MQRedirectException) {
this.handleRedirectException((MQRedirectException) t);
return true;
}
return !(t instanceof RemotingException) || this.handleRemotingException((RemotingException) t);
}
public boolean exceedMaxRetry() {
return this.retry.get() >= 3;
}
public void incrRetry() {
this.retry.incrementAndGet();
}
public SendResult wrapSendResult(SendResult sendResult) {
if (sendResult == null) {
return null;
}
SendResultForLogicalQueue newSendResult = new SendResultForLogicalQueue(sendResult, this.writableQueueRouteData.getLogicalQueueIndex());
long queueOffset = newSendResult.getQueueOffset();
if (queueOffset >= 0) {
newSendResult.setQueueOffset(LogicalQueueSendContext.this.writableQueueRouteData.toLogicalQueueOffset(queueOffset));
}
return newSendResult;
}
public boolean handleRemotingException(RemotingException e) {
if (e instanceof RemotingTooMuchRequestException) {
return false;
}
DefaultMQProducerImpl.this.mQClientFactory.updateTopicRouteInfoFromNameServer(this.mq.getTopic(), false, null, Collections.singleton(mq.getQueueId()));
this.logicalQueuesInfo = DefaultMQProducerImpl.this.getTopicPublishInfoTable().get(mq.getTopic()).getTopicRouteData().getLogicalQueuesInfo();
LogicalQueueRouteData writableQueueRouteData;
try {
writableQueueRouteData = this.getWritableQueueRouteData();
} catch (MQClientException ce) {
log.warn("getWritableQueueRouteData exception: {}", this.logicalQueuesInfo.get(mq.getQueueId()), ce);
return false;
}
if (Objects.equal(this.writableQueueRouteData.getMessageQueue(), writableQueueRouteData.getMessageQueue()) && writableQueueRouteData.isWritable()) {
// still same MessageQueue and still writable, no need to retry
return false;
}
return true;
}
}
}
......@@ -19,8 +19,8 @@ package org.apache.rocketmq.client.latency;
import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.logging.InternalLogger;
public class MQFaultStrategy {
private final static InternalLogger log = ClientLogger.getLog();
......
......@@ -28,6 +28,7 @@ public class SendResult {
private String offsetMsgId;
private String regionId;
private boolean traceOn = true;
private byte[] rawRespBody;
public SendResult() {
}
......@@ -130,4 +131,12 @@ public class SendResult {
return "SendResult [sendStatus=" + sendStatus + ", msgId=" + msgId + ", offsetMsgId=" + offsetMsgId + ", messageQueue=" + messageQueue
+ ", queueOffset=" + queueOffset + "]";
}
public void setRawRespBody(byte[] body) {
this.rawRespBody = body;
}
public byte[] getRawRespBody() {
return rawRespBody;
}
}
/*
* 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;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.MessageQueue;
public class SendResultForLogicalQueue extends SendResult {
private final String origBrokerName;
private final int origQueueId;
public SendResultForLogicalQueue(SendResult sendResult, int logicalQueueIdx) {
super(sendResult.getSendStatus(), sendResult.getMsgId(), sendResult.getOffsetMsgId(), new MessageQueue(sendResult.getMessageQueue().getTopic(), MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, logicalQueueIdx), sendResult.getQueueOffset());
this.origBrokerName = sendResult.getMessageQueue().getBrokerName();
this.origQueueId = sendResult.getMessageQueue().getQueueId();
}
public String getOrigBrokerName() {
return origBrokerName;
}
public int getOrigQueueId() {
return origQueueId;
}
@Override public String toString() {
return "SendResultForLogicalQueue{" +
"origBrokerName='" + origBrokerName + '\'' +
", origQueueId=" + origQueueId +
"} " + super.toString();
}
}
......@@ -51,7 +51,7 @@ public class ConsumeMessageOpenTracingHookImpl implements ConsumeMessageHook {
if (context == null || context.getMsgList() == null || context.getMsgList().isEmpty()) {
return;
}
List<Span> spanList = new ArrayList<>();
List<Span> spanList = new ArrayList<Span>();
for (MessageExt msg : context.getMsgList()) {
if (msg == null) {
continue;
......
......@@ -95,7 +95,9 @@ public class DefaultLitePullConsumerTest {
@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());
for (MQClientInstance instance : factoryTable.values()) {
instance.shutdown();
}
factoryTable.clear();
Field field = MQClientInstance.class.getDeclaredField("rebalanceService");
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.client.consumer;
import com.alibaba.fastjson.JSON;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.SettableFuture;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQRedirectException;
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.PullAPIWrapper;
import org.apache.rocketmq.client.impl.consumer.PullResultExt;
import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader;
import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo;
import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.assertj.core.util.Lists;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentMatchers;
import org.mockito.Mock;
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.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.eq;
import static org.mockito.ArgumentMatchers.isNull;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public class DefaultMQPullConsumerLogicalQueueTest {
private MQClientInstance mQClientFactory;
@Mock
private MQClientAPIImpl mQClientAPIImpl;
private DefaultMQPullConsumer pullConsumer;
private String topic;
private static final String cluster = "DefaultCluster";
private static final String broker1Name = "BrokerA";
private static final String broker1Addr = "127.0.0.2:10911";
private static final String broker2Name = "BrokerB";
private static final String broker2Addr = "127.0.0.3:10911";
@Before
public void init() throws Exception {
topic = "FooBar" + System.nanoTime();
mQClientFactory = spy(MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()));
FieldUtils.writeField(mQClientFactory, "mQClientAPIImpl", mQClientAPIImpl, true);
pullConsumer = new DefaultMQPullConsumer("FooBarGroup" + System.nanoTime());
pullConsumer.setNamesrvAddr("127.0.0.1:9876");
pullConsumer.start();
PullAPIWrapper pullAPIWrapper = pullConsumer.getDefaultMQPullConsumerImpl().getPullAPIWrapper();
FieldUtils.writeDeclaredField(pullAPIWrapper, "mQClientFactory", mQClientFactory, true);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRouteData());
doReturn(new FindBrokerResult(broker1Addr, false)).when(mQClientFactory).findBrokerAddressInSubscribe(eq(broker1Name), anyLong(), anyBoolean());
doReturn(new FindBrokerResult(broker2Addr, false)).when(mQClientFactory).findBrokerAddressInSubscribe(eq(broker2Name), anyLong(), anyBoolean());
}
@After
public void terminate() {
pullConsumer.shutdown();
}
@Test
public void testStart_OffsetShouldNotNUllAfterStart() {
Assert.assertNotNull(pullConsumer.getOffsetStore());
}
@Test
public void testPullMessage_Success() throws Exception {
doAnswer(new Answer<PullResultExt>() {
@Override public PullResultExt answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
return DefaultMQPullConsumerLogicalQueueTest.this.createPullResult(requestHeader, PullStatus.FOUND, Collections.singletonList(new MessageExt()));
}
}).when(mQClientAPIImpl).pullMessage(eq(broker1Addr), any(PullMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC), (PullCallback) isNull());
MessageQueue messageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, 0);
PullResult pullResult = pullConsumer.pull(messageQueue, "*", 1024, 3);
assertThat(pullResult).isNotNull();
assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND);
assertThat(pullResult.getNextBeginOffset()).isEqualTo(1024 + 1);
assertThat(pullResult.getMinOffset()).isEqualTo(123);
assertThat(pullResult.getMaxOffset()).isEqualTo(2048);
assertThat(pullResult.getMsgFoundList()).isEqualTo(Collections.emptyList());
}
@Test
public void testPullMessage_NotFound() throws Exception {
doAnswer(new Answer<PullResult>() {
@Override public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
return DefaultMQPullConsumerLogicalQueueTest.this.createPullResult(requestHeader, PullStatus.NO_NEW_MSG, new ArrayList<MessageExt>());
}
}).when(mQClientAPIImpl).pullMessage(eq(broker1Addr), any(PullMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC), (PullCallback) isNull());
MessageQueue messageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, 0);
PullResult pullResult = pullConsumer.pull(messageQueue, "*", 1024, 3);
assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.NO_NEW_MSG);
}
@Test
public void testPullMessageAsync_Success() throws Exception {
doAnswer(new Answer<PullResult>() {
@Override public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
PullResult pullResult = DefaultMQPullConsumerLogicalQueueTest.this.createPullResult(requestHeader, PullStatus.FOUND, Collections.singletonList(new MessageExt()));
PullCallback pullCallback = mock.getArgument(4);
pullCallback.onSuccess(pullResult);
return null;
}
}).when(mQClientAPIImpl).pullMessage(eq(broker1Addr), any(PullMessageRequestHeader.class), anyLong(), eq(CommunicationMode.ASYNC), any(PullCallback.class));
final SettableFuture<PullResult> future = SettableFuture.create();
MessageQueue messageQueue = new MessageQueue(topic, broker1Name, 0);
pullConsumer.pull(messageQueue, "*", 1024, 3, new PullCallback() {
@Override
public void onSuccess(PullResult pullResult) {
future.set(pullResult);
}
@Override
public void onException(Throwable e) {
future.setException(e);
}
});
PullResult pullResult = future.get(3, TimeUnit.SECONDS);
assertThat(pullResult).isNotNull();
assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND);
assertThat(pullResult.getNextBeginOffset()).isEqualTo(1024 + 1);
assertThat(pullResult.getMinOffset()).isEqualTo(123);
assertThat(pullResult.getMaxOffset()).isEqualTo(2048);
assertThat(pullResult.getMsgFoundList()).isEqualTo(Collections.emptyList());
}
@Test
public void testPullMessageSync_Redirect() throws Exception {
doAnswer(new Answer<PullResult>() {
@Override public PullResult answer(InvocationOnMock mock) throws Throwable {
throw new MQRedirectException(JSON.toJSONBytes(ImmutableList.of(
new LogicalQueueRouteData(0, 0, new MessageQueue(topic, broker1Name, 0), MessageQueueRouteState.Expired, 0, 0, 0, 0, broker1Addr),
new LogicalQueueRouteData(0, 10, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr)
)));
}
}).when(mQClientAPIImpl).pullMessage(eq(broker1Addr), any(PullMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC), (PullCallback) isNull());
doAnswer(new Answer<PullResult>() {
@Override public PullResult answer(InvocationOnMock mock) throws Throwable {
PullMessageRequestHeader requestHeader = mock.getArgument(1);
return DefaultMQPullConsumerLogicalQueueTest.this.createPullResult(requestHeader, PullStatus.FOUND, Collections.singletonList(new MessageExt()));
}
}).when(mQClientAPIImpl).pullMessage(eq(broker2Addr), any(PullMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC), (PullCallback) isNull());
MessageQueue messageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, 0);
PullResult pullResult = pullConsumer.pull(messageQueue, "*", 1024, 3);
assertThat(pullResult).isNotNull();
assertThat(pullResult.getPullStatus()).isEqualTo(PullStatus.FOUND);
assertThat(pullResult.getNextBeginOffset()).isEqualTo(1024 + 1);
assertThat(pullResult.getMinOffset()).isEqualTo(123 + 10);
assertThat(pullResult.getMaxOffset()).isEqualTo(2048 + 10);
assertThat(pullResult.getMsgFoundList()).isEqualTo(Collections.emptyList());
}
private TopicRouteData createTopicRouteData() {
TopicRouteData topicRouteData = new TopicRouteData();
topicRouteData.setFilterServerTable(new HashMap<String, List<String>>());
topicRouteData.setBrokerDatas(ImmutableList.of(
new BrokerData(cluster, broker1Name, new HashMap<Long, String>(Collections.singletonMap(MixAll.MASTER_ID, broker1Addr))),
new BrokerData(cluster, broker2Name, new HashMap<Long, String>(Collections.singletonMap(MixAll.MASTER_ID, broker2Addr)))
));
List<QueueData> queueDataList = new ArrayList<QueueData>();
QueueData queueData;
queueData = new QueueData();
queueData.setBrokerName(broker1Name);
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
queueData = new QueueData();
queueData.setBrokerName(broker2Name);
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
LogicalQueuesInfo info = new LogicalQueuesInfo();
info.put(0, Lists.newArrayList(new LogicalQueueRouteData(0, 0, new MessageQueue(topic, broker1Name, 0), MessageQueueRouteState.Normal, 0, 0, 0, 0, broker1Addr)));
topicRouteData.setLogicalQueuesInfo(info);
return topicRouteData;
}
private PullResultExt createPullResult(PullMessageRequestHeader requestHeader, PullStatus pullStatus,
List<MessageExt> messageExtList) throws Exception {
return new PullResultExt(pullStatus, requestHeader.getQueueOffset() + messageExtList.size(), 123, 2048, messageExtList, 0, new byte[] {});
}
}
\ No newline at end of file
......@@ -100,7 +100,9 @@ public class DefaultMQPushConsumerTest {
@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());
for (MQClientInstance instance : factoryTable.values()) {
instance.shutdown();
}
factoryTable.clear();
consumerGroup = "FooBarGroup" + System.currentTimeMillis();
......@@ -121,12 +123,15 @@ public class DefaultMQPushConsumerTest {
// suppress updateTopicRouteInfoFromNameServer
pushConsumer.changeInstanceNameToPID();
mQClientFactory = spy(MQClientManager.getInstance().getOrCreateMQClientInstance(pushConsumer, (RPCHook) FieldUtils.readDeclaredField(pushConsumerImpl, "rpcHook", true)));
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(null).when(mQClientFactory).queryAssignment(anyString(), anyString(), anyString(), any(MessageModel.class), anyInt());
doReturn(new FindBrokerResult("127.0.0.1:10911", false)).when(mQClientFactory).findBrokerAddressInSubscribe(anyString(), anyLong(), anyBoolean());
rebalanceImpl = spy(pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl());
rebalanceImpl = spy(pushConsumerImpl.getRebalanceImpl());
doReturn(123L).when(rebalanceImpl).computePullFromWhereWithException(any(MessageQueue.class));
Field field = DefaultMQPushConsumerImpl.class.getDeclaredField("rebalanceImpl");
field.setAccessible(true);
......@@ -136,25 +141,16 @@ public class DefaultMQPushConsumerTest {
field.setAccessible(true);
field.set(null, true);
pushConsumer.subscribe(topic, "*");
pushConsumer.start();
field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory");
field.setAccessible(true);
field.set(pushConsumerImpl, mQClientFactory);
Set<MessageQueue> messageQueueSet = new HashSet<MessageQueue>();
messageQueueSet.add(createPullRequest().getMessageQueue());
pushConsumerImpl.updateTopicSubscribeInfo(topic, messageQueueSet);
field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl");
field.setAccessible(true);
field.set(mQClientFactory, mQClientAPIImpl);
pushConsumerImpl.setmQClientFactory(mQClientFactory);
pullAPIWrapper = spy(new PullAPIWrapper(mQClientFactory, consumerGroup, false));
field = DefaultMQPushConsumerImpl.class.getDeclaredField("pullAPIWrapper");
field.setAccessible(true);
field.set(pushConsumerImpl, pullAPIWrapper);
FieldUtils.writeDeclaredField(pushConsumerImpl, "pullAPIWrapper", pullAPIWrapper, true);
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<PullResult>() {
@Override
......@@ -175,10 +171,10 @@ public class DefaultMQPushConsumerTest {
}
});
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());
pushConsumer.getDefaultMQPushConsumerImpl().updateTopicSubscribeInfo(topic, messageQueueSet);
pushConsumer.subscribe(topic, "*");
pushConsumer.start();
mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl);
}
@After
......@@ -194,7 +190,7 @@ public class DefaultMQPushConsumerTest {
@Test
public void testPullMessage_Success() throws InterruptedException, RemotingException, MQBrokerException {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<MessageExt>();
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
......@@ -217,7 +213,7 @@ public class DefaultMQPushConsumerTest {
@Test(timeout = 20000)
public void testPullMessage_SuccessWithOrderlyService() throws Exception {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<MessageExt>();
MessageListenerOrderly listenerOrderly = new MessageListenerOrderly() {
@Override
......@@ -355,11 +351,14 @@ public class DefaultMQPushConsumerTest {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final MessageExt[] messageExts = new MessageExt[1];
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(
new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(),
(msgs, context) -> {
messageExts[0] = msgs.get(0);
return null;
}));
new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(),
new MessageListenerConcurrently() {
@Override public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
ConsumeConcurrentlyContext context) {
messageExts[0] = msgs.get(0);
return null;
}
}));
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeOrderly(true);
PullMessageService pullMessageService = mQClientFactory.getPullMessageService();
......
......@@ -149,7 +149,7 @@ public class ConsumeMessageConcurrentlyServiceTest {
@Test
public void testPullMessage_ConsumeSuccess() throws InterruptedException, RemotingException, MQBrokerException, NoSuchFieldException,Exception {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<MessageExt>();
ConsumeMessageConcurrentlyService normalServie = new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
......
/*
* 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;
import com.alibaba.fastjson.JSON;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.SettableFuture;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang3.reflect.FieldUtils;
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.exception.MQRedirectException;
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.common.MixAll;
import org.apache.rocketmq.common.message.Message;
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.LogicalQueueRouteData;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo;
import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState;
import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.remoting.exception.RemotingConnectException;
import org.assertj.core.api.ThrowableAssert;
import org.assertj.core.util.Lists;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentMatchers;
import org.mockito.Mock;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.junit.MockitoJUnitRunner;
import org.mockito.stubbing.Answer;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.isNull;
import static org.mockito.ArgumentMatchers.nullable;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public class DefaultMQProducerLogicalQueueTest {
private MQClientInstance mQClientFactory;
@Mock
private MQClientAPIImpl mQClientAPIImpl;
private DefaultMQProducer producer;
private Message message;
private String topic;
private MessageQueue messageQueue;
private static final String cluster = "DefaultCluster";
private static final String broker1Name = "broker1";
private static final String broker2Name = "broker2";
private static final String broker1Addr = "127.0.0.2:10911";
private static final String broker2Addr = "127.0.0.3:10911";
@Before
public void init() throws Exception {
topic = "Foobar" + System.nanoTime();
messageQueue = new MessageQueue(topic, MixAll.LOGICAL_QUEUE_MOCK_BROKER_NAME, 0);
ConcurrentMap<String, MQClientInstance> factoryTable = (ConcurrentMap<String/* clientId */, MQClientInstance>) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true);
for (MQClientInstance instance : factoryTable.values()) {
instance.shutdown();
}
factoryTable.clear();
mQClientFactory = spy(MQClientManager.getInstance().getOrCreateMQClientInstance(new ClientConfig()));
factoryTable.put(new ClientConfig().buildMQClientId(), mQClientFactory);
String producerGroupTemp = "FooBar_PID" + System.nanoTime();
producer = new DefaultMQProducer(producerGroupTemp);
producer.setNamesrvAddr("127.0.0.1:9876");
producer.setCompressMsgBodyOverHowmuch(Integer.MAX_VALUE);
message = new Message(topic, new byte[] {'a'});
mQClientFactory.registerProducer(producerGroupTemp, producer.getDefaultMQProducerImpl());
producer.start();
FieldUtils.writeDeclaredField(producer.getDefaultMQProducerImpl(), "mQClientFactory", mQClientFactory, true);
FieldUtils.writeField(mQClientFactory, "mQClientAPIImpl", mQClientAPIImpl, true);
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),
(SendCallback) isNull(), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class)))
.thenReturn(createSendResult(SendStatus.SEND_OK));
when(mQClientAPIImpl.sendMessage(anyString(), anyString(), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), any(CommunicationMode.class),
any(SendCallback.class), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class)))
.thenAnswer(new Answer<SendResult>() {
@Override public SendResult answer(InvocationOnMock invocation) throws Throwable {
SendCallback sendCallback = invocation.getArgument(6);
sendCallback.onSuccess(DefaultMQProducerLogicalQueueTest.this.createSendResult(SendStatus.SEND_OK));
return null;
}
});
}
@After
public void terminate() {
producer.shutdown();
}
@Test
public void testSendMessageSync_Success() throws Exception {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
SendResult sendResult = producer.send(message, messageQueue);
assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
assertThat(sendResult.getOffsetMsgId()).isEqualTo("123");
assertThat(sendResult.getQueueOffset()).isEqualTo(456L);
}
@Test
public void testSendMessageSync_Redirect() throws Exception {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.sendMessage(eq(broker1Addr), eq(broker1Name), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC),
(SendCallback) isNull(), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class)))
.thenThrow(new MQRedirectException(null));
assertThatThrownBy(new ThrowableAssert.ThrowingCallable() {
@Override public void call() throws Throwable {
producer.send(message, messageQueue);
}
}).isInstanceOf(MQBrokerException.class).hasMessageContaining("redirect");
when(mQClientAPIImpl.sendMessage(eq(broker1Addr), eq(broker1Name), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC),
(SendCallback) isNull(), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class)))
.thenThrow(new MQRedirectException(JSON.toJSONBytes(ImmutableList.of(
new LogicalQueueRouteData(0, 0, new MessageQueue(topic, broker1Name, 0), MessageQueueRouteState.Expired, 0, 0, 0, 0, broker1Addr),
new LogicalQueueRouteData(0, 10, new MessageQueue(topic, broker2Name, 0), MessageQueueRouteState.Normal, 0, -1, -1, -1, broker2Addr)))));
when(mQClientAPIImpl.sendMessage(eq(broker2Addr), eq(broker2Name), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC),
(SendCallback) isNull(), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class)))
.thenReturn(createSendResult(SendStatus.SEND_OK));
SendResult sendResult = producer.send(message, messageQueue);
assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
assertThat(sendResult.getOffsetMsgId()).isEqualTo("123");
assertThat(sendResult.getQueueOffset()).isEqualTo(466L);
}
@Test
public void testSendMessageSync_RemotingException() throws Exception {
TopicRouteData topicRouteData = createTopicRoute();
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(topicRouteData);
when(mQClientAPIImpl.sendMessage(eq(broker1Addr), eq(broker1Name), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC),
(SendCallback) isNull(), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class)))
.thenThrow(new RemotingConnectException(broker1Addr));
SendResult returnSendResult = createSendResult(SendStatus.SEND_OK);
when(mQClientAPIImpl.sendMessage(eq(broker2Addr), eq(broker2Name), any(Message.class), any(SendMessageRequestHeader.class), anyLong(), eq(CommunicationMode.SYNC),
(SendCallback) isNull(), nullable(TopicPublishInfo.class), nullable(MQClientInstance.class), anyInt(), nullable(SendMessageContext.class), any(DefaultMQProducerImpl.class)))
.thenReturn(returnSendResult);
assertThatThrownBy(new ThrowableAssert.ThrowingCallable() {
@Override public void call() throws Throwable {
producer.send(message, messageQueue);
}
}).isInstanceOf(RemotingConnectException.class).hasMessageContaining(broker1Addr);
topicRouteData.getLogicalQueuesInfo().get(0).add(new LogicalQueueRouteData(0, -1, new MessageQueue(topic, broker2Name, 1), MessageQueueRouteState.WriteOnly, 0, -1, -1, -1, broker2Addr));
SendResult sendResult = producer.send(message, messageQueue);
assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
assertThat(sendResult.getOffsetMsgId()).isEqualTo("123");
assertThat(sendResult.getQueueOffset()).isEqualTo(-1L);
}
@Test
public void testSendMessageAsync_Success() throws Exception {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final SettableFuture<SendResult> future = SettableFuture.create();
producer.send(message, messageQueue, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
future.set(sendResult);
}
@Override
public void onException(Throwable e) {
future.setException(e);
}
});
SendResult sendResult = future.get(3, TimeUnit.SECONDS);
assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
assertThat(sendResult.getOffsetMsgId()).isEqualTo("123");
assertThat(sendResult.getQueueOffset()).isEqualTo(456L);
}
@Test
public void testSendMessageAsync() throws Exception {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final AtomicReference<SettableFuture<SendResult>> future = new AtomicReference<SettableFuture<SendResult>>();
SendCallback sendCallback = new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
future.get().set(sendResult);
}
@Override
public void onException(Throwable e) {
future.get().setException(e);
}
};
Message message = new Message();
message.setTopic("test");
message.setBody("hello world".getBytes());
future.set(SettableFuture.<SendResult>create());
producer.send(new Message(), messageQueue, sendCallback);
assertThatThrownBy(new ThrowableAssert.ThrowingCallable() {
@Override public void call() throws Throwable {
future.get().get(3, TimeUnit.SECONDS);
}
}).hasCauseInstanceOf(MQClientException.class).hasMessageContaining("The specified topic is blank");
//this message is send success
message.setTopic(topic);
future.set(SettableFuture.<SendResult>create());
producer.send(message, messageQueue, sendCallback, 1000);
future.get().get(3, TimeUnit.SECONDS);
}
public TopicRouteData createTopicRoute() {
TopicRouteData topicRouteData = new TopicRouteData();
topicRouteData.setFilterServerTable(new HashMap<String, List<String>>());
topicRouteData.setBrokerDatas(ImmutableList.of(
new BrokerData(cluster, broker1Name, new HashMap<Long, String>(Collections.singletonMap(MixAll.MASTER_ID, broker1Addr))),
new BrokerData(cluster, broker2Name, new HashMap<Long, String>(Collections.singletonMap(MixAll.MASTER_ID, broker2Addr)))
));
List<QueueData> queueDataList = new ArrayList<QueueData>();
QueueData queueData;
queueData = new QueueData();
queueData.setBrokerName(broker1Name);
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
queueData = new QueueData();
queueData.setBrokerName(broker2Name);
queueData.setPerm(6);
queueData.setReadQueueNums(3);
queueData.setWriteQueueNums(4);
queueData.setTopicSysFlag(0);
queueDataList.add(queueData);
topicRouteData.setQueueDatas(queueDataList);
LogicalQueuesInfo info = new LogicalQueuesInfo();
info.put(0, Lists.newArrayList(new LogicalQueueRouteData(0, 0, new MessageQueue(topic, broker1Name, 0), MessageQueueRouteState.Normal, 0, 0, 0, 0, broker1Addr)));
topicRouteData.setLogicalQueuesInfo(info);
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");
sendResult.setMessageQueue(new MessageQueue(topic, broker1Name, 0));
return sendResult;
}
}
......@@ -22,6 +22,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
......@@ -53,6 +54,7 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentMatchers;
import org.mockito.Mock;
import org.mockito.Spy;
import org.mockito.junit.MockitoJUnitRunner;
......@@ -60,6 +62,7 @@ import org.mockito.junit.MockitoJUnitRunner;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
......@@ -150,7 +153,7 @@ public class DefaultMQProducerTest {
@Test
public void testSendMessageSync_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
SendResult sendResult = producer.send(message);
assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
......@@ -160,7 +163,7 @@ public class DefaultMQProducerTest {
@Test
public void testSendMessageSync_WithBodyCompressed() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
SendResult sendResult = producer.send(bigMessage);
assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
......@@ -171,7 +174,7 @@ public class DefaultMQProducerTest {
@Test
public void testSendMessageAsync_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
final CountDownLatch countDownLatch = new CountDownLatch(1);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
producer.send(message, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
......@@ -194,7 +197,7 @@ public class DefaultMQProducerTest {
final AtomicInteger cc = new AtomicInteger(0);
final CountDownLatch countDownLatch = new CountDownLatch(6);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
SendCallback sendCallback = new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
......@@ -236,7 +239,7 @@ public class DefaultMQProducerTest {
final AtomicInteger cc = new AtomicInteger(0);
final CountDownLatch countDownLatch = new CountDownLatch(4);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
SendCallback sendCallback = new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
......@@ -278,7 +281,7 @@ public class DefaultMQProducerTest {
@Test
public void testSendMessageAsync_BodyCompressed() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
final CountDownLatch countDownLatch = new CountDownLatch(1);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
producer.send(bigMessage, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
......@@ -297,7 +300,7 @@ public class DefaultMQProducerTest {
@Test
public void testSendMessageSync_SuccessWithHook() throws Throwable {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final Throwable[] assertionErrors = new Throwable[1];
final CountDownLatch countDownLatch = new CountDownLatch(2);
producer.getDefaultMQProducerImpl().registerSendMessageHook(new SendMessageHook() {
......@@ -365,7 +368,7 @@ public class DefaultMQProducerTest {
@Test
public void testRequestMessage() throws RemotingException, RequestTimeoutException, MQClientException, InterruptedException, MQBrokerException {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final AtomicBoolean finish = new AtomicBoolean(false);
new Thread(new Runnable() {
@Override public void run() {
......@@ -391,13 +394,13 @@ public class DefaultMQProducerTest {
@Test(expected = RequestTimeoutException.class)
public void testRequestMessage_RequestTimeoutException() throws RemotingException, RequestTimeoutException, MQClientException, InterruptedException, MQBrokerException {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
Message result = producer.request(message, 3 * 1000L);
}
@Test
public void testAsyncRequest_OnSuccess() throws Exception {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final CountDownLatch countDownLatch = new CountDownLatch(1);
RequestCallback requestCallback = new RequestCallback() {
@Override public void onSuccess(Message message) {
......
......@@ -26,6 +26,7 @@ import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
......@@ -102,7 +103,9 @@ public class DefaultMQConsumerWithOpenTracingTest {
@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());
for (MQClientInstance instance : factoryTable.values()) {
instance.shutdown();
}
factoryTable.clear();
when(mQClientAPIImpl.pullMessage(anyString(), any(PullMessageRequestHeader.class),
......@@ -173,7 +176,7 @@ public class DefaultMQConsumerWithOpenTracingTest {
@Test
public void testPullMessage_WithTrace_Success() throws InterruptedException, RemotingException, MQBrokerException, MQClientException {
final CountDownLatch countDownLatch = new CountDownLatch(1);
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<MessageExt>();
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
......@@ -193,7 +196,11 @@ public class DefaultMQConsumerWithOpenTracingTest {
assertThat(msg.getBody()).isEqualTo(new byte[]{'a'});
// wait until consumeMessageAfter hook of tracer is done surely.
waitAtMost(1, TimeUnit.SECONDS).until(() -> tracer.finishedSpans().size() == 1);
waitAtMost(1, TimeUnit.SECONDS).until(new Callable<Boolean>() {
@Override public Boolean call() throws Exception {
return tracer.finishedSpans().size() == 1;
}
});
MockSpan span = tracer.finishedSpans().get(0);
assertThat(span.tags().get(Tags.MESSAGE_BUS_DESTINATION.getKey())).isEqualTo(topic);
assertThat(span.tags().get(Tags.SPAN_KIND.getKey())).isEqualTo(Tags.SPAN_KIND_CONSUMER);
......
......@@ -115,7 +115,9 @@ 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());
for (MQClientInstance instance : factoryTable.values()) {
instance.shutdown();
}
factoryTable.clear();
consumerGroup = "FooBarGroup" + System.currentTimeMillis();
......@@ -216,7 +218,7 @@ public class DefaultMQConsumerWithTraceTest {
traceProducer.getDefaultMQProducerImpl().getmQClientFactory().registerProducer(producerGroupTraceTemp, traceProducer.getDefaultMQProducerImpl());
final CountDownLatch countDownLatch = new CountDownLatch(1);
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<>();
final AtomicReference<MessageExt> messageAtomic = new AtomicReference<MessageExt>();
pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs,
......
......@@ -20,6 +20,11 @@ package org.apache.rocketmq.client.trace;
import io.opentracing.mock.MockSpan;
import io.opentracing.mock.MockTracer;
import io.opentracing.tag.Tags;
import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Set;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
......@@ -47,17 +52,14 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentMatchers;
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.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
......@@ -113,7 +115,7 @@ public class DefaultMQProducerWithOpenTracingTest {
@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());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
producer.send(message);
assertThat(tracer.finishedSpans().size()).isEqualTo(1);
MockSpan span = tracer.finishedSpans().get(0);
......
......@@ -17,6 +17,13 @@
package org.apache.rocketmq.client.trace;
import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
......@@ -42,18 +49,17 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentMatchers;
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 java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import static org.mockito.ArgumentMatchers.*;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
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)
......@@ -121,7 +127,7 @@ public class DefaultMQProducerWithTraceTest {
@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());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final CountDownLatch countDownLatch = new CountDownLatch(1);
try {
producer.send(message);
......@@ -133,7 +139,7 @@ public class DefaultMQProducerWithTraceTest {
@Test
public void testSendMessageSync_WithTrace_NoBrokerSet_Exception() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final CountDownLatch countDownLatch = new CountDownLatch(1);
try {
producer.send(message);
......
......@@ -20,6 +20,12 @@ package org.apache.rocketmq.client.trace;
import io.opentracing.mock.MockSpan;
import io.opentracing.mock.MockTracer;
import io.opentracing.tag.Tags;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Set;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
......@@ -53,18 +59,14 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentMatchers;
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.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
......@@ -131,7 +133,7 @@ public class TransactionMQProducerWithOpenTracingTest {
@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());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
producer.sendMessageInTransaction(message, null);
assertThat(tracer.finishedSpans().size()).isEqualTo(2);
......
......@@ -17,6 +17,13 @@
package org.apache.rocketmq.client.trace;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
......@@ -50,19 +57,20 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.ArgumentMatchers;
import org.mockito.Mock;
import org.mockito.Spy;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.junit.MockitoJUnitRunner;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
import org.mockito.stubbing.Answer;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.*;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.nullable;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.when;
......@@ -127,7 +135,7 @@ public class TransactionMQProducerWithTraceTest {
Field fieldHooks = DefaultMQProducerImpl.class.getDeclaredField("endTransactionHookList");
fieldHooks.setAccessible(true);
List<EndTransactionHook>hooks = new ArrayList<>();
List<EndTransactionHook>hooks = new ArrayList<EndTransactionHook>();
hooks.add(endTransactionHook);
fieldHooks.set(producer.getDefaultMQProducerImpl(), hooks);
......@@ -142,12 +150,14 @@ public class TransactionMQProducerWithTraceTest {
@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());
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong(), anyBoolean(), ArgumentMatchers.<Set<Integer>>any())).thenReturn(createTopicRoute());
final AtomicReference<EndTransactionContext> context = new AtomicReference<EndTransactionContext>();
doAnswer(new Answer() {
@Override public Object answer(InvocationOnMock mock) throws Throwable {
context.set(mock.<EndTransactionContext>getArgument(0));
return null;
}
}).when(endTransactionHook).endTransaction(ArgumentMatchers.<EndTransactionContext>any());
producer.sendMessageInTransaction(message, null);
EndTransactionContext ctx = context.get();
......
......@@ -27,6 +27,19 @@
<artifactId>rocketmq-common</artifactId>
<name>rocketmq-common ${project.version}</name>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>6</source>
<target>6</target>
</configuration>
</plugin>
</plugins>
</build>
<dependencies>
<dependency>
<groupId>${project.groupId}</groupId>
......@@ -40,5 +53,9 @@
<groupId>commons-validator</groupId>
<artifactId>commons-validator</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
</dependencies>
</project>
......@@ -222,6 +222,8 @@ public class BrokerConfig {
private boolean autoDeleteUnusedStats = false;
private long forwardTimeout = 3 * 1000;
public static String localHostName() {
try {
return InetAddress.getLocalHost().getHostName();
......@@ -908,7 +910,6 @@ public class BrokerConfig {
this.autoDeleteUnusedStats = autoDeleteUnusedStats;
}
public long getLoadBalancePollNameServerInterval() {
return loadBalancePollNameServerInterval;
}
......@@ -958,4 +959,12 @@ public class BrokerConfig {
public void setDefaultPopShareQueueNum(int defaultPopShareQueueNum) {
this.defaultPopShareQueueNum = defaultPopShareQueueNum;
}
public long getForwardTimeout() {
return forwardTimeout;
}
public void setForwardTimeout(long timeout) {
this.forwardTimeout = timeout;
}
}
......@@ -17,6 +17,7 @@
package org.apache.rocketmq.common;
import java.io.IOException;
import java.util.Map;
import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.logging.InternalLoggerFactory;
......@@ -67,6 +68,16 @@ public abstract class ConfigManager {
public abstract void decode(final String jsonString);
public synchronized <T> void persist(String topicName, T t) {
// stub for future
this.persist();
}
public synchronized <T> void persist(Map<String, T> m) {
// stub for future
this.persist();
}
public synchronized void persist() {
String jsonString = this.encode(true);
if (jsonString != null) {
......
......@@ -16,6 +16,7 @@
*/
package org.apache.rocketmq.common;
import com.alibaba.fastjson.TypeReference;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.FileInputStream;
......@@ -26,6 +27,7 @@ import java.lang.annotation.Annotation;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.lang.reflect.Type;
import java.net.Inet6Address;
import java.net.InetAddress;
import java.net.NetworkInterface;
......@@ -41,6 +43,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.rocketmq.common.annotation.ImportantField;
import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.common.help.FAQUrl;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.logging.InternalLoggerFactory;
......@@ -83,6 +86,9 @@ public class MixAll {
public static final String ACL_CONF_TOOLS_FILE = "/conf/tools.yml";
public static final String REPLY_MESSAGE_FLAG = "reply";
private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME);
public static final String LOGICAL_QUEUE_MOCK_BROKER_NAME = "__logical_queue_broker__";
public static final Type TYPE_LIST_LOGICAL_QUEUE_ROUTE_DATA = new TypeReference<List<LogicalQueueRouteData>>() {
}.getType();
public static String getWSAddr() {
String wsDomainName = System.getProperty("rocketmq.namesrv.domain", DEFAULT_NAMESRV_ADDR_LOOKUP);
......@@ -443,4 +449,11 @@ public class MixAll {
return String.format("%.1f %sB", bytes / Math.pow(unit, exp), pre);
}
public static int compareInteger(int x, int y) {
return (x < y) ? -1 : ((x == y) ? 0 : 1);
}
public static int compareLong(long x, long y) {
return (x < y) ? -1 : ((x == y) ? 0 : 1);
}
}
......@@ -37,6 +37,16 @@ public class TopicConfig {
this.topicName = topicName;
}
public TopicConfig(TopicConfig other) {
this.topicName = other.topicName;
this.readQueueNums = other.readQueueNums;
this.writeQueueNums = other.writeQueueNums;
this.perm = other.perm;
this.topicFilterType = other.topicFilterType;
this.topicSysFlag = other.topicSysFlag;
this.order = other.order;
}
public TopicConfig(String topicName, int readQueueNums, int writeQueueNums, int perm) {
this.topicName = topicName;
this.readQueueNums = readQueueNums;
......
/*
* 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;
import com.google.common.base.Objects;
public class TopicQueueId {
private final String topic;
private final int queueId;
private final int hash;
public TopicQueueId(String topic, int queueId) {
this.topic = topic;
this.queueId = queueId;
this.hash = Objects.hashCode(topic, queueId);
}
@Override public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
TopicQueueId broker = (TopicQueueId) o;
return queueId == broker.queueId && Objects.equal(topic, broker.topic);
}
@Override public int hashCode() {
return hash;
}
@Override public String toString() {
final StringBuilder sb = new StringBuilder("MessageQueueInBroker{");
sb.append("topic='").append(topic).append('\'');
sb.append(", queueId=").append(queueId);
sb.append('}');
return sb.toString();
}
}
......@@ -38,4 +38,5 @@ public class LoggerName {
public static final String WATER_MARK_LOGGER_NAME = "RocketmqWaterMark";
public static final String FILTER_LOGGER_NAME = "RocketmqFilter";
public static final String ROCKETMQ_POP_LOGGER_NAME = "RocketmqPop";
public static final String STDOUT_LOGGER_NAME = "STDOUT";
}
/*
* 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.fastjson;
import com.alibaba.fastjson.JSONException;
import com.alibaba.fastjson.parser.DefaultJSONParser;
import com.alibaba.fastjson.parser.JSONToken;
import com.alibaba.fastjson.parser.deserializer.MapDeserializer;
import com.alibaba.fastjson.parser.deserializer.ObjectDeserializer;
import java.lang.reflect.ParameterizedType;
import java.lang.reflect.Type;
import java.util.Map;
/**
* workaround https://github.com/alibaba/fastjson/issues/3730
*/
public class GenericMapSuperclassDeserializer implements ObjectDeserializer {
public static final GenericMapSuperclassDeserializer INSTANCE = new GenericMapSuperclassDeserializer();
@SuppressWarnings({"unchecked", "rawtypes"})
@Override public <T> T deserialze(DefaultJSONParser parser, Type type, Object fieldName) {
Class<?> clz = (Class<?>) type;
Type genericSuperclass = clz.getGenericSuperclass();
Map map;
try {
map = (Map) clz.newInstance();
} catch (Exception e) {
throw new JSONException("unsupport type " + type, e);
}
ParameterizedType parameterizedType = (ParameterizedType) genericSuperclass;
Type keyType = parameterizedType.getActualTypeArguments()[0];
Type valueType = parameterizedType.getActualTypeArguments()[1];
if (String.class == keyType) {
return (T) MapDeserializer.parseMap(parser, (Map<String, Object>) map, valueType, fieldName);
} else {
return (T) MapDeserializer.parseMap(parser, map, keyType, valueType, fieldName);
}
}
@Override public int getFastMatchToken() {
return JSONToken.LBRACE;
}
}
......@@ -54,6 +54,8 @@ public class MessageConst {
public static final String PROPERTY_MESSAGE_TYPE = "MSG_TYPE";
public static final String PROPERTY_POP_CK = "POP_CK";
public static final String PROPERTY_FIRST_POP_TIME = "1ST_POP_TIME";
public static final String PROPERTY_FORWARD_QUEUE_ID = "PROPERTY_FORWARD_QUEUE_ID";
public static final String PROPERTY_REDIRECT = "REDIRECT";
public static final String KEY_SEPARATOR = " ";
......
......@@ -28,6 +28,12 @@ public class MessageQueue implements Comparable<MessageQueue>, Serializable {
}
public MessageQueue(MessageQueue other) {
this.topic = other.topic;
this.brokerName = other.brokerName;
this.queueId = other.queueId;
}
public MessageQueue(String topic, String brokerName, int queueId) {
this.topic = topic;
this.brokerName = brokerName;
......
......@@ -193,6 +193,18 @@ public class RequestCode {
public static final int PUSH_REPLY_MESSAGE_TO_CLIENT = 326;
public static final int GET_TOPIC_CONFIG = 351;
public static final int QUERY_ASSIGNMENT = 400;
public static final int SET_MESSAGE_REQUEST_MODE = 401;
public static final int UPDATE_TOPIC_LOGICAL_QUEUE_MAPPING = 411;
public static final int DELETE_TOPIC_LOGICAL_QUEUE_MAPPING = 422;
public static final int QUERY_TOPIC_LOGICAL_QUEUE_MAPPING = 413;
public static final int SEAL_TOPIC_LOGICAL_QUEUE = 414;
public static final int REUSE_TOPIC_LOGICAL_QUEUE = 415;
public static final int CREATE_MESSAGE_QUEUE_FOR_LOGICAL_QUEUE = 416;
public static final int MIGRATE_TOPIC_LOGICAL_QUEUE_PREPARE = 417;
public static final int MIGRATE_TOPIC_LOGICAL_QUEUE_COMMIT = 418;
public static final int MIGRATE_TOPIC_LOGICAL_QUEUE_NOTIFY = 419;
}
......@@ -20,6 +20,7 @@ package org.apache.rocketmq.common.protocol;
import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
public class ResponseCode extends RemotingSysResponseCode {
public static final int ASYNC_AND_RETURN_NULL = -2;
public static final int FLUSH_DISK_TIMEOUT = 10;
......
......@@ -17,6 +17,7 @@
package org.apache.rocketmq.common.protocol.body;
import com.google.common.base.Objects;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
......@@ -62,4 +63,17 @@ public class ClusterInfo extends RemotingSerializable {
public String[] retrieveAllClusterNames() {
return clusterAddrTable.keySet().toArray(new String[] {});
}
@Override public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
ClusterInfo info = (ClusterInfo) o;
return Objects.equal(brokerAddrTable, info.brokerAddrTable) && Objects.equal(clusterAddrTable, info.clusterAddrTable);
}
@Override public int hashCode() {
return Objects.hashCode(brokerAddrTable, clusterAddrTable);
}
}
/*
* 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.protocol.body;
import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
public class CreateMessageQueueForLogicalQueueRequestBody extends RemotingSerializable {
private String topic;
private int logicalQueueIndex;
private MessageQueueRouteState messageQueueStatus;
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
public int getLogicalQueueIndex() {
return logicalQueueIndex;
}
public void setLogicalQueueIndex(int logicalQueueIndex) {
this.logicalQueueIndex = logicalQueueIndex;
}
public MessageQueueRouteState getMessageQueueStatus() {
return messageQueueStatus;
}
public void setMessageQueueStatus(MessageQueueRouteState messageQueueStatuses) {
this.messageQueueStatus = messageQueueStatuses;
}
}
/*
* 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.protocol.body;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
public class MigrateLogicalQueueBody extends RemotingSerializable {
private LogicalQueueRouteData fromQueueRouteData;
private LogicalQueueRouteData toQueueRouteData;
public LogicalQueueRouteData getFromQueueRouteData() {
return fromQueueRouteData;
}
public void setFromQueueRouteData(
LogicalQueueRouteData fromQueueRouteData) {
this.fromQueueRouteData = fromQueueRouteData;
}
public LogicalQueueRouteData getToQueueRouteData() {
return toQueueRouteData;
}
public void setToQueueRouteData(LogicalQueueRouteData toQueueRouteData) {
this.toQueueRouteData = toQueueRouteData;
}
}
/*
* 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.protocol.body;
import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
public class ReuseTopicLogicalQueueRequestBody extends RemotingSerializable {
private String topic;
private int queueId;
private int logicalQueueIndex;
private MessageQueueRouteState messageQueueRouteState;
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
public int getQueueId() {
return queueId;
}
public void setQueueId(int queueId) {
this.queueId = queueId;
}
public int getLogicalQueueIndex() {
return logicalQueueIndex;
}
public void setLogicalQueueIndex(int logicalQueueIndex) {
this.logicalQueueIndex = logicalQueueIndex;
}
public void setMessageQueueRouteState(MessageQueueRouteState messageQueueRouteState) {
this.messageQueueRouteState = messageQueueRouteState;
}
public MessageQueueRouteState getMessageQueueRouteState() {
return messageQueueRouteState;
}
}
/*
* 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.protocol.body;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
public class SealTopicLogicalQueueRequestBody extends RemotingSerializable {
private String topic;
private int queueId;
private int logicalQueueIndex;
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
public int getQueueId() {
return queueId;
}
public void setQueueId(int queueId) {
this.queueId = queueId;
}
public int getLogicalQueueIndex() {
return logicalQueueIndex;
}
public void setLogicalQueueIndex(int logicalQueueIndex) {
this.logicalQueueIndex = logicalQueueIndex;
}
}
......@@ -17,15 +17,18 @@
package org.apache.rocketmq.common.protocol.body;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.rocketmq.common.DataVersion;
import org.apache.rocketmq.common.TopicConfig;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
public class TopicConfigSerializeWrapper extends RemotingSerializable {
private ConcurrentMap<String, TopicConfig> topicConfigTable =
new ConcurrentHashMap<String, TopicConfig>();
private Map<String/* topic */, LogicalQueuesInfo> logicalQueuesInfoMap;
private DataVersion dataVersion = new DataVersion();
public ConcurrentMap<String, TopicConfig> getTopicConfigTable() {
......@@ -43,4 +46,12 @@ public class TopicConfigSerializeWrapper extends RemotingSerializable {
public void setDataVersion(DataVersion dataVersion) {
this.dataVersion = dataVersion;
}
public Map<String, LogicalQueuesInfo> getLogicalQueuesInfoMap() {
return logicalQueuesInfoMap;
}
public void setLogicalQueuesInfoMap(Map<String, LogicalQueuesInfo> logicalQueuesInfoMap) {
this.logicalQueuesInfoMap = logicalQueuesInfoMap;
}
}
/*
* 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.protocol.body;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
public class UpdateTopicLogicalQueueMappingRequestBody extends RemotingSerializable {
private String topic;
private int queueId;
private int logicalQueueIdx;
public int getLogicalQueueIdx() {
return logicalQueueIdx;
}
public void setLogicalQueueIdx(int logicalQueueIdx) {
this.logicalQueueIdx = logicalQueueIdx;
}
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
public int getQueueId() {
return queueId;
}
public void setQueueId(int queueId) {
this.queueId = queueId;
}
}
/*
* 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.protocol.header;
import org.apache.rocketmq.remoting.CommandCustomHeader;
import org.apache.rocketmq.remoting.annotation.CFNotNull;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
public class DeleteTopicLogicalQueueRequestHeader implements CommandCustomHeader {
@CFNotNull
private String topic;
@Override public void checkFields() throws RemotingCommandException {
}
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
}
......@@ -29,6 +29,8 @@ public class GetMaxOffsetRequestHeader implements CommandCustomHeader {
private String topic;
@CFNotNull
private Integer queueId;
private boolean committed;
private boolean logicalQueue;
@Override
public void checkFields() throws RemotingCommandException {
......@@ -49,4 +51,20 @@ public class GetMaxOffsetRequestHeader implements CommandCustomHeader {
public void setQueueId(Integer queueId) {
this.queueId = queueId;
}
public void setCommitted(boolean committed) {
this.committed = committed;
}
public boolean isCommitted() {
return committed;
}
public void setLogicalQueue(boolean logicalQueue) {
this.logicalQueue = logicalQueue;
}
public boolean getLogicalQueue() {
return logicalQueue;
}
}
/*
* 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.protocol.header;
import org.apache.rocketmq.remoting.CommandCustomHeader;
import org.apache.rocketmq.remoting.annotation.CFNotNull;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
public class GetTopicConfigRequestHeader implements CommandCustomHeader {
@Override
public void checkFields() throws RemotingCommandException {
}
@CFNotNull
private String topic;
/**
* @return the topic
*/
public String getTopic() {
return topic;
}
/**
* @param topic the topic to set
*/
public void setTopic(String topic) {
this.topic = topic;
}
}
\ 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.common.protocol.header;
import org.apache.rocketmq.remoting.CommandCustomHeader;
import org.apache.rocketmq.remoting.annotation.CFNotNull;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
public class QueryTopicLogicalQueueMappingRequestHeader implements CommandCustomHeader {
@CFNotNull
private String topic;
@Override public void checkFields() throws RemotingCommandException {
}
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
}
......@@ -20,6 +20,7 @@
*/
package org.apache.rocketmq.common.protocol.header.namesrv;
import java.util.Set;
import org.apache.rocketmq.remoting.CommandCustomHeader;
import org.apache.rocketmq.remoting.annotation.CFNotNull;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
......@@ -28,6 +29,9 @@ public class GetRouteInfoRequestHeader implements CommandCustomHeader {
@CFNotNull
private String topic;
private int sysFlag;
private Set<Integer> logicalQueueIdsFilter;
@Override
public void checkFields() throws RemotingCommandException {
}
......@@ -39,4 +43,20 @@ public class GetRouteInfoRequestHeader implements CommandCustomHeader {
public void setTopic(String topic) {
this.topic = topic;
}
public int getSysFlag() {
return sysFlag;
}
public void setSysFlag(int sysFlag) {
this.sysFlag = sysFlag;
}
public void setLogicalQueueIdsFilter(Set<Integer> filter) {
this.logicalQueueIdsFilter = filter;
}
public Set<Integer> getLogicalQueueIdsFilter() {
return logicalQueueIdsFilter;
}
}
/*
* 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.protocol.route;
import com.alibaba.fastjson.annotation.JSONField;
import com.google.common.base.Objects;
import com.google.common.base.Predicate;
import com.google.common.collect.Lists;
import java.util.List;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.message.MessageQueue;
/**
* logical queue offset -> message queue offset mapping
*/
public class LogicalQueueRouteData implements Comparable<LogicalQueueRouteData> {
private volatile int logicalQueueIndex = -1; /* -1 means not set */
private volatile long logicalQueueDelta = -1; /* inclusive, -1 means not set, occurred in writeOnly state */
private MessageQueue messageQueue;
private volatile MessageQueueRouteState state = MessageQueueRouteState.Normal;
private volatile long offsetDelta = 0; // valid when Normal/WriteOnly/ReadOnly
private volatile long offsetMax = -1; // exclusive, valid when ReadOnly
private volatile long firstMsgTimeMillis = -1; // valid when ReadOnly
private volatile long lastMsgTimeMillis = -1; // valid when ReadOnly
private String brokerAddr; /* not always set, only used by high availability forward */
public LogicalQueueRouteData() {
}
public LogicalQueueRouteData(int logicalQueueIndex, long logicalQueueDelta,
MessageQueue messageQueue, MessageQueueRouteState state, long offsetDelta, long offsetMax,
long firstMsgTimeMillis,
long lastMsgTimeMillis, String brokerAddr) {
this.logicalQueueIndex = logicalQueueIndex;
this.logicalQueueDelta = logicalQueueDelta;
this.messageQueue = messageQueue;
this.state = state;
this.offsetDelta = offsetDelta;
this.offsetMax = offsetMax;
this.firstMsgTimeMillis = firstMsgTimeMillis;
this.lastMsgTimeMillis = lastMsgTimeMillis;
this.brokerAddr = brokerAddr;
}
public LogicalQueueRouteData(LogicalQueueRouteData queueRouteData) {
copyFrom(queueRouteData);
}
public int getLogicalQueueIndex() {
return logicalQueueIndex;
}
public void setLogicalQueueIndex(int logicalQueueIndex) {
this.logicalQueueIndex = logicalQueueIndex;
}
public long getLogicalQueueDelta() {
return logicalQueueDelta;
}
public void setLogicalQueueDelta(long logicalQueueDelta) {
this.logicalQueueDelta = logicalQueueDelta;
}
public MessageQueue getMessageQueue() {
return messageQueue;
}
public void setMessageQueue(MessageQueue messageQueue) {
this.messageQueue = messageQueue;
}
public MessageQueueRouteState getState() {
return state;
}
@JSONField(serialize = false)
public int getStateOrdinal() {
return state.ordinal();
}
public void setState(MessageQueueRouteState state) {
this.state = state;
}
public String getBrokerAddr() {
return brokerAddr;
}
public void setBrokerAddr(String brokerAddr) {
this.brokerAddr = brokerAddr;
}
public long getOffsetDelta() {
return offsetDelta;
}
public void setOffsetDelta(long offsetDelta) {
this.offsetDelta = offsetDelta;
}
public long getOffsetMax() {
return offsetMax;
}
public void setOffsetMax(long offsetMax) {
this.offsetMax = offsetMax;
}
public long getFirstMsgTimeMillis() {
return firstMsgTimeMillis;
}
public void setFirstMsgTimeMillis(long firstMsgTimeMillis) {
this.firstMsgTimeMillis = firstMsgTimeMillis;
}
public long getLastMsgTimeMillis() {
return lastMsgTimeMillis;
}
public void setLastMsgTimeMillis(long lastMsgTimeMillis) {
this.lastMsgTimeMillis = lastMsgTimeMillis;
}
@Override public String toString() {
return "LogicalQueueRouteData{" +
"logicalQueueIndex=" + logicalQueueIndex +
", logicalQueueDelta=" + logicalQueueDelta +
", messageQueue=" + messageQueue +
", state=" + state +
", offsetDelta=" + offsetDelta +
", offsetMax=" + offsetMax +
", firstMsgTimeMillis=" + firstMsgTimeMillis +
", lastMsgTimeMillis=" + lastMsgTimeMillis +
", brokerAddr='" + brokerAddr + '\'' +
'}';
}
public void copyFrom(LogicalQueueRouteData queueRouteData) {
this.logicalQueueIndex = queueRouteData.logicalQueueIndex;
this.logicalQueueDelta = queueRouteData.logicalQueueDelta;
this.messageQueue = new MessageQueue(queueRouteData.getMessageQueue());
this.state = queueRouteData.state;
this.offsetDelta = queueRouteData.offsetDelta;
this.offsetMax = queueRouteData.offsetMax;
this.firstMsgTimeMillis = queueRouteData.firstMsgTimeMillis;
this.lastMsgTimeMillis = queueRouteData.lastMsgTimeMillis;
this.brokerAddr = queueRouteData.brokerAddr;
}
public long toLogicalQueueOffset(long messageQueueOffset) {
return this.logicalQueueDelta < 0 ? -1 : messageQueueOffset - this.offsetDelta + this.logicalQueueDelta;
}
public long toMessageQueueOffset(long logicalQueueOffset) {
return logicalQueueOffset - this.logicalQueueDelta + this.offsetDelta;
}
@Override public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
LogicalQueueRouteData that = (LogicalQueueRouteData) o;
return logicalQueueIndex == that.logicalQueueIndex && logicalQueueDelta == that.logicalQueueDelta && offsetDelta == that.offsetDelta && offsetMax == that.offsetMax && firstMsgTimeMillis == that.firstMsgTimeMillis && lastMsgTimeMillis == that.lastMsgTimeMillis && Objects.equal(messageQueue, that.messageQueue) && state == that.state && Objects.equal(brokerAddr, that.brokerAddr);
}
@Override public int hashCode() {
return Objects.hashCode(logicalQueueIndex, logicalQueueDelta, messageQueue, state, offsetDelta, offsetMax, firstMsgTimeMillis, lastMsgTimeMillis, brokerAddr);
}
@JSONField(serialize = false)
public long getMessagesCount() {
return this.offsetDelta >= 0 && this.offsetMax >= 0 ? this.offsetMax - this.offsetDelta : 0L;
}
@JSONField(serialize = false)
public boolean isWritable() {
return MessageQueueRouteState.Normal.equals(state) || MessageQueueRouteState.WriteOnly.equals(state);
}
@JSONField(serialize = false)
public boolean isReadable() {
return MessageQueueRouteState.Normal.equals(state) || MessageQueueRouteState.ReadOnly.equals(state);
}
@JSONField(serialize = false)
public boolean isExpired() {
return MessageQueueRouteState.Expired.equals(state);
}
@JSONField(serialize = false)
public boolean isWriteOnly() {
return MessageQueueRouteState.WriteOnly.equals(state);
}
@JSONField(serialize = false)
public int getQueueId() {
return messageQueue.getQueueId();
}
@JSONField(serialize = false)
public String getBrokerName() {
return messageQueue.getBrokerName();
}
@JSONField(serialize = false)
public String getTopic() {
return messageQueue.getTopic();
}
public boolean isSameTo(LogicalQueueRouteData o) {
if (o == null) {
return false;
}
return isSameTo(o.getMessageQueue(), o.offsetDelta);
}
public boolean isSameTo(MessageQueue mq, long offsetDelta) {
return Objects.equal(this.messageQueue, mq) && this.offsetDelta == offsetDelta;
}
/**
* First compare logicalQueueDelta, negative delta must be ordered in the last; then compare state's ordinal; then
* compare messageQueue, nulls first; then compare offsetDelta.
*/
@Override
public int compareTo(LogicalQueueRouteData o) {
long x = this.getLogicalQueueDelta();
long y = o.getLogicalQueueDelta();
int result;
{
if (x >= 0 && y >= 0) {
result = MixAll.compareLong(x, y);
} else if (x < 0 && y < 0) {
result = MixAll.compareLong(-x, -y);
} else if (x < 0) {
// o1 < 0 && o2 >= 0
result = 1;
} else {
// o1 >= 0 && o2 < 0
result = -1;
}
}
if (result == 0) {
result = MixAll.compareInteger(this.state.ordinal(), o.state.ordinal());
}
if (result == 0) {
if (this.messageQueue == null) {
if (o.messageQueue != null) {
result = -1;
}
} else {
if (o.messageQueue != null) {
result = this.messageQueue.compareTo(o.messageQueue);
} else {
result = 1;
}
}
}
if (result == 0) {
result = MixAll.compareLong(this.offsetDelta, o.offsetDelta);
}
return result;
}
public static final Predicate<LogicalQueueRouteData> READABLE_PREDICT = new Predicate<LogicalQueueRouteData>() {
@Override
public boolean apply(LogicalQueueRouteData input) {
return input != null && input.isReadable();
}
};
public List<MessageExt> filterMessages(List<MessageExt> list) {
if (this.offsetMax < 0 || list == null || list.isEmpty()) {
return list;
}
List<MessageExt> result = Lists.newArrayListWithExpectedSize(list.size());
for (MessageExt m : list) {
if (m.getQueueOffset() >= this.offsetMax) {
break;
} else {
result.add(m);
}
}
return result;
}
}
/*
* 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.protocol.route;
import com.alibaba.fastjson.parser.ParserConfig;
import com.google.common.base.Objects;
import com.google.common.collect.Lists;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.rocketmq.common.fastjson.GenericMapSuperclassDeserializer;
public class LogicalQueuesInfo extends HashMap<Integer, List<LogicalQueueRouteData>> {
// TODO whether here needs more fine-grained locks like per logical queue lock?
private final ReadWriteLock lock = new ReentrantReadWriteLock();
public LogicalQueuesInfo() {
super();
}
public LogicalQueuesInfo(Map<Integer, List<LogicalQueueRouteData>> m) {
super(m);
}
public Lock readLock() {
return lock.readLock();
}
public Lock writeLock() {
return lock.writeLock();
}
public void updateLogicalQueueRouteDataList(int logicalQueueIdx,
List<LogicalQueueRouteData> logicalQueueRouteDataList) {
this.writeLock().lock();
try {
logicalQueueRouteDataList = Lists.newLinkedList(logicalQueueRouteDataList);
List<LogicalQueueRouteData> queueRouteDataList = this.get(logicalQueueIdx);
for (LogicalQueueRouteData logicalQueueRouteData : queueRouteDataList) {
for (Iterator<LogicalQueueRouteData> it = logicalQueueRouteDataList.iterator(); it.hasNext(); ) {
LogicalQueueRouteData newQueueRouteData = it.next();
if (Objects.equal(newQueueRouteData.getMessageQueue(), logicalQueueRouteData.getMessageQueue()) && newQueueRouteData.getOffsetDelta() == logicalQueueRouteData.getOffsetDelta()) {
logicalQueueRouteData.copyFrom(newQueueRouteData);
it.remove();
break;
}
}
if (logicalQueueRouteDataList.isEmpty()) {
break;
}
}
for (LogicalQueueRouteData queueRouteData : logicalQueueRouteDataList) {
int idx = Collections.binarySearch(queueRouteDataList, queueRouteData);
if (idx < 0) {
idx = -idx - 1;
}
queueRouteDataList.add(idx, queueRouteData);
}
} finally {
this.writeLock().unlock();
}
}
static {
// workaround https://github.com/alibaba/fastjson/issues/3730
ParserConfig.getGlobalInstance().putDeserializer(LogicalQueuesInfo.class, GenericMapSuperclassDeserializer.INSTANCE);
}
}
/*
* 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.protocol.route;
import com.alibaba.fastjson.parser.ParserConfig;
import com.google.common.base.Objects;
import com.google.common.collect.Lists;
import org.apache.rocketmq.common.fastjson.GenericMapSuperclassDeserializer;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
/**
* Only used inside namesrv, between client and namesrv, to reduce cpu usage of namesrv
*/
public class LogicalQueuesInfoUnordered extends ConcurrentHashMap<Integer, Map<LogicalQueuesInfoUnordered.Key, LogicalQueueRouteData>> {
static {
// workaround https://github.com/alibaba/fastjson/issues/3730
ParserConfig.getGlobalInstance().putDeserializer(LogicalQueuesInfoUnordered.class, GenericMapSuperclassDeserializer.INSTANCE);
}
public LogicalQueuesInfoUnordered() {
super();
}
public LogicalQueuesInfoUnordered(int size) {
super(size);
}
public LogicalQueuesInfo toLogicalQueuesInfoOrdered() {
LogicalQueuesInfo logicalQueuesInfoOrdered = new LogicalQueuesInfo();
for (Map.Entry<Integer, Map<Key, LogicalQueueRouteData>> entry : this.entrySet()) {
List<LogicalQueueRouteData> list = Lists.newArrayListWithExpectedSize(entry.getValue().size());
for (LogicalQueueRouteData d : entry.getValue().values()) {
list.add(new LogicalQueueRouteData(d));
}
Collections.sort(list);
logicalQueuesInfoOrdered.put(entry.getKey(), list);
}
return logicalQueuesInfoOrdered;
}
public static class Key {
private final String brokerName;
private final int queueId;
private final long offsetDelta;
private final int hash;
public Key(String brokerName, int queueId, long offsetDelta) {
this.brokerName = brokerName;
this.queueId = queueId;
this.offsetDelta = offsetDelta;
this.hash = Objects.hashCode(brokerName, queueId, this.offsetDelta);
}
public String getBrokerName() {
return brokerName;
}
public int getQueueId() {
return queueId;
}
public long getOffsetDelta() {
return offsetDelta;
}
@Override public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
Key id = (Key) o;
return queueId == id.queueId && offsetDelta == id.offsetDelta && Objects.equal(brokerName, id.brokerName);
}
@Override public int hashCode() {
return hash;
}
@Override public String toString() {
return "Key{" +
"brokerName='" + brokerName + '\'' +
", queueId=" + queueId +
", offsetDelta=" + offsetDelta +
'}';
}
}
}
/*
* 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.protocol.route;
public enum MessageQueueRouteState {
// do not change below order, since ordinal() is used
Expired,
ReadOnly,
Normal,
WriteOnly,
;
}
......@@ -30,27 +30,32 @@ public class TopicRouteData extends RemotingSerializable {
private List<QueueData> queueDatas;
private List<BrokerData> brokerDatas;
private HashMap<String/* brokerAddr */, List<String>/* Filter Server */> filterServerTable;
private LogicalQueuesInfo logicalQueuesInfo;
public TopicRouteData cloneTopicRouteData() {
TopicRouteData topicRouteData = new TopicRouteData();
topicRouteData.setQueueDatas(new ArrayList<QueueData>());
topicRouteData.setBrokerDatas(new ArrayList<BrokerData>());
topicRouteData.setFilterServerTable(new HashMap<String, List<String>>());
topicRouteData.setOrderTopicConf(this.orderTopicConf);
public TopicRouteData() {
}
public TopicRouteData(TopicRouteData topicRouteData) {
this.queueDatas = new ArrayList<QueueData>();
this.brokerDatas = new ArrayList<BrokerData>();
this.filterServerTable = new HashMap<String, List<String>>();
this.orderTopicConf = topicRouteData.orderTopicConf;
if (this.queueDatas != null) {
topicRouteData.getQueueDatas().addAll(this.queueDatas);
if (topicRouteData.queueDatas != null) {
this.queueDatas.addAll(topicRouteData.queueDatas);
}
if (this.brokerDatas != null) {
topicRouteData.getBrokerDatas().addAll(this.brokerDatas);
if (topicRouteData.brokerDatas != null) {
this.brokerDatas.addAll(topicRouteData.brokerDatas);
}
if (this.filterServerTable != null) {
topicRouteData.getFilterServerTable().putAll(this.filterServerTable);
if (topicRouteData.filterServerTable != null) {
this.filterServerTable.putAll(topicRouteData.filterServerTable);
}
return topicRouteData;
if (topicRouteData.logicalQueuesInfo != null) {
this.logicalQueuesInfo = new LogicalQueuesInfo(topicRouteData.logicalQueuesInfo);
}
}
public List<QueueData> getQueueDatas() {
......@@ -85,6 +90,14 @@ public class TopicRouteData extends RemotingSerializable {
this.orderTopicConf = orderTopicConf;
}
public LogicalQueuesInfo getLogicalQueuesInfo() {
return logicalQueuesInfo;
}
public void setLogicalQueuesInfo(LogicalQueuesInfo logicalQueuesInfo) {
this.logicalQueuesInfo = logicalQueuesInfo;
}
@Override
public int hashCode() {
final int prime = 31;
......@@ -93,6 +106,7 @@ public class TopicRouteData extends RemotingSerializable {
result = prime * result + ((orderTopicConf == null) ? 0 : orderTopicConf.hashCode());
result = prime * result + ((queueDatas == null) ? 0 : queueDatas.hashCode());
result = prime * result + ((filterServerTable == null) ? 0 : filterServerTable.hashCode());
result = prime * result + ((logicalQueuesInfo == null) ? 0 : logicalQueuesInfo.hashCode());
return result;
}
......@@ -125,12 +139,17 @@ public class TopicRouteData extends RemotingSerializable {
return false;
} else if (!filterServerTable.equals(other.filterServerTable))
return false;
if (logicalQueuesInfo == null) {
if (other.logicalQueuesInfo != null)
return false;
} else if (!logicalQueuesInfo.equals(other.logicalQueuesInfo))
return false;
return true;
}
@Override
public String toString() {
return "TopicRouteData [orderTopicConf=" + orderTopicConf + ", queueDatas=" + queueDatas
+ ", brokerDatas=" + brokerDatas + ", filterServerTable=" + filterServerTable + "]";
+ ", brokerDatas=" + brokerDatas + ", filterServerTable=" + filterServerTable + ", logicalQueuesInfo=" + logicalQueuesInfo + "]";
}
}
/*
* 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.protocol.route;
import com.google.common.base.Objects;
public class TopicRouteDataNameSrv extends TopicRouteData {
private LogicalQueuesInfoUnordered logicalQueuesInfoUnordered;
public TopicRouteDataNameSrv() {
}
public LogicalQueuesInfoUnordered getLogicalQueuesInfoUnordered() {
return logicalQueuesInfoUnordered;
}
public void setLogicalQueuesInfoUnordered(
LogicalQueuesInfoUnordered logicalQueuesInfoUnordered) {
this.logicalQueuesInfoUnordered = logicalQueuesInfoUnordered;
}
@Override public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
if (!super.equals(o))
return false;
TopicRouteDataNameSrv srv = (TopicRouteDataNameSrv) o;
return Objects.equal(logicalQueuesInfoUnordered, srv.logicalQueuesInfoUnordered);
}
@Override public int hashCode() {
return Objects.hashCode(super.hashCode(), logicalQueuesInfoUnordered);
}
@Override public String toString() {
return "TopicRouteDataNameSrv{" +
"logicalQueuesInfoUnordered=" + logicalQueuesInfoUnordered +
"} " + super.toString();
}
public TopicRouteData toTopicRouteData() {
TopicRouteData topicRouteData = new TopicRouteData(this);
if (this.logicalQueuesInfoUnordered != null) {
topicRouteData.setLogicalQueuesInfo(this.logicalQueuesInfoUnordered.toLogicalQueuesInfoOrdered());
}
return topicRouteData;
}
}
......@@ -25,6 +25,7 @@ public class MessageSysFlag {
public final static int TRANSACTION_ROLLBACK_TYPE = 0x3 << 2;
public final static int BORNHOST_V6_FLAG = 0x1 << 4;
public final static int STOREHOSTADDRESS_V6_FLAG = 0x1 << 5;
public final static int LOGICAL_QUEUE_FLAG = 0x1 << 6;
public static int getTransactionValue(final int flag) {
return flag & TRANSACTION_ROLLBACK_TYPE;
......
/*
* 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.fastjson;
import com.alibaba.fastjson.JSON;
import java.util.Collections;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.common.protocol.route.LogicalQueueRouteData;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfo;
import org.apache.rocketmq.common.protocol.route.LogicalQueuesInfoUnordered;
import org.apache.rocketmq.common.protocol.route.MessageQueueRouteState;
import org.assertj.core.util.Lists;
import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
public class GenericMapSuperclassDeserializerTest {
@Test
public void testLogicalQueuesInfo() throws Exception {
LogicalQueuesInfo logicalQueuesInfo = new LogicalQueuesInfo();
logicalQueuesInfo.put(0, Lists.newArrayList(new LogicalQueueRouteData(1, 2, new MessageQueue("topic", "broker", 3), MessageQueueRouteState.Normal, 4, 5, 6, 7, "127.1.2.3")));
byte[] buf = JSON.toJSONBytes(logicalQueuesInfo);
LogicalQueuesInfo newLogicalQueuesInfo = JSON.parseObject(buf, LogicalQueuesInfo.class);
assertThat(newLogicalQueuesInfo).isEqualTo(logicalQueuesInfo);
}
@Test
public void testLogicalQueuesInfoUnordered() throws Exception {
LogicalQueuesInfoUnordered logicalQueuesInfoUnordered = new LogicalQueuesInfoUnordered();
MessageQueue mq = new MessageQueue("topic", "broker", 3);
logicalQueuesInfoUnordered.put(0, new ConcurrentHashMap<LogicalQueuesInfoUnordered.Key, LogicalQueueRouteData>(Collections.singletonMap(new LogicalQueuesInfoUnordered.Key(mq.getBrokerName(), mq.getQueueId(), 4), new LogicalQueueRouteData(1, 2, mq, MessageQueueRouteState.Normal, 4, 5, 6, 7, "127.1.2.3"))));
byte[] buf = JSON.toJSONBytes(logicalQueuesInfoUnordered);
LogicalQueuesInfoUnordered newLogicalQueuesInfoUnordered = JSON.parseObject(buf, LogicalQueuesInfoUnordered.class);
assertThat(newLogicalQueuesInfoUnordered).isEqualTo(logicalQueuesInfoUnordered);
}
}
\ No newline at end of file
......@@ -18,18 +18,13 @@
package org.apache.rocketmq.common.protocol.route;
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.protocol.RemotingSerializable;
import org.junit.Test;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.within;
public class TopicRouteDataTest {
......@@ -64,7 +59,7 @@ public class TopicRouteDataTest {
topicRouteData.setFilterServerTable(new HashMap<String, List<String>>());
topicRouteData.setQueueDatas(queueDataList);
assertThat(topicRouteData.cloneTopicRouteData()).isEqualTo(topicRouteData);
assertThat(new TopicRouteData(topicRouteData)).isEqualTo(topicRouteData);
}
......
......@@ -18,9 +18,10 @@ package org.apache.rocketmq.remoting.protocol;
import com.alibaba.fastjson.JSON;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
public abstract class RemotingSerializable {
private final static Charset CHARSET_UTF8 = Charset.forName("UTF-8");
public final static Charset CHARSET_UTF8 = StandardCharsets.UTF_8;
public static byte[] encode(final Object obj) {
final String json = toJson(obj, false);
......@@ -35,14 +36,17 @@ public abstract class RemotingSerializable {
}
public static <T> T decode(final byte[] data, Class<T> classOfT) {
final String json = new String(data, CHARSET_UTF8);
return fromJson(json, classOfT);
return fromJson(data, classOfT);
}
public static <T> T fromJson(String json, Class<T> classOfT) {
return JSON.parseObject(json, classOfT);
}
private static <T> T fromJson(byte[] data, Class<T> classOfT) {
return JSON.parseObject(data, classOfT);
}
public byte[] encode() {
final String json = this.toJson();
if (json != null) {
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册