/**
* Copyright (C) 2010-2013 Alibaba Group Holding Limited
*
* Licensed 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 com.alibaba.rocketmq.client.impl.consumer;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import com.alibaba.rocketmq.client.QueryResult;
import com.alibaba.rocketmq.client.Validators;
import com.alibaba.rocketmq.client.consumer.DefaultMQPullConsumer;
import com.alibaba.rocketmq.client.consumer.PullCallback;
import com.alibaba.rocketmq.client.consumer.PullResult;
import com.alibaba.rocketmq.client.consumer.store.LocalFileOffsetStore;
import com.alibaba.rocketmq.client.consumer.store.OffsetStore;
import com.alibaba.rocketmq.client.consumer.store.ReadOffsetType;
import com.alibaba.rocketmq.client.consumer.store.RemoteBrokerOffsetStore;
import com.alibaba.rocketmq.client.exception.MQBrokerException;
import com.alibaba.rocketmq.client.exception.MQClientException;
import com.alibaba.rocketmq.client.impl.CommunicationMode;
import com.alibaba.rocketmq.client.impl.MQClientManager;
import com.alibaba.rocketmq.client.impl.factory.MQClientFactory;
import com.alibaba.rocketmq.client.log.ClientLogger;
import com.alibaba.rocketmq.common.MixAll;
import com.alibaba.rocketmq.common.ServiceState;
import com.alibaba.rocketmq.common.consumer.ConsumeFromWhere;
import com.alibaba.rocketmq.common.filter.FilterAPI;
import com.alibaba.rocketmq.common.help.FAQUrl;
import com.alibaba.rocketmq.common.message.Message;
import com.alibaba.rocketmq.common.message.MessageConst;
import com.alibaba.rocketmq.common.message.MessageExt;
import com.alibaba.rocketmq.common.message.MessageQueue;
import com.alibaba.rocketmq.common.protocol.heartbeat.ConsumeType;
import com.alibaba.rocketmq.common.protocol.heartbeat.MessageModel;
import com.alibaba.rocketmq.common.protocol.heartbeat.SubscriptionData;
import com.alibaba.rocketmq.common.sysflag.PullSysFlag;
import com.alibaba.rocketmq.remoting.exception.RemotingException;
/**
* Pull方式的Consumer实现
*
* @author shijia.wxr<vintage.wang@gmail.com>
* @since 2013-7-24
*/
public class DefaultMQPullConsumerImpl implements MQConsumerInner {
private final Logger log = ClientLogger.getLog();
private final DefaultMQPullConsumer defaultMQPullConsumer;
private ServiceState serviceState = ServiceState.CREATE_JUST;
private MQClientFactory mQClientFactory;
private PullAPIWrapper pullAPIWrapper;
// 消费进度存储
private OffsetStore offsetStore;
// Rebalance实现
private RebalanceImpl rebalanceImpl = new RebalancePullImpl(this);
public DefaultMQPullConsumerImpl(final DefaultMQPullConsumer defaultMQPullConsumer) {
this.defaultMQPullConsumer = defaultMQPullConsumer;
}
public void createTopic(String key, String newTopic, int queueNum) throws MQClientException {
this.makeSureStateOK();
this.mQClientFactory.getMQAdminImpl().createTopic(key, newTopic, queueNum);
}
private void makeSureStateOK() throws MQClientException {
if (this.serviceState != ServiceState.RUNNING) {
throw new MQClientException("The consumer service state not OK, "//
+ this.serviceState//
+ FAQUrl.suggestTodo(FAQUrl.CLIENT_SERVICE_NOT_OK), null);
}
}
public long fetchConsumeOffset(MessageQueue mq, boolean fromStore) throws MQClientException {
this.makeSureStateOK();
return this.offsetStore.readOffset(mq, fromStore ? ReadOffsetType.READ_FROM_STORE
: ReadOffsetType.MEMORY_FIRST_THEN_STORE);
}
public Set<MessageQueue> fetchMessageQueuesInBalance(String topic) throws MQClientException {
this.makeSureStateOK();
if (null == topic) {
throw new IllegalArgumentException("topic is null");
}
ConcurrentHashMap<MessageQueue, ProcessQueue> mqTable = this.rebalanceImpl.getProcessQueueTable();
Set<MessageQueue> mqResult = new HashSet<MessageQueue>();
for (MessageQueue mq : mqTable.keySet()) {
if (mq.getTopic().equals(topic)) {
mqResult.add(mq);
}
}
return mqResult;
}
public List<MessageQueue> fetchPublishMessageQueues(String topic) throws MQClientException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().fetchPublishMessageQueues(topic);
}
public Set<MessageQueue> fetchSubscribeMessageQueues(String topic) throws MQClientException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().fetchSubscribeMessageQueues(topic);
}
public long earliestMsgStoreTime(MessageQueue mq) throws MQClientException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().earliestMsgStoreTime(mq);
}
@Override
public String groupName() {
return this.defaultMQPullConsumer.getConsumerGroup();
}
@Override
public MessageModel messageModel() {
return this.defaultMQPullConsumer.getMessageModel();
}
@Override
public ConsumeType consumeType() {
return ConsumeType.CONSUME_ACTIVELY;
}
@Override
public ConsumeFromWhere consumeFromWhere() {
return ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET;
}
@Override
public Set<SubscriptionData> subscriptions() {
Set<SubscriptionData> result = new HashSet<SubscriptionData>();
Set<String> topics = this.defaultMQPullConsumer.getRegisterTopics();
if (topics != null) {
synchronized (topics) {
for (String t : topics) {
SubscriptionData ms = new SubscriptionData(t, SubscriptionData.SUB_ALL);
ms.setSubVersion(0L);
result.add(ms);
}
}
}
return result;
}
@Override
public void doRebalance() {
if (this.rebalanceImpl != null) {
this.rebalanceImpl.doRebalance();
}
}
@Override
public void persistConsumerOffset() {
try {
this.makeSureStateOK();
Set<MessageQueue> mqs = new HashSet<MessageQueue>();
Set<MessageQueue> allocateMq = this.rebalanceImpl.getProcessQueueTable().keySet();
if (allocateMq != null) {
mqs.addAll(allocateMq);
}
this.offsetStore.persistAll(mqs);
}
catch (Exception e) {
log.error("group: " + this.defaultMQPullConsumer.getConsumerGroup()
+ " persistConsumerOffset exception", e);
}
}
@Override
public void updateTopicSubscribeInfo(String topic, Set<MessageQueue> info) {
Map<String, SubscriptionData> subTable = this.rebalanceImpl.getSubscriptionInner();
if (subTable != null) {
if (subTable.containsKey(topic)) {
this.rebalanceImpl.getTopicSubscribeInfoTable().put(topic, info);
}
}
}
@Override
public boolean isSubscribeTopicNeedUpdate(String topic) {
Map<String, SubscriptionData> subTable = this.rebalanceImpl.getSubscriptionInner();
if (subTable != null) {
if (subTable.containsKey(topic)) {
return !this.rebalanceImpl.topicSubscribeInfoTable.containsKey(topic);
}
}
return false;
}
public long maxOffset(MessageQueue mq) throws MQClientException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().maxOffset(mq);
}
public long minOffset(MessageQueue mq) throws MQClientException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().minOffset(mq);
}
public PullResult pull(MessageQueue mq, String subExpression, long offset, int maxNums)
throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
return this.pullSyncImpl(mq, subExpression, offset, maxNums, false);
}
private PullResult pullSyncImpl(MessageQueue mq, String subExpression, long offset, int maxNums,
boolean block) throws MQClientException, RemotingException, MQBrokerException,
InterruptedException {
this.makeSureStateOK();
if (null == mq) {
throw new MQClientException("mq is null", null);
}
if (offset < 0) {
throw new MQClientException("offset < 0", null);
}
if (maxNums <= 0) {
throw new MQClientException("maxNums <= 0", null);
}
// 自动订阅
this.subscriptionAutomatically(mq.getTopic());
int sysFlag = PullSysFlag.buildSysFlag(false, block, true);
SubscriptionData subscriptionData;
try {
subscriptionData = FilterAPI.buildSubscriptionData(mq.getTopic(), subExpression);
}
catch (Exception e) {
throw new MQClientException("parse subscription error", e);
}
long timeoutMillis =
block ? this.defaultMQPullConsumer.getConsumerTimeoutMillisWhenSuspend()
: this.defaultMQPullConsumer.getConsumerPullTimeoutMillis();
PullResult pullResult = this.pullAPIWrapper.pullKernelImpl(//
mq, // 1
subscriptionData.getSubString(), // 2
0L, // 3
offset, // 4
maxNums, // 5
sysFlag, // 6
0, // 7
this.defaultMQPullConsumer.getBrokerSuspendMaxTimeMillis(), // 8
timeoutMillis, // 9
CommunicationMode.SYNC, // 10
null// 11
);
return this.pullAPIWrapper.processPullResult(mq, pullResult, subscriptionData);
}
private void subscriptionAutomatically(final String topic) {
if (!this.rebalanceImpl.getSubscriptionInner().containsKey(topic)) {
try {
SubscriptionData subscriptionData =
FilterAPI.buildSubscriptionData(topic, SubscriptionData.SUB_ALL);
this.rebalanceImpl.subscriptionInner.putIfAbsent(topic, subscriptionData);
}
catch (Exception e) {
}
}
}
public void pull(MessageQueue mq, String subExpression, long offset, int maxNums,
PullCallback pullCallback) throws MQClientException, RemotingException, InterruptedException {
this.pullAsyncImpl(mq, subExpression, offset, maxNums, pullCallback, false);
}
private void pullAsyncImpl(//
final MessageQueue mq,//
final String subExpression,//
final long offset,//
final int maxNums,//
final PullCallback pullCallback,//
final boolean block//
) throws MQClientException, RemotingException, InterruptedException {
this.makeSureStateOK();
if (null == mq) {
throw new MQClientException("mq is null", null);
}
if (offset < 0) {
throw new MQClientException("offset < 0", null);
}
if (maxNums <= 0) {
throw new MQClientException("maxNums <= 0", null);
}
if (null == pullCallback) {
throw new MQClientException("pullCallback is null", null);
}
// 自动订阅
this.subscriptionAutomatically(mq.getTopic());
try {
int sysFlag = PullSysFlag.buildSysFlag(false, block, true);
final SubscriptionData subscriptionData;
try {
subscriptionData = FilterAPI.buildSubscriptionData(mq.getTopic(), subExpression);
}
catch (Exception e) {
throw new MQClientException("parse subscription error", e);
}
long timeoutMillis =
block ? this.defaultMQPullConsumer.getConsumerTimeoutMillisWhenSuspend()
: this.defaultMQPullConsumer.getConsumerPullTimeoutMillis();
this.pullAPIWrapper.pullKernelImpl(//
mq, // 1
subscriptionData.getSubString(), // 2
0L, // 3
offset, // 4
maxNums, // 5
sysFlag, // 6
0, // 7
this.defaultMQPullConsumer.getBrokerSuspendMaxTimeMillis(), // 8
timeoutMillis, // 9
CommunicationMode.ASYNC, // 10
new PullCallback() {
@Override
public void onException(Throwable e) {
pullCallback.onException(e);
}
@Override
public void onSuccess(PullResult pullResult) {
pullCallback.onSuccess(DefaultMQPullConsumerImpl.this.pullAPIWrapper
.processPullResult(mq, pullResult, subscriptionData));
}
});
}
catch (MQBrokerException e) {
throw new MQClientException("pullAsync unknow exception", e);
}
}
public PullResult pullBlockIfNotFound(MessageQueue mq, String subExpression, long offset, int maxNums)
throws MQClientException, RemotingException, MQBrokerException, InterruptedException {
return this.pullSyncImpl(mq, subExpression, offset, maxNums, true);
}
public void pullBlockIfNotFound(MessageQueue mq, String subExpression, long offset, int maxNums,
PullCallback pullCallback) throws MQClientException, RemotingException, InterruptedException {
this.pullAsyncImpl(mq, subExpression, offset, maxNums, pullCallback, true);
}
public QueryResult queryMessage(String topic, String key, int maxNum, long begin, long end)
throws MQClientException, InterruptedException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().queryMessage(topic, key, maxNum, begin, end);
}
public long searchOffset(MessageQueue mq, long timestamp) throws MQClientException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().searchOffset(mq, timestamp);
}
public void sendMessageBack(MessageExt msg, int delayLevel) throws RemotingException, MQBrokerException,
InterruptedException, MQClientException {
try {
this.mQClientFactory.getMQClientAPIImpl().consumerSendMessageBack(msg,
this.defaultMQPullConsumer.getConsumerGroup(), delayLevel, 3000);
}
catch (Exception e) {
log.error("sendMessageBack Exception, " + this.defaultMQPullConsumer.getConsumerGroup(), e);
Message newMsg =
new Message(MixAll.getRetryTopic(this.defaultMQPullConsumer.getConsumerGroup()),
msg.getBody());
newMsg.setFlag(msg.getFlag());
newMsg.setProperties(msg.getProperties());
newMsg.putProperty(MessageConst.PROPERTY_RETRY_TOPIC, msg.getTopic());
this.mQClientFactory.getDefaultMQProducer().send(newMsg);
}
}
public void shutdown() {
switch (this.serviceState) {
case CREATE_JUST:
break;
case RUNNING:
this.persistConsumerOffset();
this.mQClientFactory.unregisterConsumer(this.defaultMQPullConsumer.getConsumerGroup());
this.mQClientFactory.shutdown();
log.info("the consumer [{}] shutdown OK", this.defaultMQPullConsumer.getConsumerGroup());
this.serviceState = ServiceState.SHUTDOWN_ALREADY;
break;
case SHUTDOWN_ALREADY:
break;
default:
break;
}
}
public void start() throws MQClientException {
switch (this.serviceState) {
case CREATE_JUST:
this.serviceState = ServiceState.START_FAILED;
this.checkConfig();
this.copySubscription();
this.mQClientFactory =
MQClientManager.getInstance().getAndCreateMQClientFactory(this.defaultMQPullConsumer);
// 初始化Rebalance变量
this.rebalanceImpl.setConsumerGroup(this.defaultMQPullConsumer.getConsumerGroup());
this.rebalanceImpl.setMessageModel(this.defaultMQPullConsumer.getMessageModel());
this.rebalanceImpl.setAllocateMessageQueueStrategy(this.defaultMQPullConsumer
.getAllocateMessageQueueStrategy());
this.rebalanceImpl.setmQClientFactory(this.mQClientFactory);
this.pullAPIWrapper = new PullAPIWrapper(//
mQClientFactory,//
this.defaultMQPullConsumer.getConsumerGroup());
if (this.defaultMQPullConsumer.getOffsetStore() != null) {
this.offsetStore = this.defaultMQPullConsumer.getOffsetStore();
}
else {
// 广播消费/集群消费
switch (this.defaultMQPullConsumer.getMessageModel()) {
case BROADCASTING:
this.offsetStore =
new LocalFileOffsetStore(this.mQClientFactory,
this.defaultMQPullConsumer.getConsumerGroup());
break;
case CLUSTERING:
this.offsetStore =
new RemoteBrokerOffsetStore(this.mQClientFactory,
this.defaultMQPullConsumer.getConsumerGroup());
break;
default:
break;
}
}
// 加载消费进度
this.offsetStore.load();
boolean registerOK =
mQClientFactory.registerConsumer(this.defaultMQPullConsumer.getConsumerGroup(), this);
if (!registerOK) {
this.serviceState = ServiceState.CREATE_JUST;
throw new MQClientException("The consumer group["
+ this.defaultMQPullConsumer.getConsumerGroup()
+ "] has been created before, specify another name please."
+ FAQUrl.suggestTodo(FAQUrl.GROUP_NAME_DUPLICATE_URL), null);
}
mQClientFactory.start();
log.info("the consumer [{}] start OK", this.defaultMQPullConsumer.getConsumerGroup());
this.serviceState = ServiceState.RUNNING;
break;
case RUNNING:
case START_FAILED:
case SHUTDOWN_ALREADY:
throw new MQClientException("The PullConsumer service state not OK, maybe started once, "//
+ this.serviceState//
+ FAQUrl.suggestTodo(FAQUrl.CLIENT_SERVICE_NOT_OK), null);
default:
break;
}
}
private void copySubscription() throws MQClientException {
try {
// 复制用户初始设置的订阅关系
Set<String> registerTopics = this.defaultMQPullConsumer.getRegisterTopics();
if (registerTopics != null) {
for (final String topic : registerTopics) {
SubscriptionData subscriptionData =
FilterAPI.buildSubscriptionData(topic, SubscriptionData.SUB_ALL);
this.rebalanceImpl.getSubscriptionInner().put(topic, subscriptionData);
}
}
}
catch (Exception e) {
throw new MQClientException("subscription exception", e);
}
}
private void checkConfig() throws MQClientException {
// check consumerGroup
Validators.checkGroup(this.defaultMQPullConsumer.getConsumerGroup());
// consumerGroup
if (null == this.defaultMQPullConsumer.getConsumerGroup()) {
throw new MQClientException("consumerGroup is null" //
+ FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), //
null);
}
// consumerGroup
if (this.defaultMQPullConsumer.getConsumerGroup().equals(MixAll.DEFAULT_CONSUMER_GROUP)) {
throw new MQClientException("consumerGroup can not equal "//
+ MixAll.DEFAULT_CONSUMER_GROUP //
+ ", please specify another one."//
+ FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), //
null);
}
// messageModel
if (null == this.defaultMQPullConsumer.getMessageModel()) {
throw new MQClientException("messageModel is null" //
+ FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), //
null);
}
// allocateMessageQueueStrategy
if (null == this.defaultMQPullConsumer.getAllocateMessageQueueStrategy()) {
throw new MQClientException("allocateMessageQueueStrategy is null" //
+ FAQUrl.suggestTodo(FAQUrl.CLIENT_PARAMETER_CHECK_URL), //
null);
}
}
public void updateConsumeOffset(MessageQueue mq, long offset) throws MQClientException {
this.makeSureStateOK();
this.offsetStore.updateOffset(mq, offset, false);
}
public MessageExt viewMessage(String msgId) throws RemotingException, MQBrokerException,
InterruptedException, MQClientException {
this.makeSureStateOK();
return this.mQClientFactory.getMQAdminImpl().viewMessage(msgId);
}
public DefaultMQPullConsumer getDefaultMQPullConsumer() {
return defaultMQPullConsumer;
}
public OffsetStore getOffsetStore() {
return offsetStore;
}
public void setOffsetStore(OffsetStore offsetStore) {
this.offsetStore = offsetStore;
}
}