Use the store based cursor by default for Queues - which will enable very large queue support

git-svn-id: https://svn.apache.org/repos/asf/incubator/activemq/trunk@490814 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Davies 2006-12-28 21:03:53 +00:00
parent 6964747912
commit d2e606208d
7 changed files with 142 additions and 73 deletions

View File

@ -38,6 +38,7 @@ import org.apache.activemq.memory.UsageManager;
import org.apache.activemq.thread.TaskRunnerFactory; import org.apache.activemq.thread.TaskRunnerFactory;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import sun.security.x509.IssuerAlternativeNameExtension;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
@ -60,6 +61,7 @@ abstract public class AbstractRegion implements Region {
protected final TaskRunnerFactory taskRunnerFactory; protected final TaskRunnerFactory taskRunnerFactory;
protected final Object destinationsMutex = new Object(); protected final Object destinationsMutex = new Object();
protected final Map consumerChangeMutexMap = new HashMap(); protected final Map consumerChangeMutexMap = new HashMap();
protected boolean started = false;
public AbstractRegion(RegionBroker broker,DestinationStatistics destinationStatistics, UsageManager memoryManager, TaskRunnerFactory taskRunnerFactory, DestinationFactory destinationFactory) { public AbstractRegion(RegionBroker broker,DestinationStatistics destinationStatistics, UsageManager memoryManager, TaskRunnerFactory taskRunnerFactory, DestinationFactory destinationFactory) {
if (broker == null) { if (broker == null) {
@ -76,9 +78,15 @@ abstract public class AbstractRegion implements Region {
} }
public void start() throws Exception { public void start() throws Exception {
started = true;
for (Iterator i = destinations.values().iterator();i.hasNext();) {
Destination dest = (Destination)i.next();
dest.start();
}
} }
public void stop() throws Exception { public void stop() throws Exception {
started = false;
for (Iterator i = destinations.values().iterator();i.hasNext();) { for (Iterator i = destinations.values().iterator();i.hasNext();) {
Destination dest = (Destination)i.next(); Destination dest = (Destination)i.next();
dest.stop(); dest.stop();
@ -102,7 +110,7 @@ abstract public class AbstractRegion implements Region {
if (destinationInterceptor != null) { if (destinationInterceptor != null) {
dest = destinationInterceptor.intercept(dest); dest = destinationInterceptor.intercept(dest);
} }
dest.start(); dest.start();
destinations.put(destination, dest); destinations.put(destination, dest);

View File

@ -77,7 +77,7 @@ public class DestinationFactoryImpl extends DestinationFactory {
if (destination.isQueue()) { if (destination.isQueue()) {
if (destination.isTemporary()) { if (destination.isTemporary()) {
final ActiveMQTempDestination tempDest = (ActiveMQTempDestination) destination; final ActiveMQTempDestination tempDest = (ActiveMQTempDestination) destination;
return new Queue(destination, memoryManager, null, destinationStatistics, taskRunnerFactory) { return new Queue(destination, memoryManager, null, destinationStatistics, taskRunnerFactory,broker.getTempDataStore()) {
public void addSubscription(ConnectionContext context,Subscription sub) throws Exception { public void addSubscription(ConnectionContext context,Subscription sub) throws Exception {
// Only consumers on the same connection can consume from // Only consumers on the same connection can consume from
@ -90,7 +90,7 @@ public class DestinationFactoryImpl extends DestinationFactory {
}; };
} else { } else {
MessageStore store = persistenceAdapter.createQueueMessageStore((ActiveMQQueue) destination); MessageStore store = persistenceAdapter.createQueueMessageStore((ActiveMQQueue) destination);
Queue queue = new Queue(destination, memoryManager, store, destinationStatistics, taskRunnerFactory); Queue queue = new Queue(destination, memoryManager, store, destinationStatistics, taskRunnerFactory,broker.getTempDataStore());
configureQueue(queue, destination); configureQueue(queue, destination);
queue.initialize(); queue.initialize();
return queue; return queue;

View File

@ -19,22 +19,21 @@ package org.apache.activemq.broker.region;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.concurrent.ConcurrentHashMap;
import javax.jms.InvalidSelectorException; import javax.jms.InvalidSelectorException;
import javax.jms.JMSException;
import org.apache.activemq.broker.Broker; import org.apache.activemq.broker.Broker;
import org.apache.activemq.broker.ConnectionContext; import org.apache.activemq.broker.ConnectionContext;
import org.apache.activemq.broker.region.cursors.FilePendingMessageCursor;
import org.apache.activemq.broker.region.cursors.PendingMessageCursor; import org.apache.activemq.broker.region.cursors.PendingMessageCursor;
import org.apache.activemq.broker.region.cursors.StoreDurableSubscriberCursor;
import org.apache.activemq.command.ConsumerInfo; import org.apache.activemq.command.ConsumerInfo;
import org.apache.activemq.command.Message; import org.apache.activemq.command.Message;
import org.apache.activemq.command.MessageAck; import org.apache.activemq.command.MessageAck;
import org.apache.activemq.command.MessageDispatch; import org.apache.activemq.command.MessageDispatch;
import org.apache.activemq.util.SubscriptionKey; import org.apache.activemq.util.SubscriptionKey;
import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
public class DurableTopicSubscription extends PrefetchSubscription { public class DurableTopicSubscription extends PrefetchSubscription {
static private final Log log=LogFactory.getLog(PrefetchSubscription.class);
private final ConcurrentHashMap redeliveredMessages = new ConcurrentHashMap(); private final ConcurrentHashMap redeliveredMessages = new ConcurrentHashMap();
private final ConcurrentHashMap destinations = new ConcurrentHashMap(); private final ConcurrentHashMap destinations = new ConcurrentHashMap();
private final SubscriptionKey subscriptionKey; private final SubscriptionKey subscriptionKey;
@ -72,6 +71,7 @@ public class DurableTopicSubscription extends PrefetchSubscription {
} }
public void activate(ConnectionContext context, ConsumerInfo info) throws Exception { public void activate(ConnectionContext context, ConsumerInfo info) throws Exception {
log.debug("Deactivating " + this);
if( !active ) { if( !active ) {
this.active = true; this.active = true;
this.context = context; this.context = context;
@ -96,7 +96,8 @@ public class DurableTopicSubscription extends PrefetchSubscription {
} }
} }
synchronized public void deactivate(boolean keepDurableSubsActive) throws Exception { synchronized public void deactivate(boolean keepDurableSubsActive) throws Exception {
active=false; active=false;
synchronized(pending){ synchronized(pending){
pending.stop(); pending.stop();
@ -197,9 +198,12 @@ public class DurableTopicSubscription extends PrefetchSubscription {
"DurableTopicSubscription:" + "DurableTopicSubscription:" +
" consumer="+info.getConsumerId()+ " consumer="+info.getConsumerId()+
", destinations="+destinations.size()+ ", destinations="+destinations.size()+
", dispatched="+dispatched.size()+ ", total="+enqueueCounter+
", delivered="+this.prefetchExtension+ ", pending="+getPendingQueueSize()+
", pending="+getPendingQueueSize(); ", dispatched="+dispatchCounter+
", inflight="+dispatched.size()+
", prefetchExtension="+this.prefetchExtension;
} }
public String getClientId() { public String getClientId() {

View File

@ -327,6 +327,10 @@ abstract public class PrefetchSubscription extends AbstractSubscription{
return (dispatched.size()-prefetchExtension) >= (info.getPrefetchSize() *.9); return (dispatched.size()-prefetchExtension) >= (info.getPrefetchSize() *.9);
} }
public int countBeforeFull() {
return info.getPrefetchSize() + prefetchExtension - dispatched.size();
}
public int getPendingQueueSize(){ public int getPendingQueueSize(){
synchronized(pending) { synchronized(pending) {
return pending.size(); return pending.size();
@ -396,28 +400,38 @@ abstract public class PrefetchSubscription extends AbstractSubscription{
List toDispatch=null; List toDispatch=null;
synchronized(pending){ synchronized(pending){
try{ try{
pending.reset(); int numberToDispatch=countBeforeFull();
while(pending.hasNext()&&!isFull()){ if(numberToDispatch>0){
MessageReference node=pending.next(); int count=0;
pending.remove(); pending.reset();
// Message may have been sitting in the pending list a while while(pending.hasNext()&&!isFull()&&count<numberToDispatch){
// waiting for the consumer to ak the message. MessageReference node=pending.next();
if(node!=QueueMessageReference.NULL_MESSAGE&&node.isExpired()){
continue; // just drop it. if(canDispatch(node)){
pending.remove();
// Message may have been sitting in the pending list a while
// waiting for the consumer to ak the message.
if(node!=QueueMessageReference.NULL_MESSAGE&&node.isExpired()){
continue; // just drop it.
}
if(toDispatch==null){
toDispatch=new ArrayList();
}
toDispatch.add(node);
count++;
}
} }
if(toDispatch==null){
toDispatch=new ArrayList();
}
toDispatch.add(node);
} }
}finally{ }finally{
pending.release(); pending.release();
} }
} }
if(toDispatch!=null){ if(toDispatch!=null){
for(int i=0;i<toDispatch.size();i++){ synchronized(dispatched){
MessageReference node=(MessageReference)toDispatch.get(i); for(int i=0;i<toDispatch.size();i++){
dispatch(node); MessageReference node=(MessageReference)toDispatch.get(i);
dispatch(node);
}
} }
} }
}finally{ }finally{
@ -458,6 +472,7 @@ abstract public class PrefetchSubscription extends AbstractSubscription{
} }
return true; return true;
}else{ }else{
QueueMessageReference n = (QueueMessageReference) node;
return false; return false;
} }
} }

View File

@ -28,6 +28,7 @@ import javax.jms.JMSException;
import org.apache.activemq.broker.ConnectionContext; import org.apache.activemq.broker.ConnectionContext;
import org.apache.activemq.broker.region.cursors.PendingMessageCursor; import org.apache.activemq.broker.region.cursors.PendingMessageCursor;
import org.apache.activemq.broker.region.cursors.StoreQueueCursor;
import org.apache.activemq.broker.region.cursors.VMPendingMessageCursor; import org.apache.activemq.broker.region.cursors.VMPendingMessageCursor;
import org.apache.activemq.broker.region.group.MessageGroupHashBucketFactory; import org.apache.activemq.broker.region.group.MessageGroupHashBucketFactory;
import org.apache.activemq.broker.region.group.MessageGroupMap; import org.apache.activemq.broker.region.group.MessageGroupMap;
@ -44,6 +45,7 @@ import org.apache.activemq.command.MessageAck;
import org.apache.activemq.command.MessageId; import org.apache.activemq.command.MessageId;
import org.apache.activemq.filter.BooleanExpression; import org.apache.activemq.filter.BooleanExpression;
import org.apache.activemq.filter.MessageEvaluationContext; import org.apache.activemq.filter.MessageEvaluationContext;
import org.apache.activemq.kaha.Store;
import org.apache.activemq.memory.UsageManager; import org.apache.activemq.memory.UsageManager;
import org.apache.activemq.selector.SelectorParser; import org.apache.activemq.selector.SelectorParser;
import org.apache.activemq.store.MessageRecoveryListener; import org.apache.activemq.store.MessageRecoveryListener;
@ -74,7 +76,7 @@ public class Queue implements Destination, Task {
private final Valve dispatchValve = new Valve(true); private final Valve dispatchValve = new Valve(true);
private final UsageManager usageManager; private final UsageManager usageManager;
private final DestinationStatistics destinationStatistics = new DestinationStatistics(); private final DestinationStatistics destinationStatistics = new DestinationStatistics();
private PendingMessageCursor messages = new VMPendingMessageCursor(); private PendingMessageCursor messages;
private final LinkedList pagedInMessages = new LinkedList(); private final LinkedList pagedInMessages = new LinkedList();
private LockOwner exclusiveOwner; private LockOwner exclusiveOwner;
@ -92,13 +94,20 @@ public class Queue implements Destination, Task {
private final Object exclusiveLockMutex = new Object(); private final Object exclusiveLockMutex = new Object();
private final Object doDispatchMutex = new Object(); private final Object doDispatchMutex = new Object();
private TaskRunner taskRunner; private TaskRunner taskRunner;
private boolean started = false;
public Queue(ActiveMQDestination destination, final UsageManager memoryManager, MessageStore store, DestinationStatistics parentStats, public Queue(ActiveMQDestination destination, final UsageManager memoryManager, MessageStore store, DestinationStatistics parentStats,
TaskRunnerFactory taskFactory) throws Exception { TaskRunnerFactory taskFactory, Store tmpStore) throws Exception {
this.destination = destination; this.destination = destination;
this.usageManager = new UsageManager(memoryManager); this.usageManager = new UsageManager(memoryManager);
this.usageManager.setLimit(Long.MAX_VALUE); this.usageManager.setLimit(Long.MAX_VALUE);
this.store = store; this.store = store;
if(destination.isTemporary()){
this.messages=new VMPendingMessageCursor();
}else{
this.messages=new StoreQueueCursor(this,tmpStore);
}
this.taskRunner = taskFactory.createTaskRunner(this, "Queue "+destination.getPhysicalName()); this.taskRunner = taskFactory.createTaskRunner(this, "Queue "+destination.getPhysicalName());
// Let the store know what usage manager we are using so that he can // Let the store know what usage manager we are using so that he can
@ -118,18 +127,16 @@ public class Queue implements Destination, Task {
if(store!=null){ if(store!=null){
// Restore the persistent messages. // Restore the persistent messages.
messages.setUsageManager(getUsageManager()); messages.setUsageManager(getUsageManager());
messages.start();
if(messages.isRecoveryRequired()){ if(messages.isRecoveryRequired()){
store.recover(new MessageRecoveryListener(){ store.recover(new MessageRecoveryListener(){
public void recoverMessage(Message message){ public void recoverMessage(Message message){
// Message could have expired while it was being loaded.. // Message could have expired while it was being loaded..
if( message.isExpired() ) { if(message.isExpired()){
// TODO: remove message from store. // TODO remove from store
return; return;
} }
message.setRegionDestination(Queue.this);
message.setRegionDestination(Queue.this);
synchronized(messages){ synchronized(messages){
try{ try{
messages.addMessageLast(message); messages.addMessageLast(message);
@ -157,10 +164,12 @@ public class Queue implements Destination, Task {
/** /**
* Lock a node * Lock a node
*
* @param node * @param node
* @param lockOwner * @param lockOwner
* @return true if can be locked * @return true if can be locked
* @see org.apache.activemq.broker.region.Destination#lock(org.apache.activemq.broker.region.MessageReference, org.apache.activemq.broker.region.LockOwner) * @see org.apache.activemq.broker.region.Destination#lock(org.apache.activemq.broker.region.MessageReference,
* org.apache.activemq.broker.region.LockOwner)
*/ */
public boolean lock(MessageReference node,LockOwner lockOwner){ public boolean lock(MessageReference node,LockOwner lockOwner){
synchronized(exclusiveLockMutex){ synchronized(exclusiveLockMutex){
@ -309,46 +318,60 @@ public class Queue implements Destination, Task {
} }
public void send(final ConnectionContext context,final Message message) throws Exception{ public void send(final ConnectionContext context,final Message message) throws Exception{
// There is delay between the client sending it and it arriving at the // There is delay between the client sending it and it arriving at the
// destination.. it may have expired. // destination.. it may have expired.
if( message.isExpired() ) { if(message.isExpired()){
return; if (log.isDebugEnabled()) {
} log.debug("Expired message: " + message);
}
return;
}
if(context.isProducerFlowControl()){ if(context.isProducerFlowControl()){
if(usageManager.isSendFailIfNoSpace()&&usageManager.isFull()){ if(usageManager.isSendFailIfNoSpace()&&usageManager.isFull()){
throw new javax.jms.ResourceAllocationException("Usage Manager memory limit reached"); throw new javax.jms.ResourceAllocationException("Usage Manager memory limit reached");
}else{ }else{
usageManager.waitForSpace(); usageManager.waitForSpace();
// The usage manager could have delayed us by the time // The usage manager could have delayed us by the time
// we unblock the message could have expired.. // we unblock the message could have expired..
if( message.isExpired() ) { if(message.isExpired()){
return; if (log.isDebugEnabled()) {
} log.debug("Expired message: " + message);
}
return;
}
} }
} }
message.setRegionDestination(this); message.setRegionDestination(this);
if (store != null && message.isPersistent()) { if(store!=null&&message.isPersistent()){
store.addMessage(context, message); store.addMessage(context,message);
} }
if(context.isInTransaction()){ if(context.isInTransaction()){
context.getTransaction().addSynchronization(new Synchronization(){ context.getTransaction().addSynchronization(new Synchronization(){
public void afterCommit() throws Exception{ public void afterCommit() throws Exception{
//even though the message could be expired - it won't be from the store
// It could take while before we receive the commit //and it's important to keep the store/cursor in step
// operration.. by that time the message could have expired.. synchronized(messages){
if( message.isExpired() ) { messages.addMessageLast(message);
// TODO: remove message from store. }
return; // It could take while before we receive the commit
} // operration.. by that time the message could have expired..
if(message.isExpired()){
// TODO: remove message from store.
if (log.isDebugEnabled()) {
log.debug("Expired message: " + message);
}
return;
}
sendMessage(context,message); sendMessage(context,message);
} }
}); });
}else{ }else{
synchronized(messages){
messages.addMessageLast(message);
}
sendMessage(context,message); sendMessage(context,message);
} }
} }
@ -432,12 +455,19 @@ public class Queue implements Destination, Task {
} }
public void start() throws Exception { public void start() throws Exception {
started = true;
messages.start();
doPageIn(false);
} }
public void stop() throws Exception { public void stop() throws Exception {
started = false;
if( taskRunner!=null ) { if( taskRunner!=null ) {
taskRunner.shutdown(); taskRunner.shutdown();
} }
if(messages!=null){
messages.stop();
}
} }
// Properties // Properties
@ -528,6 +558,11 @@ public class Queue implements Destination, Task {
public Message[] browse() { public Message[] browse() {
ArrayList l = new ArrayList(); ArrayList l = new ArrayList();
try{
doPageIn(true);
}catch(Exception e){
log.error("caught an exception browsing " + this,e);
}
synchronized(pagedInMessages) { synchronized(pagedInMessages) {
for (Iterator i = pagedInMessages.iterator();i.hasNext();) { for (Iterator i = pagedInMessages.iterator();i.hasNext();) {
MessageReference r = (MessageReference)i.next(); MessageReference r = (MessageReference)i.next();
@ -538,7 +573,7 @@ public class Queue implements Destination, Task {
l.add(m); l.add(m);
} }
}catch(IOException e){ }catch(IOException e){
log.error("caught an exception brwsing " + this,e); log.error("caught an exception browsing " + this,e);
} }
finally { finally {
r.decrementReferenceCount(); r.decrementReferenceCount();
@ -850,11 +885,10 @@ public class Queue implements Destination, Task {
return answer; return answer;
} }
private void sendMessage(final ConnectionContext context,Message msg) throws Exception{ private void sendMessage(final ConnectionContext context,Message msg) throws Exception{
synchronized(messages){
messages.addMessageLast(msg);
}
destinationStatistics.getEnqueues().increment(); destinationStatistics.getEnqueues().increment();
destinationStatistics.getMessages().increment(); destinationStatistics.getMessages().increment();
pageInMessages(false); pageInMessages(false);
@ -863,10 +897,11 @@ public class Queue implements Destination, Task {
private List doPageIn() throws Exception{ private List doPageIn() throws Exception{
return doPageIn(true); return doPageIn(true);
} }
private List doPageIn(boolean force) throws Exception{ private List doPageIn(boolean force) throws Exception{
final int toPageIn=maximumPagedInMessages-pagedInMessages.size(); final int toPageIn=maximumPagedInMessages-pagedInMessages.size();
List result=null; List result=null;
if((force || !consumers.isEmpty())&&toPageIn>0){ if((force||!consumers.isEmpty())&&toPageIn>0){
try{ try{
dispatchValve.increment(); dispatchValve.increment();
int count=0; int count=0;
@ -877,9 +912,15 @@ public class Queue implements Destination, Task {
while(messages.hasNext()&&count<toPageIn){ while(messages.hasNext()&&count<toPageIn){
MessageReference node=messages.next(); MessageReference node=messages.next();
messages.remove(); messages.remove();
node=createMessageReference(node.getMessage()); if(!node.isExpired()){
result.add(node); node=createMessageReference(node.getMessage());
count++; result.add(node);
count++;
}else{
if (log.isDebugEnabled()) {
log.debug("Expired message: " + node);
}
}
} }
}finally{ }finally{
messages.release(); messages.release();

View File

@ -73,7 +73,7 @@ public class QueueSubscription extends PrefetchSubscription implements LockOwner
protected boolean canDispatch(MessageReference n) throws IOException { protected boolean canDispatch(MessageReference n) throws IOException {
QueueMessageReference node = (QueueMessageReference) n; QueueMessageReference node = (QueueMessageReference) n;
if( node.isAcked() ) if( node.isAcked())
return false; return false;
// Keep message groups together. // Keep message groups together.
String groupId = node.getGroupID(); String groupId = node.getGroupID();
@ -208,7 +208,7 @@ public class QueueSubscription extends PrefetchSubscription implements LockOwner
/** /**
*/ */
synchronized public void destroy() { public void destroy() {
} }
} }

View File

@ -78,7 +78,7 @@ public class RegionBroker implements Broker {
private final Region tempQueueRegion; private final Region tempQueueRegion;
private final Region tempTopicRegion; private final Region tempTopicRegion;
private BrokerService brokerService; private BrokerService brokerService;
private boolean stopped = false; private boolean started = false;
private boolean keepDurableSubsActive=false; private boolean keepDurableSubsActive=false;
protected final DestinationStatistics destinationStatistics = new DestinationStatistics(); protected final DestinationStatistics destinationStatistics = new DestinationStatistics();
@ -178,6 +178,7 @@ public class RegionBroker implements Broker {
public void start() throws Exception { public void start() throws Exception {
((TopicRegion)topicRegion).setKeepDurableSubsActive(keepDurableSubsActive); ((TopicRegion)topicRegion).setKeepDurableSubsActive(keepDurableSubsActive);
started = true;
queueRegion.start(); queueRegion.start();
topicRegion.start(); topicRegion.start();
tempQueueRegion.start(); tempQueueRegion.start();
@ -185,7 +186,7 @@ public class RegionBroker implements Broker {
} }
public void stop() throws Exception { public void stop() throws Exception {
stopped = true; started = false;
ServiceStopper ss = new ServiceStopper(); ServiceStopper ss = new ServiceStopper();
doStop(ss); doStop(ss);
ss.throwFirstException(); ss.throwFirstException();
@ -245,7 +246,6 @@ public class RegionBroker implements Broker {
if( destinations.contains(destination) ){ if( destinations.contains(destination) ){
throw new JMSException("Destination already exists: "+destination); throw new JMSException("Destination already exists: "+destination);
} }
Destination answer = null; Destination answer = null;
switch(destination.getDestinationType()) { switch(destination.getDestinationType()) {
case ActiveMQDestination.QUEUE_TYPE: case ActiveMQDestination.QUEUE_TYPE:
@ -366,7 +366,8 @@ public class RegionBroker implements Broker {
} }
public void send(ConnectionContext context, Message message) throws Exception { public void send(ConnectionContext context, Message message) throws Exception {
message.getMessageId().setBrokerSequenceId(sequenceGenerator.getNextSequenceId()); long si = sequenceGenerator.getNextSequenceId();
message.getMessageId().setBrokerSequenceId(si);
if (message.getTimestamp() > 0 && (message.getBrokerPath() == null || message.getBrokerPath().length == 0)) { if (message.getTimestamp() > 0 && (message.getBrokerPath() == null || message.getBrokerPath().length == 0)) {
//timestamp not been disabled and has not passed through a network //timestamp not been disabled and has not passed through a network
message.setTimestamp(System.currentTimeMillis()); message.setTimestamp(System.currentTimeMillis());
@ -541,7 +542,7 @@ public class RegionBroker implements Broker {
} }
public boolean isStopped(){ public boolean isStopped(){
return stopped; return !started;
} }
public Set getDurableDestinations(){ public Set getDurableDestinations(){