提交 b143ff6d 编写于 作者: S shenhui.backend

resolve conflict

...@@ -8,10 +8,12 @@ ...@@ -8,10 +8,12 @@
It offers a variety of features: It offers a variety of features:
* Pub/Sub messaging model * Pub/Sub messaging model
* Scheduled message delivery * Financial grade transactional message
* A variety of cross language clients, such as Java, C/C++, Python, Go
* Pluggable transport protocols, such as TCP, SSL, AIO
* Inbuilt message tracing capability, also support opentracing
* Versatile big-data and streaming ecosytem integration
* Message retroactivity by time or offset * Message retroactivity by time or offset
* Log hub for streaming
* Big data integration
* Reliable FIFO and strict ordered messaging in the same queue * Reliable FIFO and strict ordered messaging in the same queue
* Efficient pull&push consumption model * Efficient pull&push consumption model
* Million-level message accumulation capacity in a single queue * Million-level message accumulation capacity in a single queue
...@@ -21,9 +23,7 @@ It offers a variety of features: ...@@ -21,9 +23,7 @@ It offers a variety of features:
* Various message filter mechanics such as SQL and Tag * Various message filter mechanics such as SQL and Tag
* Docker images for isolated testing and cloud isolated clusters * Docker images for isolated testing and cloud isolated clusters
* Feature-rich administrative dashboard for configuration, metrics and monitoring * Feature-rich administrative dashboard for configuration, metrics and monitoring
* Access control list * Authentication and authorisation
* Message trace
---------- ----------
......
...@@ -13,7 +13,7 @@ ...@@ -13,7 +13,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<artifactId>rocketmq-acl</artifactId> <artifactId>rocketmq-acl</artifactId>
<name>rocketmq-acl ${project.version}</name> <name>rocketmq-acl ${project.version}</name>
......
...@@ -13,7 +13,7 @@ ...@@ -13,7 +13,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -61,6 +61,7 @@ import org.apache.rocketmq.broker.processor.ConsumerManageProcessor; ...@@ -61,6 +61,7 @@ import org.apache.rocketmq.broker.processor.ConsumerManageProcessor;
import org.apache.rocketmq.broker.processor.EndTransactionProcessor; import org.apache.rocketmq.broker.processor.EndTransactionProcessor;
import org.apache.rocketmq.broker.processor.PullMessageProcessor; import org.apache.rocketmq.broker.processor.PullMessageProcessor;
import org.apache.rocketmq.broker.processor.QueryMessageProcessor; import org.apache.rocketmq.broker.processor.QueryMessageProcessor;
import org.apache.rocketmq.broker.processor.ReplyMessageProcessor;
import org.apache.rocketmq.broker.processor.SendMessageProcessor; import org.apache.rocketmq.broker.processor.SendMessageProcessor;
import org.apache.rocketmq.broker.slave.SlaveSynchronize; import org.apache.rocketmq.broker.slave.SlaveSynchronize;
import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager;
...@@ -132,6 +133,7 @@ public class BrokerController { ...@@ -132,6 +133,7 @@ public class BrokerController {
private final SlaveSynchronize slaveSynchronize; private final SlaveSynchronize slaveSynchronize;
private final BlockingQueue<Runnable> sendThreadPoolQueue; private final BlockingQueue<Runnable> sendThreadPoolQueue;
private final BlockingQueue<Runnable> pullThreadPoolQueue; private final BlockingQueue<Runnable> pullThreadPoolQueue;
private final BlockingQueue<Runnable> replyThreadPoolQueue;
private final BlockingQueue<Runnable> queryThreadPoolQueue; private final BlockingQueue<Runnable> queryThreadPoolQueue;
private final BlockingQueue<Runnable> clientManagerThreadPoolQueue; private final BlockingQueue<Runnable> clientManagerThreadPoolQueue;
private final BlockingQueue<Runnable> heartbeatThreadPoolQueue; private final BlockingQueue<Runnable> heartbeatThreadPoolQueue;
...@@ -147,6 +149,7 @@ public class BrokerController { ...@@ -147,6 +149,7 @@ public class BrokerController {
private TopicConfigManager topicConfigManager; private TopicConfigManager topicConfigManager;
private ExecutorService sendMessageExecutor; private ExecutorService sendMessageExecutor;
private ExecutorService pullMessageExecutor; private ExecutorService pullMessageExecutor;
private ExecutorService replyMessageExecutor;
private ExecutorService queryMessageExecutor; private ExecutorService queryMessageExecutor;
private ExecutorService adminBrokerExecutor; private ExecutorService adminBrokerExecutor;
private ExecutorService clientManageExecutor; private ExecutorService clientManageExecutor;
...@@ -194,6 +197,7 @@ public class BrokerController { ...@@ -194,6 +197,7 @@ public class BrokerController {
this.sendThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getSendThreadPoolQueueCapacity()); this.sendThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getSendThreadPoolQueueCapacity());
this.pullThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getPullThreadPoolQueueCapacity()); this.pullThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getPullThreadPoolQueueCapacity());
this.replyThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getReplyThreadPoolQueueCapacity());
this.queryThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getQueryThreadPoolQueueCapacity()); this.queryThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getQueryThreadPoolQueueCapacity());
this.clientManagerThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getClientManagerThreadPoolQueueCapacity()); this.clientManagerThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getClientManagerThreadPoolQueueCapacity());
this.consumerManagerThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getConsumerManagerThreadPoolQueueCapacity()); this.consumerManagerThreadPoolQueue = new LinkedBlockingQueue<Runnable>(this.brokerConfig.getConsumerManagerThreadPoolQueueCapacity());
...@@ -277,6 +281,14 @@ public class BrokerController { ...@@ -277,6 +281,14 @@ public class BrokerController {
this.pullThreadPoolQueue, this.pullThreadPoolQueue,
new ThreadFactoryImpl("PullMessageThread_")); new ThreadFactoryImpl("PullMessageThread_"));
this.replyMessageExecutor = new BrokerFixedThreadPoolExecutor(
this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
1000 * 60,
TimeUnit.MILLISECONDS,
this.replyThreadPoolQueue,
new ThreadFactoryImpl("ProcessReplyMessageThread_"));
this.queryMessageExecutor = new BrokerFixedThreadPoolExecutor( this.queryMessageExecutor = new BrokerFixedThreadPoolExecutor(
this.brokerConfig.getQueryMessageThreadPoolNums(), this.brokerConfig.getQueryMessageThreadPoolNums(),
this.brokerConfig.getQueryMessageThreadPoolNums(), this.brokerConfig.getQueryMessageThreadPoolNums(),
...@@ -553,6 +565,17 @@ public class BrokerController { ...@@ -553,6 +565,17 @@ public class BrokerController {
this.remotingServer.registerProcessor(RequestCode.PULL_MESSAGE, this.pullMessageProcessor, this.pullMessageExecutor); this.remotingServer.registerProcessor(RequestCode.PULL_MESSAGE, this.pullMessageProcessor, this.pullMessageExecutor);
this.pullMessageProcessor.registerConsumeMessageHook(consumeMessageHookList); this.pullMessageProcessor.registerConsumeMessageHook(consumeMessageHookList);
/**
* ReplyMessageProcessor
*/
ReplyMessageProcessor replyMessageProcessor = new ReplyMessageProcessor(this);
replyMessageProcessor.registerSendMessageHook(sendMessageHookList);
this.remotingServer.registerProcessor(RequestCode.SEND_REPLY_MESSAGE, replyMessageProcessor, replyMessageExecutor);
this.remotingServer.registerProcessor(RequestCode.SEND_REPLY_MESSAGE_V2, replyMessageProcessor, replyMessageExecutor);
this.fastRemotingServer.registerProcessor(RequestCode.SEND_REPLY_MESSAGE, replyMessageProcessor, replyMessageExecutor);
this.fastRemotingServer.registerProcessor(RequestCode.SEND_REPLY_MESSAGE_V2, replyMessageProcessor, replyMessageExecutor);
/** /**
* QueryMessageProcessor * QueryMessageProcessor
*/ */
...@@ -763,6 +786,10 @@ public class BrokerController { ...@@ -763,6 +786,10 @@ public class BrokerController {
this.pullMessageExecutor.shutdown(); this.pullMessageExecutor.shutdown();
} }
if (this.replyMessageExecutor != null) {
this.replyMessageExecutor.shutdown();
}
if (this.adminBrokerExecutor != null) { if (this.adminBrokerExecutor != null) {
this.adminBrokerExecutor.shutdown(); this.adminBrokerExecutor.shutdown();
} }
...@@ -857,12 +884,9 @@ public class BrokerController { ...@@ -857,12 +884,9 @@ public class BrokerController {
if (!messageStoreConfig.isEnableDLegerCommitLog()) { if (!messageStoreConfig.isEnableDLegerCommitLog()) {
startProcessorByHa(messageStoreConfig.getBrokerRole()); startProcessorByHa(messageStoreConfig.getBrokerRole());
handleSlaveSynchronize(messageStoreConfig.getBrokerRole()); handleSlaveSynchronize(messageStoreConfig.getBrokerRole());
this.registerBrokerAll(true, false, true);
} }
this.registerBrokerAll(true, false, true);
this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
@Override @Override
......
...@@ -178,6 +178,10 @@ public class BrokerStartup { ...@@ -178,6 +178,10 @@ public class BrokerStartup {
break; break;
} }
if (messageStoreConfig.isEnableDLegerCommitLog()) {
brokerConfig.setBrokerId(-1);
}
messageStoreConfig.setHaListenPort(nettyServerConfig.getListenPort() + 1); messageStoreConfig.setHaListenPort(nettyServerConfig.getListenPort() + 1);
LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory(); LoggerContext lc = (LoggerContext) LoggerFactory.getILoggerFactory();
JoranConfigurator configurator = new JoranConfigurator(); JoranConfigurator configurator = new JoranConfigurator();
......
...@@ -17,17 +17,16 @@ ...@@ -17,17 +17,16 @@
package org.apache.rocketmq.broker.client; package org.apache.rocketmq.broker.client;
import io.netty.channel.Channel; import io.netty.channel.Channel;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import org.apache.rocketmq.broker.util.PositiveAtomicCounter; import org.apache.rocketmq.broker.util.PositiveAtomicCounter;
import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLogger;
...@@ -43,7 +42,9 @@ public class ProducerManager { ...@@ -43,7 +42,9 @@ public class ProducerManager {
private final Lock groupChannelLock = new ReentrantLock(); private final Lock groupChannelLock = new ReentrantLock();
private final HashMap<String /* group name */, HashMap<Channel, ClientChannelInfo>> groupChannelTable = private final HashMap<String /* group name */, HashMap<Channel, ClientChannelInfo>> groupChannelTable =
new HashMap<String, HashMap<Channel, ClientChannelInfo>>(); new HashMap<String, HashMap<Channel, ClientChannelInfo>>();
private final ConcurrentHashMap<String, Channel> clientChannelTable = new ConcurrentHashMap<>();
private PositiveAtomicCounter positiveAtomicCounter = new PositiveAtomicCounter(); private PositiveAtomicCounter positiveAtomicCounter = new PositiveAtomicCounter();
public ProducerManager() { public ProducerManager() {
} }
...@@ -53,7 +54,15 @@ public class ProducerManager { ...@@ -53,7 +54,15 @@ public class ProducerManager {
try { try {
if (this.groupChannelLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) { if (this.groupChannelLock.tryLock(LOCK_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) {
try { try {
newGroupChannelTable.putAll(groupChannelTable); Iterator<Map.Entry<String, HashMap<Channel, ClientChannelInfo>>> iter = groupChannelTable.entrySet().iterator();
while (iter.hasNext()) {
Map.Entry<String, HashMap<Channel, ClientChannelInfo>> entry = iter.next();
String key = entry.getKey();
HashMap<Channel, ClientChannelInfo> val = entry.getValue();
HashMap<Channel, ClientChannelInfo> tmp = new HashMap<Channel, ClientChannelInfo>();
tmp.putAll(val);
newGroupChannelTable.put(key, tmp);
}
} finally { } finally {
groupChannelLock.unlock(); groupChannelLock.unlock();
} }
...@@ -82,6 +91,7 @@ public class ProducerManager { ...@@ -82,6 +91,7 @@ public class ProducerManager {
long diff = System.currentTimeMillis() - info.getLastUpdateTimestamp(); long diff = System.currentTimeMillis() - info.getLastUpdateTimestamp();
if (diff > CHANNEL_EXPIRED_TIMEOUT) { if (diff > CHANNEL_EXPIRED_TIMEOUT) {
it.remove(); it.remove();
clientChannelTable.remove(info.getClientId());
log.warn( log.warn(
"SCAN: remove expired channel[{}] from ProducerManager groupChannelTable, producer group name: {}", "SCAN: remove expired channel[{}] from ProducerManager groupChannelTable, producer group name: {}",
RemotingHelper.parseChannelRemoteAddr(info.getChannel()), group); RemotingHelper.parseChannelRemoteAddr(info.getChannel()), group);
...@@ -113,6 +123,7 @@ public class ProducerManager { ...@@ -113,6 +123,7 @@ public class ProducerManager {
final ClientChannelInfo clientChannelInfo = final ClientChannelInfo clientChannelInfo =
clientChannelInfoTable.remove(channel); clientChannelInfoTable.remove(channel);
if (clientChannelInfo != null) { if (clientChannelInfo != null) {
clientChannelTable.remove(clientChannelInfo.getClientId());
log.info( log.info(
"NETTY EVENT: remove channel[{}][{}] from ProducerManager groupChannelTable, producer group: {}", "NETTY EVENT: remove channel[{}][{}] from ProducerManager groupChannelTable, producer group: {}",
clientChannelInfo.toString(), remoteAddr, group); clientChannelInfo.toString(), remoteAddr, group);
...@@ -146,6 +157,7 @@ public class ProducerManager { ...@@ -146,6 +157,7 @@ public class ProducerManager {
clientChannelInfoFound = channelTable.get(clientChannelInfo.getChannel()); clientChannelInfoFound = channelTable.get(clientChannelInfo.getChannel());
if (null == clientChannelInfoFound) { if (null == clientChannelInfoFound) {
channelTable.put(clientChannelInfo.getChannel(), clientChannelInfo); channelTable.put(clientChannelInfo.getChannel(), clientChannelInfo);
clientChannelTable.put(clientChannelInfo.getClientId(), clientChannelInfo.getChannel());
log.info("new producer connected, group: {} channel: {}", group, log.info("new producer connected, group: {} channel: {}", group,
clientChannelInfo.toString()); clientChannelInfo.toString());
} }
...@@ -171,6 +183,7 @@ public class ProducerManager { ...@@ -171,6 +183,7 @@ public class ProducerManager {
HashMap<Channel, ClientChannelInfo> channelTable = this.groupChannelTable.get(group); HashMap<Channel, ClientChannelInfo> channelTable = this.groupChannelTable.get(group);
if (null != channelTable && !channelTable.isEmpty()) { if (null != channelTable && !channelTable.isEmpty()) {
ClientChannelInfo old = channelTable.remove(clientChannelInfo.getChannel()); ClientChannelInfo old = channelTable.remove(clientChannelInfo.getChannel());
clientChannelTable.remove(clientChannelInfo.getClientId());
if (old != null) { if (old != null) {
log.info("unregister a producer[{}] from groupChannelTable {}", group, log.info("unregister a producer[{}] from groupChannelTable {}", group,
clientChannelInfo.toString()); clientChannelInfo.toString());
...@@ -223,4 +236,8 @@ public class ProducerManager { ...@@ -223,4 +236,8 @@ public class ProducerManager {
} }
return null; return null;
} }
public Channel findChannel(String clientId) {
return clientChannelTable.get(clientId);
}
} }
...@@ -56,7 +56,7 @@ public class EndTransactionProcessor extends AsyncNettyRequestProcessor implemen ...@@ -56,7 +56,7 @@ public class EndTransactionProcessor extends AsyncNettyRequestProcessor implemen
final RemotingCommand response = RemotingCommand.createResponseCommand(null); final RemotingCommand response = RemotingCommand.createResponseCommand(null);
final EndTransactionRequestHeader requestHeader = final EndTransactionRequestHeader requestHeader =
(EndTransactionRequestHeader)request.decodeCommandCustomHeader(EndTransactionRequestHeader.class); (EndTransactionRequestHeader)request.decodeCommandCustomHeader(EndTransactionRequestHeader.class);
LOGGER.info("Transaction request:{}", requestHeader); LOGGER.debug("Transaction request:{}", requestHeader);
if (BrokerRole.SLAVE == brokerController.getMessageStoreConfig().getBrokerRole()) { if (BrokerRole.SLAVE == brokerController.getMessageStoreConfig().getBrokerRole()) {
response.setCode(ResponseCode.SLAVE_NOT_AVAILABLE); response.setCode(ResponseCode.SLAVE_NOT_AVAILABLE);
LOGGER.warn("Message store is slave mode, so end transaction is forbidden. "); LOGGER.warn("Message store is slave mode, so end transaction is forbidden. ");
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.broker.processor;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import org.apache.rocketmq.broker.BrokerController;
import org.apache.rocketmq.broker.mqtrace.SendMessageContext;
import org.apache.rocketmq.common.TopicConfig;
import org.apache.rocketmq.common.UtilAll;
import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageAccessor;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageDecoder;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.protocol.RequestCode;
import org.apache.rocketmq.common.protocol.ResponseCode;
import org.apache.rocketmq.common.protocol.header.ReplyMessageRequestHeader;
import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeaderV2;
import org.apache.rocketmq.common.protocol.header.SendMessageResponseHeader;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.logging.InternalLoggerFactory;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
import org.apache.rocketmq.remoting.exception.RemotingException;
import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
import org.apache.rocketmq.remoting.protocol.RemotingCommand;
import org.apache.rocketmq.store.MessageExtBrokerInner;
import org.apache.rocketmq.store.PutMessageResult;
import org.apache.rocketmq.store.stats.BrokerStatsManager;
public class ReplyMessageProcessor extends AbstractSendMessageProcessor implements NettyRequestProcessor {
private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
public ReplyMessageProcessor(final BrokerController brokerController) {
super(brokerController);
}
@Override
public RemotingCommand processRequest(ChannelHandlerContext ctx,
RemotingCommand request) throws RemotingCommandException {
SendMessageContext mqtraceContext = null;
SendMessageRequestHeader requestHeader = parseRequestHeader(request);
if (requestHeader == null) {
return null;
}
mqtraceContext = buildMsgContext(ctx, requestHeader);
this.executeSendMessageHookBefore(ctx, request, mqtraceContext);
RemotingCommand response = this.processReplyMessageRequest(ctx, request, mqtraceContext, requestHeader);
this.executeSendMessageHookAfter(response, mqtraceContext);
return response;
}
@Override
protected SendMessageRequestHeader parseRequestHeader(RemotingCommand request) throws RemotingCommandException {
SendMessageRequestHeaderV2 requestHeaderV2 = null;
SendMessageRequestHeader requestHeader = null;
switch (request.getCode()) {
case RequestCode.SEND_REPLY_MESSAGE_V2:
requestHeaderV2 =
(SendMessageRequestHeaderV2) request
.decodeCommandCustomHeader(SendMessageRequestHeaderV2.class);
case RequestCode.SEND_REPLY_MESSAGE:
if (null == requestHeaderV2) {
requestHeader =
(SendMessageRequestHeader) request
.decodeCommandCustomHeader(SendMessageRequestHeader.class);
} else {
requestHeader = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV1(requestHeaderV2);
}
default:
break;
}
return requestHeader;
}
private RemotingCommand processReplyMessageRequest(final ChannelHandlerContext ctx,
final RemotingCommand request,
final SendMessageContext sendMessageContext,
final SendMessageRequestHeader requestHeader) {
final RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class);
final SendMessageResponseHeader responseHeader = (SendMessageResponseHeader) response.readCustomHeader();
response.setOpaque(request.getOpaque());
response.addExtField(MessageConst.PROPERTY_MSG_REGION, this.brokerController.getBrokerConfig().getRegionId());
response.addExtField(MessageConst.PROPERTY_TRACE_SWITCH, String.valueOf(this.brokerController.getBrokerConfig().isTraceOn()));
log.debug("receive SendReplyMessage request command, {}", request);
final long startTimstamp = this.brokerController.getBrokerConfig().getStartAcceptSendRequestTimeStamp();
if (this.brokerController.getMessageStore().now() < startTimstamp) {
response.setCode(ResponseCode.SYSTEM_ERROR);
response.setRemark(String.format("broker unable to service, until %s", UtilAll.timeMillisToHumanString2(startTimstamp)));
return response;
}
response.setCode(-1);
super.msgCheck(ctx, requestHeader, response);
if (response.getCode() != -1) {
return response;
}
final byte[] body = request.getBody();
int queueIdInt = requestHeader.getQueueId();
TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic());
if (queueIdInt < 0) {
queueIdInt = Math.abs(this.random.nextInt() % 99999999) % topicConfig.getWriteQueueNums();
}
MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
msgInner.setTopic(requestHeader.getTopic());
msgInner.setQueueId(queueIdInt);
msgInner.setBody(body);
msgInner.setFlag(requestHeader.getFlag());
MessageAccessor.setProperties(msgInner, MessageDecoder.string2messageProperties(requestHeader.getProperties()));
msgInner.setPropertiesString(requestHeader.getProperties());
msgInner.setBornTimestamp(requestHeader.getBornTimestamp());
msgInner.setBornHost(ctx.channel().remoteAddress());
msgInner.setStoreHost(this.getStoreHost());
msgInner.setReconsumeTimes(requestHeader.getReconsumeTimes() == null ? 0 : requestHeader.getReconsumeTimes());
PushReplyResult pushReplyResult = this.pushReplyMessage(ctx, requestHeader, msgInner);
this.handlePushReplyResult(pushReplyResult, response, responseHeader, queueIdInt);
if (this.brokerController.getBrokerConfig().isStoreReplyMessageEnable()) {
PutMessageResult putMessageResult = this.brokerController.getMessageStore().putMessage(msgInner);
this.handlePutMessageResult(putMessageResult, request, msgInner, responseHeader, sendMessageContext, queueIdInt);
}
return response;
}
private PushReplyResult pushReplyMessage(final ChannelHandlerContext ctx,
final SendMessageRequestHeader requestHeader,
final Message msg) {
ReplyMessageRequestHeader replyMessageRequestHeader = new ReplyMessageRequestHeader();
replyMessageRequestHeader.setBornHost(ctx.channel().remoteAddress().toString());
replyMessageRequestHeader.setStoreHost(this.getStoreHost().toString());
replyMessageRequestHeader.setStoreTimestamp(System.currentTimeMillis());
replyMessageRequestHeader.setProducerGroup(requestHeader.getProducerGroup());
replyMessageRequestHeader.setTopic(requestHeader.getTopic());
replyMessageRequestHeader.setDefaultTopic(requestHeader.getDefaultTopic());
replyMessageRequestHeader.setDefaultTopicQueueNums(requestHeader.getDefaultTopicQueueNums());
replyMessageRequestHeader.setQueueId(requestHeader.getQueueId());
replyMessageRequestHeader.setSysFlag(requestHeader.getSysFlag());
replyMessageRequestHeader.setBornTimestamp(requestHeader.getBornTimestamp());
replyMessageRequestHeader.setFlag(requestHeader.getFlag());
replyMessageRequestHeader.setProperties(requestHeader.getProperties());
replyMessageRequestHeader.setReconsumeTimes(requestHeader.getReconsumeTimes());
replyMessageRequestHeader.setUnitMode(requestHeader.isUnitMode());
RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PUSH_REPLY_MESSAGE_TO_CLIENT, replyMessageRequestHeader);
request.setBody(msg.getBody());
String senderId = msg.getProperties().get(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT);
PushReplyResult pushReplyResult = new PushReplyResult(false);
if (senderId != null) {
Channel channel = this.brokerController.getProducerManager().findChannel(senderId);
if (channel != null) {
msg.getProperties().put(MessageConst.PROPERTY_PUSH_REPLY_TIME, String.valueOf(System.currentTimeMillis()));
replyMessageRequestHeader.setProperties(MessageDecoder.messageProperties2String(msg.getProperties()));
try {
RemotingCommand pushResponse = this.brokerController.getBroker2Client().callClient(channel, request);
assert pushResponse != null;
switch (pushResponse.getCode()) {
case ResponseCode.SUCCESS: {
pushReplyResult.setPushOk(true);
break;
}
default: {
pushReplyResult.setPushOk(false);
pushReplyResult.setRemark("push reply message to " + senderId + "fail.");
log.warn("push reply message to <{}> return fail, response remark: {}", senderId, pushResponse.getRemark());
}
}
} catch (RemotingException | InterruptedException e) {
pushReplyResult.setPushOk(false);
pushReplyResult.setRemark("push reply message to " + senderId + "fail.");
log.warn("push reply message to <{}> fail. {}", senderId, channel, e);
}
} else {
pushReplyResult.setPushOk(false);
pushReplyResult.setRemark("push reply message fail, channel of <" + senderId + "> not found.");
log.warn(pushReplyResult.getRemark());
}
} else {
log.warn(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT + " is null, can not reply message");
pushReplyResult.setPushOk(false);
pushReplyResult.setRemark("reply message properties[" + MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT + "] is null");
}
return pushReplyResult;
}
private void handlePushReplyResult(PushReplyResult pushReplyResult, final RemotingCommand response,
final SendMessageResponseHeader responseHeader, int queueIdInt) {
if (!pushReplyResult.isPushOk()) {
response.setCode(ResponseCode.SYSTEM_ERROR);
response.setRemark(pushReplyResult.getRemark());
} else {
response.setCode(ResponseCode.SUCCESS);
response.setRemark(null);
//set to zore to avoid client decoding exception
responseHeader.setMsgId("0");
responseHeader.setQueueId(queueIdInt);
responseHeader.setQueueOffset(0L);
}
}
private void handlePutMessageResult(PutMessageResult putMessageResult,
final RemotingCommand request, final MessageExt msg,
final SendMessageResponseHeader responseHeader, SendMessageContext sendMessageContext,
int queueIdInt) {
if (putMessageResult == null) {
log.warn("process reply message, store putMessage return null");
return;
}
boolean putOk = false;
switch (putMessageResult.getPutMessageStatus()) {
// Success
case PUT_OK:
case FLUSH_DISK_TIMEOUT:
case FLUSH_SLAVE_TIMEOUT:
case SLAVE_NOT_AVAILABLE:
putOk = true;
break;
// Failed
case CREATE_MAPEDFILE_FAILED:
log.info("create mapped file failed, server is busy or broken.");
break;
case MESSAGE_ILLEGAL:
log.info(
"the message is illegal, maybe msg properties length limit 32k.");
break;
case PROPERTIES_SIZE_EXCEEDED:
log.info(
"the message is illegal, maybe msg body or properties length not matched. msg body length limit 128k.");
break;
case SERVICE_NOT_AVAILABLE:
log.info(
"service not available now, maybe disk full, maybe your broker machine memory too small.");
break;
case OS_PAGECACHE_BUSY:
log.info("[PC_SYNCHRONIZED]broker busy, start flow control for a while");
break;
case UNKNOWN_ERROR:
log.info("UNKNOWN_ERROR");
break;
default:
log.info("UNKNOWN_ERROR DEFAULT");
break;
}
String owner = request.getExtFields().get(BrokerStatsManager.COMMERCIAL_OWNER);
if (putOk) {
this.brokerController.getBrokerStatsManager().incTopicPutNums(msg.getTopic(), putMessageResult.getAppendMessageResult().getMsgNum(), 1);
this.brokerController.getBrokerStatsManager().incTopicPutSize(msg.getTopic(),
putMessageResult.getAppendMessageResult().getWroteBytes());
this.brokerController.getBrokerStatsManager().incBrokerPutNums(putMessageResult.getAppendMessageResult().getMsgNum());
responseHeader.setMsgId(putMessageResult.getAppendMessageResult().getMsgId());
responseHeader.setQueueId(queueIdInt);
responseHeader.setQueueOffset(putMessageResult.getAppendMessageResult().getLogicsOffset());
if (hasSendMessageHook()) {
sendMessageContext.setMsgId(responseHeader.getMsgId());
sendMessageContext.setQueueId(responseHeader.getQueueId());
sendMessageContext.setQueueOffset(responseHeader.getQueueOffset());
int commercialBaseCount = brokerController.getBrokerConfig().getCommercialBaseCount();
int wroteSize = putMessageResult.getAppendMessageResult().getWroteBytes();
int incValue = (int) Math.ceil(wroteSize / BrokerStatsManager.SIZE_PER_COUNT) * commercialBaseCount;
sendMessageContext.setCommercialSendStats(BrokerStatsManager.StatsType.SEND_SUCCESS);
sendMessageContext.setCommercialSendTimes(incValue);
sendMessageContext.setCommercialSendSize(wroteSize);
sendMessageContext.setCommercialOwner(owner);
}
} else {
if (hasSendMessageHook()) {
int wroteSize = request.getBody().length;
int incValue = (int) Math.ceil(wroteSize / BrokerStatsManager.SIZE_PER_COUNT);
sendMessageContext.setCommercialSendStats(BrokerStatsManager.StatsType.SEND_FAILURE);
sendMessageContext.setCommercialSendTimes(incValue);
sendMessageContext.setCommercialSendSize(wroteSize);
sendMessageContext.setCommercialOwner(owner);
}
}
}
class PushReplyResult {
boolean pushOk;
String remark;
public PushReplyResult(boolean pushOk) {
this.pushOk = pushOk;
remark = "";
}
public boolean isPushOk() {
return pushOk;
}
public void setPushOk(boolean pushOk) {
this.pushOk = pushOk;
}
public String getRemark() {
return remark;
}
public void setRemark(String remark) {
this.remark = remark;
}
}
}
...@@ -356,6 +356,81 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement ...@@ -356,6 +356,81 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
return true; return true;
} }
private RemotingCommand sendMessage(final ChannelHandlerContext ctx,
final RemotingCommand request,
final SendMessageContext sendMessageContext,
final SendMessageRequestHeader requestHeader) throws RemotingCommandException {
final RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class);
final SendMessageResponseHeader responseHeader = (SendMessageResponseHeader)response.readCustomHeader();
response.setOpaque(request.getOpaque());
response.addExtField(MessageConst.PROPERTY_MSG_REGION, this.brokerController.getBrokerConfig().getRegionId());
response.addExtField(MessageConst.PROPERTY_TRACE_SWITCH, String.valueOf(this.brokerController.getBrokerConfig().isTraceOn()));
log.debug("receive SendMessage request command, {}", request);
final long startTimstamp = this.brokerController.getBrokerConfig().getStartAcceptSendRequestTimeStamp();
if (this.brokerController.getMessageStore().now() < startTimstamp) {
response.setCode(ResponseCode.SYSTEM_ERROR);
response.setRemark(String.format("broker unable to service, until %s", UtilAll.timeMillisToHumanString2(startTimstamp)));
return response;
}
response.setCode(-1);
super.msgCheck(ctx, requestHeader, response);
if (response.getCode() != -1) {
return response;
}
final byte[] body = request.getBody();
int queueIdInt = requestHeader.getQueueId();
TopicConfig topicConfig = this.brokerController.getTopicConfigManager().selectTopicConfig(requestHeader.getTopic());
if (queueIdInt < 0) {
queueIdInt = Math.abs(this.random.nextInt() % 99999999) % topicConfig.getWriteQueueNums();
}
MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
msgInner.setTopic(requestHeader.getTopic());
msgInner.setQueueId(queueIdInt);
if (!handleRetryAndDLQ(requestHeader, response, request, msgInner, topicConfig)) {
return response;
}
msgInner.setBody(body);
msgInner.setFlag(requestHeader.getFlag());
MessageAccessor.setProperties(msgInner, MessageDecoder.string2messageProperties(requestHeader.getProperties()));
msgInner.setBornTimestamp(requestHeader.getBornTimestamp());
msgInner.setBornHost(ctx.channel().remoteAddress());
msgInner.setStoreHost(this.getStoreHost());
msgInner.setReconsumeTimes(requestHeader.getReconsumeTimes() == null ? 0 : requestHeader.getReconsumeTimes());
String clusterName = this.brokerController.getBrokerConfig().getBrokerClusterName();
MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_CLUSTER, clusterName);
msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
PutMessageResult putMessageResult = null;
Map<String, String> oriProps = MessageDecoder.string2messageProperties(requestHeader.getProperties());
String traFlag = oriProps.get(MessageConst.PROPERTY_TRANSACTION_PREPARED);
if (traFlag != null && Boolean.parseBoolean(traFlag)) {
if (this.brokerController.getBrokerConfig().isRejectTransactionMessage()) {
response.setCode(ResponseCode.NO_PERMISSION);
response.setRemark(
"the broker[" + this.brokerController.getBrokerConfig().getBrokerIP1()
+ "] sending transaction message is forbidden");
return response;
}
putMessageResult = this.brokerController.getTransactionalMessageService().prepareMessage(msgInner);
} else {
putMessageResult = this.brokerController.getMessageStore().putMessage(msgInner);
}
return handlePutMessageResult(putMessageResult, response, request, msgInner, responseHeader, sendMessageContext, ctx, queueIdInt);
}
private RemotingCommand handlePutMessageResult(PutMessageResult putMessageResult, RemotingCommand response, private RemotingCommand handlePutMessageResult(PutMessageResult putMessageResult, RemotingCommand response,
RemotingCommand request, MessageExt msg, RemotingCommand request, MessageExt msg,
SendMessageResponseHeader responseHeader, SendMessageContext sendMessageContext, ChannelHandlerContext ctx, SendMessageResponseHeader responseHeader, SendMessageContext sendMessageContext, ChannelHandlerContext ctx,
...@@ -501,6 +576,8 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement ...@@ -501,6 +576,8 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
messageExtBatch.setBornHost(ctx.channel().remoteAddress()); messageExtBatch.setBornHost(ctx.channel().remoteAddress());
messageExtBatch.setStoreHost(this.getStoreHost()); messageExtBatch.setStoreHost(this.getStoreHost());
messageExtBatch.setReconsumeTimes(requestHeader.getReconsumeTimes() == null ? 0 : requestHeader.getReconsumeTimes()); messageExtBatch.setReconsumeTimes(requestHeader.getReconsumeTimes() == null ? 0 : requestHeader.getReconsumeTimes());
String clusterName = this.brokerController.getBrokerConfig().getBrokerClusterName();
MessageAccessor.putProperty(messageExtBatch, MessageConst.PROPERTY_CLUSTER, clusterName);
CompletableFuture<PutMessageResult> putMessageResult = this.brokerController.getMessageStore().asyncPutMessages(messageExtBatch); CompletableFuture<PutMessageResult> putMessageResult = this.brokerController.getMessageStore().asyncPutMessages(messageExtBatch);
return handlePutMessageResultFuture(putMessageResult, response, request, messageExtBatch, responseHeader, mqtraceContext, ctx, queueIdInt); return handlePutMessageResultFuture(putMessageResult, response, request, messageExtBatch, responseHeader, mqtraceContext, ctx, queueIdInt);
......
...@@ -34,11 +34,11 @@ import org.apache.rocketmq.common.MixAll; ...@@ -34,11 +34,11 @@ import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.TopicConfig;
import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.LoggerName;
import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.constant.PermName;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.logging.InternalLoggerFactory;
import org.apache.rocketmq.common.protocol.body.KVTable; import org.apache.rocketmq.common.protocol.body.KVTable;
import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.common.protocol.body.TopicConfigSerializeWrapper;
import org.apache.rocketmq.common.sysflag.TopicSysFlag; import org.apache.rocketmq.common.sysflag.TopicSysFlag;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.logging.InternalLoggerFactory;
public class TopicConfigManager extends ConfigManager { public class TopicConfigManager extends ConfigManager {
private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
...@@ -134,6 +134,14 @@ public class TopicConfigManager extends ConfigManager { ...@@ -134,6 +134,14 @@ public class TopicConfigManager extends ConfigManager {
this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig); this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig);
} }
} }
{
String topic = this.brokerController.getBrokerConfig().getBrokerClusterName() + "_" + MixAll.REPLY_TOPIC_POSTFIX;
TopicConfig topicConfig = new TopicConfig(topic);
this.systemTopicList.add(topic);
topicConfig.setReadQueueNums(1);
topicConfig.setWriteQueueNums(1);
this.topicConfigTable.put(topicConfig.getTopicName(), topicConfig);
}
} }
public boolean isSystemTopic(final String topic) { public boolean isSystemTopic(final String topic) {
......
...@@ -30,6 +30,7 @@ import java.util.concurrent.ArrayBlockingQueue; ...@@ -30,6 +30,7 @@ import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
public abstract class AbstractTransactionalMessageCheckListener { public abstract class AbstractTransactionalMessageCheckListener {
...@@ -48,7 +49,7 @@ public abstract class AbstractTransactionalMessageCheckListener { ...@@ -48,7 +49,7 @@ public abstract class AbstractTransactionalMessageCheckListener {
thread.setName("Transaction-msg-check-thread"); thread.setName("Transaction-msg-check-thread");
return thread; return thread;
} }
}); }, new CallerRunsPolicy());
public AbstractTransactionalMessageCheckListener() { public AbstractTransactionalMessageCheckListener() {
} }
......
...@@ -132,6 +132,9 @@ public class TransactionalMessageBridge { ...@@ -132,6 +132,9 @@ public class TransactionalMessageBridge {
this.brokerController.getBrokerStatsManager().incGroupGetSize(group, topic, this.brokerController.getBrokerStatsManager().incGroupGetSize(group, topic,
getMessageResult.getBufferTotalSize()); getMessageResult.getBufferTotalSize());
this.brokerController.getBrokerStatsManager().incBrokerGetNums(getMessageResult.getMessageCount()); this.brokerController.getBrokerStatsManager().incBrokerGetNums(getMessageResult.getMessageCount());
if (foundList == null || foundList.size() == 0) {
break;
}
this.brokerController.getBrokerStatsManager().recordDiskFallBehindTime(group, topic, queueId, this.brokerController.getBrokerStatsManager().recordDiskFallBehindTime(group, topic, queueId,
this.brokerController.getMessageStore().now() - foundList.get(foundList.size() - 1) this.brokerController.getMessageStore().now() - foundList.get(foundList.size() - 1)
.getStoreTimestamp()); .getStoreTimestamp());
...@@ -142,6 +145,7 @@ public class TransactionalMessageBridge { ...@@ -142,6 +145,7 @@ public class TransactionalMessageBridge {
getMessageResult.getStatus(), topic, group, offset); getMessageResult.getStatus(), topic, group, offset);
break; break;
case NO_MESSAGE_IN_QUEUE: case NO_MESSAGE_IN_QUEUE:
case OFFSET_OVERFLOW_ONE:
pullStatus = PullStatus.NO_NEW_MSG; pullStatus = PullStatus.NO_NEW_MSG;
LOGGER.warn("No new message. GetMessageStatus={}, topic={}, groupId={}, requestOffset={}", LOGGER.warn("No new message. GetMessageStatus={}, topic={}, groupId={}, requestOffset={}",
getMessageResult.getStatus(), topic, group, offset); getMessageResult.getStatus(), topic, group, offset);
...@@ -150,7 +154,6 @@ public class TransactionalMessageBridge { ...@@ -150,7 +154,6 @@ public class TransactionalMessageBridge {
case NO_MATCHED_LOGIC_QUEUE: case NO_MATCHED_LOGIC_QUEUE:
case OFFSET_FOUND_NULL: case OFFSET_FOUND_NULL:
case OFFSET_OVERFLOW_BADLY: case OFFSET_OVERFLOW_BADLY:
case OFFSET_OVERFLOW_ONE:
case OFFSET_TOO_SMALL: case OFFSET_TOO_SMALL:
pullStatus = PullStatus.OFFSET_ILLEGAL; pullStatus = PullStatus.OFFSET_ILLEGAL;
LOGGER.warn("Offset illegal. GetMessageStatus={}, topic={}, groupId={}, requestOffset={}", LOGGER.warn("Offset illegal. GetMessageStatus={}, topic={}, groupId={}, requestOffset={}",
...@@ -176,8 +179,10 @@ public class TransactionalMessageBridge { ...@@ -176,8 +179,10 @@ public class TransactionalMessageBridge {
try { try {
List<ByteBuffer> messageBufferList = getMessageResult.getMessageBufferList(); List<ByteBuffer> messageBufferList = getMessageResult.getMessageBufferList();
for (ByteBuffer bb : messageBufferList) { for (ByteBuffer bb : messageBufferList) {
MessageExt msgExt = MessageDecoder.decode(bb); MessageExt msgExt = MessageDecoder.decode(bb, true, false);
foundList.add(msgExt); if (msgExt != null) {
foundList.add(msgExt);
}
} }
} finally { } finally {
......
...@@ -165,7 +165,8 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ ...@@ -165,7 +165,8 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
} }
if (removeMap.containsKey(i)) { if (removeMap.containsKey(i)) {
log.info("Half offset {} has been committed/rolled back", i); log.info("Half offset {} has been committed/rolled back", i);
removeMap.remove(i); Long removedOpOffset = removeMap.remove(i);
doneOpOffset.add(removedOpOffset);
} else { } else {
GetResult getResult = getHalfMsg(messageQueue, i); GetResult getResult = getHalfMsg(messageQueue, i);
MessageExt msgExt = getResult.getMsg(); MessageExt msgExt = getResult.getMsg();
...@@ -229,7 +230,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ ...@@ -229,7 +230,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
listener.resolveHalfMsg(msgExt); listener.resolveHalfMsg(msgExt);
} else { } else {
pullResult = fillOpRemoveMap(removeMap, opQueue, pullResult.getNextBeginOffset(), halfOffset, doneOpOffset); pullResult = fillOpRemoveMap(removeMap, opQueue, pullResult.getNextBeginOffset(), halfOffset, doneOpOffset);
log.info("The miss offset:{} in messageQueue:{} need to get more opMsg, result is:{}", i, log.debug("The miss offset:{} in messageQueue:{} need to get more opMsg, result is:{}", i,
messageQueue, pullResult); messageQueue, pullResult);
continue; continue;
} }
...@@ -298,7 +299,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ ...@@ -298,7 +299,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
} }
for (MessageExt opMessageExt : opMsg) { for (MessageExt opMessageExt : opMsg) {
Long queueOffset = getLong(new String(opMessageExt.getBody(), TransactionalMessageUtil.charset)); Long queueOffset = getLong(new String(opMessageExt.getBody(), TransactionalMessageUtil.charset));
log.info("Topic: {} tags: {}, OpOffset: {}, HalfOffset: {}", opMessageExt.getTopic(), log.debug("Topic: {} tags: {}, OpOffset: {}, HalfOffset: {}", opMessageExt.getTopic(),
opMessageExt.getTags(), opMessageExt.getQueueOffset(), queueOffset); opMessageExt.getTags(), opMessageExt.getQueueOffset(), queueOffset);
if (TransactionalMessageUtil.REMOVETAG.equals(opMessageExt.getTags())) { if (TransactionalMessageUtil.REMOVETAG.equals(opMessageExt.getTags())) {
if (queueOffset < miniOffset) { if (queueOffset < miniOffset) {
...@@ -466,7 +467,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ ...@@ -466,7 +467,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
@Override @Override
public boolean deletePrepareMessage(MessageExt msgExt) { public boolean deletePrepareMessage(MessageExt msgExt) {
if (this.transactionalMessageBridge.putOpMessage(msgExt, TransactionalMessageUtil.REMOVETAG)) { if (this.transactionalMessageBridge.putOpMessage(msgExt, TransactionalMessageUtil.REMOVETAG)) {
log.info("Transaction op message write successfully. messageId={}, queueId={} msgExt:{}", msgExt.getMsgId(), msgExt.getQueueId(), msgExt); log.debug("Transaction op message write successfully. messageId={}, queueId={} msgExt:{}", msgExt.getMsgId(), msgExt.getQueueId(), msgExt);
return true; return true;
} else { } else {
log.error("Transaction op message write failed. messageId is {}, queueId is {}", msgExt.getMsgId(), msgExt.getQueueId()); log.error("Transaction op message write failed. messageId is {}, queueId is {}", msgExt.getMsgId(), msgExt.getQueueId());
......
...@@ -20,6 +20,7 @@ import io.netty.channel.Channel; ...@@ -20,6 +20,7 @@ import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFuture;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.util.HashMap; import java.util.HashMap;
import org.apache.rocketmq.remoting.protocol.LanguageCode;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
...@@ -42,14 +43,14 @@ public class ProducerManagerTest { ...@@ -42,14 +43,14 @@ public class ProducerManagerTest {
@Before @Before
public void init() { public void init() {
producerManager = new ProducerManager(); producerManager = new ProducerManager();
clientInfo = new ClientChannelInfo(channel); clientInfo = new ClientChannelInfo(channel, "clientId", LanguageCode.JAVA, 0);
} }
@Test @Test
public void scanNotActiveChannel() throws Exception { public void scanNotActiveChannel() throws Exception {
producerManager.registerProducer(group, clientInfo); producerManager.registerProducer(group, clientInfo);
assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNotNull(); assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNotNull();
assertThat(producerManager.findChannel("clientId")).isNotNull();
Field field = ProducerManager.class.getDeclaredField("CHANNEL_EXPIRED_TIMEOUT"); Field field = ProducerManager.class.getDeclaredField("CHANNEL_EXPIRED_TIMEOUT");
field.setAccessible(true); field.setAccessible(true);
long CHANNEL_EXPIRED_TIMEOUT = field.getLong(producerManager); long CHANNEL_EXPIRED_TIMEOUT = field.getLong(producerManager);
...@@ -57,22 +58,28 @@ public class ProducerManagerTest { ...@@ -57,22 +58,28 @@ public class ProducerManagerTest {
when(channel.close()).thenReturn(mock(ChannelFuture.class)); when(channel.close()).thenReturn(mock(ChannelFuture.class));
producerManager.scanNotActiveChannel(); producerManager.scanNotActiveChannel();
assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNull(); assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNull();
assertThat(producerManager.findChannel("clientId")).isNull();
} }
@Test @Test
public void doChannelCloseEvent() throws Exception { public void doChannelCloseEvent() throws Exception {
producerManager.registerProducer(group, clientInfo); producerManager.registerProducer(group, clientInfo);
assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNotNull(); assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNotNull();
assertThat(producerManager.findChannel("clientId")).isNotNull();
producerManager.doChannelCloseEvent("127.0.0.1", channel); producerManager.doChannelCloseEvent("127.0.0.1", channel);
assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNull(); assertThat(producerManager.getGroupChannelTable().get(group).get(channel)).isNull();
assertThat(producerManager.findChannel("clientId")).isNull();
} }
@Test @Test
public void testRegisterProducer() throws Exception { public void testRegisterProducer() throws Exception {
producerManager.registerProducer(group, clientInfo); producerManager.registerProducer(group, clientInfo);
HashMap<Channel, ClientChannelInfo> channelMap = producerManager.getGroupChannelTable().get(group); HashMap<Channel, ClientChannelInfo> channelMap = producerManager.getGroupChannelTable().get(group);
Channel channel1 = producerManager.findChannel("clientId");
assertThat(channelMap).isNotNull(); assertThat(channelMap).isNotNull();
assertThat(channel1).isNotNull();
assertThat(channelMap.get(channel)).isEqualTo(clientInfo); assertThat(channelMap.get(channel)).isEqualTo(clientInfo);
assertThat(channel1).isEqualTo(channel);
} }
@Test @Test
...@@ -81,10 +88,23 @@ public class ProducerManagerTest { ...@@ -81,10 +88,23 @@ public class ProducerManagerTest {
HashMap<Channel, ClientChannelInfo> channelMap = producerManager.getGroupChannelTable().get(group); HashMap<Channel, ClientChannelInfo> channelMap = producerManager.getGroupChannelTable().get(group);
assertThat(channelMap).isNotNull(); assertThat(channelMap).isNotNull();
assertThat(channelMap.get(channel)).isEqualTo(clientInfo); assertThat(channelMap.get(channel)).isEqualTo(clientInfo);
Channel channel1 = producerManager.findChannel("clientId");
assertThat(channel1).isNotNull();
assertThat(channel1).isEqualTo(channel);
producerManager.unregisterProducer(group, clientInfo); producerManager.unregisterProducer(group, clientInfo);
channelMap = producerManager.getGroupChannelTable().get(group); channelMap = producerManager.getGroupChannelTable().get(group);
channel1 = producerManager.findChannel("clientId");
assertThat(channelMap).isNull(); assertThat(channelMap).isNull();
assertThat(channel1).isNull();
} }
@Test
public void testGetGroupChannelTable() throws Exception {
producerManager.registerProducer(group, clientInfo);
HashMap<Channel, ClientChannelInfo> oldMap = producerManager.getGroupChannelTable().get(group);
producerManager.unregisterProducer(group, clientInfo);
assertThat(oldMap.size()).isNotEqualTo(0);
}
} }
\ 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.broker.processor;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.Map;
import org.apache.rocketmq.broker.BrokerController;
import org.apache.rocketmq.broker.client.ClientChannelInfo;
import org.apache.rocketmq.broker.client.net.Broker2Client;
import org.apache.rocketmq.broker.transaction.TransactionalMessageService;
import org.apache.rocketmq.common.BrokerConfig;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageDecoder;
import org.apache.rocketmq.common.protocol.RequestCode;
import org.apache.rocketmq.common.protocol.ResponseCode;
import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.common.protocol.header.SendMessageResponseHeader;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
import org.apache.rocketmq.remoting.netty.NettyClientConfig;
import org.apache.rocketmq.remoting.netty.NettyServerConfig;
import org.apache.rocketmq.remoting.protocol.LanguageCode;
import org.apache.rocketmq.remoting.protocol.RemotingCommand;
import org.apache.rocketmq.store.AppendMessageResult;
import org.apache.rocketmq.store.AppendMessageStatus;
import org.apache.rocketmq.store.MessageExtBrokerInner;
import org.apache.rocketmq.store.MessageStore;
import org.apache.rocketmq.store.PutMessageResult;
import org.apache.rocketmq.store.PutMessageStatus;
import org.apache.rocketmq.store.config.MessageStoreConfig;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Spy;
import org.mockito.junit.MockitoJUnitRunner;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@RunWith(MockitoJUnitRunner.class)
public class ReplyMessageProcessorTest {
private ReplyMessageProcessor replyMessageProcessor;
@Spy
private BrokerController brokerController = new BrokerController(new BrokerConfig(), new NettyServerConfig(), new NettyClientConfig(), new MessageStoreConfig());
@Mock
private ChannelHandlerContext handlerContext;
@Mock
private MessageStore messageStore;
@Mock
private Channel channel;
private String topic = "FooBar";
private String group = "FooBarGroup";
private ClientChannelInfo clientInfo;
@Mock
private Broker2Client broker2Client;
@Before
public void init() throws IllegalAccessException, NoSuchFieldException {
clientInfo = new ClientChannelInfo(channel, "127.0.0.1", LanguageCode.JAVA, 0);
brokerController.setMessageStore(messageStore);
Field field = BrokerController.class.getDeclaredField("broker2Client");
field.setAccessible(true);
field.set(brokerController, broker2Client);
when(messageStore.now()).thenReturn(System.currentTimeMillis());
Channel mockChannel = mock(Channel.class);
when(mockChannel.remoteAddress()).thenReturn(new InetSocketAddress(1024));
when(handlerContext.channel()).thenReturn(mockChannel);
replyMessageProcessor = new ReplyMessageProcessor(brokerController);
}
@Test
public void testProcessRequest_Success() throws RemotingCommandException, InterruptedException, RemotingTimeoutException, RemotingSendRequestException {
when(messageStore.putMessage(any(MessageExtBrokerInner.class))).thenReturn(new PutMessageResult(PutMessageStatus.PUT_OK, new AppendMessageResult(AppendMessageStatus.PUT_OK)));
brokerController.getProducerManager().registerProducer(group, clientInfo);
final RemotingCommand request = createSendMessageRequestHeaderCommand(RequestCode.SEND_REPLY_MESSAGE);
when(brokerController.getBroker2Client().callClient(any(Channel.class), any(RemotingCommand.class))).thenReturn(createResponse(ResponseCode.SUCCESS, request));
RemotingCommand responseToReturn = replyMessageProcessor.processRequest(handlerContext, request);
assertThat(responseToReturn.getCode()).isEqualTo(ResponseCode.SUCCESS);
assertThat(responseToReturn.getOpaque()).isEqualTo(request.getOpaque());
}
private RemotingCommand createSendMessageRequestHeaderCommand(int requestCode) {
SendMessageRequestHeader requestHeader = createSendMessageRequestHeader();
RemotingCommand request = RemotingCommand.createRequestCommand(requestCode, requestHeader);
request.setBody(new byte[] {'a'});
request.makeCustomHeaderToNet();
return request;
}
private SendMessageRequestHeader createSendMessageRequestHeader() {
SendMessageRequestHeader requestHeader = new SendMessageRequestHeader();
requestHeader.setProducerGroup(group);
requestHeader.setTopic(topic);
requestHeader.setDefaultTopic(MixAll.AUTO_CREATE_TOPIC_KEY_TOPIC);
requestHeader.setDefaultTopicQueueNums(3);
requestHeader.setQueueId(1);
requestHeader.setSysFlag(0);
requestHeader.setBornTimestamp(System.currentTimeMillis());
requestHeader.setFlag(124);
requestHeader.setReconsumeTimes(0);
Map<String, String> map = new HashMap<String, String>();
map.put(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT, "127.0.0.1");
requestHeader.setProperties(MessageDecoder.messageProperties2String(map));
return requestHeader;
}
private RemotingCommand createResponse(int code, RemotingCommand request) {
RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class);
response.setCode(code);
response.setOpaque(request.getOpaque());
return response;
}
}
\ No newline at end of file
...@@ -176,6 +176,24 @@ public class TransactionalMessageBridgeTest { ...@@ -176,6 +176,24 @@ public class TransactionalMessageBridgeTest {
assertThat(messageExt).isNotNull(); assertThat(messageExt).isNotNull();
} }
@Test
public void testGetHalfMessageStatusFound() {
when(messageStore
.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), ArgumentMatchers.nullable(MessageFilter.class)))
.thenReturn(createGetMessageResult(GetMessageStatus.FOUND));
PullResult result = transactionBridge.getHalfMessage(0, 0, 1);
assertThat(result.getPullStatus()).isEqualTo(PullStatus.FOUND);
}
@Test
public void testGetHalfMessageNull() {
when(messageStore
.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), ArgumentMatchers.nullable(MessageFilter.class)))
.thenReturn(null);
PullResult result = transactionBridge.getHalfMessage(0, 0, 1);
assertThat(result).isNull();
}
private GetMessageResult createGetMessageResult(GetMessageStatus status) { private GetMessageResult createGetMessageResult(GetMessageStatus status) {
GetMessageResult getMessageResult = new GetMessageResult(); GetMessageResult getMessageResult = new GetMessageResult();
getMessageResult.setStatus(status); getMessageResult.setStatus(status);
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -53,6 +53,7 @@ public class ClientConfig { ...@@ -53,6 +53,7 @@ public class ClientConfig {
* Offset persistent interval for consumer * Offset persistent interval for consumer
*/ */
private int persistConsumerOffsetInterval = 1000 * 5; private int persistConsumerOffsetInterval = 1000 * 5;
private long pullTimeDelayMillsWhenException = 1000;
private boolean unitMode = false; private boolean unitMode = false;
private String unitName; private String unitName;
private boolean vipChannelEnabled = Boolean.parseBoolean(System.getProperty(SEND_MESSAGE_WITH_VIP_CHANNEL_PROPERTY, "false")); private boolean vipChannelEnabled = Boolean.parseBoolean(System.getProperty(SEND_MESSAGE_WITH_VIP_CHANNEL_PROPERTY, "false"));
...@@ -148,6 +149,7 @@ public class ClientConfig { ...@@ -148,6 +149,7 @@ public class ClientConfig {
this.pollNameServerInterval = cc.pollNameServerInterval; this.pollNameServerInterval = cc.pollNameServerInterval;
this.heartbeatBrokerInterval = cc.heartbeatBrokerInterval; this.heartbeatBrokerInterval = cc.heartbeatBrokerInterval;
this.persistConsumerOffsetInterval = cc.persistConsumerOffsetInterval; this.persistConsumerOffsetInterval = cc.persistConsumerOffsetInterval;
this.pullTimeDelayMillsWhenException = cc.pullTimeDelayMillsWhenException;
this.unitMode = cc.unitMode; this.unitMode = cc.unitMode;
this.unitName = cc.unitName; this.unitName = cc.unitName;
this.vipChannelEnabled = cc.vipChannelEnabled; this.vipChannelEnabled = cc.vipChannelEnabled;
...@@ -165,6 +167,7 @@ public class ClientConfig { ...@@ -165,6 +167,7 @@ public class ClientConfig {
cc.pollNameServerInterval = pollNameServerInterval; cc.pollNameServerInterval = pollNameServerInterval;
cc.heartbeatBrokerInterval = heartbeatBrokerInterval; cc.heartbeatBrokerInterval = heartbeatBrokerInterval;
cc.persistConsumerOffsetInterval = persistConsumerOffsetInterval; cc.persistConsumerOffsetInterval = persistConsumerOffsetInterval;
cc.pullTimeDelayMillsWhenException = pullTimeDelayMillsWhenException;
cc.unitMode = unitMode; cc.unitMode = unitMode;
cc.unitName = unitName; cc.unitName = unitName;
cc.vipChannelEnabled = vipChannelEnabled; cc.vipChannelEnabled = vipChannelEnabled;
...@@ -222,6 +225,14 @@ public class ClientConfig { ...@@ -222,6 +225,14 @@ public class ClientConfig {
this.persistConsumerOffsetInterval = persistConsumerOffsetInterval; this.persistConsumerOffsetInterval = persistConsumerOffsetInterval;
} }
public long getPullTimeDelayMillsWhenException() {
return pullTimeDelayMillsWhenException;
}
public void setPullTimeDelayMillsWhenException(long pullTimeDelayMillsWhenException) {
this.pullTimeDelayMillsWhenException = pullTimeDelayMillsWhenException;
}
public String getUnitName() { public String getUnitName() {
return unitName; return unitName;
} }
...@@ -287,12 +298,13 @@ public class ClientConfig { ...@@ -287,12 +298,13 @@ public class ClientConfig {
this.accessChannel = accessChannel; this.accessChannel = accessChannel;
} }
@Override @Override
public String toString() { public String toString() {
return "ClientConfig [namesrvAddr=" + namesrvAddr + ", clientIP=" + clientIP + ", instanceName=" + instanceName return "ClientConfig [namesrvAddr=" + namesrvAddr + ", clientIP=" + clientIP + ", instanceName=" + instanceName
+ ", clientCallbackExecutorThreads=" + clientCallbackExecutorThreads + ", pollNameServerInterval=" + pollNameServerInterval + ", clientCallbackExecutorThreads=" + clientCallbackExecutorThreads + ", pollNameServerInterval=" + pollNameServerInterval
+ ", heartbeatBrokerInterval=" + heartbeatBrokerInterval + ", persistConsumerOffsetInterval=" + ", heartbeatBrokerInterval=" + heartbeatBrokerInterval + ", persistConsumerOffsetInterval=" + persistConsumerOffsetInterval
+ persistConsumerOffsetInterval + ", unitMode=" + unitMode + ", unitName=" + unitName + ", vipChannelEnabled=" + ", pullTimeDelayMillsWhenException=" + pullTimeDelayMillsWhenException + ", unitMode=" + unitMode + ", unitName=" + unitName + ", vipChannelEnabled="
+ vipChannelEnabled + ", useTLS=" + useTLS + ", language=" + language.name() + ", namespace=" + namespace + "]"; + vipChannelEnabled + ", useTLS=" + useTLS + ", language=" + language.name() + ", namespace=" + namespace + "]";
} }
} }
...@@ -23,4 +23,6 @@ public class ClientErrorCode { ...@@ -23,4 +23,6 @@ public class ClientErrorCode {
public static final int BROKER_NOT_EXIST_EXCEPTION = 10003; public static final int BROKER_NOT_EXIST_EXCEPTION = 10003;
public static final int NO_NAME_SERVER_EXCEPTION = 10004; public static final int NO_NAME_SERVER_EXCEPTION = 10004;
public static final int NOT_FOUND_TOPIC_EXCEPTION = 10005; public static final int NOT_FOUND_TOPIC_EXCEPTION = 10005;
public static final int REQUEST_TIMEOUT_EXCEPTION = 10006;
public static final int CREATE_REPLY_MESSAGE_EXCEPTION = 10007;
} }
\ No newline at end of file
...@@ -18,7 +18,6 @@ package org.apache.rocketmq.client.consumer; ...@@ -18,7 +18,6 @@ package org.apache.rocketmq.client.consumer;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely; import org.apache.rocketmq.client.consumer.rebalance.AllocateMessageQueueAveragely;
import org.apache.rocketmq.client.consumer.store.OffsetStore; import org.apache.rocketmq.client.consumer.store.OffsetStore;
...@@ -27,6 +26,7 @@ import org.apache.rocketmq.client.impl.consumer.DefaultLitePullConsumerImpl; ...@@ -27,6 +26,7 @@ import org.apache.rocketmq.client.impl.consumer.DefaultLitePullConsumerImpl;
import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.common.protocol.NamespaceUtil;
import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.RPCHook;
...@@ -35,12 +35,12 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon ...@@ -35,12 +35,12 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon
private final DefaultLitePullConsumerImpl defaultLitePullConsumerImpl; private final DefaultLitePullConsumerImpl defaultLitePullConsumerImpl;
/** /**
* Consumers belonging to the same consumer group share a group id. The consumers in a group then * Consumers belonging to the same consumer group share a group id. The consumers in a group then divides the topic
* divides the topic as fairly amongst themselves as possible by establishing that each queue is only * as fairly amongst themselves as possible by establishing that each queue is only consumed by a single consumer
* consumed by a single consumer from the group. If all consumers are from the same group, it functions * from the group. If all consumers are from the same group, it functions as a traditional message queue. Each
* as a traditional message queue. Each message would be consumed by one consumer of the group only. * message would be consumed by one consumer of the group only. When multiple consumer groups exist, the flow of the
* When multiple consumer groups exist, the flow of the data consumption model aligns with the traditional * data consumption model aligns with the traditional publish-subscribe model. The messages are broadcast to all
* publish-subscribe model. The messages are broadcast to all consumer groups. * consumer groups.
*/ */
private String consumerGroup; private String consumerGroup;
...@@ -187,6 +187,7 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon ...@@ -187,6 +187,7 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon
@Override @Override
public void start() throws MQClientException { public void start() throws MQClientException {
setConsumerGroup(NamespaceUtil.wrapNamespace(this.getNamespace(), this.consumerGroup));
this.defaultLitePullConsumerImpl.start(); this.defaultLitePullConsumerImpl.start();
} }
...@@ -266,6 +267,21 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon ...@@ -266,6 +267,21 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon
return this.defaultLitePullConsumerImpl.committed(messageQueue); return this.defaultLitePullConsumerImpl.committed(messageQueue);
} }
@Override
public void updateNameServerAddress(String nameServerAddress) {
this.defaultLitePullConsumerImpl.updateNameServerAddr(nameServerAddress);
}
@Override
public void seekToBegin(MessageQueue messageQueue) throws MQClientException {
this.defaultLitePullConsumerImpl.seekToBegin(messageQueue);
}
@Override
public void seekToEnd(MessageQueue messageQueue) throws MQClientException {
this.defaultLitePullConsumerImpl.seekToEnd(messageQueue);
}
@Override @Override
public boolean isAutoCommit() { public boolean isAutoCommit() {
return autoCommit; return autoCommit;
...@@ -411,4 +427,8 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon ...@@ -411,4 +427,8 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon
public void setTopicMetadataCheckIntervalMillis(long topicMetadataCheckIntervalMillis) { public void setTopicMetadataCheckIntervalMillis(long topicMetadataCheckIntervalMillis) {
this.topicMetadataCheckIntervalMillis = topicMetadataCheckIntervalMillis; this.topicMetadataCheckIntervalMillis = topicMetadataCheckIntervalMillis;
} }
public void setConsumerGroup(String consumerGroup) {
this.consumerGroup = consumerGroup;
}
} }
...@@ -172,4 +172,27 @@ public interface LitePullConsumer { ...@@ -172,4 +172,27 @@ public interface LitePullConsumer {
*/ */
void registerTopicMessageQueueChangeListener(String topic, void registerTopicMessageQueueChangeListener(String topic,
TopicMessageQueueChangeListener topicMessageQueueChangeListener) throws MQClientException; TopicMessageQueueChangeListener topicMessageQueueChangeListener) throws MQClientException;
/**
* Update name server addresses.
*/
void updateNameServerAddress(String nameServerAddress);
/**
* Overrides the fetch offsets with the begin offset that the consumer will use on the next poll. If this API is
* invoked for the same message queue more than once, the latest offset will be used on the next poll(). Note that
* you may lose data if this API is arbitrarily used in the middle of consumption.
*
* @param messageQueue
*/
void seekToBegin(MessageQueue messageQueue)throws MQClientException;
/**
* Overrides the fetch offsets with the end offset that the consumer will use on the next poll. If this API is
* invoked for the same message queue more than once, the latest offset will be used on the next poll(). Note that
* you may lose data if this API is arbitrarily used in the middle of consumption.
*
* @param messageQueue
*/
void seekToEnd(MessageQueue messageQueue)throws MQClientException;
} }
/*
* 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;
import org.apache.rocketmq.common.UtilAll;
public class RequestTimeoutException extends Exception {
private static final long serialVersionUID = -5758410930844185841L;
private int responseCode;
private String errorMessage;
public RequestTimeoutException(String errorMessage, Throwable cause) {
super(errorMessage, cause);
this.responseCode = -1;
this.errorMessage = errorMessage;
}
public RequestTimeoutException(int responseCode, String errorMessage) {
super("CODE: " + UtilAll.responseCode2String(responseCode) + " DESC: "
+ errorMessage);
this.responseCode = responseCode;
this.errorMessage = errorMessage;
}
public int getResponseCode() {
return responseCode;
}
public RequestTimeoutException setResponseCode(final int responseCode) {
this.responseCode = responseCode;
return this;
}
public String getErrorMessage() {
return errorMessage;
}
public void setErrorMessage(final String errorMessage) {
this.errorMessage = errorMessage;
}
}
...@@ -16,16 +16,19 @@ ...@@ -16,16 +16,19 @@
*/ */
package org.apache.rocketmq.client.impl; package org.apache.rocketmq.client.impl;
import io.netty.channel.ChannelHandlerContext;
import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import io.netty.channel.ChannelHandlerContext;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.rocketmq.client.impl.factory.MQClientInstance; import org.apache.rocketmq.client.impl.factory.MQClientInstance;
import org.apache.rocketmq.client.impl.producer.MQProducerInner; import org.apache.rocketmq.client.impl.producer.MQProducerInner;
import org.apache.rocketmq.client.log.ClientLogger; import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.client.producer.RequestFutureTable;
import org.apache.rocketmq.client.producer.RequestResponseFuture;
import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.UtilAll;
import org.apache.rocketmq.common.message.MessageAccessor;
import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.message.MessageDecoder;
import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExt;
...@@ -42,15 +45,17 @@ import org.apache.rocketmq.common.protocol.header.ConsumeMessageDirectlyResultRe ...@@ -42,15 +45,17 @@ import org.apache.rocketmq.common.protocol.header.ConsumeMessageDirectlyResultRe
import org.apache.rocketmq.common.protocol.header.GetConsumerRunningInfoRequestHeader; import org.apache.rocketmq.common.protocol.header.GetConsumerRunningInfoRequestHeader;
import org.apache.rocketmq.common.protocol.header.GetConsumerStatusRequestHeader; import org.apache.rocketmq.common.protocol.header.GetConsumerStatusRequestHeader;
import org.apache.rocketmq.common.protocol.header.NotifyConsumerIdsChangedRequestHeader; import org.apache.rocketmq.common.protocol.header.NotifyConsumerIdsChangedRequestHeader;
import org.apache.rocketmq.common.protocol.header.ReplyMessageRequestHeader;
import org.apache.rocketmq.common.protocol.header.ResetOffsetRequestHeader; import org.apache.rocketmq.common.protocol.header.ResetOffsetRequestHeader;
import org.apache.rocketmq.common.sysflag.MessageSysFlag;
import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.common.RemotingHelper;
import org.apache.rocketmq.remoting.common.RemotingUtil;
import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.exception.RemotingCommandException;
import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor; import org.apache.rocketmq.remoting.netty.AsyncNettyRequestProcessor;
import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.RemotingCommand;
public class ClientRemotingProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor { public class ClientRemotingProcessor extends AsyncNettyRequestProcessor implements NettyRequestProcessor {
private final InternalLogger log = ClientLogger.getLog(); private final InternalLogger log = ClientLogger.getLog();
private final MQClientInstance mqClientFactory; private final MQClientInstance mqClientFactory;
...@@ -77,6 +82,9 @@ public class ClientRemotingProcessor extends AsyncNettyRequestProcessor implemen ...@@ -77,6 +82,9 @@ public class ClientRemotingProcessor extends AsyncNettyRequestProcessor implemen
case RequestCode.CONSUME_MESSAGE_DIRECTLY: case RequestCode.CONSUME_MESSAGE_DIRECTLY:
return this.consumeMessageDirectly(ctx, request); return this.consumeMessageDirectly(ctx, request);
case RequestCode.PUSH_REPLY_MESSAGE_TO_CLIENT:
return this.receiveReplyMessage(ctx, request);
default: default:
break; break;
} }
...@@ -214,4 +222,73 @@ public class ClientRemotingProcessor extends AsyncNettyRequestProcessor implemen ...@@ -214,4 +222,73 @@ public class ClientRemotingProcessor extends AsyncNettyRequestProcessor implemen
return response; return response;
} }
private RemotingCommand receiveReplyMessage(ChannelHandlerContext ctx,
RemotingCommand request) throws RemotingCommandException {
final RemotingCommand response = RemotingCommand.createResponseCommand(null);
long receiveTime = System.currentTimeMillis();
ReplyMessageRequestHeader requestHeader = (ReplyMessageRequestHeader) request.decodeCommandCustomHeader(ReplyMessageRequestHeader.class);
try {
MessageExt msg = new MessageExt();
msg.setTopic(requestHeader.getTopic());
msg.setQueueId(requestHeader.getQueueId());
msg.setStoreTimestamp(requestHeader.getStoreTimestamp());
if (requestHeader.getBornHost() != null) {
msg.setBornHost(RemotingUtil.string2SocketAddress(requestHeader.getBornHost()));
}
if (requestHeader.getStoreHost() != null) {
msg.setStoreHost(RemotingUtil.string2SocketAddress(requestHeader.getStoreHost()));
}
byte[] body = request.getBody();
if ((requestHeader.getSysFlag() & MessageSysFlag.COMPRESSED_FLAG) == MessageSysFlag.COMPRESSED_FLAG) {
try {
body = UtilAll.uncompress(body);
} catch (IOException e) {
log.warn("err when uncompress constant", e);
}
}
msg.setBody(body);
msg.setFlag(requestHeader.getFlag());
MessageAccessor.setProperties(msg, MessageDecoder.string2messageProperties(requestHeader.getProperties()));
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REPLY_MESSAGE_ARRIVE_TIME, String.valueOf(receiveTime));
msg.setBornTimestamp(requestHeader.getBornTimestamp());
msg.setReconsumeTimes(requestHeader.getReconsumeTimes() == null ? 0 : requestHeader.getReconsumeTimes());
log.debug("receive reply message :{}", msg);
processReplyMessage(msg);
response.setCode(ResponseCode.SUCCESS);
response.setRemark(null);
} catch (Exception e) {
log.warn("unknown err when receiveReplyMsg", e);
response.setCode(ResponseCode.SYSTEM_ERROR);
response.setRemark("process reply message fail");
}
return response;
}
private void processReplyMessage(MessageExt replyMsg) {
final String correlationId = replyMsg.getUserProperty(MessageConst.PROPERTY_CORRELATION_ID);
final RequestResponseFuture requestResponseFuture = RequestFutureTable.getRequestFutureTable().get(correlationId);
if (requestResponseFuture != null) {
requestResponseFuture.putResponseMessage(replyMsg);
RequestFutureTable.getRequestFutureTable().remove(correlationId);
if (requestResponseFuture.getRequestCallback() != null) {
requestResponseFuture.getRequestCallback().onSuccess(replyMsg);
} else {
requestResponseFuture.putResponseMessage(replyMsg);
}
} else {
String bornHost = replyMsg.getBornHostString();
log.warn(String.format("receive reply message, but not matched any request, CorrelationId: %s , reply from host: %s",
correlationId, bornHost));
}
}
} }
...@@ -27,7 +27,6 @@ import java.util.Map; ...@@ -27,7 +27,6 @@ import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.consumer.PullCallback; import org.apache.rocketmq.client.consumer.PullCallback;
...@@ -201,6 +200,8 @@ public class MQClientAPIImpl { ...@@ -201,6 +200,8 @@ public class MQClientAPIImpl {
this.remotingClient.registerProcessor(RequestCode.GET_CONSUMER_RUNNING_INFO, this.clientRemotingProcessor, null); this.remotingClient.registerProcessor(RequestCode.GET_CONSUMER_RUNNING_INFO, this.clientRemotingProcessor, null);
this.remotingClient.registerProcessor(RequestCode.CONSUME_MESSAGE_DIRECTLY, this.clientRemotingProcessor, null); this.remotingClient.registerProcessor(RequestCode.CONSUME_MESSAGE_DIRECTLY, this.clientRemotingProcessor, null);
this.remotingClient.registerProcessor(RequestCode.PUSH_REPLY_MESSAGE_TO_CLIENT, this.clientRemotingProcessor, null);
} }
public List<String> getNameServerAddressList() { public List<String> getNameServerAddressList() {
...@@ -304,8 +305,8 @@ public class MQClientAPIImpl { ...@@ -304,8 +305,8 @@ public class MQClientAPIImpl {
requestHeader.setDefaultGroupPerm(plainAccessConfig.getDefaultGroupPerm()); requestHeader.setDefaultGroupPerm(plainAccessConfig.getDefaultGroupPerm());
requestHeader.setDefaultTopicPerm(plainAccessConfig.getDefaultTopicPerm()); requestHeader.setDefaultTopicPerm(plainAccessConfig.getDefaultTopicPerm());
requestHeader.setWhiteRemoteAddress(plainAccessConfig.getWhiteRemoteAddress()); requestHeader.setWhiteRemoteAddress(plainAccessConfig.getWhiteRemoteAddress());
requestHeader.setTopicPerms(UtilAll.List2String(plainAccessConfig.getTopicPerms(),",")); requestHeader.setTopicPerms(UtilAll.List2String(plainAccessConfig.getTopicPerms(), ","));
requestHeader.setGroupPerms(UtilAll.List2String(plainAccessConfig.getGroupPerms(),",")); requestHeader.setGroupPerms(UtilAll.List2String(plainAccessConfig.getGroupPerms(), ","));
RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_ACL_CONFIG, requestHeader); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_ACL_CONFIG, requestHeader);
...@@ -344,7 +345,7 @@ public class MQClientAPIImpl { ...@@ -344,7 +345,7 @@ public class MQClientAPIImpl {
throw new MQClientException(response.getCode(), response.getRemark()); throw new MQClientException(response.getCode(), response.getRemark());
} }
public void updateGlobalWhiteAddrsConfig(final String addr, final String globalWhiteAddrs,final long timeoutMillis) public void updateGlobalWhiteAddrsConfig(final String addr, final String globalWhiteAddrs, final long timeoutMillis)
throws RemotingException, MQBrokerException, InterruptedException, MQClientException { throws RemotingException, MQBrokerException, InterruptedException, MQClientException {
UpdateGlobalWhiteAddrsConfigRequestHeader requestHeader = new UpdateGlobalWhiteAddrsConfigRequestHeader(); UpdateGlobalWhiteAddrsConfigRequestHeader requestHeader = new UpdateGlobalWhiteAddrsConfigRequestHeader();
...@@ -366,7 +367,8 @@ public class MQClientAPIImpl { ...@@ -366,7 +367,8 @@ public class MQClientAPIImpl {
throw new MQClientException(response.getCode(), response.getRemark()); throw new MQClientException(response.getCode(), response.getRemark());
} }
public ClusterAclVersionInfo getBrokerClusterAclInfo(final String addr,final long timeoutMillis) throws RemotingCommandException, InterruptedException, RemotingTimeoutException, public ClusterAclVersionInfo getBrokerClusterAclInfo(final String addr,
final long timeoutMillis) throws RemotingCommandException, InterruptedException, RemotingTimeoutException,
RemotingSendRequestException, RemotingConnectException, MQBrokerException { RemotingSendRequestException, RemotingConnectException, MQBrokerException {
RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CLUSTER_ACL_INFO, null); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CLUSTER_ACL_INFO, null);
...@@ -389,7 +391,7 @@ public class MQClientAPIImpl { ...@@ -389,7 +391,7 @@ public class MQClientAPIImpl {
} }
throw new MQBrokerException(response.getCode(), response.getRemark()); throw new MQBrokerException(response.getCode(), response.getRemark());
} }
public AclConfig getBrokerClusterConfig(final String addr, final long timeoutMillis) throws RemotingCommandException, InterruptedException, RemotingTimeoutException, public AclConfig getBrokerClusterConfig(final String addr, final long timeoutMillis) throws RemotingCommandException, InterruptedException, RemotingTimeoutException,
...@@ -445,13 +447,23 @@ public class MQClientAPIImpl { ...@@ -445,13 +447,23 @@ public class MQClientAPIImpl {
) throws RemotingException, MQBrokerException, InterruptedException { ) throws RemotingException, MQBrokerException, InterruptedException {
long beginStartTime = System.currentTimeMillis(); long beginStartTime = System.currentTimeMillis();
RemotingCommand request = null; RemotingCommand request = null;
if (sendSmartMsg || msg instanceof MessageBatch) { String msgType = msg.getProperty(MessageConst.PROPERTY_MESSAGE_TYPE);
SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2(requestHeader); boolean isReply = msgType != null && msgType.equals(MixAll.REPLY_MESSAGE_FLAG);
request = RemotingCommand.createRequestCommand(msg instanceof MessageBatch ? RequestCode.SEND_BATCH_MESSAGE : RequestCode.SEND_MESSAGE_V2, requestHeaderV2); if (isReply) {
if (sendSmartMsg) {
SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2(requestHeader);
request = RemotingCommand.createRequestCommand(RequestCode.SEND_REPLY_MESSAGE_V2, requestHeaderV2);
} else {
request = RemotingCommand.createRequestCommand(RequestCode.SEND_REPLY_MESSAGE, requestHeader);
}
} else { } else {
request = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE, requestHeader); if (sendSmartMsg || msg instanceof MessageBatch) {
SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2(requestHeader);
request = RemotingCommand.createRequestCommand(msg instanceof MessageBatch ? RequestCode.SEND_BATCH_MESSAGE : RequestCode.SEND_MESSAGE_V2, requestHeaderV2);
} else {
request = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE, requestHeader);
}
} }
request.setBody(msg.getBody()); request.setBody(msg.getBody());
switch (communicationMode) { switch (communicationMode) {
......
...@@ -16,16 +16,16 @@ ...@@ -16,16 +16,16 @@
*/ */
package org.apache.rocketmq.client.impl.consumer; package org.apache.rocketmq.client.impl.consumer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.HashSet;
import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.Collection;
import java.util.HashSet;
import java.util.Properties; import java.util.Properties;
import java.util.Set;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
...@@ -35,13 +35,12 @@ import java.util.concurrent.ScheduledExecutorService; ...@@ -35,13 +35,12 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.rocketmq.client.Validators; import org.apache.rocketmq.client.Validators;
import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer; import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
import org.apache.rocketmq.client.consumer.TopicMessageQueueChangeListener;
import org.apache.rocketmq.client.consumer.MessageSelector;
import org.apache.rocketmq.client.consumer.MessageQueueListener; import org.apache.rocketmq.client.consumer.MessageQueueListener;
import org.apache.rocketmq.client.consumer.MessageSelector;
import org.apache.rocketmq.client.consumer.PullResult; import org.apache.rocketmq.client.consumer.PullResult;
import org.apache.rocketmq.client.consumer.TopicMessageQueueChangeListener;
import org.apache.rocketmq.client.consumer.store.LocalFileOffsetStore; import org.apache.rocketmq.client.consumer.store.LocalFileOffsetStore;
import org.apache.rocketmq.client.consumer.store.OffsetStore; import org.apache.rocketmq.client.consumer.store.OffsetStore;
import org.apache.rocketmq.client.consumer.store.ReadOffsetType; import org.apache.rocketmq.client.consumer.store.ReadOffsetType;
...@@ -106,7 +105,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner { ...@@ -106,7 +105,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
/** /**
* Delay some time when exception occur * Delay some time when exception occur
*/ */
private static final long PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION = 1000; private long pullTimeDelayMillsWhenException = 1000;
/** /**
* Flow control interval * Flow control interval
*/ */
...@@ -156,6 +155,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner { ...@@ -156,6 +155,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
return new Thread(r, "MonitorMessageQueueChangeThread"); return new Thread(r, "MonitorMessageQueueChangeThread");
} }
}); });
this.pullTimeDelayMillsWhenException = defaultLitePullConsumer.getPullTimeDelayMillsWhenException();
} }
private void checkServiceState() { private void checkServiceState() {
...@@ -163,6 +163,10 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner { ...@@ -163,6 +163,10 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
throw new IllegalStateException(NOT_RUNNING_EXCEPTION_MESSAGE); throw new IllegalStateException(NOT_RUNNING_EXCEPTION_MESSAGE);
} }
public void updateNameServerAddr(String newAddresses) {
this.mQClientFactory.getMQClientAPIImpl().updateNameServerAddressList(newAddresses);
}
private synchronized void setSubscriptionType(SubscriptionType type) { private synchronized void setSubscriptionType(SubscriptionType type) {
if (this.subscriptionType == SubscriptionType.NONE) if (this.subscriptionType == SubscriptionType.NONE)
this.subscriptionType = type; this.subscriptionType = type;
...@@ -555,6 +559,16 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner { ...@@ -555,6 +559,16 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
} }
} }
public void seekToBegin(MessageQueue messageQueue) throws MQClientException {
long begin = minOffset(messageQueue);
this.seek(messageQueue, begin);
}
public void seekToEnd(MessageQueue messageQueue) throws MQClientException {
long begin = maxOffset(messageQueue);
this.seek(messageQueue, begin);
}
private long maxOffset(MessageQueue messageQueue) throws MQClientException { private long maxOffset(MessageQueue messageQueue) throws MQClientException {
checkServiceState(); checkServiceState();
return this.mQClientFactory.getMQAdminImpl().maxOffset(messageQueue); return this.mQClientFactory.getMQAdminImpl().maxOffset(messageQueue);
...@@ -763,8 +777,9 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner { ...@@ -763,8 +777,9 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
subscriptionData = FilterAPI.buildSubscriptionData(defaultLitePullConsumer.getConsumerGroup(), subscriptionData = FilterAPI.buildSubscriptionData(defaultLitePullConsumer.getConsumerGroup(),
topic, SubscriptionData.SUB_ALL); topic, SubscriptionData.SUB_ALL);
} }
PullResult pullResult = pull(messageQueue, subscriptionData, offset, nextPullBatchSize()); PullResult pullResult = pull(messageQueue, subscriptionData, offset, nextPullBatchSize());
switch (pullResult.getPullStatus()) { switch (pullResult.getPullStatus()) {
case FOUND: case FOUND:
final Object objLock = messageQueueLock.fetchLockObject(messageQueue); final Object objLock = messageQueueLock.fetchLockObject(messageQueue);
...@@ -783,7 +798,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner { ...@@ -783,7 +798,7 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
} }
updatePullOffset(messageQueue, pullResult.getNextBeginOffset()); updatePullOffset(messageQueue, pullResult.getNextBeginOffset());
} catch (Throwable e) { } catch (Throwable e) {
pullDelayTimeMills = PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION; pullDelayTimeMills = pullTimeDelayMillsWhenException;
log.error("An error occurred in pull message process.", e); log.error("An error occurred in pull message process.", e);
} }
...@@ -1070,4 +1085,8 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner { ...@@ -1070,4 +1085,8 @@ public class DefaultLitePullConsumerImpl implements MQConsumerInner {
} }
} }
public void setPullTimeDelayMillsWhenException(long pullTimeDelayMillsWhenException) {
this.pullTimeDelayMillsWhenException = pullTimeDelayMillsWhenException;
}
} }
...@@ -83,7 +83,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -83,7 +83,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
/** /**
* Delay some time when exception occur * Delay some time when exception occur
*/ */
private static final long PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION = 3000; private long pullTimeDelayMillsWhenException = 3000;
/** /**
* Flow control interval * Flow control interval
*/ */
...@@ -115,6 +115,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -115,6 +115,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
public DefaultMQPushConsumerImpl(DefaultMQPushConsumer defaultMQPushConsumer, RPCHook rpcHook) { public DefaultMQPushConsumerImpl(DefaultMQPushConsumer defaultMQPushConsumer, RPCHook rpcHook) {
this.defaultMQPushConsumer = defaultMQPushConsumer; this.defaultMQPushConsumer = defaultMQPushConsumer;
this.rpcHook = rpcHook; this.rpcHook = rpcHook;
this.pullTimeDelayMillsWhenException = defaultMQPushConsumer.getPullTimeDelayMillsWhenException();
} }
public void registerFilterMessageHook(final FilterMessageHook hook) { public void registerFilterMessageHook(final FilterMessageHook hook) {
...@@ -222,7 +223,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -222,7 +223,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
this.makeSureStateOK(); this.makeSureStateOK();
} catch (MQClientException e) { } catch (MQClientException e) {
log.warn("pullMessage exception, consumer state not ok", e); log.warn("pullMessage exception, consumer state not ok", e);
this.executePullRequestLater(pullRequest, PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION); this.executePullRequestLater(pullRequest, pullTimeDelayMillsWhenException);
return; return;
} }
...@@ -282,7 +283,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -282,7 +283,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
pullRequest.setNextOffset(offset); pullRequest.setNextOffset(offset);
} }
} else { } else {
this.executePullRequestLater(pullRequest, PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION); this.executePullRequestLater(pullRequest, pullTimeDelayMillsWhenException);
log.info("pull message later because not locked in broker, {}", pullRequest); log.info("pull message later because not locked in broker, {}", pullRequest);
return; return;
} }
...@@ -290,7 +291,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -290,7 +291,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
final SubscriptionData subscriptionData = this.rebalanceImpl.getSubscriptionInner().get(pullRequest.getMessageQueue().getTopic()); final SubscriptionData subscriptionData = this.rebalanceImpl.getSubscriptionInner().get(pullRequest.getMessageQueue().getTopic());
if (null == subscriptionData) { if (null == subscriptionData) {
this.executePullRequestLater(pullRequest, PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION); this.executePullRequestLater(pullRequest, pullTimeDelayMillsWhenException);
log.warn("find the consumer's subscription failed, {}", pullRequest); log.warn("find the consumer's subscription failed, {}", pullRequest);
return; return;
} }
...@@ -397,7 +398,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -397,7 +398,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
log.warn("execute the pull request exception", e); log.warn("execute the pull request exception", e);
} }
DefaultMQPushConsumerImpl.this.executePullRequestLater(pullRequest, PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION); DefaultMQPushConsumerImpl.this.executePullRequestLater(pullRequest, pullTimeDelayMillsWhenException);
} }
}; };
...@@ -444,7 +445,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -444,7 +445,7 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
); );
} catch (Exception e) { } catch (Exception e) {
log.error("pullKernelImpl exception", e); log.error("pullKernelImpl exception", e);
this.executePullRequestLater(pullRequest, PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION); this.executePullRequestLater(pullRequest, pullTimeDelayMillsWhenException);
} }
} }
...@@ -1168,4 +1169,8 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner { ...@@ -1168,4 +1169,8 @@ public class DefaultMQPushConsumerImpl implements MQConsumerInner {
this.consumeMessageService = consumeMessageService; this.consumeMessageService = consumeMessageService;
} }
public void setPullTimeDelayMillsWhenException(long pullTimeDelayMillsWhenException) {
this.pullTimeDelayMillsWhenException = pullTimeDelayMillsWhenException;
}
} }
...@@ -24,6 +24,8 @@ import java.util.HashSet; ...@@ -24,6 +24,8 @@ import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
import java.util.Set; import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
...@@ -39,6 +41,7 @@ import org.apache.rocketmq.client.Validators; ...@@ -39,6 +41,7 @@ import org.apache.rocketmq.client.Validators;
import org.apache.rocketmq.client.common.ClientErrorCode; import org.apache.rocketmq.client.common.ClientErrorCode;
import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.exception.RequestTimeoutException;
import org.apache.rocketmq.client.hook.CheckForbiddenContext; import org.apache.rocketmq.client.hook.CheckForbiddenContext;
import org.apache.rocketmq.client.hook.CheckForbiddenHook; import org.apache.rocketmq.client.hook.CheckForbiddenHook;
import org.apache.rocketmq.client.hook.SendMessageContext; import org.apache.rocketmq.client.hook.SendMessageContext;
...@@ -52,6 +55,9 @@ import org.apache.rocketmq.client.producer.DefaultMQProducer; ...@@ -52,6 +55,9 @@ import org.apache.rocketmq.client.producer.DefaultMQProducer;
import org.apache.rocketmq.client.producer.LocalTransactionExecuter; import org.apache.rocketmq.client.producer.LocalTransactionExecuter;
import org.apache.rocketmq.client.producer.LocalTransactionState; import org.apache.rocketmq.client.producer.LocalTransactionState;
import org.apache.rocketmq.client.producer.MessageQueueSelector; import org.apache.rocketmq.client.producer.MessageQueueSelector;
import org.apache.rocketmq.client.producer.RequestCallback;
import org.apache.rocketmq.client.producer.RequestFutureTable;
import org.apache.rocketmq.client.producer.RequestResponseFuture;
import org.apache.rocketmq.client.producer.SendCallback; import org.apache.rocketmq.client.producer.SendCallback;
import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.client.producer.SendResult;
import org.apache.rocketmq.client.producer.SendStatus; import org.apache.rocketmq.client.producer.SendStatus;
...@@ -79,6 +85,7 @@ import org.apache.rocketmq.common.protocol.header.CheckTransactionStateRequestHe ...@@ -79,6 +85,7 @@ import org.apache.rocketmq.common.protocol.header.CheckTransactionStateRequestHe
import org.apache.rocketmq.common.protocol.header.EndTransactionRequestHeader; import org.apache.rocketmq.common.protocol.header.EndTransactionRequestHeader;
import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader; import org.apache.rocketmq.common.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.common.sysflag.MessageSysFlag; import org.apache.rocketmq.common.sysflag.MessageSysFlag;
import org.apache.rocketmq.common.utils.CorrelationIdUtil;
import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.RPCHook;
import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.common.RemotingHelper;
...@@ -95,17 +102,16 @@ public class DefaultMQProducerImpl implements MQProducerInner { ...@@ -95,17 +102,16 @@ public class DefaultMQProducerImpl implements MQProducerInner {
new ConcurrentHashMap<String, TopicPublishInfo>(); new ConcurrentHashMap<String, TopicPublishInfo>();
private final ArrayList<SendMessageHook> sendMessageHookList = new ArrayList<SendMessageHook>(); private final ArrayList<SendMessageHook> sendMessageHookList = new ArrayList<SendMessageHook>();
private final RPCHook rpcHook; private final RPCHook rpcHook;
private final BlockingQueue<Runnable> asyncSenderThreadPoolQueue;
private final ExecutorService defaultAsyncSenderExecutor;
private final Timer timer = new Timer("RequestHouseKeepingService", true);
protected BlockingQueue<Runnable> checkRequestQueue; protected BlockingQueue<Runnable> checkRequestQueue;
protected ExecutorService checkExecutor; protected ExecutorService checkExecutor;
private ServiceState serviceState = ServiceState.CREATE_JUST; private ServiceState serviceState = ServiceState.CREATE_JUST;
private MQClientInstance mQClientFactory; private MQClientInstance mQClientFactory;
private ArrayList<CheckForbiddenHook> checkForbiddenHookList = new ArrayList<CheckForbiddenHook>(); private ArrayList<CheckForbiddenHook> checkForbiddenHookList = new ArrayList<CheckForbiddenHook>();
private int zipCompressLevel = Integer.parseInt(System.getProperty(MixAll.MESSAGE_COMPRESS_LEVEL, "5")); private int zipCompressLevel = Integer.parseInt(System.getProperty(MixAll.MESSAGE_COMPRESS_LEVEL, "5"));
private MQFaultStrategy mqFaultStrategy = new MQFaultStrategy(); private MQFaultStrategy mqFaultStrategy = new MQFaultStrategy();
private final BlockingQueue<Runnable> asyncSenderThreadPoolQueue;
private final ExecutorService defaultAsyncSenderExecutor;
private ExecutorService asyncSenderExecutor; private ExecutorService asyncSenderExecutor;
public DefaultMQProducerImpl(final DefaultMQProducer defaultMQProducer) { public DefaultMQProducerImpl(final DefaultMQProducer defaultMQProducer) {
...@@ -212,6 +218,17 @@ public class DefaultMQProducerImpl implements MQProducerInner { ...@@ -212,6 +218,17 @@ public class DefaultMQProducerImpl implements MQProducerInner {
} }
this.mQClientFactory.sendHeartbeatToAllBrokerWithLock(); this.mQClientFactory.sendHeartbeatToAllBrokerWithLock();
this.timer.scheduleAtFixedRate(new TimerTask() {
@Override
public void run() {
try {
RequestFutureTable.scanExpiredRequest();
} catch (Throwable e) {
log.error("scan RequestFutureTable exception", e);
}
}
}, 1000 * 3, 1000);
} }
private void checkConfig() throws MQClientException { private void checkConfig() throws MQClientException {
...@@ -1188,6 +1205,12 @@ public class DefaultMQProducerImpl implements MQProducerInner { ...@@ -1188,6 +1205,12 @@ public class DefaultMQProducerImpl implements MQProducerInner {
if (null == localTransactionExecuter && null == transactionListener) { if (null == localTransactionExecuter && null == transactionListener) {
throw new MQClientException("tranExecutor is null", null); throw new MQClientException("tranExecutor is null", null);
} }
// ignore DelayTimeLevel parameter
if (msg.getDelayTimeLevel() != 0) {
MessageAccessor.clearProperty(msg, MessageConst.PROPERTY_DELAY_TIME_LEVEL);
}
Validators.checkMessage(msg, this.defaultMQProducer); Validators.checkMessage(msg, this.defaultMQProducer);
SendResult sendResult = null; SendResult sendResult = null;
...@@ -1319,6 +1342,233 @@ public class DefaultMQProducerImpl implements MQProducerInner { ...@@ -1319,6 +1342,233 @@ public class DefaultMQProducerImpl implements MQProducerInner {
return this.sendDefaultImpl(msg, CommunicationMode.SYNC, null, timeout); return this.sendDefaultImpl(msg, CommunicationMode.SYNC, null, timeout);
} }
public Message request(Message msg,
long timeout) throws RequestTimeoutException, MQClientException, RemotingException, MQBrokerException, InterruptedException {
long beginTimestamp = System.currentTimeMillis();
prepareSendRequest(msg, timeout);
final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);
try {
final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, null);
RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture);
long cost = System.currentTimeMillis() - beginTimestamp;
this.sendDefaultImpl(msg, CommunicationMode.ASYNC, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
requestResponseFuture.setSendReqeustOk(true);
}
@Override
public void onException(Throwable e) {
requestResponseFuture.setSendReqeustOk(false);
requestResponseFuture.putResponseMessage(null);
requestResponseFuture.setCause(e);
}
}, timeout - cost);
Message responseMessage = requestResponseFuture.waitResponseMessage(timeout - cost);
if (responseMessage == null) {
if (requestResponseFuture.isSendRequestOk()) {
throw new RequestTimeoutException(ClientErrorCode.REQUEST_TIMEOUT_EXCEPTION,
"send request message to <" + msg.getTopic() + "> OK, but wait reply message timeout, " + timeout + " ms.");
} else {
throw new MQClientException("send request message to <" + msg.getTopic() + "> fail", requestResponseFuture.getCause());
}
}
return responseMessage;
} finally {
RequestFutureTable.getRequestFutureTable().remove(correlationId);
}
}
public void request(Message msg, final RequestCallback requestCallback, long timeout)
throws RemotingException, InterruptedException, MQClientException, MQBrokerException {
long beginTimestamp = System.currentTimeMillis();
prepareSendRequest(msg, timeout);
final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);
final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, requestCallback);
RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture);
long cost = System.currentTimeMillis() - beginTimestamp;
this.sendDefaultImpl(msg, CommunicationMode.ASYNC, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
requestResponseFuture.setSendReqeustOk(true);
}
@Override
public void onException(Throwable e) {
requestResponseFuture.setCause(e);
requestFail(correlationId);
}
}, timeout - cost);
}
public Message request(final Message msg, final MessageQueueSelector selector, final Object arg,
final long timeout) throws MQClientException, RemotingException, MQBrokerException,
InterruptedException, RequestTimeoutException {
long beginTimestamp = System.currentTimeMillis();
prepareSendRequest(msg, timeout);
final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);
try {
final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, null);
RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture);
long cost = System.currentTimeMillis() - beginTimestamp;
this.sendSelectImpl(msg, selector, arg, CommunicationMode.ASYNC, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
requestResponseFuture.setSendReqeustOk(true);
}
@Override
public void onException(Throwable e) {
requestResponseFuture.setSendReqeustOk(false);
requestResponseFuture.putResponseMessage(null);
requestResponseFuture.setCause(e);
}
}, timeout - cost);
Message responseMessage = requestResponseFuture.waitResponseMessage(timeout - cost);
if (responseMessage == null) {
if (requestResponseFuture.isSendRequestOk()) {
throw new RequestTimeoutException(ClientErrorCode.REQUEST_TIMEOUT_EXCEPTION,
"send request message to <" + msg.getTopic() + "> OK, but wait reply message timeout, " + timeout + " ms.");
} else {
throw new MQClientException("send request message to <" + msg.getTopic() + "> fail", requestResponseFuture.getCause());
}
}
return responseMessage;
} finally {
RequestFutureTable.getRequestFutureTable().remove(correlationId);
}
}
public void request(final Message msg, final MessageQueueSelector selector, final Object arg,
final RequestCallback requestCallback, final long timeout)
throws RemotingException, InterruptedException, MQClientException, MQBrokerException {
long beginTimestamp = System.currentTimeMillis();
prepareSendRequest(msg, timeout);
final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);
final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, requestCallback);
RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture);
long cost = System.currentTimeMillis() - beginTimestamp;
this.sendSelectImpl(msg, selector, arg, CommunicationMode.ASYNC, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
requestResponseFuture.setSendReqeustOk(true);
}
@Override
public void onException(Throwable e) {
requestResponseFuture.setCause(e);
requestFail(correlationId);
}
}, timeout - cost);
}
public Message request(final Message msg, final MessageQueue mq, final long timeout)
throws MQClientException, RemotingException, MQBrokerException, InterruptedException, RequestTimeoutException {
long beginTimestamp = System.currentTimeMillis();
prepareSendRequest(msg, timeout);
final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);
try {
final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, null);
RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture);
long cost = System.currentTimeMillis() - beginTimestamp;
this.sendKernelImpl(msg, mq, CommunicationMode.ASYNC, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
requestResponseFuture.setSendReqeustOk(true);
}
@Override
public void onException(Throwable e) {
requestResponseFuture.setSendReqeustOk(false);
requestResponseFuture.putResponseMessage(null);
requestResponseFuture.setCause(e);
}
}, null, timeout - cost);
Message responseMessage = requestResponseFuture.waitResponseMessage(timeout - cost);
if (responseMessage == null) {
if (requestResponseFuture.isSendRequestOk()) {
throw new RequestTimeoutException(ClientErrorCode.REQUEST_TIMEOUT_EXCEPTION,
"send request message to <" + msg.getTopic() + "> OK, but wait reply message timeout, " + timeout + " ms.");
} else {
throw new MQClientException("send request message to <" + msg.getTopic() + "> fail", requestResponseFuture.getCause());
}
}
return responseMessage;
} finally {
RequestFutureTable.getRequestFutureTable().remove(correlationId);
}
}
public void request(final Message msg, final MessageQueue mq, final RequestCallback requestCallback, long timeout)
throws RemotingException, InterruptedException, MQClientException, MQBrokerException {
long beginTimestamp = System.currentTimeMillis();
prepareSendRequest(msg, timeout);
final String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);
final RequestResponseFuture requestResponseFuture = new RequestResponseFuture(correlationId, timeout, requestCallback);
RequestFutureTable.getRequestFutureTable().put(correlationId, requestResponseFuture);
long cost = System.currentTimeMillis() - beginTimestamp;
this.sendKernelImpl(msg, mq, CommunicationMode.ASYNC, new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
requestResponseFuture.setSendReqeustOk(true);
}
@Override
public void onException(Throwable e) {
requestResponseFuture.setCause(e);
requestFail(correlationId);
}
}, null, timeout - cost);
}
private void requestFail(final String correlationId) {
RequestResponseFuture responseFuture = RequestFutureTable.getRequestFutureTable().remove(correlationId);
if (responseFuture != null) {
responseFuture.setSendReqeustOk(false);
responseFuture.putResponseMessage(null);
try {
responseFuture.executeRequestCallback();
} catch (Exception e) {
log.warn("execute requestCallback in requestFail, and callback throw", e);
}
}
}
private void prepareSendRequest(final Message msg, long timeout) {
String correlationId = CorrelationIdUtil.createCorrelationId();
String requestClientId = this.getmQClientFactory().getClientId();
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_CORRELATION_ID, correlationId);
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT, requestClientId);
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_TTL, String.valueOf(timeout));
boolean hasRouteData = this.getmQClientFactory().getTopicRouteTable().containsKey(msg.getTopic());
if (!hasRouteData) {
long beginTimestamp = System.currentTimeMillis();
this.tryToFindTopicPublishInfo(msg.getTopic());
this.getmQClientFactory().sendHeartbeatToAllBrokerWithLock();
long cost = System.currentTimeMillis() - beginTimestamp;
if (cost > 500) {
log.warn("prepare send request for <{}> cost {} ms", msg.getTopic(), cost);
}
}
}
public ConcurrentMap<String, TopicPublishInfo> getTopicPublishInfoTable() { public ConcurrentMap<String, TopicPublishInfo> getTopicPublishInfoTable() {
return topicPublishInfoTable; return topicPublishInfoTable;
} }
......
...@@ -24,6 +24,7 @@ import org.apache.rocketmq.client.QueryResult; ...@@ -24,6 +24,7 @@ import org.apache.rocketmq.client.QueryResult;
import org.apache.rocketmq.client.Validators; import org.apache.rocketmq.client.Validators;
import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.exception.RequestTimeoutException;
import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl; import org.apache.rocketmq.client.impl.producer.DefaultMQProducerImpl;
import org.apache.rocketmq.client.log.ClientLogger; import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.client.trace.AsyncTraceDispatcher; import org.apache.rocketmq.client.trace.AsyncTraceDispatcher;
...@@ -42,38 +43,29 @@ import org.apache.rocketmq.remoting.RPCHook; ...@@ -42,38 +43,29 @@ import org.apache.rocketmq.remoting.RPCHook;
import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.remoting.exception.RemotingException;
/** /**
* This class is the entry point for applications intending to send messages. * This class is the entry point for applications intending to send messages. </p>
* </p>
* *
* It's fine to tune fields which exposes getter/setter methods, but keep in mind, all of them should work well out of * It's fine to tune fields which exposes getter/setter methods, but keep in mind, all of them should work well out of
* box for most scenarios. * box for most scenarios. </p>
* </p>
* *
* This class aggregates various <code>send</code> methods to deliver messages to brokers. Each of them has pros and * This class aggregates various <code>send</code> methods to deliver messages to brokers. Each of them has pros and
* cons; you'd better understand strengths and weakness of them before actually coding. * cons; you'd better understand strengths and weakness of them before actually coding. </p>
* </p>
* *
* <p> * <p> <strong>Thread Safety:</strong> After configuring and starting process, this class can be regarded as thread-safe
* <strong>Thread Safety:</strong> After configuring and starting process, this class can be regarded as thread-safe * and used among multiple threads context. </p>
* and used among multiple threads context.
* </p>
*/ */
public class DefaultMQProducer extends ClientConfig implements MQProducer { public class DefaultMQProducer extends ClientConfig implements MQProducer {
private final InternalLogger log = ClientLogger.getLog();
/** /**
* Wrapping internal implementations for virtually all methods presented in this class. * Wrapping internal implementations for virtually all methods presented in this class.
*/ */
protected final transient DefaultMQProducerImpl defaultMQProducerImpl; protected final transient DefaultMQProducerImpl defaultMQProducerImpl;
private final InternalLogger log = ClientLogger.getLog();
/** /**
* Producer group conceptually aggregates all producer instances of exactly same role, which is particularly * Producer group conceptually aggregates all producer instances of exactly same role, which is particularly
* important when transactional messages are involved. * important when transactional messages are involved. </p>
* </p>
* *
* For non-transactional messages, it does not matter as long as it's unique per process. * For non-transactional messages, it does not matter as long as it's unique per process. </p>
* </p>
* *
* See {@linktourl http://rocketmq.apache.org/docs/core-concept/} for more discussion. * See {@linktourl http://rocketmq.apache.org/docs/core-concept/} for more discussion.
*/ */
...@@ -100,16 +92,14 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer { ...@@ -100,16 +92,14 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
private int compressMsgBodyOverHowmuch = 1024 * 4; private int compressMsgBodyOverHowmuch = 1024 * 4;
/** /**
* Maximum number of retry to perform internally before claiming sending failure in synchronous mode. * Maximum number of retry to perform internally before claiming sending failure in synchronous mode. </p>
* </p>
* *
* This may potentially cause message duplication which is up to application developers to resolve. * This may potentially cause message duplication which is up to application developers to resolve.
*/ */
private int retryTimesWhenSendFailed = 2; private int retryTimesWhenSendFailed = 2;
/** /**
* Maximum number of retry to perform internally before claiming sending failure in asynchronous mode. * Maximum number of retry to perform internally before claiming sending failure in asynchronous mode. </p>
* </p>
* *
* This may potentially cause message duplication which is up to application developers to resolve. * This may potentially cause message duplication which is up to application developers to resolve.
*/ */
...@@ -268,14 +258,10 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer { ...@@ -268,14 +258,10 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
} }
/** /**
* Start this producer instance. * Start this producer instance. </p>
* </p>
* *
* <strong> * <strong> Much internal initializing procedures are carried out to make this instance prepared, thus, it's a must
* Much internal initializing procedures are carried out to make this instance prepared, thus, it's a must to invoke * to invoke this method before sending or querying messages. </strong> </p>
* this method before sending or querying messages.
* </strong>
* </p>
* *
* @throws MQClientException if there is any unexpected error. * @throws MQClientException if there is any unexpected error.
*/ */
...@@ -316,8 +302,7 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer { ...@@ -316,8 +302,7 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
} }
/** /**
* Send message in synchronous mode. This method returns only when the sending procedure totally completes. * Send message in synchronous mode. This method returns only when the sending procedure totally completes. </p>
* </p>
* *
* <strong>Warn:</strong> this method has internal retry-mechanism, that is, internal implementation will retry * <strong>Warn:</strong> this method has internal retry-mechanism, that is, internal implementation will retry
* {@link #retryTimesWhenSendFailed} times before claiming failure. As a result, multiple messages may potentially * {@link #retryTimesWhenSendFailed} times before claiming failure. As a result, multiple messages may potentially
...@@ -359,11 +344,9 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer { ...@@ -359,11 +344,9 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
} }
/** /**
* Send message to broker asynchronously. * Send message to broker asynchronously. </p>
* </p>
* *
* This method returns immediately. On sending completion, <code>sendCallback</code> will be executed. * This method returns immediately. On sending completion, <code>sendCallback</code> will be executed. </p>
* </p>
* *
* Similar to {@link #send(Message)}, internal implementation would potentially retry up to {@link * Similar to {@link #send(Message)}, internal implementation would potentially retry up to {@link
* #retryTimesWhenSendAsyncFailed} times before claiming sending failure, which may yield message duplication and * #retryTimesWhenSendAsyncFailed} times before claiming sending failure, which may yield message duplication and
...@@ -582,6 +565,133 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer { ...@@ -582,6 +565,133 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
this.defaultMQProducerImpl.send(msg, selector, arg, sendCallback, timeout); this.defaultMQProducerImpl.send(msg, selector, arg, sendCallback, timeout);
} }
/**
* Send request message in synchronous mode. This method returns only when the consumer consume the request message and reply a message. </p>
*
* <strong>Warn:</strong> this method has internal retry-mechanism, that is, internal implementation will retry
* {@link #retryTimesWhenSendFailed} times before claiming failure. As a result, multiple messages may potentially
* delivered to broker(s). It's up to the application developers to resolve potential duplication issue.
*
* @param msg request message to send
* @param timeout request timeout
* @return reply message
* @throws MQClientException if there is any client error.
* @throws RemotingException if there is any network-tier error.
* @throws MQBrokerException if there is any broker error.
* @throws InterruptedException if the thread is interrupted.
* @throws RequestTimeoutException if request timeout.
*/
@Override
public Message request(final Message msg, final long timeout) throws RequestTimeoutException, MQClientException,
RemotingException, MQBrokerException, InterruptedException {
msg.setTopic(withNamespace(msg.getTopic()));
return this.defaultMQProducerImpl.request(msg, timeout);
}
/**
* Request asynchronously. </p>
* This method returns immediately. On receiving reply message, <code>requestCallback</code> will be executed. </p>
*
* Similar to {@link #request(Message, long)}, internal implementation would potentially retry up to {@link
* #retryTimesWhenSendAsyncFailed} times before claiming sending failure, which may yield message duplication and
* application developers are the one to resolve this potential issue.
*
* @param msg request message to send
* @param requestCallback callback to execute on request completion.
* @param timeout request timeout
* @throws MQClientException if there is any client error.
* @throws RemotingException if there is any network-tier error.
* @throws InterruptedException if the thread is interrupted.
* @throws MQBrokerException if there is any broker error.
*/
@Override
public void request(final Message msg, final RequestCallback requestCallback, final long timeout)
throws MQClientException, RemotingException, InterruptedException, MQBrokerException {
msg.setTopic(withNamespace(msg.getTopic()));
this.defaultMQProducerImpl.request(msg, requestCallback, timeout);
}
/**
* Same to {@link #request(Message, long)} with message queue selector specified.
*
* @param msg request message to send
* @param selector message queue selector, through which we get target message queue to deliver message to.
* @param arg argument to work along with message queue selector.
* @param timeout timeout of request.
* @return reply message
* @throws MQClientException if there is any client error.
* @throws RemotingException if there is any network-tier error.
* @throws MQBrokerException if there is any broker error.
* @throws InterruptedException if the thread is interrupted.
* @throws RequestTimeoutException if request timeout.
*/
@Override
public Message request(final Message msg, final MessageQueueSelector selector, final Object arg,
final long timeout) throws MQClientException, RemotingException, MQBrokerException,
InterruptedException, RequestTimeoutException {
msg.setTopic(withNamespace(msg.getTopic()));
return this.defaultMQProducerImpl.request(msg, selector, arg, timeout);
}
/**
* Same to {@link #request(Message, RequestCallback, long)} with target message selector specified.
*
* @param msg requst message to send
* @param selector message queue selector, through which we get target message queue to deliver message to.
* @param arg argument to work along with message queue selector.
* @param requestCallback callback to execute on request completion.
* @param timeout timeout of request.
* @throws MQClientException if there is any client error.
* @throws RemotingException if there is any network-tier error.
* @throws InterruptedException if the thread is interrupted.
* @throws MQBrokerException if there is any broker error.
*/
@Override
public void request(final Message msg, final MessageQueueSelector selector, final Object arg,
final RequestCallback requestCallback, final long timeout) throws MQClientException, RemotingException,
InterruptedException, MQBrokerException {
msg.setTopic(withNamespace(msg.getTopic()));
this.defaultMQProducerImpl.request(msg, selector, arg, requestCallback, timeout);
}
/**
* Same to {@link #request(Message, long)} with target message queue specified in addition.
*
* @param msg request message to send
* @param mq target message queue.
* @param timeout request timeout
* @throws MQClientException if there is any client error.
* @throws RemotingException if there is any network-tier error.
* @throws MQBrokerException if there is any broker error.
* @throws InterruptedException if the thread is interrupted.
* @throws RequestTimeoutException if request timeout.
*/
@Override
public Message request(final Message msg, final MessageQueue mq, final long timeout)
throws MQClientException, RemotingException, MQBrokerException, InterruptedException, RequestTimeoutException {
msg.setTopic(withNamespace(msg.getTopic()));
return this.defaultMQProducerImpl.request(msg, mq, timeout);
}
/**
* Same to {@link #request(Message, RequestCallback, long)} with target message queue specified.
*
* @param msg request message to send
* @param mq target message queue.
* @param requestCallback callback to execute on request completion.
* @param timeout timeout of request.
* @throws MQClientException if there is any client error.
* @throws RemotingException if there is any network-tier error.
* @throws InterruptedException if the thread is interrupted.
* @throws MQBrokerException if there is any broker error.
*/
@Override
public void request(final Message msg, final MessageQueue mq, final RequestCallback requestCallback, long timeout)
throws MQClientException, RemotingException, InterruptedException, MQBrokerException {
msg.setTopic(withNamespace(msg.getTopic()));
this.defaultMQProducerImpl.request(msg, mq, requestCallback, timeout);
}
/** /**
* Same to {@link #sendOneway(Message)} with message queue selector specified. * Same to {@link #sendOneway(Message)} with message queue selector specified.
* *
......
...@@ -21,6 +21,7 @@ import java.util.List; ...@@ -21,6 +21,7 @@ import java.util.List;
import org.apache.rocketmq.client.MQAdmin; import org.apache.rocketmq.client.MQAdmin;
import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.exception.RequestTimeoutException;
import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.message.MessageQueue;
import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.remoting.exception.RemotingException;
...@@ -98,4 +99,26 @@ public interface MQProducer extends MQAdmin { ...@@ -98,4 +99,26 @@ public interface MQProducer extends MQAdmin {
SendResult send(final Collection<Message> msgs, final MessageQueue mq, final long timeout) SendResult send(final Collection<Message> msgs, final MessageQueue mq, final long timeout)
throws MQClientException, RemotingException, MQBrokerException, InterruptedException; throws MQClientException, RemotingException, MQBrokerException, InterruptedException;
//for rpc
Message request(final Message msg, final long timeout) throws RequestTimeoutException, MQClientException,
RemotingException, MQBrokerException, InterruptedException;
void request(final Message msg, final RequestCallback requestCallback, final long timeout)
throws MQClientException, RemotingException, InterruptedException, MQBrokerException;
Message request(final Message msg, final MessageQueueSelector selector, final Object arg,
final long timeout) throws RequestTimeoutException, MQClientException, RemotingException, MQBrokerException,
InterruptedException;
void request(final Message msg, final MessageQueueSelector selector, final Object arg,
final RequestCallback requestCallback,
final long timeout) throws MQClientException, RemotingException,
InterruptedException, MQBrokerException;
Message request(final Message msg, final MessageQueue mq, final long timeout)
throws RequestTimeoutException, MQClientException, RemotingException, MQBrokerException, InterruptedException;
void request(final Message msg, final MessageQueue mq, final RequestCallback requestCallback, long timeout)
throws MQClientException, RemotingException, MQBrokerException, InterruptedException;
} }
/*
* 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.message.Message;
public interface RequestCallback {
void onSuccess(final Message message);
void onException(final Throwable e);
}
/*
* 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 java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.rocketmq.client.common.ClientErrorCode;
import org.apache.rocketmq.client.exception.RequestTimeoutException;
import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.logging.InternalLogger;
public class RequestFutureTable {
private static InternalLogger log = ClientLogger.getLog();
private static ConcurrentHashMap<String, RequestResponseFuture> requestFutureTable = new ConcurrentHashMap<String, RequestResponseFuture>();
public static ConcurrentHashMap<String, RequestResponseFuture> getRequestFutureTable() {
return requestFutureTable;
}
public static void scanExpiredRequest() {
final List<RequestResponseFuture> rfList = new LinkedList<RequestResponseFuture>();
Iterator<Map.Entry<String, RequestResponseFuture>> it = requestFutureTable.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<String, RequestResponseFuture> next = it.next();
RequestResponseFuture rep = next.getValue();
if (rep.isTimeout()) {
it.remove();
rfList.add(rep);
log.warn("remove timeout request, CorrelationId={}" + rep.getCorrelationId());
}
}
for (RequestResponseFuture rf : rfList) {
try {
Throwable cause = new RequestTimeoutException(ClientErrorCode.REQUEST_TIMEOUT_EXCEPTION, "request timeout, no reply message.");
rf.setCause(cause);
rf.executeRequestCallback();
} catch (Throwable e) {
log.warn("scanResponseTable, operationComplete Exception", e);
}
}
}
}
/*
* 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 java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.rocketmq.common.message.Message;
public class RequestResponseFuture {
private final String correlationId;
private final RequestCallback requestCallback;
private final long beginTimestamp = System.currentTimeMillis();
private final Message requestMsg = null;
private long timeoutMillis;
private CountDownLatch countDownLatch = new CountDownLatch(1);
private volatile Message responseMsg = null;
private volatile boolean sendRequestOk = true;
private volatile Throwable cause = null;
public RequestResponseFuture(String correlationId, long timeoutMillis, RequestCallback requestCallback) {
this.correlationId = correlationId;
this.timeoutMillis = timeoutMillis;
this.requestCallback = requestCallback;
}
public void executeRequestCallback() {
if (requestCallback != null) {
if (sendRequestOk && cause == null) {
requestCallback.onSuccess(responseMsg);
} else {
requestCallback.onException(cause);
}
}
}
public boolean isTimeout() {
long diff = System.currentTimeMillis() - this.beginTimestamp;
return diff > this.timeoutMillis;
}
public Message waitResponseMessage(final long timeout) throws InterruptedException {
this.countDownLatch.await(timeout, TimeUnit.MILLISECONDS);
return this.responseMsg;
}
public void putResponseMessage(final Message responseMsg) {
this.responseMsg = responseMsg;
this.countDownLatch.countDown();
}
public String getCorrelationId() {
return correlationId;
}
public long getTimeoutMillis() {
return timeoutMillis;
}
public void setTimeoutMillis(long timeoutMillis) {
this.timeoutMillis = timeoutMillis;
}
public RequestCallback getRequestCallback() {
return requestCallback;
}
public long getBeginTimestamp() {
return beginTimestamp;
}
public CountDownLatch getCountDownLatch() {
return countDownLatch;
}
public void setCountDownLatch(CountDownLatch countDownLatch) {
this.countDownLatch = countDownLatch;
}
public Message getResponseMsg() {
return responseMsg;
}
public void setResponseMsg(Message responseMsg) {
this.responseMsg = responseMsg;
}
public boolean isSendRequestOk() {
return sendRequestOk;
}
public void setSendReqeustOk(boolean sendReqeustOk) {
this.sendRequestOk = sendReqeustOk;
}
public Message getRequestMsg() {
return requestMsg;
}
public Throwable getCause() {
return cause;
}
public void setCause(Throwable cause) {
this.cause = cause;
}
}
/*
* 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.utils;
import org.apache.rocketmq.client.common.ClientErrorCode;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageAccessor;
import org.apache.rocketmq.common.message.MessageConst;
public class MessageUtil {
public static Message createReplyMessage(final Message requestMessage, final byte[] body) throws MQClientException {
if (requestMessage != null) {
Message replyMessage = new Message();
String cluster = requestMessage.getProperty(MessageConst.PROPERTY_CLUSTER);
String replyTo = requestMessage.getProperty(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT);
String correlationId = requestMessage.getProperty(MessageConst.PROPERTY_CORRELATION_ID);
String ttl = requestMessage.getProperty(MessageConst.PROPERTY_MESSAGE_TTL);
replyMessage.setBody(body);
if (cluster != null) {
String replyTopic = MixAll.getReplyTopic(cluster);
replyMessage.setTopic(replyTopic);
MessageAccessor.putProperty(replyMessage, MessageConst.PROPERTY_MESSAGE_TYPE, MixAll.REPLY_MESSAGE_FLAG);
MessageAccessor.putProperty(replyMessage, MessageConst.PROPERTY_CORRELATION_ID, correlationId);
MessageAccessor.putProperty(replyMessage, MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT, replyTo);
MessageAccessor.putProperty(replyMessage, MessageConst.PROPERTY_MESSAGE_TTL, ttl);
return replyMessage;
} else {
throw new MQClientException(ClientErrorCode.CREATE_REPLY_MESSAGE_EXCEPTION, "create reply message fail, requestMessage error, property[" + MessageConst.PROPERTY_CLUSTER + "] is null.");
}
}
throw new MQClientException(ClientErrorCode.CREATE_REPLY_MESSAGE_EXCEPTION, "create reply message fail, requestMessage cannot be null.");
}
public static String getReplyToClient(final Message msg) {
return msg.getProperty(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT);
}
}
...@@ -189,6 +189,36 @@ public class DefaultLitePullConsumerTest { ...@@ -189,6 +189,36 @@ public class DefaultLitePullConsumerTest {
litePullConsumer.shutdown(); litePullConsumer.shutdown();
} }
@Test
public void testSeek_SeekToBegin() throws Exception {
DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer();
when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L);
when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(500L);
MessageQueue messageQueue = createMessageQueue();
litePullConsumer.assign(Collections.singletonList(messageQueue));
litePullConsumer.seekToBegin(messageQueue);
Field field = DefaultLitePullConsumerImpl.class.getDeclaredField("assignedMessageQueue");
field.setAccessible(true);
AssignedMessageQueue assignedMessageQueue = (AssignedMessageQueue) field.get(litePullConsumerImpl);
assertEquals(assignedMessageQueue.getSeekOffset(messageQueue), 0L);
litePullConsumer.shutdown();
}
@Test
public void testSeek_SeekToEnd() throws Exception {
DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer();
when(mQAdminImpl.minOffset(any(MessageQueue.class))).thenReturn(0L);
when(mQAdminImpl.maxOffset(any(MessageQueue.class))).thenReturn(500L);
MessageQueue messageQueue = createMessageQueue();
litePullConsumer.assign(Collections.singletonList(messageQueue));
litePullConsumer.seekToEnd(messageQueue);
Field field = DefaultLitePullConsumerImpl.class.getDeclaredField("assignedMessageQueue");
field.setAccessible(true);
AssignedMessageQueue assignedMessageQueue = (AssignedMessageQueue) field.get(litePullConsumerImpl);
assertEquals(assignedMessageQueue.getSeekOffset(messageQueue), 500L);
litePullConsumer.shutdown();
}
@Test @Test
public void testSeek_SeekOffsetIllegal() throws Exception { public void testSeek_SeekOffsetIllegal() throws Exception {
DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer(); DefaultLitePullConsumer litePullConsumer = createStartLitePullConsumer();
......
...@@ -42,6 +42,7 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand; ...@@ -42,6 +42,7 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.mockito.Matchers;
import org.mockito.Mock; import org.mockito.Mock;
import org.mockito.invocation.InvocationOnMock; import org.mockito.invocation.InvocationOnMock;
import org.mockito.junit.MockitoJUnitRunner; import org.mockito.junit.MockitoJUnitRunner;
...@@ -164,7 +165,7 @@ public class MQClientAPIImplTest { ...@@ -164,7 +165,7 @@ public class MQClientAPIImplTest {
public Object answer(InvocationOnMock mock) throws Throwable { public Object answer(InvocationOnMock mock) throws Throwable {
InvokeCallback callback = mock.getArgument(3); InvokeCallback callback = mock.getArgument(3);
RemotingCommand request = mock.getArgument(1); RemotingCommand request = mock.getArgument(1);
ResponseFuture responseFuture = new ResponseFuture(null,request.getOpaque(), 3 * 1000, null, null); ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
responseFuture.setResponseCommand(createSuccessResponse(request)); responseFuture.setResponseCommand(createSuccessResponse(request));
callback.operationComplete(responseFuture); callback.operationComplete(responseFuture);
return null; return null;
...@@ -289,6 +290,7 @@ public class MQClientAPIImplTest { ...@@ -289,6 +290,7 @@ public class MQClientAPIImplTest {
assertThat(ex.getErrorMessage()).isEqualTo("corresponding to accessConfig has been deleted failed"); assertThat(ex.getErrorMessage()).isEqualTo("corresponding to accessConfig has been deleted failed");
} }
} }
@Test @Test
public void testResumeCheckHalfMessage_WithException() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { public void testResumeCheckHalfMessage_WithException() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
doAnswer(new Answer() { doAnswer(new Answer() {
...@@ -322,6 +324,38 @@ public class MQClientAPIImplTest { ...@@ -322,6 +324,38 @@ public class MQClientAPIImplTest {
assertThat(result).isEqualTo(true); assertThat(result).isEqualTo(true);
} }
@Test
public void testSendMessageTypeofReply() throws Exception {
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock mock) throws Throwable {
InvokeCallback callback = mock.getArgument(3);
RemotingCommand request = mock.getArgument(1);
ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
responseFuture.setResponseCommand(createSuccessResponse(request));
callback.operationComplete(responseFuture);
return null;
}
}).when(remotingClient).invokeAsync(Matchers.anyString(), Matchers.any(RemotingCommand.class), Matchers.anyLong(), Matchers.any(InvokeCallback.class));
SendMessageContext sendMessageContext = new SendMessageContext();
sendMessageContext.setProducer(new DefaultMQProducerImpl(new DefaultMQProducer()));
msg.getProperties().put("MSG_TYPE", "reply");
mqClientAPI.sendMessage(brokerAddr, brokerName, msg, new SendMessageRequestHeader(), 3 * 1000, CommunicationMode.ASYNC,
new SendCallback() {
@Override
public void onSuccess(SendResult sendResult) {
assertThat(sendResult.getSendStatus()).isEqualTo(SendStatus.SEND_OK);
assertThat(sendResult.getOffsetMsgId()).isEqualTo("123");
assertThat(sendResult.getQueueOffset()).isEqualTo(123L);
assertThat(sendResult.getMessageQueue().getQueueId()).isEqualTo(1);
}
@Override
public void onException(Throwable e) {
}
}, null, null, 0, sendMessageContext, defaultMQProducerImpl);
}
private RemotingCommand createResumeSuccessResponse(RemotingCommand request) { private RemotingCommand createResumeSuccessResponse(RemotingCommand request) {
RemotingCommand response = RemotingCommand.createResponseCommand(null); RemotingCommand response = RemotingCommand.createResponseCommand(null);
response.setCode(ResponseCode.SUCCESS); response.setCode(ResponseCode.SUCCESS);
......
/*
* 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.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyContext;
import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus;
import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.protocol.body.CMResult;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertTrue;
public class ConsumeMessageOrderlyServiceTest {
private String consumerGroup;
private String topic = "FooBar";
private String brokerName = "BrokerA";
private DefaultMQPushConsumer pushConsumer;
@Before
public void init() throws Exception {
consumerGroup = "FooBarGroup" + System.currentTimeMillis();
pushConsumer = new DefaultMQPushConsumer(consumerGroup);
}
@Test
public void testConsumeMessageDirectly_WithNoException() {
Map<ConsumeOrderlyStatus, CMResult> map = new HashMap();
map.put(ConsumeOrderlyStatus.SUCCESS, CMResult.CR_SUCCESS);
map.put(ConsumeOrderlyStatus.SUSPEND_CURRENT_QUEUE_A_MOMENT, CMResult.CR_LATER);
map.put(ConsumeOrderlyStatus.COMMIT, CMResult.CR_COMMIT);
map.put(ConsumeOrderlyStatus.ROLLBACK, CMResult.CR_ROLLBACK);
map.put(null, CMResult.CR_RETURN_NULL);
for (ConsumeOrderlyStatus consumeOrderlyStatus : map.keySet()) {
final ConsumeOrderlyStatus status = consumeOrderlyStatus;
MessageListenerOrderly listenerOrderly = new MessageListenerOrderly() {
@Override
public ConsumeOrderlyStatus consumeMessage(List<MessageExt> msgs, ConsumeOrderlyContext context) {
return status;
}
};
ConsumeMessageOrderlyService consumeMessageOrderlyService = new ConsumeMessageOrderlyService(pushConsumer.getDefaultMQPushConsumerImpl(), listenerOrderly);
MessageExt msg = new MessageExt();
msg.setTopic(topic);
assertTrue(consumeMessageOrderlyService.consumeMessageDirectly(msg, brokerName).getConsumeResult().equals(map.get(consumeOrderlyStatus)));
}
}
@Test
public void testConsumeMessageDirectly_WithException() {
MessageListenerOrderly listenerOrderly = new MessageListenerOrderly() {
@Override
public ConsumeOrderlyStatus consumeMessage(List<MessageExt> msgs, ConsumeOrderlyContext context) {
throw new RuntimeException();
}
};
ConsumeMessageOrderlyService consumeMessageOrderlyService = new ConsumeMessageOrderlyService(pushConsumer.getDefaultMQPushConsumerImpl(), listenerOrderly);
MessageExt msg = new MessageExt();
msg.setTopic(topic);
assertTrue(consumeMessageOrderlyService.consumeMessageDirectly(msg, brokerName).getConsumeResult().equals(CMResult.CR_THROW_EXCEPTION));
}
}
...@@ -21,15 +21,18 @@ import java.util.ArrayList; ...@@ -21,15 +21,18 @@ import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.ClientConfig;
import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.exception.RequestTimeoutException;
import org.apache.rocketmq.client.hook.SendMessageContext; import org.apache.rocketmq.client.hook.SendMessageContext;
import org.apache.rocketmq.client.hook.SendMessageHook; import org.apache.rocketmq.client.hook.SendMessageHook;
import org.apache.rocketmq.client.impl.CommunicationMode; import org.apache.rocketmq.client.impl.CommunicationMode;
...@@ -45,6 +48,7 @@ import org.apache.rocketmq.common.protocol.route.BrokerData; ...@@ -45,6 +48,7 @@ import org.apache.rocketmq.common.protocol.route.BrokerData;
import org.apache.rocketmq.common.protocol.route.QueueData; import org.apache.rocketmq.common.protocol.route.QueueData;
import org.apache.rocketmq.common.protocol.route.TopicRouteData; import org.apache.rocketmq.common.protocol.route.TopicRouteData;
import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.remoting.exception.RemotingException;
import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
import org.apache.rocketmq.remoting.netty.NettyRemotingClient; import org.apache.rocketmq.remoting.netty.NettyRemotingClient;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
...@@ -52,7 +56,9 @@ import org.junit.Test; ...@@ -52,7 +56,9 @@ import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.mockito.Mock; import org.mockito.Mock;
import org.mockito.Spy; import org.mockito.Spy;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.junit.MockitoJUnitRunner; 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.assertThat;
import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown; import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown;
...@@ -168,6 +174,7 @@ public class DefaultMQProducerTest { ...@@ -168,6 +174,7 @@ public class DefaultMQProducerTest {
@Test @Test
public void testSendMessageAsync_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { public void testSendMessageAsync_Success() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
final CountDownLatch countDownLatch = new CountDownLatch(1); final CountDownLatch countDownLatch = new CountDownLatch(1);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
producer.send(message, new SendCallback() { producer.send(message, new SendCallback() {
@Override @Override
public void onSuccess(SendResult sendResult) { public void onSuccess(SendResult sendResult) {
...@@ -190,9 +197,11 @@ public class DefaultMQProducerTest { ...@@ -190,9 +197,11 @@ public class DefaultMQProducerTest {
final AtomicInteger cc = new AtomicInteger(0); final AtomicInteger cc = new AtomicInteger(0);
final CountDownLatch countDownLatch = new CountDownLatch(6); final CountDownLatch countDownLatch = new CountDownLatch(6);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
SendCallback sendCallback = new SendCallback() { SendCallback sendCallback = new SendCallback() {
@Override @Override
public void onSuccess(SendResult sendResult) { public void onSuccess(SendResult sendResult) {
countDownLatch.countDown();
} }
@Override @Override
...@@ -213,20 +222,21 @@ public class DefaultMQProducerTest { ...@@ -213,20 +222,21 @@ public class DefaultMQProducerTest {
message.setTopic("test"); message.setTopic("test");
message.setBody("hello world".getBytes()); message.setBody("hello world".getBytes());
producer.send(new Message(), sendCallback); producer.send(new Message(), sendCallback);
producer.send(message, sendCallback, 1000);
producer.send(message, new MessageQueue(), sendCallback); producer.send(message, new MessageQueue(), sendCallback);
producer.send(new Message(), new MessageQueue(), sendCallback, 1000); producer.send(new Message(), new MessageQueue(), sendCallback, 1000);
producer.send(new Message(), messageQueueSelector, null, sendCallback); producer.send(new Message(), messageQueueSelector, null, sendCallback);
producer.send(message, messageQueueSelector, null, sendCallback, 1000); producer.send(message, messageQueueSelector, null, sendCallback, 1000);
//this message is send success
producer.send(message, sendCallback, 1000);
countDownLatch.await(3000L, TimeUnit.MILLISECONDS); countDownLatch.await(3000L, TimeUnit.MILLISECONDS);
assertThat(cc.get()).isEqualTo(6); assertThat(cc.get()).isEqualTo(5);
} }
@Test @Test
public void testSendMessageAsync_BodyCompressed() throws RemotingException, InterruptedException, MQBrokerException, MQClientException { public void testSendMessageAsync_BodyCompressed() throws RemotingException, InterruptedException, MQBrokerException, MQClientException {
final CountDownLatch countDownLatch = new CountDownLatch(1); final CountDownLatch countDownLatch = new CountDownLatch(1);
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
producer.send(bigMessage, new SendCallback() { producer.send(bigMessage, new SendCallback() {
@Override @Override
public void onSuccess(SendResult sendResult) { public void onSuccess(SendResult sendResult) {
...@@ -238,7 +248,6 @@ public class DefaultMQProducerTest { ...@@ -238,7 +248,6 @@ public class DefaultMQProducerTest {
@Override @Override
public void onException(Throwable e) { public void onException(Throwable e) {
countDownLatch.countDown();
} }
}); });
countDownLatch.await(3000L, TimeUnit.MILLISECONDS); countDownLatch.await(3000L, TimeUnit.MILLISECONDS);
...@@ -312,6 +321,101 @@ public class DefaultMQProducerTest { ...@@ -312,6 +321,101 @@ public class DefaultMQProducerTest {
assertThat(remotingClient.getCallbackExecutor()).isEqualTo(customized); assertThat(remotingClient.getCallbackExecutor()).isEqualTo(customized);
} }
@Test
public void testRequestMessage() throws RemotingException, RequestTimeoutException, MQClientException, InterruptedException, MQBrokerException {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
final AtomicBoolean finish = new AtomicBoolean(false);
new Thread(new Runnable() {
@Override public void run() {
ConcurrentHashMap<String, RequestResponseFuture> responseMap = RequestFutureTable.getRequestFutureTable();
assertThat(responseMap).isNotNull();
while (!finish.get()) {
try {
Thread.sleep(10);
} catch (InterruptedException e) {
}
for (Map.Entry<String, RequestResponseFuture> entry : responseMap.entrySet()) {
RequestResponseFuture future = entry.getValue();
future.putResponseMessage(message);
}
}
}
}).start();
Message result = producer.request(message, 3 * 1000L);
finish.getAndSet(true);
assertThat(result.getTopic()).isEqualTo("FooBar");
assertThat(result.getBody()).isEqualTo(new byte[] {'a'});
}
@Test(expected = RequestTimeoutException.class)
public void testRequestMessage_RequestTimeoutException() throws RemotingException, RequestTimeoutException, MQClientException, InterruptedException, MQBrokerException {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
Message result = producer.request(message, 3 * 1000L);
}
@Test
public void testAsyncRequest_OnSuccess() throws Exception {
when(mQClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRoute());
final CountDownLatch countDownLatch = new CountDownLatch(1);
RequestCallback requestCallback = new RequestCallback() {
@Override public void onSuccess(Message message) {
assertThat(message.getTopic()).isEqualTo("FooBar");
assertThat(message.getBody()).isEqualTo(new byte[] {'a'});
assertThat(message.getFlag()).isEqualTo(1);
countDownLatch.countDown();
}
@Override public void onException(Throwable e) {
}
};
producer.request(message, requestCallback, 3 * 1000L);
ConcurrentHashMap<String, RequestResponseFuture> responseMap = RequestFutureTable.getRequestFutureTable();
assertThat(responseMap).isNotNull();
for (Map.Entry<String, RequestResponseFuture> entry : responseMap.entrySet()) {
RequestResponseFuture future = entry.getValue();
future.setSendReqeustOk(true);
message.setFlag(1);
future.getRequestCallback().onSuccess(message);
}
countDownLatch.await(3000L, TimeUnit.MILLISECONDS);
}
@Test
public void testAsyncRequest_OnException() throws Exception {
final AtomicInteger cc = new AtomicInteger(0);
final CountDownLatch countDownLatch = new CountDownLatch(1);
RequestCallback requestCallback = new RequestCallback() {
@Override public void onSuccess(Message message) {
}
@Override public void onException(Throwable e) {
cc.incrementAndGet();
countDownLatch.countDown();
}
};
MessageQueueSelector messageQueueSelector = new MessageQueueSelector() {
@Override
public MessageQueue select(List<MessageQueue> mqs, Message msg, Object arg) {
return null;
}
};
try {
producer.request(message, requestCallback, 3 * 1000L);
failBecauseExceptionWasNotThrown(Exception.class);
} catch (Exception e) {
ConcurrentHashMap<String, RequestResponseFuture> responseMap = RequestFutureTable.getRequestFutureTable();
assertThat(responseMap).isNotNull();
for (Map.Entry<String, RequestResponseFuture> entry : responseMap.entrySet()) {
RequestResponseFuture future = entry.getValue();
future.getRequestCallback().onException(e);
}
}
countDownLatch.await(3000L, TimeUnit.MILLISECONDS);
assertThat(cc.get()).isEqualTo(1);
}
public static TopicRouteData createTopicRoute() { public static TopicRouteData createTopicRoute() {
TopicRouteData topicRouteData = new TopicRouteData(); TopicRouteData topicRouteData = new TopicRouteData();
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.client.producer;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.rocketmq.common.message.Message;
import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
public class RequestResponseFutureTest {
@Test
public void testExecuteRequestCallback() throws Exception {
final AtomicInteger cc = new AtomicInteger(0);
RequestResponseFuture future = new RequestResponseFuture(UUID.randomUUID().toString(), 3 * 1000L, new RequestCallback() {
@Override public void onSuccess(Message message) {
cc.incrementAndGet();
}
@Override public void onException(Throwable e) {
}
});
future.setSendReqeustOk(true);
future.executeRequestCallback();
assertThat(cc.get()).isEqualTo(1);
}
}
/*
* 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.utils;
import java.util.HashMap;
import java.util.Map;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageAccessor;
import org.apache.rocketmq.common.message.MessageConst;
import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown;
public class MessageUtilsTest {
@Test
public void testCreateReplyMessage() throws MQClientException {
Message msg = MessageUtil.createReplyMessage(createReplyMessage("clusterName"), new byte[] {'a'});
assertThat(msg.getTopic()).isEqualTo("clusterName" + "_" + MixAll.REPLY_TOPIC_POSTFIX);
assertThat(msg.getProperty(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT)).isEqualTo("127.0.0.1");
assertThat(msg.getProperty(MessageConst.PROPERTY_MESSAGE_TTL)).isEqualTo("3000");
}
@Test
public void testCreateReplyMessage_Exception() throws MQClientException {
try {
Message msg = MessageUtil.createReplyMessage(createReplyMessage(null), new byte[] {'a'});
failBecauseExceptionWasNotThrown(MQClientException.class);
} catch (MQClientException e) {
assertThat(e).hasMessageContaining("create reply message fail, requestMessage error, property[" + MessageConst.PROPERTY_CLUSTER + "] is null.");
}
}
@Test
public void testCreateReplyMessage_reqMsgIsNull() throws MQClientException {
try {
Message msg = MessageUtil.createReplyMessage(null, new byte[] {'a'});
failBecauseExceptionWasNotThrown(MQClientException.class);
} catch (MQClientException e) {
assertThat(e).hasMessageContaining("create reply message fail, requestMessage cannot be null.");
}
}
@Test
public void testGetReplyToClient() throws MQClientException {
Message msg = createReplyMessage("clusterName");
String replyToClient = MessageUtil.getReplyToClient(msg);
assertThat(replyToClient).isNotNull();
assertThat(replyToClient).isEqualTo("127.0.0.1");
}
private Message createReplyMessage(String clusterName) {
Message requestMessage = new Message();
Map map = new HashMap<String, String>();
map.put(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT, "127.0.0.1");
map.put(MessageConst.PROPERTY_CLUSTER, clusterName);
map.put(MessageConst.PROPERTY_MESSAGE_TTL, "3000");
MessageAccessor.setProperties(requestMessage, map);
return requestMessage;
}
}
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -61,6 +61,7 @@ public class BrokerConfig { ...@@ -61,6 +61,7 @@ public class BrokerConfig {
*/ */
private int sendMessageThreadPoolNums = 1; //16 + Runtime.getRuntime().availableProcessors() * 4; private int sendMessageThreadPoolNums = 1; //16 + Runtime.getRuntime().availableProcessors() * 4;
private int pullMessageThreadPoolNums = 16 + Runtime.getRuntime().availableProcessors() * 2; private int pullMessageThreadPoolNums = 16 + Runtime.getRuntime().availableProcessors() * 2;
private int processReplyMessageThreadPoolNums = 16 + Runtime.getRuntime().availableProcessors() * 2;
private int queryMessageThreadPoolNums = 8 + Runtime.getRuntime().availableProcessors(); private int queryMessageThreadPoolNums = 8 + Runtime.getRuntime().availableProcessors();
private int adminBrokerThreadPoolNums = 16; private int adminBrokerThreadPoolNums = 16;
...@@ -83,6 +84,7 @@ public class BrokerConfig { ...@@ -83,6 +84,7 @@ public class BrokerConfig {
private boolean fetchNamesrvAddrByAddressServer = false; private boolean fetchNamesrvAddrByAddressServer = false;
private int sendThreadPoolQueueCapacity = 10000; private int sendThreadPoolQueueCapacity = 10000;
private int pullThreadPoolQueueCapacity = 100000; private int pullThreadPoolQueueCapacity = 100000;
private int replyThreadPoolQueueCapacity = 10000;
private int queryThreadPoolQueueCapacity = 20000; private int queryThreadPoolQueueCapacity = 20000;
private int clientManagerThreadPoolQueueCapacity = 1000000; private int clientManagerThreadPoolQueueCapacity = 1000000;
private int consumerManagerThreadPoolQueueCapacity = 1000000; private int consumerManagerThreadPoolQueueCapacity = 1000000;
...@@ -180,6 +182,8 @@ public class BrokerConfig { ...@@ -180,6 +182,8 @@ public class BrokerConfig {
@ImportantField @ImportantField
private boolean aclEnable = false; private boolean aclEnable = false;
private boolean storeReplyMessageEnable = true;
public static String localHostName() { public static String localHostName() {
try { try {
return InetAddress.getLocalHost().getHostName(); return InetAddress.getLocalHost().getHostName();
...@@ -374,6 +378,14 @@ public class BrokerConfig { ...@@ -374,6 +378,14 @@ public class BrokerConfig {
this.pullMessageThreadPoolNums = pullMessageThreadPoolNums; this.pullMessageThreadPoolNums = pullMessageThreadPoolNums;
} }
public int getProcessReplyMessageThreadPoolNums() {
return processReplyMessageThreadPoolNums;
}
public void setProcessReplyMessageThreadPoolNums(int processReplyMessageThreadPoolNums) {
this.processReplyMessageThreadPoolNums = processReplyMessageThreadPoolNums;
}
public int getQueryMessageThreadPoolNums() { public int getQueryMessageThreadPoolNums() {
return queryMessageThreadPoolNums; return queryMessageThreadPoolNums;
} }
...@@ -470,6 +482,14 @@ public class BrokerConfig { ...@@ -470,6 +482,14 @@ public class BrokerConfig {
this.pullThreadPoolQueueCapacity = pullThreadPoolQueueCapacity; this.pullThreadPoolQueueCapacity = pullThreadPoolQueueCapacity;
} }
public int getReplyThreadPoolQueueCapacity() {
return replyThreadPoolQueueCapacity;
}
public void setReplyThreadPoolQueueCapacity(int replyThreadPoolQueueCapacity) {
this.replyThreadPoolQueueCapacity = replyThreadPoolQueueCapacity;
}
public int getQueryThreadPoolQueueCapacity() { public int getQueryThreadPoolQueueCapacity() {
return queryThreadPoolQueueCapacity; return queryThreadPoolQueueCapacity;
} }
...@@ -749,7 +769,7 @@ public class BrokerConfig { ...@@ -749,7 +769,7 @@ public class BrokerConfig {
public void setMsgTraceTopicName(String msgTraceTopicName) { public void setMsgTraceTopicName(String msgTraceTopicName) {
this.msgTraceTopicName = msgTraceTopicName; this.msgTraceTopicName = msgTraceTopicName;
} }
public boolean isTraceTopicEnable() { public boolean isTraceTopicEnable() {
return traceTopicEnable; return traceTopicEnable;
} }
...@@ -765,4 +785,12 @@ public class BrokerConfig { ...@@ -765,4 +785,12 @@ public class BrokerConfig {
public void setAclEnable(boolean aclEnable) { public void setAclEnable(boolean aclEnable) {
this.aclEnable = aclEnable; this.aclEnable = aclEnable;
} }
public boolean isStoreReplyMessageEnable() {
return storeReplyMessageEnable;
}
public void setStoreReplyMessageEnable(boolean storeReplyMessageEnable) {
this.storeReplyMessageEnable = storeReplyMessageEnable;
}
} }
...@@ -18,7 +18,7 @@ package org.apache.rocketmq.common; ...@@ -18,7 +18,7 @@ package org.apache.rocketmq.common;
public class MQVersion { public class MQVersion {
public static final int CURRENT_VERSION = Version.V4_5_2.ordinal(); public static final int CURRENT_VERSION = Version.V4_6_0.ordinal();
public static String getVersionDesc(int value) { public static String getVersionDesc(int value) {
int length = Version.values().length; int length = Version.values().length;
......
...@@ -45,8 +45,6 @@ import org.apache.rocketmq.logging.InternalLogger; ...@@ -45,8 +45,6 @@ import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.logging.InternalLoggerFactory;
public class MixAll { public class MixAll {
private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.COMMON_LOGGER_NAME);
public static final String ROCKETMQ_HOME_ENV = "ROCKETMQ_HOME"; public static final String ROCKETMQ_HOME_ENV = "ROCKETMQ_HOME";
public static final String ROCKETMQ_HOME_PROPERTY = "rocketmq.home.dir"; public static final String ROCKETMQ_HOME_PROPERTY = "rocketmq.home.dir";
public static final String NAMESRV_ADDR_ENV = "NAMESRV_ADDR"; public static final String NAMESRV_ADDR_ENV = "NAMESRV_ADDR";
...@@ -74,27 +72,26 @@ public class MixAll { ...@@ -74,27 +72,26 @@ public class MixAll {
public static final String CID_ONSAPI_OWNER_GROUP = "CID_ONSAPI_OWNER"; public static final String CID_ONSAPI_OWNER_GROUP = "CID_ONSAPI_OWNER";
public static final String CID_ONSAPI_PULL_GROUP = "CID_ONSAPI_PULL"; public static final String CID_ONSAPI_PULL_GROUP = "CID_ONSAPI_PULL";
public static final String CID_RMQ_SYS_PREFIX = "CID_RMQ_SYS_"; public static final String CID_RMQ_SYS_PREFIX = "CID_RMQ_SYS_";
public static final List<String> LOCAL_INET_ADDRESS = getLocalInetAddress(); public static final List<String> LOCAL_INET_ADDRESS = getLocalInetAddress();
public static final String LOCALHOST = localhost(); public static final String LOCALHOST = localhost();
public static final String DEFAULT_CHARSET = "UTF-8"; public static final String DEFAULT_CHARSET = "UTF-8";
public static final long MASTER_ID = 0L; public static final long MASTER_ID = 0L;
public static final long CURRENT_JVM_PID = getPID(); public static final long CURRENT_JVM_PID = getPID();
public static final String RETRY_GROUP_TOPIC_PREFIX = "%RETRY%"; public static final String RETRY_GROUP_TOPIC_PREFIX = "%RETRY%";
public static final String DLQ_GROUP_TOPIC_PREFIX = "%DLQ%"; public static final String DLQ_GROUP_TOPIC_PREFIX = "%DLQ%";
public static final String REPLY_TOPIC_POSTFIX = "REPLY_TOPIC";
public static final String SYSTEM_TOPIC_PREFIX = "rmq_sys_"; public static final String SYSTEM_TOPIC_PREFIX = "rmq_sys_";
public static final String UNIQUE_MSG_QUERY_FLAG = "_UNIQUE_KEY_QUERY"; public static final String UNIQUE_MSG_QUERY_FLAG = "_UNIQUE_KEY_QUERY";
public static final String DEFAULT_TRACE_REGION_ID = "DefaultRegion"; public static final String DEFAULT_TRACE_REGION_ID = "DefaultRegion";
public static final String CONSUME_CONTEXT_TYPE = "ConsumeContextType"; public static final String CONSUME_CONTEXT_TYPE = "ConsumeContextType";
public static final String RMQ_SYS_TRANS_HALF_TOPIC = "RMQ_SYS_TRANS_HALF_TOPIC"; public static final String RMQ_SYS_TRANS_HALF_TOPIC = "RMQ_SYS_TRANS_HALF_TOPIC";
public static final String RMQ_SYS_TRACE_TOPIC = "RMQ_SYS_TRACE_TOPIC"; public static final String RMQ_SYS_TRACE_TOPIC = "RMQ_SYS_TRACE_TOPIC";
public static final String RMQ_SYS_TRANS_OP_HALF_TOPIC = "RMQ_SYS_TRANS_OP_HALF_TOPIC"; public static final String RMQ_SYS_TRANS_OP_HALF_TOPIC = "RMQ_SYS_TRANS_OP_HALF_TOPIC";
public static final String TRANS_CHECK_MAX_TIME_TOPIC = "TRANS_CHECK_MAX_TIME_TOPIC"; public static final String TRANS_CHECK_MAX_TIME_TOPIC = "TRANS_CHECK_MAX_TIME_TOPIC";
public static final String CID_SYS_RMQ_TRANS = "CID_RMQ_SYS_TRANS"; public static final String CID_SYS_RMQ_TRANS = "CID_RMQ_SYS_TRANS";
public static final String ACL_CONF_TOOLS_FILE = "/conf/tools.yml"; public static final String 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 String getWSAddr() { public static String getWSAddr() {
String wsDomainName = System.getProperty("rocketmq.namesrv.domain", DEFAULT_NAMESRV_ADDR_LOOKUP); String wsDomainName = System.getProperty("rocketmq.namesrv.domain", DEFAULT_NAMESRV_ADDR_LOOKUP);
...@@ -110,6 +107,10 @@ public class MixAll { ...@@ -110,6 +107,10 @@ public class MixAll {
return RETRY_GROUP_TOPIC_PREFIX + consumerGroup; return RETRY_GROUP_TOPIC_PREFIX + consumerGroup;
} }
public static String getReplyTopic(final String clusterName) {
return clusterName + "_" + REPLY_TOPIC_POSTFIX;
}
public static boolean isSysConsumerGroup(final String consumerGroup) { public static boolean isSysConsumerGroup(final String consumerGroup) {
return consumerGroup.startsWith(CID_RMQ_SYS_PREFIX); return consumerGroup.startsWith(CID_RMQ_SYS_PREFIX);
} }
......
...@@ -440,13 +440,11 @@ public class UtilAll { ...@@ -440,13 +440,11 @@ public class UtilAll {
return false; return false;
} }
public static boolean isInternalV6IP(byte[] ip) { public static boolean isInternalV6IP(InetAddress inetAddr) {
if (ip.length != 16) { if (inetAddr.isAnyLocalAddress() // Wild card ipv6
throw new RuntimeException("illegal ipv6 bytes"); || inetAddr.isLinkLocalAddress() // Single broadcast ipv6 address: fe80:xx:xx...
} || inetAddr.isLoopbackAddress() //Loopback ipv6 address
|| inetAddr.isSiteLocalAddress()) { // Site local ipv6 address: fec0:xx:xx...
//FEC0:0000:0000:0000:0000:0000:0000:0000/10
if (ip[0] == (byte) 254 && ip[1] >= (byte) 192) {
return true; return true;
} }
return false; return false;
...@@ -457,9 +455,6 @@ public class UtilAll { ...@@ -457,9 +455,6 @@ public class UtilAll {
throw new RuntimeException("illegal ipv4 bytes"); throw new RuntimeException("illegal ipv4 bytes");
} }
// if (ip[0] == (byte)30 && ip[1] == (byte)10 && ip[2] == (byte)163 && ip[3] == (byte)120) {
// }
if (ip[0] >= (byte) 1 && ip[0] <= (byte) 126) { if (ip[0] >= (byte) 1 && ip[0] <= (byte) 126) {
if (ip[1] == (byte) 1 && ip[2] == (byte) 1 && ip[3] == (byte) 1) { if (ip[1] == (byte) 1 && ip[2] == (byte) 1 && ip[3] == (byte) 1) {
return false; return false;
...@@ -550,7 +545,7 @@ public class UtilAll { ...@@ -550,7 +545,7 @@ public class UtilAll {
byte[] ipByte = ip.getAddress(); byte[] ipByte = ip.getAddress();
if (ipByte.length == 16) { if (ipByte.length == 16) {
if (ipV6Check(ipByte)) { if (ipV6Check(ipByte)) {
if (!isInternalV6IP(ipByte)) { if (!isInternalV6IP(ip)) {
return ipByte; return ipByte;
} else if (internalIP == null) { } else if (internalIP == null) {
internalIP = ipByte; internalIP = ipByte;
......
...@@ -45,6 +45,13 @@ public class MessageConst { ...@@ -45,6 +45,13 @@ public class MessageConst {
public static final String PROPERTY_TRANSACTION_CHECK_TIMES = "TRANSACTION_CHECK_TIMES"; public static final String PROPERTY_TRANSACTION_CHECK_TIMES = "TRANSACTION_CHECK_TIMES";
public static final String PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS = "CHECK_IMMUNITY_TIME_IN_SECONDS"; public static final String PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS = "CHECK_IMMUNITY_TIME_IN_SECONDS";
public static final String PROPERTY_INSTANCE_ID = "INSTANCE_ID"; public static final String PROPERTY_INSTANCE_ID = "INSTANCE_ID";
public static final String PROPERTY_CORRELATION_ID = "CORRELATION_ID";
public static final String PROPERTY_MESSAGE_REPLY_TO_CLIENT = "REPLY_TO_CLIENT";
public static final String PROPERTY_MESSAGE_TTL = "TTL";
public static final String PROPERTY_REPLY_MESSAGE_ARRIVE_TIME = "ARRIVE_TIME";
public static final String PROPERTY_PUSH_REPLY_TIME = "PUSH_REPLY_TIME";
public static final String PROPERTY_CLUSTER = "CLUSTER";
public static final String PROPERTY_MESSAGE_TYPE = "MSG_TYPE";
public static final String KEY_SEPARATOR = " "; public static final String KEY_SEPARATOR = " ";
...@@ -74,5 +81,12 @@ public class MessageConst { ...@@ -74,5 +81,12 @@ public class MessageConst {
STRING_HASH_SET.add(PROPERTY_MAX_RECONSUME_TIMES); STRING_HASH_SET.add(PROPERTY_MAX_RECONSUME_TIMES);
STRING_HASH_SET.add(PROPERTY_CONSUME_START_TIMESTAMP); STRING_HASH_SET.add(PROPERTY_CONSUME_START_TIMESTAMP);
STRING_HASH_SET.add(PROPERTY_INSTANCE_ID); STRING_HASH_SET.add(PROPERTY_INSTANCE_ID);
STRING_HASH_SET.add(PROPERTY_CORRELATION_ID);
STRING_HASH_SET.add(PROPERTY_MESSAGE_REPLY_TO_CLIENT);
STRING_HASH_SET.add(PROPERTY_MESSAGE_TTL);
STRING_HASH_SET.add(PROPERTY_REPLY_MESSAGE_ARRIVE_TIME);
STRING_HASH_SET.add(PROPERTY_PUSH_REPLY_TIME);
STRING_HASH_SET.add(PROPERTY_CLUSTER);
STRING_HASH_SET.add(PROPERTY_MESSAGE_TYPE);
} }
} }
...@@ -182,4 +182,10 @@ public class RequestCode { ...@@ -182,4 +182,10 @@ public class RequestCode {
* resume logic of checking half messages that have been put in TRANS_CHECK_MAXTIME_TOPIC before * resume logic of checking half messages that have been put in TRANS_CHECK_MAXTIME_TOPIC before
*/ */
public static final int RESUME_CHECK_HALF_MESSAGE = 323; public static final int RESUME_CHECK_HALF_MESSAGE = 323;
public static final int SEND_REPLY_MESSAGE = 324;
public static final int SEND_REPLY_MESSAGE_V2 = 325;
public static final int PUSH_REPLY_MESSAGE_TO_CLIENT = 326;
} }
/*
* 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.annotation.CFNullable;
import org.apache.rocketmq.remoting.exception.RemotingCommandException;
public class ReplyMessageRequestHeader implements CommandCustomHeader {
@CFNotNull
private String producerGroup;
@CFNotNull
private String topic;
@CFNotNull
private String defaultTopic;
@CFNotNull
private Integer defaultTopicQueueNums;
@CFNotNull
private Integer queueId;
@CFNotNull
private Integer sysFlag;
@CFNotNull
private Long bornTimestamp;
@CFNotNull
private Integer flag;
@CFNullable
private String properties;
@CFNullable
private Integer reconsumeTimes;
@CFNullable
private boolean unitMode = false;
@CFNotNull
private String bornHost;
@CFNotNull
private String storeHost;
@CFNotNull
private long storeTimestamp;
public void checkFields() throws RemotingCommandException {
}
public String getProducerGroup() {
return producerGroup;
}
public void setProducerGroup(String producerGroup) {
this.producerGroup = producerGroup;
}
public String getTopic() {
return topic;
}
public void setTopic(String topic) {
this.topic = topic;
}
public String getDefaultTopic() {
return defaultTopic;
}
public void setDefaultTopic(String defaultTopic) {
this.defaultTopic = defaultTopic;
}
public Integer getDefaultTopicQueueNums() {
return defaultTopicQueueNums;
}
public void setDefaultTopicQueueNums(Integer defaultTopicQueueNums) {
this.defaultTopicQueueNums = defaultTopicQueueNums;
}
public Integer getQueueId() {
return queueId;
}
public void setQueueId(Integer queueId) {
this.queueId = queueId;
}
public Integer getSysFlag() {
return sysFlag;
}
public void setSysFlag(Integer sysFlag) {
this.sysFlag = sysFlag;
}
public Long getBornTimestamp() {
return bornTimestamp;
}
public void setBornTimestamp(Long bornTimestamp) {
this.bornTimestamp = bornTimestamp;
}
public Integer getFlag() {
return flag;
}
public void setFlag(Integer flag) {
this.flag = flag;
}
public String getProperties() {
return properties;
}
public void setProperties(String properties) {
this.properties = properties;
}
public Integer getReconsumeTimes() {
return reconsumeTimes;
}
public void setReconsumeTimes(Integer reconsumeTimes) {
this.reconsumeTimes = reconsumeTimes;
}
public boolean isUnitMode() {
return unitMode;
}
public void setUnitMode(boolean unitMode) {
this.unitMode = unitMode;
}
public String getBornHost() {
return bornHost;
}
public void setBornHost(String bornHost) {
this.bornHost = bornHost;
}
public String getStoreHost() {
return storeHost;
}
public void setStoreHost(String storeHost) {
this.storeHost = storeHost;
}
public long getStoreTimestamp() {
return storeTimestamp;
}
public void setStoreTimestamp(long storeTimestamp) {
this.storeTimestamp = storeTimestamp;
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.common.utils;
import java.util.UUID;
public class CorrelationIdUtil {
public static String createCorrelationId() {
return UUID.randomUUID().toString();
}
}
...@@ -17,6 +17,8 @@ ...@@ -17,6 +17,8 @@
package org.apache.rocketmq.common; package org.apache.rocketmq.common;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.Properties; import java.util.Properties;
import org.junit.Test; import org.junit.Test;
...@@ -99,12 +101,12 @@ public class UtilAllTest { ...@@ -99,12 +101,12 @@ public class UtilAllTest {
} }
@Test @Test
public void testIPv6Check() { public void testIPv6Check() throws UnknownHostException {
byte[] nonInternalIp = UtilAll.string2bytes("24084004018081003FAA1DDE2B3F898A"); InetAddress nonInternal = InetAddress.getByName("2408:4004:0180:8100:3FAA:1DDE:2B3F:898A");
byte[] internalIp = UtilAll.string2bytes("FEC0000000000000000000000000FFFF"); InetAddress internal = InetAddress.getByName("FE80:0000:0000:0000:0000:0000:0000:FFFF");
assertThat(UtilAll.isInternalV6IP(nonInternalIp)).isFalse(); assertThat(UtilAll.isInternalV6IP(nonInternal)).isFalse();
assertThat(UtilAll.isInternalV6IP(internalIp)).isTrue(); assertThat(UtilAll.isInternalV6IP(internal)).isTrue();
assertThat(UtilAll.ipToIPv6Str(nonInternalIp).toUpperCase()).isEqualTo("2408:4004:0180:8100:3FAA:1DDE:2B3F:898A"); assertThat(UtilAll.ipToIPv6Str(nonInternal.getAddress()).toUpperCase()).isEqualTo("2408:4004:0180:8100:3FAA:1DDE:2B3F:898A");
} }
static class DemoConfig { static class DemoConfig {
......
...@@ -17,9 +17,8 @@ ...@@ -17,9 +17,8 @@
package org.apache.rocketmq.common.message; package org.apache.rocketmq.common.message;
import java.util.Calendar;
import java.util.Date;
import org.junit.Test; import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThat;
public class MessageClientIDSetterTest { public class MessageClientIDSetterTest {
......
...@@ -20,7 +20,7 @@ ...@@ -20,7 +20,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<artifactId>rocketmq-distribution</artifactId> <artifactId>rocketmq-distribution</artifactId>
<name>rocketmq-distribution ${project.version}</name> <name>rocketmq-distribution ${project.version}</name>
......
Apache RocketMQ开发者指南 Apache RocketMQ开发者指南
-------- --------
##### 这个开发者指南是帮您快速了解,并使用 Apache RocketMQ ##### 这个开发者指南是帮您快速了解,并使用 Apache RocketMQ
### 1. 概念和特性 ### 1. 概念和特性
......
...@@ -66,7 +66,11 @@ public class AsyncProducer { ...@@ -66,7 +66,11 @@ public class AsyncProducer {
// 启动Producer实例 // 启动Producer实例
producer.start(); producer.start();
producer.setRetryTimesWhenSendAsyncFailed(0); producer.setRetryTimesWhenSendAsyncFailed(0);
for (int i = 0; i < 100; i++) {
int messageCount = 100;
// 根据消息数量实例化倒计时计算器
final CountDownLatch2 countDownLatch = new CountDownLatch2(messageCount);
for (int i = 0; i < messageCount; i++) {
final int index = i; final int index = i;
// 创建消息,并指定Topic,Tag和消息体 // 创建消息,并指定Topic,Tag和消息体
Message msg = new Message("TopicTest", Message msg = new Message("TopicTest",
...@@ -87,6 +91,8 @@ public class AsyncProducer { ...@@ -87,6 +91,8 @@ public class AsyncProducer {
} }
}); });
} }
// 等待5s
countDownLatch.await(5, TimeUnit.SECONDS);
// 如果不再发送消息,关闭Producer实例。 // 如果不再发送消息,关闭Producer实例。
producer.shutdown(); producer.shutdown();
} }
......
...@@ -16,7 +16,7 @@ ...@@ -16,7 +16,7 @@
RocketMQ的消费者可以根据Tag进行消息过滤,也支持自定义属性过滤。消息过滤目前是在Broker端实现的,优点是减少了对于Consumer无用消息的网络传输,缺点是增加了Broker的负担、而且实现相对复杂。 RocketMQ的消费者可以根据Tag进行消息过滤,也支持自定义属性过滤。消息过滤目前是在Broker端实现的,优点是减少了对于Consumer无用消息的网络传输,缺点是增加了Broker的负担、而且实现相对复杂。
## 4 消息可靠性 ## 4 消息可靠性
RocketMQ支持消息的高可靠,影响消息可靠性的几种情况: RocketMQ支持消息的高可靠,影响消息可靠性的几种情况:
1) Broker正常关闭 1) Broker正常关闭
2) Broker异常Crash 2) Broker异常Crash
3) OS Crash 3) OS Crash
4) 机器掉电,但是能立即恢复供电情况 4) 机器掉电,但是能立即恢复供电情况
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
...@@ -51,12 +51,12 @@ ...@@ -51,12 +51,12 @@
<dependency> <dependency>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-openmessaging</artifactId> <artifactId>rocketmq-openmessaging</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-acl</artifactId> <artifactId>rocketmq-acl</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</dependency> </dependency>
</dependencies> </dependencies>
</project> </project>
...@@ -103,6 +103,10 @@ public class Consumer { ...@@ -103,6 +103,10 @@ public class Consumer {
}, 10000, 10000); }, 10000, 10000);
DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(group); DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(group);
if (commandLine.hasOption('n')) {
String ns = commandLine.getOptionValue('n');
consumer.setNamesrvAddr(ns);
}
consumer.setInstanceName(Long.toString(System.currentTimeMillis())); consumer.setInstanceName(Long.toString(System.currentTimeMillis()));
if (filterType == null || expression == null) { if (filterType == null || expression == null) {
......
...@@ -24,6 +24,11 @@ import java.util.TimerTask; ...@@ -24,6 +24,11 @@ import java.util.TimerTask;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.PosixParser;
import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.producer.LocalTransactionExecuter; import org.apache.rocketmq.client.producer.LocalTransactionExecuter;
import org.apache.rocketmq.client.producer.LocalTransactionState; import org.apache.rocketmq.client.producer.LocalTransactionState;
...@@ -33,20 +38,19 @@ import org.apache.rocketmq.client.producer.TransactionMQProducer; ...@@ -33,20 +38,19 @@ import org.apache.rocketmq.client.producer.TransactionMQProducer;
import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.common.RemotingHelper;
import org.apache.rocketmq.srvutil.ServerUtil;
public class TransactionProducer { public class TransactionProducer {
private static int threadCount;
private static int messageSize;
private static boolean ischeck;
private static boolean ischeckffalse;
public static void main(String[] args) throws MQClientException, UnsupportedEncodingException { public static void main(String[] args) throws MQClientException, UnsupportedEncodingException {
threadCount = args.length >= 1 ? Integer.parseInt(args[0]) : 32; Options options = ServerUtil.buildCommandlineOptions(new Options());
messageSize = args.length >= 2 ? Integer.parseInt(args[1]) : 1024 * 2; CommandLine commandLine = ServerUtil.parseCmdLine("TransactionProducer", args, buildCommandlineOptions(options), new PosixParser());
ischeck = args.length >= 3 && Boolean.parseBoolean(args[2]);
ischeckffalse = args.length >= 4 && Boolean.parseBoolean(args[3]);
final Message msg = buildMessage(messageSize); final String topic = commandLine.hasOption('t') ? commandLine.getOptionValue('t').trim() : "BenchmarkTest";
final int threadCount = commandLine.hasOption('w') ? Integer.parseInt(commandLine.getOptionValue('w')) : 32;
final int messageSize = commandLine.hasOption('s') ? Integer.parseInt(commandLine.getOptionValue('s')) : 2048;
final boolean ischeck = commandLine.hasOption('c') ? Boolean.parseBoolean(commandLine.getOptionValue('c')) : false;
final boolean ischeckffalse = commandLine.hasOption('r') ? Boolean.parseBoolean(commandLine.getOptionValue('r')) : true;
final ExecutorService sendThreadPool = Executors.newFixedThreadPool(threadCount); final ExecutorService sendThreadPool = Executors.newFixedThreadPool(threadCount);
...@@ -98,6 +102,10 @@ public class TransactionProducer { ...@@ -98,6 +102,10 @@ public class TransactionProducer {
producer.setInstanceName(Long.toString(System.currentTimeMillis())); producer.setInstanceName(Long.toString(System.currentTimeMillis()));
producer.setTransactionCheckListener(transactionCheckListener); producer.setTransactionCheckListener(transactionCheckListener);
producer.setDefaultTopicQueueNums(1000); producer.setDefaultTopicQueueNums(1000);
if (commandLine.hasOption('n')) {
String ns = commandLine.getOptionValue('n');
producer.setNamesrvAddr(ns);
}
producer.start(); producer.start();
final TransactionExecuterBImpl tranExecuter = new TransactionExecuterBImpl(ischeck); final TransactionExecuterBImpl tranExecuter = new TransactionExecuterBImpl(ischeck);
...@@ -111,7 +119,7 @@ public class TransactionProducer { ...@@ -111,7 +119,7 @@ public class TransactionProducer {
// Thread.sleep(1000); // Thread.sleep(1000);
final long beginTimestamp = System.currentTimeMillis(); final long beginTimestamp = System.currentTimeMillis();
SendResult sendResult = SendResult sendResult =
producer.sendMessageInTransaction(msg, tranExecuter, null); producer.sendMessageInTransaction(buildMessage(messageSize, topic), tranExecuter, null);
if (sendResult != null) { if (sendResult != null) {
statsBenchmark.getSendRequestSuccessCount().incrementAndGet(); statsBenchmark.getSendRequestSuccessCount().incrementAndGet();
statsBenchmark.getReceiveResponseSuccessCount().incrementAndGet(); statsBenchmark.getReceiveResponseSuccessCount().incrementAndGet();
...@@ -138,18 +146,45 @@ public class TransactionProducer { ...@@ -138,18 +146,45 @@ public class TransactionProducer {
} }
} }
private static Message buildMessage(final int messageSize) throws UnsupportedEncodingException { private static Message buildMessage(final int messageSize, String topic) {
Message msg = new Message(); try {
msg.setTopic("BenchmarkTest"); Message msg = new Message();
msg.setTopic(topic);
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
for (int i = 0; i < messageSize; i += 10) { for (int i = 0; i < messageSize; i += 10) {
sb.append("hello baby"); sb.append("hello baby");
}
msg.setBody(sb.toString().getBytes(RemotingHelper.DEFAULT_CHARSET));
return msg;
} catch (UnsupportedEncodingException e) {
throw new RuntimeException(e);
} }
}
public static Options buildCommandlineOptions(final Options options) {
Option opt = new Option("w", "threadCount", true, "Thread count, Default: 32");
opt.setRequired(false);
options.addOption(opt);
opt = new Option("s", "messageSize", true, "Message Size, Default: 2048");
opt.setRequired(false);
options.addOption(opt);
opt = new Option("t", "topic", true, "Topic name, Default: BenchmarkTest");
opt.setRequired(false);
options.addOption(opt);
opt = new Option("c", "check", true, "Check the message, Default: false");
opt.setRequired(false);
options.addOption(opt);
opt = new Option("r", "checkResult", true, "Message check result, Default: true");
opt.setRequired(false);
options.addOption(opt);
msg.setBody(sb.toString().getBytes(RemotingHelper.DEFAULT_CHARSET));
return msg; return options;
} }
} }
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.example.rpc;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.log.ClientLogger;
import org.apache.rocketmq.client.producer.DefaultMQProducer;
import org.apache.rocketmq.client.producer.RequestCallback;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.logging.InternalLogger;
import org.apache.rocketmq.remoting.common.RemotingHelper;
public class AsyncRequestProducer {
private static final InternalLogger log = ClientLogger.getLog();
public static void main(String[] args) throws MQClientException, InterruptedException {
String producerGroup = "please_rename_unique_group_name";
String topic = "RequestTopic";
long ttl = 3000;
DefaultMQProducer producer = new DefaultMQProducer(producerGroup);
producer.start();
try {
Message msg = new Message(topic,
"",
"Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET));
long begin = System.currentTimeMillis();
producer.request(msg, new RequestCallback() {
@Override
public void onSuccess(Message message) {
long cost = System.currentTimeMillis() - begin;
System.out.printf("request to <%s> cost: %d replyMessage: %s %n", topic, cost, message);
}
@Override
public void onException(Throwable e) {
System.err.printf("request to <%s> fail.", topic);
}
}, ttl);
} catch (Exception e) {
log.warn("", e);
}
/* shutdown after your request callback is finished */
// producer.shutdown();
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.example.rpc;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.producer.DefaultMQProducer;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.remoting.common.RemotingHelper;
public class RequestProducer {
public static void main(String[] args) throws MQClientException, InterruptedException {
String producerGroup = "please_rename_unique_group_name";
String topic = "RequestTopic";
long ttl = 3000;
DefaultMQProducer producer = new DefaultMQProducer(producerGroup);
producer.start();
try {
Message msg = new Message(topic,
"",
"Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET));
long begin = System.currentTimeMillis();
Message retMsg = producer.request(msg, ttl);
long cost = System.currentTimeMillis() - begin;
System.out.printf("request to <%s> cost: %d replyMessage: %s %n", topic, cost, retMsg);
} catch (Exception e) {
e.printStackTrace();
}
producer.shutdown();
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.rocketmq.example.rpc;
import java.util.List;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext;
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus;
import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.producer.DefaultMQProducer;
import org.apache.rocketmq.client.producer.SendResult;
import org.apache.rocketmq.client.utils.MessageUtil;
import org.apache.rocketmq.common.consumer.ConsumeFromWhere;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.remoting.exception.RemotingException;
public class ResponseConsumer {
public static void main(String[] args) throws InterruptedException, MQClientException {
String producerGroup = "please_rename_unique_group_name";
String consumerGroup = "please_rename_unique_group_name";
String topic = "RequestTopic";
// create a producer to send reply message
DefaultMQProducer replyProducer = new DefaultMQProducer(producerGroup);
replyProducer.start();
// create consumer
DefaultMQPushConsumer consumer = new DefaultMQPushConsumer(consumerGroup);
consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET);
// recommend client configs
consumer.setPullTimeDelayMillsWhenException(0L);
consumer.registerMessageListener(new MessageListenerConcurrently() {
@Override
public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs, ConsumeConcurrentlyContext context) {
System.out.printf("%s Receive New Messages: %s %n", Thread.currentThread().getName(), msgs);
for (MessageExt msg : msgs) {
try {
System.out.printf("handle message: %s", msg.toString());
String replyTo = MessageUtil.getReplyToClient(msg);
byte[] replyContent = "reply message contents.".getBytes();
// create reply message with given util, do not create reply message by yourself
Message replyMessage = MessageUtil.createReplyMessage(msg, replyContent);
// send reply message with producer
SendResult replyResult = replyProducer.send(replyMessage, 3000);
System.out.printf("reply to %s , %s %n", replyTo, replyResult.toString());
} catch (MQClientException | RemotingException | MQBrokerException | InterruptedException e) {
e.printStackTrace();
}
}
return ConsumeConcurrentlyStatus.CONSUME_SUCCESS;
}
});
consumer.subscribe(topic, "*");
consumer.start();
System.out.printf("Consumer Started.%n");
}
}
...@@ -20,7 +20,7 @@ ...@@ -20,7 +20,7 @@
<parent> <parent>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
<artifactId>rocketmq-logappender</artifactId> <artifactId>rocketmq-logappender</artifactId>
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -20,7 +20,7 @@ ...@@ -20,7 +20,7 @@
<parent> <parent>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -16,8 +16,7 @@ ...@@ -16,8 +16,7 @@
limitations under the License. limitations under the License.
--> -->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<parent> <parent>
<groupId>org.apache</groupId> <groupId>org.apache</groupId>
...@@ -30,7 +29,7 @@ ...@@ -30,7 +29,7 @@
<inceptionYear>2012</inceptionYear> <inceptionYear>2012</inceptionYear>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
<packaging>pom</packaging> <packaging>pom</packaging>
<name>Apache RocketMQ ${project.version}</name> <name>Apache RocketMQ ${project.version}</name>
<url>http://rocketmq.apache.org/</url> <url>http://rocketmq.apache.org/</url>
...@@ -160,7 +159,7 @@ ...@@ -160,7 +159,7 @@
</executions> </executions>
<configuration> <configuration>
<rules> <rules>
<banCircularDependencies/> <banCircularDependencies />
</rules> </rules>
<fail>true</fail> <fail>true</fail>
</configuration> </configuration>
...@@ -569,7 +568,7 @@ ...@@ -569,7 +568,7 @@
<dependency> <dependency>
<groupId>com.alibaba</groupId> <groupId>com.alibaba</groupId>
<artifactId>fastjson</artifactId> <artifactId>fastjson</artifactId>
<version>1.2.51</version> <version>1.2.61</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.javassist</groupId> <groupId>org.javassist</groupId>
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -280,7 +280,9 @@ public class HAService { ...@@ -280,7 +280,9 @@ public class HAService {
if (!this.requestsRead.isEmpty()) { if (!this.requestsRead.isEmpty()) {
for (CommitLog.GroupCommitRequest req : this.requestsRead) { for (CommitLog.GroupCommitRequest req : this.requestsRead) {
boolean transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset(); boolean transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset();
for (int i = 0; !transferOK && i < 5; i++) { long waitUntilWhen = HAService.this.defaultMessageStore.getSystemClock().now()
+ HAService.this.defaultMessageStore.getMessageStoreConfig().getSyncFlushTimeout();
while (!transferOK && HAService.this.defaultMessageStore.getSystemClock().now() < waitUntilWhen) {
this.notifyTransferObject.waitForRunning(1000); this.notifyTransferObject.waitForRunning(1000);
transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset(); transferOK = HAService.this.push2SlaveMaxOffset.get() >= req.getNextOffset();
} }
......
...@@ -20,7 +20,7 @@ ...@@ -20,7 +20,7 @@
<parent> <parent>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
...@@ -19,7 +19,7 @@ ...@@ -19,7 +19,7 @@
<parent> <parent>
<groupId>org.apache.rocketmq</groupId> <groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-all</artifactId> <artifactId>rocketmq-all</artifactId>
<version>4.6.0-SNAPSHOT</version> <version>4.6.1-SNAPSHOT</version>
</parent> </parent>
<modelVersion>4.0.0</modelVersion> <modelVersion>4.0.0</modelVersion>
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册