/* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 * (the "License"); you may not use this file except in compliance with * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.rocketmq.client.impl.consumer; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyContext; import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; import org.apache.rocketmq.client.consumer.listener.ConsumeReturnType; import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; import org.apache.rocketmq.client.hook.ConsumeMessageContext; import org.apache.rocketmq.client.log.ClientLogger; import org.apache.rocketmq.client.stat.ConsumerStatsManager; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.message.*; import org.apache.rocketmq.common.protocol.body.CMResult; import org.apache.rocketmq.common.protocol.body.ConsumeMessageDirectlyResult; import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.slf4j.Logger; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.concurrent.*; public class ConsumeMessageOrderlyService implements ConsumeMessageService { private static final Logger log = ClientLogger.getLog(); private final static long MAX_TIME_CONSUME_CONTINUOUSLY = Long.parseLong(System.getProperty("rocketmq.client.maxTimeConsumeContinuously", "60000")); private final DefaultMQPushConsumerImpl defaultMQPushConsumerImpl; private final DefaultMQPushConsumer defaultMQPushConsumer; private final MessageListenerOrderly messageListener; private final BlockingQueue<Runnable> consumeRequestQueue; private final ThreadPoolExecutor consumeExecutor; private final String consumerGroup; private final MessageQueueLock messageQueueLock = new MessageQueueLock(); private final ScheduledExecutorService scheduledExecutorService; private volatile boolean stopped = false; public ConsumeMessageOrderlyService(DefaultMQPushConsumerImpl defaultMQPushConsumerImpl, MessageListenerOrderly messageListener) { this.defaultMQPushConsumerImpl = defaultMQPushConsumerImpl; this.messageListener = messageListener; this.defaultMQPushConsumer = this.defaultMQPushConsumerImpl.getDefaultMQPushConsumer(); this.consumerGroup = this.defaultMQPushConsumer.getConsumerGroup(); this.consumeRequestQueue = new LinkedBlockingQueue<Runnable>(); this.consumeExecutor = new ThreadPoolExecutor(// this.defaultMQPushConsumer.getConsumeThreadMin(), // this.defaultMQPushConsumer.getConsumeThreadMax(), // 1000 * 60, // TimeUnit.MILLISECONDS, // this.consumeRequestQueue, // new ThreadFactoryImpl("ConsumeMessageThread_")); this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("ConsumeMessageScheduledThread_")); } public void start() { if (MessageModel.CLUSTERING.equals(ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.messageModel())) { //每隔20S更新Borker上的当前MessageQueue分布式锁的最后刷新时间 this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override public void run() { ConsumeMessageOrderlyService.this.lockMQPeriodically(); } }, 1000 * 1, ProcessQueue.REBALANCE_LOCK_INTERVAL, TimeUnit.MILLISECONDS); } } public void shutdown() { this.stopped = true; this.scheduledExecutorService.shutdown(); this.consumeExecutor.shutdown(); if (MessageModel.CLUSTERING.equals(this.defaultMQPushConsumerImpl.messageModel())) { this.unlockAllMQ(); } } public synchronized void unlockAllMQ() { this.defaultMQPushConsumerImpl.getRebalanceImpl().unlockAll(false); } @Override public void updateCorePoolSize(int corePoolSize) { if (corePoolSize > 0 // && corePoolSize <= Short.MAX_VALUE // && corePoolSize < this.defaultMQPushConsumer.getConsumeThreadMax()) { this.consumeExecutor.setCorePoolSize(corePoolSize); } } @Override public void incCorePoolSize() { } @Override public void decCorePoolSize() { } @Override public int getCorePoolSize() { return this.consumeExecutor.getCorePoolSize(); } @Override public ConsumeMessageDirectlyResult consumeMessageDirectly(MessageExt msg, String brokerName) { ConsumeMessageDirectlyResult result = new ConsumeMessageDirectlyResult(); result.setOrder(true); List<MessageExt> msgs = new ArrayList<MessageExt>(); msgs.add(msg); MessageQueue mq = new MessageQueue(); mq.setBrokerName(brokerName); mq.setTopic(msg.getTopic()); mq.setQueueId(msg.getQueueId()); ConsumeOrderlyContext context = new ConsumeOrderlyContext(mq); final long beginTime = System.currentTimeMillis(); log.info("consumeMessageDirectly receive new messge: {}", msg); try { ConsumeOrderlyStatus status = this.messageListener.consumeMessage(msgs, context); if (status != null) { switch (status) { case COMMIT: result.setConsumeResult(CMResult.CR_COMMIT); break; case ROLLBACK: result.setConsumeResult(CMResult.CR_ROLLBACK); break; case SUCCESS: result.setConsumeResult(CMResult.CR_SUCCESS); break; case SUSPEND_CURRENT_QUEUE_A_MOMENT: result.setConsumeResult(CMResult.CR_LATER); break; default: break; } } else { result.setConsumeResult(CMResult.CR_RETURN_NULL); } } catch (Throwable e) { result.setConsumeResult(CMResult.CR_THROW_EXCEPTION); result.setRemark(RemotingHelper.exceptionSimpleDesc(e)); log.warn(String.format("consumeMessageDirectly exception: %s Group: %s Msgs: %s MQ: %s", // RemotingHelper.exceptionSimpleDesc(e), // ConsumeMessageOrderlyService.this.consumerGroup, // msgs, // mq), e); } result.setAutoCommit(context.isAutoCommit()); result.setSpentTimeMills(System.currentTimeMillis() - beginTime); log.info("consumeMessageDirectly Result: {}", result); return result; } /** * 当 dispathToConsume=true 时提交消费请求,不指定拉取的消息,仅指明MessageQueue,ProcessQueue * 因此,这个消费请求会一直消费,直到{@link ProcessQueue#msgTreeMap}里没有消息 * 可能出现这种情况,消息消费的速度慢于拉取的速度,那么一个消费请求会一直持续消费 * 也就是一个线程一直维持着消费消息,不释放MessageQueue的{@link MessageQueueLock}锁 * 其他线程干瞪眼等待 * * @param msgs * @param processQueue * @param messageQueue * @param dispathToConsume */ @Override public void submitConsumeRequest(final List<MessageExt> msgs, final ProcessQueue processQueue, final MessageQueue messageQueue, final boolean dispathToConsume) { if (dispathToConsume) { ConsumeRequest consumeRequest = new ConsumeRequest(processQueue, messageQueue); this.consumeExecutor.submit(consumeRequest); } } /** * 默认每20S锁定当前持有的所有MessageQueue */ public synchronized void lockMQPeriodically() { if (!this.stopped) { this.defaultMQPushConsumerImpl.getRebalanceImpl().lockAll(); } } /** * 向Broker请求指定MessageQueue的分布式锁,成功则10ms后开始消费,失败则3S后开始消费,在消费时还需要再次获取分布式锁 * * @param mq * @param processQueue * @param delayMills */ public void tryLockLaterAndReconsume(final MessageQueue mq, final ProcessQueue processQueue, final long delayMills) { this.scheduledExecutorService.schedule(new Runnable() { @Override public void run() { boolean lockOK = ConsumeMessageOrderlyService.this.lockOneMQ(mq); if (lockOK) { ConsumeMessageOrderlyService.this.submitConsumeRequestLater(processQueue, mq, 10); } else { ConsumeMessageOrderlyService.this.submitConsumeRequestLater(processQueue, mq, 3000); } } }, delayMills, TimeUnit.MILLISECONDS); } /** * 锁定 Broker 消息队列 * * @param mq 消息队列 * @return 是否成功 */ public synchronized boolean lockOneMQ(final MessageQueue mq) { if (!this.stopped) { return this.defaultMQPushConsumerImpl.getRebalanceImpl().lock(mq); } return false; } /** * 提交延迟消费请求 * 一般挂起1S * * @param processQueue 消息处理队列 * @param messageQueue 消息队列 * @param suspendTimeMillis 延迟时长 */ private void submitConsumeRequestLater(final ProcessQueue processQueue, final MessageQueue messageQueue, final long suspendTimeMillis ) { long timeMillis = suspendTimeMillis; if (timeMillis == -1) { timeMillis = this.defaultMQPushConsumer.getSuspendCurrentQueueTimeMillis(); //1000 } if (timeMillis < 10) { timeMillis = 10; } else if (timeMillis > 30000) { timeMillis = 30000; } this.scheduledExecutorService.schedule(new Runnable() { @Override public void run() { ConsumeMessageOrderlyService.this.submitConsumeRequest(null, processQueue, messageQueue, true); } }, timeMillis, TimeUnit.MILLISECONDS); } /** * 处理消费结果,并返回是否继续消费 * * @param msgs 消息 * @param status 消费结果状态 * @param context 消费Context * @param consumeRequest 消费请求 * @return 是否继续消费 */ public boolean processConsumeResult(final List<MessageExt> msgs, final ConsumeOrderlyStatus status, final ConsumeOrderlyContext context, final ConsumeRequest consumeRequest ) { boolean continueConsume = true; long commitOffset = -1L; if (context.isAutoCommit()) { //默认 AutoCommit = true switch (status) { case COMMIT: case ROLLBACK: log.warn("the message queue consume result is illegal, we think you want to ack these message {}", consumeRequest.getMessageQueue()); case SUCCESS: // 提交消息已消费成功到消息处理队列 commitOffset = consumeRequest.getProcessQueue().commit(); // 统计 this.getConsumerStatsManager().incConsumeOKTPS(consumerGroup, consumeRequest.getMessageQueue().getTopic(), msgs.size()); break; case SUSPEND_CURRENT_QUEUE_A_MOMENT: // 统计 this.getConsumerStatsManager().incConsumeFailedTPS(consumerGroup, consumeRequest.getMessageQueue().getTopic(), msgs.size()); // 统计是否对这些消息重消费,只要重消费次数未达到最大值,都会返回true if (checkReconsumeTimes(msgs)) { // 设置消息重新消费,将{@link #msgTreeMapTemp}里的消息重新放回{@link #msgTreeMap},这样重新消费时就能再次消费这些消息 consumeRequest.getProcessQueue().makeMessageToCosumeAgain(msgs); // 提交延迟消费请求,在1S后重新开启消费 this.submitConsumeRequestLater(consumeRequest.getProcessQueue(), consumeRequest.getMessageQueue(), context.getSuspendCurrentQueueTimeMillis()); continueConsume = false; } else { commitOffset = consumeRequest.getProcessQueue().commit(); } break; default: break; } } else { switch (status) { case SUCCESS: this.getConsumerStatsManager().incConsumeOKTPS(consumerGroup, consumeRequest.getMessageQueue().getTopic(), msgs.size()); break; case COMMIT: // 提交消息已消费成功到消息处理队列 commitOffset = consumeRequest.getProcessQueue().commit(); break; case ROLLBACK: // 设置消息重新消费 consumeRequest.getProcessQueue().rollback(); this.submitConsumeRequestLater(// consumeRequest.getProcessQueue(), // consumeRequest.getMessageQueue(), // context.getSuspendCurrentQueueTimeMillis()); continueConsume = false; break; case SUSPEND_CURRENT_QUEUE_A_MOMENT: // 计算是否暂时挂起(暂停)消费N毫秒,默认:10ms this.getConsumerStatsManager().incConsumeFailedTPS(consumerGroup, consumeRequest.getMessageQueue().getTopic(), msgs.size()); if (checkReconsumeTimes(msgs)) { // 设置消息重新消费 consumeRequest.getProcessQueue().makeMessageToCosumeAgain(msgs); // 提交延迟消费请求 this.submitConsumeRequestLater(// consumeRequest.getProcessQueue(), // consumeRequest.getMessageQueue(), // context.getSuspendCurrentQueueTimeMillis()); continueConsume = false; } break; default: break; } } // 消息处理队列未dropped,提交有效消费进度 if (commitOffset >= 0 && !consumeRequest.getProcessQueue().isDropped()) { this.defaultMQPushConsumerImpl.getOffsetStore().updateOffset(consumeRequest.getMessageQueue(), commitOffset, false); } return continueConsume; } public ConsumerStatsManager getConsumerStatsManager() { return this.defaultMQPushConsumerImpl.getConsumerStatsManager(); } private int getMaxReconsumeTimes() { // default reconsume times: Integer.MAX_VALUE if (this.defaultMQPushConsumer.getMaxReconsumeTimes() == -1) { return Integer.MAX_VALUE; } else { return this.defaultMQPushConsumer.getMaxReconsumeTimes(); } } /** * 计算是否要暂停消费 * 不暂停条件:存在消息都超过最大消费次数并且都发回broker成功 * * @param msgs 消息 * @return 是否要暂停 */ private boolean checkReconsumeTimes(List<MessageExt> msgs) { boolean suspend = false; if (msgs != null && !msgs.isEmpty()) { for (MessageExt msg : msgs) { if (msg.getReconsumeTimes() >= getMaxReconsumeTimes()) { MessageAccessor.setReconsumeTime(msg, String.valueOf(msg.getReconsumeTimes())); if (!sendMessageBack(msg)) { suspend = true; msg.setReconsumeTimes(msg.getReconsumeTimes() + 1); } } else { suspend = true; msg.setReconsumeTimes(msg.getReconsumeTimes() + 1); } } } return suspend; } /** * 发回消息。 * 消息发回broker后,对应的消息队列是死信队列。 * * @param msg 消息 * @return 是否发送成功 */ public boolean sendMessageBack(final MessageExt msg) { try { // max reconsume times exceeded then send to dead letter queue. Message newMsg = new Message(MixAll.getRetryTopic(this.defaultMQPushConsumer.getConsumerGroup()), msg.getBody()); String originMsgId = MessageAccessor.getOriginMessageId(msg); MessageAccessor.setOriginMessageId(newMsg, UtilAll.isBlank(originMsgId) ? msg.getMsgId() : originMsgId); newMsg.setFlag(msg.getFlag()); MessageAccessor.setProperties(newMsg, msg.getProperties()); MessageAccessor.putProperty(newMsg, MessageConst.PROPERTY_RETRY_TOPIC, msg.getTopic()); MessageAccessor.setReconsumeTime(newMsg, String.valueOf(msg.getReconsumeTimes())); MessageAccessor.setMaxReconsumeTimes(newMsg, String.valueOf(getMaxReconsumeTimes())); newMsg.setDelayTimeLevel(3 + msg.getReconsumeTimes()); this.defaultMQPushConsumer.getDefaultMQPushConsumerImpl().getmQClientFactory().getDefaultMQProducer().send(newMsg); return true; } catch (Exception e) { log.error("sendMessageBack exception, group: " + this.consumerGroup + " msg: " + msg.toString(), e); } return false; } /** * 消费请求 */ class ConsumeRequest implements Runnable { /** * 消息处理队列 */ private final ProcessQueue processQueue; /** * 消息队列 */ private final MessageQueue messageQueue; public ConsumeRequest(ProcessQueue processQueue, MessageQueue messageQueue) { this.processQueue = processQueue; this.messageQueue = messageQueue; } public ProcessQueue getProcessQueue() { return processQueue; } public MessageQueue getMessageQueue() { return messageQueue; } @Override public void run() { if (this.processQueue.isDropped()) { log.warn("run, the message queue not be able to consume, because it's dropped. {}", this.messageQueue); return; } // 获得 Consumer 消息队列锁 final Object objLock = messageQueueLock.fetchLockObject(this.messageQueue); synchronized (objLock) { // (广播模式) || (集群模式 && Broker消息队列锁有效) if (MessageModel.BROADCASTING.equals(ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.messageModel()) || (this.processQueue.isLocked() && !this.processQueue.isLockExpired())) { final long beginTime = System.currentTimeMillis(); // 循环 for (boolean continueConsume = true; continueConsume; ) { //负载均衡后,当前MessageQueue不在属于自己,马上设置其ProcessQueue的dropped = true ,终止剩余消息的消费 if (this.processQueue.isDropped()) { log.warn("the message queue not be able to consume, because it's dropped. {}", this.messageQueue); break; } // ProcessQueue未被锁定,向Broker请求锁定相应的MessageQueue,成功了锁定ProcessQueue,然后再次消费 if (MessageModel.CLUSTERING.equals(ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.messageModel()) && !this.processQueue.isLocked()) { log.warn("the message queue not locked, so consume later, {}", this.messageQueue); ConsumeMessageOrderlyService.this.tryLockLaterAndReconsume(this.messageQueue, this.processQueue, 10); break; } // ProcessQueue锁过期(30S),向Broker请求锁定相应的MessageQueue,成功了锁定ProcessQueue,然后再次消费 if (MessageModel.CLUSTERING.equals(ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.messageModel()) && this.processQueue.isLockExpired()) { log.warn("the message queue lock expired, so consume later, {}", this.messageQueue); ConsumeMessageOrderlyService.this.tryLockLaterAndReconsume(this.messageQueue, this.processQueue, 10); break; } // 当前轮次消费时间超过连续时长,默认:60s,提交延迟消费请求。默认情况下,每消费1分钟休息10ms。 long interval = System.currentTimeMillis() - beginTime; if (interval > MAX_TIME_CONSUME_CONTINUOUSLY) { ConsumeMessageOrderlyService.this.submitConsumeRequestLater(processQueue, messageQueue, 10); break; } // 获取消费消息。此处和并发消息请求不同,并发消息请求已经带了消费哪些消息。 final int consumeBatchSize = ConsumeMessageOrderlyService.this.defaultMQPushConsumer.getConsumeMessageBatchMaxSize(); //按顺序提取msgTreeMap里的相应数量消息至msgTreeMapTemp临时集合中,此举会删除msgTreeMap里的那些消息,也就是这些消息目前只在临时集合msgTreeMapTemp中 List<MessageExt> msgs = this.processQueue.takeMessags(consumeBatchSize); if (!msgs.isEmpty()) { final ConsumeOrderlyContext context = new ConsumeOrderlyContext(this.messageQueue); ConsumeOrderlyStatus status = null; // Hook:before ConsumeMessageContext consumeMessageContext = null; if (ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.hasHook()) { consumeMessageContext = new ConsumeMessageContext(); consumeMessageContext .setConsumerGroup(ConsumeMessageOrderlyService.this.defaultMQPushConsumer.getConsumerGroup()); consumeMessageContext.setMq(messageQueue); consumeMessageContext.setMsgList(msgs); consumeMessageContext.setSuccess(false); // init the consume context type consumeMessageContext.setProps(new HashMap<String, String>()); ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.executeHookBefore(consumeMessageContext); } // 执行消费 long beginTimestamp = System.currentTimeMillis(); ConsumeReturnType returnType = ConsumeReturnType.SUCCESS; boolean hasException = false; try { this.processQueue.getLockConsume().lock(); // 获取队列消费锁 if (this.processQueue.isDropped()) { log.warn("consumeMessage, the message queue not be able to consume, because it's dropped. {}", this.messageQueue); break; } status = messageListener.consumeMessage(Collections.unmodifiableList(msgs), context); } catch (Throwable e) { log.warn("consumeMessage exception: {} Group: {} Msgs: {} MQ: {}", // RemotingHelper.exceptionSimpleDesc(e), // ConsumeMessageOrderlyService.this.consumerGroup, // msgs, // messageQueue); hasException = true; } finally { this.processQueue.getLockConsume().unlock(); // 释放队列消费锁 } if (null == status || ConsumeOrderlyStatus.ROLLBACK == status || ConsumeOrderlyStatus.SUSPEND_CURRENT_QUEUE_A_MOMENT == status) { log.warn("consumeMessage Orderly return not OK, Group: {} Msgs: {} MQ: {}", ConsumeMessageOrderlyService.this.consumerGroup, msgs, messageQueue); } // 解析消费结果状态 long consumeRT = System.currentTimeMillis() - beginTimestamp; if (null == status) { if (hasException) { returnType = ConsumeReturnType.EXCEPTION; } else { returnType = ConsumeReturnType.RETURNNULL; } } else if (consumeRT >= defaultMQPushConsumer.getConsumeTimeout() * 60 * 1000) { returnType = ConsumeReturnType.TIME_OUT; } else if (ConsumeOrderlyStatus.SUSPEND_CURRENT_QUEUE_A_MOMENT == status) { returnType = ConsumeReturnType.FAILED; } else if (ConsumeOrderlyStatus.SUCCESS == status) { returnType = ConsumeReturnType.SUCCESS; } if (ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.hasHook()) { consumeMessageContext.getProps().put(MixAll.CONSUME_CONTEXT_TYPE, returnType.name()); } if (null == status) { status = ConsumeOrderlyStatus.SUSPEND_CURRENT_QUEUE_A_MOMENT; } // Hook:after if (ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.hasHook()) { consumeMessageContext.setStatus(status.toString()); consumeMessageContext .setSuccess(ConsumeOrderlyStatus.SUCCESS == status || ConsumeOrderlyStatus.COMMIT == status); ConsumeMessageOrderlyService.this.defaultMQPushConsumerImpl.executeHookAfter(consumeMessageContext); } ConsumeMessageOrderlyService.this.getConsumerStatsManager() .incConsumeRT(ConsumeMessageOrderlyService.this.consumerGroup, messageQueue.getTopic(), consumeRT); // 处理消费结果 continueConsume = ConsumeMessageOrderlyService.this.processConsumeResult(msgs, status, context, this); } else { continueConsume = false; } } } else { if (this.processQueue.isDropped()) { log.warn("the message queue not be able to consume, because it's dropped. {}", this.messageQueue); return; } ConsumeMessageOrderlyService.this.tryLockLaterAndReconsume(this.messageQueue, this.processQueue, 100); } } } } }