mirror of https://github.com/apache/activemq.git
rework cursor store sync w.r.t to index order. resolve issues with skipped dispatch and duplicate dispatch. https://issues.apache.org/jira/browse/AMQ-4485 https://issues.apache.org/jira/browse/AMQ-5266
This commit is contained in:
parent
b2afb8c969
commit
54e2e3bef2
|
@ -30,13 +30,11 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.DelayQueue;
|
||||
import java.util.concurrent.Delayed;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -47,7 +45,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
import javax.jms.InvalidSelectorException;
|
||||
import javax.jms.JMSException;
|
||||
import javax.jms.ResourceAllocationException;
|
||||
import javax.transaction.xa.XAException;
|
||||
|
||||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.broker.ConnectionContext;
|
||||
|
@ -75,12 +72,12 @@ 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.command.TransactionId;
|
||||
import org.apache.activemq.filter.BooleanExpression;
|
||||
import org.apache.activemq.filter.MessageEvaluationContext;
|
||||
import org.apache.activemq.filter.NonCachedMessageEvaluationContext;
|
||||
import org.apache.activemq.selector.SelectorParser;
|
||||
import org.apache.activemq.state.ProducerState;
|
||||
import org.apache.activemq.store.IndexListener;
|
||||
import org.apache.activemq.store.ListenableFuture;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.apache.activemq.store.MessageStore;
|
||||
|
@ -88,7 +85,6 @@ import org.apache.activemq.thread.Task;
|
|||
import org.apache.activemq.thread.TaskRunner;
|
||||
import org.apache.activemq.thread.TaskRunnerFactory;
|
||||
import org.apache.activemq.transaction.Synchronization;
|
||||
import org.apache.activemq.transaction.Transaction;
|
||||
import org.apache.activemq.usage.Usage;
|
||||
import org.apache.activemq.usage.UsageListener;
|
||||
import org.apache.activemq.util.BrokerSupport;
|
||||
|
@ -101,7 +97,7 @@ import org.slf4j.MDC;
|
|||
* The Queue is a List of MessageEntry objects that are dispatched to matching
|
||||
* subscriptions.
|
||||
*/
|
||||
public class Queue extends BaseDestination implements Task, UsageListener {
|
||||
public class Queue extends BaseDestination implements Task, UsageListener, IndexListener {
|
||||
protected static final Logger LOG = LoggerFactory.getLogger(Queue.class);
|
||||
protected final TaskRunnerFactory taskFactory;
|
||||
protected TaskRunner taskRunner;
|
||||
|
@ -241,6 +237,9 @@ public class Queue extends BaseDestination implements Task, UsageListener {
|
|||
super(brokerService, store, destination, parentStats);
|
||||
this.taskFactory = taskFactory;
|
||||
this.dispatchSelector = new QueueDispatchSelector(destination);
|
||||
if (store != null) {
|
||||
store.registerIndexListener(this);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -746,158 +745,81 @@ public class Queue extends BaseDestination implements Task, UsageListener {
|
|||
}
|
||||
}
|
||||
|
||||
final ConcurrentHashMap<Transaction, SendSync> sendSyncs = new ConcurrentHashMap<Transaction, SendSync>();
|
||||
private final LinkedList<Transaction> orderIndexUpdates = new LinkedList<Transaction>();
|
||||
|
||||
// roll up all message sends
|
||||
class SendSync extends Synchronization {
|
||||
|
||||
class MessageContext {
|
||||
public Message message;
|
||||
public ConnectionContext context;
|
||||
|
||||
public MessageContext(ConnectionContext context, Message message) {
|
||||
this.context = context;
|
||||
this.message = message;
|
||||
}
|
||||
}
|
||||
|
||||
final Transaction transaction;
|
||||
List<MessageContext> additions = new ArrayList<MessageContext>();
|
||||
|
||||
public SendSync(Transaction transaction) {
|
||||
this.transaction = transaction;
|
||||
}
|
||||
|
||||
public void add(ConnectionContext context, Message message) {
|
||||
additions.add(new MessageContext(context, message));
|
||||
}
|
||||
private final LinkedList<MessageContext> indexOrderedCursorUpdates = new LinkedList<>();
|
||||
|
||||
@Override
|
||||
public void beforeCommit() throws Exception {
|
||||
synchronized (orderIndexUpdates) {
|
||||
orderIndexUpdates.addLast(transaction);
|
||||
public void onAdd(MessageContext messageContext) {
|
||||
synchronized (indexOrderedCursorUpdates) {
|
||||
indexOrderedCursorUpdates.addLast(messageContext);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterCommit() throws Exception {
|
||||
ArrayList<SendSync> syncs = new ArrayList<SendSync>(200);
|
||||
private void doPendingCursorAdditions() throws Exception {
|
||||
LinkedList<MessageContext> orderedUpdates = new LinkedList<>();
|
||||
sendLock.lockInterruptibly();
|
||||
try {
|
||||
synchronized (orderIndexUpdates) {
|
||||
Transaction next = orderIndexUpdates.peek();
|
||||
while( next!=null && next.isCommitted() ) {
|
||||
syncs.add(sendSyncs.remove(orderIndexUpdates.removeFirst()));
|
||||
next = orderIndexUpdates.peek();
|
||||
synchronized (indexOrderedCursorUpdates) {
|
||||
MessageContext candidate = indexOrderedCursorUpdates.peek();
|
||||
while (candidate != null && candidate.message.getMessageId().getFutureOrSequenceLong() != null) {
|
||||
candidate = indexOrderedCursorUpdates.removeFirst();
|
||||
// check for duplicate adds suppressed by the store
|
||||
if (candidate.message.getMessageId().getFutureOrSequenceLong() instanceof Long && ((Long)candidate.message.getMessageId().getFutureOrSequenceLong()).compareTo(-1l) == 0) {
|
||||
LOG.warn("{} messageStore indicated duplicate add attempt for {}, suppressing duplicate dispatch", this, candidate.message.getMessageId());
|
||||
} else {
|
||||
orderedUpdates.add(candidate);
|
||||
}
|
||||
candidate = indexOrderedCursorUpdates.peek();
|
||||
}
|
||||
}
|
||||
for (SendSync sync : syncs) {
|
||||
sync.processSend();
|
||||
for (MessageContext messageContext : orderedUpdates) {
|
||||
if (!cursorAdd(messageContext.message)) {
|
||||
// cursor suppressed a duplicate
|
||||
messageContext.duplicate = true;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
sendLock.unlock();
|
||||
}
|
||||
for (SendSync sync : syncs) {
|
||||
sync.processSent();
|
||||
}
|
||||
}
|
||||
|
||||
// called with sendLock
|
||||
private void processSend() throws Exception {
|
||||
|
||||
for (Iterator<MessageContext> iterator = additions.iterator(); iterator.hasNext(); ) {
|
||||
MessageContext messageContext = iterator.next();
|
||||
// It could take while before we receive the commit
|
||||
// op, by that time the message could have expired..
|
||||
if (broker.isExpired(messageContext.message)) {
|
||||
broker.messageExpired(messageContext.context, messageContext.message, null);
|
||||
destinationStatistics.getExpired().increment();
|
||||
iterator.remove();
|
||||
continue;
|
||||
}
|
||||
sendMessage(messageContext.message);
|
||||
messageContext.message.decrementReferenceCount();
|
||||
}
|
||||
}
|
||||
|
||||
private void processSent() throws Exception {
|
||||
for (MessageContext messageContext : additions) {
|
||||
for (MessageContext messageContext : orderedUpdates) {
|
||||
if (!messageContext.duplicate) {
|
||||
messageSent(messageContext.context, messageContext.message);
|
||||
}
|
||||
if (messageContext.onCompletion != null) {
|
||||
messageContext.onCompletion.run();
|
||||
}
|
||||
}
|
||||
orderedUpdates.clear();
|
||||
}
|
||||
|
||||
final class CursorAddSync extends Synchronization {
|
||||
|
||||
private final MessageContext messageContext;
|
||||
|
||||
CursorAddSync(MessageContext messageContext) {
|
||||
this.messageContext = messageContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterCommit() throws Exception {
|
||||
if (store != null && messageContext.message.isPersistent()) {
|
||||
doPendingCursorAdditions();
|
||||
} else {
|
||||
cursorAdd(messageContext.message);
|
||||
messageSent(messageContext.context, messageContext.message);
|
||||
}
|
||||
messageContext.message.decrementReferenceCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void afterRollback() throws Exception {
|
||||
try {
|
||||
for (MessageContext messageContext : additions) {
|
||||
messageContext.message.decrementReferenceCount();
|
||||
}
|
||||
} finally {
|
||||
sendSyncs.remove(transaction);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class OrderedNonTransactionWorkTx extends Transaction {
|
||||
|
||||
@Override
|
||||
public void commit(boolean onePhase) throws XAException, IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void rollback() throws XAException, IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int prepare() throws XAException, IOException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TransactionId getTransactionId() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Logger getLog() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCommitted() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addSynchronization(Synchronization s) {
|
||||
try {
|
||||
s.beforeCommit();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to add not transactional message to orderedWork", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// called while holding the sendLock
|
||||
private void registerSendSync(Message message, ConnectionContext context) {
|
||||
final Transaction transaction =
|
||||
message.isInTransaction() ? context.getTransaction()
|
||||
: new OrderedNonTransactionWorkTx();
|
||||
Queue.SendSync currentSync = sendSyncs.get(transaction);
|
||||
if (currentSync == null) {
|
||||
currentSync = new Queue.SendSync(transaction);
|
||||
transaction.addSynchronization(currentSync);
|
||||
sendSyncs.put(transaction, currentSync);
|
||||
}
|
||||
currentSync.add(context, message);
|
||||
}
|
||||
|
||||
void doMessageSend(final ProducerBrokerExchange producerExchange, final Message message) throws IOException,
|
||||
Exception {
|
||||
final ConnectionContext context = producerExchange.getConnectionContext();
|
||||
ListenableFuture<Object> result = null;
|
||||
boolean needsOrderingWithTransactions = context.isInTransaction();
|
||||
|
||||
producerExchange.incrementSend();
|
||||
checkUsage(context, producerExchange, message);
|
||||
|
@ -922,26 +844,11 @@ public class Queue extends BaseDestination implements Task, UsageListener {
|
|||
throw e;
|
||||
}
|
||||
}
|
||||
// did a transaction commit beat us to the index?
|
||||
synchronized (orderIndexUpdates) {
|
||||
needsOrderingWithTransactions |= !orderIndexUpdates.isEmpty();
|
||||
}
|
||||
if (needsOrderingWithTransactions ) {
|
||||
// If this is a transacted message.. increase the usage now so that
|
||||
// a big TX does not blow up
|
||||
// our memory. This increment is decremented once the tx finishes..
|
||||
message.incrementReferenceCount();
|
||||
|
||||
registerSendSync(message, context);
|
||||
} else {
|
||||
// Add to the pending list, this takes care of incrementing the
|
||||
// usage manager.
|
||||
sendMessage(message);
|
||||
}
|
||||
orderedCursorAdd(message, context);
|
||||
} finally {
|
||||
sendLock.unlock();
|
||||
}
|
||||
if (!needsOrderingWithTransactions) {
|
||||
if (store == null || (!context.isInTransaction() && !message.isPersistent())) {
|
||||
messageSent(context, message);
|
||||
}
|
||||
if (result != null && message.isResponseRequired() && !result.isCancelled()) {
|
||||
|
@ -954,6 +861,17 @@ public class Queue extends BaseDestination implements Task, UsageListener {
|
|||
}
|
||||
}
|
||||
|
||||
private void orderedCursorAdd(Message message, ConnectionContext context) throws Exception {
|
||||
if (context.isInTransaction()) {
|
||||
context.getTransaction().addSynchronization(new CursorAddSync(new MessageContext(context, message, null)));
|
||||
} else if (store != null && message.isPersistent()) {
|
||||
doPendingCursorAdditions();
|
||||
} else {
|
||||
// no ordering issue with non persistent messages
|
||||
cursorAdd(message);
|
||||
}
|
||||
}
|
||||
|
||||
private void checkUsage(ConnectionContext context,ProducerBrokerExchange producerBrokerExchange, Message message) throws ResourceAllocationException, IOException, InterruptedException {
|
||||
if (message.isPersistent()) {
|
||||
if (store != null && systemUsage.getStoreUsage().isFull(getStoreUsageHighWaterMark())) {
|
||||
|
@ -1860,10 +1778,10 @@ public class Queue extends BaseDestination implements Task, UsageListener {
|
|||
}
|
||||
}
|
||||
|
||||
final void sendMessage(final Message msg) throws Exception {
|
||||
final boolean cursorAdd(final Message msg) throws Exception {
|
||||
messagesLock.writeLock().lock();
|
||||
try {
|
||||
messages.addMessageLast(msg);
|
||||
return messages.addMessageLast(msg);
|
||||
} finally {
|
||||
messagesLock.writeLock().unlock();
|
||||
}
|
||||
|
|
|
@ -748,6 +748,7 @@ public class RegionBroker extends EmptyBroker {
|
|||
if (deadLetterStrategy.isSendToDeadLetterQueue(message)) {
|
||||
// message may be inflight to other subscriptions so do not modify
|
||||
message = message.copy();
|
||||
message.getMessageId().setFutureOrSequenceLong(null);
|
||||
stampAsExpired(message);
|
||||
message.setExpiration(0);
|
||||
if (!message.isPersistent()) {
|
||||
|
|
|
@ -82,12 +82,12 @@ public abstract class AbstractPendingMessageCursor implements PendingMessageCurs
|
|||
public void addMessageFirst(MessageReference node) throws Exception {
|
||||
}
|
||||
|
||||
public void addMessageLast(MessageReference node) throws Exception {
|
||||
public boolean addMessageLast(MessageReference node) throws Exception {
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean tryAddMessageLast(MessageReference node, long maxWaitTime) throws Exception {
|
||||
addMessageLast(node);
|
||||
return true;
|
||||
return addMessageLast(node);
|
||||
}
|
||||
|
||||
public void addRecoveredMessage(MessageReference node) throws Exception {
|
||||
|
|
|
@ -18,12 +18,13 @@ package org.apache.activemq.broker.region.cursors;
|
|||
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.Future;
|
||||
import org.apache.activemq.broker.region.Destination;
|
||||
import org.apache.activemq.broker.region.MessageReference;
|
||||
import org.apache.activemq.broker.region.Subscription;
|
||||
import org.apache.activemq.command.Message;
|
||||
import org.apache.activemq.command.MessageId;
|
||||
import org.apache.activemq.command.TransactionId;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -40,8 +41,8 @@ public abstract class AbstractStoreCursor extends AbstractPendingMessageCursor i
|
|||
protected boolean batchResetNeeded = false;
|
||||
private boolean storeHasMessages = false;
|
||||
protected int size;
|
||||
private MessageId lastCachedId;
|
||||
private TransactionId lastTx;
|
||||
private LinkedList<MessageId> pendingCachedIds = new LinkedList<>();
|
||||
MessageId lastCachedId = null;
|
||||
protected boolean hadSpace = false;
|
||||
|
||||
protected AbstractStoreCursor(Destination destination) {
|
||||
|
@ -100,13 +101,12 @@ public abstract class AbstractStoreCursor extends AbstractPendingMessageCursor i
|
|||
recovered = true;
|
||||
storeHasMessages = true;
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(this + " - cursor got duplicate: " + message.getMessageId() + "," + message.getPriority() + ", cached=" + cached, new Throwable("duplicate message detected"));
|
||||
} else {
|
||||
LOG.warn("{} - cursor got duplicate {}", regionDestination.getActiveMQDestination(), message.getMessageId());
|
||||
}
|
||||
if (!cached || message.getMessageId().getEntryLocator() != null) {
|
||||
// came from the store or was added to the jdbc store
|
||||
LOG.warn("{} - cursor got duplicate {} seq: {}", this, message.getMessageId(), message.getMessageId().getFutureOrSequenceLong());
|
||||
|
||||
// a duplicate from the store - needs to be removed/acked - otherwise it will get redispatched on restart
|
||||
// jdbc store will store duplicates and will set entry locator to sequence long.
|
||||
// REVISIT - this seems too hacky - see use case AMQ4952Test
|
||||
if (!cached || message.getMessageId().getEntryLocator() instanceof Long) {
|
||||
duplicate(message);
|
||||
}
|
||||
}
|
||||
|
@ -189,21 +189,24 @@ public abstract class AbstractStoreCursor extends AbstractPendingMessageCursor i
|
|||
return result;
|
||||
}
|
||||
|
||||
|
||||
public final synchronized void addMessageLast(MessageReference node) throws Exception {
|
||||
public final synchronized boolean addMessageLast(MessageReference node) throws Exception {
|
||||
boolean disableCache = false;
|
||||
if (hasSpace()) {
|
||||
if (!isCacheEnabled() && size==0 && isStarted() && useCache) {
|
||||
LOG.trace("{} - enabling cache for empty store {}", this, node.getMessageId());
|
||||
LOG.trace("{} - enabling cache for empty store {} {}", this, node.getMessageId(), node.getMessageId().getFutureOrSequenceLong());
|
||||
setCacheEnabled(true);
|
||||
}
|
||||
if (isCacheEnabled()) {
|
||||
if (recoverMessage(node.getMessage(),true)) {
|
||||
lastCachedId = node.getMessageId();
|
||||
lastTx = node.getMessage().getTransactionId();
|
||||
if (node.getMessageId().getFutureOrSequenceLong() instanceof Future) {
|
||||
pruneLastCached();
|
||||
pendingCachedIds.add(node.getMessageId());
|
||||
} else {
|
||||
setLastCachedId(node.getMessageId());
|
||||
}
|
||||
} else {
|
||||
dealWithDuplicates();
|
||||
return;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
|
@ -213,16 +216,62 @@ public abstract class AbstractStoreCursor extends AbstractPendingMessageCursor i
|
|||
if (disableCache && isCacheEnabled()) {
|
||||
setCacheEnabled(false);
|
||||
// sync with store on disabling the cache
|
||||
if (!pendingCachedIds.isEmpty() || lastCachedId != null) {
|
||||
LOG.trace("{} - disabling cache. current Id: {} seq: {}, batchList size: {}",
|
||||
new Object[]{this, node.getMessageId(), node.getMessageId().getFutureOrSequenceLong(), batchList.size()});
|
||||
collapseLastCachedIds();
|
||||
if (lastCachedId != null) {
|
||||
LOG.debug("{} - disabling cache, lastCachedId: {} last-tx: {} current node Id: {} node-tx: {} batchList size: {}",
|
||||
new Object[]{ this, lastCachedId, lastTx, node.getMessageId(), node.getMessage().getTransactionId(), batchList.size() });
|
||||
setBatch(lastCachedId);
|
||||
lastCachedId = null;
|
||||
lastTx = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
this.storeHasMessages = true;
|
||||
size++;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
private void pruneLastCached() {
|
||||
for (Iterator<MessageId> it = pendingCachedIds.iterator(); it.hasNext(); ) {
|
||||
MessageId candidate = it.next();
|
||||
final Object futureOrLong = candidate.getFutureOrSequenceLong();
|
||||
if (futureOrLong instanceof Future) {
|
||||
Future future = (Future) futureOrLong;
|
||||
if (future.isCancelled()) {
|
||||
it.remove();
|
||||
}
|
||||
} else {
|
||||
// store complete - track via lastCachedId
|
||||
setLastCachedId(candidate);
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void collapseLastCachedIds() throws Exception {
|
||||
for (MessageId candidate : pendingCachedIds) {
|
||||
final Object futureOrLong = candidate.getFutureOrSequenceLong();
|
||||
if (futureOrLong instanceof Future) {
|
||||
Future future = (Future) futureOrLong;
|
||||
try {
|
||||
future.get();
|
||||
// future should be replaced with sequence by this time
|
||||
} catch (CancellationException ignored) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
setLastCachedId(candidate);
|
||||
}
|
||||
pendingCachedIds.clear();
|
||||
}
|
||||
|
||||
private void setLastCachedId(MessageId candidate) {
|
||||
if (lastCachedId == null) {
|
||||
lastCachedId = candidate;
|
||||
} else if (Long.compare(((Long) candidate.getFutureOrSequenceLong()), ((Long) lastCachedId.getFutureOrSequenceLong())) > 0) {
|
||||
lastCachedId = candidate;
|
||||
}
|
||||
}
|
||||
|
||||
protected void setBatch(MessageId messageId) throws Exception {
|
||||
|
@ -260,8 +309,7 @@ public abstract class AbstractStoreCursor extends AbstractPendingMessageCursor i
|
|||
|
||||
|
||||
public synchronized void gc() {
|
||||
for (Iterator<MessageReference>i = batchList.iterator();i.hasNext();) {
|
||||
MessageReference msg = i.next();
|
||||
for (MessageReference msg : batchList) {
|
||||
rollback(msg.getMessageId());
|
||||
msg.decrementReferenceCount();
|
||||
}
|
||||
|
@ -272,7 +320,6 @@ public abstract class AbstractStoreCursor extends AbstractPendingMessageCursor i
|
|||
}
|
||||
|
||||
protected final synchronized void fillBatch() {
|
||||
//LOG.trace("{} - fillBatch", this);
|
||||
if (batchResetNeeded) {
|
||||
resetSize();
|
||||
setMaxBatchSize(Math.min(regionDestination.getMaxPageSize(), size));
|
||||
|
@ -313,7 +360,7 @@ public abstract class AbstractStoreCursor extends AbstractPendingMessageCursor i
|
|||
public String toString() {
|
||||
return super.toString() + ":" + regionDestination.getActiveMQDestination().getPhysicalName() + ",batchResetNeeded=" + batchResetNeeded
|
||||
+ ",storeHasMessages=" + this.storeHasMessages + ",size=" + this.size + ",cacheEnabled=" + isCacheEnabled()
|
||||
+ ",maxBatchSize:" + maxBatchSize + ",hasSpace:" + hasSpace();
|
||||
+ ",maxBatchSize:" + maxBatchSize + ",hasSpace:" + hasSpace() + ",pendingCachedIds.size:" + pendingCachedIds.size() + ",lastCachedId:" + lastCachedId;
|
||||
}
|
||||
|
||||
protected abstract void doFillBatch() throws Exception;
|
||||
|
|
|
@ -203,8 +203,8 @@ public class FilePendingMessageCursor extends AbstractPendingMessageCursor imple
|
|||
* @throws Exception
|
||||
*/
|
||||
@Override
|
||||
public synchronized void addMessageLast(MessageReference node) throws Exception {
|
||||
tryAddMessageLast(node, 0);
|
||||
public synchronized boolean addMessageLast(MessageReference node) throws Exception {
|
||||
return tryAddMessageLast(node, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -81,10 +81,12 @@ public interface PendingMessageCursor extends Service {
|
|||
* add message to await dispatch
|
||||
*
|
||||
* @param node
|
||||
* @return boolean true if successful, false if cursor traps a duplicate
|
||||
* @throws IOException
|
||||
* @throws Exception
|
||||
*/
|
||||
void addMessageLast(MessageReference node) throws Exception;
|
||||
boolean addMessageLast(MessageReference node) throws Exception;
|
||||
|
||||
/**
|
||||
* add message to await dispatch - if it can
|
||||
*
|
||||
|
|
|
@ -94,6 +94,7 @@ class QueueStorePrefetch extends AbstractStoreCursor {
|
|||
|
||||
@Override
|
||||
protected void setBatch(MessageId messageId) throws Exception {
|
||||
LOG.trace("{} setBatch {} loc: {}", this, messageId, messageId.getEntryLocator());
|
||||
store.setBatch(messageId);
|
||||
batchResetNeeded = false;
|
||||
}
|
||||
|
|
|
@ -183,7 +183,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized void addMessageLast(MessageReference node) throws Exception {
|
||||
public synchronized boolean addMessageLast(MessageReference node) throws Exception {
|
||||
if (node != null) {
|
||||
Message msg = node.getMessage();
|
||||
if (isStarted()) {
|
||||
|
@ -206,6 +206,7 @@ public class StoreDurableSubscriberCursor extends AbstractPendingMessageCursor {
|
|||
}
|
||||
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -87,7 +87,8 @@ public class StoreQueueCursor extends AbstractPendingMessageCursor {
|
|||
pendingCount = 0;
|
||||
}
|
||||
|
||||
public synchronized void addMessageLast(MessageReference node) throws Exception {
|
||||
public synchronized boolean addMessageLast(MessageReference node) throws Exception {
|
||||
boolean result = true;
|
||||
if (node != null) {
|
||||
Message msg = node.getMessage();
|
||||
if (started) {
|
||||
|
@ -97,9 +98,10 @@ public class StoreQueueCursor extends AbstractPendingMessageCursor {
|
|||
}
|
||||
}
|
||||
if (msg.isPersistent()) {
|
||||
persistent.addMessageLast(node);
|
||||
result = persistent.addMessageLast(node);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public synchronized void addMessageFirst(MessageReference node) throws Exception {
|
||||
|
|
|
@ -97,15 +97,15 @@ public class VMPendingMessageCursor extends AbstractPendingMessageCursor {
|
|||
* @param node
|
||||
*/
|
||||
|
||||
public synchronized void addMessageLast(MessageReference node) {
|
||||
public synchronized boolean addMessageLast(MessageReference node) {
|
||||
node.incrementReferenceCount();
|
||||
list.addMessageLast(node);
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* add message to await dispatch
|
||||
*
|
||||
* @param position
|
||||
* @param node
|
||||
*/
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ abstract public class AbstractMessageStore implements MessageStore {
|
|||
public static final ListenableFuture<Object> FUTURE;
|
||||
protected final ActiveMQDestination destination;
|
||||
protected boolean prioritizedMessages;
|
||||
protected IndexListener indexListener;
|
||||
|
||||
public AbstractMessageStore(ActiveMQDestination destination) {
|
||||
this.destination = destination;
|
||||
|
@ -114,10 +115,16 @@ abstract public class AbstractMessageStore implements MessageStore {
|
|||
removeMessage(context, ack);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateMessage(Message message) throws IOException {
|
||||
throw new IOException("update is not supported by: " + this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerIndexListener(IndexListener indexListener) {
|
||||
this.indexListener = indexListener;
|
||||
}
|
||||
|
||||
static {
|
||||
FUTURE = new InlineListenableFuture();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,47 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import org.apache.activemq.broker.ConnectionContext;
|
||||
import org.apache.activemq.command.Message;
|
||||
|
||||
/**
|
||||
* callback when the index is updated, allows ordered work to be seen by destinations
|
||||
*/
|
||||
public interface IndexListener {
|
||||
|
||||
final class MessageContext {
|
||||
public Message message;
|
||||
public ConnectionContext context;
|
||||
public Runnable onCompletion;
|
||||
public boolean duplicate;
|
||||
|
||||
public MessageContext(ConnectionContext context, Message message, Runnable onCompletion) {
|
||||
this.context = context;
|
||||
this.message = message;
|
||||
this.onCompletion = onCompletion;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* called with some global index lock held so that a listener can do order
|
||||
* dependent work
|
||||
* non null MessageContext.onCompletion called when work is done
|
||||
*/
|
||||
public void onAdd(MessageContext messageContext);
|
||||
|
||||
}
|
|
@ -195,4 +195,6 @@ public interface MessageStore extends Service {
|
|||
public boolean isPrioritizedMessages();
|
||||
|
||||
void updateMessage(Message message) throws IOException;
|
||||
|
||||
void registerIndexListener(IndexListener indexListener);
|
||||
}
|
||||
|
|
|
@ -160,4 +160,9 @@ public class ProxyMessageStore implements MessageStore {
|
|||
public void updateMessage(Message message) throws IOException {
|
||||
delegate.updateMessage(message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerIndexListener(IndexListener indexListener) {
|
||||
delegate.registerIndexListener(indexListener);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -208,4 +208,9 @@ public class ProxyTopicMessageStore implements TopicMessageStore {
|
|||
public void updateMessage(Message message) throws IOException {
|
||||
delegate.updateMessage(message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void registerIndexListener(IndexListener indexListener) {
|
||||
delegate.registerIndexListener(indexListener);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.activemq.command.ActiveMQDestination;
|
|||
import org.apache.activemq.command.Message;
|
||||
import org.apache.activemq.command.MessageAck;
|
||||
import org.apache.activemq.command.MessageId;
|
||||
import org.apache.activemq.store.IndexListener;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.apache.activemq.store.AbstractMessageStore;
|
||||
|
||||
|
@ -41,6 +42,7 @@ public class MemoryMessageStore extends AbstractMessageStore {
|
|||
|
||||
protected final Map<MessageId, Message> messageTable;
|
||||
protected MessageId lastBatchId;
|
||||
protected long sequenceId;
|
||||
|
||||
public MemoryMessageStore(ActiveMQDestination destination) {
|
||||
this(destination, new LinkedHashMap<MessageId, Message>());
|
||||
|
@ -56,6 +58,10 @@ public class MemoryMessageStore extends AbstractMessageStore {
|
|||
messageTable.put(message.getMessageId(), message);
|
||||
}
|
||||
message.incrementReferenceCount();
|
||||
message.getMessageId().setFutureOrSequenceLong(sequenceId++);
|
||||
if (indexListener != null) {
|
||||
indexListener.onAdd(new IndexListener.MessageContext(context, message, null));
|
||||
}
|
||||
}
|
||||
|
||||
// public void addMessageReference(ConnectionContext context,MessageId
|
||||
|
|
|
@ -37,6 +37,7 @@ public class MessageId implements DataStructure, Comparable<MessageId> {
|
|||
private transient AtomicReference<Object> dataLocator = new AtomicReference<Object>();
|
||||
private transient Object entryLocator;
|
||||
private transient Object plistLocator;
|
||||
private transient Object futureOrSequenceLong;
|
||||
|
||||
public MessageId() {
|
||||
this.producerId = new ProducerId();
|
||||
|
@ -186,6 +187,7 @@ public class MessageId implements DataStructure, Comparable<MessageId> {
|
|||
copy.brokerSequenceId = brokerSequenceId;
|
||||
copy.dataLocator = dataLocator;
|
||||
copy.entryLocator = entryLocator;
|
||||
copy.futureOrSequenceLong = futureOrSequenceLong;
|
||||
copy.plistLocator = plistLocator;
|
||||
copy.textView = textView;
|
||||
return copy;
|
||||
|
@ -219,6 +221,14 @@ public class MessageId implements DataStructure, Comparable<MessageId> {
|
|||
this.dataLocator.set(value);
|
||||
}
|
||||
|
||||
public Object getFutureOrSequenceLong() {
|
||||
return futureOrSequenceLong;
|
||||
}
|
||||
|
||||
public void setFutureOrSequenceLong(Object futureOrSequenceLong) {
|
||||
this.futureOrSequenceLong = futureOrSequenceLong;
|
||||
}
|
||||
|
||||
public Object getEntryLocator() {
|
||||
return entryLocator;
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.activemq.command.Message;
|
|||
import org.apache.activemq.command.MessageAck;
|
||||
import org.apache.activemq.command.MessageId;
|
||||
import org.apache.activemq.store.AbstractMessageStore;
|
||||
import org.apache.activemq.store.IndexListener;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.apache.activemq.util.ByteSequence;
|
||||
import org.apache.activemq.util.ByteSequenceData;
|
||||
|
@ -101,7 +102,7 @@ public class JDBCMessageStore extends AbstractMessageStore {
|
|||
}
|
||||
}
|
||||
|
||||
public void addMessage(ConnectionContext context, Message message) throws IOException {
|
||||
public void addMessage(final ConnectionContext context, final Message message) throws IOException {
|
||||
MessageId messageId = message.getMessageId();
|
||||
if (audit != null && audit.isDuplicate(message)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -126,8 +127,26 @@ public class JDBCMessageStore extends AbstractMessageStore {
|
|||
long sequenceId;
|
||||
synchronized (pendingAdditions) {
|
||||
sequenceId = persistenceAdapter.getNextSequenceId();
|
||||
if (message.isInTransaction()) {
|
||||
trackPendingSequence(c, sequenceId);
|
||||
final long sequence = sequenceId;
|
||||
pendingAdditions.add(sequence);
|
||||
c.onCompletion(new Runnable() {
|
||||
public void run() {
|
||||
// message added to db
|
||||
message.getMessageId().setFutureOrSequenceLong(sequence);
|
||||
message.getMessageId().setEntryLocator(sequence);
|
||||
}
|
||||
});
|
||||
|
||||
if (indexListener != null) {
|
||||
indexListener.onAdd(new IndexListener.MessageContext(context, message, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
// cursor add complete
|
||||
synchronized (pendingAdditions) { pendingAdditions.remove(sequence); }
|
||||
}
|
||||
}));
|
||||
} else {
|
||||
pendingAdditions.remove(sequence);
|
||||
}
|
||||
}
|
||||
try {
|
||||
|
@ -139,20 +158,10 @@ public class JDBCMessageStore extends AbstractMessageStore {
|
|||
} finally {
|
||||
c.close();
|
||||
}
|
||||
message.getMessageId().setEntryLocator(sequenceId);
|
||||
onAdd(message, sequenceId, message.getPriority());
|
||||
}
|
||||
|
||||
// jdbc commit order is random with concurrent connections - limit scan to lowest pending
|
||||
private void trackPendingSequence(final TransactionContext transactionContext, final long sequenceId) {
|
||||
synchronized (pendingAdditions) { pendingAdditions.add(sequenceId); }
|
||||
transactionContext.onCompletion(new Runnable() {
|
||||
public void run() {
|
||||
synchronized (pendingAdditions) { pendingAdditions.remove(sequenceId); }
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private long minPendingSequeunceId() {
|
||||
synchronized (pendingAdditions) {
|
||||
if (!pendingAdditions.isEmpty()) {
|
||||
|
@ -237,8 +246,8 @@ public class JDBCMessageStore extends AbstractMessageStore {
|
|||
|
||||
public void removeMessage(ConnectionContext context, MessageAck ack) throws IOException {
|
||||
|
||||
long seq = ack.getLastMessageId().getEntryLocator() != null ?
|
||||
(Long) ack.getLastMessageId().getEntryLocator() :
|
||||
long seq = ack.getLastMessageId().getFutureOrSequenceLong() != null ?
|
||||
(Long) ack.getLastMessageId().getFutureOrSequenceLong() :
|
||||
persistenceAdapter.getStoreSequenceIdForMessageId(ack.getLastMessageId(), destination)[0];
|
||||
|
||||
// Get a connection and remove the message from the DB
|
||||
|
@ -251,9 +260,9 @@ public class JDBCMessageStore extends AbstractMessageStore {
|
|||
} finally {
|
||||
c.close();
|
||||
}
|
||||
if (context != null && context.getXid() != null) {
|
||||
ack.getLastMessageId().setEntryLocator(seq);
|
||||
}
|
||||
//if (context != null && context.getXid() != null) {
|
||||
// ack.getLastMessageId().setEntryLocator(seq);
|
||||
//}
|
||||
}
|
||||
|
||||
public void recover(final MessageRecoveryListener listener) throws Exception {
|
||||
|
@ -341,7 +350,7 @@ public class JDBCMessageStore extends AbstractMessageStore {
|
|||
public boolean recoverMessage(long sequenceId, byte[] data) throws Exception {
|
||||
Message msg = (Message)wireFormat.unmarshal(new ByteSequence(data));
|
||||
msg.getMessageId().setBrokerSequenceId(sequenceId);
|
||||
msg.getMessageId().setEntryLocator(sequenceId);
|
||||
msg.getMessageId().setFutureOrSequenceLong(sequenceId);
|
||||
listener.recoverMessage(msg);
|
||||
lastRecoveredSequenceId.set(sequenceId);
|
||||
lastRecoveredPriority.set(msg.getPriority());
|
||||
|
|
|
@ -773,7 +773,7 @@ public class JDBCPersistenceAdapter extends DataSourceServiceSupport implements
|
|||
public void commitAdd(ConnectionContext context, MessageId messageId) throws IOException {
|
||||
TransactionContext c = getTransactionContext(context);
|
||||
try {
|
||||
long sequence = (Long)messageId.getEntryLocator();
|
||||
long sequence = (Long)messageId.getFutureOrSequenceLong();
|
||||
getAdapter().doCommitAddOp(c, sequence);
|
||||
} catch (SQLException e) {
|
||||
JDBCPersistenceAdapter.log("JDBC Failure: ", e);
|
||||
|
@ -786,7 +786,7 @@ public class JDBCPersistenceAdapter extends DataSourceServiceSupport implements
|
|||
public void commitRemove(ConnectionContext context, MessageAck ack) throws IOException {
|
||||
TransactionContext c = getTransactionContext(context);
|
||||
try {
|
||||
getAdapter().doRemoveMessage(c, (Long)ack.getLastMessageId().getEntryLocator(), null);
|
||||
getAdapter().doRemoveMessage(c, (Long)ack.getLastMessageId().getFutureOrSequenceLong(), null);
|
||||
} catch (SQLException e) {
|
||||
JDBCPersistenceAdapter.log("JDBC Failure: ", e);
|
||||
throw IOExceptionSupport.create("Failed to commit last ack: " + ack + ". Reason: " + e,e);
|
||||
|
|
|
@ -108,7 +108,7 @@ public class JdbcMemoryTransactionStore extends MemoryTransactionStore {
|
|||
jdbcPersistenceAdapter.commitAdd(context, message.getMessageId());
|
||||
((JDBCMessageStore)addMessageCommand.getMessageStore()).onAdd(
|
||||
message,
|
||||
(Long)message.getMessageId().getEntryLocator(),
|
||||
(Long)message.getMessageId().getFutureOrSequenceLong(),
|
||||
message.getPriority());
|
||||
|
||||
}
|
||||
|
@ -170,7 +170,7 @@ public class JdbcMemoryTransactionStore extends MemoryTransactionStore {
|
|||
|
||||
public void recoverAdd(long id, byte[] messageBytes) throws IOException {
|
||||
final Message message = (Message) ((JDBCPersistenceAdapter)persistenceAdapter).getWireFormat().unmarshal(new ByteSequence(messageBytes));
|
||||
message.getMessageId().setEntryLocator(id);
|
||||
message.getMessageId().setFutureOrSequenceLong(id);
|
||||
Tx tx = getPreparedTx(message.getTransactionId());
|
||||
tx.add(new AddMessageCommand() {
|
||||
MessageStore messageStore;
|
||||
|
@ -187,7 +187,7 @@ public class JdbcMemoryTransactionStore extends MemoryTransactionStore {
|
|||
@Override
|
||||
public void run(ConnectionContext context) throws IOException {
|
||||
((JDBCPersistenceAdapter)persistenceAdapter).commitAdd(null, message.getMessageId());
|
||||
((JDBCMessageStore)messageStore).onAdd(message, ((Long)message.getMessageId().getEntryLocator()).longValue(), message.getPriority());
|
||||
((JDBCMessageStore)messageStore).onAdd(message, ((Long)message.getMessageId().getFutureOrSequenceLong()).longValue(), message.getPriority());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -200,7 +200,7 @@ public class JdbcMemoryTransactionStore extends MemoryTransactionStore {
|
|||
|
||||
public void recoverAck(long id, byte[] xid, byte[] message) throws IOException {
|
||||
Message msg = (Message) ((JDBCPersistenceAdapter)persistenceAdapter).getWireFormat().unmarshal(new ByteSequence(message));
|
||||
msg.getMessageId().setEntryLocator(id);
|
||||
msg.getMessageId().setFutureOrSequenceLong(id);
|
||||
Tx tx = getPreparedTx(new XATransactionId(xid));
|
||||
final MessageAck ack = new MessageAck(msg, MessageAck.STANDARD_ACK_TYPE, 1);
|
||||
tx.add(new RemoveMessageCommand() {
|
||||
|
|
|
@ -396,7 +396,7 @@ public class DefaultJDBCAdapter implements JDBCAdapter {
|
|||
if (this.batchStatements) {
|
||||
s.addBatch();
|
||||
} else if (s.executeUpdate() != 1) {
|
||||
throw new SQLException("Failed to remove message");
|
||||
throw new SQLException("Failed to remove message seq: " + seq);
|
||||
}
|
||||
} finally {
|
||||
cleanupExclusiveLock.readLock().unlock();
|
||||
|
@ -935,7 +935,7 @@ public class DefaultJDBCAdapter implements JDBCAdapter {
|
|||
this.batchStatements = batchStatements;
|
||||
// The next lines are deprecated and should be removed in a future release
|
||||
// and is here in case someone created their own
|
||||
this.batchStatments = batchStatements;
|
||||
// this.batchStatments = batchStatements;
|
||||
}
|
||||
|
||||
// Note - remove batchStatment in future distributions. Here for backward compatibility
|
||||
|
@ -1168,8 +1168,12 @@ public class DefaultJDBCAdapter implements JDBCAdapter {
|
|||
printQuery(s,System.out); }
|
||||
|
||||
public static void dumpTables(java.sql.Connection c) throws SQLException {
|
||||
printQuery(c, "Select * from ACTIVEMQ_MSGS ORDER BY ID", System.out);
|
||||
printQuery(c, "Select * from ACTIVEMQ_ACKS", System.out);
|
||||
printQuery(c, "SELECT COUNT(*) from ACTIVEMQ_MSGS", System.out);
|
||||
|
||||
//printQuery(c, "SELECT COUNT(*) from ACTIVEMQ_ACKS", System.out);
|
||||
|
||||
//printQuery(c, "Select * from ACTIVEMQ_MSGS ORDER BY ID", System.out);
|
||||
//printQuery(c, "Select * from ACTIVEMQ_ACKS", System.out);
|
||||
}
|
||||
|
||||
public static void printQuery(java.sql.Connection c, String query, java.io.PrintStream out)
|
||||
|
|
|
@ -29,6 +29,8 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.CancellationException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.FutureTask;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
@ -57,6 +59,7 @@ import org.apache.activemq.command.TransactionId;
|
|||
import org.apache.activemq.openwire.OpenWireFormat;
|
||||
import org.apache.activemq.protobuf.Buffer;
|
||||
import org.apache.activemq.store.AbstractMessageStore;
|
||||
import org.apache.activemq.store.IndexListener;
|
||||
import org.apache.activemq.store.ListenableFuture;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.apache.activemq.store.MessageStore;
|
||||
|
@ -83,7 +86,7 @@ import org.apache.activemq.wireformat.WireFormat;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
||||
public class KahaDBStore extends MessageDatabase implements PersistenceAdapter, MessageDatabase.SerialExecution<Location> {
|
||||
static final Logger LOG = LoggerFactory.getLogger(KahaDBStore.class);
|
||||
private static final int MAX_ASYNC_JOBS = 10000;
|
||||
|
||||
|
@ -121,6 +124,7 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
return txid;
|
||||
}
|
||||
};
|
||||
serialExecutor = this;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -207,7 +211,7 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
// In case the recovered store used a different OpenWire version log a warning
|
||||
// to assist in determining why journal reads fail.
|
||||
if (metadata.openwireVersion != brokerService.getStoreOpenWireVersion()) {
|
||||
LOG.warn("Receovered Store uses a different OpenWire version[{}] " +
|
||||
LOG.warn("Recovered Store uses a different OpenWire version[{}] " +
|
||||
"than the version configured[{}].",
|
||||
metadata.openwireVersion, brokerService.getStoreOpenWireVersion());
|
||||
}
|
||||
|
@ -286,21 +290,6 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
super.doStop(stopper);
|
||||
}
|
||||
|
||||
@Override
|
||||
void rollbackStatsOnDuplicate(KahaDestination commandDestination) {
|
||||
if (brokerService != null) {
|
||||
RegionBroker regionBroker = (RegionBroker) brokerService.getRegionBroker();
|
||||
if (regionBroker != null) {
|
||||
ActiveMQDestination activeMQDestination = convert(commandDestination);
|
||||
Destination destination = regionBroker.getDestinationMap(activeMQDestination).get(activeMQDestination);
|
||||
if (destination != null) {
|
||||
destination.getDestinationStatistics().getMessages().decrement();
|
||||
destination.getDestinationStatistics().getEnqueues().decrement();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Location findMessageLocation(final String key, final KahaDestination destination) throws IOException {
|
||||
return pageFile.tx().execute(new Transaction.CallableClosure<Location, IOException>() {
|
||||
@Override
|
||||
|
@ -358,6 +347,17 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
this.forceRecoverIndex = forceRecoverIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Location execute(Callable<Location> c) throws Exception {
|
||||
if (isConcurrentStoreAndDispatchQueues()) {
|
||||
FutureTask<Location> future = new FutureTask<>(c);
|
||||
this.queueExecutor.execute(future);
|
||||
return future.get();
|
||||
} else {
|
||||
return c.call();
|
||||
}
|
||||
}
|
||||
|
||||
public class KahaDBMessageStore extends AbstractMessageStore {
|
||||
protected final Map<AsyncJobKey, StoreTask> asyncTaskMap = new HashMap<AsyncJobKey, StoreTask>();
|
||||
protected KahaDestination dest;
|
||||
|
@ -385,7 +385,25 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
StoreQueueTask result = new StoreQueueTask(this, context, message);
|
||||
result.aquireLocks();
|
||||
addQueueTask(this, result);
|
||||
return result.getFuture();
|
||||
final ListenableFuture<Object> future = result.getFuture();
|
||||
if (indexListener != null) {
|
||||
// allow concurrent dispatch by setting entry locator,
|
||||
// wait for add completion to remove potential pending addition
|
||||
message.getMessageId().setFutureOrSequenceLong(future);
|
||||
indexListener.onAdd(new IndexListener.MessageContext(context, message, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
future.get();
|
||||
trackPendingAddComplete(dest, (Long) message.getMessageId().getFutureOrSequenceLong());
|
||||
} catch (CancellationException okNothingToTrack) {
|
||||
} catch (Exception e) {
|
||||
LOG.warn("{} unexpected exception tracking completion of async add of {}", this, message.getMessageId(), e);
|
||||
}
|
||||
}
|
||||
}));
|
||||
}
|
||||
return future;
|
||||
} else {
|
||||
return super.asyncAddQueueMessage(context, message);
|
||||
}
|
||||
|
@ -423,7 +441,7 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void addMessage(ConnectionContext context, Message message) throws IOException {
|
||||
public void addMessage(final ConnectionContext context, final Message message) throws IOException {
|
||||
KahaAddMessageCommand command = new KahaAddMessageCommand();
|
||||
command.setDestination(dest);
|
||||
command.setMessageId(message.getMessageId().toProducerKey());
|
||||
|
@ -432,8 +450,25 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
command.setPrioritySupported(isPrioritizedMessages());
|
||||
org.apache.activemq.util.ByteSequence packet = wireFormat.marshal(message);
|
||||
command.setMessage(new Buffer(packet.getData(), packet.getOffset(), packet.getLength()));
|
||||
store(command, isEnableJournalDiskSyncs() && message.isResponseRequired(), null, null);
|
||||
|
||||
store(command, isEnableJournalDiskSyncs() && message.isResponseRequired(), new IndexAware() {
|
||||
@Override
|
||||
public void sequenceAssignedWithIndexLocked(final long sequence) {
|
||||
final Object possibleFuture = message.getMessageId().getFutureOrSequenceLong();
|
||||
message.getMessageId().setFutureOrSequenceLong(sequence);
|
||||
if (indexListener != null) {
|
||||
trackPendingAdd(dest, sequence);
|
||||
if (possibleFuture == null) {
|
||||
// sync add (for async future present from getFutureOrSequenceLong)
|
||||
indexListener.onAdd(new IndexListener.MessageContext(context, message, new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
trackPendingAddComplete(dest, sequence);
|
||||
}
|
||||
}));
|
||||
}
|
||||
}
|
||||
}
|
||||
}, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -582,6 +617,7 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
continue;
|
||||
}
|
||||
Message msg = loadMessage(entry.getValue().location);
|
||||
msg.getMessageId().setFutureOrSequenceLong(entry.getKey());
|
||||
listener.recoverMessage(msg);
|
||||
counter++;
|
||||
if (counter >= maxReturned) {
|
||||
|
@ -643,7 +679,7 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setBatch(MessageId identity) throws IOException {
|
||||
public void setBatch(final MessageId identity) throws IOException {
|
||||
try {
|
||||
final String key = identity.toProducerKey();
|
||||
lockAsyncJobQueue();
|
||||
|
@ -660,6 +696,8 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter {
|
|||
Long location = sd.messageIdIndex.get(tx, key);
|
||||
if (location != null) {
|
||||
sd.orderIndex.setBatch(tx, location);
|
||||
} else {
|
||||
LOG.warn("{} Location {} not found in order index for {}", this, identity.getFutureOrSequenceLong(), identity);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
|
|
@ -42,9 +42,7 @@ import org.apache.activemq.store.ProxyTopicMessageStore;
|
|||
import org.apache.activemq.store.TopicMessageStore;
|
||||
import org.apache.activemq.store.TransactionRecoveryListener;
|
||||
import org.apache.activemq.store.TransactionStore;
|
||||
import org.apache.activemq.store.kahadb.MessageDatabase.AddOpperation;
|
||||
import org.apache.activemq.store.kahadb.MessageDatabase.Operation;
|
||||
import org.apache.activemq.store.kahadb.MessageDatabase.RemoveOpperation;
|
||||
import org.apache.activemq.store.kahadb.data.KahaCommitCommand;
|
||||
import org.apache.activemq.store.kahadb.data.KahaPrepareCommand;
|
||||
import org.apache.activemq.store.kahadb.data.KahaRollbackCommand;
|
||||
|
@ -254,7 +252,7 @@ public class KahaDBTransactionStore implements TransactionStore {
|
|||
return tx;
|
||||
}
|
||||
|
||||
public void commit(TransactionId txid, boolean wasPrepared, Runnable preCommit, Runnable postCommit)
|
||||
public void commit(TransactionId txid, boolean wasPrepared, final Runnable preCommit, Runnable postCommit)
|
||||
throws IOException {
|
||||
if (txid != null) {
|
||||
if (!txid.isXATransaction() && theStore.isConcurrentStoreAndDispatchTransactions()) {
|
||||
|
@ -294,7 +292,10 @@ public class KahaDBTransactionStore implements TransactionStore {
|
|||
|
||||
} else {
|
||||
KahaTransactionInfo info = getTransactionInfo(txid);
|
||||
theStore.store(new KahaCommitCommand().setTransactionInfo(info), theStore.isEnableJournalDiskSyncs(), preCommit, postCommit);
|
||||
if (preCommit != null) {
|
||||
preCommit.run();
|
||||
}
|
||||
theStore.store(new KahaCommitCommand().setTransactionInfo(info), theStore.isEnableJournalDiskSyncs(), null, postCommit);
|
||||
forgetRecoveredAcks(txid, false);
|
||||
}
|
||||
}else {
|
||||
|
@ -336,13 +337,13 @@ public class KahaDBTransactionStore implements TransactionStore {
|
|||
ArrayList<MessageAck> ackList = new ArrayList<MessageAck>();
|
||||
|
||||
for (Operation op : entry.getValue()) {
|
||||
if (op.getClass() == AddOpperation.class) {
|
||||
AddOpperation addOp = (AddOpperation) op;
|
||||
if (op.getClass() == MessageDatabase.AddOperation.class) {
|
||||
MessageDatabase.AddOperation addOp = (MessageDatabase.AddOperation) op;
|
||||
Message msg = (Message) wireFormat().unmarshal(new DataInputStream(addOp.getCommand().getMessage()
|
||||
.newInput()));
|
||||
messageList.add(msg);
|
||||
} else {
|
||||
RemoveOpperation rmOp = (RemoveOpperation) op;
|
||||
MessageDatabase.RemoveOperation rmOp = (MessageDatabase.RemoveOperation) op;
|
||||
Buffer ackb = rmOp.getCommand().getAck();
|
||||
MessageAck ack = (MessageAck) wireFormat().unmarshal(new DataInputStream(ackb.newInput()));
|
||||
ackList.add(ack);
|
||||
|
|
|
@ -38,6 +38,7 @@ import java.util.HashSet;
|
|||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
@ -45,6 +46,7 @@ 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;
|
||||
|
@ -254,8 +256,10 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
private boolean enableIndexPageCaching = true;
|
||||
ReentrantReadWriteLock checkpointLock = new ReentrantReadWriteLock();
|
||||
|
||||
public MessageDatabase() {
|
||||
interface SerialExecution<V> {
|
||||
public V execute(Callable<V> c) throws Exception;
|
||||
}
|
||||
SerialExecution<Location> serialExecutor;
|
||||
|
||||
@Override
|
||||
public void doStart() throws Exception {
|
||||
|
@ -517,12 +521,12 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
}
|
||||
KahaDestination destination;
|
||||
boolean isAdd = false;
|
||||
if (operation instanceof AddOpperation) {
|
||||
AddOpperation add = (AddOpperation) operation;
|
||||
if (operation instanceof AddOperation) {
|
||||
AddOperation add = (AddOperation) operation;
|
||||
destination = add.getCommand().getDestination();
|
||||
isAdd = true;
|
||||
} else {
|
||||
RemoveOpperation removeOpperation = (RemoveOpperation) operation;
|
||||
RemoveOperation removeOpperation = (RemoveOperation) operation;
|
||||
destination = removeOpperation.getCommand().getDestination();
|
||||
}
|
||||
opCount opCount = destinationOpCount.get(destination);
|
||||
|
@ -884,7 +888,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
lastRecoveryPosition = nextRecoveryPosition;
|
||||
metadata.lastUpdate = lastRecoveryPosition;
|
||||
JournalCommand<?> message = load(lastRecoveryPosition);
|
||||
process(message, lastRecoveryPosition, (Runnable)null, (Runnable)null);
|
||||
process(message, lastRecoveryPosition, (IndexAware) null);
|
||||
nextRecoveryPosition = journal.getNextLocation(lastRecoveryPosition);
|
||||
}
|
||||
} finally {
|
||||
|
@ -951,20 +955,34 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
}
|
||||
|
||||
public Location store(JournalCommand<?> data, Runnable onJournalStoreComplete) throws IOException {
|
||||
return store(data, false, null,null, onJournalStoreComplete);
|
||||
return store(data, false, null, null, onJournalStoreComplete);
|
||||
}
|
||||
|
||||
public Location store(JournalCommand<?> data, boolean sync, Runnable before,Runnable after) throws IOException {
|
||||
public Location store(JournalCommand<?> data, boolean sync, IndexAware before,Runnable after) throws IOException {
|
||||
return store(data, sync, before, after, null);
|
||||
}
|
||||
|
||||
public Location store(final KahaCommitCommand data, final boolean sync, final IndexAware before, final Runnable after) throws IOException {
|
||||
try {
|
||||
return serialExecutor.execute(new Callable<Location>() {
|
||||
@Override
|
||||
public Location call() throws Exception {
|
||||
return store(data, sync, before, after, null);
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to execute commit", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* All updated are are funneled through this method. The updates are converted
|
||||
* to a JournalMessage which is logged to the journal and then the data from
|
||||
* the JournalMessage is used to update the index just like it would be done
|
||||
* during a recovery process.
|
||||
*/
|
||||
public Location store(JournalCommand<?> data, boolean sync, Runnable before, Runnable after, Runnable onJournalStoreComplete) throws IOException {
|
||||
public Location store(JournalCommand<?> data, boolean sync, IndexAware before, Runnable after, Runnable onJournalStoreComplete) throws IOException {
|
||||
try {
|
||||
ByteSequence sequence = toByteSequence(data);
|
||||
|
||||
|
@ -975,7 +993,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
long start = System.currentTimeMillis();
|
||||
location = onJournalStoreComplete == null ? journal.write(sequence, sync) : journal.write(sequence, onJournalStoreComplete) ;
|
||||
long start2 = System.currentTimeMillis();
|
||||
process(data, location, before, after);
|
||||
process(data, location, before);
|
||||
|
||||
long end = System.currentTimeMillis();
|
||||
if( LOG_SLOW_ACCESS_TIME>0 && end-start > LOG_SLOW_ACCESS_TIME) {
|
||||
|
@ -1049,7 +1067,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
return;
|
||||
}
|
||||
}
|
||||
process(data, location, (Runnable) null, (Runnable) null);
|
||||
process(data, location, (IndexAware) null);
|
||||
} else {
|
||||
// just recover producer audit
|
||||
data.visit(new Visitor() {
|
||||
|
@ -1067,11 +1085,11 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
// from the recovery method too so they need to be idempotent
|
||||
// /////////////////////////////////////////////////////////////////
|
||||
|
||||
void process(JournalCommand<?> data, final Location location, final Runnable before, final Runnable after) throws IOException {
|
||||
void process(JournalCommand<?> data, final Location location, final IndexAware onSequenceAssignedCallback) throws IOException {
|
||||
data.visit(new Visitor() {
|
||||
@Override
|
||||
public void visit(KahaAddMessageCommand command) throws IOException {
|
||||
process(command, location);
|
||||
process(command, location, onSequenceAssignedCallback);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1086,7 +1104,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
|
||||
@Override
|
||||
public void visit(KahaCommitCommand command) throws IOException {
|
||||
process(command, location, before, after);
|
||||
process(command, location, onSequenceAssignedCallback);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1127,19 +1145,23 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
protected void process(final KahaAddMessageCommand command, final Location location) throws IOException {
|
||||
protected void process(final KahaAddMessageCommand command, final Location location, final IndexAware runWithIndexLock) throws IOException {
|
||||
if (command.hasTransactionInfo()) {
|
||||
List<Operation> inflightTx = getInflightTx(command.getTransactionInfo(), location);
|
||||
inflightTx.add(new AddOpperation(command, location));
|
||||
List<Operation> inflightTx = getInflightTx(command.getTransactionInfo());
|
||||
inflightTx.add(new AddOperation(command, location, runWithIndexLock));
|
||||
} else {
|
||||
this.indexLock.writeLock().lock();
|
||||
try {
|
||||
pageFile.tx().execute(new Transaction.Closure<IOException>() {
|
||||
@Override
|
||||
public void execute(Transaction tx) throws IOException {
|
||||
updateIndex(tx, command, location);
|
||||
long assignedIndex = updateIndex(tx, command, location);
|
||||
if (runWithIndexLock != null) {
|
||||
runWithIndexLock.sequenceAssignedWithIndexLocked(assignedIndex);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
} finally {
|
||||
this.indexLock.writeLock().unlock();
|
||||
}
|
||||
|
@ -1164,8 +1186,8 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
@SuppressWarnings("rawtypes")
|
||||
protected void process(final KahaRemoveMessageCommand command, final Location location) throws IOException {
|
||||
if (command.hasTransactionInfo()) {
|
||||
List<Operation> inflightTx = getInflightTx(command.getTransactionInfo(), location);
|
||||
inflightTx.add(new RemoveOpperation(command, location));
|
||||
List<Operation> inflightTx = getInflightTx(command.getTransactionInfo());
|
||||
inflightTx.add(new RemoveOperation(command, location));
|
||||
} else {
|
||||
this.indexLock.writeLock().lock();
|
||||
try {
|
||||
|
@ -1219,7 +1241,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
protected void process(KahaCommitCommand command, Location location, final Runnable before, final Runnable after) throws IOException {
|
||||
protected void process(KahaCommitCommand command, final Location location, final IndexAware before) throws IOException {
|
||||
TransactionId key = TransactionIdConversion.convert(command.getTransactionInfo());
|
||||
List<Operation> inflightTx;
|
||||
synchronized (inflightTransactions) {
|
||||
|
@ -1231,18 +1253,15 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
if (inflightTx == null) {
|
||||
// only non persistent messages in this tx
|
||||
if (before != null) {
|
||||
before.run();
|
||||
before.sequenceAssignedWithIndexLocked(-1);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
final List<Operation> messagingTx = inflightTx;
|
||||
this.indexLock.writeLock().lock();
|
||||
try {
|
||||
// run before with the index lock so that queue can order cursor updates with index updates
|
||||
if (before != null) {
|
||||
before.run();
|
||||
}
|
||||
indexLock.writeLock().lock();
|
||||
try {
|
||||
pageFile.tx().execute(new Transaction.Closure<IOException>() {
|
||||
@Override
|
||||
public void execute(Transaction tx) throws IOException {
|
||||
|
@ -1253,7 +1272,11 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
});
|
||||
metadata.lastUpdate = location;
|
||||
} finally {
|
||||
this.indexLock.writeLock().unlock();
|
||||
indexLock.writeLock().unlock();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("serial execution of commit failed", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1287,13 +1310,13 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
protected final ReentrantReadWriteLock indexLock = new ReentrantReadWriteLock();
|
||||
private final HashSet<Integer> journalFilesBeingReplicated = new HashSet<Integer>();
|
||||
|
||||
void updateIndex(Transaction tx, KahaAddMessageCommand command, Location location) throws IOException {
|
||||
long updateIndex(Transaction tx, KahaAddMessageCommand command, Location location) throws IOException {
|
||||
StoredDestination sd = getStoredDestination(command.getDestination(), tx);
|
||||
|
||||
// Skip adding the message to the index if this is a topic and there are
|
||||
// no subscriptions.
|
||||
if (sd.subscriptions != null && sd.subscriptions.isEmpty(tx)) {
|
||||
return;
|
||||
return -1;
|
||||
}
|
||||
|
||||
// Add the message.
|
||||
|
@ -1308,12 +1331,11 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
addAckLocationForNewMessage(tx, sd, id);
|
||||
}
|
||||
} else {
|
||||
// If the message ID as indexed, then the broker asked us to
|
||||
// store a DUP message. Bad BOY! Don't do it, and log a warning.
|
||||
// If the message ID is indexed, then the broker asked us to store a duplicate before the message was dispatched and acked, we ignore this add attempt
|
||||
LOG.warn("Duplicate message add attempt rejected. Destination: {}://{}, Message id: {}", command.getDestination().getType(), command.getDestination().getName(), command.getMessageId());
|
||||
sd.messageIdIndex.put(tx, command.getMessageId(), previous);
|
||||
sd.locationIndex.remove(tx, location);
|
||||
rollbackStatsOnDuplicate(command.getDestination());
|
||||
id = -1;
|
||||
}
|
||||
} else {
|
||||
// restore the previous value.. Looks like this was a redo of a previously
|
||||
|
@ -1324,6 +1346,21 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
// record this id in any event, initial send or recovery
|
||||
metadata.producerSequenceIdTracker.isDuplicate(command.getMessageId());
|
||||
metadata.lastUpdate = location;
|
||||
return id;
|
||||
}
|
||||
|
||||
void trackPendingAdd(KahaDestination destination, Long seq) {
|
||||
StoredDestination sd = storedDestinations.get(key(destination));
|
||||
if (sd != null) {
|
||||
sd.trackPendingAdd(seq);
|
||||
}
|
||||
}
|
||||
|
||||
void trackPendingAddComplete(KahaDestination destination, Long seq) {
|
||||
StoredDestination sd = storedDestinations.get(key(destination));
|
||||
if (sd != null) {
|
||||
sd.trackPendingAddComplete(seq);
|
||||
}
|
||||
}
|
||||
|
||||
void updateIndex(Transaction tx, KahaUpdateMessageCommand updateMessageCommand, Location location) throws IOException {
|
||||
|
@ -1345,8 +1382,6 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
metadata.lastUpdate = location;
|
||||
}
|
||||
|
||||
abstract void rollbackStatsOnDuplicate(KahaDestination commandDestination);
|
||||
|
||||
void updateIndex(Transaction tx, KahaRemoveMessageCommand command, Location ackLocation) throws IOException {
|
||||
StoredDestination sd = getStoredDestination(command.getDestination(), tx);
|
||||
if (!command.hasSubscriptionKey()) {
|
||||
|
@ -1864,6 +1899,14 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
// Transient data used to track which Messages are no longer needed.
|
||||
final TreeMap<Long, Long> messageReferences = new TreeMap<Long, Long>();
|
||||
final HashSet<String> subscriptionCache = new LinkedHashSet<String>();
|
||||
|
||||
public void trackPendingAdd(Long seq) {
|
||||
orderIndex.trackPendingAdd(seq);
|
||||
}
|
||||
|
||||
public void trackPendingAddComplete(Long seq) {
|
||||
orderIndex.trackPendingAddComplete(seq);
|
||||
}
|
||||
}
|
||||
|
||||
protected class StoredDestinationMarshaller extends VariableMarshaller<StoredDestination> {
|
||||
|
@ -2361,7 +2404,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
private List<Operation> getInflightTx(KahaTransactionInfo info, Location location) {
|
||||
private List<Operation> getInflightTx(KahaTransactionInfo info) {
|
||||
TransactionId key = TransactionIdConversion.convert(info);
|
||||
List<Operation> tx;
|
||||
synchronized (inflightTransactions) {
|
||||
|
@ -2399,22 +2442,26 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
abstract public void execute(Transaction tx) throws IOException;
|
||||
}
|
||||
|
||||
class AddOpperation extends Operation<KahaAddMessageCommand> {
|
||||
|
||||
public AddOpperation(KahaAddMessageCommand command, Location location) {
|
||||
class AddOperation extends Operation<KahaAddMessageCommand> {
|
||||
final IndexAware runWithIndexLock;
|
||||
public AddOperation(KahaAddMessageCommand command, Location location, IndexAware runWithIndexLock) {
|
||||
super(command, location);
|
||||
this.runWithIndexLock = runWithIndexLock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(Transaction tx) throws IOException {
|
||||
updateIndex(tx, command, location);
|
||||
long seq = updateIndex(tx, command, location);
|
||||
if (runWithIndexLock != null) {
|
||||
runWithIndexLock.sequenceAssignedWithIndexLocked(seq);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class RemoveOpperation extends Operation<KahaRemoveMessageCommand> {
|
||||
class RemoveOperation extends Operation<KahaRemoveMessageCommand> {
|
||||
|
||||
public RemoveOpperation(KahaRemoveMessageCommand command, Location location) {
|
||||
public RemoveOperation(KahaRemoveMessageCommand command, Location location) {
|
||||
super(command, location);
|
||||
}
|
||||
|
||||
|
@ -2765,6 +2812,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
Long lastHighKey;
|
||||
Long lastLowKey;
|
||||
byte lastGetPriority;
|
||||
final List<Long> pendingAdditions = new LinkedList<Long>();
|
||||
|
||||
MessageKeys remove(Transaction tx, Long key) throws IOException {
|
||||
MessageKeys result = defaultPriorityIndex.remove(tx, key);
|
||||
|
@ -2928,7 +2976,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
void getDeleteList(Transaction tx, ArrayList<Entry<Long, MessageKeys>> deletes,
|
||||
BTreeIndex<Long, MessageKeys> index, Long sequenceId) throws IOException {
|
||||
|
||||
Iterator<Entry<Long, MessageKeys>> iterator = index.iterator(tx, sequenceId);
|
||||
Iterator<Entry<Long, MessageKeys>> iterator = index.iterator(tx, sequenceId, null);
|
||||
deletes.add(iterator.next());
|
||||
}
|
||||
|
||||
|
@ -2963,11 +3011,11 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
}
|
||||
|
||||
Iterator<Entry<Long, MessageKeys>> iterator(Transaction tx) throws IOException{
|
||||
return new MessageOrderIterator(tx,cursor);
|
||||
return new MessageOrderIterator(tx,cursor,this);
|
||||
}
|
||||
|
||||
Iterator<Entry<Long, MessageKeys>> iterator(Transaction tx, MessageOrderCursor m) throws IOException{
|
||||
return new MessageOrderIterator(tx,m);
|
||||
return new MessageOrderIterator(tx,m,this);
|
||||
}
|
||||
|
||||
public byte lastGetPriority() {
|
||||
|
@ -2980,21 +3028,45 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
(cursor.lowPriorityCursorPosition > 0 && cursor.lowPriorityCursorPosition >= sequence);
|
||||
}
|
||||
|
||||
public void trackPendingAdd(Long seq) {
|
||||
synchronized (pendingAdditions) {
|
||||
pendingAdditions.add(seq);
|
||||
}
|
||||
}
|
||||
|
||||
public void trackPendingAddComplete(Long seq) {
|
||||
synchronized (pendingAdditions) {
|
||||
pendingAdditions.remove(seq);
|
||||
}
|
||||
}
|
||||
|
||||
public Long minPendingAdd() {
|
||||
synchronized (pendingAdditions) {
|
||||
if (!pendingAdditions.isEmpty()) {
|
||||
return pendingAdditions.get(0);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
class MessageOrderIterator implements Iterator<Entry<Long, MessageKeys>>{
|
||||
Iterator<Entry<Long, MessageKeys>>currentIterator;
|
||||
final Iterator<Entry<Long, MessageKeys>>highIterator;
|
||||
final Iterator<Entry<Long, MessageKeys>>defaultIterator;
|
||||
final Iterator<Entry<Long, MessageKeys>>lowIterator;
|
||||
|
||||
MessageOrderIterator(Transaction tx, MessageOrderCursor m) throws IOException {
|
||||
this.defaultIterator = defaultPriorityIndex.iterator(tx, m.defaultCursorPosition);
|
||||
MessageOrderIterator(Transaction tx, MessageOrderCursor m, MessageOrderIndex messageOrderIndex) throws IOException {
|
||||
Long pendingAddLimiter = messageOrderIndex.minPendingAdd();
|
||||
this.defaultIterator = defaultPriorityIndex.iterator(tx, m.defaultCursorPosition, pendingAddLimiter);
|
||||
if (highPriorityIndex != null) {
|
||||
this.highIterator = highPriorityIndex.iterator(tx, m.highPriorityCursorPosition);
|
||||
this.highIterator = highPriorityIndex.iterator(tx, m.highPriorityCursorPosition, pendingAddLimiter);
|
||||
} else {
|
||||
this.highIterator = null;
|
||||
}
|
||||
if (lowPriorityIndex != null) {
|
||||
this.lowIterator = lowPriorityIndex.iterator(tx, m.lowPriorityCursorPosition);
|
||||
this.lowIterator = lowPriorityIndex.iterator(tx, m.lowPriorityCursorPosition, pendingAddLimiter);
|
||||
} else {
|
||||
this.lowIterator = null;
|
||||
}
|
||||
|
@ -3117,4 +3189,8 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe
|
|||
public void setIndexDirectory(File indexDirectory) {
|
||||
this.indexDirectory = indexDirectory;
|
||||
}
|
||||
|
||||
interface IndexAware {
|
||||
public void sequenceAssignedWithIndexLocked(long index);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -229,7 +229,11 @@ public class BTreeIndex<Key,Value> implements Index<Key,Value> {
|
|||
}
|
||||
|
||||
synchronized public Iterator<Map.Entry<Key,Value>> iterator(final Transaction tx, Key initialKey) throws IOException {
|
||||
return getRoot(tx).iterator(tx, initialKey);
|
||||
return getRoot(tx).iterator(tx, initialKey, null);
|
||||
}
|
||||
|
||||
synchronized public Iterator<Map.Entry<Key,Value>> iterator(final Transaction tx, Key initialKey, Key maxKey) throws IOException {
|
||||
return getRoot(tx).iterator(tx, initialKey, maxKey);
|
||||
}
|
||||
|
||||
synchronized public void visit(Transaction tx, BTreeVisitor<Key, Value> visitor) throws IOException {
|
||||
|
|
|
@ -80,14 +80,19 @@ public final class BTreeNode<Key,Value> {
|
|||
private final class BTreeIterator implements Iterator<Map.Entry<Key, Value>> {
|
||||
|
||||
private final Transaction tx;
|
||||
private final Key endKey;
|
||||
BTreeNode<Key,Value> current;
|
||||
int nextIndex;
|
||||
Map.Entry<Key,Value> nextEntry;
|
||||
|
||||
private BTreeIterator(Transaction tx, BTreeNode<Key,Value> current, int nextIndex) {
|
||||
private BTreeIterator(Transaction tx, BTreeNode<Key, Value> current, int nextIndex, Key endKey) {
|
||||
this.tx = tx;
|
||||
this.current = current;
|
||||
this.nextIndex=nextIndex;
|
||||
this.endKey = endKey;
|
||||
if (endKey != null && endKey.equals(0l)) {
|
||||
Thread.dumpStack();
|
||||
}
|
||||
}
|
||||
|
||||
synchronized private void findNextPage() {
|
||||
|
@ -107,6 +112,10 @@ public final class BTreeNode<Key,Value> {
|
|||
break;
|
||||
}
|
||||
} else {
|
||||
if (endKey != null && current.keys[nextIndex].equals(endKey)) {
|
||||
System.err.println("Stopping iterator on reaching: " + endKey);
|
||||
break;
|
||||
}
|
||||
nextEntry = new KeyValueEntry(current.keys[nextIndex], current.values[nextIndex]);
|
||||
nextIndex++;
|
||||
break;
|
||||
|
@ -631,23 +640,23 @@ public final class BTreeNode<Key,Value> {
|
|||
return node;
|
||||
}
|
||||
|
||||
public Iterator<Map.Entry<Key,Value>> iterator(final Transaction tx, Key startKey) throws IOException {
|
||||
public Iterator<Map.Entry<Key,Value>> iterator(final Transaction tx, Key startKey, Key endKey) throws IOException {
|
||||
if (startKey == null) {
|
||||
return iterator(tx);
|
||||
}
|
||||
if( isBranch() ) {
|
||||
return getLeafNode(tx, this, startKey).iterator(tx, startKey);
|
||||
return getLeafNode(tx, this, startKey).iterator(tx, startKey, endKey);
|
||||
} else {
|
||||
int idx = Arrays.binarySearch(keys, startKey);
|
||||
if (idx < 0) {
|
||||
idx = -(idx + 1);
|
||||
}
|
||||
return new BTreeIterator(tx, this, idx);
|
||||
return new BTreeIterator(tx, this, idx, endKey);
|
||||
}
|
||||
}
|
||||
|
||||
public Iterator<Map.Entry<Key,Value>> iterator(final Transaction tx) throws IOException {
|
||||
return new BTreeIterator(tx, getFirstLeafNode(tx), 0);
|
||||
return new BTreeIterator(tx, getFirstLeafNode(tx), 0, null);
|
||||
}
|
||||
|
||||
public void clear(Transaction tx) throws IOException {
|
||||
|
|
|
@ -162,6 +162,35 @@ public class BTreeIndexTest extends IndexTestSupport {
|
|||
tx.commit();
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testLimitedIteration() throws Exception {
|
||||
createPageFileAndIndex(500);
|
||||
BTreeIndex<String,Long> index = ((BTreeIndex<String,Long>)this.index);
|
||||
this.index.load(tx);
|
||||
tx.commit();
|
||||
|
||||
// Insert in reverse order..
|
||||
doInsertReverse(1000);
|
||||
|
||||
this.index.unload(tx);
|
||||
tx.commit();
|
||||
this.index.load(tx);
|
||||
tx.commit();
|
||||
|
||||
// BTree should iterate it in sorted order up to limit
|
||||
int counter=0;
|
||||
for (Iterator<Map.Entry<String,Long>> i = index.iterator(tx, key(0), key(500)); i.hasNext();) {
|
||||
Map.Entry<String,Long> entry = i.next();
|
||||
assertEquals(key(counter),entry.getKey());
|
||||
assertEquals(counter,(long)entry.getValue());
|
||||
counter++;
|
||||
}
|
||||
|
||||
assertEquals("got to 500", 500, counter);
|
||||
this.index.unload(tx);
|
||||
tx.commit();
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testVisitor() throws Exception {
|
||||
createPageFileAndIndex(100);
|
||||
|
|
|
@ -104,6 +104,7 @@ class CountDownFuture[T <: AnyRef]() extends ListenableFuture[T] {
|
|||
var value:T = _
|
||||
var error:Throwable = _
|
||||
var listener:Runnable = _
|
||||
var id:MessageId = _
|
||||
|
||||
def cancel(mayInterruptIfRunning: Boolean) = false
|
||||
def isCancelled = false
|
||||
|
@ -115,6 +116,9 @@ class CountDownFuture[T <: AnyRef]() extends ListenableFuture[T] {
|
|||
|
||||
def set(v:T) = {
|
||||
value = v
|
||||
if (id != null) {
|
||||
id.setFutureOrSequenceLong(id.getEntryLocator.asInstanceOf[EntryLocator].seq)
|
||||
}
|
||||
latch.countDown()
|
||||
fireListener
|
||||
}
|
||||
|
@ -326,6 +330,8 @@ class DelayableUOW(val manager:DBManager) extends BaseRetained {
|
|||
val entry = QueueEntryRecord(id, queueKey, queueSeq)
|
||||
assert(id.getEntryLocator == null)
|
||||
id.setEntryLocator(EntryLocator(queueKey, queueSeq))
|
||||
id.setFutureOrSequenceLong(countDownFuture)
|
||||
countDownFuture.id = id
|
||||
|
||||
val a = this.synchronized {
|
||||
if( !delay )
|
||||
|
|
|
@ -245,7 +245,7 @@ class LevelDBStore extends LockableServiceSupport with BrokerServiceAware with P
|
|||
val (msgs, acks) = db.getXAActions(transaction.xacontainer_id)
|
||||
transaction.xarecovery = (msgs, acks.map(_.ack))
|
||||
for ( msg <- msgs ) {
|
||||
transaction.add(createMessageStore(msg.getDestination), msg, false);
|
||||
transaction.add(createMessageStore(msg.getDestination), new IndexListener.MessageContext(null, msg, null), false);
|
||||
}
|
||||
for ( record <- acks ) {
|
||||
var ack = record.ack
|
||||
|
@ -348,27 +348,27 @@ class LevelDBStore extends LockableServiceSupport with BrokerServiceAware with P
|
|||
}
|
||||
}
|
||||
|
||||
def add(store:LevelDBStore#LevelDBMessageStore, message: Message, delay:Boolean) = {
|
||||
def add(store:LevelDBStore#LevelDBMessageStore, messageContext:IndexListener.MessageContext, delay:Boolean) = {
|
||||
commitActions += new TransactionAction() {
|
||||
def commit(uow:DelayableUOW) = {
|
||||
if( prepared ) {
|
||||
uow.dequeue(xacontainer_id, message.getMessageId)
|
||||
uow.dequeue(xacontainer_id, messageContext.message.getMessageId)
|
||||
}
|
||||
var copy = message.getMessageId.copy()
|
||||
var copy = messageContext.message.getMessageId.copy()
|
||||
copy.setEntryLocator(null)
|
||||
message.setMessageId(copy)
|
||||
store.doAdd(uow, message, delay)
|
||||
messageContext.message.setMessageId(copy)
|
||||
store.doAdd(uow, messageContext, delay)
|
||||
}
|
||||
|
||||
def prepare(uow:DelayableUOW) = {
|
||||
// add it to the xa container instead of the actual store container.
|
||||
uow.enqueue(xacontainer_id, xaseqcounter.incrementAndGet, message, delay)
|
||||
xarecovery._1 += message
|
||||
uow.enqueue(xacontainer_id, xaseqcounter.incrementAndGet, messageContext.message, delay)
|
||||
xarecovery._1 += messageContext.message
|
||||
}
|
||||
|
||||
def rollback(uow:DelayableUOW) = {
|
||||
if( prepared ) {
|
||||
uow.dequeue(xacontainer_id, message.getMessageId)
|
||||
uow.dequeue(xacontainer_id, messageContext.message.getMessageId)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -676,14 +676,19 @@ class LevelDBStore extends LockableServiceSupport with BrokerServiceAware with P
|
|||
|
||||
def cursorResetPosition = 0L
|
||||
|
||||
def doAdd(uow: DelayableUOW, message: Message, delay:Boolean): CountDownFuture[AnyRef] = {
|
||||
def doAdd(uow: DelayableUOW, messageContext:IndexListener.MessageContext, delay:Boolean): CountDownFuture[AnyRef] = {
|
||||
check_running
|
||||
val seq = lastSeq.incrementAndGet()
|
||||
message.incrementReferenceCount()
|
||||
messageContext.message.incrementReferenceCount()
|
||||
uow.addCompleteListener({
|
||||
message.decrementReferenceCount()
|
||||
messageContext.message.decrementReferenceCount()
|
||||
})
|
||||
uow.enqueue(key, seq, message, delay)
|
||||
val future = uow.enqueue(key, seq, messageContext.message, delay)
|
||||
messageContext.message.getMessageId.setFutureOrSequenceLong(future)
|
||||
if (indexListener != null) {
|
||||
indexListener.onAdd(messageContext)
|
||||
}
|
||||
future
|
||||
}
|
||||
|
||||
override def asyncAddQueueMessage(context: ConnectionContext, message: Message) = asyncAddQueueMessage(context, message, false)
|
||||
|
@ -691,11 +696,11 @@ class LevelDBStore extends LockableServiceSupport with BrokerServiceAware with P
|
|||
check_running
|
||||
message.getMessageId.setEntryLocator(null)
|
||||
if( message.getTransactionId!=null ) {
|
||||
transaction(message.getTransactionId).add(this, message, delay)
|
||||
transaction(message.getTransactionId).add(this, new IndexListener.MessageContext(context, message, null), delay)
|
||||
DONE
|
||||
} else {
|
||||
withUow { uow=>
|
||||
doAdd(uow, message, delay)
|
||||
doAdd(uow, new IndexListener.MessageContext(context, message, null), delay)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -78,6 +78,7 @@ public class StoreQueueCursorNoDuplicateTest extends TestCase {
|
|||
queueMessageStore, destinationStatistics, null);
|
||||
|
||||
queueMessageStore.start();
|
||||
queueMessageStore.registerIndexListener(null);
|
||||
|
||||
QueueStorePrefetch underTest = new QueueStorePrefetch(queue, brokerService.getBroker());
|
||||
SystemUsage systemUsage = new SystemUsage();
|
||||
|
|
|
@ -0,0 +1,462 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import javax.jms.JMSException;
|
||||
import javax.jms.Message;
|
||||
import javax.jms.MessageListener;
|
||||
import javax.jms.MessageProducer;
|
||||
import javax.jms.QueueConnection;
|
||||
import javax.jms.QueueReceiver;
|
||||
import javax.jms.QueueSession;
|
||||
import javax.jms.Session;
|
||||
import javax.jms.TextMessage;
|
||||
import javax.management.ObjectName;
|
||||
import org.apache.activemq.ActiveMQConnection;
|
||||
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.JmsMultipleBrokersTestSupport;
|
||||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.broker.jmx.QueueViewMBean;
|
||||
import org.apache.activemq.broker.region.RegionBroker;
|
||||
import org.apache.activemq.broker.region.policy.PolicyEntry;
|
||||
import org.apache.activemq.broker.region.policy.PolicyMap;
|
||||
import org.apache.activemq.command.ActiveMQDestination;
|
||||
import org.apache.activemq.command.ActiveMQMessage;
|
||||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.apache.activemq.command.BrokerInfo;
|
||||
import org.apache.activemq.network.DiscoveryNetworkConnector;
|
||||
import org.apache.activemq.network.NetworkConnector;
|
||||
import org.apache.activemq.store.kahadb.KahaDBPersistenceAdapter;
|
||||
import org.apache.activemq.util.TimeUtils;
|
||||
import org.apache.activemq.util.Wait;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class AMQ4485LowLimitTest extends JmsMultipleBrokersTestSupport {
|
||||
static final String payload = new String(new byte[10 * 1024]);
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AMQ4485LowLimitTest.class);
|
||||
final int portBase = 61600;
|
||||
final int numBrokers = 8;
|
||||
final int numProducers = 30;
|
||||
final int numMessages = 1000;
|
||||
final int consumerSleepTime = 40;
|
||||
StringBuilder brokersUrl = new StringBuilder();
|
||||
HashMap<ActiveMQQueue, AtomicInteger> accumulators = new HashMap<ActiveMQQueue, AtomicInteger>();
|
||||
private ArrayList<Throwable> exceptions = new ArrayList<Throwable>();
|
||||
|
||||
protected void buildUrlList() throws Exception {
|
||||
for (int i = 0; i < numBrokers; i++) {
|
||||
brokersUrl.append("tcp://localhost:" + (portBase + i));
|
||||
if (i != numBrokers - 1) {
|
||||
brokersUrl.append(',');
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected BrokerService createBroker(int brokerid) throws Exception {
|
||||
return createBroker(brokerid, true);
|
||||
}
|
||||
|
||||
protected BrokerService createBroker(int brokerid, boolean addToNetwork) throws Exception {
|
||||
|
||||
BrokerService broker = new BrokerService();
|
||||
broker.setPersistent(true);
|
||||
broker.setDeleteAllMessagesOnStartup(true);
|
||||
broker.getManagementContext().setCreateConnector(false);
|
||||
|
||||
|
||||
broker.setUseJmx(true);
|
||||
broker.setBrokerName("B" + brokerid);
|
||||
broker.addConnector(new URI("tcp://localhost:" + (portBase + brokerid)));
|
||||
|
||||
if (addToNetwork) {
|
||||
addNetworkConnector(broker);
|
||||
}
|
||||
broker.setSchedulePeriodForDestinationPurge(0);
|
||||
//broker.getSystemUsage().setSendFailIfNoSpace(true);
|
||||
broker.getSystemUsage().getMemoryUsage().setLimit(256 * 1024 * 1024l);
|
||||
|
||||
|
||||
PolicyMap policyMap = new PolicyMap();
|
||||
PolicyEntry policyEntry = new PolicyEntry();
|
||||
policyEntry.setExpireMessagesPeriod(0);
|
||||
policyEntry.setQueuePrefetch(1000);
|
||||
policyEntry.setMemoryLimit(2 * 1024 * 1024l);
|
||||
policyEntry.setProducerFlowControl(false);
|
||||
policyEntry.setEnableAudit(true);
|
||||
policyEntry.setUseCache(true);
|
||||
policyMap.put(new ActiveMQQueue("GW.>"), policyEntry);
|
||||
|
||||
PolicyEntry inPolicyEntry = new PolicyEntry();
|
||||
inPolicyEntry.setExpireMessagesPeriod(0);
|
||||
inPolicyEntry.setQueuePrefetch(1000);
|
||||
inPolicyEntry.setMemoryLimit(5 * 1024 * 1024l);
|
||||
inPolicyEntry.setProducerFlowControl(true);
|
||||
inPolicyEntry.setEnableAudit(true);
|
||||
inPolicyEntry.setUseCache(true);
|
||||
policyMap.put(new ActiveMQQueue("IN"), inPolicyEntry);
|
||||
|
||||
broker.setDestinationPolicy(policyMap);
|
||||
|
||||
KahaDBPersistenceAdapter kahaDBPersistenceAdapter = (KahaDBPersistenceAdapter) broker.getPersistenceAdapter();
|
||||
kahaDBPersistenceAdapter.setConcurrentStoreAndDispatchQueues(true);
|
||||
|
||||
brokers.put(broker.getBrokerName(), new BrokerItem(broker));
|
||||
return broker;
|
||||
}
|
||||
|
||||
private void addNetworkConnector(BrokerService broker) throws Exception {
|
||||
StringBuilder networkConnectorUrl = new StringBuilder("static:(").append(brokersUrl.toString());
|
||||
networkConnectorUrl.append(')');
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
NetworkConnector nc = new DiscoveryNetworkConnector(new URI(networkConnectorUrl.toString()));
|
||||
nc.setName("Bridge-" + i);
|
||||
nc.setNetworkTTL(1);
|
||||
nc.setDecreaseNetworkConsumerPriority(true);
|
||||
nc.setDynamicOnly(true);
|
||||
nc.setPrefetchSize(100);
|
||||
nc.setDynamicallyIncludedDestinations(
|
||||
Arrays.asList(new ActiveMQDestination[]{new ActiveMQQueue("GW.*")}));
|
||||
broker.addNetworkConnector(nc);
|
||||
}
|
||||
}
|
||||
|
||||
// used to explore contention with concurrentStoreandDispatch - sync commit and task queue reversing
|
||||
// order of cursor add and sequence assignment
|
||||
public void x_testInterleavedSend() throws Exception {
|
||||
|
||||
BrokerService b = createBroker(0, false);
|
||||
b.start();
|
||||
|
||||
ActiveMQConnectionFactory connectionFactory = new ActiveMQConnectionFactory("tcp://localhost:" + (portBase + 0));
|
||||
connectionFactory.setWatchTopicAdvisories(false);
|
||||
|
||||
QueueConnection c1 = connectionFactory.createQueueConnection();
|
||||
QueueConnection c2 = connectionFactory.createQueueConnection();
|
||||
QueueConnection c3 = connectionFactory.createQueueConnection();
|
||||
|
||||
c1.start();
|
||||
c2.start();
|
||||
c3.start();
|
||||
|
||||
ActiveMQQueue dest = new ActiveMQQueue("IN");
|
||||
final Session s1 = c1.createQueueSession(true, Session.SESSION_TRANSACTED);
|
||||
final TextMessage txMessage = s1.createTextMessage("TX");
|
||||
final TextMessage noTxMessage = s1.createTextMessage("NO_TX");
|
||||
|
||||
final MessageProducer txProducer = s1.createProducer(dest);
|
||||
final MessageProducer nonTxProducer = c2.createQueueSession(false, Session.AUTO_ACKNOWLEDGE).createProducer(dest);
|
||||
|
||||
txProducer.send(txMessage);
|
||||
|
||||
ExecutorService executorService = Executors.newFixedThreadPool(2);
|
||||
executorService.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
s1.commit();
|
||||
} catch (JMSException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
executorService.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
nonTxProducer.send(noTxMessage);
|
||||
} catch (JMSException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
executorService.shutdown();
|
||||
executorService.awaitTermination(10, TimeUnit.MINUTES);
|
||||
|
||||
}
|
||||
|
||||
public void testBrokers() throws Exception {
|
||||
|
||||
buildUrlList();
|
||||
|
||||
for (int i = 0; i < numBrokers; i++) {
|
||||
createBroker(i);
|
||||
}
|
||||
|
||||
startAllBrokers();
|
||||
waitForBridgeFormation(numBrokers - 1);
|
||||
|
||||
verifyPeerBrokerInfos(numBrokers - 1);
|
||||
|
||||
|
||||
final List<ConsumerState> consumerStates = startAllGWConsumers(numBrokers);
|
||||
|
||||
startAllGWFanoutConsumers(numBrokers);
|
||||
|
||||
LOG.info("Waiting for percolation of consumers..");
|
||||
TimeUnit.SECONDS.sleep(5);
|
||||
|
||||
LOG.info("Produce mesages..");
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
// produce
|
||||
produce(numMessages);
|
||||
|
||||
assertTrue("Got all sent", Wait.waitFor(new Wait.Condition() {
|
||||
@Override
|
||||
public boolean isSatisified() throws Exception {
|
||||
for (ConsumerState tally : consumerStates) {
|
||||
final int expected = numMessages * (tally.destination.isComposite() ? tally.destination.getCompositeDestinations().length : 1);
|
||||
LOG.info("Tally for: " + tally.brokerName + ", dest: " + tally.destination + " - " + tally.accumulator.get());
|
||||
if (tally.accumulator.get() != expected) {
|
||||
LOG.info("Tally for: " + tally.brokerName + ", dest: " + tally.destination + " - " + tally.accumulator.get() + " != " + expected + ", " + tally.expected);
|
||||
if (tally.accumulator.get() > expected - 50) {
|
||||
dumpQueueStat(tally.destination);
|
||||
}
|
||||
if (tally.expected.size() == 1) {
|
||||
startConsumer(tally.brokerName, tally.destination);
|
||||
};
|
||||
return false;
|
||||
}
|
||||
LOG.info("got tally on " + tally.brokerName);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}, 1000 * 60 * 1000l, 20*1000));
|
||||
|
||||
assertTrue("No exceptions:" + exceptions, exceptions.isEmpty());
|
||||
|
||||
LOG.info("done");
|
||||
long duration = System.currentTimeMillis() - startTime;
|
||||
LOG.info("Duration:" + TimeUtils.printDuration(duration));
|
||||
}
|
||||
|
||||
private void startConsumer(String brokerName, ActiveMQDestination destination) throws Exception {
|
||||
int id = Integer.parseInt(brokerName.substring(1));
|
||||
ActiveMQConnectionFactory connectionFactory = new ActiveMQConnectionFactory("tcp://localhost:" + (portBase + id));
|
||||
connectionFactory.setWatchTopicAdvisories(false);
|
||||
QueueConnection queueConnection = connectionFactory.createQueueConnection();
|
||||
queueConnection.start();
|
||||
|
||||
queueConnection.createQueueSession(false, Session.AUTO_ACKNOWLEDGE).createConsumer(destination);
|
||||
queueConnection.close();
|
||||
}
|
||||
|
||||
private void dumpQueueStat(ActiveMQDestination destination) throws Exception {
|
||||
Collection<BrokerItem> brokerList = brokers.values();
|
||||
for (Iterator<BrokerItem> i = brokerList.iterator(); i.hasNext(); ) {
|
||||
BrokerService brokerService = i.next().broker;
|
||||
for (ObjectName objectName : brokerService.getAdminView().getQueues()) {
|
||||
//if (objectName.toString().contains(destination.getQualifiedName())) {
|
||||
QueueViewMBean qViewMBean = (QueueViewMBean) brokerService.getManagementContext().newProxyInstance(objectName, QueueViewMBean.class, false);
|
||||
LOG.info(brokerService.getBrokerName() + ", " + qViewMBean.getName() + " Size: " + qViewMBean.getEnqueueCount());
|
||||
//}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void startAllGWFanoutConsumers(int nBrokers) throws Exception {
|
||||
|
||||
StringBuffer compositeDest = new StringBuffer();
|
||||
for (int k = 0; k < nBrokers; k++) {
|
||||
compositeDest.append("GW." + k);
|
||||
if (k + 1 != nBrokers) {
|
||||
compositeDest.append(',');
|
||||
}
|
||||
}
|
||||
ActiveMQQueue compositeQ = new ActiveMQQueue(compositeDest.toString());
|
||||
|
||||
for (int id = 0; id < nBrokers; id++) {
|
||||
ActiveMQConnectionFactory connectionFactory = new ActiveMQConnectionFactory("failover:(tcp://localhost:" + (portBase + id) + ")");
|
||||
connectionFactory.setWatchTopicAdvisories(false);
|
||||
|
||||
QueueConnection queueConnection = connectionFactory.createQueueConnection();
|
||||
queueConnection.start();
|
||||
|
||||
final QueueSession queueSession = queueConnection.createQueueSession(true, Session.SESSION_TRANSACTED);
|
||||
|
||||
final MessageProducer producer = queueSession.createProducer(compositeQ);
|
||||
queueSession.createReceiver(new ActiveMQQueue("IN")).setMessageListener(new MessageListener() {
|
||||
@Override
|
||||
public void onMessage(Message message) {
|
||||
try {
|
||||
producer.send(message);
|
||||
queueSession.commit();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to fanout to GW: " + message, e);
|
||||
}
|
||||
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private List<ConsumerState> startAllGWConsumers(int nBrokers) throws Exception {
|
||||
List<ConsumerState> consumerStates = new LinkedList<ConsumerState>();
|
||||
for (int id = 0; id < nBrokers; id++) {
|
||||
ActiveMQConnectionFactory connectionFactory = new ActiveMQConnectionFactory("failover:(tcp://localhost:" + (portBase + id) + ")");
|
||||
connectionFactory.setWatchTopicAdvisories(false);
|
||||
|
||||
QueueConnection queueConnection = connectionFactory.createQueueConnection();
|
||||
queueConnection.start();
|
||||
|
||||
final QueueSession queueSession = queueConnection.createQueueSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||
|
||||
ActiveMQQueue destination = new ActiveMQQueue("GW." + id);
|
||||
QueueReceiver queueReceiver = queueSession.createReceiver(destination);
|
||||
|
||||
final ConsumerState consumerState = new ConsumerState();
|
||||
consumerState.brokerName = ((ActiveMQConnection) queueConnection).getBrokerName();
|
||||
consumerState.receiver = queueReceiver;
|
||||
consumerState.destination = destination;
|
||||
for (int j = 0; j < numMessages * (consumerState.destination.isComposite() ? consumerState.destination.getCompositeDestinations().length : 1); j++) {
|
||||
consumerState.expected.add(j);
|
||||
}
|
||||
|
||||
if (!accumulators.containsKey(destination)) {
|
||||
accumulators.put(destination, new AtomicInteger(0));
|
||||
}
|
||||
consumerState.accumulator = accumulators.get(destination);
|
||||
|
||||
queueReceiver.setMessageListener(new MessageListener() {
|
||||
@Override
|
||||
public void onMessage(Message message) {
|
||||
try {
|
||||
if (consumerSleepTime > 0) {
|
||||
TimeUnit.MILLISECONDS.sleep(consumerSleepTime);
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
try {
|
||||
consumerState.accumulator.incrementAndGet();
|
||||
try {
|
||||
consumerState.expected.remove(((ActiveMQMessage) message).getProperty("NUM"));
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
//queueSession.commit();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to commit slow receipt of " + message, e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
consumerStates.add(consumerState);
|
||||
|
||||
}
|
||||
return consumerStates;
|
||||
}
|
||||
|
||||
private void produce(final int numMessages) throws Exception {
|
||||
ExecutorService executorService = Executors.newFixedThreadPool(numProducers);
|
||||
final AtomicInteger toSend = new AtomicInteger(numMessages);
|
||||
for (int i = 1; i <= numProducers; i++) {
|
||||
final int id = i % numBrokers;
|
||||
executorService.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
ActiveMQConnectionFactory connectionFactory = new ActiveMQConnectionFactory("failover:(tcp://localhost:" + (portBase + id) + ")");
|
||||
connectionFactory.setWatchTopicAdvisories(false);
|
||||
QueueConnection queueConnection = connectionFactory.createQueueConnection();
|
||||
queueConnection.start();
|
||||
QueueSession queueSession = queueConnection.createQueueSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||
MessageProducer producer = queueSession.createProducer(null);
|
||||
int val = 0;
|
||||
while ((val = toSend.decrementAndGet()) >= 0) {
|
||||
|
||||
int id = numMessages - val - 1;
|
||||
|
||||
ActiveMQQueue compositeQ = new ActiveMQQueue("IN");
|
||||
//LOG.info("Send to: " + ((ActiveMQConnection) queueConnection).getBrokerName() + ", " + val + ", dest:" + compositeQ);
|
||||
Message textMessage = queueSession.createTextMessage(((ActiveMQConnection) queueConnection).getBrokerName() + "->" + id + " payload:" + payload);
|
||||
textMessage.setIntProperty("NUM", id);
|
||||
producer.send(compositeQ, textMessage);
|
||||
}
|
||||
queueConnection.close();
|
||||
|
||||
} catch (Throwable throwable) {
|
||||
throwable.printStackTrace();
|
||||
exceptions.add(throwable);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyPeerBrokerInfo(BrokerItem brokerItem, final int max) throws Exception {
|
||||
final BrokerService broker = brokerItem.broker;
|
||||
final RegionBroker regionBroker = (RegionBroker) broker.getRegionBroker();
|
||||
Wait.waitFor(new Wait.Condition() {
|
||||
@Override
|
||||
public boolean isSatisified() throws Exception {
|
||||
LOG.info("verify infos " + broker.getBrokerName() + ", len: " + regionBroker.getPeerBrokerInfos().length);
|
||||
return max == regionBroker.getPeerBrokerInfos().length;
|
||||
}
|
||||
});
|
||||
LOG.info("verify infos " + broker.getBrokerName() + ", len: " + regionBroker.getPeerBrokerInfos().length);
|
||||
List<String> missing = new ArrayList<String>();
|
||||
for (int i = 0; i < max; i++) {
|
||||
missing.add("B" + i);
|
||||
}
|
||||
if (max != regionBroker.getPeerBrokerInfos().length) {
|
||||
for (BrokerInfo info : regionBroker.getPeerBrokerInfos()) {
|
||||
LOG.info(info.getBrokerName());
|
||||
missing.remove(info.getBrokerName());
|
||||
}
|
||||
LOG.info("Broker infos off.." + missing);
|
||||
}
|
||||
assertEquals(broker.getBrokerName(), max, regionBroker.getPeerBrokerInfos().length);
|
||||
}
|
||||
|
||||
private void verifyPeerBrokerInfos(final int max) throws Exception {
|
||||
Collection<BrokerItem> brokerList = brokers.values();
|
||||
for (Iterator<BrokerItem> i = brokerList.iterator(); i.hasNext(); ) {
|
||||
verifyPeerBrokerInfo(i.next(), max);
|
||||
}
|
||||
}
|
||||
|
||||
protected void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
class ConsumerState {
|
||||
AtomicInteger accumulator;
|
||||
String brokerName;
|
||||
QueueReceiver receiver;
|
||||
ActiveMQDestination destination;
|
||||
ConcurrentLinkedQueue<Integer> expected = new ConcurrentLinkedQueue<Integer>();
|
||||
}
|
||||
}
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
package org.apache.activemq.bugs;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -31,18 +32,31 @@ import org.apache.activemq.ActiveMQSession;
|
|||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.apache.activemq.command.ActiveMQTextMessage;
|
||||
import org.apache.activemq.store.kahadb.KahaDBPersistenceAdapter;
|
||||
import org.apache.activemq.util.Wait;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
@RunWith(value = Parameterized.class)
|
||||
public class AMQ5212Test {
|
||||
|
||||
BrokerService brokerService;
|
||||
|
||||
@Parameterized.Parameter(0)
|
||||
public boolean concurrentStoreAndDispatchQ = true;
|
||||
|
||||
@Parameterized.Parameters(name = "concurrentStoreAndDispatch={0}")
|
||||
public static Iterable<Object[]> getTestParameters() {
|
||||
return Arrays.asList(new Object[][]{{Boolean.TRUE}, {Boolean.FALSE}});
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
start(true);
|
||||
|
@ -53,6 +67,7 @@ public class AMQ5212Test {
|
|||
if (deleteAllMessages) {
|
||||
brokerService.deleteAllMessages();
|
||||
}
|
||||
((KahaDBPersistenceAdapter)brokerService.getPersistenceAdapter()).setConcurrentStoreAndDispatchQueues(concurrentStoreAndDispatchQ);
|
||||
brokerService.addConnector("tcp://localhost:0");
|
||||
brokerService.setAdvisorySupport(false);
|
||||
brokerService.start();
|
||||
|
@ -118,6 +133,12 @@ public class AMQ5212Test {
|
|||
executorService.shutdown();
|
||||
executorService.awaitTermination(5, TimeUnit.MINUTES);
|
||||
|
||||
Wait.waitFor(new Wait.Condition() {
|
||||
@Override
|
||||
public boolean isSatisified() throws Exception {
|
||||
return expectedTotalEnqueue == brokerService.getAdminView().getTotalEnqueueCount();
|
||||
}
|
||||
});
|
||||
assertEquals("total enqueue as expected", expectedTotalEnqueue, brokerService.getAdminView().getTotalEnqueueCount());
|
||||
}
|
||||
|
||||
|
|
|
@ -17,13 +17,14 @@
|
|||
package org.apache.activemq.bugs;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import javax.jms.Message;
|
||||
import javax.jms.MessageConsumer;
|
||||
import javax.jms.MessageProducer;
|
||||
|
@ -41,10 +42,13 @@ import org.apache.activemq.broker.region.policy.PolicyMap;
|
|||
import org.apache.activemq.command.ActiveMQQueue;
|
||||
import org.apache.activemq.store.jdbc.JDBCPersistenceAdapter;
|
||||
import org.apache.activemq.store.jdbc.adapter.DefaultJDBCAdapter;
|
||||
import org.apache.activemq.store.kahadb.KahaDBPersistenceAdapter;
|
||||
import org.apache.derby.jdbc.EmbeddedDataSource;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -56,12 +60,54 @@ import static org.junit.Assert.assertEquals;
|
|||
* <p/>
|
||||
* Will kick of publisher and consumer simultaneously, and will usually result in stuck messages on the queue.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class AMQ5266Test {
|
||||
static Logger LOG = LoggerFactory.getLogger(AMQ5266Test.class);
|
||||
String activemqURL = "tcp://localhost:61617";
|
||||
BrokerService brokerService;
|
||||
private EmbeddedDataSource dataSource;
|
||||
|
||||
public int messageSize = 1000;
|
||||
|
||||
@Parameterized.Parameter(0)
|
||||
public int publisherMessagesPerThread = 1000;
|
||||
|
||||
@Parameterized.Parameter(1)
|
||||
public int publisherThreadCount = 20;
|
||||
|
||||
@Parameterized.Parameter(2)
|
||||
public int consumerThreadsPerQueue = 5;
|
||||
|
||||
@Parameterized.Parameter(3)
|
||||
public int destMemoryLimit = 50 * 1024;
|
||||
|
||||
@Parameterized.Parameter(4)
|
||||
public boolean useCache = true;
|
||||
|
||||
@Parameterized.Parameter(5)
|
||||
public boolean useDefaultStore = false;
|
||||
|
||||
@Parameterized.Parameters(name="#{0},producerThreads:{1},consumerThreads:{2},mL:{3},useCache:{4},useDefaultStore:{5}")
|
||||
public static Iterable<Object[]> parameters() {
|
||||
return Arrays.asList(new Object[][]{
|
||||
// jdbc
|
||||
{1000, 20, 5, 50*1024, true, false},
|
||||
{100, 20, 5, 50*1024, false, false},
|
||||
{1000, 5, 20, 50*1024, true, false},
|
||||
{1000, 20, 20, 1024*1024, true, false},
|
||||
{1000, 100, 100, 1024*1024, true, false},
|
||||
|
||||
// default store
|
||||
{1000, 20, 5, 50*1024, true, true},
|
||||
{100, 20, 5, 50*1024, false, true},
|
||||
{1000, 5, 20, 50*1024, true, true},
|
||||
{1000, 20, 20, 1024*1024, true, true},
|
||||
{1000, 100, 100, 1024*1024, true, true}
|
||||
});
|
||||
}
|
||||
|
||||
public int consumerBatchSize = 5;
|
||||
|
||||
@Before
|
||||
public void startBroker() throws Exception {
|
||||
brokerService = new BrokerService();
|
||||
|
@ -70,19 +116,28 @@ public class AMQ5266Test {
|
|||
dataSource.setDatabaseName("target/derbyDb");
|
||||
dataSource.setCreateDatabase("create");
|
||||
|
||||
JDBCPersistenceAdapter persistenceAdapter = new JDBCPersistenceAdapter();
|
||||
persistenceAdapter.setDataSource(dataSource);
|
||||
brokerService.setPersistenceAdapter(persistenceAdapter);
|
||||
JDBCPersistenceAdapter jdbcPersistenceAdapter = new JDBCPersistenceAdapter();
|
||||
jdbcPersistenceAdapter.setDataSource(dataSource);
|
||||
jdbcPersistenceAdapter.setUseLock(false);
|
||||
|
||||
if (!useDefaultStore) {
|
||||
brokerService.setPersistenceAdapter(jdbcPersistenceAdapter);
|
||||
} else {
|
||||
KahaDBPersistenceAdapter kahaDBPersistenceAdapter = (KahaDBPersistenceAdapter) brokerService.getPersistenceAdapter();
|
||||
kahaDBPersistenceAdapter.setConcurrentStoreAndDispatchQueues(true);
|
||||
}
|
||||
brokerService.setDeleteAllMessagesOnStartup(true);
|
||||
|
||||
|
||||
PolicyMap policyMap = new PolicyMap();
|
||||
PolicyEntry defaultEntry = new PolicyEntry();
|
||||
defaultEntry.setEnableAudit(false);
|
||||
defaultEntry.setUseCache(false);
|
||||
defaultEntry.setUseConsumerPriority(false); // java.lang.IllegalArgumentException: Comparison method violates its general contract!
|
||||
defaultEntry.setMaxAuditDepth(publisherThreadCount);
|
||||
defaultEntry.setEnableAudit(true);
|
||||
defaultEntry.setUseCache(useCache);
|
||||
defaultEntry.setMaxPageSize(1000);
|
||||
defaultEntry.setOptimizedDispatch(false);
|
||||
defaultEntry.setMemoryLimit(1024 * 1024);
|
||||
defaultEntry.setMemoryLimit(destMemoryLimit);
|
||||
defaultEntry.setExpireMessagesPeriod(0);
|
||||
policyMap.setDefaultEntry(defaultEntry);
|
||||
brokerService.setDestinationPolicy(policyMap);
|
||||
|
@ -110,11 +165,6 @@ public class AMQ5266Test {
|
|||
|
||||
String activemqQueues = "activemq,activemq2";//,activemq3,activemq4,activemq5,activemq6,activemq7,activemq8,activemq9";
|
||||
|
||||
int publisherMessagesPerThread = 1000;
|
||||
int publisherThreadCount = 5;
|
||||
|
||||
int consumerThreadsPerQueue = 5;
|
||||
int consumerBatchSize = 25;
|
||||
int consumerWaitForConsumption = 5 * 60 * 1000;
|
||||
|
||||
ExportQueuePublisher publisher = null;
|
||||
|
@ -145,28 +195,36 @@ public class AMQ5266Test {
|
|||
|
||||
publisher.waitForCompletion();
|
||||
|
||||
distinctPublishedCount = publisher.getIDs().size();
|
||||
List publishedIds = publisher.getIDs();
|
||||
distinctPublishedCount = new TreeSet(publishedIds).size();
|
||||
|
||||
LOG.info("Publisher Complete. Distinct IDs Published: " + distinctPublishedCount);
|
||||
LOG.info("Publisher Complete. Published: " + publishedIds.size() + ", Distinct IDs Published: " + distinctPublishedCount);
|
||||
|
||||
|
||||
long endWait = System.currentTimeMillis() + consumerWaitForConsumption;
|
||||
|
||||
|
||||
while (!consumer.completed() && System.currentTimeMillis() < endWait) {
|
||||
try {
|
||||
int secs = (int) (endWait - System.currentTimeMillis()) / 1000;
|
||||
LOG.info("Waiting For Consumer Completion. Time left: " + secs + " secs");
|
||||
if (!useDefaultStore) {
|
||||
DefaultJDBCAdapter.dumpTables(dataSource.getConnection());
|
||||
}
|
||||
Thread.sleep(10000);
|
||||
} catch (Exception e) {
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("\nConsumer Complete. Shutting Down.");
|
||||
LOG.info("\nConsumer Complete: " + consumer.completed() +", Shutting Down.");
|
||||
|
||||
consumer.shutdown();
|
||||
|
||||
TimeUnit.SECONDS.sleep(2);
|
||||
LOG.info("DB Contents START");
|
||||
if (!useDefaultStore) {
|
||||
DefaultJDBCAdapter.dumpTables(dataSource.getConnection());
|
||||
}
|
||||
LOG.info("DB Contents END");
|
||||
|
||||
LOG.info("Consumer Stats:");
|
||||
|
||||
for (Map.Entry<String, List<String>> entry : consumer.getIDs().entrySet()) {
|
||||
|
@ -199,7 +257,8 @@ public class AMQ5266Test {
|
|||
// Collection of distinct IDs that the publisher has published.
|
||||
// After a message is published, its UUID will be written to this list for tracking.
|
||||
// This list of IDs (or distinct count) will be used to compare to the consumed list of IDs.
|
||||
private Set<String> ids = Collections.synchronizedSet(new TreeSet<String>());
|
||||
//private Set<String> ids = Collections.synchronizedSet(new TreeSet<String>());
|
||||
private List<String> ids = Collections.synchronizedList(new ArrayList<String>());
|
||||
private List<PublisherThread> threads;
|
||||
|
||||
public ExportQueuePublisher(String activemqURL, String activemqQueues, int messagesPerThread, int threadCount) throws Exception {
|
||||
|
@ -216,7 +275,7 @@ public class AMQ5266Test {
|
|||
}
|
||||
}
|
||||
|
||||
public Set<String> getIDs() {
|
||||
public List<String> getIDs() {
|
||||
return ids;
|
||||
}
|
||||
|
||||
|
@ -241,7 +300,7 @@ public class AMQ5266Test {
|
|||
return queueConnection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
}
|
||||
|
||||
private QueueConnection newQueueConnection() throws Exception {
|
||||
private synchronized QueueConnection newQueueConnection() throws Exception {
|
||||
|
||||
if (connectionFactory == null) {
|
||||
connectionFactory = new ActiveMQConnectionFactory(amqUser, amqPassword, activemqURL);
|
||||
|
@ -285,7 +344,7 @@ public class AMQ5266Test {
|
|||
// Loop until we've published enough messages
|
||||
while (count-- > 0) {
|
||||
|
||||
TextMessage tm = session.createTextMessage("test");
|
||||
TextMessage tm = session.createTextMessage(getMessageText());
|
||||
String id = UUID.randomUUID().toString();
|
||||
tm.setStringProperty("KEY", id);
|
||||
ids.add(id); // keep track of the key to compare against consumer
|
||||
|
@ -320,6 +379,28 @@ public class AMQ5266Test {
|
|||
|
||||
}
|
||||
|
||||
String messageText;
|
||||
private String getMessageText() {
|
||||
|
||||
if (messageText == null) {
|
||||
|
||||
synchronized (this) {
|
||||
|
||||
if (messageText == null) {
|
||||
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (int i = 0; i < messageSize; i++) {
|
||||
sb.append("X");
|
||||
}
|
||||
messageText = sb.toString();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return messageText;
|
||||
}
|
||||
|
||||
|
||||
public class ExportQueueConsumer {
|
||||
|
||||
private final String amqUser = ActiveMQConnection.DEFAULT_USER;
|
||||
|
@ -404,7 +485,7 @@ public class AMQ5266Test {
|
|||
return queueConnection.createSession(true, Session.SESSION_TRANSACTED);
|
||||
}
|
||||
|
||||
private QueueConnection newQueueConnection() throws Exception {
|
||||
private synchronized QueueConnection newQueueConnection() throws Exception {
|
||||
|
||||
if (connectionFactory == null) {
|
||||
connectionFactory = new ActiveMQConnectionFactory(amqUser, amqPassword, activemqURL);
|
||||
|
@ -452,7 +533,7 @@ public class AMQ5266Test {
|
|||
qName = queueName;
|
||||
qc = newQueueConnection();
|
||||
session = newSession(qc);
|
||||
Queue q = session.createQueue(queueName);
|
||||
Queue q = session.createQueue(queueName + "?consumer.prefetchSize=" + batchSize);
|
||||
mc = session.createConsumer(q);
|
||||
|
||||
idList = idsByQueue.get(queueName);
|
||||
|
@ -469,6 +550,7 @@ public class AMQ5266Test {
|
|||
|
||||
if (idList.size() >= totalToExpect) {
|
||||
LOG.info("Got {} for q: {}", +idList.size(), qName);
|
||||
session.commit();
|
||||
break;
|
||||
}
|
||||
Message m = mc.receive(4000);
|
||||
|
@ -497,7 +579,9 @@ public class AMQ5266Test {
|
|||
// Sleep a little before trying to read after not getting a message
|
||||
|
||||
try {
|
||||
if (idList.size() < totalToExpect) {
|
||||
LOG.info("did not receive on {}, current count: {}", qName, idList.size());
|
||||
}
|
||||
//sleep(3000);
|
||||
} catch (Exception e) {
|
||||
}
|
||||
|
|
|
@ -106,7 +106,6 @@ public class DemandForwardingBridgeTest extends NetworkTestSupport {
|
|||
}));
|
||||
|
||||
assertEquals("broker dest stat dequeues", 1, destinationStatistics.getDequeues().getCount());
|
||||
assertEquals("remote broker dest stat dequeues", 1, remoteBroker.getDestination(destination).getDestinationStatistics().getDequeues().getCount());
|
||||
}
|
||||
|
||||
public void initCombosForTestAddConsumerThenSend() {
|
||||
|
|
|
@ -123,7 +123,7 @@ public class MemoryLimitTest extends TestSupport {
|
|||
|
||||
// consume one message
|
||||
MessageConsumer consumer = sess.createConsumer(queue);
|
||||
Message msg = consumer.receive();
|
||||
Message msg = consumer.receive(5000);
|
||||
msg.acknowledge();
|
||||
|
||||
// this should free some space and allow us to get new batch of messages in the memory
|
||||
|
@ -132,12 +132,12 @@ public class MemoryLimitTest extends TestSupport {
|
|||
@Override
|
||||
public boolean isSatisified() throws Exception {
|
||||
LOG.info("Destination usage: " + dest.getMemoryUsage());
|
||||
return dest.getMemoryUsage().getPercentUsage() >= 478;
|
||||
return dest.getMemoryUsage().getPercentUsage() >= 470;
|
||||
}
|
||||
}));
|
||||
|
||||
LOG.info("Broker usage: " + broker.getSystemUsage().getMemoryUsage());
|
||||
assertTrue(broker.getSystemUsage().getMemoryUsage().getPercentUsage() >= 478);
|
||||
assertTrue(broker.getSystemUsage().getMemoryUsage().getPercentUsage() >= 470);
|
||||
|
||||
// let's make sure we can consume all messages
|
||||
for (int i = 1; i < 2000; i++) {
|
||||
|
|
Loading…
Reference in New Issue