AMQ-7077 AMQ-6421 - tie queue consumer slow flag detection to abort slow consumer policy, fire advisory when set.

This commit is contained in:
gtully 2018-10-17 11:04:47 +01:00
parent 79d07a396e
commit 4dee173962
13 changed files with 167 additions and 23 deletions

View File

@ -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);

View File

@ -363,4 +363,7 @@ public abstract class AbstractSubscription implements Subscription {
}
}
public CopyOnWriteArrayList<Destination> getDestinations() {
return destinations;
}
}

View File

@ -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;
}

View File

@ -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();

View File

@ -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

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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");

View File

@ -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) {

View File

@ -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) {

View File

@ -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();

View File

@ -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);

View File

@ -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();
}
}