git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@1297252 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Timothy A. Bish 2012-03-05 22:08:51 +00:00
parent 0dfbc5054d
commit 3a5b48a4db
4 changed files with 320 additions and 100 deletions

View File

@ -212,7 +212,7 @@ public class DurableTopicSubscription extends PrefetchSubscription implements Us
}
}
}
prefetchExtension = 0;
prefetchExtension.set(0);
}
@ -272,10 +272,9 @@ public class DurableTopicSubscription extends PrefetchSubscription implements Us
node.decrementReferenceCount();
}
public synchronized String toString() {
return "DurableTopicSubscription-" + getSubscriptionKey() + ", id=" + info.getConsumerId() + ", active=" + isActive() + ", destinations=" + durableDestinations.size() + ", total=" + enqueueCounter + ", pending="
+ getPendingQueueSize() + ", dispatched=" + dispatchCounter + ", inflight=" + dispatched.size() + ", prefetchExtension=" + this.prefetchExtension;
+ getPendingQueueSize() + ", dispatched=" + dispatchCounter + ", inflight=" + dispatched.size() + ", prefetchExtension=" + getPrefetchExtension();
}
public SubscriptionKey getSubscriptionKey() {
@ -301,8 +300,7 @@ public class DurableTopicSubscription extends PrefetchSubscription implements Us
}
}
synchronized(dispatched) {
for (Iterator iter = dispatched.iterator(); iter.hasNext();) {
MessageReference node = (MessageReference) iter.next();
for (MessageReference node : dispatched) {
node.decrementReferenceCount();
}
dispatched.clear();

View File

@ -23,8 +23,11 @@ import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import javax.jms.InvalidSelectorException;
import javax.jms.JMSException;
import org.apache.activemq.broker.Broker;
import org.apache.activemq.broker.ConnectionContext;
import org.apache.activemq.broker.region.cursors.PendingMessageCursor;
@ -47,8 +50,6 @@ import org.slf4j.LoggerFactory;
/**
* A subscription that honors the pre-fetch option of the ConsumerInfo.
*
*
*/
public abstract class PrefetchSubscription extends AbstractSubscription {
@ -57,7 +58,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
protected PendingMessageCursor pending;
protected final List<MessageReference> dispatched = new CopyOnWriteArrayList<MessageReference>();
protected int prefetchExtension;
protected final AtomicInteger prefetchExtension = new AtomicInteger();
protected boolean usePrefetchExtension = true;
protected long enqueueCounter;
protected long dispatchCounter;
@ -84,17 +85,13 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
* Allows a message to be pulled on demand by a client
*/
public Response pullMessage(ConnectionContext context, MessagePull pull) throws Exception {
// The slave should not deliver pull messages. TODO: when the slave
// becomes a master,
// He should send a NULL message to all the consumers to 'wake them up'
// in case
// they were waiting for a message.
// The slave should not deliver pull messages.
// TODO: when the slave becomes a master, He should send a NULL message to all the
// consumers to 'wake them up' in case they were waiting for a message.
if (getPrefetchSize() == 0 && !isSlave()) {
final long dispatchCounterBeforePull;
synchronized(this) {
prefetchExtension++;
dispatchCounterBeforePull = dispatchCounter;
}
prefetchExtension.incrementAndGet();
final long dispatchCounterBeforePull = dispatchCounter;
// Have the destination push us some messages.
for (Destination dest : destinations) {
@ -113,7 +110,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
}
if (pull.getTimeout() > 0) {
scheduler.executeAfterDelay(new Runnable() {
@Override
public void run() {
pullTimeout(dispatchCounterBeforePull);
}
@ -230,10 +227,23 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
if (ack.getLastMessageId().equals(messageId)) {
// contract prefetch if dispatch required a pull
if (getPrefetchSize() == 0) {
prefetchExtension = Math.max(0, prefetchExtension - index);
// Protect extension update against parallel updates.
while (true) {
int currentExtension = prefetchExtension.get();
int newExtension = Math.max(0, currentExtension - index);
if (prefetchExtension.compareAndSet(currentExtension, newExtension)) {
break;
}
}
} else if (usePrefetchExtension && context.isInTransaction()) {
// extend prefetch window only if not a pulling consumer
prefetchExtension = Math.max(prefetchExtension, index);
while (true) {
int currentExtension = prefetchExtension.get();
int newExtension = Math.max(currentExtension, index);
if (prefetchExtension.compareAndSet(currentExtension, newExtension)) {
break;
}
}
}
destination = node.getRegionDestination();
callDispatchMatched = true;
@ -264,7 +274,15 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
} else {
registerRemoveSync(context, node);
}
prefetchExtension = Math.max(0, prefetchExtension - 1);
// Protect extension update against parallel updates.
while (true) {
int currentExtension = prefetchExtension.get();
int newExtension = Math.max(0, currentExtension - 1);
if (prefetchExtension.compareAndSet(currentExtension, newExtension)) {
break;
}
}
acknowledge(context, ack, node);
destination = node.getRegionDestination();
callDispatchMatched = true;
@ -286,7 +304,13 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
}
if (ack.getLastMessageId().equals(node.getMessageId())) {
if (usePrefetchExtension) {
prefetchExtension = Math.max(prefetchExtension, index + 1);
while (true) {
int currentExtension = prefetchExtension.get();
int newExtension = Math.max(currentExtension, index + 1);
if (prefetchExtension.compareAndSet(currentExtension, newExtension)) {
break;
}
}
}
destination = node.getRegionDestination();
callDispatchMatched = true;
@ -351,8 +375,13 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
index++;
acknowledge(context, ack, node);
if (ack.getLastMessageId().equals(messageId)) {
prefetchExtension = Math.max(0, prefetchExtension
- (index + 1));
while (true) {
int currentExtension = prefetchExtension.get();
int newExtension = Math.max(0, currentExtension - (index + 1));
if (prefetchExtension.compareAndSet(currentExtension, newExtension)) {
break;
}
}
destination = node.getRegionDestination();
callDispatchMatched = true;
break;
@ -479,26 +508,26 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
* @return
*/
public boolean isFull() {
return dispatched.size() - prefetchExtension >= info.getPrefetchSize();
return dispatched.size() - prefetchExtension.get() >= info.getPrefetchSize();
}
/**
* @return true when 60% or more room is left for dispatching messages
*/
public boolean isLowWaterMark() {
return (dispatched.size() - prefetchExtension) <= (info.getPrefetchSize() * .4);
return (dispatched.size() - prefetchExtension.get()) <= (info.getPrefetchSize() * .4);
}
/**
* @return true when 10% or less room is left for dispatching messages
*/
public boolean isHighWaterMark() {
return (dispatched.size() - prefetchExtension) >= (info.getPrefetchSize() * .9);
return (dispatched.size() - prefetchExtension.get()) >= (info.getPrefetchSize() * .9);
}
@Override
public int countBeforeFull() {
return info.getPrefetchSize() + prefetchExtension - dispatched.size();
return info.getPrefetchSize() + prefetchExtension.get() - dispatched.size();
}
public int getPendingQueueSize() {
@ -646,7 +675,13 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
dispatchCounter++;
dispatched.add(node);
} else {
prefetchExtension = Math.max(0, prefetchExtension - 1);
while (true) {
int currentExtension = prefetchExtension.get();
int newExtension = Math.max(0, currentExtension - 1);
if (prefetchExtension.compareAndSet(currentExtension, newExtension)) {
break;
}
}
}
if (info.isDispatchAsync()) {
md.setTransmitCallback(new Runnable() {
@ -711,20 +746,19 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
* @return MessageDispatch
*/
protected MessageDispatch createMessageDispatch(MessageReference node, Message message) {
MessageDispatch md = new MessageDispatch();
md.setConsumerId(info.getConsumerId());
if (node == QueueMessageReference.NULL_MESSAGE) {
MessageDispatch md = new MessageDispatch();
md.setMessage(null);
md.setConsumerId(info.getConsumerId());
md.setDestination(null);
return md;
} else {
MessageDispatch md = new MessageDispatch();
md.setConsumerId(info.getConsumerId());
md.setDestination(node.getRegionDestination().getActiveMQDestination());
md.setMessage(message);
md.setRedeliveryCounter(node.getRedeliveryCounter());
return md;
}
return md;
}
/**
@ -770,4 +804,8 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
public void setUsePrefetchExtension(boolean usePrefetchExtension) {
this.usePrefetchExtension = usePrefetchExtension;
}
protected int getPrefetchExtension() {
return this.prefetchExtension.get();
}
}

View File

@ -60,7 +60,17 @@ import org.apache.activemq.broker.region.group.MessageGroupMapFactory;
import org.apache.activemq.broker.region.policy.DispatchPolicy;
import org.apache.activemq.broker.region.policy.RoundRobinDispatchPolicy;
import org.apache.activemq.broker.util.InsertionCountList;
import org.apache.activemq.command.*;
import org.apache.activemq.command.ActiveMQDestination;
import org.apache.activemq.command.ActiveMQMessage;
import org.apache.activemq.command.ConsumerId;
import org.apache.activemq.command.ExceptionResponse;
import org.apache.activemq.command.Message;
import org.apache.activemq.command.MessageAck;
import org.apache.activemq.command.MessageDispatchNotification;
import org.apache.activemq.command.MessageId;
import org.apache.activemq.command.ProducerAck;
import org.apache.activemq.command.ProducerInfo;
import org.apache.activemq.command.Response;
import org.apache.activemq.filter.BooleanExpression;
import org.apache.activemq.filter.MessageEvaluationContext;
import org.apache.activemq.filter.NonCachedMessageEvaluationContext;

View File

@ -0,0 +1,174 @@
/**
* 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.activemq.bugs;
import static org.junit.Assert.assertEquals;
import java.util.Random;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import javax.jms.Connection;
import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Queue;
import javax.jms.Session;
import org.apache.activemq.ActiveMQConnectionFactory;
import org.apache.activemq.ActiveMQSession;
import org.apache.activemq.broker.BrokerService;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class AMQ3732Test {
private static Logger LOG = LoggerFactory.getLogger(AMQ3529Test.class);
private ActiveMQConnectionFactory connectionFactory;
private Connection connection;
private Session session;
private BrokerService broker;
private String connectionUri;
private final Random pause = new Random();
private final long NUM_MESSAGES = 25000;
private final AtomicLong totalConsumed = new AtomicLong();
@Before
public void startBroker() throws Exception {
broker = new BrokerService();
broker.setDeleteAllMessagesOnStartup(true);
broker.setPersistent(false);
broker.setUseJmx(false);
broker.addConnector("tcp://0.0.0.0:0");
broker.start();
broker.waitUntilStarted();
connectionUri = broker.getTransportConnectors().get(0).getPublishableConnectString();
connectionFactory = new ActiveMQConnectionFactory(connectionUri);
connectionFactory.getPrefetchPolicy().setAll(0);
}
@After
public void stopBroker() throws Exception {
connection.close();
broker.stop();
broker.waitUntilStopped();
}
@Test(timeout = 1200000)
public void testInterruptionAffects() throws Exception {
connection = connectionFactory.createConnection();
connection.start();
session = connection.createSession(false, ActiveMQSession.INDIVIDUAL_ACKNOWLEDGE);
Queue queue = session.createQueue("AMQ3732Test");
final LinkedBlockingQueue<Message> workQueue = new LinkedBlockingQueue<Message>();
final MessageConsumer consumer1 = session.createConsumer(queue);
final MessageConsumer consumer2 = session.createConsumer(queue);
final MessageProducer producer = session.createProducer(queue);
Thread consumer1Thread = new Thread(new Runnable() {
@Override
public void run() {
try {
while (totalConsumed.get() < NUM_MESSAGES) {
Message message = consumer1.receiveNoWait();
if (message != null) {
workQueue.add(message);
}
}
} catch(Exception e) {
LOG.error("Caught an unexpected error: ", e);
}
}
});
consumer1Thread.start();
Thread consumer2Thread = new Thread(new Runnable() {
@Override
public void run() {
try {
while (totalConsumed.get() < NUM_MESSAGES) {
Message message = consumer2.receive(50);
if (message != null) {
workQueue.add(message);
}
}
} catch(Exception e) {
LOG.error("Caught an unexpected error: ", e);
}
}
});
consumer2Thread.start();
Thread producerThread = new Thread(new Runnable() {
@Override
public void run() {
try {
for (int i = 0; i < NUM_MESSAGES; ++i) {
producer.send(session.createTextMessage("TEST"));
TimeUnit.MILLISECONDS.sleep(pause.nextInt(10));
}
} catch(Exception e) {
LOG.error("Caught an unexpected error: ", e);
}
}
});
producerThread.start();
Thread ackingThread = new Thread(new Runnable() {
@Override
public void run() {
try {
while (totalConsumed.get() < NUM_MESSAGES) {
Message message = workQueue.take();
message.acknowledge();
totalConsumed.incrementAndGet();
if ((totalConsumed.get() % 100) == 0) {
LOG.info("Consumed " + totalConsumed.get() + " messages so far.");
}
}
} catch(Exception e) {
LOG.error("Caught an unexpected error: ", e);
}
}
});
ackingThread.start();
producerThread.join();
consumer1Thread.join();
consumer2Thread.join();
ackingThread.join();
assertEquals(NUM_MESSAGES, totalConsumed.get());
}
}