https://issues.apache.org/jira/browse/AMQ-5626 - fix and test. on restart, next index needs to ingore priority to find the last used entry

This commit is contained in:
gtully 2015-03-02 14:44:27 +00:00
parent 260e28ecad
commit ecebd2413b
2 changed files with 226 additions and 19 deletions

View File

@ -46,7 +46,6 @@ import java.util.Set;
import java.util.SortedSet;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -2837,25 +2836,22 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
void configureLast(Transaction tx) throws IOException {
// Figure out the next key using the last entry in the destination.
if (highPriorityIndex != null) {
Entry<Long, MessageKeys> lastEntry = highPriorityIndex.getLast(tx);
TreeSet<Long> orderedSet = new TreeSet<Long>();
addLast(orderedSet, highPriorityIndex, tx);
addLast(orderedSet, defaultPriorityIndex, tx);
addLast(orderedSet, lowPriorityIndex, tx);
if (!orderedSet.isEmpty()) {
nextMessageId = orderedSet.last() + 1;
}
}
private void addLast(TreeSet<Long> orderedSet, BTreeIndex<Long, MessageKeys> index, Transaction tx) throws IOException {
if (index != null) {
Entry<Long, MessageKeys> lastEntry = index.getLast(tx);
if (lastEntry != null) {
nextMessageId = lastEntry.getKey() + 1;
} else {
lastEntry = defaultPriorityIndex.getLast(tx);
if (lastEntry != null) {
nextMessageId = lastEntry.getKey() + 1;
} else {
lastEntry = lowPriorityIndex.getLast(tx);
if (lastEntry != null) {
nextMessageId = lastEntry.getKey() + 1;
}
}
}
} else {
Entry<Long, MessageKeys> lastEntry = defaultPriorityIndex.getLast(tx);
if (lastEntry != null) {
nextMessageId = lastEntry.getKey() + 1;
orderedSet.add(lastEntry.getKey());
}
}
}

View File

@ -0,0 +1,211 @@
/**
* 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.store.kahadb;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import javax.jms.ConnectionFactory;
import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.Session;
import javax.management.ObjectName;
import org.apache.activemq.ActiveMQConnection;
import org.apache.activemq.ActiveMQConnectionFactory;
import org.apache.activemq.broker.BrokerService;
import org.apache.activemq.broker.TransportConnector;
import org.apache.activemq.broker.jmx.DestinationView;
import org.apache.activemq.broker.jmx.QueueView;
import org.apache.activemq.broker.region.policy.PolicyEntry;
import org.apache.activemq.broker.region.policy.PolicyMap;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.fail;
public class AMQ5626Test {
private static final Logger LOG = LoggerFactory.getLogger(AMQ5626Test.class);
private static final String QUEUE_NAME = "TesQ";
private BrokerService brokerService;
private URI brokerUri;
@Before
public void setup() throws Exception {
createBroker(true);
}
private void createBroker(boolean deleteMessagesOnStart) throws Exception {
brokerService = new BrokerService();
PolicyMap policyMap = new PolicyMap();
List<PolicyEntry> entries = new ArrayList<PolicyEntry>();
PolicyEntry pe = new PolicyEntry();
pe.setPrioritizedMessages(true);
pe.setExpireMessagesPeriod(0);
pe.setQueue(QUEUE_NAME);
entries.add(pe);
policyMap.setPolicyEntries(entries);
brokerService.setDestinationPolicy(policyMap);
TransportConnector transportConnector = new TransportConnector();
transportConnector.setName("openwire");
transportConnector.setUri(new URI("tcp://0.0.0.0:0"));
brokerService.addConnector(transportConnector);
brokerService.setDeleteAllMessagesOnStartup(deleteMessagesOnStart);
brokerService.getManagementContext().setCreateConnector(false);
brokerService.start();
brokerService.waitUntilStarted();
brokerUri = transportConnector.getPublishableConnectURI();
}
@Test
public void testPriorityMessages() throws Exception {
ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(brokerUri);
ActiveMQConnection connection = (ActiveMQConnection) connectionFactory.createConnection();
connection.start();
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
MessageProducer producer = session.createProducer(session.createQueue(QUEUE_NAME));
Message message = session.createMessage();
// 0,1
producer.setPriority(9);
producer.send(message);
producer.send(message);
// 2,3
producer.setPriority(4);
producer.send(message);
producer.send(message);
connection.close();
stopRestartBroker();
connectionFactory = new ActiveMQConnectionFactory(brokerUri);
connection = (ActiveMQConnection) connectionFactory.createConnection();
connection.start();
session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
producer = session.createProducer(session.createQueue(QUEUE_NAME));
// 4
producer.setPriority(4);
producer.send(message);
displayQueueViews(brokerService);
// consume 5
MessageConsumer jmsConsumer = session.createConsumer(session.createQueue(QUEUE_NAME));
for (int i = 0; i < 5; i++) {
message = jmsConsumer.receive(4000);
assertNotNull("Got message i=" + i, message);
LOG.info("received: " + message.getJMSMessageID() + ", priority:" + message.getJMSPriority());
}
connection.close();
}
private void displayQueueViews(BrokerService broker) throws Exception {
Map<ObjectName, DestinationView> queueViews = broker.getAdminView().getBroker().getQueueViews();
for (ObjectName key : queueViews.keySet()) {
DestinationView destinationView = queueViews.get(key);
if (destinationView instanceof QueueView) {
QueueView queueView = (QueueView) destinationView;
LOG.info("ObjectName " + key);
LOG.info("QueueView name : " + queueView.getName());
LOG.info("QueueView cursorSize : " + queueView.cursorSize());
LOG.info("QueueView queueSize : " + queueView.getQueueSize());
LOG.info("QueueView enqueue count : " + queueView.getEnqueueCount());
LOG.info("QueueView dequeue count : " + queueView.getDequeueCount());
LOG.info("QueueView inflight count : " + queueView.getInFlightCount());
}
}
}
private QueueView getQueueView(BrokerService broker, String queueName) throws Exception {
Map<ObjectName, DestinationView> queueViews = broker.getAdminView().getBroker().getQueueViews();
for (ObjectName key : queueViews.keySet()) {
DestinationView destinationView = queueViews.get(key);
if (destinationView instanceof QueueView) {
QueueView queueView = (QueueView) destinationView;
if (queueView.getName().equals(queueName)) {
return queueView;
}
}
}
return null;
}
private synchronized void stopRestartBroker() {
try {
LOG.info(">>>SHUTTING BROKER DOWN");
brokerService.stop();
brokerService.waitUntilStopped();
//restart it
createBroker(false);
brokerService.start();
brokerService.waitUntilStarted();
LOG.info(">>>BROKER RESTARTED..");
} catch (Exception e) {
LOG.error("FAILED TO STOP/START BROKER EXCEPTION", e);
fail("FAILED TO STOP/START BROKER" + e);
}
}
@After
public void teardown() throws Exception {
try {
brokerService.stop();
} catch (Exception ex) {
LOG.error("FAILED TO STOP/START BROKER EXCEPTION", ex);
}
}
}