mirror of https://github.com/apache/activemq.git
Updated Rapid Persistence Adaptor to do batching for cursors
git-svn-id: https://svn.apache.org/repos/asf/incubator/activemq/trunk@477567 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ecb50234ec
commit
878b9645ce
|
@ -91,7 +91,7 @@ public class KahaMessageStore implements MessageStore, UsageListener{
|
|||
Message msg=(Message)messageContainer.get(entry);
|
||||
if(msg.getMessageId().equals(identity)){
|
||||
result=msg;
|
||||
cache.put(identity,msg);
|
||||
cache.put(identity,entry);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -186,7 +186,7 @@ public class KahaMessageStore implements MessageStore, UsageListener{
|
|||
* @throws Exception
|
||||
* @see org.apache.activemq.store.MessageStore#recoverNextMessages(org.apache.activemq.command.MessageId, int, org.apache.activemq.store.MessageRecoveryListener)
|
||||
*/
|
||||
public void recoverNextMessages(int maxReturned,MessageRecoveryListener listener) throws Exception{
|
||||
public synchronized void recoverNextMessages(int maxReturned,MessageRecoveryListener listener) throws Exception{
|
||||
StoreEntry entry = batchEntry;
|
||||
if (entry == null) {
|
||||
entry= messageContainer.getFirst();
|
||||
|
@ -242,7 +242,6 @@ public class KahaMessageStore implements MessageStore, UsageListener{
|
|||
if(newPercentUsage==100){
|
||||
cache.clear();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.activemq.kaha.StoreEntry;
|
|||
import org.apache.activemq.memory.UsageManager;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.apache.activemq.store.TopicMessageStore;
|
||||
import org.apache.activemq.store.rapid.RapidMessageReference;
|
||||
|
||||
/**
|
||||
* @version $Revision: 1.5 $
|
||||
|
@ -149,11 +150,9 @@ public class KahaTopicMessageStore extends KahaMessageStore implements TopicMess
|
|||
listener.recoverMessage((Message)msg);
|
||||
}
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
}else{
|
||||
listener.finished();
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
|
||||
public void recoverNextMessages(String clientId,String subscriptionName,int maxReturned,
|
||||
|
@ -236,31 +235,7 @@ public class KahaTopicMessageStore extends KahaMessageStore implements TopicMess
|
|||
messageContainer.add(messageRef);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the destination
|
||||
* @see org.apache.activemq.store.MessageStore#getDestination()
|
||||
*/
|
||||
public ActiveMQDestination getDestination(){
|
||||
return destination;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param identity
|
||||
* @return the Message
|
||||
* @throws IOException
|
||||
* @see org.apache.activemq.store.MessageStore#getMessage(org.apache.activemq.command.MessageId)
|
||||
*/
|
||||
public Message getMessage(MessageId identity) throws IOException{
|
||||
Message result=null;
|
||||
for(Iterator i=messageContainer.iterator();i.hasNext();){
|
||||
Message msg=(Message)i.next();
|
||||
if(msg.getMessageId().equals(identity)){
|
||||
result=msg;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param identity
|
||||
|
@ -272,21 +247,6 @@ public class KahaTopicMessageStore extends KahaMessageStore implements TopicMess
|
|||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @throws Exception
|
||||
* @see org.apache.activemq.store.MessageStore#recover(org.apache.activemq.store.MessageRecoveryListener)
|
||||
*/
|
||||
public void recover(MessageRecoveryListener listener) throws Exception{
|
||||
for(Iterator iter=messageContainer.iterator();iter.hasNext();){
|
||||
Object msg=iter.next();
|
||||
if(msg.getClass()==String.class){
|
||||
listener.recoverMessageReference((String)msg);
|
||||
}else{
|
||||
listener.recoverMessage((Message)msg);
|
||||
}
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param context
|
||||
|
@ -302,22 +262,6 @@ public class KahaTopicMessageStore extends KahaMessageStore implements TopicMess
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param context
|
||||
* @param ack
|
||||
* @throws IOException
|
||||
* @see org.apache.activemq.store.MessageStore#removeMessage(org.apache.activemq.broker.ConnectionContext,
|
||||
* org.apache.activemq.command.MessageAck)
|
||||
*/
|
||||
public void removeMessage(ConnectionContext context,MessageAck ack) throws IOException{
|
||||
for(Iterator i=messageContainer.iterator();i.hasNext();){
|
||||
Message msg=(Message)i.next();
|
||||
if(msg.getMessageId().equals(ack.getLastMessageId())){
|
||||
i.remove();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void resetBatching(String clientId,String subscriptionName){
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
|
@ -326,25 +270,4 @@ public class KahaTopicMessageStore extends KahaMessageStore implements TopicMess
|
|||
topicSubContainer.reset();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public MessageId getNextMessageIdToDeliver(String clientId,String subscriptionName) throws IOException{
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clientId
|
||||
* @param subscriptionName
|
||||
* @param id
|
||||
* @return previous messageId
|
||||
* @throws IOException
|
||||
* @see org.apache.activemq.store.TopicMessageStore#getPreviousMessageIdToDeliver(java.lang.String,
|
||||
* java.lang.String, org.apache.activemq.command.MessageId)
|
||||
*/
|
||||
public MessageId getPreviousMessageIdToDeliver(String clientId,String subscriptionName,MessageId id)
|
||||
throws IOException{
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,43 +22,43 @@ import org.apache.activemq.kaha.StoreEntry;
|
|||
*
|
||||
* @version $Revision: 1.10 $
|
||||
*/
|
||||
class TopicSubContainer{
|
||||
public class TopicSubContainer{
|
||||
|
||||
private ListContainer listContainer;
|
||||
private StoreEntry batchEntry;
|
||||
|
||||
TopicSubContainer(ListContainer container){
|
||||
public TopicSubContainer(ListContainer container){
|
||||
this.listContainer = container;
|
||||
}
|
||||
/**
|
||||
* @return the batchEntry
|
||||
*/
|
||||
StoreEntry getBatchEntry(){
|
||||
public StoreEntry getBatchEntry(){
|
||||
return this.batchEntry;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param batchEntry the batchEntry to set
|
||||
*/
|
||||
void setBatchEntry(StoreEntry batchEntry){
|
||||
public void setBatchEntry(StoreEntry batchEntry){
|
||||
this.batchEntry=batchEntry;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the listContainer
|
||||
*/
|
||||
ListContainer getListContainer(){
|
||||
public ListContainer getListContainer(){
|
||||
return this.listContainer;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param listContainer the listContainer to set
|
||||
*/
|
||||
void setListContainer(ListContainer container){
|
||||
public void setListContainer(ListContainer container){
|
||||
this.listContainer=container;
|
||||
}
|
||||
|
||||
void reset() {
|
||||
public void reset() {
|
||||
batchEntry = null;
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,10 @@ public class RapidMessageReference {
|
|||
public final MessageId messageId;
|
||||
public final Location location;
|
||||
|
||||
public RapidMessageReference(MessageId messageId, Location location) {
|
||||
this.messageId = messageId;
|
||||
this.location=location;
|
||||
}
|
||||
public RapidMessageReference(Message message, Location location) {
|
||||
this.messageId = message.getMessageId();
|
||||
this.location=location;
|
||||
|
|
|
@ -0,0 +1,46 @@
|
|||
/**
|
||||
*
|
||||
* 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.rapid;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import org.apache.activeio.journal.active.Location;
|
||||
import org.apache.activemq.command.MessageId;
|
||||
import org.apache.activemq.kaha.Marshaller;
|
||||
|
||||
public class RapidMessageReferenceMarshaller implements Marshaller{
|
||||
|
||||
|
||||
|
||||
public Object readPayload(DataInput dataIn) throws IOException{
|
||||
MessageId mid = new MessageId(dataIn.readUTF());
|
||||
Location loc = new Location(dataIn.readInt(),dataIn.readInt());
|
||||
RapidMessageReference rmr = new RapidMessageReference(mid,loc);
|
||||
return rmr;
|
||||
}
|
||||
|
||||
public void writePayload(Object object,DataOutput dataOut) throws IOException{
|
||||
RapidMessageReference rmr = (RapidMessageReference)object;
|
||||
dataOut.writeUTF(rmr.getMessageId().toString());
|
||||
dataOut.writeInt(rmr.getLocation().getLogFileId());
|
||||
dataOut.writeInt(rmr.getLocation().getLogFileOffset());
|
||||
|
||||
}
|
||||
}
|
|
@ -30,11 +30,15 @@ import org.apache.activemq.command.JournalQueueAck;
|
|||
import org.apache.activemq.command.Message;
|
||||
import org.apache.activemq.command.MessageAck;
|
||||
import org.apache.activemq.command.MessageId;
|
||||
import org.apache.activemq.kaha.ListContainer;
|
||||
import org.apache.activemq.kaha.MapContainer;
|
||||
import org.apache.activemq.kaha.StoreEntry;
|
||||
import org.apache.activemq.memory.UsageListener;
|
||||
import org.apache.activemq.memory.UsageManager;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.apache.activemq.store.MessageStore;
|
||||
import org.apache.activemq.transaction.Synchronization;
|
||||
import org.apache.activemq.util.LRUCache;
|
||||
import org.apache.activemq.util.TransactionTemplate;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -44,31 +48,42 @@ import org.apache.commons.logging.LogFactory;
|
|||
*
|
||||
* @version $Revision: 1.14 $
|
||||
*/
|
||||
public class RapidMessageStore implements MessageStore {
|
||||
public class RapidMessageStore implements MessageStore, UsageListener {
|
||||
|
||||
private static final Log log = LogFactory.getLog(RapidMessageStore.class);
|
||||
|
||||
protected final RapidPersistenceAdapter peristenceAdapter;
|
||||
protected final RapidTransactionStore transactionStore;
|
||||
protected final MapContainer messageContainer;
|
||||
protected final ListContainer messageContainer;
|
||||
protected final ActiveMQDestination destination;
|
||||
protected final TransactionTemplate transactionTemplate;
|
||||
protected final LRUCache cache;
|
||||
protected UsageManager usageManager;
|
||||
protected StoreEntry batchEntry = null;
|
||||
|
||||
// private LinkedHashMap messages = new LinkedHashMap();
|
||||
// private ArrayList messageAcks = new ArrayList();
|
||||
|
||||
// /** A MessageStore that we can use to retrieve messages quickly. */
|
||||
// private LinkedHashMap cpAddedMessageIds;
|
||||
|
||||
protected Location lastLocation;
|
||||
protected HashSet inFlightTxLocations = new HashSet();
|
||||
|
||||
public RapidMessageStore(RapidPersistenceAdapter adapter, ActiveMQDestination destination, MapContainer container) {
|
||||
public RapidMessageStore(RapidPersistenceAdapter adapter, ActiveMQDestination destination, ListContainer container, int maximumCacheSize) {
|
||||
this.peristenceAdapter = adapter;
|
||||
this.transactionStore = adapter.getTransactionStore();
|
||||
this.messageContainer = container;
|
||||
this.destination = destination;
|
||||
this.transactionTemplate = new TransactionTemplate(adapter, new ConnectionContext());
|
||||
this.cache=new LRUCache(maximumCacheSize,maximumCacheSize,0.75f,false);
|
||||
// populate the cache
|
||||
StoreEntry entry=messageContainer.getFirst();
|
||||
int count = 0;
|
||||
if(entry!=null){
|
||||
do{
|
||||
RapidMessageReference msg = (RapidMessageReference)messageContainer.get(entry);
|
||||
cache.put(msg.getMessageId(),entry);
|
||||
entry = messageContainer.getNext(entry);
|
||||
count++;
|
||||
}while(entry!=null && count < maximumCacheSize);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -76,7 +91,7 @@ public class RapidMessageStore implements MessageStore {
|
|||
* Not synchronized since the Journal has better throughput if you increase
|
||||
* the number of concurrent writes that it is doing.
|
||||
*/
|
||||
public void addMessage(ConnectionContext context, final Message message) throws IOException {
|
||||
public synchronized void addMessage(ConnectionContext context, final Message message) throws IOException {
|
||||
|
||||
final MessageId id = message.getMessageId();
|
||||
|
||||
|
@ -118,12 +133,9 @@ public class RapidMessageStore implements MessageStore {
|
|||
}
|
||||
}
|
||||
|
||||
private void addMessage(final RapidMessageReference messageReference) {
|
||||
synchronized (this) {
|
||||
lastLocation = messageReference.getLocation();
|
||||
MessageId id = messageReference.getMessageId();
|
||||
messageContainer.put(id.toString(), messageReference);
|
||||
}
|
||||
private synchronized void addMessage(final RapidMessageReference messageReference){
|
||||
StoreEntry item=messageContainer.placeLast(messageReference);
|
||||
cache.put(messageReference.getMessageId(),item);
|
||||
}
|
||||
|
||||
static protected String toString(Location location) {
|
||||
|
@ -141,7 +153,7 @@ public class RapidMessageStore implements MessageStore {
|
|||
public void replayAddMessage(ConnectionContext context, Message message, Location location) {
|
||||
try {
|
||||
RapidMessageReference messageReference = new RapidMessageReference(message, location);
|
||||
messageContainer.put(message.getMessageId().toString(), messageReference);
|
||||
addMessage(messageReference);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
log.warn("Could not replay add for message '" + message.getMessageId() + "'. Message may have already been added. reason: " + e);
|
||||
|
@ -160,7 +172,7 @@ public class RapidMessageStore implements MessageStore {
|
|||
if( !context.isInTransaction() ) {
|
||||
if( debug )
|
||||
log.debug("Journalled message remove for: "+ack.getLastMessageId()+", at: "+location);
|
||||
removeMessage(ack, location);
|
||||
removeMessage(ack.getLastMessageId());
|
||||
} else {
|
||||
if( debug )
|
||||
log.debug("Journalled transacted message remove for: "+ack.getLastMessageId()+", at: "+location);
|
||||
|
@ -174,7 +186,7 @@ public class RapidMessageStore implements MessageStore {
|
|||
log.debug("Transacted message remove commit for: "+ack.getLastMessageId()+", at: "+location);
|
||||
synchronized( RapidMessageStore.this ) {
|
||||
inFlightTxLocations.remove(location);
|
||||
removeMessage(ack, location);
|
||||
removeMessage(ack.getLastMessageId());
|
||||
}
|
||||
}
|
||||
public void afterRollback() throws Exception {
|
||||
|
@ -189,32 +201,53 @@ public class RapidMessageStore implements MessageStore {
|
|||
}
|
||||
}
|
||||
|
||||
private void removeMessage(final MessageAck ack, final Location location) {
|
||||
synchronized (this) {
|
||||
lastLocation = location;
|
||||
MessageId id = ack.getLastMessageId();
|
||||
messageContainer.remove(id.toString());
|
||||
|
||||
public synchronized void removeMessage(MessageId msgId) throws IOException{
|
||||
StoreEntry entry=(StoreEntry)cache.remove(msgId);
|
||||
if(entry!=null){
|
||||
entry = messageContainer.refresh(entry);
|
||||
messageContainer.remove(entry);
|
||||
}else{
|
||||
for (entry = messageContainer.getFirst();entry != null; entry = messageContainer.getNext(entry)) {
|
||||
RapidMessageReference msg=(RapidMessageReference)messageContainer.get(entry);
|
||||
if(msg.getMessageId().equals(msgId)){
|
||||
messageContainer.remove(entry);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void replayRemoveMessage(ConnectionContext context, MessageAck ack) {
|
||||
try {
|
||||
MessageId id = ack.getLastMessageId();
|
||||
messageContainer.remove(id.toString());
|
||||
removeMessage(id);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
log.warn("Could not replay acknowledge for message '" + ack.getLastMessageId() + "'. Message may have already been acknowledged. reason: " + e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public Message getMessage(MessageId id) throws IOException {
|
||||
RapidMessageReference messageReference = (RapidMessageReference) messageContainer.get(id.toString());
|
||||
if (messageReference == null )
|
||||
|
||||
public synchronized Message getMessage(MessageId identity) throws IOException{
|
||||
RapidMessageReference result=null;
|
||||
StoreEntry entry=(StoreEntry)cache.get(identity);
|
||||
if(entry!=null){
|
||||
entry = messageContainer.refresh(entry);
|
||||
result = (RapidMessageReference)messageContainer.get(entry);
|
||||
}else{
|
||||
for (entry = messageContainer.getFirst();entry != null; entry = messageContainer.getNext(entry)) {
|
||||
RapidMessageReference msg=(RapidMessageReference)messageContainer.get(entry);
|
||||
if(msg.getMessageId().equals(identity)){
|
||||
result=msg;
|
||||
cache.put(identity,entry);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (result == null )
|
||||
return null;
|
||||
return (Message) peristenceAdapter.readCommand(messageReference.getLocation());
|
||||
return (Message) peristenceAdapter.readCommand(result.getLocation());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -225,28 +258,32 @@ public class RapidMessageStore implements MessageStore {
|
|||
* @param listener
|
||||
* @throws Exception
|
||||
*/
|
||||
public void recover(final MessageRecoveryListener listener) throws Exception {
|
||||
|
||||
for(Iterator iter=messageContainer.values().iterator();iter.hasNext();){
|
||||
public synchronized void recover(MessageRecoveryListener listener) throws Exception{
|
||||
for(Iterator iter=messageContainer.iterator();iter.hasNext();){
|
||||
RapidMessageReference messageReference=(RapidMessageReference) iter.next();
|
||||
Message m = (Message) peristenceAdapter.readCommand(messageReference.getLocation());
|
||||
listener.recoverMessage(m);
|
||||
}
|
||||
listener.finished();
|
||||
|
||||
}
|
||||
|
||||
public void start() throws Exception {
|
||||
public void start() {
|
||||
if( this.usageManager != null )
|
||||
this.usageManager.addUsageListener(this);
|
||||
}
|
||||
|
||||
public void stop() throws Exception {
|
||||
public void stop() {
|
||||
if( this.usageManager != null )
|
||||
this.usageManager.removeUsageListener(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* @see org.apache.activemq.store.MessageStore#removeAllMessages(ConnectionContext)
|
||||
*/
|
||||
public void removeAllMessages(ConnectionContext context) throws IOException {
|
||||
public synchronized void removeAllMessages(ConnectionContext context) throws IOException {
|
||||
messageContainer.clear();
|
||||
cache.clear();
|
||||
}
|
||||
|
||||
public ActiveMQDestination getDestination() {
|
||||
|
@ -254,15 +291,16 @@ public class RapidMessageStore implements MessageStore {
|
|||
}
|
||||
|
||||
public void addMessageReference(ConnectionContext context, MessageId messageId, long expirationTime, String messageRef) throws IOException {
|
||||
throw new IOException("The journal does not support message references.");
|
||||
throw new IOException("Does not support message references.");
|
||||
}
|
||||
|
||||
public String getMessageReference(MessageId identity) throws IOException {
|
||||
throw new IOException("The journal does not support message references.");
|
||||
throw new IOException("Does not support message references.");
|
||||
}
|
||||
|
||||
|
||||
public void setUsageManager(UsageManager usageManager) {
|
||||
this.usageManager = usageManager;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -289,13 +327,50 @@ public class RapidMessageStore implements MessageStore {
|
|||
|
||||
|
||||
public int getMessageCount(){
|
||||
return 0;
|
||||
return messageContainer.size();
|
||||
}
|
||||
|
||||
public void recoverNextMessages(int maxReturned,MessageRecoveryListener listener) throws Exception{
|
||||
public synchronized void recoverNextMessages(int maxReturned,MessageRecoveryListener listener) throws Exception{
|
||||
StoreEntry entry=batchEntry;
|
||||
if(entry==null){
|
||||
entry=messageContainer.getFirst();
|
||||
}else{
|
||||
entry=messageContainer.refresh(entry);
|
||||
entry=messageContainer.getNext(entry);
|
||||
}
|
||||
if(entry!=null){
|
||||
int count=0;
|
||||
do{
|
||||
RapidMessageReference messageReference=(RapidMessageReference)messageContainer.get(entry);
|
||||
Message msg=(Message)peristenceAdapter.readCommand(messageReference.getLocation());
|
||||
if(msg!=null){
|
||||
Message message=(Message)msg;
|
||||
listener.recoverMessage(message);
|
||||
count++;
|
||||
}
|
||||
batchEntry=entry;
|
||||
entry=messageContainer.getNext(entry);
|
||||
}while(entry!=null&&count<maxReturned);
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
|
||||
public void resetBatching(){
|
||||
batchEntry = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the store supports cursors
|
||||
*/
|
||||
public boolean isSupportForCursors() {
|
||||
return true;
|
||||
}
|
||||
|
||||
public synchronized void onMemoryUseChanged(UsageManager memoryManager,int oldPercentUsage,int newPercentUsage){
|
||||
if (newPercentUsage == 100) {
|
||||
cache.clear();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -1,20 +1,17 @@
|
|||
/**
|
||||
*
|
||||
* 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
|
||||
* 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.
|
||||
* 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.rapid;
|
||||
|
||||
import java.io.File;
|
||||
|
@ -32,7 +29,6 @@ import java.util.concurrent.ThreadFactory;
|
|||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.activeio.journal.InvalidRecordLocationException;
|
||||
import org.apache.activeio.journal.Journal;
|
||||
import org.apache.activeio.journal.JournalEventListener;
|
||||
|
@ -52,6 +48,7 @@ import org.apache.activemq.command.JournalTrace;
|
|||
import org.apache.activemq.command.JournalTransaction;
|
||||
import org.apache.activemq.command.Message;
|
||||
import org.apache.activemq.command.MessageAck;
|
||||
import org.apache.activemq.kaha.ListContainer;
|
||||
import org.apache.activemq.kaha.MapContainer;
|
||||
import org.apache.activemq.kaha.Store;
|
||||
import org.apache.activemq.kaha.StoreFactory;
|
||||
|
@ -65,6 +62,8 @@ import org.apache.activemq.store.TopicMessageStore;
|
|||
import org.apache.activemq.store.TransactionStore;
|
||||
import org.apache.activemq.store.kahadaptor.AtomicIntegerMarshaller;
|
||||
import org.apache.activemq.store.kahadaptor.CommandMarshaller;
|
||||
import org.apache.activemq.store.kahadaptor.KahaTopicMessageStore;
|
||||
import org.apache.activemq.store.kahadaptor.TopicSubAckMarshaller;
|
||||
import org.apache.activemq.store.rapid.RapidTransactionStore.Tx;
|
||||
import org.apache.activemq.store.rapid.RapidTransactionStore.TxOperation;
|
||||
import org.apache.activemq.thread.Scheduler;
|
||||
|
@ -78,9 +77,8 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
/**
|
||||
* An implementation of {@link PersistenceAdapter} designed for use with a
|
||||
* {@link Journal} and then check pointing asynchronously on a timeout with some
|
||||
* other long term persistent storage.
|
||||
* An implementation of {@link PersistenceAdapter} designed for use with a {@link Journal} and then check pointing
|
||||
* asynchronously on a timeout with some other long term persistent storage.
|
||||
*
|
||||
* @org.apache.xbean.XBean
|
||||
*
|
||||
|
@ -90,34 +88,27 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
|
||||
private static final Log log=LogFactory.getLog(RapidPersistenceAdapter.class);
|
||||
private final Journal journal;
|
||||
|
||||
private final WireFormat wireFormat=new OpenWireFormat();
|
||||
|
||||
private final ConcurrentHashMap queues=new ConcurrentHashMap();
|
||||
private final ConcurrentHashMap topics=new ConcurrentHashMap();
|
||||
|
||||
private long checkpointInterval=1000*60*5;
|
||||
private long lastCheckpointRequest=System.currentTimeMillis();
|
||||
private int maxCheckpointWorkers=10;
|
||||
private int maxCheckpointMessageAddSize=5000;
|
||||
|
||||
private RapidTransactionStore transactionStore=new RapidTransactionStore(this);
|
||||
private ThreadPoolExecutor checkpointExecutor;
|
||||
|
||||
private TaskRunner checkpointTask;
|
||||
private CountDownLatch nextCheckpointCountDownLatch=new CountDownLatch(1);
|
||||
private boolean fullCheckPoint;
|
||||
|
||||
private AtomicBoolean started=new AtomicBoolean(false);
|
||||
|
||||
Store store;
|
||||
private boolean useExternalMessageReferences;
|
||||
|
||||
|
||||
private final Runnable periodicCheckpointTask=createPeriodicCheckpointTask();
|
||||
private int maximumDestinationCacheSize=2000;
|
||||
|
||||
final Runnable createPeriodicCheckpointTask(){
|
||||
return new Runnable(){
|
||||
|
||||
public void run(){
|
||||
if(System.currentTimeMillis()>lastCheckpointRequest+checkpointInterval){
|
||||
checkpoint(false,true);
|
||||
|
@ -127,20 +118,17 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
}
|
||||
|
||||
public RapidPersistenceAdapter(Journal journal,TaskRunnerFactory taskRunnerFactory) throws IOException{
|
||||
|
||||
this.journal=journal;
|
||||
journal.setJournalEventListener(this);
|
||||
|
||||
File dir=((JournalImpl)journal).getLogDirectory();
|
||||
String name=dir.getAbsolutePath()+File.separator+"kaha.db";
|
||||
store=StoreFactory.open(name,"rw");
|
||||
|
||||
checkpointTask=taskRunnerFactory.createTaskRunner(new Task(){
|
||||
|
||||
public boolean iterate(){
|
||||
return doCheckpoint();
|
||||
}
|
||||
},"ActiveMQ Checkpoint Worker");
|
||||
|
||||
}
|
||||
|
||||
public Set getDestinations(){
|
||||
|
@ -161,8 +149,7 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
private MessageStore createMessageStore(ActiveMQDestination destination) throws IOException{
|
||||
if(destination.isQueue()){
|
||||
return createQueueMessageStore((ActiveMQQueue)destination);
|
||||
}
|
||||
else {
|
||||
}else{
|
||||
return createTopicMessageStore((ActiveMQTopic)destination);
|
||||
}
|
||||
}
|
||||
|
@ -170,8 +157,8 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
public MessageStore createQueueMessageStore(ActiveMQQueue destination) throws IOException{
|
||||
RapidMessageStore store=(RapidMessageStore)queues.get(destination);
|
||||
if(store==null){
|
||||
MapContainer messageContainer=getMapContainer(destination,"topic-data");
|
||||
store = new RapidMessageStore(this, destination, messageContainer);
|
||||
ListContainer messageContainer=getListContainer(destination,"topic-data");
|
||||
store=new RapidMessageStore(this,destination,messageContainer,maximumDestinationCacheSize);
|
||||
queues.put(destination,store);
|
||||
}
|
||||
return store;
|
||||
|
@ -189,21 +176,28 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
return container;
|
||||
}
|
||||
|
||||
public TopicMessageStore createTopicMessageStore(ActiveMQTopic destination) throws IOException {
|
||||
RapidTopicMessageStore store = (RapidTopicMessageStore) topics.get(destination);
|
||||
if (store == null) {
|
||||
|
||||
MapContainer messageContainer=getMapContainer(destination,"topic-data");
|
||||
MapContainer subsContainer=getMapContainer(destination.toString()+"-subscriptions","topic-subs");
|
||||
MapContainer ackContainer=this.store.getMapContainer(destination.toString(),"topic-acks");
|
||||
|
||||
ackContainer.setKeyMarshaller(new StringMarshaller());
|
||||
ackContainer.setValueMarshaller(new AtomicIntegerMarshaller());
|
||||
|
||||
store = new RapidTopicMessageStore(this, destination, messageContainer, subsContainer, ackContainer);
|
||||
topics.put(destination, store);
|
||||
protected ListContainer getListContainer(Object id,String containerName) throws IOException{
|
||||
Store store=getStore();
|
||||
ListContainer container=store.getListContainer(id,containerName);
|
||||
container.setMaximumCacheSize(0);
|
||||
container.setMarshaller(new RapidMessageReferenceMarshaller());
|
||||
container.load();
|
||||
return container;
|
||||
}
|
||||
return store;
|
||||
|
||||
public TopicMessageStore createTopicMessageStore(ActiveMQTopic destination) throws IOException{
|
||||
TopicMessageStore rc=(TopicMessageStore)topics.get(destination);
|
||||
if(rc==null){
|
||||
Store store=getStore();
|
||||
ListContainer messageContainer=getListContainer(destination,"topic-data");
|
||||
MapContainer subsContainer=getMapContainer(destination.toString()+"-Subscriptions","topic-subs");
|
||||
ListContainer ackContainer=store.getListContainer(destination.toString(),"topic-acks");
|
||||
ackContainer.setMarshaller(new TopicSubAckMarshaller());
|
||||
rc=new RapidTopicMessageStore(this,store,messageContainer,ackContainer,subsContainer,destination,
|
||||
maximumDestinationCacheSize);
|
||||
topics.put(destination,rc);
|
||||
}
|
||||
return rc;
|
||||
}
|
||||
|
||||
public TransactionStore createTransactionStore() throws IOException{
|
||||
|
@ -227,8 +221,9 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
public synchronized void start() throws Exception{
|
||||
if(!started.compareAndSet(false,true))
|
||||
return;
|
||||
checkpointExecutor=new ThreadPoolExecutor(maxCheckpointWorkers,maxCheckpointWorkers,30,TimeUnit.SECONDS,
|
||||
new LinkedBlockingQueue(),new ThreadFactory(){
|
||||
|
||||
checkpointExecutor = new ThreadPoolExecutor(maxCheckpointWorkers, maxCheckpointWorkers, 30, TimeUnit.SECONDS, new LinkedBlockingQueue(), new ThreadFactory() {
|
||||
public Thread newThread(Runnable runable){
|
||||
Thread t=new Thread(runable,"Journal checkpoint worker");
|
||||
t.setPriority(7);
|
||||
|
@ -236,30 +231,22 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
}
|
||||
});
|
||||
// checkpointExecutor.allowCoreThreadTimeOut(true);
|
||||
|
||||
createTransactionStore();
|
||||
recover();
|
||||
|
||||
// Do a checkpoint periodically.
|
||||
Scheduler.executePeriodically(periodicCheckpointTask,checkpointInterval/10);
|
||||
|
||||
}
|
||||
|
||||
public void stop() throws Exception{
|
||||
|
||||
if(!started.compareAndSet(true,false))
|
||||
return;
|
||||
|
||||
Scheduler.cancel(periodicCheckpointTask);
|
||||
|
||||
// Take one final checkpoint and stop checkpoint processing.
|
||||
checkpoint(false,true);
|
||||
checkpointTask.shutdown();
|
||||
checkpointExecutor.shutdown();
|
||||
|
||||
queues.clear();
|
||||
topics.clear();
|
||||
|
||||
IOException firstException=null;
|
||||
try{
|
||||
journal.close();
|
||||
|
@ -267,7 +254,6 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
firstException=IOExceptionSupport.create("Failed to close journals: "+e,e);
|
||||
}
|
||||
store.close();
|
||||
|
||||
if(firstException!=null){
|
||||
throw firstException;
|
||||
}
|
||||
|
@ -275,7 +261,6 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
|
||||
// Properties
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* @return Returns the wireFormat.
|
||||
*/
|
||||
|
@ -285,10 +270,9 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
|
||||
// Implementation methods
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* The Journal give us a call back so that we can move old data out of the
|
||||
* journal. Taking a checkpoint does this for us.
|
||||
* The Journal give us a call back so that we can move old data out of the journal. Taking a checkpoint does this
|
||||
* for us.
|
||||
*
|
||||
* @see org.apache.activemq.journal.JournalEventListener#overflowNotification(org.apache.activemq.journal.RecordLocation)
|
||||
*/
|
||||
|
@ -298,6 +282,7 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
|
||||
/**
|
||||
* When we checkpoint we move all the journalled data to long term storage.
|
||||
*
|
||||
* @param stopping
|
||||
*
|
||||
* @param b
|
||||
|
@ -306,7 +291,6 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
try{
|
||||
if(journal==null)
|
||||
throw new IllegalStateException("Journal is closed.");
|
||||
|
||||
long now=System.currentTimeMillis();
|
||||
CountDownLatch latch=null;
|
||||
synchronized(this){
|
||||
|
@ -316,21 +300,19 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
this.fullCheckPoint=true;
|
||||
}
|
||||
}
|
||||
|
||||
checkpointTask.wakeup();
|
||||
|
||||
if(sync){
|
||||
log.debug("Waking for checkpoint to complete.");
|
||||
latch.await();
|
||||
}
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
}catch(InterruptedException e){
|
||||
log.warn("Request to start checkpoint failed: "+e,e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This does the actual checkpoint.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public boolean doCheckpoint(){
|
||||
|
@ -343,12 +325,9 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
this.fullCheckPoint=false;
|
||||
}
|
||||
try{
|
||||
|
||||
log.debug("Checkpoint started.");
|
||||
RecordLocation newMark=null;
|
||||
|
||||
ArrayList futureTasks=new ArrayList(queues.size()+topics.size());
|
||||
|
||||
//
|
||||
// We do many partial checkpoints (fullCheckpoint==false) to move topic messages
|
||||
// to long term store as soon as possible.
|
||||
|
@ -363,34 +342,34 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
try{
|
||||
final RapidMessageStore ms=(RapidMessageStore)iterator.next();
|
||||
FutureTask task=new FutureTask(new Callable(){
|
||||
|
||||
public Object call() throws Exception{
|
||||
return ms.checkpoint();
|
||||
}});
|
||||
}
|
||||
});
|
||||
futureTasks.add(task);
|
||||
checkpointExecutor.execute(task);
|
||||
}
|
||||
catch (Exception e) {
|
||||
}catch(Exception e){
|
||||
log.error("Failed to checkpoint a message store: "+e,e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Iterator iterator=topics.values().iterator();
|
||||
while(iterator.hasNext()){
|
||||
try{
|
||||
final RapidTopicMessageStore ms=(RapidTopicMessageStore)iterator.next();
|
||||
FutureTask task=new FutureTask(new Callable(){
|
||||
|
||||
public Object call() throws Exception{
|
||||
return ms.checkpoint();
|
||||
}});
|
||||
}
|
||||
});
|
||||
futureTasks.add(task);
|
||||
checkpointExecutor.execute(task);
|
||||
}
|
||||
catch (Exception e) {
|
||||
}catch(Exception e){
|
||||
log.error("Failed to checkpoint a message store: "+e,e);
|
||||
}
|
||||
}
|
||||
|
||||
try{
|
||||
for(Iterator iter=futureTasks.iterator();iter.hasNext();){
|
||||
FutureTask ft=(FutureTask)iter.next();
|
||||
|
@ -405,21 +384,16 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
}catch(Throwable e){
|
||||
log.error("Failed to checkpoint a message store: "+e,e);
|
||||
}
|
||||
|
||||
|
||||
if(fullCheckpoint){
|
||||
try{
|
||||
if(newMark!=null){
|
||||
log.debug("Marking journal at: "+newMark);
|
||||
journal.setMark(newMark,true);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
}catch(Exception e){
|
||||
log.error("Failed to mark the Journal: "+e,e);
|
||||
}
|
||||
|
||||
// TODO: do we need to implement a periodic clean up?
|
||||
|
||||
// if (longTermPersistence instanceof JDBCPersistenceAdapter) {
|
||||
// // We may be check pointing more often than the checkpointInterval if under high use
|
||||
// // But we don't want to clean up the db that often.
|
||||
|
@ -430,16 +404,13 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
// }
|
||||
// }
|
||||
}
|
||||
|
||||
log.debug("Checkpoint done.");
|
||||
}
|
||||
finally {
|
||||
}finally{
|
||||
latch.countDown();
|
||||
}
|
||||
synchronized(this){
|
||||
return this.fullCheckPoint;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -451,18 +422,15 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
try{
|
||||
Packet data=journal.read(location);
|
||||
return (DataStructure)wireFormat.unmarshal(toByteSequence(data));
|
||||
}
|
||||
catch (InvalidRecordLocationException e) {
|
||||
}catch(InvalidRecordLocationException e){
|
||||
throw createReadException(location,e);
|
||||
}
|
||||
catch (IOException e) {
|
||||
}catch(IOException e){
|
||||
throw createReadException(location,e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Move all the messages that were in the journal into long term storage. We
|
||||
* just replay and do a checkpoint.
|
||||
* Move all the messages that were in the journal into long term storage. We just replay and do a checkpoint.
|
||||
*
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
|
@ -470,58 +438,49 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
* @throws IllegalStateException
|
||||
*/
|
||||
private void recover() throws IllegalStateException,InvalidRecordLocationException,IOException,IOException{
|
||||
|
||||
Location pos=null;
|
||||
int transactionCounter=0;
|
||||
|
||||
log.info("Journal Recovery Started.");
|
||||
ConnectionContext context=new ConnectionContext();
|
||||
|
||||
// While we have records in the journal.
|
||||
while((pos=(Location)journal.getNextRecordLocation(pos))!=null){
|
||||
Packet data=journal.read(pos);
|
||||
DataStructure c=(DataStructure)wireFormat.unmarshal(toByteSequence(data));
|
||||
|
||||
if(c instanceof Message){
|
||||
Message message=(Message)c;
|
||||
RapidMessageStore store=(RapidMessageStore)createMessageStore(message.getDestination());
|
||||
if(message.isInTransaction()){
|
||||
transactionStore.addMessage(store,message,pos);
|
||||
}
|
||||
else {
|
||||
}else{
|
||||
store.replayAddMessage(context,message,pos);
|
||||
transactionCounter++;
|
||||
}
|
||||
}else{
|
||||
switch(c.getDataStructureType()){
|
||||
case JournalQueueAck.DATA_STRUCTURE_TYPE:
|
||||
{
|
||||
case JournalQueueAck.DATA_STRUCTURE_TYPE: {
|
||||
JournalQueueAck command=(JournalQueueAck)c;
|
||||
RapidMessageStore store=(RapidMessageStore)createMessageStore(command.getDestination());
|
||||
if(command.getMessageAck().isInTransaction()){
|
||||
transactionStore.removeMessage(store,command.getMessageAck(),pos);
|
||||
}
|
||||
else {
|
||||
}else{
|
||||
store.replayRemoveMessage(context,command.getMessageAck());
|
||||
transactionCounter++;
|
||||
}
|
||||
}
|
||||
break;
|
||||
case JournalTopicAck.DATA_STRUCTURE_TYPE:
|
||||
{
|
||||
case JournalTopicAck.DATA_STRUCTURE_TYPE: {
|
||||
JournalTopicAck command=(JournalTopicAck)c;
|
||||
RapidTopicMessageStore store=(RapidTopicMessageStore)createMessageStore(command.getDestination());
|
||||
if(command.getTransactionId()!=null){
|
||||
transactionStore.acknowledge(store,command,pos);
|
||||
}
|
||||
else {
|
||||
store.replayAcknowledge(context, command.getClientId(), command.getSubscritionName(), command.getMessageId());
|
||||
}else{
|
||||
store.replayAcknowledge(context,command.getClientId(),command.getSubscritionName(),command
|
||||
.getMessageId());
|
||||
transactionCounter++;
|
||||
}
|
||||
}
|
||||
break;
|
||||
case JournalTransaction.DATA_STRUCTURE_TYPE:
|
||||
{
|
||||
case JournalTransaction.DATA_STRUCTURE_TYPE: {
|
||||
JournalTransaction command=(JournalTransaction)c;
|
||||
try{
|
||||
// Try to replay the packet.
|
||||
|
@ -535,7 +494,6 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
if(tx==null)
|
||||
break; // We may be trying to replay a commit that
|
||||
// was already committed.
|
||||
|
||||
// Replay the committed operations.
|
||||
for(Iterator iter=tx.getOperations().iterator();iter.hasNext();){
|
||||
TxOperation op=(TxOperation)iter.next();
|
||||
|
@ -547,8 +505,8 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
}
|
||||
if(op.operationType==TxOperation.ACK_OPERATION_TYPE){
|
||||
JournalTopicAck ack=(JournalTopicAck)op.data;
|
||||
((RapidTopicMessageStore) op.store).replayAcknowledge(context, ack.getClientId(), ack.getSubscritionName(), ack
|
||||
.getMessageId());
|
||||
((RapidTopicMessageStore)op.store).replayAcknowledge(context,ack.getClientId(),ack
|
||||
.getSubscritionName(),ack.getMessageId());
|
||||
}
|
||||
}
|
||||
transactionCounter++;
|
||||
|
@ -558,8 +516,7 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
transactionStore.replayRollback(command.getTransactionId());
|
||||
break;
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
}catch(IOException e){
|
||||
log.error("Recovery Failure: Could not replay: "+c+", reason: "+e,e);
|
||||
}
|
||||
}
|
||||
|
@ -573,10 +530,8 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
RecordLocation location=writeTraceMessage("RECOVERED",true);
|
||||
journal.setMark(location,true);
|
||||
|
||||
log.info("Journal Recovered: "+transactionCounter+" message(s) in transactions recovered.");
|
||||
}
|
||||
|
||||
|
@ -637,11 +592,14 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
}catch(Throwable e){
|
||||
throw IOExceptionSupport.create(e);
|
||||
}
|
||||
|
||||
if(store!=null){
|
||||
if(store.isInitialized()){
|
||||
store.clear();
|
||||
}else{
|
||||
store.delete();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public int getMaxCheckpointMessageAddSize(){
|
||||
return maxCheckpointMessageAddSize;
|
||||
|
@ -675,13 +633,27 @@ public class RapidPersistenceAdapter implements PersistenceAdapter, JournalEvent
|
|||
return store;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the maximumDestinationCacheSize
|
||||
*/
|
||||
public int getMaximumDestinationCacheSize(){
|
||||
return this.maximumDestinationCacheSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param maximumDestinationCacheSize the maximumDestinationCacheSize to set
|
||||
*/
|
||||
public void setMaximumDestinationCacheSize(int maximumDestinationCacheSize){
|
||||
this.maximumDestinationCacheSize=maximumDestinationCacheSize;
|
||||
}
|
||||
|
||||
public Packet toPacket(ByteSequence sequence){
|
||||
return new ByteArrayPacket(new org.apache.activeio.packet.ByteSequence(sequence.data, sequence.offset, sequence.length));
|
||||
return new ByteArrayPacket(new org.apache.activeio.packet.ByteSequence(sequence.data,sequence.offset,
|
||||
sequence.length));
|
||||
}
|
||||
|
||||
public ByteSequence toByteSequence(Packet packet){
|
||||
org.apache.activeio.packet.ByteSequence sequence=packet.asByteSequence();
|
||||
return new ByteSequence(sequence.getData(),sequence.getOffset(),sequence.getLength());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -15,32 +15,27 @@
|
|||
package org.apache.activemq.store.rapid;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.activeio.journal.active.Location;
|
||||
import org.apache.activemq.broker.ConnectionContext;
|
||||
import org.apache.activemq.command.ActiveMQTopic;
|
||||
import org.apache.activemq.command.JournalTopicAck;
|
||||
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.command.SubscriptionInfo;
|
||||
import org.apache.activemq.kaha.ListContainer;
|
||||
import org.apache.activemq.kaha.MapContainer;
|
||||
import org.apache.activemq.kaha.Marshaller;
|
||||
import org.apache.activemq.kaha.Store;
|
||||
import org.apache.activemq.kaha.StringMarshaller;
|
||||
import org.apache.activemq.kaha.StoreEntry;
|
||||
import org.apache.activemq.store.MessageRecoveryListener;
|
||||
import org.apache.activemq.store.TopicMessageStore;
|
||||
import org.apache.activemq.transaction.Synchronization;
|
||||
import org.apache.activemq.util.SubscriptionKey;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.activemq.store.kahadaptor.ConsumerMessageRef;
|
||||
import org.apache.activemq.store.kahadaptor.ConsumerMessageRefMarshaller;
|
||||
import org.apache.activemq.store.kahadaptor.TopicSubAck;
|
||||
import org.apache.activemq.store.kahadaptor.TopicSubContainer;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -50,78 +45,61 @@ import org.apache.commons.logging.LogFactory;
|
|||
*/
|
||||
public class RapidTopicMessageStore extends RapidMessageStore implements TopicMessageStore{
|
||||
|
||||
private static final Log log=LogFactory.getLog(RapidTopicMessageStore.class);
|
||||
private HashMap ackedLastAckLocations=new HashMap();
|
||||
private final MapContainer subscriberContainer;
|
||||
private final MapContainer ackContainer;
|
||||
private final Store store;
|
||||
private Map subscriberAcks=new ConcurrentHashMap();
|
||||
private ListContainer ackContainer;
|
||||
private Map subscriberContainer;
|
||||
private Store store;
|
||||
private Map subscriberMessages=new ConcurrentHashMap();
|
||||
|
||||
public RapidTopicMessageStore(RapidPersistenceAdapter adapter,ActiveMQTopic destination,
|
||||
MapContainer messageContainer,MapContainer subsContainer,MapContainer ackContainer) throws IOException{
|
||||
super(adapter,destination,messageContainer);
|
||||
this.subscriberContainer=subsContainer;
|
||||
public RapidTopicMessageStore(RapidPersistenceAdapter adapter, Store store,ListContainer messageContainer,ListContainer ackContainer,
|
||||
MapContainer subsContainer,ActiveMQDestination destination,int maximumCacheSize) throws IOException{
|
||||
super(adapter,destination,messageContainer,maximumCacheSize);
|
||||
this.store=store;
|
||||
this.ackContainer=ackContainer;
|
||||
this.store=adapter.getStore();
|
||||
subscriberContainer=subsContainer;
|
||||
// load all the Ack containers
|
||||
for(Iterator i=subscriberContainer.keySet().iterator();i.hasNext();){
|
||||
Object key=i.next();
|
||||
addSubscriberAckContainer(key);
|
||||
addSubscriberMessageContainer(key);
|
||||
}
|
||||
}
|
||||
|
||||
public void recoverSubscription(String clientId,String subscriptionName,final MessageRecoveryListener listener)
|
||||
throws Exception{
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
ListContainer list=(ListContainer)subscriberAcks.get(key);
|
||||
if(list!=null){
|
||||
for(Iterator i=list.iterator();i.hasNext();){
|
||||
Object msg=messageContainer.get(i.next());
|
||||
if(msg!=null){
|
||||
if(msg.getClass()==String.class){
|
||||
listener.recoverMessageReference((String)msg);
|
||||
}else{
|
||||
listener.recoverMessage((Message)msg);
|
||||
public synchronized void addMessage(ConnectionContext context,Message message) throws IOException{
|
||||
int subscriberCount=subscriberMessages.size();
|
||||
if(subscriberCount>0){
|
||||
final Location location = peristenceAdapter.writeCommand(message, message.isResponseRequired());
|
||||
final RapidMessageReference md = new RapidMessageReference(message, location);
|
||||
StoreEntry messageEntry=messageContainer.placeLast(md);
|
||||
TopicSubAck tsa=new TopicSubAck();
|
||||
tsa.setCount(subscriberCount);
|
||||
tsa.setMessageEntry(messageEntry);
|
||||
StoreEntry ackEntry=ackContainer.placeLast(tsa);
|
||||
for(Iterator i=subscriberMessages.values().iterator();i.hasNext();){
|
||||
TopicSubContainer container=(TopicSubContainer)i.next();
|
||||
ConsumerMessageRef ref=new ConsumerMessageRef();
|
||||
ref.setAckEntry(ackEntry);
|
||||
ref.setMessageEntry(messageEntry);
|
||||
container.getListContainer().add(ref);
|
||||
}
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
}else{
|
||||
listener.finished();
|
||||
}
|
||||
}
|
||||
|
||||
public void recoverNextMessages(String clientId,String subscriptionName,MessageId lastMessageId,int maxReturned,
|
||||
MessageRecoveryListener listener) throws Exception{
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
ListContainer list=(ListContainer)subscriberAcks.get(key);
|
||||
if(list!=null){
|
||||
boolean startFound=false;
|
||||
int count=0;
|
||||
for(Iterator i=list.iterator();i.hasNext()&&count<maxReturned;){
|
||||
Object msg=messageContainer.get(i.next());
|
||||
if(msg!=null){
|
||||
if(msg.getClass()==String.class){
|
||||
String ref=msg.toString();
|
||||
if(startFound||lastMessageId==null){
|
||||
listener.recoverMessageReference(ref);
|
||||
count++;
|
||||
}else if(!startFound||ref.equals(lastMessageId.toString())){
|
||||
startFound=true;
|
||||
}
|
||||
public synchronized void acknowledge(ConnectionContext context,String clientId,String subscriptionName,
|
||||
MessageId messageId) throws IOException{
|
||||
String subcriberId=getSubscriptionKey(clientId,subscriptionName);
|
||||
TopicSubContainer container=(TopicSubContainer)subscriberMessages.get(subcriberId);
|
||||
if(container!=null){
|
||||
ConsumerMessageRef ref=(ConsumerMessageRef)container.getListContainer().removeFirst();
|
||||
if(ref!=null){
|
||||
TopicSubAck tsa=(TopicSubAck)ackContainer.get(ref.getAckEntry());
|
||||
if(tsa!=null){
|
||||
if(tsa.decrementCount()<=0){
|
||||
ackContainer.remove(ref.getAckEntry());
|
||||
messageContainer.remove(tsa.getMessageEntry());
|
||||
}else{
|
||||
Message message=(Message)msg;
|
||||
if(startFound||lastMessageId==null){
|
||||
listener.recoverMessage(message);
|
||||
count++;
|
||||
}else if(!startFound&&message.getMessageId().equals(lastMessageId)){
|
||||
startFound=true;
|
||||
ackContainer.update(ref.getAckEntry(),tsa);
|
||||
}
|
||||
}
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
}else{
|
||||
listener.finished();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -129,7 +107,7 @@ public class RapidTopicMessageStore extends RapidMessageStore implements TopicMe
|
|||
return (SubscriptionInfo)subscriberContainer.get(getSubscriptionKey(clientId,subscriptionName));
|
||||
}
|
||||
|
||||
public void addSubsciption(String clientId,String subscriptionName,String selector,boolean retroactive)
|
||||
public synchronized void addSubsciption(String clientId,String subscriptionName,String selector,boolean retroactive)
|
||||
throws IOException{
|
||||
SubscriptionInfo info=new SubscriptionInfo();
|
||||
info.setDestination(destination);
|
||||
|
@ -142,122 +120,82 @@ public class RapidTopicMessageStore extends RapidMessageStore implements TopicMe
|
|||
if(!subscriberContainer.containsKey(key)){
|
||||
subscriberContainer.put(key,info);
|
||||
}
|
||||
addSubscriberAckContainer(key);
|
||||
addSubscriberMessageContainer(key);
|
||||
}
|
||||
|
||||
public synchronized void addMessage(ConnectionContext context,Message message) throws IOException{
|
||||
int subscriberCount=subscriberAcks.size();
|
||||
if(subscriberCount>0){
|
||||
String id=message.getMessageId().toString();
|
||||
ackContainer.put(id,new AtomicInteger(subscriberCount));
|
||||
for(Iterator i=subscriberAcks.keySet().iterator();i.hasNext();){
|
||||
Object key=i.next();
|
||||
ListContainer container=store.getListContainer(key,"durable-subs");
|
||||
container.add(id);
|
||||
}
|
||||
super.addMessage(context,message);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*/
|
||||
public void acknowledge(ConnectionContext context,String clientId,String subscriptionName,final MessageId messageId)
|
||||
throws IOException{
|
||||
final boolean debug=log.isDebugEnabled();
|
||||
JournalTopicAck ack=new JournalTopicAck();
|
||||
ack.setDestination(destination);
|
||||
ack.setMessageId(messageId);
|
||||
ack.setMessageSequenceId(messageId.getBrokerSequenceId());
|
||||
ack.setSubscritionName(subscriptionName);
|
||||
ack.setClientId(clientId);
|
||||
ack.setTransactionId(context.getTransaction()!=null?context.getTransaction().getTransactionId():null);
|
||||
final Location location=peristenceAdapter.writeCommand(ack,false);
|
||||
final SubscriptionKey key=new SubscriptionKey(clientId,subscriptionName);
|
||||
if(!context.isInTransaction()){
|
||||
if(debug)
|
||||
log.debug("Journalled acknowledge for: "+messageId+", at: "+location);
|
||||
acknowledge(messageId,location,key);
|
||||
public synchronized void deleteSubscription(String clientId,String subscriptionName){
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
subscriberContainer.remove(key);
|
||||
TopicSubContainer container=(TopicSubContainer)subscriberMessages.get(key);
|
||||
for(Iterator i=container.getListContainer().iterator();i.hasNext();){
|
||||
ConsumerMessageRef ref=(ConsumerMessageRef)i.next();
|
||||
if(ref!=null){
|
||||
TopicSubAck tsa=(TopicSubAck)ackContainer.get(ref.getAckEntry());
|
||||
if(tsa!=null){
|
||||
if(tsa.decrementCount()<=0){
|
||||
ackContainer.remove(ref.getAckEntry());
|
||||
messageContainer.remove(tsa.getMessageEntry());
|
||||
}else{
|
||||
if(debug)
|
||||
log.debug("Journalled transacted acknowledge for: "+messageId+", at: "+location);
|
||||
synchronized(this){
|
||||
inFlightTxLocations.add(location);
|
||||
ackContainer.update(ref.getAckEntry(),tsa);
|
||||
}
|
||||
}
|
||||
}
|
||||
transactionStore.acknowledge(this,ack,location);
|
||||
context.getTransaction().addSynchronization(new Synchronization(){
|
||||
|
||||
public void afterCommit() throws Exception{
|
||||
if(debug)
|
||||
log.debug("Transacted acknowledge commit for: "+messageId+", at: "+location);
|
||||
synchronized(RapidTopicMessageStore.this){
|
||||
inFlightTxLocations.remove(location);
|
||||
acknowledge(messageId,location,key);
|
||||
}
|
||||
}
|
||||
|
||||
public void afterRollback() throws Exception{
|
||||
if(debug)
|
||||
log.debug("Transacted acknowledge rollback for: "+messageId+", at: "+location);
|
||||
synchronized(RapidTopicMessageStore.this){
|
||||
inFlightTxLocations.remove(location);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
public void replayAcknowledge(ConnectionContext context,String clientId,String subscritionName,MessageId messageId){
|
||||
try{
|
||||
synchronized(this){
|
||||
String subcriberId=getSubscriptionKey(clientId,subscritionName);
|
||||
String id=messageId.toString();
|
||||
ListContainer container=(ListContainer)subscriberAcks.get(subcriberId);
|
||||
public void recoverSubscription(String clientId,String subscriptionName,MessageRecoveryListener listener)
|
||||
throws Exception{
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
TopicSubContainer container=(TopicSubContainer)subscriberMessages.get(key);
|
||||
if(container!=null){
|
||||
// container.remove(id);
|
||||
container.removeFirst();
|
||||
AtomicInteger count=(AtomicInteger)ackContainer.remove(id);
|
||||
if(count!=null){
|
||||
if(count.decrementAndGet()>0){
|
||||
ackContainer.put(id,count);
|
||||
}else{
|
||||
// no more references to message messageContainer so remove it
|
||||
messageContainer.remove(messageId.toString());
|
||||
for(Iterator i=container.getListContainer().iterator();i.hasNext();){
|
||||
ConsumerMessageRef ref=(ConsumerMessageRef)i.next();
|
||||
RapidMessageReference messageReference=(RapidMessageReference)messageContainer.get(ref
|
||||
.getMessageEntry());
|
||||
if(messageReference!=null){
|
||||
Message m=(Message)peristenceAdapter.readCommand(messageReference.getLocation());
|
||||
listener.recoverMessage(m);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}catch(Throwable e){
|
||||
log.debug("Could not replay acknowledge for message '"+messageId
|
||||
+"'. Message may have already been acknowledged. reason: "+e);
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param messageId
|
||||
* @param location
|
||||
* @param key
|
||||
*/
|
||||
private void acknowledge(MessageId messageId,Location location,SubscriptionKey key){
|
||||
synchronized(this){
|
||||
lastLocation=location;
|
||||
ackedLastAckLocations.put(key,messageId);
|
||||
String subcriberId=getSubscriptionKey(key.getClientId(),key.getSubscriptionName());
|
||||
String id=messageId.toString();
|
||||
ListContainer container=(ListContainer)subscriberAcks.get(subcriberId);
|
||||
public void recoverNextMessages(String clientId,String subscriptionName,int maxReturned,
|
||||
MessageRecoveryListener listener) throws Exception{
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
TopicSubContainer container=(TopicSubContainer)subscriberMessages.get(key);
|
||||
if(container!=null){
|
||||
// container.remove(id);
|
||||
container.removeFirst();
|
||||
AtomicInteger count=(AtomicInteger)ackContainer.remove(id);
|
||||
if(count!=null){
|
||||
if(count.decrementAndGet()>0){
|
||||
ackContainer.put(id,count);
|
||||
int count=0;
|
||||
StoreEntry entry=container.getBatchEntry();
|
||||
if(entry==null){
|
||||
entry=container.getListContainer().getFirst();
|
||||
}else{
|
||||
// no more references to message messageContainer so remove it
|
||||
messageContainer.remove(messageId.toString());
|
||||
}
|
||||
entry=container.getListContainer().refresh(entry);
|
||||
entry=container.getListContainer().getNext(entry);
|
||||
}
|
||||
if(entry!=null){
|
||||
do{
|
||||
ConsumerMessageRef consumerRef=(ConsumerMessageRef)container.getListContainer().get(entry);
|
||||
RapidMessageReference messageReference=(RapidMessageReference)messageContainer.get(consumerRef
|
||||
.getMessageEntry());
|
||||
if(messageReference!=null){
|
||||
Message m=(Message)peristenceAdapter.readCommand(messageReference.getLocation());
|
||||
listener.recoverMessage(m);
|
||||
count++;
|
||||
}
|
||||
container.setBatchEntry(entry);
|
||||
entry=container.getListContainer().getNext(entry);
|
||||
}while(entry!=null&&count<maxReturned);
|
||||
}
|
||||
}
|
||||
listener.finished();
|
||||
}
|
||||
|
||||
|
||||
public SubscriptionInfo[] getAllSubscriptions() throws IOException{
|
||||
return (SubscriptionInfo[])subscriberContainer.values().toArray(
|
||||
new SubscriptionInfo[subscriberContainer.size()]);
|
||||
}
|
||||
|
||||
protected String getSubscriptionKey(String clientId,String subscriberName){
|
||||
|
@ -266,73 +204,102 @@ public class RapidTopicMessageStore extends RapidMessageStore implements TopicMe
|
|||
return result;
|
||||
}
|
||||
|
||||
public Location checkpoint() throws IOException{
|
||||
ArrayList cpAckedLastAckLocations;
|
||||
// swap out the hash maps..
|
||||
synchronized(this){
|
||||
cpAckedLastAckLocations=new ArrayList(this.ackedLastAckLocations.values());
|
||||
this.ackedLastAckLocations=new HashMap();
|
||||
}
|
||||
Location rc=super.checkpoint();
|
||||
if(!cpAckedLastAckLocations.isEmpty()){
|
||||
Collections.sort(cpAckedLastAckLocations);
|
||||
Location t=(Location)cpAckedLastAckLocations.get(0);
|
||||
if(rc==null||t.compareTo(rc)<0){
|
||||
rc=t;
|
||||
}
|
||||
}
|
||||
return rc;
|
||||
}
|
||||
|
||||
public void deleteSubscription(String clientId,String subscriptionName) throws IOException{
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
subscriberContainer.remove(key);
|
||||
ListContainer list=(ListContainer)subscriberAcks.get(key);
|
||||
for(Iterator i=list.iterator();i.hasNext();){
|
||||
String id=i.next().toString();
|
||||
AtomicInteger count=(AtomicInteger)ackContainer.remove(id);
|
||||
if(count!=null){
|
||||
if(count.decrementAndGet()>0){
|
||||
ackContainer.put(id,count);
|
||||
}else{
|
||||
// no more references to message messageContainer so remove it
|
||||
messageContainer.remove(id);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public SubscriptionInfo[] getAllSubscriptions() throws IOException{
|
||||
return (SubscriptionInfo[])subscriberContainer.values().toArray(
|
||||
new SubscriptionInfo[subscriberContainer.size()]);
|
||||
}
|
||||
|
||||
protected void addSubscriberAckContainer(Object key) throws IOException{
|
||||
ListContainer container=store.getListContainer(key,"durable-subs");
|
||||
Marshaller marshaller=new StringMarshaller();
|
||||
protected void addSubscriberMessageContainer(Object key) throws IOException{
|
||||
ListContainer container=store.getListContainer(key,"topic-subs");
|
||||
Marshaller marshaller=new ConsumerMessageRefMarshaller();
|
||||
container.setMarshaller(marshaller);
|
||||
subscriberAcks.put(key,container);
|
||||
TopicSubContainer tsc=new TopicSubContainer(container);
|
||||
subscriberMessages.put(key,tsc);
|
||||
}
|
||||
|
||||
|
||||
public int getMessageCount(String clientId,String subscriberName) throws IOException{
|
||||
String key=getSubscriptionKey(clientId,subscriberName);
|
||||
ListContainer list=(ListContainer)subscriberAcks.get(key);
|
||||
return list.size();
|
||||
TopicSubContainer container=(TopicSubContainer)subscriberMessages.get(key);
|
||||
return container.getListContainer().size();
|
||||
}
|
||||
|
||||
public void resetBatching(String clientId,String subscriptionName,MessageId nextId){
|
||||
/**
|
||||
* @param context
|
||||
* @param messageId
|
||||
* @param expirationTime
|
||||
* @param messageRef
|
||||
* @throws IOException
|
||||
* @see org.apache.activemq.store.MessageStore#addMessageReference(org.apache.activemq.broker.ConnectionContext,
|
||||
* org.apache.activemq.command.MessageId, long, java.lang.String)
|
||||
*/
|
||||
public void addMessageReference(ConnectionContext context,MessageId messageId,long expirationTime,String messageRef)
|
||||
throws IOException{
|
||||
throw new IOException("Not supported");
|
||||
}
|
||||
|
||||
|
||||
public void recoverNextMessages(String clientId,String subscriptionName,int maxReturned,MessageRecoveryListener listener) throws Exception{
|
||||
|
||||
|
||||
/**
|
||||
* @param identity
|
||||
* @return String
|
||||
* @throws IOException
|
||||
* @see org.apache.activemq.store.MessageStore#getMessageReference(org.apache.activemq.command.MessageId)
|
||||
*/
|
||||
public String getMessageReference(MessageId identity) throws IOException{
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
public void resetBatching(String clientId,String subscriptionName){
|
||||
|
||||
|
||||
/**
|
||||
* @param context
|
||||
* @throws IOException
|
||||
* @see org.apache.activemq.store.MessageStore#removeAllMessages(org.apache.activemq.broker.ConnectionContext)
|
||||
*/
|
||||
public synchronized void removeAllMessages(ConnectionContext context) throws IOException{
|
||||
messageContainer.clear();
|
||||
ackContainer.clear();
|
||||
for(Iterator i=subscriberMessages.values().iterator();i.hasNext();){
|
||||
TopicSubContainer container=(TopicSubContainer)i.next();
|
||||
container.getListContainer().clear();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public synchronized void resetBatching(String clientId,String subscriptionName){
|
||||
String key=getSubscriptionKey(clientId,subscriptionName);
|
||||
TopicSubContainer topicSubContainer=(TopicSubContainer)subscriberMessages.get(key);
|
||||
if(topicSubContainer!=null){
|
||||
topicSubContainer.reset();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public Location checkpoint() throws IOException{
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
public synchronized void replayAcknowledge(ConnectionContext context,String clientId,String subscriptionName,MessageId messageId){
|
||||
String subcriberId=getSubscriptionKey(clientId,subscriptionName);
|
||||
TopicSubContainer container=(TopicSubContainer)subscriberMessages.get(subcriberId);
|
||||
if(container!=null){
|
||||
ConsumerMessageRef ref=(ConsumerMessageRef)container.getListContainer().removeFirst();
|
||||
if(ref!=null){
|
||||
TopicSubAck tsa=(TopicSubAck)ackContainer.get(ref.getAckEntry());
|
||||
if(tsa!=null){
|
||||
if(tsa.decrementCount()<=0){
|
||||
ackContainer.remove(ref.getAckEntry());
|
||||
messageContainer.remove(tsa.getMessageEntry());
|
||||
}else{
|
||||
ackContainer.update(ref.getAckEntry(),tsa);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,45 @@
|
|||
/**
|
||||
*
|
||||
* 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.perf;
|
||||
|
||||
import java.io.File;
|
||||
import org.apache.activeio.journal.active.JournalImpl;
|
||||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.store.rapid.RapidPersistenceAdapter;
|
||||
/**
|
||||
* @version $Revision: 1.3 $
|
||||
*/
|
||||
public class RapidStoreQueueTest extends SimpleQueueTest{
|
||||
|
||||
|
||||
protected void configureBroker(BrokerService answer) throws Exception{
|
||||
|
||||
File dataFileDir = new File("activemq-data/perfTest");
|
||||
File journalDir = new File(dataFileDir, "journal").getCanonicalFile();
|
||||
JournalImpl journal = new JournalImpl(journalDir, 3, 1024*1024*20);
|
||||
|
||||
RapidPersistenceAdapter adaptor = new RapidPersistenceAdapter(journal,answer.getTaskRunnerFactory());
|
||||
|
||||
|
||||
answer.setPersistenceAdapter(adaptor);
|
||||
answer.addConnector(bindAddress);
|
||||
answer.setDeleteAllMessagesOnStartup(true);
|
||||
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue