tidied up some of the synchronization

git-svn-id: https://svn.apache.org/repos/asf/incubator/activemq/trunk@382746 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Davies 2006-03-03 10:06:41 +00:00
parent c916749e78
commit 6d003a89e2
1 changed files with 96 additions and 107 deletions

View File

@ -2,17 +2,14 @@
* *
* Copyright 2005-2006 The Apache Software Foundation * Copyright 2005-2006 The Apache Software Foundation
* *
* Licensed under the Apache License, Version 2.0 (the "License"); * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* you may not use this file except in compliance with the License. * the License. You may obtain a copy of the License at
* You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* distributed under the License is distributed on an "AS IS" BASIS, * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * specific language governing permissions and limitations under the License.
* See the License for the specific language governing permissions and
* limitations under the License.
*/ */
package org.apache.activemq.broker.region; package org.apache.activemq.broker.region;
@ -34,39 +31,44 @@ import org.apache.activemq.memory.UsageManager;
import org.apache.activemq.transaction.Synchronization; import org.apache.activemq.transaction.Synchronization;
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 edu.emory.mathcs.backport.java.util.concurrent.atomic.AtomicInteger;
public class TopicSubscription extends AbstractSubscription {
private static final Log log = LogFactory.getLog(TopicSubscription.class); public class TopicSubscription extends AbstractSubscription{
final protected LinkedList matched = new LinkedList(); private static final Log log=LogFactory.getLog(TopicSubscription.class);
final protected ActiveMQDestination dlqDestination = new ActiveMQQueue("ActiveMQ.DLQ"); final protected LinkedList matched=new LinkedList();
final protected ActiveMQDestination dlqDestination=new ActiveMQQueue("ActiveMQ.DLQ");
final protected UsageManager usageManager; final protected UsageManager usageManager;
protected int dispatched=0; protected AtomicInteger dispatched=new AtomicInteger();
protected int delivered=0; protected AtomicInteger delivered=new AtomicInteger();
private int maximumPendingMessages = -1; private int maximumPendingMessages=-1;
private final Object matchedListMutex=new Object();
public TopicSubscription(Broker broker,ConnectionContext context, ConsumerInfo info, UsageManager usageManager) throws InvalidSelectorException { public TopicSubscription(Broker broker,ConnectionContext context,ConsumerInfo info,UsageManager usageManager)
super(broker,context, info); throws InvalidSelectorException{
super(broker,context,info);
this.usageManager=usageManager; this.usageManager=usageManager;
} }
public void add(MessageReference node) throws InterruptedException, IOException { public void add(MessageReference node) throws InterruptedException,IOException{
node.incrementReferenceCount(); node.incrementReferenceCount();
if( !isFull() && !isSlaveBroker()) { if(!isFull()&&!isSlaveBroker()){
// if maximumPendingMessages is set we will only discard messages which // if maximumPendingMessages is set we will only discard messages which
// have not been dispatched (i.e. we allow the prefetch buffer to be filled) // have not been dispatched (i.e. we allow the prefetch buffer to be filled)
dispatch(node); dispatch(node);
} else { }else{
if (maximumPendingMessages != 0) { if(maximumPendingMessages!=0){
synchronized (matched) { synchronized(matchedListMutex){
matched.addLast(node); matched.addLast(node);
// NOTE - be careful about the slaveBroker! // NOTE - be careful about the slaveBroker!
if (maximumPendingMessages > 0) { if(maximumPendingMessages>0){
log.warn("discarding " + (matched.size() - maximumPendingMessages) + " messages for slow consumer");
// lets discard old messages as we are a slow consumer // lets discard old messages as we are a slow consumer
while (!matched.isEmpty() && matched.size() > maximumPendingMessages) { while(!matched.isEmpty()&&matched.size()>maximumPendingMessages){
MessageReference oldMessage = (MessageReference) matched.removeFirst(); MessageReference oldMessage=(MessageReference) matched.removeFirst();
oldMessage.decrementReferenceCount(); oldMessage.decrementReferenceCount();
if (log.isDebugEnabled()){
log.debug("Discarding message " + oldMessage);
}
} }
} }
} }
@ -74,13 +76,13 @@ public class TopicSubscription extends AbstractSubscription {
} }
} }
public void processMessageDispatchNotification(MessageDispatchNotification mdn){ public void processMessageDispatchNotification(MessageDispatchNotification mdn){
synchronized(matched){ synchronized(matchedListMutex){
for (Iterator i = matched.iterator(); i.hasNext();){ for(Iterator i=matched.iterator();i.hasNext();){
MessageReference node = (MessageReference)i.next(); MessageReference node=(MessageReference) i.next();
if (node.getMessageId().equals(mdn.getMessageId())){ if(node.getMessageId().equals(mdn.getMessageId())){
i.remove(); i.remove();
dispatched++; dispatched.incrementAndGet();
node.decrementReferenceCount(); node.decrementReferenceCount();
break; break;
} }
@ -88,111 +90,98 @@ public class TopicSubscription extends AbstractSubscription {
} }
} }
public void acknowledge(final ConnectionContext context, final MessageAck ack) throws Throwable { public void acknowledge(final ConnectionContext context,final MessageAck ack) throws Throwable{
// Handle the standard acknowledgment case. // Handle the standard acknowledgment case.
boolean wasFull = isFull(); boolean wasFull=isFull();
if( ack.isStandardAck() || ack.isPoisonAck() ) { if(ack.isStandardAck()||ack.isPoisonAck()){
if ( context.isInTransaction() ) { if(context.isInTransaction()){
delivered += ack.getMessageCount(); delivered.addAndGet(ack.getMessageCount());
context.getTransaction().addSynchronization(new Synchronization(){ context.getTransaction().addSynchronization(new Synchronization(){
public void afterCommit() throws Throwable { public void afterCommit() throws Throwable{
synchronized(TopicSubscription.this) { dispatched.addAndGet(-ack.getMessageCount());
dispatched -= ack.getMessageCount(); delivered.set(Math.max(0,delivered.get()-ack.getMessageCount()));
delivered = Math.max(0, delivered - ack.getMessageCount());
}
} }
}); });
} else { }else{
dispatched -= ack.getMessageCount(); dispatched.addAndGet(-ack.getMessageCount());
delivered = Math.max(0, delivered - ack.getMessageCount()); delivered.set(Math.max(0,delivered.get()-ack.getMessageCount()));
} }
if(wasFull&&!isFull()){
if( wasFull && !isFull() ) {
dispatchMatched(); dispatchMatched();
} }
return; return;
}else if(ack.isDeliveredAck()){
} else if( ack.isDeliveredAck() ) {
// Message was delivered but not acknowledged: update pre-fetch counters. // Message was delivered but not acknowledged: update pre-fetch counters.
delivered += ack.getMessageCount(); delivered.addAndGet(ack.getMessageCount());
if( wasFull && !isFull() ) { if(wasFull&&!isFull()){
dispatchMatched(); dispatchMatched();
} }
return; return;
} }
throw new JMSException("Invalid acknowledgment: "+ack); throw new JMSException("Invalid acknowledgment: "+ack);
} }
public int pending(){ public int pending(){
return matched.size() - dispatched; return matched.size()-dispatched.get();
} }
public int dispatched(){ public int dispatched(){
return dispatched; return dispatched.get();
} }
public int delivered(){ public int delivered(){
return delivered; return delivered.get();
} }
public int getMaximumPendingMessages() { public int getMaximumPendingMessages(){
return maximumPendingMessages; return maximumPendingMessages;
} }
/** /**
* Sets the maximum number of pending messages that can be matched against this consumer * Sets the maximum number of pending messages that can be matched against this consumer before old messages are
* before old messages are discarded. * discarded.
*/ */
public void setMaximumPendingMessages(int maximumPendingMessages) { public void setMaximumPendingMessages(int maximumPendingMessages){
this.maximumPendingMessages = maximumPendingMessages; this.maximumPendingMessages=maximumPendingMessages;
} }
private boolean isFull() { private boolean isFull(){
return dispatched-delivered >= info.getPrefetchSize(); return dispatched.get()-delivered.get()>=info.getPrefetchSize();
} }
private void dispatchMatched() throws IOException { private void dispatchMatched() throws IOException{
for (Iterator iter = matched.iterator(); iter.hasNext() && !isFull();) { synchronized(matchedListMutex){
MessageReference message = (MessageReference) iter.next(); for(Iterator iter=matched.iterator();iter.hasNext()&&!isFull();){
iter.remove(); MessageReference message=(MessageReference) iter.next();
dispatch(message); iter.remove();
dispatch(message);
}
} }
} }
private void dispatch(final MessageReference node) throws IOException { private void dispatch(final MessageReference node) throws IOException{
Message message=(Message) node;
Message message = (Message) node;
// Make sure we can dispatch a message. // Make sure we can dispatch a message.
MessageDispatch md = new MessageDispatch(); MessageDispatch md=new MessageDispatch();
md.setMessage(message); md.setMessage(message);
md.setConsumerId( info.getConsumerId() ); md.setConsumerId(info.getConsumerId());
md.setDestination( node.getRegionDestination().getActiveMQDestination() ); md.setDestination(node.getRegionDestination().getActiveMQDestination());
dispatched.incrementAndGet();
dispatched++; if(info.isDispatchAsync()){
if( info.isDispatchAsync() ) {
md.setConsumer(new Runnable(){ md.setConsumer(new Runnable(){
public void run() { public void run(){
node.decrementReferenceCount(); node.decrementReferenceCount();
} }
}); });
context.getConnection().dispatchAsync(md); context.getConnection().dispatchAsync(md);
} else { }else{
context.getConnection().dispatchSync(md); context.getConnection().dispatchSync(md);
node.decrementReferenceCount(); node.decrementReferenceCount();
} }
} }
public String toString() { public String toString(){
return return "TopicSubscription:"+" consumer="+info.getConsumerId()+", destinations="+destinations.size()
"TopicSubscription:" + +", dispatched="+dispatched+", delivered="+this.delivered+", matched="+this.matched.size();
" consumer="+info.getConsumerId()+
", destinations="+destinations.size()+
", dispatched="+dispatched+
", delivered="+this.delivered+
", matched="+this.matched.size();
} }
} }