mirror of https://github.com/apache/activemq.git
AMQ-7077 AMQ-6421 - tie queue consumer slow flag detection to abort slow consumer policy, fire advisory when set.
This commit is contained in:
parent
79d07a396e
commit
4dee173962
|
@ -32,7 +32,7 @@ import javax.management.openmbean.TabularData;
|
|||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.broker.jmx.AbortSlowConsumerStrategyViewMBean;
|
||||
import org.apache.activemq.broker.jmx.QueueViewMBean;
|
||||
import org.apache.activemq.broker.region.policy.AbortSlowConsumerStrategy;
|
||||
import org.apache.activemq.broker.region.policy.AbortSlowAckConsumerStrategy;
|
||||
import org.apache.activemq.broker.region.policy.PolicyEntry;
|
||||
import org.apache.activemq.broker.region.policy.PolicyMap;
|
||||
import org.apache.activemq.transport.amqp.client.AmqpClient;
|
||||
|
@ -45,14 +45,14 @@ import org.apache.activemq.util.Wait;
|
|||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Test the handling of consumer abort when the AbortSlowConsumerStrategy is used.
|
||||
* Test the handling of consumer abort when the AbortSlowAckConsumerStrategy is used.
|
||||
*/
|
||||
public class AmqpSlowReceiverTest extends AmqpClientTestSupport {
|
||||
|
||||
private final long DEFAULT_CHECK_PERIOD = 1000;
|
||||
private final long DEFAULT_MAX_SLOW_DURATION = 3000;
|
||||
|
||||
private AbortSlowConsumerStrategy strategy;
|
||||
private AbortSlowAckConsumerStrategy strategy;
|
||||
|
||||
@Test(timeout = 60 * 1000)
|
||||
public void testSlowConsumerIsAborted() throws Exception {
|
||||
|
@ -105,7 +105,7 @@ public class AmqpSlowReceiverTest extends AmqpClientTestSupport {
|
|||
AbortSlowConsumerStrategyViewMBean abortPolicy = (AbortSlowConsumerStrategyViewMBean)
|
||||
brokerService.getManagementContext().newProxyInstance(slowConsumerPolicyMBeanName, AbortSlowConsumerStrategyViewMBean.class, true);
|
||||
|
||||
TimeUnit.SECONDS.sleep(3);
|
||||
TimeUnit.SECONDS.sleep(6);
|
||||
|
||||
TabularData slowOnes = abortPolicy.getSlowConsumers();
|
||||
assertEquals("one slow consumers", 1, slowOnes.size());
|
||||
|
@ -148,10 +148,11 @@ public class AmqpSlowReceiverTest extends AmqpClientTestSupport {
|
|||
|
||||
@Override
|
||||
protected void performAdditionalConfiguration(BrokerService brokerService) throws Exception {
|
||||
strategy = new AbortSlowConsumerStrategy();
|
||||
strategy = new AbortSlowAckConsumerStrategy();
|
||||
strategy.setAbortConnection(false);
|
||||
strategy.setCheckPeriod(DEFAULT_CHECK_PERIOD);
|
||||
strategy.setMaxSlowDuration(DEFAULT_MAX_SLOW_DURATION);
|
||||
strategy.setMaxTimeSinceLastAck(DEFAULT_MAX_SLOW_DURATION);
|
||||
|
||||
PolicyEntry policy = new PolicyEntry();
|
||||
policy.setSlowConsumerStrategy(strategy);
|
||||
|
|
|
@ -363,4 +363,7 @@ public abstract class AbstractSubscription implements Subscription {
|
|||
}
|
||||
}
|
||||
|
||||
public CopyOnWriteArrayList<Destination> getDestinations() {
|
||||
return destinations;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -671,7 +671,7 @@ public abstract class PrefetchSubscription extends AbstractSubscription {
|
|||
// decrement after dispatch has taken ownership to avoid usage jitter
|
||||
node.decrementReferenceCount();
|
||||
}
|
||||
} else if (!isSlowConsumer()) {
|
||||
} else if (!pending.isEmpty() && !isSlowConsumer()) {
|
||||
setSlowConsumer(true);
|
||||
slowConsumerTargets = destinations;
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
||||
import org.apache.activemq.broker.Broker;
|
||||
import org.apache.activemq.broker.ConnectionContext;
|
||||
import org.apache.activemq.broker.region.AbstractSubscription;
|
||||
import org.apache.activemq.broker.region.Destination;
|
||||
import org.apache.activemq.broker.region.Subscription;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -128,6 +129,15 @@ public class AbortSlowAckConsumerStrategy extends AbortSlowConsumerStrategy {
|
|||
LOG.debug("sub: {} is now slow", subscriber.getConsumerInfo().getConsumerId());
|
||||
SlowConsumerEntry entry = new SlowConsumerEntry(subscriber.getContext());
|
||||
entry.mark(); // mark consumer on first run
|
||||
if (subscriber instanceof AbstractSubscription) {
|
||||
AbstractSubscription abstractSubscription = (AbstractSubscription) subscriber;
|
||||
if (!abstractSubscription.isSlowConsumer()) {
|
||||
abstractSubscription.setSlowConsumer(true);
|
||||
for (Destination destination: abstractSubscription.getDestinations()) {
|
||||
// destination.slowConsumer(broker.getAdminConnectionContext(), abstractSubscription);
|
||||
}
|
||||
}
|
||||
}
|
||||
slowConsumers.put(subscriber, entry);
|
||||
} else if (getMaxSlowCount() > 0) {
|
||||
slowConsumers.get(subscriber).slow();
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.activemq.broker.policy;
|
|||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import javax.jms.ConnectionFactory;
|
||||
|
@ -42,8 +44,14 @@ public class AbortSlowAckConsumer0Test extends AbortSlowConsumer0Test {
|
|||
protected long maxTimeSinceLastAck = 5 * 1000;
|
||||
protected AbortSlowAckConsumerStrategy strategy;
|
||||
|
||||
@Parameterized.Parameters(name = "isTopic({0})")
|
||||
public static Collection<Object[]> getTestParameters() {
|
||||
return Arrays.asList(new Object[][]{{Boolean.TRUE}, {Boolean.FALSE}});
|
||||
}
|
||||
|
||||
public AbortSlowAckConsumer0Test(Boolean isTopic) {
|
||||
super(isTopic);
|
||||
super();
|
||||
this.topic = isTopic;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,11 +26,23 @@ import org.apache.activemq.broker.region.policy.PolicyMap;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
||||
@RunWith(value = Parameterized.class)
|
||||
public class AbortSlowAckConsumer1Test extends AbortSlowConsumer1Test {
|
||||
|
||||
protected long maxTimeSinceLastAck = 5 * 1000;
|
||||
|
||||
@Parameterized.Parameters(name = "abortConnection({0})-isTopic({1})")
|
||||
public static Collection<Object[]> getTestParameters() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
{Boolean.TRUE, Boolean.TRUE},
|
||||
{Boolean.FALSE, Boolean.TRUE},
|
||||
{Boolean.TRUE, Boolean.FALSE},
|
||||
{Boolean.FALSE, Boolean.FALSE}});
|
||||
}
|
||||
|
||||
public AbortSlowAckConsumer1Test(Boolean abortConnection, Boolean topic) {
|
||||
super(abortConnection, topic);
|
||||
}
|
||||
|
|
|
@ -26,11 +26,20 @@ import org.apache.activemq.broker.region.policy.PolicyMap;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
||||
@RunWith(value = Parameterized.class)
|
||||
public class AbortSlowAckConsumer2Test extends AbortSlowConsumer2Test {
|
||||
|
||||
protected long maxTimeSinceLastAck = 5 * 1000;
|
||||
|
||||
|
||||
@Parameterized.Parameters(name = "isTopic({0})")
|
||||
public static Collection<Object[]> getTestParameters() {
|
||||
return Arrays.asList(new Object[][]{{Boolean.TRUE}, {Boolean.FALSE}});
|
||||
}
|
||||
|
||||
public AbortSlowAckConsumer2Test(Boolean topic) {
|
||||
super(topic);
|
||||
}
|
||||
|
|
|
@ -49,24 +49,19 @@ import org.apache.activemq.command.ActiveMQTopic;
|
|||
import org.apache.activemq.util.MessageIdList;
|
||||
import org.apache.activemq.util.SocketProxy;
|
||||
import org.apache.activemq.util.Wait;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@RunWith(value = Parameterized.class)
|
||||
public class AbortSlowConsumer0Test extends AbortSlowConsumerBase {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AbortSlowConsumer0Test.class);
|
||||
|
||||
@Parameterized.Parameters(name = "isTopic({0})")
|
||||
public static Collection<Object[]> getTestParameters() {
|
||||
return Arrays.asList(new Object[][]{{Boolean.TRUE}, {Boolean.FALSE}});
|
||||
}
|
||||
|
||||
public AbortSlowConsumer0Test(Boolean isTopic) {
|
||||
this.topic = isTopic;
|
||||
public AbortSlowConsumer0Test() {
|
||||
this.topic = true;
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -125,8 +120,11 @@ public class AbortSlowConsumer0Test extends AbortSlowConsumerBase {
|
|||
assertEquals("no slow consumers left", 0, slowOnes.size());
|
||||
|
||||
// verify mbean gone with destination
|
||||
broker.getAdminView().removeTopic(amqDest.getPhysicalName());
|
||||
|
||||
if (topic) {
|
||||
broker.getAdminView().removeTopic(amqDest.getPhysicalName());
|
||||
} else {
|
||||
broker.getAdminView().removeQueue(amqDest.getPhysicalName());
|
||||
}
|
||||
try {
|
||||
abortPolicy.getSlowConsumers();
|
||||
fail("expect not found post destination removal");
|
||||
|
|
|
@ -42,9 +42,7 @@ public class AbortSlowConsumer1Test extends AbortSlowConsumerBase {
|
|||
public static Collection<Object[]> getTestParameters() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
{Boolean.TRUE, Boolean.TRUE},
|
||||
{Boolean.TRUE, Boolean.FALSE},
|
||||
{Boolean.FALSE, Boolean.TRUE},
|
||||
{Boolean.FALSE, Boolean.FALSE}});
|
||||
{Boolean.FALSE, Boolean.TRUE}});
|
||||
}
|
||||
|
||||
public AbortSlowConsumer1Test(Boolean abortConnection, Boolean topic) {
|
||||
|
|
|
@ -31,7 +31,7 @@ public class AbortSlowConsumer2Test extends AbortSlowConsumerBase {
|
|||
|
||||
@Parameterized.Parameters(name = "isTopic({0})")
|
||||
public static Collection<Object[]> getTestParameters() {
|
||||
return Arrays.asList(new Object[][]{{Boolean.TRUE}, {Boolean.FALSE}});
|
||||
return Arrays.asList(new Object[][]{{Boolean.TRUE}});
|
||||
}
|
||||
|
||||
public AbortSlowConsumer2Test(Boolean isTopic) {
|
||||
|
|
|
@ -41,7 +41,6 @@ public class AbortSlowConsumerBase extends JmsMultipleClientsTestSupport impleme
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
exceptions.clear();
|
||||
topic = true;
|
||||
underTest = createSlowConsumerStrategy();
|
||||
super.setUp();
|
||||
createDestination();
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.activemq.ActiveMQConnection;
|
|||
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.ActiveMQMessageConsumer;
|
||||
import org.apache.activemq.broker.TransportConnector;
|
||||
import org.apache.activemq.broker.region.policy.AbortSlowAckConsumerStrategy;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -108,12 +109,13 @@ public class AMQ5844Test {
|
|||
broker.setBrokerName("Main");
|
||||
|
||||
PolicyEntry policy = new PolicyEntry();
|
||||
AbortSlowConsumerStrategy abortSlowConsumerStrategy = new AbortSlowConsumerStrategy();
|
||||
AbortSlowAckConsumerStrategy abortSlowConsumerStrategy = new AbortSlowAckConsumerStrategy();
|
||||
abortSlowConsumerStrategy.setAbortConnection(false);
|
||||
//time in milliseconds between checks for slow subscriptions
|
||||
abortSlowConsumerStrategy.setCheckPeriod(checkPeriod);
|
||||
//time in milliseconds that a sub can remain slow before triggering an abort
|
||||
abortSlowConsumerStrategy.setMaxSlowDuration(maxSlowDuration);
|
||||
abortSlowConsumerStrategy.setMaxTimeSinceLastAck(maxSlowDuration);
|
||||
|
||||
policy.setSlowConsumerStrategy(abortSlowConsumerStrategy);
|
||||
policy.setQueuePrefetch(0);
|
||||
|
|
|
@ -0,0 +1,104 @@
|
|||
/**
|
||||
* 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 org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.advisory.AdvisorySupport;
|
||||
import org.apache.activemq.broker.BrokerFactory;
|
||||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.broker.region.policy.AbortSlowAckConsumerStrategy;
|
||||
import org.apache.activemq.broker.region.policy.PolicyEntry;
|
||||
import org.apache.activemq.broker.region.policy.PolicyMap;
|
||||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.jms.*;
|
||||
import java.net.URI;
|
||||
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
public class AMQ7077Test {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AMQ7077Test.class);
|
||||
|
||||
private BrokerService brokerService;
|
||||
private String connectionUri;
|
||||
|
||||
protected ConnectionFactory createConnectionFactory() throws Exception {
|
||||
ActiveMQConnectionFactory conFactory = new ActiveMQConnectionFactory(connectionUri);
|
||||
conFactory.setWatchTopicAdvisories(false);
|
||||
return conFactory;
|
||||
}
|
||||
|
||||
protected AbortSlowAckConsumerStrategy createSlowConsumerStrategy() {
|
||||
AbortSlowAckConsumerStrategy strategy = new AbortSlowAckConsumerStrategy();
|
||||
strategy.setCheckPeriod(500);
|
||||
strategy.setMaxTimeSinceLastAck(1000);
|
||||
strategy.setMaxSlowCount(2);
|
||||
strategy.setIgnoreIdleConsumers(false);
|
||||
return strategy;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
brokerService = BrokerFactory.createBroker(new URI("broker://()/localhost?persistent=false&useJmx=true"));
|
||||
PolicyEntry policy = new PolicyEntry();
|
||||
|
||||
policy.setSlowConsumerStrategy(createSlowConsumerStrategy());
|
||||
policy.setQueuePrefetch(10);
|
||||
policy.setTopicPrefetch(10);
|
||||
policy.setAdvisoryForSlowConsumers(true);
|
||||
PolicyMap pMap = new PolicyMap();
|
||||
pMap.put(new ActiveMQQueue(">"), policy);
|
||||
brokerService.setUseJmx(false);
|
||||
brokerService.setDestinationPolicy(pMap);
|
||||
brokerService.addConnector("tcp://0.0.0.0:0");
|
||||
brokerService.start();
|
||||
|
||||
connectionUri = brokerService.getTransportConnectorByScheme("tcp").getPublishableConnectString();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAdvisoryOnSlowAckDetection() throws Exception {
|
||||
Connection connection = createConnectionFactory().createConnection();
|
||||
connection.start();
|
||||
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||
|
||||
Destination destination = session.createQueue("DD");
|
||||
|
||||
MessageConsumer consumer = session.createConsumer(destination);
|
||||
// will be idle and can get removed but will be marked slow and now produce an advisory
|
||||
|
||||
MessageConsumer advisoryConsumer = session.createConsumer(AdvisorySupport.getSlowConsumerAdvisoryTopic(destination));
|
||||
Message message = advisoryConsumer.receive(10000);
|
||||
if (message == null) {
|
||||
message = advisoryConsumer.receive(2000);
|
||||
}
|
||||
assertNotNull("Got advisory", message);
|
||||
connection.close();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
brokerService.stop();
|
||||
brokerService.waitUntilStopped();
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue