/**
* 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.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.slf4j.Logger;
import com.alibaba.rocketmq.client.log.ClientLogger;
import com.alibaba.rocketmq.common.message.MessageExt;
/**
* 正在被消费的队列,含消息
*
* @author shijia.wxr<vintage.wang@gmail.com>
* @since 2013-7-24
*/
public class ProcessQueue {
// 客户端本地Lock存活最大时间,超过则自动过期,单位ms
public final static long RebalanceLockMaxLiveTime = Long.parseLong(System.getProperty(
"rocketmq.client.rebalance.lockMaxLiveTime", "30000"));
// 定时Lock间隔时间,单位ms
public final static long RebalanceLockInterval = Long.parseLong(System.getProperty(
"rocketmq.client.rebalance.lockInterval", "20000"));
private final Logger log = ClientLogger.getLog();
private final ReadWriteLock lockTreeMap = new ReentrantReadWriteLock();
private final TreeMap<Long, MessageExt> msgTreeMap = new TreeMap<Long, MessageExt>();
private volatile long queueOffsetMax = 0L;
private final AtomicLong msgCount = new AtomicLong();
// 当前Q是否被rebalance丢弃
private volatile boolean droped = false;
private volatile long lastPullTimestamp = System.currentTimeMillis();
private final static long PullMaxIdleTime = Long.parseLong(System.getProperty(
"rocketmq.client.pull.pullMaxIdleTime", "120000"));
/**
* 顺序消息专用
*/
private final Lock lockConsume = new ReentrantLock();
// 是否从Broker锁定
private volatile boolean locked = false;
// 最后一次锁定成功时间戳
private volatile long lastLockTimestamp = System.currentTimeMillis();
// 是否正在被消费
private volatile boolean consuming = false;
// 事务方式消费,未提交的消息
private final TreeMap<Long, MessageExt> msgTreeMapTemp = new TreeMap<Long, MessageExt>();
public boolean isLockExpired() {
boolean result = (System.currentTimeMillis() - this.lastLockTimestamp) > RebalanceLockMaxLiveTime;
return result;
}
public boolean isPullExpired() {
boolean result = (System.currentTimeMillis() - this.lastPullTimestamp) > PullMaxIdleTime;
return result;
}
/**
* @return 是否需要分发当前队列到消费线程池
*/
public boolean putMessage(final List<MessageExt> msgs) {
boolean dispathToConsume = false;
try {
this.lockTreeMap.writeLock().lockInterruptibly();
try {
int validMsgCnt = 0;
for (MessageExt msg : msgs) {
MessageExt old = msgTreeMap.put(msg.getQueueOffset(), msg);
if (null == old) {
validMsgCnt++;
this.queueOffsetMax = msg.getQueueOffset();
}
}
msgCount.addAndGet(validMsgCnt);
if (!msgTreeMap.isEmpty() && !this.consuming) {
dispathToConsume = true;
this.consuming = true;
}
}
finally {
this.lockTreeMap.writeLock().unlock();
}
}
catch (InterruptedException e) {
log.error("putMessage exception", e);
}
return dispathToConsume;
}
/**
* 获取当前队列的最大跨度
*/
public long getMaxSpan() {
try {
this.lockTreeMap.readLock().lockInterruptibly();
try {
if (!this.msgTreeMap.isEmpty()) {
return this.msgTreeMap.lastKey() - this.msgTreeMap.firstKey();
}
}
finally {
this.lockTreeMap.readLock().unlock();
}
}
catch (InterruptedException e) {
log.error("getMaxSpan exception", e);
}
return 0;
}
/**
* 删除已经消费过的消息,返回最小Offset,这个Offset对应的消息未消费
*
* @param msgs
* @return
*/
public long removeMessage(final List<MessageExt> msgs) {
long result = -1;
try {
this.lockTreeMap.writeLock().lockInterruptibly();
try {
if (!msgTreeMap.isEmpty()) {
result = this.queueOffsetMax + 1;
int removedCnt = 0;
for (MessageExt msg : msgs) {
MessageExt prev = msgTreeMap.remove(msg.getQueueOffset());
if (prev != null) {
removedCnt--;
}
}
msgCount.addAndGet(removedCnt);
if (!msgTreeMap.isEmpty()) {
result = msgTreeMap.firstKey();
}
}
}
finally {
this.lockTreeMap.writeLock().unlock();
}
}
catch (Throwable t) {
log.error("removeMessage exception", t);
}
return result;
}
public TreeMap<Long, MessageExt> getMsgTreeMap() {
return msgTreeMap;
}
public AtomicLong getMsgCount() {
return msgCount;
}
public boolean isDroped() {
return droped;
}
public void setDroped(boolean droped) {
this.droped = droped;
}
/**
* ========================================================================
* 以下部分为顺序消息专有操作
*/
public void setLocked(boolean locked) {
this.locked = locked;
}
public boolean isLocked() {
return locked;
}
public void rollback() {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
try {
this.msgTreeMap.putAll(this.msgTreeMapTemp);
this.msgTreeMapTemp.clear();
}
finally {
this.lockTreeMap.writeLock().unlock();
}
}
catch (InterruptedException e) {
log.error("rollback exception", e);
}
}
public long commit() {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
try {
Long offset = this.msgTreeMapTemp.lastKey();
msgCount.addAndGet(this.msgTreeMapTemp.size() * (-1));
this.msgTreeMapTemp.clear();
if (offset != null) {
return offset + 1;
}
}
finally {
this.lockTreeMap.writeLock().unlock();
}
}
catch (InterruptedException e) {
log.error("commit exception", e);
}
return -1;
}
public void makeMessageToCosumeAgain(List<MessageExt> msgs) {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
try {
// 临时Table删除
// 正常Table增加
for (MessageExt msg : msgs) {
this.msgTreeMapTemp.remove(msg.getQueueOffset());
this.msgTreeMap.put(msg.getQueueOffset(), msg);
}
}
finally {
this.lockTreeMap.writeLock().unlock();
}
}
catch (InterruptedException e) {
log.error("makeMessageToCosumeAgain exception", e);
}
}
/**
* 如果取不到消息,则将正在消费状态置为false
*
* @param batchSize
* @return
*/
public List<MessageExt> takeMessags(final int batchSize) {
List<MessageExt> result = new ArrayList<MessageExt>(batchSize);
try {
this.lockTreeMap.writeLock().lockInterruptibly();
try {
if (!this.msgTreeMap.isEmpty()) {
for (int i = 0; i < batchSize; i++) {
Map.Entry<Long, MessageExt> entry = this.msgTreeMap.pollFirstEntry();
if (entry != null) {
result.add(entry.getValue());
msgTreeMapTemp.put(entry.getKey(), entry.getValue());
}
else {
break;
}
}
}
if (result.isEmpty()) {
consuming = false;
}
}
finally {
this.lockTreeMap.writeLock().unlock();
}
}
catch (InterruptedException e) {
log.error("takeMessags exception", e);
}
return result;
}
public void clear() {
try {
this.lockTreeMap.writeLock().lockInterruptibly();
try {
this.msgTreeMap.clear();
this.msgTreeMapTemp.clear();
this.msgCount.set(0);
this.queueOffsetMax = 0L;
}
finally {
this.lockTreeMap.writeLock().unlock();
}
}
catch (InterruptedException e) {
log.error("rollback exception", e);
}
}
public long getLastLockTimestamp() {
return lastLockTimestamp;
}
public void setLastLockTimestamp(long lastLockTimestamp) {
this.lastLockTimestamp = lastLockTimestamp;
}
public Lock getLockConsume() {
return lockConsume;
}
public long getLastPullTimestamp() {
return lastPullTimestamp;
}
public void setLastPullTimestamp(long lastPullTimestamp) {
this.lastPullTimestamp = lastPullTimestamp;
}
}