mirror of https://github.com/apache/activemq.git
git-svn-id: https://svn.apache.org/repos/asf/activemq/trunk@1297252 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0dfbc5054d
commit
3a5b48a4db
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue