mirror of https://github.com/apache/activemq.git
Fixes for Master-Slave functionality
git-svn-id: https://svn.apache.org/repos/asf/incubator/activemq/trunk@372050 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
32446c1f1d
commit
8b04e15826
|
@ -23,6 +23,7 @@ import org.apache.activemq.broker.InsertableMutableBrokerFilter;
|
|||
import org.apache.activemq.broker.MutableBrokerFilter;
|
||||
import org.apache.activemq.command.Command;
|
||||
import org.apache.activemq.command.ConnectionInfo;
|
||||
import org.apache.activemq.command.ConsumerInfo;
|
||||
import org.apache.activemq.command.ExceptionResponse;
|
||||
import org.apache.activemq.command.Message;
|
||||
import org.apache.activemq.command.MessageAck;
|
||||
|
@ -30,6 +31,7 @@ import org.apache.activemq.command.MessageDispatch;
|
|||
import org.apache.activemq.command.MessageDispatchNotification;
|
||||
import org.apache.activemq.command.ProducerInfo;
|
||||
import org.apache.activemq.command.RemoveInfo;
|
||||
import org.apache.activemq.command.RemoveSubscriptionInfo;
|
||||
import org.apache.activemq.command.Response;
|
||||
import org.apache.activemq.command.SessionInfo;
|
||||
import org.apache.activemq.command.TransactionId;
|
||||
|
@ -132,6 +134,17 @@ public class MasterBroker extends InsertableMutableBrokerFilter{
|
|||
sendAsyncToSlave(new RemoveInfo(info.getProducerId()));
|
||||
}
|
||||
|
||||
public void addConsumer(ConnectionContext context, ConsumerInfo info) throws Throwable {
|
||||
super.addConsumer(context, info);
|
||||
sendAsyncToSlave(info);
|
||||
}
|
||||
|
||||
|
||||
public void removeSubscription(ConnectionContext context, RemoveSubscriptionInfo info) throws Throwable {
|
||||
super.removeSubscription(context, info);
|
||||
sendAsyncToSlave(info);
|
||||
}
|
||||
|
||||
|
||||
|
||||
public void beginTransaction(ConnectionContext context, TransactionId xid) throws Throwable{
|
||||
|
@ -163,6 +176,7 @@ public class MasterBroker extends InsertableMutableBrokerFilter{
|
|||
super.rollbackTransaction(context, xid);
|
||||
TransactionInfo info = new TransactionInfo(context.getConnectionId(),xid,TransactionInfo.ROLLBACK);
|
||||
sendAsyncToSlave(info);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -174,7 +188,7 @@ public class MasterBroker extends InsertableMutableBrokerFilter{
|
|||
public void commitTransaction(ConnectionContext context, TransactionId xid, boolean onePhase) throws Throwable{
|
||||
super.commitTransaction(context, xid,onePhase);
|
||||
TransactionInfo info = new TransactionInfo(context.getConnectionId(),xid,TransactionInfo.COMMIT_ONE_PHASE);
|
||||
sendAsyncToSlave(info);
|
||||
sendSyncToSlave(info);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -205,26 +219,40 @@ public class MasterBroker extends InsertableMutableBrokerFilter{
|
|||
}
|
||||
|
||||
public void send(ConnectionContext context, Message message) throws Throwable{
|
||||
/**
|
||||
* A message can be dispatched before the super.send() method returns
|
||||
* so - here the order is switched to avoid problems on the slave
|
||||
* with receiving acks for messages not received yey
|
||||
*/
|
||||
sendToSlave(message);
|
||||
super.send(context,message);
|
||||
sendAsyncToSlave(message);
|
||||
}
|
||||
|
||||
|
||||
public void acknowledge(ConnectionContext context, MessageAck ack) throws Throwable{
|
||||
super.acknowledge(context, ack);
|
||||
sendAsyncToSlave(ack);
|
||||
sendToSlave(ack);
|
||||
}
|
||||
|
||||
|
||||
protected void sendToSlave(Message message){
|
||||
/*
|
||||
if (message.isPersistent()){
|
||||
|
||||
if (message.isPersistent() && !message.isInTransaction()){
|
||||
sendSyncToSlave(message);
|
||||
}else{
|
||||
sendAsyncToSlave(message);
|
||||
}
|
||||
*/
|
||||
sendAsyncToSlave(message);
|
||||
|
||||
|
||||
}
|
||||
|
||||
protected void sendToSlave(MessageAck ack){
|
||||
|
||||
if (ack.isInTransaction()){
|
||||
sendAsyncToSlave(ack);
|
||||
}else{
|
||||
sendSyncToSlave(ack);
|
||||
}
|
||||
}
|
||||
|
||||
protected void sendAsyncToSlave(Command command){
|
||||
|
|
|
@ -233,6 +233,6 @@ public class MasterConnector implements Service{
|
|||
private void shutDown(){
|
||||
masterActive.set(false);
|
||||
broker.masterFailed();
|
||||
//ServiceSupport.dispose(this);
|
||||
ServiceSupport.dispose(this);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,6 +57,8 @@ public class IndirectMessageReference implements MessageReference {
|
|||
private Message message;
|
||||
/** The number of times the message has requested being hardened */
|
||||
private int referenceCount;
|
||||
/** the size of the message **/
|
||||
private int cachedSize = 0;
|
||||
|
||||
/**
|
||||
* Only used by the END_OF_BROWSE_MARKER singleton
|
||||
|
@ -69,6 +71,7 @@ public class IndirectMessageReference implements MessageReference {
|
|||
this.groupID = null;
|
||||
this.groupSequence = 0;
|
||||
this.targetConsumerId=null;
|
||||
this.cachedSize = message != null ? message.getSize() : 0;
|
||||
}
|
||||
|
||||
public IndirectMessageReference(Destination destination, Message message) {
|
||||
|
@ -82,6 +85,7 @@ public class IndirectMessageReference implements MessageReference {
|
|||
|
||||
this.referenceCount=1;
|
||||
message.incrementReferenceCount();
|
||||
this.cachedSize = message != null ? message.getSize() : 0;
|
||||
}
|
||||
|
||||
synchronized public Message getMessageHardRef() {
|
||||
|
@ -202,4 +206,12 @@ public class IndirectMessageReference implements MessageReference {
|
|||
public ConsumerId getTargetConsumerId() {
|
||||
return targetConsumerId;
|
||||
}
|
||||
|
||||
public int getSize(){
|
||||
Message msg = message;
|
||||
if (msg != null){
|
||||
return msg.getSize();
|
||||
}
|
||||
return cachedSize;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,5 +46,6 @@ public interface MessageReference {
|
|||
public int incrementReferenceCount();
|
||||
public int decrementReferenceCount();
|
||||
public ConsumerId getTargetConsumerId();
|
||||
public int getSize();
|
||||
|
||||
}
|
||||
|
|
|
@ -2,17 +2,14 @@
|
|||
*
|
||||
* Copyright 2005-2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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 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.broker.region;
|
||||
|
||||
|
@ -29,14 +26,11 @@ import org.apache.activemq.command.MessageId;
|
|||
import org.apache.activemq.transaction.Synchronization;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import javax.jms.InvalidSelectorException;
|
||||
import javax.jms.JMSException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
|
||||
/**
|
||||
* A subscription that honors the pre-fetch option of the ConsumerInfo.
|
||||
*
|
||||
|
@ -44,17 +38,15 @@ import java.util.LinkedList;
|
|||
*/
|
||||
abstract public class PrefetchSubscription extends AbstractSubscription{
|
||||
static private final Log log=LogFactory.getLog(PrefetchSubscription.class);
|
||||
|
||||
final protected LinkedList matched=new LinkedList();
|
||||
final protected LinkedList dispatched=new LinkedList();
|
||||
|
||||
protected int delivered=0;
|
||||
|
||||
int preLoadLimit=1024*100;
|
||||
int preLoadSize=0;
|
||||
boolean dispatching=false;
|
||||
|
||||
public PrefetchSubscription(Broker broker,ConnectionContext context, ConsumerInfo info) throws InvalidSelectorException {
|
||||
public PrefetchSubscription(Broker broker,ConnectionContext context,ConsumerInfo info)
|
||||
throws InvalidSelectorException{
|
||||
super(broker,context,info);
|
||||
}
|
||||
|
||||
|
@ -66,7 +58,6 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
matched.addLast(node);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void processMessageDispatchNotification(MessageDispatchNotification mdn){
|
||||
|
@ -78,8 +69,7 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
try{
|
||||
MessageDispatch md=createMessageDispatch(node,node.getMessage());
|
||||
dispatched.addLast(node);
|
||||
|
||||
incrementPreloadSize(node.getMessage().getSize());
|
||||
incrementPreloadSize(node.getSize());
|
||||
node.decrementReferenceCount();
|
||||
}catch(Exception e){
|
||||
log.error("Problem processing MessageDispatchNotification: "+mdn,e);
|
||||
|
@ -91,24 +81,19 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
}
|
||||
|
||||
synchronized public void acknowledge(final ConnectionContext context,final MessageAck ack) throws Throwable{
|
||||
|
||||
// Handle the standard acknowledgment case.
|
||||
boolean wasFull=isFull();
|
||||
if(ack.isStandardAck()){
|
||||
|
||||
// Acknowledge all dispatched messages up till the message id of the acknowledgment.
|
||||
int index=0;
|
||||
boolean inAckRange=false;
|
||||
for(Iterator iter=dispatched.iterator();iter.hasNext();){
|
||||
final MessageReference node=(MessageReference) iter.next();
|
||||
MessageId messageId=node.getMessageId();
|
||||
|
||||
if(ack.getFirstMessageId()==null||ack.getFirstMessageId().equals(messageId)){
|
||||
inAckRange=true;
|
||||
}
|
||||
|
||||
if(inAckRange){
|
||||
|
||||
// Don't remove the nodes until we are committed.
|
||||
if(!context.isInTransaction()){
|
||||
iter.remove();
|
||||
|
@ -117,7 +102,6 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
context.getTransaction().addSynchronization(new Synchronization(){
|
||||
public void afterCommit() throws Throwable{
|
||||
synchronized(PrefetchSubscription.this){
|
||||
|
||||
// Now that we are committed, we can remove the nodes.
|
||||
boolean inAckRange=false;
|
||||
int index=0;
|
||||
|
@ -136,12 +120,10 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
index++;
|
||||
acknowledge(context,ack,node);
|
||||
if(ack.getLastMessageId().equals(messageId)){
|
||||
|
@ -149,7 +131,6 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
delivered=Math.max(delivered,index+1);
|
||||
else
|
||||
delivered=Math.max(0,delivered-(index+1));
|
||||
|
||||
if(wasFull&&!isFull()){
|
||||
dispatchMatched();
|
||||
}
|
||||
|
@ -158,12 +139,9 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
// System.out.println("no match: "+ack.getLastMessageId()+","+messageId);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
log.info("Could not correlate acknowledgment with dispatched message: "+ack);
|
||||
|
||||
}else if(ack.isDeliveredAck()){
|
||||
|
||||
// Message was delivered but not acknowledged: update pre-fetch counters.
|
||||
// Acknowledge all dispatched messages up till the message id of the acknowledgment.
|
||||
int index=0;
|
||||
|
@ -178,47 +156,39 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
}
|
||||
}
|
||||
throw new JMSException("Could not correlate acknowledgment with dispatched message: "+ack);
|
||||
|
||||
}else if(ack.isPoisonAck()){
|
||||
|
||||
// TODO: what if the message is already in a DLQ???
|
||||
|
||||
// Handle the poison ACK case: we need to send the message to a DLQ
|
||||
if(ack.isInTransaction())
|
||||
throw new JMSException("Poison ack cannot be transacted: "+ack);
|
||||
|
||||
// Acknowledge all dispatched messages up till the message id of the acknowledgment.
|
||||
int index=0;
|
||||
boolean inAckRange=false;
|
||||
for(Iterator iter=dispatched.iterator();iter.hasNext();){
|
||||
final MessageReference node=(MessageReference) iter.next();
|
||||
MessageId messageId=node.getMessageId();
|
||||
|
||||
if(ack.getFirstMessageId()==null||ack.getFirstMessageId().equals(messageId)){
|
||||
inAckRange=true;
|
||||
}
|
||||
|
||||
if(inAckRange){
|
||||
|
||||
// Send the message to the DLQ
|
||||
node.incrementReferenceCount();
|
||||
try{
|
||||
Message message=node.getMessage();
|
||||
if(message!=null){
|
||||
|
||||
// The original destination and transaction id do not get filled when the message is first sent,
|
||||
// The original destination and transaction id do not get filled when the message is first
|
||||
// sent,
|
||||
// it is only populated if the message is routed to another destination like the DLQ
|
||||
if(message.getOriginalDestination()!=null)
|
||||
message.setOriginalDestination(message.getDestination());
|
||||
if(message.getOriginalTransactionId()!=null)
|
||||
message.setOriginalTransactionId(message.getTransactionId());
|
||||
|
||||
DeadLetterStrategy deadLetterStrategy=node.getRegionDestination().getDeadLetterStrategy();
|
||||
ActiveMQDestination deadLetterDestination = deadLetterStrategy.getDeadLetterQueueFor(message.getDestination());
|
||||
ActiveMQDestination deadLetterDestination=deadLetterStrategy.getDeadLetterQueueFor(message
|
||||
.getDestination());
|
||||
message.setDestination(deadLetterDestination);
|
||||
message.setTransactionId(null);
|
||||
message.evictMarshlledForm();
|
||||
|
||||
boolean originalFlowControl=context.isProducerFlowControl();
|
||||
try{
|
||||
context.setProducerFlowControl(false);
|
||||
|
@ -226,19 +196,15 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
}finally{
|
||||
context.setProducerFlowControl(originalFlowControl);
|
||||
}
|
||||
|
||||
}
|
||||
}finally{
|
||||
node.decrementReferenceCount();
|
||||
}
|
||||
|
||||
iter.remove();
|
||||
index++;
|
||||
acknowledge(context,ack,node);
|
||||
if(ack.getLastMessageId().equals(messageId)){
|
||||
|
||||
delivered=Math.max(0,delivered-(index+1));
|
||||
|
||||
if(wasFull&&!isFull()){
|
||||
dispatchMatched();
|
||||
}
|
||||
|
@ -248,7 +214,6 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
}
|
||||
throw new JMSException("Could not correlate acknowledgment with dispatched message: "+ack);
|
||||
}
|
||||
|
||||
throw new JMSException("Invalid acknowledgment: "+ack);
|
||||
}
|
||||
|
||||
|
@ -271,24 +236,17 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
private void dispatch(final MessageReference node) throws IOException{
|
||||
node.incrementReferenceCount();
|
||||
|
||||
final Message message=node.getMessage();
|
||||
if(message==null){
|
||||
return;
|
||||
}
|
||||
|
||||
// Make sure we can dispatch a message.
|
||||
if(canDispatch(node)&&!isSlaveBroker()){
|
||||
|
||||
MessageDispatch md=createMessageDispatch(node,message);
|
||||
dispatched.addLast(node);
|
||||
|
||||
incrementPreloadSize(node.getMessage().getSize());
|
||||
|
||||
if(info.isDispatchAsync()){
|
||||
md.setConsumer(new Runnable(){
|
||||
public void run(){
|
||||
|
@ -307,19 +265,15 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
// We were not allowed to dispatch that message (an other consumer grabbed it before we did)
|
||||
node.decrementReferenceCount();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
synchronized private void onDispatch(final MessageReference node,final Message message){
|
||||
|
||||
boolean wasFull=isFull();
|
||||
decrementPreloadSize(message.getSize());
|
||||
node.decrementReferenceCount();
|
||||
|
||||
if(node.getRegionDestination()!=null){
|
||||
node.getRegionDestination().getDestinationStatistics().onMessageDequeue(message);
|
||||
context.getConnection().getStatistics().onMessageDequeue(message);
|
||||
|
||||
if(wasFull&&!isFull()){
|
||||
try{
|
||||
dispatchMatched();
|
||||
|
@ -328,7 +282,6 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private int incrementPreloadSize(int size){
|
||||
|
@ -341,10 +294,10 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
return preLoadSize;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @param node
|
||||
* @param message TODO
|
||||
* @param message
|
||||
* TODO
|
||||
* @return
|
||||
*/
|
||||
protected MessageDispatch createMessageDispatch(MessageReference node,Message message){
|
||||
|
@ -360,16 +313,16 @@ abstract public class PrefetchSubscription extends AbstractSubscription {
|
|||
* Use when a matched message is about to be dispatched to the client.
|
||||
*
|
||||
* @param node
|
||||
* @return false if the message should not be dispatched to the client (another sub may have already dispatched it for example).
|
||||
* @return false if the message should not be dispatched to the client (another sub may have already dispatched it
|
||||
* for example).
|
||||
*/
|
||||
abstract protected boolean canDispatch(MessageReference node);
|
||||
|
||||
/**
|
||||
* Used during acknowledgment to remove the message.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void acknowledge(ConnectionContext context, final MessageAck ack, final MessageReference node) throws IOException {
|
||||
}
|
||||
|
||||
|
||||
protected void acknowledge(ConnectionContext context,final MessageAck ack,final MessageReference node)
|
||||
throws IOException{}
|
||||
}
|
||||
|
|
|
@ -1,97 +0,0 @@
|
|||
/**
|
||||
*
|
||||
* Copyright 2005-2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.broker.ft;
|
||||
|
||||
import java.net.URI;
|
||||
|
||||
import javax.jms.*;
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.xbean.BrokerFactoryBean;
|
||||
import org.springframework.context.support.AbstractApplicationContext;
|
||||
import org.springframework.context.support.ClassPathXmlApplicationContext;
|
||||
import org.springframework.core.io.ClassPathResource;
|
||||
|
||||
public class FTBrokerTest extends TestCase {
|
||||
|
||||
protected static final int MESSAGE_COUNT = 10;
|
||||
protected BrokerService master;
|
||||
protected BrokerService slave;
|
||||
protected Connection connection;
|
||||
protected String uriString = "failover://(tcp://localhost:62001,tcp://localhost:62002)?randomize=false";
|
||||
//protected String uriString = "tcp://localhost:62001";
|
||||
|
||||
protected void setUp() throws Exception {
|
||||
BrokerFactoryBean brokerFactory = new BrokerFactoryBean(new ClassPathResource("org/apache/activemq/broker/ft/master.xml"));
|
||||
brokerFactory.afterPropertiesSet();
|
||||
master = brokerFactory.getBroker();
|
||||
brokerFactory = new BrokerFactoryBean(new ClassPathResource("org/apache/activemq/broker/ft/slave.xml"));
|
||||
brokerFactory.afterPropertiesSet();
|
||||
slave = brokerFactory.getBroker();
|
||||
//uriString = "failover://(" + master.getVmConnectorURI() + "," + slave.getVmConnectorURI() + ")?randomize=false";
|
||||
//uriString = "failover://(" + master.getVmConnectorURI() + "," + slave.getVmConnectorURI() + ")";
|
||||
System.out.println("URI = " + uriString);
|
||||
URI uri = new URI(uriString);
|
||||
ConnectionFactory fac = new ActiveMQConnectionFactory(uri);
|
||||
connection = fac.createConnection();
|
||||
master.start();
|
||||
slave.start();
|
||||
//wait for thing to connect
|
||||
Thread.sleep(1000);
|
||||
connection.start();
|
||||
super.setUp();
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
protected void tearDown() throws Exception {
|
||||
try {
|
||||
connection.close();
|
||||
slave.stop();
|
||||
master.stop();
|
||||
}catch(Throwable e){
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
public void testFTBroker() throws Exception{
|
||||
|
||||
Session session = connection.createSession(false,Session.AUTO_ACKNOWLEDGE);
|
||||
Queue queue = session.createQueue(getClass().toString());
|
||||
MessageProducer producer = session.createProducer(queue);
|
||||
for (int i = 0; i < MESSAGE_COUNT; i++){
|
||||
Message msg = session.createTextMessage("test: " + i);
|
||||
producer.send(msg);
|
||||
}
|
||||
master.stop();
|
||||
session = connection.createSession(false,Session.AUTO_ACKNOWLEDGE);
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
for (int i = 0; i < MESSAGE_COUNT; i++){
|
||||
System.out.println("GOT MSG: " + consumer.receive(1000));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/**
|
||||
*
|
||||
* Copyright 2005-2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.broker.ft;
|
||||
|
||||
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||
import org.apache.activemq.JmsTopicSendReceiveWithTwoConnectionsTest;
|
||||
import org.apache.activemq.broker.BrokerService;
|
||||
import org.apache.activemq.xbean.BrokerFactoryBean;
|
||||
import org.springframework.core.io.ClassPathResource;
|
||||
|
||||
/**
|
||||
*Test failover for Queues
|
||||
*
|
||||
*/
|
||||
public class QueueMasterSlaveTest extends JmsTopicSendReceiveWithTwoConnectionsTest{
|
||||
|
||||
|
||||
|
||||
protected BrokerService master;
|
||||
protected BrokerService slave;
|
||||
protected int inflightMessageCount = 0;
|
||||
protected int failureCount = 50;
|
||||
protected String uriString="failover://(tcp://localhost:62001,tcp://localhost:62002)?randomize=false";
|
||||
|
||||
protected void setUp() throws Exception{
|
||||
failureCount = super.messageCount/2;
|
||||
super.topic = isTopic();
|
||||
BrokerFactoryBean brokerFactory=new BrokerFactoryBean(new ClassPathResource("org/apache/activemq/broker/ft/master.xml"));
|
||||
brokerFactory.afterPropertiesSet();
|
||||
master=brokerFactory.getBroker();
|
||||
brokerFactory=new BrokerFactoryBean(new ClassPathResource("org/apache/activemq/broker/ft/slave.xml"));
|
||||
brokerFactory.afterPropertiesSet();
|
||||
slave=brokerFactory.getBroker();
|
||||
master.start();
|
||||
slave.start();
|
||||
// wait for thing to connect
|
||||
Thread.sleep(1000);
|
||||
super.setUp();
|
||||
|
||||
}
|
||||
|
||||
protected void tearDown() throws Exception{
|
||||
super.tearDown();
|
||||
slave.stop();
|
||||
master.stop();
|
||||
}
|
||||
|
||||
protected ActiveMQConnectionFactory createConnectionFactory() throws Exception{
|
||||
return new ActiveMQConnectionFactory(uriString);
|
||||
}
|
||||
|
||||
protected void messageSent() throws Exception{
|
||||
if (++inflightMessageCount >= failureCount){
|
||||
inflightMessageCount = 0;
|
||||
master.stop();
|
||||
}
|
||||
}
|
||||
|
||||
protected boolean isTopic(){
|
||||
return false;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue