JBoss hornetq SVN: r7965 - trunk/docs/user-manual/en.
by do-not-reply@jboss.org
Author: jmesnil
Date: 2009-09-17 04:54:21 -0400 (Thu, 17 Sep 2009)
New Revision: 7965
Modified:
trunk/docs/user-manual/en/configuring-transports.xml
Log:
transport configuration documentation
* clarify that acceptors can be configured with a comma-separated
list of hosts or IP addresses
Modified: trunk/docs/user-manual/en/configuring-transports.xml
===================================================================
--- trunk/docs/user-manual/en/configuring-transports.xml 2009-09-17 03:08:24 UTC (rev 7964)
+++ trunk/docs/user-manual/en/configuring-transports.xml 2009-09-17 08:54:21 UTC (rev 7965)
@@ -201,9 +201,11 @@
</listitem>
<listitem>
<para><literal>hornetq.remoting.netty.host</literal>. This specified the host
- name or ip address to connect to (when configuring a connector) or to listen
+ name or IP address to connect to (when configuring a connector) or to listen
on (when configuring an acceptor). The default value for this property is
- <literal>localhost</literal>. Note that if you want your servers
+ <literal>localhost</literal>. When configuring acceptors, multiple hosts
+ or IP addresses can be specified by separating them with commas.
+ Note that if you want your servers
accessible from other nodes, don't bind to localhost!</para>
</listitem>
<listitem>
16 years, 3 months
JBoss hornetq SVN: r7964 - trunk/src/main/org/hornetq/core/journal/impl.
by do-not-reply@jboss.org
Author: clebert.suconic(a)jboss.com
Date: 2009-09-16 23:08:24 -0400 (Wed, 16 Sep 2009)
New Revision: 7964
Modified:
trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java
Log:
Fixing configuration
Modified: trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java 2009-09-16 21:29:29 UTC (rev 7963)
+++ trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java 2009-09-17 03:08:24 UTC (rev 7964)
@@ -200,7 +200,7 @@
private final AtomicBoolean compactorRunning = new AtomicBoolean();
private ExecutorService filesExecutor = null;
-
+
private ExecutorService compactorExecutor = null;
// Lock used during the append of records
@@ -2075,46 +2075,48 @@
}
}
- // TODO: make this configurable
- if (nCleanup > 5)
+ if (compactMinFiles > 0)
{
- for (JournalFile file : dataFiles)
+ if (nCleanup > getMinCompact())
{
- if (file.isNeedCleanup())
+ for (JournalFile file : dataFiles)
{
- final JournalFile cleanupFile = file;
-
- if (compactorRunning.compareAndSet(false, true))
+ if (file.isNeedCleanup())
{
- // The cleanup should happen rarely.
- // but when it happens it needs to use a different thread,
- // or opening new files or any other executor's usage will be blocked while the cleanUp is being
- // processed.
-
- compactorExecutor.execute(new Runnable()
+ final JournalFile cleanupFile = file;
+
+ if (compactorRunning.compareAndSet(false, true))
{
- public void run()
+ // The cleanup should happen rarely.
+ // but when it happens it needs to use a different thread,
+ // or opening new files or any other executor's usage will be blocked while the cleanUp is being
+ // processed.
+
+ compactorExecutor.execute(new Runnable()
{
- try
+ public void run()
{
- cleanUp(cleanupFile);
- }
- catch (Exception e)
- {
- log.warn(e.getMessage(), e);
- }
- finally
- {
- compactorRunning.set(false);
- if (autoReclaim)
+ try
{
- scheduleReclaim();
+ cleanUp(cleanupFile);
}
+ catch (Exception e)
+ {
+ log.warn(e.getMessage(), e);
+ }
+ finally
+ {
+ compactorRunning.set(false);
+ if (autoReclaim)
+ {
+ scheduleReclaim();
+ }
+ }
}
- }
- });
+ });
+ }
+ return true;
}
- return true;
}
}
}
@@ -2127,6 +2129,14 @@
return false;
}
+ /**
+ * @return
+ */
+ private float getMinCompact()
+ {
+ return (compactMinFiles * compactPercentage);
+ }
+
public synchronized void cleanUp(final JournalFile file) throws Exception
{
if (state != STATE_LOADED)
@@ -2141,18 +2151,18 @@
JournalCleaner cleaner = null;
ArrayList<JournalFile> dependencies = new ArrayList<JournalFile>();
lockAppend.lock();
-
+
try
{
- log.info("Cleaning up file " + file);
-
+ log.info("Cleaning up file " + file);
+
if (file.getPosCount() == 0)
{
// nothing to be done
return;
}
-
+
// We don't want this file to be reclaimed during the cleanup
file.incPosCount();
@@ -2163,7 +2173,7 @@
if (jrnFile.resetNegCount(file))
{
dependencies.add(jrnFile);
- jrnFile.incPosCount(); // this file can't be reclaimed while cleanup is being done
+ jrnFile.incPosCount(); // this file can't be reclaimed while cleanup is being done
}
}
@@ -2179,7 +2189,7 @@
cleaner.flush();
cleaner.fixDependencies(file, dependencies);
-
+
for (JournalFile jrnfile : dependencies)
{
jrnfile.decPosCount();
@@ -2199,10 +2209,10 @@
finally
{
compactingLock.readLock().unlock();
- log.info("Clean up on file " + file + " done");
+ log.info("Clean up on file " + file + " done");
}
- }
+ }
public void checkCompact() throws Exception
{
@@ -2444,7 +2454,7 @@
}
filesExecutor = Executors.newSingleThreadExecutor();
-
+
compactorExecutor = Executors.newCachedThreadPool();
fileFactory.start();
16 years, 3 months
JBoss hornetq SVN: r7963 - in trunk: src/main/org/hornetq/core/journal/impl and 5 other directories.
by do-not-reply@jboss.org
Author: clebert.suconic(a)jboss.com
Date: 2009-09-16 17:29:29 -0400 (Wed, 16 Sep 2009)
New Revision: 7963
Added:
trunk/src/main/org/hornetq/core/journal/impl/AbstractJournalUpdateTask.java
trunk/src/main/org/hornetq/core/journal/impl/JournalCleaner.java
Modified:
trunk/src/main/org/hornetq/core/journal/Journal.java
trunk/src/main/org/hornetq/core/journal/impl/JournalCompactor.java
trunk/src/main/org/hornetq/core/journal/impl/JournalFile.java
trunk/src/main/org/hornetq/core/journal/impl/JournalFileImpl.java
trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java
trunk/src/main/org/hornetq/core/journal/impl/JournalTransaction.java
trunk/src/main/org/hornetq/core/journal/impl/Reclaimer.java
trunk/tests/src/org/hornetq/tests/integration/client/CompactingTest.java
trunk/tests/src/org/hornetq/tests/integration/journal/NIOJournalCompactTest.java
trunk/tests/src/org/hornetq/tests/stress/journal/AddAndRemoveStressTest.java
trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/AlignedJournalImplTest.java
trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestBase.java
trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestUnit.java
trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/ReclaimerTest.java
trunk/tests/src/org/hornetq/tests/util/ListJournal.java
Log:
HORNETQ-35 - Journal cleanup
Modified: trunk/src/main/org/hornetq/core/journal/Journal.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/Journal.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/src/main/org/hornetq/core/journal/Journal.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -81,8 +81,9 @@
void perfBlast(int pages) throws Exception;
- /** This method is called automatically when a new file is opened */
- void checkAndReclaimFiles() throws Exception;
+ /** This method is called automatically when a new file is opened.
+ * @return true if it needs to re-check due to cleanup or other factors */
+ boolean checkReclaimStatus() throws Exception;
/** This method check for the need of compacting based on the minCompactPercentage
* This method is usually called automatically when new files are opened
Added: trunk/src/main/org/hornetq/core/journal/impl/AbstractJournalUpdateTask.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/AbstractJournalUpdateTask.java (rev 0)
+++ trunk/src/main/org/hornetq/core/journal/impl/AbstractJournalUpdateTask.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -0,0 +1,238 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.core.journal.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import org.hornetq.core.buffers.ChannelBuffer;
+import org.hornetq.core.buffers.ChannelBuffers;
+import org.hornetq.core.journal.SequentialFile;
+import org.hornetq.core.journal.SequentialFileFactory;
+import org.hornetq.core.logging.Logger;
+import org.hornetq.core.remoting.spi.HornetQBuffer;
+import org.hornetq.utils.ConcurrentHashSet;
+import org.hornetq.utils.Pair;
+
+/**
+ *
+ * Super class for Journal maintenances such as clean up and Compactor
+ *
+ * @author <mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
+ *
+ *
+ */
+public abstract class AbstractJournalUpdateTask implements JournalReaderCallback
+{
+
+ // Constants -----------------------------------------------------
+
+ // Attributes ----------------------------------------------------
+ protected static final String FILE_COMPACT_CONTROL = "journal-rename-control.ctr";
+
+ private static final Logger log = Logger.getLogger(AbstractJournalUpdateTask.class);
+
+ protected final JournalImpl journal;
+
+ protected final SequentialFileFactory fileFactory;
+
+ protected JournalFile currentFile;
+
+ protected SequentialFile sequentialFile;
+
+ protected int fileID;
+
+ protected int nextOrderingID;
+
+ private ChannelBuffer writingChannel;
+
+ private final Set<Long> recordsSnapshot = new ConcurrentHashSet<Long>();
+
+ protected final List<JournalFile> newDataFiles = new ArrayList<JournalFile>();
+
+ // Static --------------------------------------------------------
+
+ // Constructors --------------------------------------------------
+
+ protected AbstractJournalUpdateTask(final SequentialFileFactory fileFactory,
+ final JournalImpl journal,
+ final Set<Long> recordsSnapshot,
+ final int nextOrderingID)
+ {
+ super();
+ this.journal = journal;
+ this.fileFactory = fileFactory;
+ this.nextOrderingID = nextOrderingID;
+ this.recordsSnapshot.addAll(recordsSnapshot);
+ }
+
+ // Public --------------------------------------------------------
+
+ /**
+ * @param tmpRenameFile
+ * @param files
+ * @param newFiles
+ */
+ public static SequentialFile writeControlFile(final SequentialFileFactory fileFactory,
+ final List<JournalFile> files,
+ final List<JournalFile> newFiles,
+ final List<Pair<String, String>> renames) throws Exception
+ {
+
+ SequentialFile controlFile = fileFactory.createSequentialFile(FILE_COMPACT_CONTROL, 1);
+
+ try
+ {
+ controlFile.open(1);
+
+ ChannelBuffer renameBuffer = ChannelBuffers.dynamicBuffer(1);
+
+ renameBuffer.writeInt(-1);
+ renameBuffer.writeInt(-1);
+
+ HornetQBuffer filesToRename = ChannelBuffers.dynamicBuffer(1);
+
+ // DataFiles first
+
+ if (files == null)
+ {
+ filesToRename.writeInt(0);
+ }
+ else
+ {
+ filesToRename.writeInt(files.size());
+
+ for (JournalFile file : files)
+ {
+ filesToRename.writeUTF(file.getFile().getFileName());
+ }
+ }
+
+ // New Files second
+
+ if (newFiles == null)
+ {
+ filesToRename.writeInt(0);
+ }
+ else
+ {
+ filesToRename.writeInt(newFiles.size());
+
+ for (JournalFile file : newFiles)
+ {
+ filesToRename.writeUTF(file.getFile().getFileName());
+ }
+ }
+
+ // Renames from clean up third
+ if (renames == null)
+ {
+ filesToRename.writeInt(0);
+ }
+ else
+ {
+ filesToRename.writeInt(renames.size());
+ for (Pair<String, String> rename : renames)
+ {
+ filesToRename.writeUTF(rename.a);
+ filesToRename.writeUTF(rename.b);
+ }
+ }
+
+ JournalImpl.writeAddRecord(-1,
+ 1,
+ (byte)0,
+ new JournalImpl.ByteArrayEncoding(filesToRename.array()),
+ JournalImpl.SIZE_ADD_RECORD + filesToRename.array().length,
+ renameBuffer);
+
+ ByteBuffer writeBuffer = fileFactory.newBuffer(renameBuffer.writerIndex());
+
+ writeBuffer.put(renameBuffer.array(), 0, renameBuffer.writerIndex());
+
+ writeBuffer.rewind();
+
+ controlFile.write(writeBuffer, true);
+
+ return controlFile;
+ }
+ finally
+ {
+ controlFile.close();
+ }
+ }
+
+ /** Write pending output into file */
+ public void flush() throws Exception
+ {
+ if (writingChannel != null)
+ {
+ sequentialFile.position(0);
+ sequentialFile.write(writingChannel.toByteBuffer(), true);
+ sequentialFile.close();
+ newDataFiles.add(currentFile);
+ }
+
+ writingChannel = null;
+ }
+
+ public boolean lookupRecord(final long id)
+ {
+ return recordsSnapshot.contains(id);
+ }
+
+ // Package protected ---------------------------------------------
+
+ // Protected -----------------------------------------------------
+ /**
+ * @throws Exception
+ */
+
+ protected void openFile() throws Exception
+ {
+ flush();
+
+ ByteBuffer bufferWrite = fileFactory.newBuffer(journal.getFileSize());
+ writingChannel = ChannelBuffers.wrappedBuffer(bufferWrite);
+
+ currentFile = journal.getFile(false, false, false, true);
+ sequentialFile = currentFile.getFile();
+
+ sequentialFile.open(1);
+ fileID = nextOrderingID++;
+ currentFile = new JournalFileImpl(sequentialFile, fileID);
+
+ writingChannel.writeInt(fileID);
+ }
+
+ protected void addToRecordsSnaptsho(long id)
+ {
+ recordsSnapshot.add(id);
+ }
+
+ /**
+ * @return the writingChannel
+ */
+ protected ChannelBuffer getWritingChannel()
+ {
+ return writingChannel;
+ }
+
+ // Private -------------------------------------------------------
+
+ // Inner classes -------------------------------------------------
+
+}
Added: trunk/src/main/org/hornetq/core/journal/impl/JournalCleaner.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/JournalCleaner.java (rev 0)
+++ trunk/src/main/org/hornetq/core/journal/impl/JournalCleaner.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -0,0 +1,282 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.core.journal.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.hornetq.core.journal.RecordInfo;
+import org.hornetq.core.journal.SequentialFileFactory;
+import org.hornetq.utils.DataConstants;
+
+/**
+ * A JournalCleaner
+ *
+ * @author <mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
+ *
+ *
+ */
+public class JournalCleaner extends AbstractJournalUpdateTask
+{
+
+ // Constants -----------------------------------------------------
+
+ // Attributes ----------------------------------------------------
+
+ private final HashMap<Long, AtomicInteger> transactionCounter = new HashMap<Long, AtomicInteger>();
+
+ // Static --------------------------------------------------------
+
+ // Constructors --------------------------------------------------
+ /**
+ * @param fileFactory
+ * @param journal
+ * @param nextOrderingID
+ */
+ protected JournalCleaner(final SequentialFileFactory fileFactory,
+ final JournalImpl journal,
+ final Set<Long> recordsSnapshot,
+ final int nextOrderingID) throws Exception
+ {
+ super(fileFactory, journal, recordsSnapshot, nextOrderingID);
+ openFile();
+ }
+
+ // Public --------------------------------------------------------
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#markAsDataFile(org.hornetq.core.journal.impl.JournalFile)
+ */
+ public void markAsDataFile(final JournalFile file)
+ {
+ // nothing to be done here
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadAddRecord(org.hornetq.core.journal.RecordInfo)
+ */
+ public void onReadAddRecord(final RecordInfo info) throws Exception
+ {
+ if (lookupRecord(info.id))
+ {
+ int size = JournalImpl.SIZE_ADD_RECORD + info.data.length;
+
+ JournalImpl.writeAddRecord(fileID,
+ info.id,
+ info.getUserRecordType(),
+ new JournalImpl.ByteArrayEncoding(info.data),
+ size,
+ getWritingChannel());
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadAddRecordTX(long, org.hornetq.core.journal.RecordInfo)
+ */
+ public void onReadAddRecordTX(final long transactionID, final RecordInfo recordInfo) throws Exception
+ {
+ if (lookupRecord(recordInfo.id))
+ {
+ incrementTransactionCounter(transactionID);
+
+ int size = JournalImpl.SIZE_ADD_RECORD_TX + recordInfo.data.length;
+
+ JournalImpl.writeAddRecordTX(fileID,
+ transactionID,
+ recordInfo.id,
+ recordInfo.getUserRecordType(),
+ new JournalImpl.ByteArrayEncoding(recordInfo.data),
+ size,
+ getWritingChannel());
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadCommitRecord(long, int)
+ */
+ public void onReadCommitRecord(final long transactionID, final int numberOfRecords) throws Exception
+ {
+ int txcounter = getTransactionCounter(transactionID);
+
+ JournalImpl.writeTransaction(fileID,
+ JournalImpl.COMMIT_RECORD,
+ transactionID,
+ null,
+ JournalImpl.SIZE_COMPLETE_TRANSACTION_RECORD,
+ txcounter,
+ getWritingChannel());
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadDeleteRecord(long)
+ */
+ public void onReadDeleteRecord(final long recordID) throws Exception
+ {
+ JournalImpl.writeDeleteRecord(fileID, recordID, JournalImpl.SIZE_DELETE_RECORD, getWritingChannel());
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadDeleteRecordTX(long, org.hornetq.core.journal.RecordInfo)
+ */
+ public void onReadDeleteRecordTX(final long transactionID, final RecordInfo recordInfo) throws Exception
+ {
+ int size = JournalImpl.SIZE_DELETE_RECORD_TX + recordInfo.data.length;
+
+ incrementTransactionCounter(transactionID);
+
+ JournalImpl.writeDeleteRecordTransactional(fileID,
+ transactionID,
+ recordInfo.id,
+ new JournalImpl.ByteArrayEncoding(recordInfo.data),
+ size,
+ getWritingChannel());
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadPrepareRecord(long, byte[], int)
+ */
+ public void onReadPrepareRecord(final long transactionID, final byte[] extraData, final int numberOfRecords) throws Exception
+ {
+ int txcounter = getTransactionCounter(transactionID);
+
+ int size = JournalImpl.SIZE_COMPLETE_TRANSACTION_RECORD + extraData.length + DataConstants.SIZE_INT;
+
+ JournalImpl.writeTransaction(fileID,
+ JournalImpl.PREPARE_RECORD,
+ transactionID,
+ new JournalImpl.ByteArrayEncoding(extraData),
+ size,
+ txcounter,
+ getWritingChannel());
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadRollbackRecord(long)
+ */
+ public void onReadRollbackRecord(final long transactionID) throws Exception
+ {
+ JournalImpl.writeRollback(fileID, transactionID, getWritingChannel());
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadUpdateRecord(org.hornetq.core.journal.RecordInfo)
+ */
+ public void onReadUpdateRecord(final RecordInfo recordInfo) throws Exception
+ {
+ if (lookupRecord(recordInfo.id))
+ {
+ int size = JournalImpl.SIZE_UPDATE_RECORD + recordInfo.data.length;
+ JournalImpl.writeUpdateRecord(fileID,
+ recordInfo.id,
+ recordInfo.userRecordType,
+ new JournalImpl.ByteArrayEncoding(recordInfo.data),
+ size,
+ getWritingChannel());
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalReaderCallback#onReadUpdateRecordTX(long, org.hornetq.core.journal.RecordInfo)
+ */
+ public void onReadUpdateRecordTX(final long transactionID, final RecordInfo recordInfo) throws Exception
+ {
+ if (lookupRecord(recordInfo.id))
+ {
+ incrementTransactionCounter(transactionID);
+ int size = JournalImpl.SIZE_UPDATE_RECORD_TX + recordInfo.data.length;
+ JournalImpl.writeUpdateRecordTX(fileID,
+ transactionID,
+ recordInfo.id,
+ recordInfo.userRecordType,
+ new JournalImpl.ByteArrayEncoding(recordInfo.data),
+ size,
+ getWritingChannel());
+ }
+ }
+
+ /**
+ * Read files that depend on this file.
+ * Commits and rollbacks are also counted as negatives. We need to fix those also.
+ * @param dependencies
+ */
+ public void fixDependencies(final JournalFile originalFile, final ArrayList<JournalFile> dependencies) throws Exception
+ {
+ for (JournalFile dependency : dependencies)
+ {
+ fixDependency(originalFile, dependency);
+ }
+
+ }
+
+ // Package protected ---------------------------------------------
+
+ // Protected -----------------------------------------------------
+
+ protected int incrementTransactionCounter(final long transactionID)
+ {
+ AtomicInteger counter = transactionCounter.get(transactionID);
+ if (counter == null)
+ {
+ counter = new AtomicInteger(0);
+ transactionCounter.put(transactionID, counter);
+ }
+
+ return counter.incrementAndGet();
+ }
+
+ protected int getTransactionCounter(final long transactionID)
+ {
+ AtomicInteger counter = transactionCounter.get(transactionID);
+ if (counter == null)
+ {
+ return 0;
+ }
+ else
+ {
+ return counter.intValue();
+ }
+ }
+
+ // Private -------------------------------------------------------
+ private void fixDependency(final JournalFile originalFile, final JournalFile dependency) throws Exception
+ {
+ JournalReaderCallback txfix = new JournalReaderCallbackAbstract()
+ {
+ public void onReadCommitRecord(long transactionID, int numberOfRecords) throws Exception
+ {
+ if (transactionCounter.containsKey(transactionID))
+ {
+ dependency.incNegCount(originalFile);
+ }
+ }
+
+ public void onReadRollbackRecord(long transactionID) throws Exception
+ {
+ if (transactionCounter.containsKey(transactionID))
+ {
+ dependency.incNegCount(originalFile);
+ }
+ }
+ };
+
+ JournalImpl.readJournalFile(fileFactory, dependency, txfix);
+ }
+
+
+ // Inner classes -------------------------------------------------
+
+}
Modified: trunk/src/main/org/hornetq/core/journal/impl/JournalCompactor.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/JournalCompactor.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/src/main/org/hornetq/core/journal/impl/JournalCompactor.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -13,7 +13,6 @@
package org.hornetq.core.journal.impl;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedList;
@@ -29,9 +28,8 @@
import org.hornetq.core.journal.SequentialFileFactory;
import org.hornetq.core.journal.impl.JournalImpl.JournalRecord;
import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.spi.HornetQBuffer;
-import org.hornetq.utils.ConcurrentHashSet;
import org.hornetq.utils.DataConstants;
+import org.hornetq.utils.Pair;
/**
* A JournalCompactor
@@ -40,31 +38,11 @@
*
*
*/
-public class JournalCompactor implements JournalReaderCallback
+public class JournalCompactor extends AbstractJournalUpdateTask
{
- private static final String FILE_COMPACT_CONTROL = "journal-rename-control.ctr";
-
private static final Logger log = Logger.getLogger(JournalCompactor.class);
-
- private final JournalImpl journal;
-
- private final SequentialFileFactory fileFactory;
-
- private JournalFile currentFile;
-
- private SequentialFile sequentialFile;
-
- private int fileID;
-
- private ChannelBuffer writingChannel;
-
- private int nextOrderingID;
-
- private final List<JournalFile> newDataFiles = new ArrayList<JournalFile>();
-
- private final Set<Long> recordsSnapshot = new ConcurrentHashSet<Long>();;
-
+
// Snapshot of transactions that were pending when the compactor started
private final Map<Long, PendingTransaction> pendingTransactions = new ConcurrentHashMap<Long, PendingTransaction>();
@@ -77,71 +55,10 @@
* we cache those updates. As soon as we are done we take the right account. */
private final LinkedList<CompactCommand> pendingCommands = new LinkedList<CompactCommand>();
- /**
- * @param tmpRenameFile
- * @param files
- * @param newFiles
- */
- public static SequentialFile writeControlFile(final SequentialFileFactory fileFactory,
- final List<JournalFile> files,
- final List<JournalFile> newFiles) throws Exception
- {
-
- SequentialFile controlFile = fileFactory.createSequentialFile(FILE_COMPACT_CONTROL, 1);
-
- try
- {
- controlFile.open(1);
-
- ChannelBuffer renameBuffer = ChannelBuffers.dynamicBuffer(1);
-
- renameBuffer.writeInt(-1);
- renameBuffer.writeInt(-1);
-
- HornetQBuffer filesToRename = ChannelBuffers.dynamicBuffer(1);
-
- // DataFiles first
-
- filesToRename.writeInt(files.size());
-
- for (JournalFile file : files)
- {
- filesToRename.writeUTF(file.getFile().getFileName());
- }
-
- filesToRename.writeInt(newFiles.size());
-
- for (JournalFile file : newFiles)
- {
- filesToRename.writeUTF(file.getFile().getFileName());
- }
-
- JournalImpl.writeAddRecord(-1,
- 1,
- (byte)0,
- new JournalImpl.ByteArrayEncoding(filesToRename.array()),
- JournalImpl.SIZE_ADD_RECORD + filesToRename.array().length,
- renameBuffer);
-
- ByteBuffer writeBuffer = fileFactory.newBuffer(renameBuffer.writerIndex());
-
- writeBuffer.put(renameBuffer.array(), 0, renameBuffer.writerIndex());
-
- writeBuffer.rewind();
-
- controlFile.write(writeBuffer, true);
-
- return controlFile;
- }
- finally
- {
- controlFile.close();
- }
- }
-
public static SequentialFile readControlFile(final SequentialFileFactory fileFactory,
final List<String> dataFiles,
- final List<String> newFiles) throws Exception
+ final List<String> newFiles,
+ final List<Pair<String, String>> renameFile) throws Exception
{
SequentialFile controlFile = fileFactory.createSequentialFile(FILE_COMPACT_CONTROL, 1);
@@ -182,6 +99,14 @@
newFiles.add(input.readUTF());
}
+ int numberRenames = input.readInt();
+ for (int i = 0; i < numberRenames; i++)
+ {
+ String from = input.readUTF();
+ String to = input.readUTF();
+ renameFile.add(new Pair<String, String>(from, to));
+ }
+
}
return controlFile;
@@ -212,10 +137,7 @@
final Set<Long> recordsSnapshot,
final int firstFileID)
{
- this.fileFactory = fileFactory;
- this.journal = journal;
- this.recordsSnapshot.addAll(recordsSnapshot);
- nextOrderingID = firstFileID;
+ super(fileFactory, journal, recordsSnapshot, firstFileID);
}
/** This methods informs the Compactor about the existence of a pending (non committed) transaction */
@@ -248,7 +170,7 @@
{
for (long id : ids)
{
- recordsSnapshot.add(id);
+ addToRecordsSnaptsho(id);
}
}
@@ -256,7 +178,7 @@
{
for (long id : ids2)
{
- recordsSnapshot.add(id);
+ addToRecordsSnaptsho(id);
}
}
}
@@ -284,40 +206,20 @@
pendingCommands.add(new UpdateCompactCommand(id, usedFile, size));
}
- public boolean lookupRecord(final long id)
- {
- return recordsSnapshot.contains(id);
- }
-
private void checkSize(final int size) throws Exception
{
- if (writingChannel == null)
+ if (getWritingChannel() == null)
{
openFile();
}
else
{
- if (writingChannel.writerIndex() + size > writingChannel.capacity())
+ if (getWritingChannel().writerIndex() + size > getWritingChannel().capacity())
{
openFile();
}
}
}
-
- /** Write pending output into file */
- public void flush() throws Exception
- {
- if (writingChannel != null)
- {
- sequentialFile.position(0);
- sequentialFile.write(writingChannel.toByteBuffer(), true);
- sequentialFile.close();
- newDataFiles.add(currentFile);
- }
-
- writingChannel = null;
- }
-
/**
* Replay pending counts that happened during compacting
*/
@@ -342,7 +244,7 @@
public void onReadAddRecord(final RecordInfo info) throws Exception
{
- if (recordsSnapshot.contains(info.id))
+ if (lookupRecord(info.id))
{
int size = JournalImpl.SIZE_ADD_RECORD + info.data.length;
@@ -353,7 +255,7 @@
info.getUserRecordType(),
new JournalImpl.ByteArrayEncoding(info.data),
size,
- writingChannel);
+ getWritingChannel());
newRecords.put(info.id, new JournalRecord(currentFile, size));
}
@@ -377,7 +279,7 @@
info.getUserRecordType(),
new JournalImpl.ByteArrayEncoding(info.data),
size,
- writingChannel);
+ getWritingChannel());
}
else
{
@@ -388,6 +290,7 @@
public void onReadCommitRecord(final long transactionID, final int numberOfRecords) throws Exception
{
+
if (pendingTransactions.get(transactionID) != null)
{
// Sanity check, this should never happen
@@ -421,7 +324,7 @@
info.id,
new JournalImpl.ByteArrayEncoding(info.data),
size,
- writingChannel);
+ getWritingChannel());
newTransaction.addNegative(currentFile, info.id);
}
@@ -447,11 +350,10 @@
JournalImpl.writeTransaction(fileID,
JournalImpl.PREPARE_RECORD,
transactionID,
- newTransaction,
new JournalImpl.ByteArrayEncoding(extraData),
size,
newTransaction.getCounter(currentFile),
- writingChannel);
+ getWritingChannel());
newTransaction.prepare(currentFile);
@@ -470,7 +372,7 @@
public void onReadUpdateRecord(final RecordInfo info) throws Exception
{
- if (recordsSnapshot.contains(info.id))
+ if (lookupRecord(info.id))
{
int size = JournalImpl.SIZE_UPDATE_RECORD + info.data.length;
@@ -492,7 +394,7 @@
info.userRecordType,
new JournalImpl.ByteArrayEncoding(info.data),
size,
- writingChannel);
+ getWritingChannel());
}
}
@@ -513,7 +415,7 @@
info.userRecordType,
new JournalImpl.ByteArrayEncoding(info.data),
size,
- writingChannel);
+ getWritingChannel());
newTransaction.addPositive(currentFile, info.id, size);
}
@@ -538,26 +440,6 @@
return newTransaction;
}
- /**
- * @throws Exception
- */
- private void openFile() throws Exception
- {
- flush();
-
- ByteBuffer bufferWrite = fileFactory.newBuffer(journal.getFileSize());
- writingChannel = ChannelBuffers.wrappedBuffer(bufferWrite);
-
- currentFile = journal.getFile(false, false, false, true);
- sequentialFile = currentFile.getFile();
-
- sequentialFile.open(1);
- fileID = nextOrderingID++;
- currentFile = new JournalFileImpl(sequentialFile, fileID);
-
- writingChannel.writeInt(fileID);
- }
-
private static abstract class CompactCommand
{
abstract void execute() throws Exception;
@@ -599,7 +481,7 @@
private long id;
private JournalFile usedFile;
-
+
private final int size;
public UpdateCompactCommand(final long id, final JournalFile usedFile, final int size)
Modified: trunk/src/main/org/hornetq/core/journal/impl/JournalFile.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/JournalFile.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/src/main/org/hornetq/core/journal/impl/JournalFile.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -19,8 +19,6 @@
*
* A JournalFile
*
- * TODO combine this with JournalFileImpl
- *
* @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
*
*/
@@ -33,6 +31,8 @@
int getNegCount(JournalFile file);
void incNegCount(JournalFile file);
+
+ boolean resetNegCount(JournalFile file);
int getPosCount();
@@ -49,6 +49,10 @@
void setCanReclaim(boolean canDelete);
boolean isCanReclaim();
+
+ void setNeedCleanup(boolean needCleanup);
+
+ boolean isNeedCleanup();
long getOffset();
Modified: trunk/src/main/org/hornetq/core/journal/impl/JournalFileImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/JournalFileImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/src/main/org/hornetq/core/journal/impl/JournalFileImpl.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -44,6 +44,8 @@
private final AtomicInteger liveBytes = new AtomicInteger(0);
private boolean canReclaim;
+
+ private boolean needCleanup;
private final Map<JournalFile, AtomicInteger> negCounts = new ConcurrentHashMap<JournalFile, AtomicInteger>();
@@ -70,7 +72,18 @@
{
return canReclaim;
}
+
+ public boolean isNeedCleanup()
+ {
+ return needCleanup;
+ }
+ public void setNeedCleanup(boolean needCleanup)
+ {
+ this.needCleanup = needCleanup;
+ }
+
+
public void setCanReclaim(final boolean canReclaim)
{
this.canReclaim = canReclaim;
@@ -94,6 +107,11 @@
return count.intValue();
}
}
+
+ public boolean resetNegCount(JournalFile file)
+ {
+ return negCounts.remove(file) != null;
+ }
public void incPosCount()
{
Modified: trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/src/main/org/hornetq/core/journal/impl/JournalImpl.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -33,11 +33,11 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.hornetq.core.buffers.ChannelBuffer;
@@ -81,7 +81,7 @@
private static final Logger log = Logger.getLogger(JournalImpl.class);
- private static final boolean trace = log.isTraceEnabled();
+ private static final boolean trace = false;
/** This is to be set to true at DEBUG & development only */
private static final boolean LOAD_TRACE = false;
@@ -197,13 +197,16 @@
// This will be set only while the JournalCompactor is being executed
private volatile JournalCompactor compactor;
- // Latch used to wait compactor finish, to make sure we won't stop the journal with the compactor running
private final AtomicBoolean compactorRunning = new AtomicBoolean();
private ExecutorService filesExecutor = null;
+
+ private ExecutorService compactorExecutor = null;
// Lock used during the append of records
- private final Semaphore lockAppend = new Semaphore(1);
+ // This lock doesn't represent a global lock.
+ // After a record is appended, the usedFile can't be changed until the positives and negatives are updated
+ private final ReentrantLock lockAppend = new ReentrantLock();
/** We don't lock the journal while compacting, however we need to lock it while taking and updating snapshots */
private final ReadWriteLock compactingLock = new ReentrantReadWriteLock();
@@ -313,7 +316,6 @@
public static void writeTransaction(final int fileID,
final byte recordType,
final long txID,
- final JournalTransaction tx,
final EncodingSupport transactionData,
final int size,
final int numberOfRecords,
@@ -322,7 +324,7 @@
bb.writeByte(recordType);
bb.writeInt(fileID); // skip ID part
bb.writeLong(txID);
- bb.writeInt(numberOfRecords); // skip number of pendingTransactions part
+ bb.writeInt(numberOfRecords);
if (transactionData != null)
{
@@ -364,6 +366,18 @@
}
/**
+ * @param txID
+ * @param bb
+ */
+ public static void writeRollback(final int fileID, final long txID, ChannelBuffer bb)
+ {
+ bb.writeByte(ROLLBACK_RECORD);
+ bb.writeInt(fileID);
+ bb.writeLong(txID);
+ bb.writeInt(SIZE_ROLLBACK_RECORD);
+ }
+
+ /**
* @param id
* @param recordType
* @param record
@@ -410,6 +424,19 @@
}
/**
+ * @param id
+ * @param size
+ * @param bb
+ */
+ public static void writeDeleteRecord(final int fileId, final long id, int size, ChannelBuffer bb)
+ {
+ bb.writeByte(DELETE_RECORD);
+ bb.writeInt(fileId);
+ bb.writeLong(id);
+ bb.writeInt(size);
+ }
+
+ /**
* @param txID
* @param id
* @param record
@@ -836,7 +863,7 @@
callback = getSyncCallback(sync);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, false, sync, null, callback);
@@ -845,7 +872,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
finally
@@ -896,7 +923,7 @@
callback = getSyncCallback(sync);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, false, sync, null, callback);
@@ -914,7 +941,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
finally
@@ -956,14 +983,11 @@
ChannelBuffer bb = newBuffer(size);
- bb.writeByte(DELETE_RECORD);
- bb.writeInt(-1); // skip ID part
- bb.writeLong(id);
- bb.writeInt(size);
+ writeDeleteRecord(-1, id, size, bb);
callback = getSyncCallback(sync);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, false, sync, null, callback);
@@ -982,7 +1006,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
finally
@@ -1025,7 +1049,7 @@
JournalTransaction tx = getTransactionInfo(txID);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, false, false, tx, null);
@@ -1034,7 +1058,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
finally
@@ -1074,7 +1098,7 @@
JournalTransaction tx = getTransactionInfo(txID);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, false, false, tx, null);
@@ -1083,7 +1107,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
finally
@@ -1116,7 +1140,7 @@
JournalTransaction tx = getTransactionInfo(txID);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, false, false, tx, null);
@@ -1125,7 +1149,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
finally
@@ -1174,9 +1198,9 @@
int size = SIZE_COMPLETE_TRANSACTION_RECORD + transactionData.getEncodeSize() + DataConstants.SIZE_INT;
ChannelBuffer bb = newBuffer(size);
- writeTransaction(-1, PREPARE_RECORD, txID, tx, transactionData, size, -1, bb);
+ writeTransaction(-1, PREPARE_RECORD, txID, transactionData, size, -1, bb);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, true, sync, tx, null);
@@ -1185,7 +1209,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
@@ -1236,9 +1260,15 @@
ChannelBuffer bb = newBuffer(SIZE_COMPLETE_TRANSACTION_RECORD);
- writeTransaction(-1, COMMIT_RECORD, txID, tx, null, SIZE_COMPLETE_TRANSACTION_RECORD, -1, bb);
+ writeTransaction(-1,
+ COMMIT_RECORD,
+ txID,
+ null,
+ SIZE_COMPLETE_TRANSACTION_RECORD,
+ -1 /* number of records on this transaction will be filled later inside append record */,
+ bb);
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, true, sync, tx, null);
@@ -1247,7 +1277,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
@@ -1283,16 +1313,11 @@
throw new IllegalStateException("Cannot find tx with id " + txID);
}
- int size = SIZE_ROLLBACK_RECORD;
+ ChannelBuffer bb = newBuffer(SIZE_ROLLBACK_RECORD);
- ChannelBuffer bb = newBuffer(size);
+ writeRollback(-1, txID, bb);
- bb.writeByte(ROLLBACK_RECORD);
- bb.writeInt(-1); // skip ID part
- bb.writeLong(txID);
- bb.writeInt(size);
-
- lockAppend.acquire();
+ lockAppend.lock();
try
{
JournalFile usedFile = appendRecord(bb, false, sync, tx, null);
@@ -1301,7 +1326,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
@@ -1464,7 +1489,7 @@
// This is where most of the work is done, taking most of the time of the compacting routine.
// Notice there are no locks while this is being done.
-
+
// Read the files, and use the JournalCompactor class to create the new outputFiles, and the new collections as
// well
for (final JournalFile file : dataFilesToProcess)
@@ -1479,12 +1504,12 @@
// Usually tests will use this to hold the compacting while other structures are being updated.
onCompactDone();
- SequentialFile controlFile = createControlFile(dataFilesToProcess, compactor.getNewDataFiles());
-
List<JournalFile> newDatafiles = null;
JournalCompactor localCompactor = compactor;
+ SequentialFile controlFile = createControlFile(dataFilesToProcess, compactor.getNewDataFiles(), null);
+
compactingLock.writeLock().lock();
try
{
@@ -2005,18 +2030,21 @@
state = STATE_LOADED;
- checkAndReclaimFiles();
+ checkReclaimStatus();
return maxID;
}
- public void checkAndReclaimFiles() throws Exception
+ /**
+ * @return true if cleanup was called
+ */
+ public boolean checkReclaimStatus() throws Exception
{
- // We can't start compacting while compacting is working
+ // We can't start reclaim while compacting is working
compactingLock.readLock().lock();
try
{
- checkReclaimStatus();
+ reclaimer.scan(getDataFiles());
for (JournalFile file : dataFiles)
{
@@ -2037,13 +2065,145 @@
addFreeFile(file);
}
}
+
+ int nCleanup = 0;
+ for (JournalFile file : dataFiles)
+ {
+ if (file.isNeedCleanup())
+ {
+ nCleanup++;
+ }
+ }
+
+ // TODO: make this configurable
+ if (nCleanup > 5)
+ {
+ for (JournalFile file : dataFiles)
+ {
+ if (file.isNeedCleanup())
+ {
+ final JournalFile cleanupFile = file;
+
+ if (compactorRunning.compareAndSet(false, true))
+ {
+ // The cleanup should happen rarely.
+ // but when it happens it needs to use a different thread,
+ // or opening new files or any other executor's usage will be blocked while the cleanUp is being
+ // processed.
+
+ compactorExecutor.execute(new Runnable()
+ {
+ public void run()
+ {
+ try
+ {
+ cleanUp(cleanupFile);
+ }
+ catch (Exception e)
+ {
+ log.warn(e.getMessage(), e);
+ }
+ finally
+ {
+ compactorRunning.set(false);
+ if (autoReclaim)
+ {
+ scheduleReclaim();
+ }
+ }
+ }
+ });
+ }
+ return true;
+ }
+ }
+ }
}
finally
{
compactingLock.readLock().unlock();
}
+
+ return false;
}
+ public synchronized void cleanUp(final JournalFile file) throws Exception
+ {
+ if (state != STATE_LOADED)
+ {
+ return;
+ }
+
+ compactingLock.readLock().lock();
+
+ try
+ {
+ JournalCleaner cleaner = null;
+ ArrayList<JournalFile> dependencies = new ArrayList<JournalFile>();
+ lockAppend.lock();
+
+ try
+ {
+
+ log.info("Cleaning up file " + file);
+
+ if (file.getPosCount() == 0)
+ {
+ // nothing to be done
+ return;
+ }
+
+ // We don't want this file to be reclaimed during the cleanup
+ file.incPosCount();
+
+ // The file will have all the deleted records removed, so all the NegCount towards the file being cleaned up
+ // could be reset
+ for (JournalFile jrnFile : dataFiles)
+ {
+ if (jrnFile.resetNegCount(file))
+ {
+ dependencies.add(jrnFile);
+ jrnFile.incPosCount(); // this file can't be reclaimed while cleanup is being done
+ }
+ }
+
+ cleaner = new JournalCleaner(fileFactory, this, records.keySet(), file.getFileID());
+ }
+ finally
+ {
+ lockAppend.unlock();
+ }
+
+ readJournalFile(fileFactory, file, cleaner);
+
+ cleaner.flush();
+
+ cleaner.fixDependencies(file, dependencies);
+
+ for (JournalFile jrnfile : dependencies)
+ {
+ jrnfile.decPosCount();
+ }
+ file.decPosCount();
+
+ SequentialFile tmpFile = cleaner.currentFile.getFile();
+ String tmpFileName = tmpFile.getFileName();
+ String cleanedFileName = file.getFile().getFileName();
+
+ SequentialFile controlFile = createControlFile(null, null, new Pair<String, String>(tmpFileName,
+ cleanedFileName));
+ file.getFile().delete();
+ tmpFile.renameTo(cleanedFileName);
+ controlFile.delete();
+ }
+ finally
+ {
+ compactingLock.readLock().unlock();
+ log.info("Clean up on file " + file + " done");
+ }
+
+ }
+
public void checkCompact() throws Exception
{
if (compactMinFiles == 0)
@@ -2072,10 +2232,9 @@
return;
}
- // We can't use the executor for the compacting... or we would lock files opening and creation (besides other
- // operations)
- // that would freeze the journal while compacting
- Thread t = new Thread()
+ // We can't use the executor for the compacting... or we would dead lock because of file open and creation
+ // operations (that will use the executor)
+ compactorExecutor.execute(new Runnable()
{
public void run()
{
@@ -2093,9 +2252,7 @@
compactorRunning.set(false);
}
}
- };
-
- t.start();
+ });
}
}
@@ -2114,7 +2271,7 @@
public String debug() throws Exception
{
- checkReclaimStatus();
+ reclaimer.scan(getDataFiles());
StringBuilder builder = new StringBuilder();
@@ -2245,19 +2402,19 @@
compactingLock.readLock().lock();
try
{
- lockAppend.acquire();
+ lockAppend.lock();
try
{
moveNextFile(true);
if (autoReclaim)
{
- checkAndReclaimFiles();
+ checkReclaimStatus();
}
debugWait();
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
finally
@@ -2287,6 +2444,8 @@
}
filesExecutor = Executors.newSingleThreadExecutor();
+
+ compactorExecutor = Executors.newCachedThreadPool();
fileFactory.start();
@@ -2302,7 +2461,7 @@
throw new IllegalStateException("Journal is already stopped");
}
- lockAppend.acquire();
+ lockAppend.lock();
try
{
@@ -2337,7 +2496,7 @@
}
finally
{
- lockAppend.release();
+ lockAppend.unlock();
}
}
@@ -2347,6 +2506,23 @@
// Protected
// -----------------------------------------------------------------------------
+ protected SequentialFile createControlFile(List<JournalFile> files,
+ List<JournalFile> newFiles,
+ Pair<String, String> cleanupRename) throws Exception
+ {
+ ArrayList<Pair<String, String>> cleanupList;
+ if (cleanupRename == null)
+ {
+ cleanupList = null;
+ }
+ else
+ {
+ cleanupList = new ArrayList<Pair<String, String>>();
+ cleanupList.add(cleanupRename);
+ }
+ return AbstractJournalUpdateTask.writeControlFile(fileFactory, files, newFiles, cleanupList);
+ }
+
protected void deleteControlFile(final SequentialFile controlFile) throws Exception
{
controlFile.delete();
@@ -2374,14 +2550,6 @@
{
}
- /**
- * @throws Exception
- */
- protected SequentialFile createControlFile(final List<JournalFile> files, final List<JournalFile> newFiles) throws Exception
- {
- return JournalCompactor.writeControlFile(fileFactory, files, newFiles);
- }
-
// Private
// -----------------------------------------------------------------------------
@@ -2406,11 +2574,6 @@
}
}
- private void checkReclaimStatus() throws Exception
- {
- reclaimer.scan(getDataFiles());
- }
-
// Discard the old JournalFile and set it with a new ID
private JournalFile reinitializeFile(final JournalFile file) throws Exception
{
@@ -2535,7 +2698,7 @@
file.read(bb);
int fileID = bb.getInt();
-
+
fileFactory.releaseBuffer(bb);
bb = null;
@@ -2546,7 +2709,7 @@
}
int fileNameID = getFileNameID(fileName);
-
+
// The compactor could create a fileName but use a previously assigned ID.
// Because of that we need to take both parts into account
if (nextFileID.get() < fileNameID)
@@ -2554,7 +2717,6 @@
nextFileID.set(fileNameID);
}
-
orderedFiles.add(new JournalFileImpl(file, fileID));
file.close();
@@ -2594,7 +2756,7 @@
throw new IllegalArgumentException("Record is too large to store " + size);
}
- // Disable auto flush on the timer. The Timer should'nt flush anything
+ // Disable auto flush on the timer. The Timer should'nt flush anything
currentFile.getFile().disableAutoFlush();
if (!currentFile.getFile().fits(size))
@@ -2669,13 +2831,13 @@
}
}
-
+
/** Get the ID part of the name */
private int getFileNameID(String fileName)
{
try
{
- return Integer.parseInt(fileName.substring(filePrefix.length()+1, fileName.indexOf('.')));
+ return Integer.parseInt(fileName.substring(filePrefix.length() + 1, fileName.indexOf('.')));
}
catch (Throwable e)
{
@@ -2819,34 +2981,7 @@
if (autoReclaim && !synchronous)
{
- filesExecutor.execute(new Runnable()
- {
- public void run()
- {
- try
- {
- checkAndReclaimFiles();
- }
- catch (Exception e)
- {
- log.error(e.getMessage(), e);
- }
- }
- });
- filesExecutor.execute(new Runnable()
- {
- public void run()
- {
- try
- {
- checkCompact();
- }
- catch (Exception e)
- {
- log.error(e.getMessage(), e);
- }
- }
- });
+ scheduleReclaim();
}
JournalFile nextFile = null;
@@ -2863,6 +2998,32 @@
return nextFile;
}
+ private void scheduleReclaim()
+ {
+ if (state != STATE_LOADED)
+ {
+ return;
+ }
+
+ filesExecutor.execute(new Runnable()
+ {
+ public void run()
+ {
+ try
+ {
+ if (!checkReclaimStatus())
+ {
+ checkCompact();
+ }
+ }
+ catch (Exception e)
+ {
+ log.error(e.getMessage(), e);
+ }
+ }
+ });
+ }
+
/**
*
* Open a file and place it into the openedFiles queue
@@ -3002,8 +3163,9 @@
{
ArrayList<String> dataFiles = new ArrayList<String>();
ArrayList<String> newFiles = new ArrayList<String>();
+ ArrayList<Pair<String, String>> renames = new ArrayList<Pair<String, String>>();
- SequentialFile controlFile = JournalCompactor.readControlFile(fileFactory, dataFiles, newFiles);
+ SequentialFile controlFile = JournalCompactor.readControlFile(fileFactory, dataFiles, newFiles, renames);
if (controlFile != null)
{
for (String dataFile : dataFiles)
@@ -3026,6 +3188,19 @@
}
}
+ for (Pair<String, String> rename : renames)
+ {
+ SequentialFile fileTmp = fileFactory.createSequentialFile(rename.a, 1);
+ SequentialFile fileTo = fileFactory.createSequentialFile(rename.b, 1);
+ // We should do the rename only if the tmp file still exist, or else we could
+ // delete a valid file depending on where the crash occured during the control file delete
+ if (fileTmp.exists())
+ {
+ fileTo.delete();
+ fileTmp.renameTo(rename.b);
+ }
+ }
+
controlFile.delete();
}
@@ -3236,7 +3411,7 @@
{
try
{
- lockAppend.acquire();
+ lockAppend.lock();
HornetQBuffer bb = newBuffer(128 * 1024);
@@ -3245,7 +3420,7 @@
appendRecord(bb, false, false, null, null);
}
- lockAppend.release();
+ lockAppend.unlock();
}
catch (Exception e)
{
Modified: trunk/src/main/org/hornetq/core/journal/impl/JournalTransaction.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/JournalTransaction.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/src/main/org/hornetq/core/journal/impl/JournalTransaction.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -281,7 +281,7 @@
}
/**
- * The caller of this method needs to guarantee lock.acquire at the journal. (unless this is being called from load what is a single thread process).
+ * The caller of this method needs to guarantee appendLock.lock at the journal. (unless this is being called from load what is a single thread process).
* */
public void commit(final JournalFile file)
{
@@ -371,7 +371,7 @@
}
/**
- * The caller of this method needs to guarantee lock.acquire before calling this method if being used outside of the lock context.
+ * The caller of this method needs to guarantee appendLock.lock before calling this method if being used outside of the lock context.
* or else potFilesMap could be affected
* */
public void rollback(final JournalFile file)
@@ -402,7 +402,7 @@
}
/**
- * The caller of this method needs to guarantee lock.acquire before calling this method if being used outside of the lock context.
+ * The caller of this method needs to guarantee appendLock.lock before calling this method if being used outside of the lock context.
* or else potFilesMap could be affected
* */
public void prepare(final JournalFile file)
Modified: trunk/src/main/org/hornetq/core/journal/impl/Reclaimer.java
===================================================================
--- trunk/src/main/org/hornetq/core/journal/impl/Reclaimer.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/src/main/org/hornetq/core/journal/impl/Reclaimer.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -17,8 +17,6 @@
/**
*
- * <p>A ReclaimerTest</p>
- *
* <p>The journal consists of an ordered list of journal files Fn where 0 <= n <= N</p>
*
* <p>A journal file can contain either positives (pos) or negatives (neg)</p>
@@ -33,6 +31,7 @@
* which are also marked for deletion in the same pass of the algorithm.</p>
*
* @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
+ * @author <a href="mailto:clebert.suconic@jboss.com">Clebert Suconic</a>
*
*/
public class Reclaimer
@@ -54,6 +53,8 @@
JournalFile currentFile = files[i];
+ currentFile.setNeedCleanup(false);
+
int posCount = currentFile.getPosCount();
int totNeg = 0;
@@ -101,6 +102,7 @@
trace(currentFile + " Can't be reclaimed because " + file + " has negative values");
}
+ file.setNeedCleanup(true);
currentFile.setCanReclaim(false);
break;
Modified: trunk/tests/src/org/hornetq/tests/integration/client/CompactingTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/integration/client/CompactingTest.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/integration/client/CompactingTest.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -25,6 +25,7 @@
import org.hornetq.core.config.Configuration;
import org.hornetq.core.config.impl.ConfigurationImpl;
import org.hornetq.core.exception.HornetQException;
+import org.hornetq.core.message.Message;
import org.hornetq.core.server.HornetQServer;
import org.hornetq.core.server.JournalType;
import org.hornetq.tests.util.ServiceTestBase;
@@ -67,6 +68,101 @@
// Public --------------------------------------------------------
+ public void testCleanupAIO() throws Throwable
+ {
+ for (int i = 0; i < 3; i++)
+ {
+ System.out.println("Test # " + i);
+ internalTestCleanup(JournalType.ASYNCIO);
+ tearDown();
+ setUp();
+ }
+ }
+
+ public void testCleanupNIO() throws Throwable
+ {
+ for (int i = 0; i < 3; i++)
+ {
+ System.out.println("Test # " + i);
+ internalTestCleanup(JournalType.NIO);
+ tearDown();
+ setUp();
+ }
+ }
+
+ private void internalTestCleanup(JournalType journalType) throws Throwable
+ {
+ setupServer(journalType);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ ClientProducer prod = session.createProducer(AD1);
+
+ for (int i = 0; i < 500; i++)
+ {
+ prod.send(session.createClientMessage(true));
+ }
+
+ session.commit();
+
+ prod.close();
+
+ ClientConsumer cons = session.createConsumer(Q2);
+ prod = session.createProducer(AD2);
+
+ session.start();
+
+ for (int i = 0; i < 200; i++)
+ {
+ System.out.println("Iteration " + i);
+ for (int j = 0; j < 1000; j++)
+ {
+ Message msg = session.createClientMessage(true);
+ msg.getBody().writeBytes(new byte[1024]);
+
+ prod.send(msg);
+ }
+
+ session.commit();
+
+ for (int j = 0; j < 1000; j++)
+ {
+ ClientMessage msg = cons.receive(2000);
+ assertNotNull(msg);
+ msg.acknowledge();
+ }
+
+ session.commit();
+
+ }
+
+ assertNull(cons.receiveImmediate());
+
+ session.close();
+
+ server.stop();
+
+ server.start();
+
+ session = sf.createSession(false, true, true);
+ cons = session.createConsumer(Q1);
+ session.start();
+
+ for (int i = 0; i < 500; i++)
+ {
+ ClientMessage msg = cons.receive(1000);
+ assertNotNull(msg);
+ msg.acknowledge();
+ }
+
+ assertNull(cons.receiveImmediate());
+
+ prod = session.createProducer(AD2);
+
+ session.close();
+
+ }
+
public void testMultiProducerAndCompactAIO() throws Throwable
{
internalTestMultiProducer(JournalType.ASYNCIO);
@@ -107,11 +203,11 @@
{
session.close();
}
-
+
server.stop();
-
+
setupServer(journalType);
-
+
final AtomicInteger numberOfMessages = new AtomicInteger(0);
final int NUMBER_OF_FAST_MESSAGES = 100000;
final int SLOW_INTERVAL = 100;
@@ -292,7 +388,7 @@
assertNotNull(msg);
msg.acknowledge();
}
-
+
assertNull(cons.receiveImmediate());
}
@@ -327,7 +423,7 @@
config.setJournalType(journalType);
- config.setJournalCompactMinFiles(3);
+ config.setJournalCompactMinFiles(10);
config.setJournalCompactPercentage(50);
server = createServer(true, config);
@@ -363,16 +459,27 @@
}
sess.close();
-
- sf = createInVMFactory();
}
@Override
protected void tearDown() throws Exception
{
- sf.close();
+ try
+ {
+ if (sf != null)
+ {
+ sf.close();
+ }
- server.stop();
+ if (server != null)
+ {
+ server.stop();
+ }
+ }
+ catch (Exception e)
+ {
+ e.printStackTrace(); // system.out -> junit reports
+ }
server = null;
Modified: trunk/tests/src/org/hornetq/tests/integration/journal/NIOJournalCompactTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/integration/journal/NIOJournalCompactTest.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/integration/journal/NIOJournalCompactTest.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -68,17 +68,26 @@
SequentialFile file = fileFactory.createSequentialFile("file-" + i + ".tst.new", 1);
newFiles.add(new JournalFileImpl(file, 0));
}
+
+ ArrayList<Pair<String, String>> renames = new ArrayList<Pair<String, String>>();
+ renames.add(new Pair<String, String>("a", "b"));
+ renames.add(new Pair<String, String>("c", "d"));
+
+
- JournalCompactor.writeControlFile(fileFactory, dataFiles, newFiles);
+ JournalCompactor.writeControlFile(fileFactory, dataFiles, newFiles, renames);
ArrayList<String> strDataFiles = new ArrayList<String>();
ArrayList<String> strNewFiles = new ArrayList<String>();
+
+ ArrayList<Pair<String, String>> renamesRead = new ArrayList<Pair<String, String>>();
- assertNotNull(JournalCompactor.readControlFile(fileFactory, strDataFiles, strNewFiles));
+ assertNotNull(JournalCompactor.readControlFile(fileFactory, strDataFiles, strNewFiles, renamesRead));
assertEquals(dataFiles.size(), strDataFiles.size());
assertEquals(newFiles.size(), strNewFiles.size());
+ assertEquals(renames.size(), renamesRead.size());
Iterator<String> iterDataFiles = strDataFiles.iterator();
for (JournalFile file : dataFiles)
@@ -94,6 +103,16 @@
}
assertFalse(iterNewFiles.hasNext());
+
+ Iterator<Pair<String,String>> iterRename = renames.iterator();
+ for (Pair<String,String> rename : renamesRead)
+ {
+ Pair<String, String> original = iterRename.next();
+ assertEquals(original.a, rename.a);
+ assertEquals(original.b, rename.b);
+ }
+ assertFalse(iterNewFiles.hasNext());
+
}
public void testCrashRenamingFiles() throws Exception
@@ -197,11 +216,11 @@
{
@Override
- protected SequentialFile createControlFile(List<JournalFile> files, List<JournalFile> newFiles) throws Exception
+ protected SequentialFile createControlFile(List<JournalFile> files, List<JournalFile> newFiles, Pair<String, String> pair) throws Exception
{
if (createControlFile)
{
- return super.createControlFile(files, newFiles);
+ return super.createControlFile(files, newFiles, pair);
}
else
{
@@ -517,7 +536,7 @@
journal.forceMoveNextFile();
- journal.checkAndReclaimFiles();
+ journal.checkReclaimStatus();
}
long transactionID = 0;
@@ -666,9 +685,9 @@
long id = idGenerator.generateID();
listToDelete.add(id);
+ // Append Record Transaction will make the recordSize as exactly recordLength (discounting SIZE_ADD_RECORD_TX)
addTx(tx, id);
- // Append Record Transaction will make the recordSize as exactly recordLength (discounting SIZE_ADD_RECORD_TX)
expectedSizes.add(recordLength);
journal.forceMoveNextFile();
Modified: trunk/tests/src/org/hornetq/tests/stress/journal/AddAndRemoveStressTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/stress/journal/AddAndRemoveStressTest.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/stress/journal/AddAndRemoveStressTest.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -204,7 +204,7 @@
}
impl.forceMoveNextFile();
- impl.checkAndReclaimFiles();
+ impl.checkReclaimStatus();
impl.stop();
Modified: trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/AlignedJournalImplTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/AlignedJournalImplTest.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/AlignedJournalImplTest.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -156,7 +156,7 @@
journalImpl.forceMoveNextFile();
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
setupAndLoadJournal(JOURNAL_SIZE, 10);
@@ -270,7 +270,7 @@
journalImpl.setAutoReclaim(false);
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
journalImpl.debugWait();
@@ -316,7 +316,7 @@
journalImpl.setAutoReclaim(false);
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
journalImpl.debugWait();
@@ -356,7 +356,7 @@
assertEquals(1000, records.get(0).id);
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
log.debug(journalImpl.debug());
@@ -484,7 +484,7 @@
assertEquals(10, records.size());
assertEquals(0, transactions.size());
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(10, journalImpl.getDataFilesCount());
@@ -504,7 +504,7 @@
journalImpl.appendAddRecord(101, (byte)1, new SimpleEncoding(5, (byte)1), false);
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(1, journalImpl.getDataFilesCount());
@@ -595,7 +595,7 @@
assertEquals(0, records.size());
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(0, journalImpl.getDataFilesCount());
@@ -660,7 +660,7 @@
assertEquals(20, records.size());
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
}
@@ -695,7 +695,7 @@
journalImpl.forceMoveNextFile();
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
setupAndLoadJournal(JOURNAL_SIZE, 100, 2);
@@ -756,7 +756,7 @@
assertEquals(0, records.size());
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(0, journalImpl.getDataFilesCount());
@@ -798,7 +798,7 @@
// the
// file
journalImpl.forceMoveNextFile();
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
setupAndLoadJournal(JOURNAL_SIZE, 100);
@@ -836,7 +836,7 @@
journalImpl.appendCommitRecord(2l, false);
journalImpl.forceMoveNextFile();
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
setupAndLoadJournal(JOURNAL_SIZE, 100);
@@ -931,7 +931,7 @@
assertEquals((byte)1, transactions.get(0).extraData[i]);
}
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(10, journalImpl.getDataFilesCount());
@@ -943,7 +943,7 @@
assertEquals(10, records.size());
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
for (int i = 0; i < 10; i++)
{
@@ -980,7 +980,7 @@
// Reclaiming should still be able to reclaim a file if a transaction was
// ignored
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(2, factory.listFiles("tt").size());
@@ -1053,7 +1053,7 @@
journalImpl.forceMoveNextFile();
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(0, journalImpl.getDataFilesCount());
@@ -1147,7 +1147,7 @@
journalImpl.forceMoveNextFile();
journalImpl.debugWait();
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(0, transactions.size());
assertEquals(0, journalImpl.getDataFilesCount());
@@ -1238,7 +1238,7 @@
journalImpl.debugWait();
- journalImpl.checkAndReclaimFiles();
+ journalImpl.checkReclaimStatus();
assertEquals(0, journalImpl.getDataFilesCount());
Modified: trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestBase.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestBase.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestBase.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -104,7 +104,7 @@
protected void checkAndReclaimFiles() throws Exception
{
journal.debugWait();
- journal.checkAndReclaimFiles();
+ journal.checkReclaimStatus();
journal.debugWait();
}
Modified: trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestUnit.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestUnit.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/JournalImplTestUnit.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -1057,7 +1057,7 @@
// Now restart
- journal.checkAndReclaimFiles();
+ journal.checkReclaimStatus();
System.out.println("journal:" + journal.debug());
@@ -3094,7 +3094,7 @@
System.out.println("*****************************************");
journal.forceMoveNextFile();
- journal.checkAndReclaimFiles();
+ journal.checkReclaimStatus();
assertEquals(0, journal.getDataFilesCount());
Modified: trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/ReclaimerTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/ReclaimerTest.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/unit/core/journal/impl/ReclaimerTest.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -354,6 +354,23 @@
assertCanDelete(1);
assertCantDelete(2);
}
+
+ public void testCleanup() throws Exception
+ {
+ setup(3);
+ setupPosNeg(0, 11, 0, 0, 0);
+ setupPosNeg(1, 1, 10, 0, 0);
+ setupPosNeg(2, 1, 0, 1, 0);
+
+ reclaimer.scan(files);
+
+ debugFiles();
+
+ assertCantDelete(0);
+ assertTrue(files[0].isNeedCleanup());
+ assertCantDelete(1);
+ assertCantDelete(2);
+ }
public void testThreeFiles10() throws Exception
{
@@ -708,6 +725,18 @@
}
}
}
+
+ private void debugFiles()
+ {
+ for (int i = 0 ; i < files.length; i++)
+ {
+ System.out.println("[" + i + "]=" + files[i].getPosCount() + ", canDelete = " + files[i].isCanReclaim() + ", cleanup = " + files[i].isNeedCleanup());
+ for (int j = 0 ; j <= i; j++)
+ {
+ System.out.println("..." + files[i].getNegCount(files[j]));
+ }
+ }
+ }
private void assertCanDelete(final int... fileNumber)
{
@@ -738,7 +767,10 @@
private int posCount;
private boolean canDelete;
+
+ private boolean needCleanup;
+
public void extendOffset(final int delta)
{
}
@@ -913,5 +945,30 @@
{
return 0;
}
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalFile#isNeedCleanup()
+ */
+ public boolean isNeedCleanup()
+ {
+ return this.needCleanup;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalFile#resetNegCount(org.hornetq.core.journal.impl.JournalFile)
+ */
+ public boolean resetNegCount(JournalFile file)
+ {
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.journal.impl.JournalFile#setNeedCleanup(boolean)
+ */
+ public void setNeedCleanup(boolean needCleanup)
+ {
+ this.needCleanup = needCleanup;
+
+ }
}
}
Modified: trunk/tests/src/org/hornetq/tests/util/ListJournal.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/util/ListJournal.java 2009-09-16 16:46:12 UTC (rev 7962)
+++ trunk/tests/src/org/hornetq/tests/util/ListJournal.java 2009-09-16 21:29:29 UTC (rev 7963)
@@ -83,7 +83,7 @@
System.out.println("user record: " + record);
}
- journal.checkAndReclaimFiles();
+ journal.checkReclaimStatus();
System.out.println("Data = " + journal.debug());
16 years, 3 months
JBoss hornetq SVN: r7962 - in branches/Branch_Replication_Changes: src/main/org/hornetq/core/client/impl and 19 other directories.
by do-not-reply@jboss.org
Author: timfox
Date: 2009-09-16 12:46:12 -0400 (Wed, 16 Sep 2009)
New Revision: 7962
Added:
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTestBase.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomReattachTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomReattachTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReattachTest.java
Removed:
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/AutomaticFailoverWithDiscoveryTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverExpiredMessageTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverManagementTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverPreAcknowledgeTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverScheduledMessageTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTestBase.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailureOnCreateConnectionTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/JustReplicationTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/LargeMessageMultiThreadFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTestBase.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/OrderingOnBackupTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverMultiThreadTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PreserveOrderDuringFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectWithBackupTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReplicateConnectionFailureTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleAutomaticFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleManualFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SplitBrainTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XALargeMessageMultiThreadFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XAMultiThreadRandomFailoverTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/LargeMessageMultiThreadFailoverStressTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/PagingFailoverStressTest.java
Modified:
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSession.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSessionFactory.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerInternal.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientProducerImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionFactoryImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionInternal.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionPacketHandler.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ConnectionManagerImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/DelegatingSession.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/exception/HornetQException.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AcceptorControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AddressControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BridgeControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BroadcastGroupControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/ClusterConnectionControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DiscoveryGroupControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DivertControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/QueueControlImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/persistence/impl/journal/JournalStorageManager.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/ChannelImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/RemotingConnectionImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/invm/InVMConnection.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/HornetQExceptionMessage.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/SessionCreateConsumerMessage.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/HornetQ.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/cluster/impl/BridgeImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/QueueImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerConsumerImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java
branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQObjectMessage.java
branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQSession.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadFailoverSupport.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/paging/PageCrashTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/soak/failover/RandomFailoverSoakTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/MultiThreadRandomFailoverStressTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/RandomFailoverStressTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/client/impl/LargeMessageBufferTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/paging/impl/PagingManagerImplTest.java
branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/util/ServiceTestBase.java
Log:
replication changes
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSession.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSession.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSession.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -66,6 +66,8 @@
ClientConsumer createConsumer(SimpleString queueName, SimpleString filterString) throws HornetQException;
ClientConsumer createConsumer(SimpleString queueName, SimpleString filterString, boolean browseOnly) throws HornetQException;
+
+ ClientConsumer createConsumer(SimpleString queueName, boolean browseOnly) throws HornetQException;
ClientConsumer createConsumer(SimpleString queueName,
SimpleString filterString,
@@ -78,6 +80,8 @@
ClientConsumer createConsumer(String queueName, String filterString) throws HornetQException;
ClientConsumer createConsumer(String queueName, String filterString, boolean browseOnly) throws HornetQException;
+
+ ClientConsumer createConsumer(String queueName, boolean browseOnly) throws HornetQException;
ClientConsumer createConsumer(String queueName, String filterString, int windowSize, int maxRate, boolean browseOnly) throws HornetQException;
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSessionFactory.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSessionFactory.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/ClientSessionFactory.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -27,15 +27,19 @@
public interface ClientSessionFactory
{
ClientSession createXASession() throws HornetQException;
-
+
ClientSession createTransactedSession() throws HornetQException;
-
+
ClientSession createSession() throws HornetQException;
-
+
ClientSession createSession(boolean autoCommitSends, boolean autoCommitAcks) throws HornetQException;
+
+ ClientSession createSession(boolean autoCommitSends, boolean autoCommitAcks, int ackBatchSize) throws HornetQException;
ClientSession createSession(boolean xa, boolean autoCommitSends, boolean autoCommitAcks) throws HornetQException;
+ ClientSession createSession(boolean xa, boolean autoCommitSends, boolean autoCommitAcks, boolean preAcknowledge) throws HornetQException;
+
ClientSession createSession(String username,
String password,
boolean xa,
@@ -44,18 +48,17 @@
boolean preAcknowledge,
int ackBatchSize) throws HornetQException;
- ClientSession createSession(boolean xa, boolean autoCommitSends, boolean autoCommitAcks, boolean preAcknowledge) throws HornetQException;
-
+
List<Pair<TransportConfiguration, TransportConfiguration>> getStaticConnectors();
-
+
void setStaticConnectors(List<Pair<TransportConfiguration, TransportConfiguration>> staticConnectors);
-
+
long getClientFailureCheckPeriod();
void setClientFailureCheckPeriod(long clientFailureCheckPeriod);
-
+
boolean isCacheLargeMessagesClient();
-
+
void setCacheLargeMessagesClient(boolean cached);
long getConnectionTTL();
@@ -71,7 +74,7 @@
void setMaxConnections(int maxConnections);
int getMinLargeMessageSize();
-
+
void setMinLargeMessageSize(int minLargeMessageSize);
int getConsumerWindowSize();
@@ -97,9 +100,9 @@
boolean isBlockOnPersistentSend();
void setBlockOnPersistentSend(boolean blockOnPersistentSend);
-
+
boolean isBlockOnNonPersistentSend();
-
+
void setBlockOnNonPersistentSend(boolean blockOnNonPersistentSend);
boolean isAutoGroup();
@@ -145,19 +148,19 @@
boolean isFailoverOnServerShutdown();
void setFailoverOnServerShutdown(boolean failoverOnServerShutdown);
-
+
String getConnectionLoadBalancingPolicyClassName();
void setConnectionLoadBalancingPolicyClassName(String loadBalancingPolicyClassName);
-
- String getDiscoveryAddress();
+ String getDiscoveryAddress();
+
void setDiscoveryAddress(String discoveryAddress);
int getDiscoveryPort();
void setDiscoveryPort(int discoveryPort);
-
+
long getDiscoveryRefreshTimeout();
void setDiscoveryRefreshTimeout(long discoveryRefreshTimeout);
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -29,6 +29,7 @@
import org.hornetq.core.remoting.impl.wireformat.SessionReceiveContinuationMessage;
import org.hornetq.core.remoting.impl.wireformat.SessionReceiveMessage;
import org.hornetq.utils.Future;
+import org.hornetq.utils.SimpleString;
import org.hornetq.utils.TokenBucketLimiter;
/**
@@ -60,7 +61,13 @@
private final Channel channel;
private final long id;
-
+
+ private final SimpleString filterString;
+
+ private final SimpleString queueName;
+
+ private boolean browseOnly;
+
private final Executor sessionExecutor;
private final int clientWindowSize;
@@ -106,6 +113,9 @@
public ClientConsumerImpl(final ClientSessionInternal session,
final long id,
+ final SimpleString queueName,
+ final SimpleString filterString,
+ final boolean browseOnly,
final int clientWindowSize,
final int ackBatchSize,
final TokenBucketLimiter rateLimiter,
@@ -113,6 +123,12 @@
final Channel channel)
{
this.id = id;
+
+ this.queueName = queueName;
+
+ this.filterString = filterString;
+
+ this.browseOnly = browseOnly;
this.channel = channel;
@@ -329,7 +345,7 @@
public void stop() throws HornetQException
{
waitForOnMessageToComplete();
-
+
synchronized (this)
{
if (stopped)
@@ -340,6 +356,15 @@
stopped = true;
}
}
+
+ public void clearAtFailover()
+ {
+ clearBuffer();
+
+ lastAckedMessage = null;
+
+ creditsToSend = 0;
+ }
public synchronized void start()
{
@@ -360,9 +385,24 @@
{
return id;
}
+
+ public SimpleString getFilterString()
+ {
+ return filterString;
+ }
+ public SimpleString getQueueName()
+ {
+ return queueName;
+ }
+
+ public boolean isBrowseOnly()
+ {
+ return browseOnly;
+ }
+
public synchronized void handleMessage(final ClientMessageInternal message) throws Exception
- {
+ {
if (closing)
{
// This is ok - we just ignore the message
@@ -446,14 +486,14 @@
flowControlBeforeConsumption(message);
}
- buffer.clear();
+ clearBuffer();
}
// Need to send credits for the messages in the buffer
waitForOnMessageToComplete();
}
-
+
public int getClientWindowSize()
{
return clientWindowSize;
@@ -584,7 +624,7 @@
* @param credits
*/
private void sendCredits(final int credits)
- {
+ {
channel.send(new SessionConsumerFlowCreditMessage(id, credits));
}
@@ -755,7 +795,7 @@
}
session.removeConsumer(this);
- }
+ }
private void clearBuffer()
{
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerInternal.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerInternal.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientConsumerInternal.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -18,6 +18,7 @@
import org.hornetq.core.exception.HornetQException;
import org.hornetq.core.remoting.impl.wireformat.SessionReceiveContinuationMessage;
import org.hornetq.core.remoting.impl.wireformat.SessionReceiveMessage;
+import org.hornetq.utils.SimpleString;
/**
*
@@ -29,6 +30,12 @@
public interface ClientConsumerInternal extends ClientConsumer
{
long getID();
+
+ SimpleString getQueueName();
+
+ SimpleString getFilterString();
+
+ boolean isBrowseOnly();
void handleMessage(ClientMessageInternal message) throws Exception;
@@ -39,6 +46,8 @@
void flowControl(final int messageBytes, final boolean discountSlowConsumer) throws HornetQException;
void clear() throws HornetQException;
+
+ void clearAtFailover();
int getClientWindowSize();
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientProducerImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientProducerImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientProducerImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -215,6 +215,8 @@
boolean sendBlocking = msg.isDurable() ? blockOnPersistentSend : blockOnNonPersistentSend;
SessionSendMessage message = new SessionSendMessage(msg, sendBlocking);
+
+ session.workDone();
if (msg.getBodyInputStream() != null || msg.getEncodeSize() >= minLargeMessageSize || msg.isLargeMessage())
{
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionFactoryImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionFactoryImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionFactoryImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -726,7 +726,20 @@
preAcknowledge,
ackBatchSize);
}
+
+
+ public ClientSession createSession(boolean autoCommitSends, boolean autoCommitAcks, int ackBatchSize) throws HornetQException
+ {
+ return createSessionInternal(null,
+ null,
+ false,
+ autoCommitSends,
+ autoCommitAcks,
+ preAcknowledge,
+ ackBatchSize);
+ }
+
public ClientSession createXASession() throws HornetQException
{
return createSessionInternal(null, null, true, false, false, preAcknowledge, this.ackBatchSize);
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -12,13 +12,14 @@
*/
package org.hornetq.core.client.impl;
+import static org.hornetq.core.exception.HornetQException.TRANSACTION_ROLLED_BACK;
import static org.hornetq.utils.SimpleString.toSimpleString;
import java.util.HashSet;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executor;
import javax.transaction.xa.XAException;
@@ -38,9 +39,9 @@
import org.hornetq.core.remoting.Packet;
import org.hornetq.core.remoting.RemotingConnection;
import org.hornetq.core.remoting.impl.wireformat.CreateQueueMessage;
+import org.hornetq.core.remoting.impl.wireformat.CreateSessionMessage;
+import org.hornetq.core.remoting.impl.wireformat.CreateSessionResponseMessage;
import org.hornetq.core.remoting.impl.wireformat.PacketImpl;
-import org.hornetq.core.remoting.impl.wireformat.ReattachSessionMessage;
-import org.hornetq.core.remoting.impl.wireformat.ReattachSessionResponseMessage;
import org.hornetq.core.remoting.impl.wireformat.RollbackMessage;
import org.hornetq.core.remoting.impl.wireformat.SessionAcknowledgeMessage;
import org.hornetq.core.remoting.impl.wireformat.SessionBindingQueryMessage;
@@ -68,6 +69,7 @@
import org.hornetq.core.remoting.impl.wireformat.SessionXASetTimeoutMessage;
import org.hornetq.core.remoting.impl.wireformat.SessionXASetTimeoutResponseMessage;
import org.hornetq.core.remoting.impl.wireformat.SessionXAStartMessage;
+import org.hornetq.core.remoting.spi.Connection;
import org.hornetq.core.remoting.spi.HornetQBuffer;
import org.hornetq.utils.ConcurrentHashSet;
import org.hornetq.utils.IDGenerator;
@@ -107,6 +109,10 @@
private final String name;
+ private final String username;
+
+ private final String password;
+
private final boolean xa;
private final Executor executor;
@@ -115,7 +121,8 @@
private final Set<ClientProducerInternal> producers = new ConcurrentHashSet<ClientProducerInternal>();
- private final Map<Long, ClientConsumerInternal> consumers = new ConcurrentHashMap<Long, ClientConsumerInternal>();
+ // Consumers must be an ordered map so if we fail we recreate them in the same order with the same ids
+ private final Map<Long, ClientConsumerInternal> consumers = new LinkedHashMap<Long, ClientConsumerInternal>();
private volatile boolean closed;
@@ -135,6 +142,8 @@
private final int consumerMaxRate;
+ private final int producerWindowSize;
+
private final int producerMaxRate;
private final boolean blockOnNonPersistentSend;
@@ -158,12 +167,18 @@
private SendAcknowledgementHandler sendAckHandler;
- private volatile boolean closedSent;
+ // private volatile boolean closedSent;
+ private volatile boolean rollbackOnly;
+
+ private volatile boolean workDone;
+
// Constructors ----------------------------------------------------------------------------
public ClientSessionImpl(final ConnectionManager connectionManager,
final String name,
+ final String username,
+ final String password,
final boolean xa,
final boolean autoCommitSends,
final boolean autoCommitAcks,
@@ -173,6 +188,7 @@
final int ackBatchSize,
final int consumerWindowSize,
final int consumerMaxRate,
+ final int producerWindowSize,
final int producerMaxRate,
final boolean blockOnNonPersistentSend,
final boolean blockOnPersistentSend,
@@ -187,6 +203,10 @@
this.name = name;
+ this.username = username;
+
+ this.password = password;
+
this.remotingConnection = remotingConnection;
this.executor = executor;
@@ -213,6 +233,8 @@
this.consumerMaxRate = consumerMaxRate;
+ this.producerWindowSize = producerWindowSize;
+
this.producerMaxRate = producerMaxRate;
this.blockOnNonPersistentSend = blockOnNonPersistentSend;
@@ -337,11 +359,22 @@
{
return createConsumer(queueName, filterString, consumerWindowSize, consumerMaxRate, browseOnly);
}
+
+ public ClientConsumer createConsumer(final SimpleString queueName,
+ final boolean browseOnly) throws HornetQException
+ {
+ return createConsumer(queueName, null, consumerWindowSize, consumerMaxRate, browseOnly);
+ }
public ClientConsumer createConsumer(final String queueName, final String filterString, final boolean browseOnly) throws HornetQException
{
return createConsumer(toSimpleString(queueName), toSimpleString(filterString), browseOnly);
}
+
+ public ClientConsumer createConsumer(final String queueName, final boolean browseOnly) throws HornetQException
+ {
+ return createConsumer(toSimpleString(queueName), null, browseOnly);
+ }
/*
* Note, we DO NOT currently support direct consumers (i.e. consumers we're delivery occurs on the remoting thread.
@@ -419,9 +452,17 @@
{
checkClosed();
+ if (rollbackOnly)
+ {
+ throw new HornetQException(TRANSACTION_ROLLED_BACK,
+ "The transaction was rolled back on failover to a backup server");
+ }
+
flushAcks();
channel.sendBlocking(new PacketImpl(PacketImpl.SESS_COMMIT));
+
+ workDone = false;
}
public void rollback() throws HornetQException
@@ -429,7 +470,7 @@
rollback(false);
}
- public void rollback(final boolean isLastMessageAsDelived) throws HornetQException
+ public void rollback(final boolean isLastMessageAsDelivered) throws HornetQException
{
checkClosed();
@@ -454,7 +495,7 @@
// Acks must be flushed here *after connection is stopped and all onmessages finished executing
flushAcks();
- channel.sendBlocking(new RollbackMessage(isLastMessageAsDelived));
+ channel.sendBlocking(new RollbackMessage(isLastMessageAsDelivered));
if (wasStarted)
{
@@ -527,7 +568,7 @@
if (!started)
{
for (ClientConsumerInternal clientConsumerInternal : consumers.values())
- {
+ {
clientConsumerInternal.start();
}
@@ -617,7 +658,7 @@
{
checkClosed();
- //We don't send expiries for pre-ack since message will already have been acked on server
+ // We don't send expiries for pre-ack since message will already have been acked on server
if (!preAcknowledge)
{
SessionExpiredMessage message = new SessionExpiredMessage(consumerID, messageID);
@@ -638,7 +679,7 @@
public void removeConsumer(final ClientConsumerInternal consumer) throws HornetQException
{
- consumers.remove(consumer.getID());
+ consumers.remove(consumer.getID());
}
public void removeProducer(final ClientProducerInternal producer)
@@ -658,8 +699,11 @@
{
log.trace("Setting up flowControlSize to " + message.getRequiredBufferSize() + " on message = " + clMessage);
}
+
clMessage.setFlowControlSize(message.getRequiredBufferSize());
+ workDone();
+
consumer.handleMessage(message.getClientMessage());
}
}
@@ -670,6 +714,8 @@
if (consumer != null)
{
+ workDone();
+
consumer.handleLargeMessage(message);
}
}
@@ -680,6 +726,8 @@
if (consumer != null)
{
+ workDone();
+
consumer.handleLargeMessageContinuation(continuation);
}
}
@@ -695,7 +743,7 @@
{
closeChildren();
- closedSent = true;
+ // closedSent = true;
channel.sendBlocking(new SessionCloseMessage());
}
@@ -726,6 +774,81 @@
sendAckHandler = handler;
}
+ // // Needs to be synchronized to prevent issues with occurring concurrently with close()
+ // public synchronized boolean handleOldFailover(final RemotingConnection backupConnection)
+ // {
+ // if (closed)
+ // {
+ // return true;
+ // }
+ //
+ // boolean ok = false;
+ //
+ // // We lock the channel to prevent any packets to be added to the resend
+ // // cache during the failover process
+ // channel.lock();
+ //
+ // try
+ // {
+ // channel.transferConnection(backupConnection);
+ //
+ // backupConnection.syncIDGeneratorSequence(remotingConnection.getIDGeneratorSequence());
+ //
+ // remotingConnection = backupConnection;
+ //
+ // Packet request = new ReattachSessionMessage(name, channel.getLastReceivedCommandID());
+ //
+ // Channel channel1 = backupConnection.getChannel(1, -1, false);
+ //
+ // ReattachSessionResponseMessage response = (ReattachSessionResponseMessage)channel1.sendBlocking(request);
+ //
+ // if (response.isSessionFound())
+ // {
+ // channel.replayCommands(response.getLastReceivedCommandID(), channel.getID());
+ //
+ // ok = true;
+ // }
+ // else
+ // {
+ // if (closedSent)
+ // {
+ // // a session re-attach may fail, if the session close was sent before failover started, hit the server,
+ // // processed, then before the response was received back, failover occurred, re-attach was attempted. in
+ // // this case it's ok - we don't want to call any failure listeners and we don't want to halt the rest of
+ // // the failover process.
+ // //
+ // // however if session re-attach fails and the session was not in a call to close, then we DO want to call
+ // // the session listeners so we return false
+ // //
+ // // Also session reattach will fail if the server is restarted - so the session is lost
+ // ok = true;
+ // }
+ // else
+ // {
+ // log.warn(System.identityHashCode(this) + " Session not found on server when attempting to re-attach");
+ // }
+ //
+ // channel.returnBlocking();
+ // }
+ //
+ // }
+ // catch (Throwable t)
+ // {
+ // log.error("Failed to handle failover", t);
+ // }
+ // finally
+ // {
+ // channel.unlock();
+ // }
+ //
+ // return ok;
+ // }
+
+ public void workDone()
+ {
+ workDone = true;
+ }
+
// Needs to be synchronized to prevent issues with occurring concurrently with close()
public synchronized boolean handleFailover(final RemotingConnection backupConnection)
{
@@ -733,55 +856,130 @@
{
return true;
}
+
+ log.info("session handling failover");
boolean ok = false;
- // We lock the channel to prevent any packets to be added to the resend
- // cache during the failover process
+ // Need to stop all consumers outside the lock
+ for (ClientConsumerInternal consumer : consumers.values())
+ {
+ try
+ {
+ consumer.stop();
+ }
+ catch (HornetQException e)
+ {
+ log.error("Failed to stop consumer", e);
+ }
+
+ consumer.clearAtFailover();
+ }
+
+ // We lock the channel to prevent any packets being sent during the failover process
channel.lock();
try
{
channel.transferConnection(backupConnection);
- backupConnection.syncIDGeneratorSequence(remotingConnection.getIDGeneratorSequence());
-
remotingConnection = backupConnection;
- Packet request = new ReattachSessionMessage(name, channel.getLastReceivedCommandID());
+ Packet request = new CreateSessionMessage(name,
+ channel.getID(),
+ version,
+ username,
+ password,
+ minLargeMessageSize,
+ xa,
+ autoCommitSends,
+ autoCommitAcks,
+ preAcknowledge,
+ producerWindowSize);
Channel channel1 = backupConnection.getChannel(1, -1, false);
- ReattachSessionResponseMessage response = (ReattachSessionResponseMessage)channel1.sendBlocking(request);
+ CreateSessionResponseMessage response = (CreateSessionResponseMessage)channel1.sendBlocking(request);
- if (!response.isRemoved())
+ if (response.isCreated())
{
- channel.replayCommands(response.getLastReceivedCommandID(), channel.getID());
+ // Session was created ok
- ok = true;
- }
- else
- {
- if (closedSent)
+ // Now we need to recreate the consumers
+
+ for (Map.Entry<Long, ClientConsumerInternal> entry : consumers.entrySet())
{
- // a session re-attach may fail, if the session close was sent before failover started, hit the server,
- // processed, then before the response was received back, failover occurred, re-attach was attempted. in
- // this case it's ok - we don't want to call any failure listeners and we don't want to halt the rest of
- // the failover process.
- //
- // however if session re-attach fails and the session was not in a call to close, then we DO want to call
- // the session listeners so we return false
- //
- // Also session reattach will fail if the server is restarted - so the session is lost
- ok = true;
+ SessionCreateConsumerMessage createConsumerRequest = new SessionCreateConsumerMessage(entry.getKey(),
+ entry.getValue().getQueueName(),
+ entry.getValue().getFilterString(),
+ entry.getValue().isBrowseOnly(),
+ false);
+
+ createConsumerRequest.setChannelID(channel.getID());
+
+ Connection conn = channel.getConnection().getTransportConnection();
+
+ HornetQBuffer buffer = conn.createBuffer(createConsumerRequest.getRequiredBufferSize());
+
+ createConsumerRequest.encode(buffer);
+
+ conn.write(buffer, false);
+
+ int clientWindowSize = calcWindowSize(entry.getValue().getClientWindowSize());
+
+ if (clientWindowSize != 0)
+ {
+ SessionConsumerFlowCreditMessage packet = new SessionConsumerFlowCreditMessage(entry.getKey(), clientWindowSize);
+
+ packet.setChannelID(channel.getID());
+
+ buffer = conn.createBuffer(packet.getRequiredBufferSize());
+
+ packet.encode(buffer);
+
+ conn.write(buffer, false);
+ }
}
- else
+
+ if ((!autoCommitAcks || !autoCommitSends) && workDone)
{
- log.warn(System.identityHashCode(this) + " Session not found on server when attempting to re-attach");
+ // Session is transacted - set for rollback only
+
+ // FIXME - there is a race condition here - a commit could sneak in before this is set
+ rollbackOnly = true;
}
- channel.returnBlocking();
+ // Now start the session if it was already started
+ if (started)
+ {
+ for (ClientConsumerInternal consumer : consumers.values())
+ {
+ consumer.start();
+ }
+
+ Packet packet = new PacketImpl(PacketImpl.SESS_START);
+
+ packet.setChannelID(channel.getID());
+
+ Connection conn = channel.getConnection().getTransportConnection();
+
+ HornetQBuffer buffer = conn.createBuffer(packet.getRequiredBufferSize());
+
+ packet.encode(buffer);
+
+ conn.write(buffer, false);
+ }
+
+ ok = true;
}
+ else
+ {
+ // This means the server we failed onto is not ready to take new sessions - perhaps it hasn't actually
+ // failed over
+ }
+
+ // We cause any blocking calls to return - since they won't get responses.
+ channel.returnBlocking();
}
catch (Throwable t)
{
@@ -823,6 +1021,11 @@
public void commit(final Xid xid, final boolean onePhase) throws XAException
{
checkXA();
+
+ if (rollbackOnly)
+ {
+ throw new XAException(XAException.XA_RBOTHER);
+ }
// Note - don't need to flush acks since the previous end would have
// done this
@@ -833,6 +1036,8 @@
{
SessionXAResponseMessage response = (SessionXAResponseMessage)channel.sendBlocking(packet);
+ workDone = false;
+
if (response.isError())
{
throw new XAException(response.getResponseCode());
@@ -848,6 +1053,12 @@
public void end(final Xid xid, final int flags) throws XAException
{
checkXA();
+
+ if (rollbackOnly)
+ {
+ throw new XAException(XAException.XA_RBOTHER);
+ }
+
try
{
Packet packet;
@@ -945,6 +1156,11 @@
{
checkXA();
+ if (rollbackOnly)
+ {
+ throw new XAException(XAException.XA_RBOTHER);
+ }
+
// Note - don't need to flush acks since the previous end would have
// done this
@@ -1028,6 +1244,8 @@
start();
}
+ workDone = false;
+
if (response.isError())
{
throw new XAException(response.getResponseCode());
@@ -1133,30 +1351,8 @@
// Private
// ----------------------------------------------------------------------------
- /**
- * @param queueName
- * @param filterString
- * @param windowSize
- * @param browseOnly
- * @return
- * @throws HornetQException
- */
- private ClientConsumer internalCreateConsumer(final SimpleString queueName,
- final SimpleString filterString,
- final int windowSize,
- final int maxRate,
- final boolean browseOnly) throws HornetQException
+ private int calcWindowSize(final int windowSize)
{
- checkClosed();
-
- SessionCreateConsumerMessage request = new SessionCreateConsumerMessage(queueName, filterString, browseOnly);
-
- channel.sendBlocking(request);
-
- // The actual windows size that gets used is determined by the user since
- // could be overridden on the queue settings
- // The value we send is just a hint
-
int clientWindowSize;
if (windowSize == -1)
{
@@ -1183,11 +1379,43 @@
{
throw new IllegalArgumentException("Invalid window size " + windowSize);
}
-
+
+ return clientWindowSize;
+ }
+
+ /**
+ * @param queueName
+ * @param filterString
+ * @param windowSize
+ * @param browseOnly
+ * @return
+ * @throws HornetQException
+ */
+ private ClientConsumer internalCreateConsumer(final SimpleString queueName,
+ final SimpleString filterString,
+ final int windowSize,
+ final int maxRate,
+ final boolean browseOnly) throws HornetQException
+ {
+ checkClosed();
+
long consumerID = idGenerator.generateID();
+ SessionCreateConsumerMessage request = new SessionCreateConsumerMessage(consumerID, queueName, filterString, browseOnly, true);
+
+ channel.sendBlocking(request);
+
+ // The actual windows size that gets used is determined by the user since
+ // could be overridden on the queue settings
+ // The value we send is just a hint
+
+ int clientWindowSize = calcWindowSize(windowSize);
+
ClientConsumerInternal consumer = new ClientConsumerImpl(this,
consumerID,
+ queueName,
+ filterString,
+ browseOnly,
clientWindowSize,
ackBatchSize,
consumerMaxRate > 0 ? new TokenBucketLimiterImpl(maxRate,
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionInternal.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionInternal.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionInternal.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -64,4 +64,6 @@
void setForceNotSameRM(boolean force);
ConnectionManager getConnectionManager();
+
+ void workDone();
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionPacketHandler.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionPacketHandler.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ClientSessionPacketHandler.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -58,6 +58,7 @@
case SESS_RECEIVE_CONTINUATION:
{
SessionReceiveContinuationMessage continuation = (SessionReceiveContinuationMessage)packet;
+
clientSession.handleReceiveContinuation(continuation.getConsumerID(), continuation);
break;
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ConnectionManagerImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ConnectionManagerImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/ConnectionManagerImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -351,6 +351,8 @@
ClientSessionInternal session = new ClientSessionImpl(this,
name,
+ username,
+ password,
xa,
autoCommitSends,
autoCommitAcks,
@@ -360,6 +362,7 @@
ackBatchSize,
consumerWindowSize,
consumerMaxRate,
+ producerWindowSize,
producerMaxRate,
blockOnNonPersistentSend,
blockOnPersistentSend,
@@ -638,31 +641,24 @@
connection.destroy();
}
- closeConnectionsAndCallFailureListeners(me);
+ closeAllConnections();
}
}
else
{
- closeConnectionsAndCallFailureListeners(me);
+ closeAllConnections();
}
+
+ //We always call the failure listeners
+ callFailureListeners(me);
}
}
- private void closeConnectionsAndCallFailureListeners(final HornetQException me)
+ private void closeAllConnections()
{
refCount = 0;
mapIterator = null;
- checkCloseConnections();
-
- // TODO (after beta5) should really execute on different thread then remove the async in HornetQConnection
-
- // threadPool.execute(new Runnable()
- // {
- // public void run()
- // {
- callFailureListeners(me);
- // }
- // });
+ checkCloseConnections();
}
private void callFailureListeners(final HornetQException me)
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/DelegatingSession.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/DelegatingSession.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/client/impl/DelegatingSession.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -43,7 +43,7 @@
*
*/
public class DelegatingSession implements ClientSessionInternal
-{
+{
private static final Logger log = Logger.getLogger(DelegatingSession.class);
private final ClientSessionInternal session;
@@ -184,7 +184,17 @@
{
return session.createConsumer(queueName);
}
+
+ public ClientConsumer createConsumer(SimpleString queueName, boolean browseOnly) throws HornetQException
+ {
+ return session.createConsumer(queueName, browseOnly);
+ }
+ public ClientConsumer createConsumer(String queueName, boolean browseOnly) throws HornetQException
+ {
+ return session.createConsumer(queueName, browseOnly);
+ }
+
public ClientProducer createProducer() throws HornetQException
{
return session.createProducer();
@@ -465,4 +475,9 @@
{
session.setForceNotSameRM(force);
}
+
+ public void workDone()
+ {
+ session.workDone();
+ }
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/exception/HornetQException.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/exception/HornetQException.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/exception/HornetQException.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -59,6 +59,8 @@
public static final int SESSION_EXISTS = 109;
public static final int LARGE_MESSAGE_ERROR_BODY = 110;
+
+ public static final int TRANSACTION_ROLLED_BACK = 111;
// Native Error codes ----------------------------------------------
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AcceptorControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AcceptorControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AcceptorControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -15,6 +15,8 @@
import java.util.Map;
+import javax.management.StandardMBean;
+
import org.hornetq.core.config.TransportConfiguration;
import org.hornetq.core.management.AcceptorControl;
import org.hornetq.core.remoting.spi.Acceptor;
@@ -26,7 +28,7 @@
*
* Created 11 dec. 2008 17:09:04
*/
-public class AcceptorControlImpl implements AcceptorControl
+public class AcceptorControlImpl extends StandardMBean implements AcceptorControl
{
// Constants -----------------------------------------------------
@@ -42,7 +44,9 @@
// Constructors --------------------------------------------------
public AcceptorControlImpl(final Acceptor acceptor, final TransportConfiguration configuration)
+ throws Exception
{
+ super(AcceptorControl.class);
this.acceptor = acceptor;
this.configuration = configuration;
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AddressControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AddressControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/AddressControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -16,6 +16,8 @@
import java.util.Iterator;
import java.util.Set;
+import javax.management.StandardMBean;
+
import org.hornetq.core.logging.Logger;
import org.hornetq.core.management.AddressControl;
import org.hornetq.core.postoffice.Binding;
@@ -34,7 +36,7 @@
* @version <tt>$Revision$</tt>
*
*/
-public class AddressControlImpl implements AddressControl
+public class AddressControlImpl extends StandardMBean implements AddressControl
{
// Constants -----------------------------------------------------
@@ -54,9 +56,11 @@
// Constructors --------------------------------------------------
public AddressControlImpl(final SimpleString address,
- final PostOffice postOffice,
- final HierarchicalRepository<Set<Role>> securityRepository)
+ final PostOffice postOffice,
+ final HierarchicalRepository<Set<Role>> securityRepository)
+ throws Exception
{
+ super(AddressControl.class);
this.address = address;
this.postOffice = postOffice;
this.securityRepository = securityRepository;
@@ -110,7 +114,7 @@
}
return objRoles;
}
-
+
public String getRolesAsJSON() throws Exception
{
JSONArray json = new JSONArray();
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BridgeControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BridgeControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BridgeControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -13,6 +13,8 @@
package org.hornetq.core.management.impl;
+import javax.management.StandardMBean;
+
import org.hornetq.core.config.cluster.BridgeConfiguration;
import org.hornetq.core.management.BridgeControl;
import org.hornetq.core.server.cluster.Bridge;
@@ -24,7 +26,7 @@
*
* Created 11 dec. 2008 17:09:04
*/
-public class BridgeControlImpl implements BridgeControl
+public class BridgeControlImpl extends StandardMBean implements BridgeControl
{
// Constants -----------------------------------------------------
@@ -40,7 +42,9 @@
// Constructors --------------------------------------------------
public BridgeControlImpl(final Bridge bridge, final BridgeConfiguration configuration)
+ throws Exception
{
+ super(BridgeControl.class);
this.bridge = bridge;
this.configuration = configuration;
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BroadcastGroupControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BroadcastGroupControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/BroadcastGroupControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -13,6 +13,8 @@
package org.hornetq.core.management.impl;
+import javax.management.StandardMBean;
+
import org.hornetq.core.config.cluster.BroadcastGroupConfiguration;
import org.hornetq.core.management.BroadcastGroupControl;
import org.hornetq.core.server.cluster.BroadcastGroup;
@@ -27,7 +29,7 @@
*
* Created 11 dec. 2008 17:09:04
*/
-public class BroadcastGroupControlImpl implements BroadcastGroupControl
+public class BroadcastGroupControlImpl extends StandardMBean implements BroadcastGroupControl
{
// Constants -----------------------------------------------------
@@ -43,7 +45,9 @@
// Constructors --------------------------------------------------
public BroadcastGroupControlImpl(final BroadcastGroup broadcastGroup, final BroadcastGroupConfiguration configuration)
+ throws Exception
{
+ super(BroadcastGroupControl.class);
this.broadcastGroup = broadcastGroup;
this.configuration = configuration;
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/ClusterConnectionControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/ClusterConnectionControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/ClusterConnectionControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -13,6 +13,8 @@
package org.hornetq.core.management.impl;
+import javax.management.StandardMBean;
+
import org.hornetq.core.config.cluster.ClusterConnectionConfiguration;
import org.hornetq.core.management.ClusterConnectionControl;
import org.hornetq.core.server.cluster.ClusterConnection;
@@ -25,7 +27,7 @@
*
* @author <a href="jmesnil(a)redhat.com">Jeff Mesnil</a>
*/
-public class ClusterConnectionControlImpl implements ClusterConnectionControl
+public class ClusterConnectionControlImpl extends StandardMBean implements ClusterConnectionControl
{
// Constants -----------------------------------------------------
@@ -41,8 +43,9 @@
// Constructors --------------------------------------------------
public ClusterConnectionControlImpl(final ClusterConnection clusterConnection,
- ClusterConnectionConfiguration configuration)
+ ClusterConnectionConfiguration configuration) throws Exception
{
+ super(ClusterConnectionControl.class);
this.clusterConnection = clusterConnection;
this.configuration = configuration;
}
@@ -63,7 +66,7 @@
{
return configuration.getMaxHops();
}
-
+
public String getName()
{
return configuration.getName();
@@ -77,25 +80,26 @@
public Object[] getStaticConnectorNamePairs()
{
Object[] ret = new Object[configuration.getStaticConnectorNamePairs().size()];
-
+
int i = 0;
- for (Pair<String, String> pair: configuration.getStaticConnectorNamePairs())
+ for (Pair<String, String> pair : configuration.getStaticConnectorNamePairs())
{
String[] opair = new String[2];
-
+
opair[0] = pair.a;
opair[1] = pair.b != null ? pair.b : null;
-
+
ret[i++] = opair;
}
-
- return ret;
+
+ return ret;
}
- public String getStaticConnectorNamePairsAsJSON() throws Exception {
+ public String getStaticConnectorNamePairsAsJSON() throws Exception
+ {
JSONArray array = new JSONArray();
- for (Pair<String, String> pair: configuration.getStaticConnectorNamePairs())
+ for (Pair<String, String> pair : configuration.getStaticConnectorNamePairs())
{
JSONObject p = new JSONObject();
p.put("a", pair.a);
@@ -104,7 +108,7 @@
}
return array.toString();
}
-
+
public boolean isDuplicateDetection()
{
return configuration.isDuplicateDetection();
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DiscoveryGroupControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DiscoveryGroupControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DiscoveryGroupControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -13,6 +13,8 @@
package org.hornetq.core.management.impl;
+import javax.management.StandardMBean;
+
import org.hornetq.core.cluster.DiscoveryGroup;
import org.hornetq.core.config.cluster.DiscoveryGroupConfiguration;
import org.hornetq.core.management.DiscoveryGroupControl;
@@ -24,7 +26,7 @@
*
* Created 11 dec. 2008 17:09:04
*/
-public class DiscoveryGroupControlImpl implements DiscoveryGroupControl
+public class DiscoveryGroupControlImpl extends StandardMBean implements DiscoveryGroupControl
{
// Constants -----------------------------------------------------
@@ -40,7 +42,9 @@
// Constructors --------------------------------------------------
public DiscoveryGroupControlImpl(final DiscoveryGroup acceptor, final DiscoveryGroupConfiguration configuration)
+ throws Exception
{
+ super(DiscoveryGroupControl.class);
this.discoveryGroup = acceptor;
this.configuration = configuration;
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DivertControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DivertControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/DivertControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -13,6 +13,8 @@
package org.hornetq.core.management.impl;
+import javax.management.StandardMBean;
+
import org.hornetq.core.config.cluster.DivertConfiguration;
import org.hornetq.core.management.DivertControl;
import org.hornetq.core.server.Divert;
@@ -24,7 +26,7 @@
*
* Created 11 dec. 2008 17:09:04
*/
-public class DivertControlImpl implements DivertControl
+public class DivertControlImpl extends StandardMBean implements DivertControl
{
// Constants -----------------------------------------------------
@@ -42,7 +44,9 @@
// DivertControlMBean implementation ---------------------------
public DivertControlImpl(final Divert divert, final DivertConfiguration configuration)
+ throws Exception
{
+ super(DivertControl.class);
this.divert = divert;
this.configuration = configuration;
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -30,6 +30,7 @@
import javax.management.NotificationEmitter;
import javax.management.NotificationFilter;
import javax.management.NotificationListener;
+import javax.management.StandardMBean;
import javax.transaction.xa.Xid;
import org.hornetq.core.config.Configuration;
@@ -58,7 +59,7 @@
* @version <tt>$Revision$</tt>
*
*/
-public class HornetQServerControlImpl implements HornetQServerControl, NotificationEmitter
+public class HornetQServerControlImpl extends StandardMBean implements HornetQServerControl, NotificationEmitter
{
// Constants -----------------------------------------------------
@@ -94,6 +95,7 @@
final MessageCounterManager messageCounterManager,
final NotificationBroadcasterSupport broadcaster) throws Exception
{
+ super(HornetQServerControl.class);
this.postOffice = postOffice;
this.configuration = configuration;
this.resourceManager = resourceManager;
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/QueueControlImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/QueueControlImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/management/impl/QueueControlImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -16,6 +16,8 @@
import java.util.List;
import java.util.Map;
+import javax.management.StandardMBean;
+
import org.hornetq.core.exception.HornetQException;
import org.hornetq.core.filter.Filter;
import org.hornetq.core.filter.impl.FilterImpl;
@@ -40,7 +42,7 @@
* @version <tt>$Revision$</tt>
*
*/
-public class QueueControlImpl implements QueueControl
+public class QueueControlImpl extends StandardMBean implements QueueControl
{
// Constants -----------------------------------------------------
@@ -65,11 +67,11 @@
for (int i = 0; i < messages.length; i++)
{
Map<String, Object> message = messages[i];
- array.put(new JSONObject(message));
+ array.put(new JSONObject(message));
}
return array.toString();
}
-
+
/**
* Returns null if the string is null or empty
*/
@@ -78,7 +80,8 @@
if (filterStr == null || filterStr.trim().length() == 0)
{
return null;
- } else
+ }
+ else
{
return new FilterImpl(new SimpleString(filterStr));
}
@@ -87,10 +90,12 @@
// Constructors --------------------------------------------------
public QueueControlImpl(final Queue queue,
- final String address,
- final PostOffice postOffice,
- final HierarchicalRepository<AddressSettings> addressSettingsRepository)
+ final String address,
+ final PostOffice postOffice,
+ final HierarchicalRepository<AddressSettings> addressSettingsRepository)
+ throws Exception
{
+ super(QueueControl.class);
this.queue = queue;
this.address = address;
this.postOffice = postOffice;
@@ -206,12 +211,12 @@
AddressSettings addressSettings = addressSettingsRepository.getMatch(address);
SimpleString sExpiryAddress = new SimpleString(expiryAddress);
-
+
if (expiryAddress != null)
{
addressSettings.setExpiryAddress(sExpiryAddress);
}
-
+
queue.setExpiryAddress(sExpiryAddress);
}
@@ -227,14 +232,14 @@
}
return messages;
}
-
+
public String listScheduledMessagesAsJSON() throws Exception
{
return toJSON(listScheduledMessages());
}
public Map<String, Object>[] listMessages(final String filterStr) throws Exception
- {
+ {
try
{
Filter filter = createFilter(filterStr);
@@ -253,7 +258,7 @@
throw new IllegalStateException(e.getMessage());
}
}
-
+
public String listMessagesAsJSON(String filter) throws Exception
{
return toJSON(listMessages(filter));
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/persistence/impl/journal/JournalStorageManager.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/persistence/impl/journal/JournalStorageManager.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/persistence/impl/journal/JournalStorageManager.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -94,7 +94,7 @@
public static final int SIZE_FIELDS = SIZE_INT + SIZE_LONG + SIZE_LONG + SIZE_BYTE;
// Message journal record types
-
+
public static final byte ADD_LARGE_MESSAGE = 30;
public static final byte ADD_MESSAGE = 31;
@@ -130,15 +130,15 @@
private final boolean syncNonTransactional;
private final int perfBlastPages;
-
+
private final boolean createBindingsDir;
-
+
private final String bindingsDir;
-
+
private final boolean createJournalDir;
-
+
private final String journalDir;
-
+
private final String largeMessagesDirectory;
public JournalStorageManager(final Configuration config, final Executor executor)
@@ -156,7 +156,7 @@
{
throw new NullPointerException("bindings-dir is null");
}
-
+
createBindingsDir = config.isCreateBindingsDir();
journalDir = config.getJournalDirectory();
@@ -165,7 +165,7 @@
{
throw new NullPointerException("journal-dir is null");
}
-
+
createJournalDir = config.isCreateJournalDir();
SequentialFileFactory bindingsFF = new NIOSequentialFileFactory(bindingsDir);
@@ -212,8 +212,7 @@
{
throw new IllegalArgumentException("Unsupported journal type " + config.getJournalType());
}
-
-
+
this.idGenerator = new BatchingIDGenerator(0, CHECKPOINT_BATCH_SIZE, bindingsJournal);
messageJournal = new JournalImpl(config.getJournalFileSize(),
@@ -275,7 +274,7 @@
throw new HornetQException(HornetQException.ILLEGAL_STATE, "MessageId was not assigned to Message");
}
- // Note that we don't sync, the add reference that comes immediately after will sync
+ // Note that we don't sync, the add reference that comes immediately after will sync if appropriate
if (message.isLargeMessage())
{
@@ -350,7 +349,7 @@
}
}
-
+
public void storePageTransaction(final long txID, final PageTransactionInfo pageTransaction) throws Exception
{
if (pageTransaction.getRecordID() != 0)
@@ -476,7 +475,7 @@
List<PreparedTransactionInfo> preparedTransactions = new ArrayList<PreparedTransactionInfo>();
messageJournal.load(records, preparedTransactions);
-
+
Map<Long, ServerMessage> messages = new HashMap<Long, ServerMessage>();
Map<Long, Map<Long, AddMessageRecord>> queueMap = new HashMap<Long, Map<Long, AddMessageRecord>>();
@@ -498,29 +497,29 @@
LargeMessageEncoding messageEncoding = new LargeMessageEncoding(largeMessage);
messageEncoding.decode(buff);
-
+
Long originalMessageID = (Long)largeMessage.getProperties().getProperty(MessageImpl.HDR_ORIG_MESSAGE_ID);
-
+
// Using the linked file by the original file
if (originalMessageID != null)
{
LargeServerMessage originalMessage = (LargeServerMessage)messages.get(originalMessageID);
-
+
if (originalMessage == null)
{
- // this could happen if the message was deleted but the file still exists as the file still being used
- originalMessage = createLargeMessage();
+ // this could happen if the message was deleted but the file still exists as the file still being
+ // used
+ originalMessage = createLargeMessage();
originalMessage.setMessageID(originalMessageID);
originalMessage.setComplete(true);
messages.put(originalMessageID, originalMessage);
}
-
+
originalMessage.incrementRefCount();
-
+
largeMessage.setLinkedMessage(originalMessage);
largeMessage.setComplete(true);
}
-
messages.put(record.id, largeMessage);
@@ -951,7 +950,7 @@
bindingEncoding.setPersistenceID(id);
- queueBindingInfos.add(bindingEncoding);
+ queueBindingInfos.add(bindingEncoding);
}
else if (rec == PERSISTENT_ID_RECORD)
{
@@ -985,7 +984,7 @@
checkAndCreateDir(bindingsDir, createBindingsDir);
checkAndCreateDir(journalDir, createJournalDir);
-
+
checkAndCreateDir(largeMessagesDirectory, createJournalDir);
cleanupIncompleteFiles();
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/ChannelImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/ChannelImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/ChannelImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -194,6 +194,7 @@
{
lock.unlock();
}
+
// Must block on semaphore outside the main lock or this can prevent failover from occurring, also after the
// packet is sent to assure we get some credits back
if (sendSemaphore != null && packet.getType() != PACKETS_CONFIRMED)
@@ -207,7 +208,6 @@
throw new IllegalStateException("Semaphore interrupted");
}
}
-
}
}
@@ -293,8 +293,12 @@
if (response.getType() == PacketImpl.EXCEPTION)
{
final HornetQExceptionMessage mem = (HornetQExceptionMessage)response;
+
+ HornetQException e = mem.getException();
+
+ e.fillInStackTrace();
- throw mem.getException();
+ throw e;
}
}
finally
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/RemotingConnectionImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/RemotingConnectionImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/RemotingConnectionImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -65,8 +65,6 @@
private volatile boolean destroyed;
- // private volatile boolean active;
-
private final boolean client;
// Channels 0-9 are reserved for the system
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/invm/InVMConnection.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/invm/InVMConnection.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/invm/InVMConnection.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -123,7 +123,7 @@
if (!closed)
{
buffer.readInt(); // read and discard
-
+
handler.bufferReceived(id, buffer);
}
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/HornetQExceptionMessage.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/HornetQExceptionMessage.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/HornetQExceptionMessage.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -14,6 +14,7 @@
package org.hornetq.core.remoting.impl.wireformat;
import org.hornetq.core.exception.HornetQException;
+import org.hornetq.core.logging.Logger;
import org.hornetq.core.remoting.spi.HornetQBuffer;
import org.hornetq.utils.DataConstants;
@@ -27,7 +28,10 @@
public class HornetQExceptionMessage extends PacketImpl
{
// Constants -----------------------------------------------------
+
+ private static final Logger log = Logger.getLogger(HornetQExceptionMessage.class);
+
// Attributes ----------------------------------------------------
private HornetQException exception;
@@ -39,7 +43,7 @@
public HornetQExceptionMessage(final HornetQException exception)
{
super(EXCEPTION);
-
+
this.exception = exception;
}
@@ -76,6 +80,7 @@
{
int code = buffer.readInt();
String msg = buffer.readNullableString();
+
exception = new HornetQException(code, msg);
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/SessionCreateConsumerMessage.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/SessionCreateConsumerMessage.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/remoting/impl/wireformat/SessionCreateConsumerMessage.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -28,25 +28,33 @@
// Attributes ----------------------------------------------------
+ private long id;
+
private SimpleString queueName;
private SimpleString filterString;
private boolean browseOnly;
+
+ private boolean requiresResponse;
// Static --------------------------------------------------------
// Constructors --------------------------------------------------
- public SessionCreateConsumerMessage(final SimpleString queueName,
+ public SessionCreateConsumerMessage(final long id,
+ final SimpleString queueName,
final SimpleString filterString,
- final boolean browseOnly)
+ final boolean browseOnly,
+ final boolean requiresResponse)
{
super(SESS_CREATECONSUMER);
+ this.id = id;
this.queueName = queueName;
this.filterString = filterString;
this.browseOnly = browseOnly;
+ this.requiresResponse = requiresResponse;
}
public SessionCreateConsumerMessage()
@@ -65,6 +73,11 @@
buff.append("]");
return buff.toString();
}
+
+ public long getID()
+ {
+ return id;
+ }
public SimpleString getQueueName()
{
@@ -80,28 +93,37 @@
{
return browseOnly;
}
+
+ public boolean isRequiresResponse()
+ {
+ return requiresResponse;
+ }
public int getRequiredBufferSize()
{
- return BASIC_PACKET_SIZE + queueName.sizeof() +
+ return BASIC_PACKET_SIZE + DataConstants.SIZE_LONG + queueName.sizeof() +
SimpleString.sizeofNullableString(filterString) +
- DataConstants.SIZE_BOOLEAN;
+ 2 * DataConstants.SIZE_BOOLEAN ;
}
@Override
public void encodeBody(final HornetQBuffer buffer)
{
+ buffer.writeLong(id);
buffer.writeSimpleString(queueName);
buffer.writeNullableSimpleString(filterString);
buffer.writeBoolean(browseOnly);
+ buffer.writeBoolean(requiresResponse);
}
@Override
public void decodeBody(final HornetQBuffer buffer)
{
+ id = buffer.readLong();
queueName = buffer.readSimpleString();
filterString = buffer.readNullableSimpleString();
browseOnly = buffer.readBoolean();
+ requiresResponse = buffer.readBoolean();
}
@Override
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/HornetQ.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/HornetQ.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/HornetQ.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -41,9 +41,9 @@
HornetQSecurityManager securityManager = new HornetQSecurityManagerImpl();
HornetQServer server = newHornetQServer(config,
- ManagementFactory.getPlatformMBeanServer(),
- securityManager,
- enablePersistence);
+ ManagementFactory.getPlatformMBeanServer(),
+ securityManager,
+ enablePersistence);
return server;
}
@@ -54,8 +54,8 @@
}
public static HornetQServer newHornetQServer(final Configuration config,
- final MBeanServer mbeanServer,
- final boolean enablePersistence)
+ final MBeanServer mbeanServer,
+ final boolean enablePersistence)
{
HornetQSecurityManager securityManager = new HornetQSecurityManagerImpl();
@@ -70,8 +70,8 @@
}
public static HornetQServer newHornetQServer(final Configuration config,
- final MBeanServer mbeanServer,
- final HornetQSecurityManager securityManager)
+ final MBeanServer mbeanServer,
+ final HornetQSecurityManager securityManager)
{
HornetQServer server = newHornetQServer(config, mbeanServer, securityManager, true);
@@ -79,9 +79,9 @@
}
public static HornetQServer newHornetQServer(final Configuration config,
- final MBeanServer mbeanServer,
- final HornetQSecurityManager securityManager,
- final boolean enablePersistence)
+ final MBeanServer mbeanServer,
+ final HornetQSecurityManager securityManager,
+ final boolean enablePersistence)
{
config.setPersistenceEnabled(enablePersistence);
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/cluster/impl/BridgeImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/cluster/impl/BridgeImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/cluster/impl/BridgeImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -259,8 +259,6 @@
public void stop() throws Exception
{
- log.info("Stopping bridge " + name);
-
if (started)
{
// We need to stop the csf here otherwise the stop runnable never runs since the createobjectsrunnable is
@@ -290,8 +288,6 @@
log.warn("unable to send notification when broadcast group is stopped", e);
}
}
- log.info("Stopped bridge " + name);
-
}
public boolean isStarted()
@@ -471,8 +467,6 @@
public void connectionFailed(final HornetQException me)
{
- log.info("bridge " + name + " failed " + me);
-
fail();
}
@@ -721,8 +715,6 @@
log.error("Failed to stop", e);
}
- log.info("Bridge " + name + " closed objects");
-
if (!createObjects())
{
started = false;
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -466,21 +466,19 @@
if (!checkActivate())
{
- return new ReattachSessionResponseMessage(-1, false, false);
+ return new ReattachSessionResponseMessage(-1, false);
}
if (session == null)
{
- create the session
-
- return new ReattachSessionResponseMessage(-1, false, true);
+ return new ReattachSessionResponseMessage(-1, false);
}
else
{
// Reconnect the channel to the new connection
int serverLastReceivedCommandID = session.transferConnection(connection, lastReceivedCommandID);
- return new ReattachSessionResponseMessage(serverLastReceivedCommandID, true, true);
+ return new ReattachSessionResponseMessage(serverLastReceivedCommandID, true);
}
}
@@ -509,7 +507,7 @@
"interoperate properly");
return null;
}
-
+
if (!checkActivate())
{
//Backup server is not ready to accept connections
@@ -550,7 +548,7 @@
queueFactory,
this,
configuration.getManagementAddress());
-
+
sessions.put(name, session);
ServerSessionPacketHandler handler = new ServerSessionPacketHandler(session);
@@ -772,7 +770,7 @@
// Private
// --------------------------------------------------------------------------------------
- private boolean checkActivate() throws Exception
+ private synchronized boolean checkActivate() throws Exception
{
if (configuration.isBackup())
{
@@ -780,9 +778,11 @@
if (configuration.isSharedStore())
{
- //load shared store
-
+ //Complete the startup procedure
+
configuration.setBackup(false);
+
+ initialisePart2();
}
else
{
@@ -1102,7 +1102,7 @@
}
else
{
- throw new HornetQException(HornetQException.QUEUE_EXISTS);
+ throw new HornetQException(HornetQException.QUEUE_EXISTS, "Queue " + queueName + " already exists");
}
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/QueueImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/QueueImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/QueueImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -78,8 +78,6 @@
public static final int REDISTRIBUTOR_BATCH_SIZE = 100;
- private static final boolean trace = log.isTraceEnabled();
-
public static final int NUM_PRIORITIES = 10;
private volatile long persistenceID = -1;
@@ -736,8 +734,7 @@
}
public void expire(final MessageReference ref) throws Exception
- {
- log.info("expiring ref " + this.expiryAddress);
+ {
if (expiryAddress != null)
{
move(expiryAddress, ref, true);
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerConsumerImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerConsumerImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerConsumerImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -298,7 +298,7 @@
{
lock.unlock();
}
-
+
// Outside the lock
if (started)
{
@@ -419,10 +419,6 @@
private void promptDelivery()
{
- if (trace)
- {
- log.trace("Starting prompt delivery");
- }
lock.lock();
try
{
@@ -461,8 +457,7 @@
private HandleStatus doHandle(final MessageReference ref) throws Exception
{
if (availableCredits != null && availableCredits.get() <= 0)
- {
- // log.info("busy - available credits is " + availableCredits.get());
+ {
return HandleStatus.BUSY;
}
@@ -568,10 +563,6 @@
if (availableCredits != null)
{
availableCredits.addAndGet(-packet.getRequiredBufferSize());
- if (trace)
- {
- log.trace("Taking " + packet.getRequiredBufferSize() + " out of flow control");
- }
}
channel.send(packet);
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -93,8 +93,6 @@
import org.hornetq.core.transaction.ResourceManager;
import org.hornetq.core.transaction.Transaction;
import org.hornetq.core.transaction.impl.TransactionImpl;
-import org.hornetq.utils.IDGenerator;
-import org.hornetq.utils.SimpleIDGenerator;
import org.hornetq.utils.SimpleString;
import org.hornetq.utils.TypedProperties;
@@ -156,8 +154,6 @@
private final List<Runnable> failureRunners = new ArrayList<Runnable>();
- private final IDGenerator idGenerator = new SimpleIDGenerator(0);
-
private final String name;
private final HornetQServer server;
@@ -390,8 +386,8 @@
{
theQueue = (Queue)binding.getBindable();
}
-
- ServerConsumer consumer = new ServerConsumerImpl(idGenerator.generateID(),
+
+ ServerConsumer consumer = new ServerConsumerImpl(packet.getID(),
this,
(QueueBinding)binding,
filter,
@@ -404,7 +400,7 @@
updateDeliveries,
executor,
managementService);
-
+
consumers.put(consumer.getID(), consumer);
if (!browseOnly)
@@ -517,6 +513,7 @@
}
else
{
+ log.error("Failed to create queue", e);
response = new HornetQExceptionMessage(new HornetQException(HornetQException.INTERNAL_ERROR));
}
}
@@ -664,7 +661,7 @@
}
public void handleAcknowledge(final SessionAcknowledgeMessage packet)
- {
+ {
Packet response = null;
try
@@ -1380,15 +1377,16 @@
}
public void handleReceiveConsumerCredits(final SessionConsumerFlowCreditMessage packet)
- {
+ {
try
{
consumers.get(packet.getConsumerID()).receiveCredits(packet.getCredits());
}
catch (Exception e)
{
- log.error("Failed to receive credits", e);
+ log.error("Failed to receive credits " + this.server.getConfiguration().isBackup(), e);
}
+
channel.confirm(packet);
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQObjectMessage.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQObjectMessage.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQObjectMessage.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -356,8 +356,8 @@
}
catch (Exception e)
{
- JMSException je = new JMSException("Failed to deserialize object");
- je.setLinkedException(e);
+ JMSException je = new JMSException(e.getMessage());
+ je.setStackTrace(e.getStackTrace());
throw je;
}
}
Modified: branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQSession.java
===================================================================
--- branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQSession.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/src/main/org/hornetq/jms/client/HornetQSession.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -308,11 +308,11 @@
// Constructors --------------------------------------------------
public HornetQSession(final HornetQConnection connection,
- final boolean transacted,
- final boolean xa,
- final int ackMode,
- final ClientSession session,
- final int sessionType)
+ final boolean transacted,
+ final boolean xa,
+ final int ackMode,
+ final ClientSession session,
+ final int sessionType)
{
this.connection = connection;
@@ -513,28 +513,28 @@
{
HornetQDestination jbd = (HornetQDestination)destination;
- //TODO Uncomment when https://jira.jboss.org/jira/browse/JBMESSAGING-1565 is complete
-// if (jbd != null)
-// {
-// if (jbd instanceof Queue)
-// {
-// SessionQueueQueryResponseMessage response = session.queueQuery(jbd.getSimpleAddress());
-//
-// if (!response.isExists())
-// {
-// throw new InvalidDestinationException("Queue " + jbd.getName() + " does not exist");
-// }
-// }
-// else
-// {
-// SessionBindingQueryResponseMessage response = session.bindingQuery(jbd.getSimpleAddress());
-//
-// if (!response.isExists())
-// {
-// throw new InvalidDestinationException("Topic " + jbd.getName() + " does not exist");
-// }
-// }
-// }
+ // TODO Uncomment when https://jira.jboss.org/jira/browse/JBMESSAGING-1565 is complete
+ // if (jbd != null)
+ // {
+ // if (jbd instanceof Queue)
+ // {
+ // SessionQueueQueryResponseMessage response = session.queueQuery(jbd.getSimpleAddress());
+ //
+ // if (!response.isExists())
+ // {
+ // throw new InvalidDestinationException("Queue " + jbd.getName() + " does not exist");
+ // }
+ // }
+ // else
+ // {
+ // SessionBindingQueryResponseMessage response = session.bindingQuery(jbd.getSimpleAddress());
+ //
+ // if (!response.isExists())
+ // {
+ // throw new InvalidDestinationException("Topic " + jbd.getName() + " does not exist");
+ // }
+ // }
+ // }
ClientProducer producer = session.createProducer(jbd == null ? null : jbd.getSimpleAddress());
@@ -574,9 +574,10 @@
if (jbdest.isTemporary() && !connection.containsTemporaryQueue(jbdest.getSimpleAddress()))
{
- throw new JMSException("Can not create consumer for temporary destination " + destination + " from another JMS connection");
+ throw new JMSException("Can not create consumer for temporary destination " + destination +
+ " from another JMS connection");
}
-
+
HornetQMessageConsumer consumer = createConsumer(jbdest, null, messageSelector, noLocal);
return consumer;
@@ -674,9 +675,9 @@
}
private HornetQMessageConsumer createConsumer(final HornetQDestination dest,
- final String subscriptionName,
- String selectorString,
- final boolean noLocal) throws JMSException
+ final String subscriptionName,
+ String selectorString,
+ final boolean noLocal) throws JMSException
{
try
{
@@ -686,7 +687,9 @@
{
connection.setHasNoLocal();
- String filter = HornetQConnection.CONNECTION_ID_PROPERTY_NAME.toString() + "<>'" + connection.getUID() + "'";
+ String filter = HornetQConnection.CONNECTION_ID_PROPERTY_NAME.toString() + "<>'" +
+ connection.getUID() +
+ "'";
if (selectorString != null)
{
@@ -758,7 +761,7 @@
}
queueName = new SimpleString(HornetQTopic.createQueueNameForDurableSubscription(connection.getClientID(),
- subscriptionName));
+ subscriptionName));
SessionQueueQueryResponseMessage subResponse = session.queueQuery(queueName);
@@ -807,11 +810,11 @@
}
HornetQMessageConsumer jbc = new HornetQMessageConsumer(this,
- consumer,
- noLocal,
- dest,
- selectorString,
- autoDeleteQueueName);
+ consumer,
+ noLocal,
+ dest,
+ selectorString,
+ autoDeleteQueueName);
consumers.add(jbc);
@@ -947,7 +950,7 @@
}
SimpleString queueName = new SimpleString(HornetQTopic.createQueueNameForDurableSubscription(connection.getClientID(),
- name));
+ name));
try
{
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/AutomaticFailoverWithDiscoveryTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/AutomaticFailoverWithDiscoveryTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/AutomaticFailoverWithDiscoveryTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,178 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A AutomaticFailoverWithDiscoveryTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- * @author <a href="mailto:clebert.suconic@jboss.com">Clebert Suconic</a>
- *
- * Created 8 Dec 2008 14:52:21
- *
- *
- */
-public class AutomaticFailoverWithDiscoveryTest extends FailoverTestBase
-{
- private static final Logger log = Logger.getLogger(AutomaticFailoverWithDiscoveryTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private final String groupAddress = "230.1.2.3";
-
- private final int groupPort = 8765;
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testFailover() throws Exception
- {
- ClientSessionFactoryImpl sf = new ClientSessionFactoryImpl(groupAddress, groupPort);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn1 = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- boolean outOfOrder = false;
-
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- if (i != (Integer)message2.getProperty(new SimpleString("count")))
- {
- System.out.println("Messages received out of order, " + i +
- " != " +
- message2.getProperty(new SimpleString("count")));
- outOfOrder = true;
- }
-
- message2.acknowledge();
- }
-
- session.close();
-
- session = sf.createSession(false, true, true);
-
- consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = numMessages / 2; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- if (i != (Integer)message2.getProperty(new SimpleString("count")))
- {
- System.out.println("Messages received out of order, " + i +
- " != " +
- message2.getProperty(new SimpleString("count")));
- outOfOrder = true;
- }
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = consumer.receive(250);
-
- if (message3 != null)
- {
- do
- {
- System.out.println("Message " + message3.getProperty(new SimpleString("count")) + " was duplicated");
- message3 = consumer.receive(1000);
- }
- while (message3 != null);
- fail("Duplicated messages received on test");
- }
-
- session.close();
-
- assertFalse("Messages received out of order, look at System.out for more details", outOfOrder);
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
- setupGroupServers(false, "bc1", 5432, groupAddress, groupPort);
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverExpiredMessageTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverExpiredMessageTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverExpiredMessageTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,298 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A FailoverExpiredMessageTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 5 Nov 2008 09:33:32
- *
- *
- */
-public class FailoverExpiredMessageTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(FailoverExpiredMessageTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveService;
-
- private HornetQServer backupService;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- /*
- * Set messages to expire very soon, send a load of them, so at some of them get expired when they reach the client
- * After failover make sure all are received ok
- */
- public void testExpiredBeforeConsumption() throws Exception
- {
- ClientSessionFactoryInternal sf1 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf1.setProducerWindowSize(32 * 1024);
-
- ClientSession session1 = sf1.createSession(false, true, true);
-
- session1.createQueue(ADDRESS, ADDRESS, null, false);
-
- session1.start();
-
- ClientProducer producer = session1.createProducer(ADDRESS);
-
- final int numMessages = 10000;
-
- //Set time to live so at least some of them will more than likely expire before they are consumed by the client
-
- long now = System.currentTimeMillis();
-
- long expire = now + 5000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session1.createClientMessage(HornetQTextMessage.TYPE,
- false,
- expire,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer1 = session1.createConsumer(ADDRESS);
-
- final RemotingConnection conn1 = ((ClientSessionInternal)session1).getConnection();
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- //Sleep a little while to ensure that some messages are consumed before failover
- Thread.sleep(5000);
- }
- catch (InterruptedException e)
- {
- }
-
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
- }
- };
-
- t.start();
-
- int count = 0;
-
- while (true)
- {
- ClientMessage message = consumer1.receive(1000);
-
- if (message != null)
- {
- message.acknowledge();
-
- //We sleep a little to make sure messages aren't consumed too quickly and some
- //will expire before reaching consumer
- Thread.sleep(1);
-
- count++;
- }
- else
- {
- break;
- }
- }
-
- t.join();
-
- session1.close();
-
- //Make sure no more messages
- ClientSession session2 = sf1.createSession(false, true, true);
-
- session2.start();
-
- ClientConsumer consumer2 = session2.createConsumer(ADDRESS);
-
- ClientMessage message = consumer2.receive(1000);
-
- assertNull(message);
-
- session2.close();
- }
-
- public void testExpireViaReaperOnLive() throws Exception
- {
- ClientSessionFactoryInternal sf1 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf1.setProducerWindowSize(32 * 1024);
-
- ClientSession session1 = sf1.createSession(false, true, true);
-
- session1.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session1.createProducer(ADDRESS);
-
- final int numMessages = 10000;
-
- //Set time to live so messages are expired on the server
-
- long now = System.currentTimeMillis();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session1.createClientMessage(HornetQTextMessage.TYPE,
- false,
- now,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- Thread.sleep(4 * expireScanPeriod);
-
- //Messages should all be expired now
-
- ClientConsumer consumer1 = session1.createConsumer(ADDRESS);
-
- session1.start();
-
- RemotingConnection conn1 = ((ClientSessionInternal)session1).getConnection();
-
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientMessage message = consumer1.receive(1000);
-
- assertNull(message);
-
- session1.close();
-
- //Make sure no more messages
- ClientSession session2 = sf1.createSession(false, true, true);
-
- session2.start();
-
- ClientConsumer consumer2 = session2.createConsumer(ADDRESS);
-
- message = consumer2.receive(1000);
-
- assertNull(message);
-
- session2.close();
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- private final long expireScanPeriod = 1000;
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setMessageExpiryScanPeriod(expireScanPeriod);
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setMessageExpiryScanPeriod(expireScanPeriod);
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams, "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- backupService.stop();
-
- liveService.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupService = null;
-
- liveService = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
-
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverManagementTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverManagementTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverManagementTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,302 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import static org.hornetq.core.config.impl.ConfigurationImpl.DEFAULT_MANAGEMENT_ADDRESS;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientMessageImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.client.management.impl.ManagementHelper;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.management.ResourceNames;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A FailoverManagementTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 5 Nov 2008 15:05:14
- *
- *
- */
-public class FailoverManagementTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(FailoverManagementTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveService;
-
- private HornetQServer backupService;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testManagementMessages() throws Exception
- {
- ClientSessionFactoryInternal sf1 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf1.setProducerWindowSize(32 * 1024);
-
- ClientSession session1 = sf1.createSession(false, true, true);
-
- session1.createQueue(ADDRESS, ADDRESS, null, false);
-
- SimpleString replyTo = new SimpleString("replyto");
-
- session1.createQueue(replyTo, new SimpleString("replyto"), null, false);
-
- ClientProducer producer = session1.createProducer(ADDRESS);
-
- final int numMessages = 10;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage msg = session1.createClientMessage(false);
-
- producer.send(msg);
- }
-
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage managementMessage = session1.createClientMessage(false);
-
- ManagementHelper.putAttribute(managementMessage,
- ResourceNames.CORE_QUEUE + ADDRESS,
- "messageCount");
- managementMessage.putStringProperty(ClientMessageImpl.REPLYTO_HEADER_NAME, replyTo);
-
- producer.send(DEFAULT_MANAGEMENT_ADDRESS, managementMessage);
- }
-
- ClientConsumer consumer1 = session1.createConsumer(replyTo);
-
- final RemotingConnection conn1 = ((ClientSessionInternal)session1).getConnection();
-
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- //Send the other half
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage managementMessage = session1.createClientMessage(false);
-
- ManagementHelper.putAttribute(managementMessage,
- ResourceNames.CORE_QUEUE + ADDRESS,
- "messageCount");
- managementMessage.putStringProperty(ClientMessageImpl.REPLYTO_HEADER_NAME, replyTo);
-
- producer.send(DEFAULT_MANAGEMENT_ADDRESS, managementMessage);
- }
-
- session1.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer1.receive(1000);
-
- assertNotNull(message);
-
- message.acknowledge();
-
- assertTrue(ManagementHelper.isAttributesResult(message));
-
- assertEquals(numMessages, ManagementHelper.getResult(message));
- }
-
- session1.close();
-
- //Make sure no more messages
- ClientSession session2 = sf1.createSession(false, true, true);
-
- session2.start();
-
- ClientConsumer consumer2 = session2.createConsumer(replyTo);
-
- ClientMessage message = consumer2.receive(1000);
-
- assertNull(message);
-
- session2.close();
- }
-
- public void testManagementMessages2() throws Exception
- {
- ClientSessionFactoryInternal sf1 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf1.setProducerWindowSize(32 * 1024);
-
- ClientSession session1 = sf1.createSession(false, true, true);
-
- session1.createQueue(ADDRESS, ADDRESS, null, false);
-
- SimpleString replyTo = new SimpleString("replyto");
-
- session1.createQueue(replyTo, new SimpleString("replyto"), null, false);
-
- ClientProducer producer = session1.createProducer(ADDRESS);
-
- final int numMessages = 10;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage msg = session1.createClientMessage(false);
-
- producer.send(msg);
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage managementMessage = session1.createClientMessage(false);
-
- ManagementHelper.putAttribute(managementMessage,
- ResourceNames.CORE_QUEUE + ADDRESS,
- "messageCount");
- managementMessage.putStringProperty(ClientMessageImpl.REPLYTO_HEADER_NAME, replyTo);
-
- producer.send(DEFAULT_MANAGEMENT_ADDRESS, managementMessage);
- }
-
- ClientConsumer consumer1 = session1.createConsumer(replyTo);
-
-
- session1.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer1.receive(1000);
-
- assertNotNull(message);
-
- if (i == 0)
- {
- //Fail after receipt but before ack
- final RemotingConnection conn1 = ((ClientSessionInternal)session1).getConnection();
-
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
- }
-
- message.acknowledge();
-
- assertTrue(ManagementHelper.isAttributesResult(message));
-
- assertEquals(numMessages, ManagementHelper.getResult(message));
- }
-
- session1.close();
-
- //Make sure no more messages
- ClientSession session2 = sf1.createSession(false, true, true);
-
- session2.start();
-
- ClientConsumer consumer2 = session2.createConsumer(replyTo);
-
- ClientMessage message = consumer2.receive(1000);
-
- assertNull(message);
-
- session2.close();
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams, "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- backupService.stop();
-
- liveService.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupService = null;
-
- liveService = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
-
-
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverPreAcknowledgeTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverPreAcknowledgeTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverPreAcknowledgeTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,209 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- * @author <a href="mailto:andy.taylor@jboss.org">Andy Taylor</a>
- */
-public class FailoverPreAcknowledgeTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(FailoverPreAcknowledgeTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveService;
-
- private HornetQServer backupService;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testPreAcknowledgeFailoverTest() throws Exception
- {
- ClientSessionFactoryInternal sf1 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf1.setProducerWindowSize(32 * 1024);
-
- ClientSession session1 = sf1.createSession(false, true, true, true);
-
- session1.createQueue(ADDRESS, ADDRESS, null, false);
-
- session1.start();
-
- ClientProducer producer = session1.createProducer(ADDRESS);
-
- final int numMessages = 10000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session1.createClientMessage(HornetQTextMessage.TYPE,
- false);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("more aardvarks");
- producer.send(message);
- }
- ClientConsumer consumer1 = session1.createConsumer(ADDRESS);
-
- final RemotingConnection conn1 = ((ClientSessionInternal)session1).getConnection();
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- //Sleep a little while to ensure that some messages are consumed before failover
- Thread.sleep(5000);
- }
- catch (InterruptedException e)
- {
- }
-
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
- }
- };
-
- t.start();
-
- int count = 0;
-
- while (true)
- {
- ClientMessage message = consumer1.receive(1000);
-
- if (message != null)
- {
- message.acknowledge();
-
- count++;
- }
- else
- {
- break;
- }
- }
-
- t.join();
-
- session1.close();
-
- //Make sure no more messages
- ClientSession session2 = sf1.createSession(false, true, true);
-
- session2.start();
-
- ClientConsumer consumer2 = session2.createConsumer(ADDRESS);
-
- ClientMessage message = consumer2.receive(1000);
-
- try
- {
- assertNull(message);
- }
- finally
- {
- session2.close();
- }
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams, "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- backupService.stop();
-
- liveService.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupService = null;
-
- liveService = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
-
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverScheduledMessageTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverScheduledMessageTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverScheduledMessageTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,240 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.message.impl.MessageImpl;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A FailoverScheduledMessageTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 5 Nov 2008 11:18:51
- *
- */
-public class FailoverScheduledMessageTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(FailoverScheduledMessageTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveService;
-
- private HornetQServer backupService;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- /*
- * Send some scheduled messsages on live
- * Let some fire on live
- * Failover
- * Let rest fire on backup
- * Assert no duplicates and all are received ok
- */
- public void testScheduled() throws Exception
- {
- ClientSessionFactoryInternal sf1 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf1.setProducerWindowSize(32 * 1024);
-
- ClientSession session1 = sf1.createSession(false, true, true);
-
- session1.createQueue(ADDRESS, ADDRESS, null, false);
-
- session1.start();
-
- ClientProducer producer = session1.createProducer(ADDRESS);
-
- final int numMessages = 10;
-
- long now = System.currentTimeMillis();
-
- final long delay = 200;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session1.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- long deliveryTime = now + delay * i;
- message.putLongProperty(MessageImpl.HDR_SCHEDULED_DELIVERY_TIME, deliveryTime);
- producer.send(message);
- }
-
- ClientConsumer consumer1 = session1.createConsumer(ADDRESS);
-
- final RemotingConnection conn1 = ((ClientSessionInternal)session1).getConnection();
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- //Sleep a little while to ensure that some messages are consumed before failover
- Thread.sleep(delay * numMessages / 2);
- }
- catch (InterruptedException e)
- {
- }
-
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
- }
- };
-
- t.start();
-
- ClientSession session2 = null;
- try
- {
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer1.receive(delay * 2);
-
- assertNotNull(message);
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer1.receive(delay * 2);
-
- assertNull(message);
-
- t.join();
-
- session1.close();
-
- //Make sure no more messages
- session2 = sf1.createSession(false, true, true);
-
- session2.start();
-
- ClientConsumer consumer2 = session2.createConsumer(ADDRESS);
-
- message = consumer2.receive(1000);
-
- assertNull(message);
-
- session2.close();
- }
- finally
- {
- if (session1 != null)
- {
- session1.close();
- }
- if (session2 != null)
- {
- session2.close();
- }
- }
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams, "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- backupService.stop();
-
- liveService.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupService = null;
-
- liveService = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
-
-
Copied: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTest.java (from rev 7946, branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleManualFailoverTest.java)
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTest.java (rev 0)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -0,0 +1,1932 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.tests.integration.cluster.failover;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import javax.transaction.xa.XAException;
+import javax.transaction.xa.XAResource;
+import javax.transaction.xa.Xid;
+
+import org.hornetq.core.client.ClientConsumer;
+import org.hornetq.core.client.ClientMessage;
+import org.hornetq.core.client.ClientProducer;
+import org.hornetq.core.client.ClientSession;
+import org.hornetq.core.client.ClientSessionFactory;
+import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
+import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
+import org.hornetq.core.client.impl.ClientSessionInternal;
+import org.hornetq.core.config.Configuration;
+import org.hornetq.core.config.TransportConfiguration;
+import org.hornetq.core.exception.HornetQException;
+import org.hornetq.core.logging.Logger;
+import org.hornetq.core.remoting.FailureListener;
+import org.hornetq.core.remoting.Interceptor;
+import org.hornetq.core.remoting.Packet;
+import org.hornetq.core.remoting.RemotingConnection;
+import org.hornetq.core.remoting.impl.invm.InVMRegistry;
+import org.hornetq.core.remoting.impl.invm.TransportConstants;
+import org.hornetq.core.remoting.impl.wireformat.PacketImpl;
+import org.hornetq.core.server.HornetQServer;
+import org.hornetq.core.transaction.impl.XidImpl;
+import org.hornetq.jms.client.HornetQTextMessage;
+import org.hornetq.tests.util.ServiceTestBase;
+import org.hornetq.utils.SimpleString;
+
+/**
+ *
+ * A FailoverTest
+ *
+ * Tests:
+ *
+ * Failover via shared storage manager:
+ *
+ *
+ * 5) Failover due to failure on create session
+ *
+ * 6) Replicate above tests on JMS API
+ *
+ * 7) Repeat above tests using replicated journal
+ *
+ * 8) Test with different values of auto commit acks and autocomit sends
+ *
+ * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
+ *
+ */
+public class FailoverTest extends ServiceTestBase
+{
+ private static final Logger log = Logger.getLogger(FailoverTest.class);
+
+ // Constants -----------------------------------------------------
+
+ // Attributes ----------------------------------------------------
+
+ private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
+
+ private HornetQServer server0Service;
+
+ private HornetQServer server1Service;
+
+ private Map<String, Object> server1Params = new HashMap<String, Object>();
+
+ // Static --------------------------------------------------------
+
+ // Constructors --------------------------------------------------
+
+ // Public --------------------------------------------------------
+
+ public void testNonTransacted() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ // Only the persistent messages will survive
+
+ if (i % 2 == 0)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+ }
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testTransactedMessagesSentSoRollback() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(false, false);
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session.commit();
+
+ fail("Should throw exception");
+ }
+ catch (HornetQException e)
+ {
+ assertEquals(HornetQException.TRANSACTION_ROLLED_BACK, e.getCode());
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ ClientMessage message = consumer.receive(500);
+
+ assertNull(message);
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testTransactedMessagesNotSentSoNoRollback() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(false, false);
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session.commit();
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ // committing again should work since didn't send anything since last commit
+
+ session.commit();
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ // Only the persistent messages will survive
+
+ if (i % 2 == 0)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+ }
+
+ session.commit();
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testTransactedMessagesConsumedSoRollback() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session1 = sf.createSession(false, false);
+
+ session1.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session1.addFailureListener(new MyListener());
+
+ ClientProducer producer = session1.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session1.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session1.commit();
+
+ ClientSession session2 = sf.createSession(false, false);
+
+ ClientConsumer consumer = session2.createConsumer(ADDRESS);
+
+ session2.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session2).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session2.commit();
+
+ fail("Should throw exception");
+ }
+ catch (HornetQException e)
+ {
+ assertEquals(HornetQException.TRANSACTION_ROLLED_BACK, e.getCode());
+ }
+
+ session1.close();
+
+ session2.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testTransactedMessagesNotConsumedSoNoRollback() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session1 = sf.createSession(false, false);
+
+ session1.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session1.addFailureListener(new MyListener());
+
+ ClientProducer producer = session1.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session1.createClientMessage(true);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session1.commit();
+
+ ClientSession session2 = sf.createSession(false, false);
+
+ ClientConsumer consumer = session2.createConsumer(ADDRESS);
+
+ session2.start();
+
+ for (int i = 0; i < numMessages / 2; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ session2.commit();
+
+ consumer.close();
+
+ RemotingConnection conn = ((ClientSessionInternal)session2).getConnection();
+
+ log.info("Failing connection**");
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ log.info("** creating the consumer");
+
+ consumer = session2.createConsumer(ADDRESS);
+
+ for (int i = numMessages / 2; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ log.info("got message " + message);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ session2.commit();
+
+ session1.close();
+
+ session2.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testXAMessagesSentSoRollbackOnEnd() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(true, false, false);
+
+ Xid xid = new XidImpl("uhuhuhu".getBytes(), 126512, "auhsduashd".getBytes());
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ session.start(xid, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session.end(xid, XAResource.TMSUCCESS);
+
+ fail("Should throw exception");
+ }
+ catch (XAException e)
+ {
+ assertEquals(XAException.XA_RBOTHER, e.errorCode);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ ClientMessage message = consumer.receive(500);
+
+ assertNull(message);
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testXAMessagesSentSoRollbackOnPrepare() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(true, false, false);
+
+ Xid xid = new XidImpl("uhuhuhu".getBytes(), 126512, "auhsduashd".getBytes());
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ session.start(xid, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session.end(xid, XAResource.TMSUCCESS);
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session.prepare(xid);
+
+ fail("Should throw exception");
+ }
+ catch (XAException e)
+ {
+ assertEquals(XAException.XA_RBOTHER, e.errorCode);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ ClientMessage message = consumer.receive(500);
+
+ assertNull(message);
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ // This might happen if 1PC optimisation kicks in
+ public void testXAMessagesSentSoRollbackOnCommit() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(true, false, false);
+
+ Xid xid = new XidImpl("uhuhuhu".getBytes(), 126512, "auhsduashd".getBytes());
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ session.start(xid, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session.end(xid, XAResource.TMSUCCESS);
+
+ session.prepare(xid);
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session.commit(xid, true);
+
+ fail("Should throw exception");
+ }
+ catch (XAException e)
+ {
+ assertEquals(XAException.XA_RBOTHER, e.errorCode);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ ClientMessage message = consumer.receive(500);
+
+ assertNull(message);
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testXAMessagesNotSentSoNoRollbackOnCommit() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(true, false, false);
+
+ Xid xid = new XidImpl("uhuhuhu".getBytes(), 126512, "auhsduashd".getBytes());
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ session.start(xid, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session.end(xid, XAResource.TMSUCCESS);
+
+ session.prepare(xid);
+
+ session.commit(xid, false);
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ Xid xid2 = new XidImpl("tfytftyf".getBytes(), 54654, "iohiuohiuhgiu".getBytes());
+
+ session.start(xid2, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ // Only the persistent messages will survive
+
+ if (i % 2 == 0)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+ }
+
+ session.end(xid2, XAResource.TMSUCCESS);
+
+ session.prepare(xid2);
+
+ session.commit(xid2, false);
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testXAMessagesConsumedSoRollbackOnEnd() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session1 = sf.createSession(false, false);
+
+ session1.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session1.addFailureListener(new MyListener());
+
+ ClientProducer producer = session1.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session1.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session1.commit();
+
+ ClientSession session2 = sf.createSession(true, false, false);
+
+ ClientConsumer consumer = session2.createConsumer(ADDRESS);
+
+ session2.start();
+
+ Xid xid = new XidImpl("uhuhuhu".getBytes(), 126512, "auhsduashd".getBytes());
+
+ session2.start(xid, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session2).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session2.end(xid, XAResource.TMSUCCESS);
+
+ fail("Should throw exception");
+ }
+ catch (XAException e)
+ {
+ assertEquals(XAException.XA_RBOTHER, e.errorCode);
+ }
+
+ session1.close();
+
+ session2.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testXAMessagesConsumedSoRollbackOnPrepare() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session1 = sf.createSession(false, false);
+
+ session1.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session1.addFailureListener(new MyListener());
+
+ ClientProducer producer = session1.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session1.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session1.commit();
+
+ ClientSession session2 = sf.createSession(true, false, false);
+
+ ClientConsumer consumer = session2.createConsumer(ADDRESS);
+
+ session2.start();
+
+ Xid xid = new XidImpl("uhuhuhu".getBytes(), 126512, "auhsduashd".getBytes());
+
+ session2.start(xid, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ session2.end(xid, XAResource.TMSUCCESS);
+
+ RemotingConnection conn = ((ClientSessionInternal)session2).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session2.prepare(xid);
+
+ fail("Should throw exception");
+ }
+ catch (XAException e)
+ {
+ assertEquals(XAException.XA_RBOTHER, e.errorCode);
+ }
+
+ session1.close();
+
+ session2.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ // 1PC optimisation
+ public void testXAMessagesConsumedSoRollbackOnCommit() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session1 = sf.createSession(false, false);
+
+ session1.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session1.addFailureListener(new MyListener());
+
+ ClientProducer producer = session1.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session1.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ session1.commit();
+
+ ClientSession session2 = sf.createSession(true, false, false);
+
+ ClientConsumer consumer = session2.createConsumer(ADDRESS);
+
+ session2.start();
+
+ Xid xid = new XidImpl("uhuhuhu".getBytes(), 126512, "auhsduashd".getBytes());
+
+ session2.start(xid, XAResource.TMNOFLAGS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ session2.end(xid, XAResource.TMSUCCESS);
+
+ session2.prepare(xid);
+
+ RemotingConnection conn = ((ClientSessionInternal)session2).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ try
+ {
+ session2.commit(xid, true);
+
+ fail("Should throw exception");
+ }
+ catch (XAException e)
+ {
+ assertEquals(XAException.XA_RBOTHER, e.errorCode);
+ }
+
+ session1.close();
+
+ session2.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testCreateNewFactoryAfterFailover() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ ClientSession session = sendAndConsume(sf);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ conn.addFailureListener(new MyListener());
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ session.close();
+
+ sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ session = sendAndConsume(sf);
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testFailoverMultipleSessionsWithConsumers() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ final int numSessions = 10;
+
+ final int numConsumersPerSession = 5;
+
+ Map<ClientSession, List<ClientConsumer>> sessionConsumerMap = new HashMap<ClientSession, List<ClientConsumer>>();
+
+ class MyListener implements FailureListener
+ {
+ CountDownLatch latch = new CountDownLatch(1);
+
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ List<MyListener> listeners = new ArrayList<MyListener>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ ClientSession session = sf.createSession(true, true);
+
+ List<ClientConsumer> consumers = new ArrayList<ClientConsumer>();
+
+ for (int j = 0; j < numConsumersPerSession; j++)
+ {
+ SimpleString queueName = new SimpleString("queue" + i + "-" + j);
+
+ session.createQueue(ADDRESS, queueName, null, true);
+
+ ClientConsumer consumer = session.createConsumer(queueName);
+
+ consumers.add(consumer);
+ }
+
+ sessionConsumerMap.put(session, consumers);
+ }
+
+ ClientSession sendSession = sf.createSession(true, true);
+
+ ClientProducer producer = sendSession.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sendSession.createClientMessage(true);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)sendSession).getConnection();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ for (MyListener listener : listeners)
+ {
+ boolean ok = listener.latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+ }
+
+ for (ClientSession session : sessionConsumerMap.keySet())
+ {
+ session.start();
+ }
+
+ for (List<ClientConsumer> consumerList : sessionConsumerMap.values())
+ {
+ for (ClientConsumer consumer : consumerList)
+ {
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+ }
+ }
+
+ for (ClientSession session : sessionConsumerMap.keySet())
+ {
+ session.close();
+ }
+
+ sendSession.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testFailoverFailMultipleUnderlyingConnections() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ class MyListener implements FailureListener
+ {
+ CountDownLatch latch = new CountDownLatch(1);
+
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ ClientSession session1 = sf.createSession(true, true);
+ ClientSession session2 = sf.createSession(true, true);
+ ClientSession session3 = sf.createSession(true, true);
+
+ SimpleString queueName1 = new SimpleString("queue1");
+ session1.createQueue(ADDRESS, queueName1, null, true);
+ MyListener listener1 = new MyListener();
+ session1.addFailureListener(listener1);
+
+ SimpleString queueName2 = new SimpleString("queue2");
+ session2.createQueue(ADDRESS, queueName2, null, true);
+ MyListener listener2 = new MyListener();
+ session2.addFailureListener(listener2);
+
+ SimpleString queueName3 = new SimpleString("queue3");
+ session3.createQueue(ADDRESS, queueName3, null, true);
+ MyListener listener3 = new MyListener();
+ session3.addFailureListener(listener3);
+
+ ClientConsumer consumer1 = session1.createConsumer(queueName1);
+ ClientConsumer consumer2 = session1.createConsumer(queueName2);
+ ClientConsumer consumer3 = session1.createConsumer(queueName3);
+
+ ClientProducer producer = session1.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session1.createClientMessage(true);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ // Fail all the connections
+
+ RemotingConnection conn1 = ((ClientSessionInternal)session1).getConnection();
+ RemotingConnection conn2 = ((ClientSessionInternal)session2).getConnection();
+ RemotingConnection conn3 = ((ClientSessionInternal)session3).getConnection();
+
+ assertTrue(conn1 != conn2);
+ assertTrue(conn2 != conn3);
+ assertTrue(conn1 != conn3);
+
+ conn2.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+ conn3.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = listener1.latch.await(1000, TimeUnit.MILLISECONDS);
+ assertTrue(ok);
+ ok = listener2.latch.await(1000, TimeUnit.MILLISECONDS);
+ assertTrue(ok);
+ ok = listener3.latch.await(1000, TimeUnit.MILLISECONDS);
+ assertTrue(ok);
+
+ session1.start();
+ session2.start();
+ session3.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer1.receive(1000);
+ assertNotNull(message);
+ assertEquals("message" + i, message.getBody().readString());
+ assertEquals(i, message.getProperty("counter"));
+ message.acknowledge();
+
+ message = consumer2.receive(1000);
+ assertNotNull(message);
+ assertEquals("message" + i, message.getBody().readString());
+ assertEquals(i, message.getProperty("counter"));
+ message.acknowledge();
+
+ message = consumer3.receive(1000);
+ assertNotNull(message);
+ assertEquals("message" + i, message.getBody().readString());
+ assertEquals(i, message.getProperty("counter"));
+ message.acknowledge();
+ }
+
+ session1.close();
+ session2.close();
+ session3.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ // Package protected ---------------------------------------------
+
+ // Protected -----------------------------------------------------
+
+ protected void setUp() throws Exception
+ {
+ super.setUp();
+
+ server1Params.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
+ Configuration config1 = super.createDefaultConfig();
+ config1.getAcceptorConfigurations().clear();
+ config1.getAcceptorConfigurations()
+ .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory", server1Params));
+ config1.setSecurityEnabled(false);
+ config1.setSharedStore(true);
+ config1.setBackup(true);
+ server1Service = super.createServer(true, config1);
+
+ Configuration config0 = super.createDefaultConfig();
+ config0.getAcceptorConfigurations().clear();
+ config0.getAcceptorConfigurations()
+ .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
+ config0.setSecurityEnabled(false);
+ config0.setSharedStore(true);
+ server0Service = super.createServer(true, config0);
+
+ server1Service.start();
+ server0Service.start();
+ }
+
+ protected void tearDown() throws Exception
+ {
+ server1Service.stop();
+
+ server0Service.stop();
+
+ assertEquals(0, InVMRegistry.instance.size());
+
+ server1Service = null;
+
+ server0Service = null;
+
+ server1Params = null;
+
+ super.tearDown();
+ }
+
+ // Private -------------------------------------------------------
+
+ private ClientSession sendAndConsume(final ClientSessionFactory sf) throws Exception
+ {
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message2 = consumer.receive();
+
+ assertEquals("aardvarks", message2.getBody().readString());
+
+ assertEquals(i, message2.getProperty(new SimpleString("count")));
+
+ message2.acknowledge();
+ }
+
+ ClientMessage message3 = consumer.receive(250);
+
+ assertNull(message3);
+
+ return session;
+ }
+
+ /*
+ * Browser will get reset to beginning after failover
+ */
+ public void testFailWithBrowser() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS, true);
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ log.info("after failover");
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ // Only the persistent messages will survive
+
+ if (i % 2 == 0)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ log.info("got message " + i);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+ }
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testFailThenReceiveMoreMessagesAfterFailover() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+
+ ClientSession session = sf.createSession(true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ // Should get the same ones after failover since we didn't ack
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ // Only the persistent messages will survive
+
+ if (i % 2 == 0)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+ }
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testFailThenReceiveMoreMessagesAfterFailover2() throws Exception
+ {
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+ sf.setBlockOnAcknowledge(true);
+
+ ClientSession session = sf.createSession(true, true, 0);
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 100;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ // Send some more
+
+ for (int i = numMessages; i < numMessages * 2; i++)
+ {
+ ClientMessage message = session.createClientMessage(i % 2 == 0);
+
+ message.getBody().writeString("message" + i);
+
+ message.putIntProperty("counter", i);
+
+ producer.send(message);
+ }
+
+ // Should get the same ones after failover since we didn't ack
+
+ for (int i = numMessages; i < numMessages * 2; i++)
+ {
+ ClientMessage message = consumer.receive(1000);
+
+ assertNotNull(message);
+
+ assertEquals("message" + i, message.getBody().readString());
+
+ assertEquals(i, message.getProperty("counter"));
+
+ message.acknowledge();
+ }
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+ }
+
+ public void testForceBlockingReturn() throws Exception
+ {
+ server0Service.stop();
+
+ //Add an interceptor to delay the send method so we can get time to cause failover before it returns
+
+ server0Service.getConfiguration().getInterceptorClassNames().add(DelayInterceptor.class.getCanonicalName());
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
+ new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
+ server1Params));
+
+ server0Service.start();
+
+ sf.setBlockOnNonPersistentSend(true);
+ sf.setBlockOnPersistentSend(true);
+ sf.setBlockOnAcknowledge(true);
+
+ final ClientSession session = sf.createSession(true, true, 0);
+
+ session.createQueue(ADDRESS, ADDRESS, null, true);
+
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ class MyListener implements FailureListener
+ {
+ public void connectionFailed(HornetQException me)
+ {
+ latch.countDown();
+ }
+ }
+
+ session.addFailureListener(new MyListener());
+
+ final ClientProducer producer = session.createProducer(ADDRESS);
+
+ class Sender extends Thread
+ {
+ public void run()
+ {
+ ClientMessage message = session.createClientMessage(true);
+
+ message.getBody().writeString("message");
+
+ try
+ {
+ producer.send(message);
+ }
+ catch (HornetQException e)
+ {
+ this.e = e;
+ }
+ }
+
+ volatile HornetQException e;
+ }
+
+ Sender sender = new Sender();
+
+ sender.start();
+
+ Thread.sleep(500);
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Simulate failure on connection
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ // Wait to be informed of failure
+
+ boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+
+ sender.join();
+
+ assertNotNull(sender.e);
+
+ assertEquals(sender.e.getCode(), HornetQException.UNBLOCKED);
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+
+
+ }
+
+
+ // Inner classes -------------------------------------------------
+}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTestBase.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTestBase.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailoverTestBase.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,308 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.cluster.BroadcastGroupConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.core.server.JournalType;
-import org.hornetq.core.settings.impl.AddressSettings;
-import org.hornetq.tests.util.ServiceTestBase;
-import org.hornetq.utils.Pair;
-
-/**
- * A FailoverTestBase
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- * Created Dec 8, 2008 6:59:53 PM
- *
- *
- */
-public class FailoverTestBase extends ServiceTestBase
-{
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- protected Map<String, Object> backupParams = new HashMap<String, Object>();
-
- protected HornetQServer liveServer;
-
- protected HornetQServer backupServer;
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- protected ClientSessionFactory createFailoverFactory()
- {
- return new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
- }
-
- protected ClientSessionFactory createBackupFactory()
- {
- return new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
- }
-
- protected void setUpFileBased(final int maxGlobalSize) throws Exception
- {
- setUpFileBased(maxGlobalSize, 20 * 1024);
- }
-
- protected void setUpFileBased(final int maxGlobalSize, final int pageSize) throws Exception
- {
- setUpFailoverServers(true, maxGlobalSize, pageSize);
- }
-
- /*
- *
-
- */
-
- protected void setUpFailoverServers(boolean fileBased,
- final int maxAddressSize,
- final int pageSize) throws Exception
- {
- deleteDirectory(new File(getTestDir()));
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupConf.setClustered(true);
- backupConf.setBackup(true);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName(), backupParams));
-
- if (fileBased)
- {
- clearData(getTestDir() + "/backup");
-
- backupConf.setJournalDirectory(getJournalDir(getTestDir() + "/backup"));
- backupConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/backup"));
- backupConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/backup"));
- backupConf.setPagingDirectory(getPageDir(getTestDir() + "/backup"));
- backupConf.setJournalFileSize(100 * 1024);
- backupConf.setJournalCompactPercentage(0);
-
- backupConf.setJournalType(JournalType.ASYNCIO);
-
- backupServer = HornetQ.newHornetQServer(backupConf);
- }
- else
- {
- backupServer = HornetQ.newHornetQServer(backupConf, false);
- }
-
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.setClustered(true);
-
- TransportConfiguration liveTC = new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName());
- liveConf.getAcceptorConfigurations().add(liveTC);
-
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
-
- TransportConfiguration backupTC = new TransportConfiguration(INVM_CONNECTOR_FACTORY,
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
-
- if (fileBased)
- {
- liveConf.setJournalDirectory(getJournalDir(getTestDir() + "/live"));
- liveConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/live"));
- liveConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/live"));
- liveConf.setPagingDirectory(getPageDir(getTestDir() + "/live"));
- liveConf.setJournalCompactMinFiles(0);
-
- liveConf.setJournalFileSize(100 * 1024);
-
- liveConf.setJournalType(JournalType.ASYNCIO);
- }
-
- if (fileBased)
- {
- liveServer = HornetQ.newHornetQServer(liveConf);
- }
- else
- {
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- }
-
- AddressSettings settings = new AddressSettings();
- settings.setPageSizeBytes(pageSize);
- settings.setMaxSizeBytes(maxAddressSize);
- settings.setPageSizeBytes(pageSize);
-
- liveServer.getAddressSettingsRepository().addMatch("#", settings);
- backupServer.getAddressSettingsRepository().addMatch("#", settings);
-
- clearData(getTestDir() + "/live");
-
- liveServer.start();
- }
-
- protected void setupGroupServers(boolean fileBased, String bcGroupName, int localBindPort, String groupAddress, int groupPort) throws Exception
- {
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupConf.setClustered(true);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
-
- if (fileBased)
- {
- clearData(getTestDir() + "/backup");
-
- backupConf.setJournalDirectory(getJournalDir(getTestDir() + "/backup"));
- backupConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/backup"));
- backupConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/backup"));
- backupConf.setPagingDirectory(getPageDir(getTestDir() + "/backup"));
- backupConf.setJournalFileSize(100 * 1024);
-
- backupConf.setJournalType(JournalType.ASYNCIO);
-
- backupServer = HornetQ.newHornetQServer(backupConf);
-
- }
- else
- {
- backupServer = HornetQ.newHornetQServer(backupConf, false);
- }
-
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- TransportConfiguration liveTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory");
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams);
- connectors.put(backupTC.getName(), backupTC);
- connectors.put(liveTC.getName(), liveTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveConf.setClustered(true);
-
- List<Pair<String, String>> connectorNames = new ArrayList<Pair<String, String>>();
- connectorNames.add(new Pair<String, String>(liveTC.getName(), backupTC.getName()));
-
- final long broadcastPeriod = 250;
-
- BroadcastGroupConfiguration bcConfig1 = new BroadcastGroupConfiguration(bcGroupName,
- null,
- localBindPort,
- groupAddress,
- groupPort,
- broadcastPeriod,
- connectorNames);
-
- List<BroadcastGroupConfiguration> bcConfigs1 = new ArrayList<BroadcastGroupConfiguration>();
- bcConfigs1.add(bcConfig1);
- liveConf.setBroadcastGroupConfigurations(bcConfigs1);
-
- if (fileBased)
- {
- liveConf.setJournalDirectory(getJournalDir(getTestDir() + "/live"));
- liveConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/live"));
- liveConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/live"));
- liveConf.setPagingDirectory(getPageDir(getTestDir() + "/live"));
-
- liveConf.setJournalFileSize(100 * 1024);
-
- liveConf.setJournalType(JournalType.ASYNCIO);
- liveServer = HornetQ.newHornetQServer(liveConf);
- }
- else
- {
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- }
-
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- liveServer.start();
-
- }
-
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected void tearDown() throws Exception
- {
- stopServers();
-
- backupServer = null;
-
- liveServer = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- protected void stopServers() throws Exception
- {
- if (backupServer != null && backupServer.isStarted())
- {
- backupServer.stop();
- }
-
- if (liveServer != null && liveServer.isStarted())
- {
- liveServer.stop();
- }
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupServer = null;
-
- liveServer = null;
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailureOnCreateConnectionTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailureOnCreateConnectionTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/FailureOnCreateConnectionTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,183 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMConnector;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A FailureOnCreateConnectionTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 4 Nov 2008 16:54:50
- *
- *
- */
-public class FailureOnCreateConnectionTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(FailureOnCreateConnectionTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer service;
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testFailureOnCreateConnectionReconnectSameServerOneFailure() throws Exception
- {
- testFailureOnCreateConnectionReconnectSameServer(1);
- }
-
- public void testFailureOnCreateConnectionReconnectSameServerMultipleFailures() throws Exception
- {
- testFailureOnCreateConnectionReconnectSameServer(10);
- }
-
-
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- service = HornetQ.newHornetQServer(liveConf, false);
- service.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- InVMConnector.resetFailures();
-
- service.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- service = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- private void testFailureOnCreateConnectionReconnectSameServer(final int numFailures) throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = -1;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- InVMConnector.failOnCreateConnection = true;
- //One failure only
- InVMConnector.numberOfFailures = numFailures;
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- producer.close();
-
- consumer.close();
-
- session.close();
-
- sf.close();
- }
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/JustReplicationTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/JustReplicationTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/JustReplicationTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,219 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.concurrent.CountDownLatch;
-
-import org.hornetq.core.buffers.ChannelBuffers;
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.remoting.spi.HornetQBuffer;
-import org.hornetq.utils.SimpleString;
-
-/**
- * A LargeMessageFailoverTest
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- * Created Dec 8, 2008 7:09:38 PM
- *
- *
- */
-public class JustReplicationTest extends FailoverTestBase
-{
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testFactory() throws Exception
- {
- final ClientSessionFactory factory = createFailoverFactory(); // Just a regular factory with Backup configured
-
- final int NUMBER_OF_THREADS = 5;
- final int NUMBER_OF_ITERATIONS = 5;
- final int NUMBER_OF_SESSIONS = 5;
-
- final CountDownLatch flagAlign = new CountDownLatch(NUMBER_OF_THREADS);
- final CountDownLatch flagStart = new CountDownLatch(1);
-
- class LocalThread extends Thread
- {
-
- Throwable e;
-
- @Override
- public void run()
- {
- try
- {
- flagAlign.countDown();
- flagStart.await();
-
-
- for (int i = 0; i < NUMBER_OF_ITERATIONS; i++)
- {
- ClientSession sessions[] = new ClientSession[NUMBER_OF_SESSIONS];
- for (int j = 0; j < NUMBER_OF_SESSIONS; j++)
- {
- sessions[j] = factory.createSession(false, true, true);
- sessions[j].start();
- }
-
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- }
- }
- catch (Throwable e)
- {
- this.e = e;
- }
- }
- }
-
- LocalThread t[] = new LocalThread[NUMBER_OF_THREADS];
-
- for (int i = 0; i < t.length; i++)
- {
- t[i] = new LocalThread();
- t[i].start();
- }
-
- flagAlign.await();
- flagStart.countDown();
-
- for (LocalThread localT : t)
- {
- localT.join();
- }
-
- for (LocalThread localT : t)
- {
- if (localT.e != null)
- {
- throw new Exception(localT.e.getMessage(), localT.e);
- }
- }
-
- }
-
- public void testJustReplication() throws Exception
- {
- ClientSessionFactory factory = createFailoverFactory();
- factory.setBlockOnAcknowledge(true);
- factory.setBlockOnNonPersistentSend(true);
- factory.setBlockOnPersistentSend(true);
-
- factory.setMinLargeMessageSize(10 * 1024);
-
- ClientSession session = factory.createSession(null, null, false, true, true, false, 0);
-
- final int numberOfMessages = 500;
-
- final int numberOfBytes = 15000;
-
- try
- {
- session.createQueue(ADDRESS, ADDRESS, null, true);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- // he remotingConnection could be used to force a failure
- // final RemotingConnection conn = ((ClientSessionImpl)session).getConnection();
-
- for (int i = 0; i < numberOfMessages; i++)
- {
- ClientMessage message = session.createClientMessage(true);
-
- HornetQBuffer buffer = ChannelBuffers.buffer(15000);
- buffer.setInt(0, i);
- buffer.writerIndex(buffer.capacity());
-
- message.setBody(buffer);
-
- producer.send(message);
-
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numberOfMessages; i++)
- {
- ClientMessage message = consumer.receive(5000);
-
- assertNotNull(message);
-
- message.acknowledge();
-
- HornetQBuffer buffer = message.getBody();
-
- assertEquals(numberOfBytes, buffer.writerIndex());
-
- assertEquals(i, buffer.readInt());
- }
-
- assertNull(consumer.receive(500));
- }
- finally
- {
- try
- {
- session.close();
- }
- catch (Exception ignored)
- {
- }
- }
-
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- setUpFileBased(100 * 1024 * 1024);
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/LargeMessageMultiThreadFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/LargeMessageMultiThreadFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/LargeMessageMultiThreadFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,153 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.JournalType;
-
-/**
- * A LargeMessageMultiThreadFailoverTest
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- * Created Jan 18, 2009 4:52:09 PM
- *
- *
- */
-public class LargeMessageMultiThreadFailoverTest extends MultiThreadRandomFailoverTestBase
-{
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
- private final byte[] FIVE_HUNDRED_BYTES = new byte[500];
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected int getLatchWait()
- {
- return 60000;
- }
-
- protected ClientSessionFactoryInternal createSessionFactory()
- {
- ClientSessionFactoryInternal sf = super.createSessionFactory();
-
- sf.setMinLargeMessageSize(200);
-
- return sf;
-
- }
-
- protected void start() throws Exception
- {
- InVMRegistry.instance.clear();
-
- startJournal();
- }
-
- protected void startJournal() throws Exception
- {
- deleteDirectory(new File(getTestDir()));
-
- Configuration backupConf = new ConfigurationImpl();
-
- backupConf.setJournalDirectory(getJournalDir(getTestDir() + "/backup"));
- backupConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/backup"));
- backupConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/backup"));
- backupConf.setPagingDirectory(getPageDir(getTestDir() + "/backup"));
- backupConf.setJournalFileSize(100 * 1024);
-
- backupConf.setJournalType(JournalType.ASYNCIO);
-
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
-
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName(), backupParams));
- backupConf.setBackup(true);
-
- backupServer = HornetQ.newHornetQServer(backupConf);
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
-
- liveConf.setJournalDirectory(getJournalDir(getTestDir() + "/live"));
- liveConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/live"));
- liveConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/live"));
- liveConf.setPagingDirectory(getPageDir(getTestDir() + "/live"));
-
- liveConf.setJournalFileSize(100 * 1024);
-
- liveConf.setJournalType(JournalType.ASYNCIO);
-
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName()));
-
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
-
- TransportConfiguration backupTC = new TransportConfiguration(INVM_CONNECTOR_FACTORY,
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveServer = HornetQ.newHornetQServer(liveConf);
-
- liveServer.start();
-
- }
-
- @Override
- protected void setBody(final ClientMessage message) throws Exception
- {
- message.getBody().writeBytes(FIVE_HUNDRED_BYTES);
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTestBase#checkSize(org.hornetq.core.client.ClientMessage)
- */
- @Override
- protected boolean checkSize(ClientMessage message)
- {
- return 500 == message.getBodySize();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-
-}
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadFailoverSupport.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadFailoverSupport.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadFailoverSupport.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -132,7 +132,7 @@
// Case a failure happened here, it should print the Thread dump
// Sending it to System.out, as it would show on the Tests report
- System.out.println(threadDump(" - fired by MultiThreadRandomFailoverTestBase::runTestMultipleThreads (" + t.getLocalizedMessage() + ")"));
+ System.out.println(threadDump(" - fired by MultiThreadRandomReattachTestBase::runTestMultipleThreads (" + t.getLocalizedMessage() + ")"));
}
}
}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,79 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-
-/**
- * A MultiThreadRandomFailoverStressTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- */
-public class MultiThreadRandomFailoverTest extends MultiThreadRandomFailoverTestBase
-{
- @Override
- protected void start() throws Exception
- {
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupServer = HornetQ.newHornetQServer(backupConf, false);
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- liveServer.start();
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTestBase#setBody(org.hornetq.core.client.ClientMessage)
- */
- @Override
- protected void setBody(final ClientMessage message) throws Exception
- {
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTestBase#checkSize(org.hornetq.core.client.ClientMessage)
- */
- @Override
- protected boolean checkSize(final ClientMessage message)
- {
- return 0 == message.getBody().writerIndex();
- }
-
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTestBase.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTestBase.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomFailoverTestBase.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,1523 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.MessageHandler;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQBytesMessage;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.utils.SimpleString;
-
-/**
- * A MultiThreadRandomFailoverTestBase
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- *
- */
-public abstract class MultiThreadRandomFailoverTestBase extends MultiThreadFailoverSupport
-{
- private final Logger log = Logger.getLogger(getClass());
-
- // Constants -----------------------------------------------------
-
- private static final int RECEIVE_TIMEOUT = 30000;
-
- private final int LATCH_WAIT = getLatchWait();
-
- private int NUM_THREADS = getNumThreads();
-
- // Attributes ----------------------------------------------------
- protected static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- protected HornetQServer liveServer;
-
- protected HornetQServer backupServer;
-
- protected Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testA() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestA(sf, threadNum);
- }
- }, NUM_THREADS, false);
-
- }
-
- public void testB() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestB(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testC() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestC(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testD() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestD(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testE() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestE(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testF() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestF(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testG() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestG(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testH() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestH(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testI() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestI(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testJ() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestJ(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testK() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestK(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- public void testL() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestL(sf);
- }
- }, NUM_THREADS, true, 10);
- }
-
- // public void testM() throws Exception
- // {
- // runTestMultipleThreads(new RunnableT()
- // {
- // public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- // {
- // doTestM(sf, threadNum);
- // }
- // }, NUM_THREADS);
- // }
-
- public void testN() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestN(sf, threadNum);
- }
- }, NUM_THREADS, false);
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected abstract void start() throws Exception;
-
- protected abstract void setBody(ClientMessage message) throws Exception;
-
- protected abstract boolean checkSize(ClientMessage message);
-
- protected int getNumThreads()
- {
- return 10;
- }
-
- protected ClientSession createAutoCommitSession(ClientSessionFactory sf) throws Exception
- {
- return sf.createSession(false, true, true);
- }
-
- protected ClientSession createTransactionalSession(ClientSessionFactory sf) throws Exception
- {
- return sf.createSession(false, false, false);
- }
-
- protected void doTestA(final ClientSessionFactory sf, final int threadNum, final ClientSession session2) throws Exception
- {
- SimpleString subName = new SimpleString("sub" + threadNum);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, subName, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- ClientConsumer consumer = session.createConsumer(subName);
-
- final int numMessages = 100;
-
- sendMessages(session, producer, numMessages, threadNum);
-
- session.start();
-
- MyHandler handler = new MyHandler(threadNum, numMessages);
-
- consumer.setMessageHandler(handler);
-
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
-
- producer.close();
-
- consumer.close();
-
- session.deleteQueue(subName);
-
- session.close();
- }
-
- protected void doTestA(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = createAutoCommitSession(sf);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
-
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler(threadNum, numMessages);
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
- }
-
- sessSend.close();
-
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestB(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = createAutoCommitSession(sf);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler(threadNum, numMessages);
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
- }
-
- sessSend.close();
-
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
-
- }
-
- protected void doTestC(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = createTransactionalSession(sf);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, false, false);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.rollback();
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.commit();
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler(threadNum, numMessages);
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
-
- handler.reset();
- }
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
- }
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestD(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + " sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, false, false);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, false, false);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.rollback();
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.commit();
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler(threadNum, numMessages);
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
- }
-
- handlers.clear();
-
- // Set handlers to null
- for (ClientConsumer consumer : consumers)
- {
- consumer.setMessageHandler(null);
- }
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- // New handlers
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler(threadNum, numMessages);
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed on rollback: " + handler.failure);
- }
- }
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + " sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- // Now with synchronous receive()
-
- protected void doTestE(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, true, true);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- consumeMessages(consumers, numMessages, threadNum);
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestF(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, true, true);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- consumeMessages(consumers, numMessages, threadNum);
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestG(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, false, false);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, false, false);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.rollback();
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.commit();
-
- consumeMessages(consumers, numMessages, threadNum);
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- consumeMessages(consumers, numMessages, threadNum);
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestH(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, false, false);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, false, false);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.rollback();
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.commit();
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- consumeMessages(consumers, numMessages, threadNum);
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- consumeMessages(consumers, numMessages, threadNum);
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestI(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- ClientSession sessCreate = sf.createSession(false, true, true);
-
- sessCreate.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
-
- ClientSession sess = sf.createSession(false, true, true);
-
- sess.start();
-
- ClientConsumer consumer = sess.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
-
- ClientProducer producer = sess.createProducer(ADDRESS);
-
- ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- producer.send(message);
-
- ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(message2);
-
- message2.acknowledge();
-
- sess.close();
-
- sessCreate.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
-
- sessCreate.close();
- }
-
- protected void doTestJ(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- ClientSession sessCreate = sf.createSession(false, true, true);
-
- sessCreate.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
-
- ClientSession sess = sf.createSession(false, true, true);
-
- sess.start();
-
- ClientConsumer consumer = sess.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
-
- ClientProducer producer = sess.createProducer(ADDRESS);
-
- ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- producer.send(message);
-
- ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(message2);
-
- message2.acknowledge();
-
- sess.close();
-
- sessCreate.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
-
- sessCreate.close();
- }
-
- protected void doTestK(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- ClientSession s = sf.createSession(false, false, false);
-
- s.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
-
- final int numConsumers = 100;
-
- for (int i = 0; i < numConsumers; i++)
- {
- ClientConsumer consumer = s.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
-
- consumer.close();
- }
-
- s.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
-
- s.close();
- }
-
- /*
- * This test tests failure during create connection
- */
- protected void doTestL(final ClientSessionFactory sf) throws Exception
- {
- final int numSessions = 10;
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession session = sf.createSession(false, false, false);
-
- session.close();
- }
- }
-
- // Browsers
- // FIXME - this test won't work until we use a proper iterator for browsing a queue.
- // Making a copy of the queue for a browser consumer doesn't work well with replication since
- // When replicating the create consumer (browser) to the backup, when executed on the backup the
- // backup may have different messages in its queue since been added on different threads.
- // So when replicating deliveries they may not be found.
- // https://jira.jboss.org/jira/browse/JBMESSAGING-1433
- // protected void doTestM(final ClientSessionFactory sf, final int threadNum) throws Exception
- // {
- // long start = System.currentTimeMillis();
- //
- // ClientSession sessSend = sf.createSession(false, true, true, false);
- //
- // ClientSession sessConsume = sf.createSession(false, true, true, false);
- //
- // sessConsume.createQueue(ADDRESS, new SimpleString(threadNum + "sub"), null, false, false);
- //
- // final int numMessages = 100;
- //
- // ClientProducer producer = sessSend.createProducer(ADDRESS);
- //
- // sendMessages(sessSend, producer, numMessages, threadNum);
- //
- // ClientConsumer browser = sessConsume.createConsumer(new SimpleString(threadNum + "sub"),
- // null, false, true);
- //
- // Map<Integer, Integer> consumerCounts = new HashMap<Integer, Integer>();
- //
- // for (int i = 0; i < numMessages; i++)
- // {
- // ClientMessage msg = browser.receive(RECEIVE_TIMEOUT);
- //
- // assertNotNull(msg);
- //
- // int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
- // int cnt = (Integer)msg.getProperty(new SimpleString("count"));
- //
- // Integer c = consumerCounts.get(tn);
- // if (c == null)
- // {
- // c = new Integer(cnt);
- // }
- //
- // if (cnt != c.intValue())
- // {
- // throw new Exception("Invalid count, expected " + c + " got " + cnt);
- // }
- //
- // c++;
- //
- // //Wrap
- // if (c == numMessages)
- // {
- // c = 0;
- // }
- //
- // consumerCounts.put(tn, c);
- //
- // msg.acknowledge();
- // }
- //
- // sessConsume.close();
- //
- // sessConsume = sf.createSession(false, true, true, false);
- //
- // browser = sessConsume.createConsumer(new SimpleString(threadNum + "sub"),
- // null, false, true);
- //
- // //Messages should still be there
- //
- // consumerCounts.clear();
- //
- // for (int i = 0; i < numMessages; i++)
- // {
- // ClientMessage msg = browser.receive(RECEIVE_TIMEOUT);
- //
- // assertNotNull(msg);
- //
- // int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
- // int cnt = (Integer)msg.getProperty(new SimpleString("count"));
- //
- // Integer c = consumerCounts.get(tn);
- // if (c == null)
- // {
- // c = new Integer(cnt);
- // }
- //
- // if (cnt != c.intValue())
- // {
- // throw new Exception("Invalid count, expected " + c + " got " + cnt);
- // }
- //
- // c++;
- //
- // //Wrap
- // if (c == numMessages)
- // {
- // c = 0;
- // }
- //
- // consumerCounts.put(tn, c);
- //
- // msg.acknowledge();
- // }
- //
- // sessConsume.close();
- //
- // sessSend.deleteQueue(new SimpleString(threadNum + "sub"));
- //
- // sessSend.close();
- //
- // long end = System.currentTimeMillis();
- //
- // log.info("duration " + (end - start));
- // }
-
- protected void doTestN(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- ClientSession sessCreate = sf.createSession(false, true, true);
-
- sessCreate.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
-
- ClientSession sess = sf.createSession(false, true, true);
-
- sess.stop();
-
- sess.start();
-
- sess.stop();
-
- ClientConsumer consumer = sess.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
-
- ClientProducer producer = sess.createProducer(ADDRESS);
-
- ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- producer.send(message);
-
- sess.start();
-
- ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(message2);
-
- message2.acknowledge();
-
- sess.stop();
-
- sess.start();
-
- sess.close();
-
- sessCreate.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
-
- sessCreate.close();
- }
-
- protected int getLatchWait()
- {
- return 60000;
- }
-
- protected int getNumIterations()
- {
- return 3;
- }
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- log.info("************ Starting test " + getName());
-
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- for (int i = 0 ; i < 10; i ++)
- log.info("************* Ending test " + getName());
-
- if (liveServer != null && liveServer.isStarted())
- {
- liveServer.stop();
- }
- liveServer = null;
- if (backupServer != null && backupServer.isStarted())
- {
- backupServer.stop();
- }
- backupServer = null;
-
- liveServer = null;
-
- backupServer = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- private void runTestMultipleThreads(final RunnableT runnable,
- final int numThreads,
- final boolean failOnCreateConnection) throws Exception
- {
- runTestMultipleThreads(runnable, numThreads, failOnCreateConnection, 1000);
- }
-
- private void runTestMultipleThreads(final RunnableT runnable,
- final int numThreads,
- final boolean failOnCreateConnection,
- final long failDelay) throws Exception
- {
-
- runMultipleThreadsFailoverTest(runnable, numThreads, getNumIterations(), failOnCreateConnection, failDelay);
- }
-
- /**
- * @return
- */
- protected ClientSessionFactoryInternal createSessionFactory()
- {
- final ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
- sf.setReconnectAttempts(-1);
- sf.setProducerWindowSize(32 * 1024);
-
- return sf;
- }
-
- protected void stop() throws Exception
- {
- log.info("** Stopping server");
- backupServer.stop();
-
- liveServer.stop();
-
- System.gc();
-
- assertEquals(0, InVMRegistry.instance.size());
- }
-
- private void sendMessages(final ClientSession sessSend,
- final ClientProducer producer,
- final int numMessages,
- final int threadNum) throws Exception
- {
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQBytesMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("threadnum"), threadNum);
- message.putIntProperty(new SimpleString("count"), i);
- setBody(message);
- producer.send(message);
- }
- }
-
- private void consumeMessages(final Set<ClientConsumer> consumers, final int numMessages, final int threadNum) throws Exception
- {
- // We make sure the messages arrive in the order they were sent from a particular producer
- Map<ClientConsumer, Map<Integer, Integer>> counts = new HashMap<ClientConsumer, Map<Integer, Integer>>();
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- Map<Integer, Integer> consumerCounts = counts.get(consumer);
-
- if (consumerCounts == null)
- {
- consumerCounts = new HashMap<Integer, Integer>();
- counts.put(consumer, consumerCounts);
- }
-
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
- int cnt = (Integer)msg.getProperty(new SimpleString("count"));
-
- Integer c = consumerCounts.get(tn);
- if (c == null)
- {
- c = new Integer(cnt);
- }
-
- if (tn == threadNum && cnt != c.intValue())
- {
- throw new Exception("Invalid count, expected " + tn + ": " + c + " got " + cnt);
- }
-
- c++;
-
- // Wrap
- if (c == numMessages)
- {
- c = 0;
- }
-
- consumerCounts.put(tn, c);
-
- msg.acknowledge();
- }
- }
- }
-
- // Inner classes -------------------------------------------------
-
- private class MyHandler implements MessageHandler
- {
- CountDownLatch latch = new CountDownLatch(1);
-
- private final Map<Integer, Integer> counts = new HashMap<Integer, Integer>();
-
- volatile String failure;
-
- final int tn;
-
- final int numMessages;
-
- volatile boolean done;
-
- synchronized void reset()
- {
- counts.clear();
-
- done = false;
-
- failure = null;
-
- latch = new CountDownLatch(1);
- }
-
- MyHandler(final int threadNum, final int numMessages)
- {
- tn = threadNum;
-
- this.numMessages = numMessages;
- }
-
- public synchronized void onMessage(final ClientMessage message)
- {
- try
- {
- message.acknowledge();
- }
- catch (HornetQException me)
- {
- log.error("Failed to process", me);
- }
-
- if (done)
- {
- return;
- }
-
- int threadNum = (Integer)message.getProperty(new SimpleString("threadnum"));
- int cnt = (Integer)message.getProperty(new SimpleString("count"));
-
- Integer c = counts.get(threadNum);
- if (c == null)
- {
- c = new Integer(cnt);
- }
-
- if (tn == threadNum && cnt != c.intValue())
- {
- failure = "Invalid count, expected " + threadNum + ":" + c + " got " + cnt;
- log.error(failure);
-
- latch.countDown();
- }
-
- if (!checkSize(message))
- {
- failure = "Invalid size on message";
- log.error(failure);
- latch.countDown();
- }
-
- if (tn == threadNum && c == numMessages - 1)
- {
- done = true;
- latch.countDown();
- }
-
- c++;
- // Wrap around at numMessages
- if (c == numMessages)
- {
- c = 0;
- }
-
- counts.put(threadNum, c);
-
- }
- }
-}
\ No newline at end of file
Added: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTest.java (rev 0)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.tests.integration.cluster.failover;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.hornetq.core.client.ClientMessage;
+import org.hornetq.core.config.Configuration;
+import org.hornetq.core.config.TransportConfiguration;
+import org.hornetq.core.config.impl.ConfigurationImpl;
+import org.hornetq.core.server.HornetQ;
+
+
+/**
+ *
+ * A MultiThreadRandomReattachTest
+ *
+ * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
+ *
+ *
+ */
+public class MultiThreadRandomReattachTest extends MultiThreadRandomReattachTestBase
+{
+ @Override
+ protected void start() throws Exception
+ {
+ Configuration liveConf = new ConfigurationImpl();
+ liveConf.setSecurityEnabled(false);
+ liveConf.getAcceptorConfigurations()
+ .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
+ Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
+ TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory");
+ connectors.put(backupTC.getName(), backupTC);
+ liveConf.setConnectorConfigurations(connectors);
+ liveConf.setBackupConnectorName(backupTC.getName());
+ liveServer = HornetQ.newHornetQServer(liveConf, false);
+ liveServer.start();
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomReattachTestBase#setBody(org.hornetq.core.client.ClientMessage)
+ */
+ @Override
+ protected void setBody(final ClientMessage message) throws Exception
+ {
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomReattachTestBase#checkSize(org.hornetq.core.client.ClientMessage)
+ */
+ @Override
+ protected boolean checkSize(final ClientMessage message)
+ {
+ return 0 == message.getBody().writerIndex();
+ }
+
+}
Added: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTestBase.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTestBase.java (rev 0)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/MultiThreadRandomReattachTestBase.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -0,0 +1,1500 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.tests.integration.cluster.failover;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.hornetq.core.client.ClientConsumer;
+import org.hornetq.core.client.ClientMessage;
+import org.hornetq.core.client.ClientProducer;
+import org.hornetq.core.client.ClientSession;
+import org.hornetq.core.client.ClientSessionFactory;
+import org.hornetq.core.client.MessageHandler;
+import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
+import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
+import org.hornetq.core.config.TransportConfiguration;
+import org.hornetq.core.exception.HornetQException;
+import org.hornetq.core.logging.Logger;
+import org.hornetq.core.remoting.impl.invm.InVMRegistry;
+import org.hornetq.core.server.HornetQServer;
+import org.hornetq.jms.client.HornetQBytesMessage;
+import org.hornetq.jms.client.HornetQTextMessage;
+import org.hornetq.utils.SimpleString;
+
+/**
+ * A MultiThreadRandomReattachTestBase
+ *
+ * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
+ * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
+ *
+ *
+ */
+public abstract class MultiThreadRandomReattachTestBase extends MultiThreadFailoverSupport
+{
+ private final Logger log = Logger.getLogger(getClass());
+
+ // Constants -----------------------------------------------------
+
+ private static final int RECEIVE_TIMEOUT = 30000;
+
+ private final int LATCH_WAIT = getLatchWait();
+
+ private int NUM_THREADS = getNumThreads();
+
+ // Attributes ----------------------------------------------------
+ protected static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
+
+ protected HornetQServer liveServer;
+
+ // Static --------------------------------------------------------
+
+ // Constructors --------------------------------------------------
+
+ // Public --------------------------------------------------------
+
+ public void testA() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestA(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+
+ }
+
+ public void testB() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestB(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testC() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestC(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testD() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestD(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testE() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestE(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testF() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestF(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testG() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestG(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testH() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestH(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testI() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestI(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testJ() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestJ(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testK() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestK(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ public void testL() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestL(sf);
+ }
+ }, NUM_THREADS, true, 10);
+ }
+
+ // public void testM() throws Exception
+ // {
+ // runTestMultipleThreads(new RunnableT()
+ // {
+ // public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ // {
+ // doTestM(sf, threadNum);
+ // }
+ // }, NUM_THREADS);
+ // }
+
+ public void testN() throws Exception
+ {
+ runTestMultipleThreads(new RunnableT()
+ {
+ @Override
+ public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ doTestN(sf, threadNum);
+ }
+ }, NUM_THREADS, false);
+ }
+
+ // Package protected ---------------------------------------------
+
+ // Protected -----------------------------------------------------
+
+ protected abstract void start() throws Exception;
+
+ protected abstract void setBody(ClientMessage message) throws Exception;
+
+ protected abstract boolean checkSize(ClientMessage message);
+
+ protected int getNumThreads()
+ {
+ return 10;
+ }
+
+ protected ClientSession createAutoCommitSession(ClientSessionFactory sf) throws Exception
+ {
+ return sf.createSession(false, true, true);
+ }
+
+ protected ClientSession createTransactionalSession(ClientSessionFactory sf) throws Exception
+ {
+ return sf.createSession(false, false, false);
+ }
+
+ protected void doTestA(final ClientSessionFactory sf, final int threadNum, final ClientSession session2) throws Exception
+ {
+ SimpleString subName = new SimpleString("sub" + threadNum);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, subName, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ ClientConsumer consumer = session.createConsumer(subName);
+
+ final int numMessages = 100;
+
+ sendMessages(session, producer, numMessages, threadNum);
+
+ session.start();
+
+ MyHandler handler = new MyHandler(threadNum, numMessages);
+
+ consumer.setMessageHandler(handler);
+
+ boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
+
+ if (!ok)
+ {
+ throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
+ " threadnum " +
+ threadNum);
+ }
+
+ if (handler.failure != null)
+ {
+ throw new Exception("Handler failed: " + handler.failure);
+ }
+
+ producer.close();
+
+ consumer.close();
+
+ session.deleteQueue(subName);
+
+ session.close();
+ }
+
+ protected void doTestA(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ ClientSession sessConsume = createAutoCommitSession(sf);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler(threadNum, numMessages);
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
+
+ if (!ok)
+ {
+ throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
+ " threadnum " +
+ threadNum);
+ }
+
+ if (handler.failure != null)
+ {
+ throw new Exception("Handler failed: " + handler.failure);
+ }
+ }
+
+ sessSend.close();
+
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestB(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ ClientSession sessConsume = createAutoCommitSession(sf);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler(threadNum, numMessages);
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
+
+ if (!ok)
+ {
+ throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
+ " threadnum " +
+ threadNum);
+ }
+
+ if (handler.failure != null)
+ {
+ throw new Exception("Handler failed: " + handler.failure);
+ }
+ }
+
+ sessSend.close();
+
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+
+ }
+
+ protected void doTestC(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ ClientSession sessConsume = createTransactionalSession(sf);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, false, false);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.rollback();
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.commit();
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler(threadNum, numMessages);
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
+
+ if (!ok)
+ {
+ throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
+ " threadnum " +
+ threadNum);
+ }
+
+ if (handler.failure != null)
+ {
+ throw new Exception("Handler failed: " + handler.failure);
+ }
+
+ handler.reset();
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestD(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + " sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, false, false);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, false, false);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.rollback();
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.commit();
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler(threadNum, numMessages);
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
+
+ if (!ok)
+ {
+ throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
+ " threadnum " +
+ threadNum);
+ }
+
+ if (handler.failure != null)
+ {
+ throw new Exception("Handler failed: " + handler.failure);
+ }
+ }
+
+ handlers.clear();
+
+ // Set handlers to null
+ for (ClientConsumer consumer : consumers)
+ {
+ consumer.setMessageHandler(null);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ // New handlers
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler(threadNum, numMessages);
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
+
+ if (!ok)
+ {
+ throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
+ " threadnum " +
+ threadNum);
+ }
+
+ if (handler.failure != null)
+ {
+ throw new Exception("Handler failed on rollback: " + handler.failure);
+ }
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + " sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ // Now with synchronous receive()
+
+ protected void doTestE(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, true, true);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ consumeMessages(consumers, numMessages, threadNum);
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestF(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, true, true);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ consumeMessages(consumers, numMessages, threadNum);
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestG(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, false, false);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, false, false);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.rollback();
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.commit();
+
+ consumeMessages(consumers, numMessages, threadNum);
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ consumeMessages(consumers, numMessages, threadNum);
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestH(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, false, false);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, false, false);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.rollback();
+
+ sendMessages(sessSend, producer, numMessages, threadNum);
+
+ sessSend.commit();
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ consumeMessages(consumers, numMessages, threadNum);
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ consumeMessages(consumers, numMessages, threadNum);
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString(threadNum + "sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestI(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ ClientSession sessCreate = sf.createSession(false, true, true);
+
+ sessCreate.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
+
+ ClientSession sess = sf.createSession(false, true, true);
+
+ sess.start();
+
+ ClientConsumer consumer = sess.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
+
+ ClientProducer producer = sess.createProducer(ADDRESS);
+
+ ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ producer.send(message);
+
+ ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(message2);
+
+ message2.acknowledge();
+
+ sess.close();
+
+ sessCreate.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
+
+ sessCreate.close();
+ }
+
+ protected void doTestJ(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ ClientSession sessCreate = sf.createSession(false, true, true);
+
+ sessCreate.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
+
+ ClientSession sess = sf.createSession(false, true, true);
+
+ sess.start();
+
+ ClientConsumer consumer = sess.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
+
+ ClientProducer producer = sess.createProducer(ADDRESS);
+
+ ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ producer.send(message);
+
+ ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(message2);
+
+ message2.acknowledge();
+
+ sess.close();
+
+ sessCreate.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
+
+ sessCreate.close();
+ }
+
+ protected void doTestK(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ ClientSession s = sf.createSession(false, false, false);
+
+ s.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
+
+ final int numConsumers = 100;
+
+ for (int i = 0; i < numConsumers; i++)
+ {
+ ClientConsumer consumer = s.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
+
+ consumer.close();
+ }
+
+ s.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
+
+ s.close();
+ }
+
+ /*
+ * This test tests failure during create connection
+ */
+ protected void doTestL(final ClientSessionFactory sf) throws Exception
+ {
+ final int numSessions = 10;
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ ClientSession session = sf.createSession(false, false, false);
+
+ session.close();
+ }
+ }
+
+ // Browsers
+ // FIXME - this test won't work until we use a proper iterator for browsing a queue.
+ // Making a copy of the queue for a browser consumer doesn't work well with replication since
+ // When replicating the create consumer (browser) to the backup, when executed on the backup the
+ // backup may have different messages in its queue since been added on different threads.
+ // So when replicating deliveries they may not be found.
+ // https://jira.jboss.org/jira/browse/JBMESSAGING-1433
+ // protected void doTestM(final ClientSessionFactory sf, final int threadNum) throws Exception
+ // {
+ // long start = System.currentTimeMillis();
+ //
+ // ClientSession sessSend = sf.createSession(false, true, true, false);
+ //
+ // ClientSession sessConsume = sf.createSession(false, true, true, false);
+ //
+ // sessConsume.createQueue(ADDRESS, new SimpleString(threadNum + "sub"), null, false, false);
+ //
+ // final int numMessages = 100;
+ //
+ // ClientProducer producer = sessSend.createProducer(ADDRESS);
+ //
+ // sendMessages(sessSend, producer, numMessages, threadNum);
+ //
+ // ClientConsumer browser = sessConsume.createConsumer(new SimpleString(threadNum + "sub"),
+ // null, false, true);
+ //
+ // Map<Integer, Integer> consumerCounts = new HashMap<Integer, Integer>();
+ //
+ // for (int i = 0; i < numMessages; i++)
+ // {
+ // ClientMessage msg = browser.receive(RECEIVE_TIMEOUT);
+ //
+ // assertNotNull(msg);
+ //
+ // int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
+ // int cnt = (Integer)msg.getProperty(new SimpleString("count"));
+ //
+ // Integer c = consumerCounts.get(tn);
+ // if (c == null)
+ // {
+ // c = new Integer(cnt);
+ // }
+ //
+ // if (cnt != c.intValue())
+ // {
+ // throw new Exception("Invalid count, expected " + c + " got " + cnt);
+ // }
+ //
+ // c++;
+ //
+ // //Wrap
+ // if (c == numMessages)
+ // {
+ // c = 0;
+ // }
+ //
+ // consumerCounts.put(tn, c);
+ //
+ // msg.acknowledge();
+ // }
+ //
+ // sessConsume.close();
+ //
+ // sessConsume = sf.createSession(false, true, true, false);
+ //
+ // browser = sessConsume.createConsumer(new SimpleString(threadNum + "sub"),
+ // null, false, true);
+ //
+ // //Messages should still be there
+ //
+ // consumerCounts.clear();
+ //
+ // for (int i = 0; i < numMessages; i++)
+ // {
+ // ClientMessage msg = browser.receive(RECEIVE_TIMEOUT);
+ //
+ // assertNotNull(msg);
+ //
+ // int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
+ // int cnt = (Integer)msg.getProperty(new SimpleString("count"));
+ //
+ // Integer c = consumerCounts.get(tn);
+ // if (c == null)
+ // {
+ // c = new Integer(cnt);
+ // }
+ //
+ // if (cnt != c.intValue())
+ // {
+ // throw new Exception("Invalid count, expected " + c + " got " + cnt);
+ // }
+ //
+ // c++;
+ //
+ // //Wrap
+ // if (c == numMessages)
+ // {
+ // c = 0;
+ // }
+ //
+ // consumerCounts.put(tn, c);
+ //
+ // msg.acknowledge();
+ // }
+ //
+ // sessConsume.close();
+ //
+ // sessSend.deleteQueue(new SimpleString(threadNum + "sub"));
+ //
+ // sessSend.close();
+ //
+ // long end = System.currentTimeMillis();
+ //
+ // log.info("duration " + (end - start));
+ // }
+
+ protected void doTestN(final ClientSessionFactory sf, final int threadNum) throws Exception
+ {
+ ClientSession sessCreate = sf.createSession(false, true, true);
+
+ sessCreate.createQueue(ADDRESS, new SimpleString(threadNum + ADDRESS.toString()), null, false);
+
+ ClientSession sess = sf.createSession(false, true, true);
+
+ sess.stop();
+
+ sess.start();
+
+ sess.stop();
+
+ ClientConsumer consumer = sess.createConsumer(new SimpleString(threadNum + ADDRESS.toString()));
+
+ ClientProducer producer = sess.createProducer(ADDRESS);
+
+ ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ producer.send(message);
+
+ sess.start();
+
+ ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(message2);
+
+ message2.acknowledge();
+
+ sess.stop();
+
+ sess.start();
+
+ sess.close();
+
+ sessCreate.deleteQueue(new SimpleString(threadNum + ADDRESS.toString()));
+
+ sessCreate.close();
+ }
+
+ protected int getLatchWait()
+ {
+ return 60000;
+ }
+
+ protected int getNumIterations()
+ {
+ return 3;
+ }
+
+ @Override
+ protected void setUp() throws Exception
+ {
+ super.setUp();
+
+ log.info("************ Starting test " + getName());
+ }
+
+ @Override
+ protected void tearDown() throws Exception
+ {
+ if (liveServer != null && liveServer.isStarted())
+ {
+ liveServer.stop();
+ }
+
+ liveServer = null;
+
+ super.tearDown();
+ }
+
+ // Private -------------------------------------------------------
+
+ private void runTestMultipleThreads(final RunnableT runnable,
+ final int numThreads,
+ final boolean failOnCreateConnection) throws Exception
+ {
+ runTestMultipleThreads(runnable, numThreads, failOnCreateConnection, 1000);
+ }
+
+ private void runTestMultipleThreads(final RunnableT runnable,
+ final int numThreads,
+ final boolean failOnCreateConnection,
+ final long failDelay) throws Exception
+ {
+
+ runMultipleThreadsFailoverTest(runnable, numThreads, getNumIterations(), failOnCreateConnection, failDelay);
+ }
+
+ /**
+ * @return
+ */
+ protected ClientSessionFactoryInternal createSessionFactory()
+ {
+ final ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+ sf.setReconnectAttempts(-1);
+ sf.setProducerWindowSize(32 * 1024);
+
+ return sf;
+ }
+
+ protected void stop() throws Exception
+ {
+ liveServer.stop();
+
+ System.gc();
+
+ assertEquals(0, InVMRegistry.instance.size());
+ }
+
+ private void sendMessages(final ClientSession sessSend,
+ final ClientProducer producer,
+ final int numMessages,
+ final int threadNum) throws Exception
+ {
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQBytesMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("threadnum"), threadNum);
+ message.putIntProperty(new SimpleString("count"), i);
+ setBody(message);
+ producer.send(message);
+ }
+ }
+
+ private void consumeMessages(final Set<ClientConsumer> consumers, final int numMessages, final int threadNum) throws Exception
+ {
+ // We make sure the messages arrive in the order they were sent from a particular producer
+ Map<ClientConsumer, Map<Integer, Integer>> counts = new HashMap<ClientConsumer, Map<Integer, Integer>>();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ Map<Integer, Integer> consumerCounts = counts.get(consumer);
+
+ if (consumerCounts == null)
+ {
+ consumerCounts = new HashMap<Integer, Integer>();
+ counts.put(consumer, consumerCounts);
+ }
+
+ ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(msg);
+
+ int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
+ int cnt = (Integer)msg.getProperty(new SimpleString("count"));
+
+ Integer c = consumerCounts.get(tn);
+ if (c == null)
+ {
+ c = new Integer(cnt);
+ }
+
+ if (tn == threadNum && cnt != c.intValue())
+ {
+ throw new Exception("Invalid count, expected " + tn + ": " + c + " got " + cnt);
+ }
+
+ c++;
+
+ // Wrap
+ if (c == numMessages)
+ {
+ c = 0;
+ }
+
+ consumerCounts.put(tn, c);
+
+ msg.acknowledge();
+ }
+ }
+ }
+
+ // Inner classes -------------------------------------------------
+
+ private class MyHandler implements MessageHandler
+ {
+ CountDownLatch latch = new CountDownLatch(1);
+
+ private final Map<Integer, Integer> counts = new HashMap<Integer, Integer>();
+
+ volatile String failure;
+
+ final int tn;
+
+ final int numMessages;
+
+ volatile boolean done;
+
+ synchronized void reset()
+ {
+ counts.clear();
+
+ done = false;
+
+ failure = null;
+
+ latch = new CountDownLatch(1);
+ }
+
+ MyHandler(final int threadNum, final int numMessages)
+ {
+ tn = threadNum;
+
+ this.numMessages = numMessages;
+ }
+
+ public synchronized void onMessage(final ClientMessage message)
+ {
+ try
+ {
+ message.acknowledge();
+ }
+ catch (HornetQException me)
+ {
+ log.error("Failed to process", me);
+ }
+
+ if (done)
+ {
+ return;
+ }
+
+ int threadNum = (Integer)message.getProperty(new SimpleString("threadnum"));
+ int cnt = (Integer)message.getProperty(new SimpleString("count"));
+
+ Integer c = counts.get(threadNum);
+ if (c == null)
+ {
+ c = new Integer(cnt);
+ }
+
+ if (tn == threadNum && cnt != c.intValue())
+ {
+ failure = "Invalid count, expected " + threadNum + ":" + c + " got " + cnt;
+ log.error(failure);
+
+ latch.countDown();
+ }
+
+ if (!checkSize(message))
+ {
+ failure = "Invalid size on message";
+ log.error(failure);
+ latch.countDown();
+ }
+
+ if (tn == threadNum && c == numMessages - 1)
+ {
+ done = true;
+ latch.countDown();
+ }
+
+ c++;
+ // Wrap around at numMessages
+ if (c == numMessages)
+ {
+ c = 0;
+ }
+
+ counts.put(threadNum, c);
+
+ }
+ }
+}
\ No newline at end of file
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,81 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.integration.transports.netty.TransportConstants;
-
-/**
- * A NettyMultiThreadRandomFailoverTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 18 Feb 2009 08:01:20
- *
- *
- */
-public class NettyMultiThreadRandomFailoverTest extends MultiThreadRandomFailoverTest
-{
- @Override
- protected void start() throws Exception
- {
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setJMXManagementEnabled(false);
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.PORT_PROP_NAME, TransportConstants.DEFAULT_PORT + 1);
- backupConf.getAcceptorConfigurations().clear();
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.integration.transports.netty.NettyAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupServer = HornetQ.newHornetQServer(backupConf, false);
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
- backupConf.setJMXManagementEnabled(false);
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations().clear();
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.integration.transports.netty.NettyAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.integration.transports.netty.NettyConnectorFactory",
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- liveServer.start();
- }
-
- @Override
- protected ClientSessionFactoryInternal createSessionFactory()
- {
- final ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.integration.transports.netty.NettyConnectorFactory"),
- new TransportConfiguration("org.hornetq.integration.transports.netty.NettyConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
- return sf;
- }
-
-}
Added: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomReattachTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomReattachTest.java (rev 0)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/NettyMultiThreadRandomReattachTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.tests.integration.cluster.failover;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
+import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
+import org.hornetq.core.config.Configuration;
+import org.hornetq.core.config.TransportConfiguration;
+import org.hornetq.core.config.impl.ConfigurationImpl;
+import org.hornetq.core.server.HornetQ;
+
+/**
+ * A NettyMultiThreadRandomReattachTest
+ *
+ * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
+ *
+ * Created 18 Feb 2009 08:01:20
+ *
+ *
+ */
+public class NettyMultiThreadRandomReattachTest extends MultiThreadRandomReattachTest
+{
+ @Override
+ protected void start() throws Exception
+ {
+ Configuration liveConf = new ConfigurationImpl();
+ liveConf.setJMXManagementEnabled(false);
+ liveConf.setSecurityEnabled(false);
+ liveConf.getAcceptorConfigurations().clear();
+ liveConf.getAcceptorConfigurations()
+ .add(new TransportConfiguration("org.hornetq.integration.transports.netty.NettyAcceptorFactory"));
+ Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
+ TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.integration.transports.netty.NettyConnectorFactory");
+ connectors.put(backupTC.getName(), backupTC);
+ liveConf.setConnectorConfigurations(connectors);
+ liveConf.setBackupConnectorName(backupTC.getName());
+ liveServer = HornetQ.newHornetQServer(liveConf, false);
+ liveServer.start();
+ }
+
+ @Override
+ protected ClientSessionFactoryInternal createSessionFactory()
+ {
+ final ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.integration.transports.netty.NettyConnectorFactory"));
+
+ sf.setProducerWindowSize(32 * 1024);
+ return sf;
+ }
+
+}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/OrderingOnBackupTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/OrderingOnBackupTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/OrderingOnBackupTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,856 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.transaction.xa.XAResource;
-import javax.transaction.xa.Xid;
-
-import org.hornetq.core.buffers.ChannelBuffers;
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.MessageHandler;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.paging.Page;
-import org.hornetq.core.paging.PagedMessage;
-import org.hornetq.core.paging.PagingManager;
-import org.hornetq.core.paging.impl.TestSupportPageStore;
-import org.hornetq.core.postoffice.QueueBinding;
-import org.hornetq.core.server.MessageReference;
-import org.hornetq.core.server.ServerMessage;
-import org.hornetq.core.server.impl.QueueImpl;
-import org.hornetq.tests.util.RandomUtil;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * It validates if the messages are in the same ordering on the page system between the backup and live nodes.
- *
- * This test is valid as long as we want to guarantee strict ordering on both nodes for paged messages between backup and live nodes.
- *
- * If we change this concept anyway this test may become invalid and we would need to delete it.
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- *
- */
-public class OrderingOnBackupTest extends FailoverTestBase
-{
-
- // Constants -----------------------------------------------------
-
- private static final Logger log = Logger.getLogger(OrderingOnBackupTest.class);
-
- // Attributes ----------------------------------------------------
-
- // Static --------------------------------------------------------
-
- private static void debug(String message)
- {
- log.info(message);
- }
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
- public void testPageOrderingLiveAndBackupProducerOnly() throws Exception
- {
- internalTestPageOrderingLiveAndBackup(false);
- }
-
- public void testPageOrderingLiveAndBackupConsume() throws Exception
- {
- internalTestPageOrderingLiveAndBackup(true);
- }
-
- private void internalTestPageOrderingLiveAndBackup(boolean consumeMessages) throws Exception
- {
- final SimpleString threadIDKey = new SimpleString("THREAD_ID");
- final SimpleString sequenceIDKey = new SimpleString("SEQUENCE_ID");
- final SimpleString ADDRESS = new SimpleString("SOME_QUEUE");
-
- final int NUMBER_OF_THREADS = 10;
- final int NUMBER_OF_MESSAGES = 200;
-
- final int NUMBER_OF_HANDLERS = consumeMessages ? NUMBER_OF_THREADS : 0;
-
- setUpFailoverServers(true, 100 * 1024, 50 * 1024);
-
- final ClientSessionFactory factory = createFailoverFactory();
-
- ClientSession session = factory.createSession(false, true, true);
- for (int i = 0; i < NUMBER_OF_THREADS; i++)
- {
- session.createQueue(ADDRESS, ADDRESS.concat("-" + i), true);
- }
- session.close();
-
- MyHandler handlers[] = new MyHandler[NUMBER_OF_HANDLERS];
-
- for (int i = 0; i < handlers.length; i++)
- {
- handlers[i] = new MyHandler(factory, ADDRESS.concat("-" + i), NUMBER_OF_MESSAGES * 10);
- }
-
- final CountDownLatch flagAlign = new CountDownLatch(NUMBER_OF_THREADS);
- final CountDownLatch flagStart = new CountDownLatch(1);
-
- class ProducerThread extends Thread
- {
- Throwable e;
-
- final int threadID;
-
- ProducerThread(int threadID)
- {
- this.threadID = threadID;
- }
-
- public void run()
- {
- try
- {
- ClientSession session = factory.createSession(false, true, true);
- ClientProducer producer = session.createProducer(ADDRESS);
-
- // I want to jinx all this by having everybody start sending at the same time
- flagAlign.countDown();
- flagStart.await();
-
- for (int i = 0; i < NUMBER_OF_MESSAGES; i++)
- {
- ClientMessage msg = session.createClientMessage(true);
- msg.setBody(ChannelBuffers.wrappedBuffer(new byte[512]));
- msg.getProperties().putIntProperty(threadIDKey, this.threadID);
- msg.getProperties().putIntProperty(sequenceIDKey, i);
- producer.send(msg);
- }
-
- session.close();
-
- }
- catch (Throwable e)
- {
- // System.out => Hudson/JUNIT reports
- e.printStackTrace();
- this.e = e;
- }
-
- }
- }
-
- ProducerThread threads[] = new ProducerThread[NUMBER_OF_THREADS];
-
- for (int i = 0; i < threads.length; i++)
- {
- threads[i] = new ProducerThread(i);
- threads[i].start();
- }
-
- assertTrue("Error initializing some of the threads", flagAlign.await(10, TimeUnit.SECONDS));
-
- flagStart.countDown();
-
- for (ProducerThread t : threads)
- {
- t.join();
- }
-
- for (ProducerThread t : threads)
- {
- if (t.e != null)
- {
- throw new Exception("Test Failed", t.e);
- }
- }
-
- for (MyHandler handler : handlers)
- {
- handler.close();
- if (handler.failure != null)
- {
- throw new Exception("Failure on consumer", handler.failure);
- }
- }
-
- PagingManager livePagingManager = liveServer.getPostOffice().getPagingManager();
- PagingManager backupPagingManager = backupServer.getPostOffice().getPagingManager();
-
- TestSupportPageStore livePagingStore = (TestSupportPageStore)livePagingManager.getPageStore(ADDRESS);
- TestSupportPageStore backupPagingStore = (TestSupportPageStore)backupPagingManager.getPageStore(ADDRESS);
-
- debug("Pages: " + livePagingStore.getNumberOfPages() + " on backup: " + backupPagingStore.getNumberOfPages());
-
- if (consumeMessages)
- {
- if (livePagingStore.getNumberOfPages() == backupPagingStore.getNumberOfPages() - 1)
- {
- // The live node may have one extra page in front of the backup
- backupPagingStore.depage();
- }
- }
-
- assertEquals(livePagingStore.getNumberOfPages(), backupPagingStore.getNumberOfPages());
-
- Page livePage = null;
- Page backupPage = null;
-
- while (true)
- {
- livePage = livePagingStore.depage();
-
- if (livePage == null)
- {
- assertNull(backupPagingStore.depage());
- break;
- }
-
- backupPage = backupPagingStore.depage();
-
- assertNotNull(backupPage);
-
- livePage.open();
- backupPage.open();
-
- List<PagedMessage> liveMessages = livePage.read();
- List<PagedMessage> backupMessages = backupPage.read();
-
- livePage.close();
- backupPage.close();
-
- assertEquals(liveMessages.size(), backupMessages.size());
-
- Iterator<PagedMessage> backupIterator = backupMessages.iterator();
-
- for (PagedMessage liveMsg : liveMessages)
- {
- PagedMessage backupMsg = backupIterator.next();
- assertNotNull(backupMsg);
-
- ServerMessage liveSrvMsg = liveMsg.getMessage(null);
- ServerMessage backupSrvMsg = liveMsg.getMessage(null);
-
- assertEquals(liveSrvMsg.getMessageID(), backupSrvMsg.getMessageID());
- assertEquals(liveSrvMsg.getProperty(threadIDKey), backupSrvMsg.getProperty(threadIDKey));
- assertEquals(liveSrvMsg.getProperty(sequenceIDKey), backupSrvMsg.getProperty(sequenceIDKey));
- }
- }
-
- }
-
- public void testDeliveryOrderOnTransactionalRollbackMultiThreadXA() throws Exception
- {
- internalTestDeliveryOrderOnTransactionalRollbackMultiThread(true);
- }
-
- public void testDeliveryOrderOnTransactionalRollbackMultiThread() throws Exception
- {
- internalTestDeliveryOrderOnTransactionalRollbackMultiThread(false);
- }
-
- public void internalTestDeliveryOrderOnTransactionalRollbackMultiThread(final boolean isXA) throws Exception
- {
-
- final SimpleString ADDRESS = new SimpleString("TEST");
- final SimpleString PROPERTY_KEY = new SimpleString("KEY-STR");
-
- final AtomicInteger errors = new AtomicInteger(0);
-
- int NTHREADS = 30;
- final int NMESSAGES = 1000;
-
- class ProdThread extends Thread
- {
- final CountDownLatch latchAlign;
-
- final CountDownLatch latchStart;
-
- final ClientSessionFactory sf;
-
- ProdThread(final CountDownLatch latchAlign, final CountDownLatch latchStart, final ClientSessionFactory sf)
- {
- this.latchAlign = latchAlign;
- this.latchStart = latchStart;
- this.sf = sf;
- }
-
- @Override
- public void run()
- {
- ClientSession sess = null;
- try
- {
- latchAlign.countDown();
- latchStart.await();
-
- sess = sf.createSession(false, false, false);
-
- ClientProducer prod = sess.createProducer(ADDRESS);
-
- for (int i = 0; i < NMESSAGES; i++)
- {
- ClientMessage msg = createTextMessage(sess, "test" + i, false);
- msg.putStringProperty(PROPERTY_KEY, RandomUtil.randomSimpleString());
- prod.send(msg);
- }
-
- sess.commit();
- }
- catch (Throwable e)
- {
- e.printStackTrace();
- errors.incrementAndGet();
- }
- finally
- {
- try
- {
- sess.close();
- }
- catch (Throwable ignored)
- {
- }
- }
-
- }
- };
-
- class ConsumerThread extends Thread
- {
- final ClientSessionFactory sf;
-
- volatile ClientSession sess;
-
- final CountDownLatch latchAlign;
-
- final CountDownLatch latchStart;
-
- Xid xid = null;
-
- final boolean rollback;
-
- ConsumerThread(final ClientSessionFactory sf,
- final CountDownLatch latchAlign,
- final CountDownLatch latchStart,
- final boolean rollback)
- {
- this.sf = sf;
- this.latchAlign = latchAlign;
- this.latchStart = latchStart;
- this.rollback = rollback;
- }
-
- public void close()
- {
- try
- {
- if (xid != null)
- {
- sess.rollback(xid);
- xid = null;
- }
- sess.close();
- sess = null;
- }
- catch (Exception e)
- {
- e.printStackTrace();
- }
-
- }
-
- @Override
- public void run()
- {
-
- ClientConsumer cons = null;
- try
- {
- latchAlign.countDown();
- latchStart.await();
-
- sess = sf.createSession(isXA, false, false);
-
- if (isXA)
- {
- xid = newXID();
- sess.start(xid, XAResource.TMNOFLAGS);
- }
-
- cons = sess.createConsumer(ADDRESS);
-
- sess.start();
-
- ClientMessage msg = null;
-
- while ((msg = cons.receive(1000)) != null)
- {
- msg.acknowledge();
- }
- }
- catch (Throwable e)
- {
- e.printStackTrace();
- errors.incrementAndGet();
- }
- finally
- {
- try
- {
- if (isXA)
- {
- sess.end(xid, XAResource.TMSUCCESS);
- }
- if (rollback)
- {
- if (isXA)
- {
- sess.rollback(xid);
- xid = null;
- }
- else
- {
- sess.rollback();
- }
- cons.close();
- }
- }
- catch (Exception e)
- {
- e.printStackTrace();
- errors.incrementAndGet();
- }
-
- }
- }
- };
-
- this.setUpFailoverServers(false, -1, 512);
-
- ClientSessionFactory sf = createFailoverFactory();
- sf.setConsumerWindowSize(-1);
-
- ClientSession s = sf.createSession(false, true, true);
-
- s.createQueue(ADDRESS, ADDRESS, true);
-
- s.close();
-
- CountDownLatch latchAlign = new CountDownLatch(NTHREADS);
-
- CountDownLatch latchStart = new CountDownLatch(1);
-
- ProdThread pthreads[] = new ProdThread[NTHREADS];
-
- for (int i = 0; i < NTHREADS; i++)
- {
- pthreads[i] = new ProdThread(latchAlign, latchStart, sf);
- pthreads[i].start();
- }
-
- latchAlign.await();
- latchStart.countDown();
-
- for (Thread t : pthreads)
- {
- t.join();
- }
-
- assertEquals(0, errors.get());
-
- compareQueues(ADDRESS, PROPERTY_KEY, NTHREADS * NMESSAGES);
-
- ConsumerThread cthreads[] = new ConsumerThread[NTHREADS];
-
- log.info("********************** Consuming messages ****************************");
-
- latchAlign = new CountDownLatch(NTHREADS);
-
- latchStart = new CountDownLatch(1);
-
- for (int i = 0; i < NTHREADS; i++)
- {
- // 50% of the consumers will close the session without ACKing messages what cause them to be redelivered.
- // This shouldn't affect delivery on backup
- cthreads[i] = new ConsumerThread(sf, latchAlign, latchStart, i % 2 == 0);
- cthreads[i].start();
- }
-
- latchAlign.await();
- latchStart.countDown();
-
- for (ConsumerThread t : cthreads)
- {
- t.join();
- }
-
- assertEquals(0, errors.get());
-
- for (ConsumerThread t : cthreads)
- {
- if (t.sess != null)
- {
- t.close();
- }
- }
-
- sf.close();
-
- compareQueues(ADDRESS, PROPERTY_KEY, NTHREADS * NMESSAGES);
-
- stopServers();
- }
-
- public void testDeliveryOrderOnRedeliveryMultiThread() throws Exception
- {
-
- final SimpleString ADDRESS = new SimpleString("TEST");
- final SimpleString PROPERTY_KEY = new SimpleString("KEY-STR");
-
- final AtomicInteger errors = new AtomicInteger(0);
-
- int NTHREADS = 30;
- final int NMESSAGES = 1000;
-
- class ProdThread extends Thread
- {
- final CountDownLatch latchAlign;
-
- final CountDownLatch latchStart;
-
- final ClientSessionFactory sf;
-
- ProdThread(final CountDownLatch latchAlign, final CountDownLatch latchStart, final ClientSessionFactory sf)
- {
- this.latchAlign = latchAlign;
- this.latchStart = latchStart;
- this.sf = sf;
- }
-
- @Override
- public void run()
- {
- ClientSession sess = null;
- try
- {
- latchAlign.countDown();
- latchStart.await();
-
- sess = sf.createSession(false, true, true);
-
- ClientProducer prod = sess.createProducer(ADDRESS);
-
- for (int i = 0; i < NMESSAGES; i++)
- {
- ClientMessage msg = createTextMessage(sess, "test" + i, false);
- msg.putStringProperty(PROPERTY_KEY, RandomUtil.randomSimpleString());
- prod.send(msg);
- }
- }
- catch (Throwable e)
- {
- e.printStackTrace();
- errors.incrementAndGet();
- }
- finally
- {
- try
- {
- sess.close();
- }
- catch (Throwable ignored)
- {
- }
- }
-
- }
- };
-
- class ConsumerThread extends Thread
- {
- final ClientSessionFactory sf;
-
- volatile ClientSession sess;
-
- final CountDownLatch latchAlign;
-
- final CountDownLatch latchStart;
-
- final boolean closeSession;
-
- ConsumerThread(final ClientSessionFactory sf,
- final CountDownLatch latchAlign,
- final CountDownLatch latchStart,
- final boolean closeSession)
- {
- this.sf = sf;
- this.latchAlign = latchAlign;
- this.latchStart = latchStart;
- this.closeSession = closeSession;
- }
-
- @Override
- public void run()
- {
- ClientConsumer cons = null;
- try
- {
- latchAlign.countDown();
- latchStart.await();
-
- sess = sf.createSession(false, true, true);
-
- cons = sess.createConsumer(ADDRESS);
-
- sess.start();
-
- ClientMessage msg = null;
-
- while ((msg = cons.receive(1000)) != null)
- {
- // do not ack. Forcing it to come back to head of queue thorugh cancel & rollback
- // debug("Received Msg = " + getTextMessage(msg));
- }
- }
- catch (Throwable e)
- {
- e.printStackTrace();
- errors.incrementAndGet();
- }
- finally
- {
- try
- {
- sess.commit();
- }
- catch (HornetQException e)
- {
- e.printStackTrace();
- }
- if (closeSession)
- {
- try
- {
- cons.close();
- }
- catch (Throwable ignored)
- {
- }
- }
- }
- }
- };
-
- this.setUpFailoverServers(false, -1, 512);
-
- ClientSessionFactory sf = createFailoverFactory();
- sf.setConsumerWindowSize(-1);
-
- ClientSession s = sf.createSession(false, true, true);
-
- s.createQueue(ADDRESS, ADDRESS, true);
-
- s.close();
-
- CountDownLatch latchAlign = new CountDownLatch(NTHREADS);
-
- CountDownLatch latchStart = new CountDownLatch(1);
-
- ProdThread pthreads[] = new ProdThread[NTHREADS];
-
- for (int i = 0; i < NTHREADS; i++)
- {
- pthreads[i] = new ProdThread(latchAlign, latchStart, sf);
- pthreads[i].start();
- }
-
- latchAlign.await();
- latchStart.countDown();
-
- for (Thread t : pthreads)
- {
- t.join();
- }
-
- assertEquals(0, errors.get());
-
- compareQueues(ADDRESS, PROPERTY_KEY, NTHREADS * NMESSAGES);
-
- ConsumerThread cthreads[] = new ConsumerThread[NTHREADS];
-
- log.info("********************** Consuming messages ****************************");
-
- latchAlign = new CountDownLatch(NTHREADS);
-
- latchStart = new CountDownLatch(1);
-
- for (int i = 0; i < NTHREADS; i++)
- {
- // 50% of the consumers will close the session without ACKing messages what cause them to be redelivered.
- // This shouldn't affect delivery on backup
- cthreads[i] = new ConsumerThread(sf, latchAlign, latchStart, i % 2 == 0);
- cthreads[i].start();
- }
-
- latchAlign.await();
- latchStart.countDown();
-
- for (ConsumerThread t : cthreads)
- {
- t.join();
- }
-
- assertEquals(0, errors.get());
-
- for (ConsumerThread t : cthreads)
- {
- if (t.sess != null)
- {
- t.sess.close();
- }
- }
-
- sf.close();
-
- compareQueues(ADDRESS, PROPERTY_KEY, NTHREADS * NMESSAGES);
-
- stopServers();
- // ClientProducer p = s
-
- }
-
- /**
- * Compare if a Queue on Backup and Live server are identical
- * @param ADDRESS
- * @param propertyToAssert
- * @param NTHREADS
- * @param NMESSAGES
- * @throws Exception
- */
- private void compareQueues(final SimpleString ADDRESS,
- final SimpleString propertyToAssert,
- int expectedNumberOfMessages) throws Exception
- {
- List<QueueBinding> blive = getLocalQueueBindings(liveServer.getPostOffice(), ADDRESS.toString());
- List<QueueBinding> bbackup = getLocalQueueBindings(backupServer.getPostOffice(), ADDRESS.toString());
-
- assertEquals(1, blive.size());
- assertEquals(1, bbackup.size());
-
- QueueImpl qlive = (QueueImpl)blive.get(0).getQueue();
- QueueImpl qbackup = (QueueImpl)bbackup.get(0).getQueue();
-
- assertEquals(expectedNumberOfMessages, qlive.list(null).size());
-
- assertEquals(expectedNumberOfMessages, qbackup.list(null).size());
-
- Iterator<MessageReference> iterBackup = qbackup.list(null).iterator();
-
- for (MessageReference refLive : qlive.list(null))
- {
- assertTrue(iterBackup.hasNext());
- MessageReference refBackup = iterBackup.next();
-
- assertEquals(refLive.getMessage().getMessageID(), refBackup.getMessage().getMessageID());
- assertNotNull(refLive.getMessage().getProperty(propertyToAssert));
- assertEquals(refLive.getMessage().getProperty(propertyToAssert), refBackup.getMessage()
- .getProperty(propertyToAssert));
- }
-
- assertFalse(iterBackup.hasNext());
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-
- class MyHandler implements MessageHandler
- {
- final ClientSession session;
-
- final ClientConsumer consumer;
-
- volatile boolean started = true;
-
- final int msgs;
-
- volatile int receivedMsgs = 0;
-
- final CountDownLatch latch;
-
- Throwable failure;
-
- MyHandler(ClientSessionFactory sf, SimpleString address, final int msgs) throws Exception
- {
- this.session = sf.createSession(null, null, false, true, true, false, 0);
- this.consumer = session.createConsumer(address);
- consumer.setMessageHandler(this);
- this.session.start();
- this.msgs = msgs;
- latch = new CountDownLatch(msgs);
- }
-
- public synchronized void close() throws Exception
- {
- session.close();
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.client.MessageHandler#onMessage(org.hornetq.core.client.ClientMessage)
- */
- public synchronized void onMessage(ClientMessage message)
- {
- try
- {
- if (!started)
- {
- throw new IllegalStateException("Stopped Handler received message");
- }
-
- if (receivedMsgs++ == msgs)
- {
- debug("done");
- started = false;
- session.stop();
- }
-
- message.acknowledge();
-
- if (!started)
- {
- latch.countDown();
- }
-
- }
- catch (Throwable e)
- {
- this.failure = e;
- }
- }
-
- }
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverMultiThreadTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverMultiThreadTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverMultiThreadTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,610 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import javax.transaction.xa.Xid;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.MessageHandler;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.core.server.JournalType;
-import org.hornetq.core.settings.impl.AddressSettings;
-import org.hornetq.jms.client.HornetQBytesMessage;
-import org.hornetq.utils.SimpleString;
-
-/**
- * A PagingFailoverMultiThreadTest
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- *
- */
-public class PagingFailoverMultiThreadTest extends MultiThreadFailoverSupport
-{
-
- // Constants -----------------------------------------------------
- private static final int RECEIVE_TIMEOUT = 20000;
-
- final int PAGE_SIZE = 512;
-
- final int MAX_GLOBAL = 40 * PAGE_SIZE;
-
- final boolean CREATE_AT_START = true;
-
- private final int LATCH_WAIT = 50000;
-
- private final int NUM_THREADS = 10;
-
- private final int NUM_SESSIONS = 10;
-
- private final Logger log = Logger.getLogger(this.getClass());
-
- // Attributes ----------------------------------------------------
-
- protected static final SimpleString ADDRESS_GLOBAL = new SimpleString("FailoverTestAddress");
-
- protected HornetQServer liveServer;
-
- protected HornetQServer backupServer;
-
- protected Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testFoo()
- {
-
- }
-
- // Currently disabled - https://jira.jboss.org/jira/browse/JBMESSAGING-1558
- public void disabled_testB() throws Exception
- {
- runMultipleThreadsFailoverTest(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestB(sf, threadNum);
- }
- }, NUM_THREADS, 20, false, 1000);
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected void setBody(final ClientMessage message) throws Exception
- {
- message.getBody().writeBytes(new byte[256]);
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTestBase#checkSize(org.hornetq.core.client.ClientMessage)
- */
- protected boolean checkSize(final ClientMessage message)
- {
- return 256 == message.getBody().writerIndex();
- }
-
- protected SimpleString createAddressName(int threadNum)
- {
- return ADDRESS_GLOBAL.concat("_thread-" + threadNum);
- }
-
- protected SimpleString createSubName(int thread, int sequence)
- {
- return new SimpleString(thread + "sub" + sequence);
- }
-
- protected void doTestB(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- SimpleString ADDRESS = createAddressName(threadNum);
-
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 1;
-
- Set<MyInfo> infos = new HashSet<MyInfo>();
-
- for (int i = 0; i < NUM_SESSIONS; i++)
- {
- SimpleString subName = createSubName(threadNum, i);
-
- ClientSession sessConsume = sf.createSession(null, null, false, true, true, false, 0);
-
- if (!CREATE_AT_START)
- {
- sessConsume.createQueue(ADDRESS, subName, null, true);
- }
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- infos.add(new MyInfo(sessConsume, consumer));
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- for (MyInfo info : infos)
- {
- info.session.start();
- }
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (MyInfo info : infos)
- {
- MyHandler handler = new MyHandler(threadNum, numMessages, info.session, info.consumer);
-
- handler.start();
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
-
- assertNull(handler.consumer.receive(250));
- }
-
- sessSend.close();
-
- for (MyInfo info : infos)
- {
- info.session.close();
- }
-
- if (!CREATE_AT_START)
- {
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
-
- }
-
- protected void stop() throws Exception
- {
- backupServer.stop();
-
- liveServer.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupServer = null;
-
- liveServer = null;
- }
-
- private void sendMessages(final ClientSession sessSend,
- final ClientProducer producer,
- final int numMessages,
- final int threadNum) throws Exception
- {
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQBytesMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("threadnum"), threadNum);
- message.putIntProperty(new SimpleString("count"), i);
- setBody(message);
- producer.send(message);
- }
- }
-
- private void consumeMessages(final Set<ClientConsumer> consumers, final int numMessages, final int threadNum) throws Exception
- {
- // We make sure the messages arrive in the order they were sent from a particular producer
- Map<ClientConsumer, Map<Integer, Integer>> counts = new HashMap<ClientConsumer, Map<Integer, Integer>>();
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- Map<Integer, Integer> consumerCounts = counts.get(consumer);
-
- if (consumerCounts == null)
- {
- consumerCounts = new HashMap<Integer, Integer>();
- counts.put(consumer, consumerCounts);
- }
-
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
- int cnt = (Integer)msg.getProperty(new SimpleString("count"));
-
- Integer c = consumerCounts.get(tn);
- if (c == null)
- {
- c = new Integer(cnt);
- }
-
- if (tn == threadNum && cnt != c.intValue())
- {
- throw new Exception("Invalid count, expected " + tn + ": " + c + " got " + cnt);
- }
-
- c++;
-
- // Wrap
- if (c == numMessages)
- {
- c = 0;
- }
-
- consumerCounts.put(tn, c);
-
- msg.acknowledge();
- }
- }
- }
-
- /**
- * @return
- */
- protected ClientSessionFactoryInternal createSessionFactory()
- {
- final ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
- sf.setProducerWindowSize(32 * 1024);
- return sf;
- }
-
- @Override
- protected void start() throws Exception
- {
- setUpFailoverServers(true, MAX_GLOBAL, PAGE_SIZE);
-
- if (CREATE_AT_START)
- {
- // TODO: Remove this part here
- ClientSessionFactory sf = createSessionFactory();
-
- ClientSession session = sf.createSession(false, true, true);
-
- for (int threadNum = 0; threadNum < NUM_THREADS; threadNum++)
- {
- SimpleString ADDRESS = createAddressName(threadNum);
-
- for (int i = 0; i < NUM_SESSIONS; i++)
- {
- SimpleString subName = createSubName(threadNum, i);
- session.createQueue(ADDRESS, subName, null, true);
- }
- }
- session.close();
-
- }
-
- }
-
- protected void setUpFailoverServers(boolean fileBased, final int maxGlobalSize, final int pageSize) throws Exception
- {
- deleteDirectory(new File(getTestDir()));
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupConf.setClustered(true);
- backupConf.setBackup(true);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName(), backupParams));
-
- if (fileBased)
- {
- clearData(getTestDir() + "/backup");
-
- backupConf.setJournalDirectory(getJournalDir(getTestDir() + "/backup"));
- backupConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/backup"));
- backupConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/backup"));
- backupConf.setPagingDirectory(getPageDir(getTestDir() + "/backup"));
- backupConf.setJournalFileSize(100 * 1024);
-
- backupConf.setJournalType(JournalType.ASYNCIO);
-
- backupServer = HornetQ.newHornetQServer(backupConf);
-
- AddressSettings defaultSetting = new AddressSettings();
- defaultSetting.setPageSizeBytes(pageSize);
- defaultSetting.setMaxSizeBytes(maxGlobalSize);
-
- backupServer.getAddressSettingsRepository().addMatch("#", defaultSetting);
-
- }
- else
- {
- backupServer = HornetQ.newHornetQServer(backupConf, false);
- }
-
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.setClustered(true);
-
- TransportConfiguration liveTC = new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName());
- liveConf.getAcceptorConfigurations().add(liveTC);
-
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
-
- TransportConfiguration backupTC = new TransportConfiguration(INVM_CONNECTOR_FACTORY,
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
-
- if (fileBased)
- {
- liveConf.setJournalDirectory(getJournalDir(getTestDir() + "/live"));
- liveConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/live"));
- liveConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/live"));
- liveConf.setPagingDirectory(getPageDir(getTestDir() + "/live"));
-
- liveConf.setJournalFileSize(100 * 1024);
-
- liveConf.setJournalType(JournalType.ASYNCIO);
-
- liveServer = HornetQ.newHornetQServer(liveConf);
-
- AddressSettings defaultSetting = new AddressSettings();
- defaultSetting.setPageSizeBytes(pageSize);
- defaultSetting.setMaxSizeBytes(maxGlobalSize);
-
- liveServer.getAddressSettingsRepository().addMatch("#", defaultSetting);
-
- }
- else
- {
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- }
-
- AddressSettings settings = new AddressSettings();
- settings.setPageSizeBytes(pageSize);
-
- liveServer.getAddressSettingsRepository().addMatch("#", settings);
- backupServer.getAddressSettingsRepository().addMatch("#", settings);
-
- clearData(getTestDir() + "/live");
-
- liveServer.start();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
- private class MyInfo
- {
- final ClientSession session;
-
- final ClientConsumer consumer;
-
- public MyInfo(final ClientSession session, final ClientConsumer consumer)
- {
- this.session = session;
- this.consumer = consumer;
- }
- }
-
- private class MyHandler implements MessageHandler
- {
- CountDownLatch latch = new CountDownLatch(1);
-
- private final Map<Integer, Integer> counts = new HashMap<Integer, Integer>();
-
- volatile String failure;
-
- final int tn;
-
- final int numMessages;
-
- final ClientSession session;
-
- final ClientConsumer consumer;
-
- volatile Xid xid;
-
- volatile boolean done;
-
- volatile boolean started = false;
-
- volatile boolean commit = false;
-
- synchronized void start() throws Exception
- {
- counts.clear();
-
- done = false;
-
- failure = null;
-
- latch = new CountDownLatch(1);
-
- started = true;
- consumer.setMessageHandler(this);
- session.start();
- }
-
- synchronized void stop() throws Exception
- {
- session.stop();
- // FIXME: Remove this line when https://jira.jboss.org/jira/browse/JBMESSAGING-1549 is done
- consumer.setMessageHandler(null);
- started = false;
- }
-
- synchronized void close() throws Exception
- {
- stop();
- session.close();
- }
-
- MyHandler(final int threadNum, final int numMessages, final ClientSession session, final ClientConsumer consumer) throws Exception
- {
- tn = threadNum;
-
- this.numMessages = numMessages;
-
- this.session = session;
-
- this.consumer = consumer;
-
- }
-
- public void setCommitOnComplete(boolean commit)
- {
- this.commit = commit;
- }
-
- public synchronized void onMessage(final ClientMessage message)
- {
-
- if (!started)
- {
- this.failure = "Received message with session stopped (thread = " + tn + ")";
- log.error(failure);
- return;
- }
-
- // log.info("*** handler got message");
- try
- {
- message.acknowledge();
- }
- catch (HornetQException me)
- {
- log.error("Failed to process", me);
- }
-
- if (done)
- {
- return;
- }
-
- int threadNum = (Integer)message.getProperty(new SimpleString("threadnum"));
- int cnt = (Integer)message.getProperty(new SimpleString("count"));
-
- Integer c = counts.get(threadNum);
- if (c == null)
- {
- c = new Integer(cnt);
- }
-
- // log.info(System.identityHashCode(this) + " consumed message " + threadNum + ":" + cnt);
-
- if (tn == threadNum && cnt != c.intValue())
- {
- failure = "Invalid count, expected " + threadNum + ":" + c + " got " + cnt;
- log.error(failure);
-
- latch.countDown();
- }
-
- if (!checkSize(message))
- {
- failure = "Invalid size on message";
- log.error(failure);
- latch.countDown();
- }
-
- if (tn == threadNum && c == numMessages - 1)
- {
- done = true;
- try
- {
- this.stop();
- }
- catch (Exception e)
- {
- this.failure = e.getMessage();
- e.printStackTrace();
- }
- latch.countDown();
- }
-
- c++;
- // Wrap around at numMessages
- if (c == numMessages)
- {
- c = 0;
- }
-
- counts.put(threadNum, c);
-
- }
- }
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PagingFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,571 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.paging.PagingManager;
-import org.hornetq.core.paging.PagingStore;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.RemotingConnectionImpl;
-import org.hornetq.core.remoting.impl.invm.InVMConnector;
-import org.hornetq.tests.util.RandomUtil;
-import org.hornetq.utils.SimpleString;
-
-/**
- * A PagingFailoverTest
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- * Created Feb 5, 2009 10:57:42 AM
- *
- *
- */
-public class PagingFailoverTest extends FailoverTestBase
-{
- // Constants -----------------------------------------------------
-
- private final Logger log = Logger.getLogger(PagingFailoverTest.class);
-
- final int RECEIVE_TIMEOUT = 2000;
-
- // Attributes ----------------------------------------------------
-
- // Static --------------------------------------------------------
-
- protected static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
-
- public void testMultithreadFailoverReplicationOnly() throws Throwable
- {
- setUpFileBased(getMaxGlobal(), getPageSize());
-
- int numberOfProducedMessages = multiThreadProducer(getNumberOfThreads(), false);
-
- System.out.println(numberOfProducedMessages + " messages produced");
-
- int numberOfConsumedMessages = multiThreadConsumer(getNumberOfThreads(), false, false);
-
- assertEquals(numberOfProducedMessages, numberOfConsumedMessages);
-
- System.out.println("Done!");
-
- }
-
- public void testMultithreadFailoverOnProducing() throws Throwable
- {
- setUpFileBased(getMaxGlobal(), getPageSize());
-
- int numberOfProducedMessages = multiThreadProducer(getNumberOfThreads(), true);
-
- System.out.println(numberOfProducedMessages + " messages produced");
-
- int numberOfConsumedMessages = multiThreadConsumer(getNumberOfThreads(), true, false);
-
- assertEquals(numberOfProducedMessages, numberOfConsumedMessages);
- }
-
- public void testMultithreadFailoverOnConsume() throws Throwable
- {
- setUpFileBased(getMaxGlobal(), getPageSize());
-
- int numberOfProducedMessages = multiThreadProducer(getNumberOfThreads(), false);
-
- System.out.println(numberOfProducedMessages + " messages produced");
-
- int numberOfConsumedMessages = multiThreadConsumer(getNumberOfThreads(), false, true);
-
- assertEquals(numberOfProducedMessages, numberOfConsumedMessages);
-
- }
-
-
- public void testFailoverOnPaging() throws Exception
- {
- testPaging(true);
- }
-
- public void testReplicationOnPaging() throws Exception
- {
- testPaging(false);
- }
-
- private void testPaging(final boolean fail) throws Exception
- {
- setUpFileBased(100 * 1024);
-
- ClientSession session = null;
- try
- {
- ClientSessionFactory sf1 = createFailoverFactory();
-
- sf1.setBlockOnAcknowledge(true);
- sf1.setBlockOnNonPersistentSend(true);
- sf1.setBlockOnPersistentSend(true);
-
- session = sf1.createSession(null, null, false, true, true, false, 0);
-
- session.createQueue(ADDRESS, ADDRESS, null, true);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = getNumberOfMessages();
-
- PagingManager pmLive = liveServer.getPostOffice().getPagingManager();
- PagingStore storeLive = pmLive.getPageStore(ADDRESS);
-
- PagingManager pmBackup = backupServer.getPostOffice().getPagingManager();
- PagingStore storeBackup = pmBackup.getPageStore(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(true);
- message.getBody().writeInt(i);
-
- producer.send(message);
-
- if (storeLive.isPaging())
- {
- assertTrue(storeBackup.isPaging());
- }
- }
-
- session.close();
- session = sf1.createSession(null, null, false, true, true, false, 0);
- session.start();
-
- final RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- assertEquals("GloblSize", pmLive.getTotalMemory(), pmBackup.getTotalMemory());
-
- assertEquals("PageSizeLive", storeLive.getAddressSize(), pmLive.getTotalMemory());
-
- assertEquals("PageSizeBackup", storeBackup.getAddressSize(), pmBackup.getTotalMemory());
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
-
- if (fail && i == numMessages / 2)
- {
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
- }
-
- ClientMessage message = consumer.receive(RECEIVE_TIMEOUT);
-
-
- assertNotNull(message);
-
- message.acknowledge();
-
- assertEquals(i, message.getBody().readInt());
-
- }
-
- session.close();
- session = null;
-
- if (!fail)
- {
- assertEquals(0, pmLive.getTotalMemory());
- assertEquals(0, storeLive.getAddressSize());
- }
- assertEquals(0, pmBackup.getTotalMemory());
- assertEquals(0, storeBackup.getAddressSize());
-
- }
- finally
- {
- if (session != null)
- {
- try
- {
- session.close();
- }
- catch (Exception ignored)
- {
- // eat it
- }
- }
- }
-
- }
-
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected int getNumberOfMessages()
- {
- return 500;
- }
-
- protected int getNumberOfThreads()
- {
- return 5;
- }
-
- protected int getMaxGlobal()
- {
- return 10024;
- }
-
- protected int getPageSize()
- {
- return 5120;
- }
-
- protected void fail(final ClientSession session) throws Exception
- {
- RemotingConnectionImpl conn = (RemotingConnectionImpl)((ClientSessionInternal)session).getConnection();
- System.out.println("Forcing a failure");
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED, "blah"));
-
- }
-
-
- // Private -------------------------------------------------------
-
- /**
- * @throws Exception
- * @throws InterruptedException
- * @throws Throwable
- */
- protected int multiThreadConsumer(int numberOfThreads, final boolean connectedOnBackup, final boolean fail) throws Exception,
- InterruptedException,
- Throwable
- {
- ClientSession session = null;
- try
- {
- final AtomicInteger numberOfMessages = new AtomicInteger(0);
-
- final ClientSessionFactory factory;
- final PagingStore store;
-
- if (connectedOnBackup)
- {
- factory = createBackupFactory();
- store = backupServer.getPostOffice().getPagingManager().getPageStore(ADDRESS);
- }
- else
- {
- factory = createFailoverFactory();
- store = liveServer.getPostOffice().getPagingManager().getPageStore(ADDRESS);
- }
-
- factory.setBlockOnNonPersistentSend(true);
- factory.setBlockOnAcknowledge(true);
- factory.setBlockOnPersistentSend(true);
-
- session = factory.createSession(false, true, true, false);
-
- final int initialNumberOfPages = store.getNumberOfPages();
-
- System.out.println("It has initially " + initialNumberOfPages);
-
- final CountDownLatch startFlag = new CountDownLatch(1);
- final CountDownLatch alignSemaphore = new CountDownLatch(numberOfThreads);
-
- class Consumer extends Thread
- {
- volatile Throwable e;
-
- ClientSession session;
-
- public Consumer() throws Exception
- {
- session = factory.createSession(null, null, false, true, true, false, 0);
- }
-
- @Override
- public void run()
- {
- boolean started = false;
-
- try
- {
-
- try
- {
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- alignSemaphore.countDown();
-
- started = true;
-
- startFlag.await();
-
- while (true)
- {
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
- if (msg == null)
- {
- break;
- }
-
- if (numberOfMessages.incrementAndGet() % 1000 == 0)
- {
- System.out.println(numberOfMessages + " messages read");
- }
-
- msg.acknowledge();
- }
-
- }
- finally
- {
- session.close();
- }
- }
- catch (Throwable e)
- {
- // Using System.out, as it would appear on the test output
- e.printStackTrace();
- if (!started)
- {
- alignSemaphore.countDown();
- }
- this.e = e;
- }
- }
- }
-
- Consumer[] consumers = new Consumer[numberOfThreads];
-
- for (int i = 0; i < numberOfThreads; i++)
- {
- consumers[i] = new Consumer();
- }
-
- for (int i = 0; i < numberOfThreads; i++)
- {
- consumers[i].start();
- }
-
- alignSemaphore.await();
-
- startFlag.countDown();
-
- if (fail)
- {
- // Fail after some time
- Thread.sleep((long)(1000 * RandomUtil.randomDouble()));
- while (store.getNumberOfPages() == initialNumberOfPages)
- {
- Thread.sleep(100);
- }
-
- System.out.println("The system has already depaged " + (initialNumberOfPages - store.getNumberOfPages()) +
- ", failing now");
-
- fail(session);
- }
-
- for (Thread t : consumers)
- {
- t.join();
- }
-
- for (Consumer p : consumers)
- {
- if (p.e != null)
- {
- throw p.e;
- }
- }
-
- return numberOfMessages.intValue();
- }
- finally
- {
- if (session != null)
- {
- try
- {
- session.close();
- }
- catch (Exception ignored)
- {
- }
- }
- }
- }
-
- /**
- * @throws Exception
- * @throws HornetQException
- * @throws InterruptedException
- * @throws Throwable
- */
- protected int multiThreadProducer(final int numberOfThreads, final boolean failover) throws Exception,
- HornetQException,
- InterruptedException,
- Throwable
- {
-
- final AtomicInteger numberOfMessages = new AtomicInteger(0);
- final PagingStore store = liveServer.getPostOffice().getPagingManager().getPageStore(ADDRESS);
-
- final ClientSessionFactory factory = createFailoverFactory();
-
- factory.setBlockOnNonPersistentSend(true);
- factory.setBlockOnAcknowledge(true);
- factory.setBlockOnPersistentSend(true);
-
- ClientSession session = factory.createSession(false, true, true, false);
- try
- {
- try
- {
- session.createQueue(ADDRESS, ADDRESS, null, true);
- }
- catch (Exception e)
- {
- }
-
- final CountDownLatch startFlag = new CountDownLatch(1);
- final CountDownLatch alignSemaphore = new CountDownLatch(numberOfThreads);
- final CountDownLatch flagPaging = new CountDownLatch(numberOfThreads);
-
- class Producer extends Thread
- {
- volatile Throwable e;
-
- @Override
- public void run()
- {
- boolean started = false;
- try
- {
- ClientSession session = factory.createSession(false, true, true);
- try
- {
- ClientProducer producer = session.createProducer(ADDRESS);
-
- alignSemaphore.countDown();
-
- started = true;
- startFlag.await();
-
- while (!store.isPaging())
- {
-
- ClientMessage msg = session.createClientMessage(true);
-
- producer.send(msg);
- numberOfMessages.incrementAndGet();
- }
-
- flagPaging.countDown();
-
- for (int i = 0; i < 100; i++)
- {
-
- ClientMessage msg = session.createClientMessage(true);
-
- producer.send(msg);
- numberOfMessages.incrementAndGet();
-
- }
-
- }
- finally
- {
- session.close();
- }
- }
- catch (Throwable e)
- {
- // Using System.out, as it would appear on the test output
- e.printStackTrace();
- if (!started)
- {
- alignSemaphore.countDown();
- }
- flagPaging.countDown();
- this.e = e;
- }
- }
- }
-
- Producer[] producers = new Producer[numberOfThreads];
-
- for (int i = 0; i < numberOfThreads; i++)
- {
- producers[i] = new Producer();
- producers[i].start();
- }
-
- alignSemaphore.await();
-
- // Start producing only when all the sessions are opened
- startFlag.countDown();
-
- if (failover)
- {
- flagPaging.await(); // for this test I want everybody on the paging part
-
- Thread.sleep(1500);
-
- fail(session);
-
- }
-
- for (Thread t : producers)
- {
- t.join();
- }
-
- for (Producer p : producers)
- {
- if (p.e != null)
- {
- throw p.e;
- }
- }
-
- return numberOfMessages.intValue();
-
- }
- finally
- {
- session.close();
- InVMConnector.resetFailures();
- }
-
- }
-
-
- // Inner classes -------------------------------------------------
-
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PreserveOrderDuringFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PreserveOrderDuringFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/PreserveOrderDuringFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,206 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.Interceptor;
-import org.hornetq.core.remoting.Packet;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.wireformat.SessionSendMessage;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A AutomaticFailoverWithDiscoveryTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- * @author <a href="mailto:clebert.suconic@jboss.com">Clebert Suconic</a>
- *
- * Created 8 Dec 2008 14:52:21
- *
- *
- */
-public class PreserveOrderDuringFailoverTest extends FailoverTestBase
-{
- private static final Logger log = Logger.getLogger(PreserveOrderDuringFailoverTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverOrderTestAddress");
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testOrdering() throws Exception
- {
- for (int i = 0; i < 20; i++)
- {
- log.info("testOrdering # " + i);
- setUpFailoverServers(false, -1, -1);
- failoverOrderTest();
- stopServers();
- }
- }
-
- protected void failoverOrderTest() throws Exception
- {
- ClientSessionFactory sf = createFailoverFactory();
-
- ClientSession session = sf.createSession(false, true, true);
-
- final RemotingConnection conn1 = ((ClientSessionInternal)session).getConnection();
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- Interceptor failInterceptor = new Interceptor()
- {
- int msg = 0;
-
- public boolean intercept(final Packet packet, final RemotingConnection conn) throws HornetQException
- {
- if (packet instanceof SessionSendMessage)
- {
- if (msg++ == 554)
- {
- // Simulate failure on connection
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
- return false;
- }
- }
- else
- {
- System.out.println("packet " + packet.getClass().getName());
- }
-
- return true;
- }
- };
-
- liveServer.getRemotingService().addInterceptor(failInterceptor);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- boolean outOfOrder = false;
-
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- if (i != (Integer)message2.getProperty(new SimpleString("count")))
- {
- System.out.println("Messages received out of order, " + i +
- " != " +
- message2.getProperty(new SimpleString("count")));
- outOfOrder = true;
- }
-
- message2.acknowledge();
- }
-
- session.close();
-
- session = sf.createSession(false, true, true);
-
- consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = numMessages / 2; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- if (i != (Integer)message2.getProperty(new SimpleString("count")))
- {
- System.out.println("Messages received out of order, " + i +
- " != " +
- message2.getProperty(new SimpleString("count")));
- outOfOrder = true;
- }
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = consumer.receive(250);
-
- if (message3 != null)
- {
- do
- {
- System.out.println("Message " + message3.getProperty(new SimpleString("count")) + " was duplicated");
- message3 = consumer.receive(1000);
- }
- while (message3 != null);
- fail("Duplicated messages received on test");
- }
-
- session.close();
-
- assertFalse("Messages received out of order, look at System.out for more details", outOfOrder);
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,1509 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.MessageHandler;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- * A RandomFailoverSoakTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- */
-public class RandomFailoverTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(RandomFailoverTest.class);
-
- // Constants -----------------------------------------------------
-
- private static final int RECEIVE_TIMEOUT = 10000;
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveService;
-
- private HornetQServer backupService;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- private Timer timer;
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testA() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestA(sf);
- }
- });
- }
-
- public void testB() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestB(sf);
- }
- });
- }
-
- public void testC() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestC(sf);
- }
- });
- }
-
- public void testD() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestD(sf);
- }
- });
- }
-
- public void testE() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestE(sf);
- }
- });
- }
-
- public void testF() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestF(sf);
- }
- });
- }
-
- public void testG() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestG(sf);
- }
- });
- }
-
- public void testH() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestH(sf);
- }
- });
- }
-
- public void testI() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestI(sf);
- }
- });
- }
-
- public void testJ() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestJ(sf);
- }
- });
- }
-
- public void testK() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestK(sf);
- }
- });
- }
-
- public void testL() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestL(sf);
- }
- });
- }
-
- public void testN() throws Exception
- {
- runTest(new RunnableT()
- {
- public void run(final ClientSessionFactory sf) throws Exception
- {
- doTestN(sf);
- }
- });
- }
-
- public void runTest(final RunnableT runnable) throws Exception
- {
- final int numIts = getNumIterations();
-
- for (int its = 0; its < numIts; its++)
- {
- start();
-
- ClientSessionFactoryImpl sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, false, false);
-
- Failer failer = startFailer(1000, session);
-
- do
- {
- runnable.run(sf);
- }
- while (!failer.isExecuted());
-
- session.close();
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
-
- stop();
- }
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected void doTestA(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, true, true);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- class MyHandler implements MessageHandler
- {
- final CountDownLatch latch = new CountDownLatch(1);
-
- volatile int count;
-
- public void onMessage(ClientMessage message)
- {
- if (count == numMessages)
- {
- fail("Too many messages");
- }
-
- assertEquals(count, message.getProperty(new SimpleString("count")));
-
- count++;
-
- try
- {
- message.acknowledge();
- }
- catch (HornetQException me)
- {
- log.error("Failed to process", me);
- }
-
- if (count == numMessages)
- {
- latch.countDown();
- }
- }
- }
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler();
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(5000, TimeUnit.MILLISECONDS);
-
- assertTrue("Didn't receive all messages", ok);
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestB(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 50;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, true, true);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- class MyHandler implements MessageHandler
- {
- final CountDownLatch latch = new CountDownLatch(1);
-
- volatile int count;
-
- public void onMessage(ClientMessage message)
- {
- if (count == numMessages)
- {
- fail("Too many messages");
- }
-
- assertEquals(count, message.getProperty(new SimpleString("count")));
-
- count++;
-
- if (count == numMessages)
- {
- latch.countDown();
- }
- }
- }
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler();
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
- }
-
- sessSend.close();
-
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
-
- }
-
- protected void doTestC(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, false, false);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.rollback();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.commit();
-
- class MyHandler implements MessageHandler
- {
- final CountDownLatch latch = new CountDownLatch(1);
-
- volatile int count;
-
- public void onMessage(ClientMessage message)
- {
- if (count == numMessages)
- {
- fail("Too many messages");
- }
-
- assertEquals(count, message.getProperty(new SimpleString("count")));
-
- count++;
-
- if (count == numMessages)
- {
- latch.countDown();
- }
- }
- }
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler();
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
- }
-
- handlers.clear();
-
- // New handlers
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler();
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
- }
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestD(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, false, false);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.rollback();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.commit();
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- class MyHandler implements MessageHandler
- {
- final CountDownLatch latch = new CountDownLatch(1);
-
- volatile int count;
-
- public void onMessage(ClientMessage message)
- {
- if (count == numMessages)
- {
- fail("Too many messages");
- }
-
- assertEquals(count, message.getProperty(new SimpleString("count")));
-
- count++;
-
- if (count == numMessages)
- {
- latch.countDown();
- }
- }
- }
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler();
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
- }
-
- handlers.clear();
-
- // New handlers
- for (ClientConsumer consumer : consumers)
- {
- MyHandler handler = new MyHandler();
-
- consumer.setMessageHandler(handler);
-
- handlers.add(handler);
- }
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
- }
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- // Now with synchronous receive()
-
- protected void doTestE(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, true, true);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- assertEquals(i, msg.getProperty(new SimpleString("count")));
-
- msg.acknowledge();
- }
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receiveImmediate();
-
- assertNull(msg);
- }
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestF(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, true, true);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, true, true);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- if (msg == null)
- {
- throw new IllegalStateException("Failed to receive message " + i);
- }
-
- assertNotNull(msg);
-
- assertEquals(i, msg.getProperty(new SimpleString("count")));
-
- msg.acknowledge();
- }
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receiveImmediate();
-
- assertNull(msg);
- }
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- assertEquals(1, ((ClientSessionFactoryImpl)sf).numSessions());
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestG(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, false, false);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, false, false);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.rollback();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.commit();
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- assertEquals(i, msg.getProperty(new SimpleString("count")));
-
- msg.acknowledge();
- }
- }
-
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receiveImmediate();
-
- assertNull(msg);
- }
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- assertEquals(i, msg.getProperty(new SimpleString("count")));
-
- msg.acknowledge();
- }
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receiveImmediate();
-
- assertNull(msg);
- }
- }
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestH(final ClientSessionFactory sf) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
- Set<ClientSession> sessions = new HashSet<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- ClientSession sessConsume = sf.createSession(false, false, false);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- consumers.add(consumer);
-
- sessions.add(sessConsume);
- }
-
- ClientSession sessSend = sf.createSession(false, false, false);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.rollback();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- producer.send(message);
- }
-
- sessSend.commit();
-
- for (ClientSession session : sessions)
- {
- session.start();
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- assertEquals(i, msg.getProperty(new SimpleString("count")));
-
- msg.acknowledge();
- }
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receiveImmediate();
-
- assertNull(msg);
- }
- }
-
- for (ClientSession session : sessions)
- {
- session.rollback();
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- assertEquals(i, msg.getProperty(new SimpleString("count")));
-
- msg.acknowledge();
- }
- }
-
- for (int i = 0; i < numMessages; i++)
- {
- for (ClientConsumer consumer : consumers)
- {
- ClientMessage msg = consumer.receiveImmediate();
-
- assertNull(msg);
- }
- }
-
- for (ClientSession session : sessions)
- {
- session.commit();
- }
-
- sessSend.close();
- for (ClientSession session : sessions)
- {
- session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString("sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestI(final ClientSessionFactory sf) throws Exception
- {
- ClientSession sessCreate = sf.createSession(false, true, true);
-
- sessCreate.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientSession sess = sf.createSession(false, true, true);
-
- sess.start();
-
- ClientConsumer consumer = sess.createConsumer(ADDRESS);
-
- ClientProducer producer = sess.createProducer(ADDRESS);
-
- ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- producer.send(message);
-
- ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(message2);
-
- message2.acknowledge();
-
- sess.close();
-
- sessCreate.deleteQueue(ADDRESS);
-
- sessCreate.close();
- }
-
- protected void doTestJ(final ClientSessionFactory sf) throws Exception
- {
- ClientSession sessCreate = sf.createSession(false, true, true);
-
- sessCreate.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientSession sess = sf.createSession(false, true, true);
-
- sess.start();
-
- ClientConsumer consumer = sess.createConsumer(ADDRESS);
-
- ClientProducer producer = sess.createProducer(ADDRESS);
-
- ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- producer.send(message);
-
- ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(message2);
-
- message2.acknowledge();
-
- sess.close();
-
- sessCreate.deleteQueue(ADDRESS);
-
- sessCreate.close();
- }
-
- protected void doTestK(final ClientSessionFactory sf) throws Exception
- {
- ClientSession s = sf.createSession(false, false, false);
-
- s.createQueue(ADDRESS, ADDRESS, null, false);
-
- final int numConsumers = 100;
-
- for (int i = 0; i < numConsumers; i++)
- {
- ClientConsumer consumer = s.createConsumer(ADDRESS);
-
- consumer.close();
- }
-
- s.deleteQueue(ADDRESS);
-
- s.close();
- }
-
- protected void doTestL(final ClientSessionFactory sf) throws Exception
- {
- final int numSessions = 10;
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession session = sf.createSession(false, false, false);
-
- session.close();
- }
- }
-
- protected void doTestN(final ClientSessionFactory sf) throws Exception
- {
- ClientSession sessCreate = sf.createSession(false, true, true);
-
- sessCreate.createQueue(ADDRESS, new SimpleString(ADDRESS.toString()), null, false);
-
- ClientSession sess = sf.createSession(false, true, true);
-
- sess.stop();
-
- sess.start();
-
- sess.stop();
-
- ClientConsumer consumer = sess.createConsumer(new SimpleString(ADDRESS.toString()));
-
- ClientProducer producer = sess.createProducer(ADDRESS);
-
- ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- producer.send(message);
-
- sess.start();
-
- ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(message2);
-
- message2.acknowledge();
-
- sess.stop();
-
- sess.start();
-
- sess.close();
-
- sessCreate.deleteQueue(new SimpleString(ADDRESS.toString()));
-
- sessCreate.close();
- }
-
- protected int getNumIterations()
- {
- return 2;
- }
-
- protected void setUp() throws Exception
- {
- super.setUp();
-
- timer = new Timer(true);
- }
-
- protected void tearDown() throws Exception
- {
- timer.cancel();
-
- InVMRegistry.instance.clear();
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- private Failer startFailer(final long time, final ClientSession session)
- {
- Failer failer = new Failer((ClientSessionInternal)session);
-
- timer.schedule(failer, (long)(time * Math.random()), 100);
-
- return failer;
- }
-
- private void start() throws Exception
- {
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- private void stop() throws Exception
- {
- backupService.stop();
-
- liveService.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupService = null;
-
- liveService = null;
- }
-
- // Inner classes -------------------------------------------------
-
- class Failer extends TimerTask
- {
- private final ClientSessionInternal session;
-
- private boolean executed;
-
- public Failer(final ClientSessionInternal session)
- {
- this.session = session;
- }
-
- public synchronized void run()
- {
- log.info("** Failing connection");
-
- session.getConnection().fail(new HornetQException(HornetQException.NOT_CONNECTED, "oops"));
-
- log.info("** Fail complete");
-
- cancel();
-
- executed = true;
- }
-
- public synchronized boolean isExecuted()
- {
- return executed;
- }
- }
-
- public abstract class RunnableT
- {
- abstract void run(final ClientSessionFactory sf) throws Exception;
- }
-}
Copied: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomReattachTest.java (from rev 7946, branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomFailoverTest.java)
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomReattachTest.java (rev 0)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/RandomReattachTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -0,0 +1,1486 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.tests.integration.cluster.failover;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.hornetq.core.client.ClientConsumer;
+import org.hornetq.core.client.ClientMessage;
+import org.hornetq.core.client.ClientProducer;
+import org.hornetq.core.client.ClientSession;
+import org.hornetq.core.client.ClientSessionFactory;
+import org.hornetq.core.client.MessageHandler;
+import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
+import org.hornetq.core.client.impl.ClientSessionInternal;
+import org.hornetq.core.config.Configuration;
+import org.hornetq.core.config.TransportConfiguration;
+import org.hornetq.core.config.impl.ConfigurationImpl;
+import org.hornetq.core.exception.HornetQException;
+import org.hornetq.core.logging.Logger;
+import org.hornetq.core.remoting.impl.invm.InVMRegistry;
+import org.hornetq.core.server.HornetQ;
+import org.hornetq.core.server.HornetQServer;
+import org.hornetq.jms.client.HornetQTextMessage;
+import org.hornetq.tests.util.UnitTestCase;
+import org.hornetq.utils.SimpleString;
+
+/**
+ * A RandomFailoverSoakTest
+ *
+ * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
+ */
+public class RandomReattachTest extends UnitTestCase
+{
+ private static final Logger log = Logger.getLogger(RandomReattachTest.class);
+
+ // Constants -----------------------------------------------------
+
+ private static final int RECEIVE_TIMEOUT = 10000;
+
+ // Attributes ----------------------------------------------------
+
+ private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
+
+ private HornetQServer liveService;
+
+ private Timer timer;
+
+ // Static --------------------------------------------------------
+
+ // Constructors --------------------------------------------------
+
+ // Public --------------------------------------------------------
+
+ public void testA() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestA(sf);
+ }
+ });
+ }
+
+ public void testB() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestB(sf);
+ }
+ });
+ }
+
+ public void testC() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestC(sf);
+ }
+ });
+ }
+
+ public void testD() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestD(sf);
+ }
+ });
+ }
+
+ public void testE() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestE(sf);
+ }
+ });
+ }
+
+ public void testF() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestF(sf);
+ }
+ });
+ }
+
+ public void testG() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestG(sf);
+ }
+ });
+ }
+
+ public void testH() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestH(sf);
+ }
+ });
+ }
+
+ public void testI() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestI(sf);
+ }
+ });
+ }
+
+ public void testJ() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestJ(sf);
+ }
+ });
+ }
+
+ public void testK() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestK(sf);
+ }
+ });
+ }
+
+ public void testL() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestL(sf);
+ }
+ });
+ }
+
+ public void testN() throws Exception
+ {
+ runTest(new RunnableT()
+ {
+ public void run(final ClientSessionFactory sf) throws Exception
+ {
+ doTestN(sf);
+ }
+ });
+ }
+
+ public void runTest(final RunnableT runnable) throws Exception
+ {
+ final int numIts = getNumIterations();
+
+ for (int its = 0; its < numIts; its++)
+ {
+ start();
+
+ ClientSessionFactoryImpl sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setProducerWindowSize(32 * 1024);
+
+ ClientSession session = sf.createSession(false, false, false);
+
+ Failer failer = startFailer(1000, session);
+
+ do
+ {
+ runnable.run(sf);
+ }
+ while (!failer.isExecuted());
+
+ session.close();
+
+ assertEquals(0, sf.numSessions());
+
+ assertEquals(0, sf.numConnections());
+
+ stop();
+ }
+ }
+
+ // Package protected ---------------------------------------------
+
+ // Protected -----------------------------------------------------
+
+ protected void doTestA(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, true, true);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ class MyHandler implements MessageHandler
+ {
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ volatile int count;
+
+ public void onMessage(ClientMessage message)
+ {
+ if (count == numMessages)
+ {
+ fail("Too many messages");
+ }
+
+ assertEquals(count, message.getProperty(new SimpleString("count")));
+
+ count++;
+
+ try
+ {
+ message.acknowledge();
+ }
+ catch (HornetQException me)
+ {
+ log.error("Failed to process", me);
+ }
+
+ if (count == numMessages)
+ {
+ latch.countDown();
+ }
+ }
+ }
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler();
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(5000, TimeUnit.MILLISECONDS);
+
+ assertTrue("Didn't receive all messages", ok);
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestB(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 50;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, true, true);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ class MyHandler implements MessageHandler
+ {
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ volatile int count;
+
+ public void onMessage(ClientMessage message)
+ {
+ if (count == numMessages)
+ {
+ fail("Too many messages");
+ }
+
+ assertEquals(count, message.getProperty(new SimpleString("count")));
+
+ count++;
+
+ if (count == numMessages)
+ {
+ latch.countDown();
+ }
+ }
+ }
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler();
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+ }
+
+ sessSend.close();
+
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+
+ }
+
+ protected void doTestC(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, false, false);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.rollback();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.commit();
+
+ class MyHandler implements MessageHandler
+ {
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ volatile int count;
+
+ public void onMessage(ClientMessage message)
+ {
+ if (count == numMessages)
+ {
+ fail("Too many messages");
+ }
+
+ assertEquals(count, message.getProperty(new SimpleString("count")));
+
+ count++;
+
+ if (count == numMessages)
+ {
+ latch.countDown();
+ }
+ }
+ }
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler();
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+ }
+
+ handlers.clear();
+
+ // New handlers
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler();
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestD(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, false, false);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.rollback();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.commit();
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ class MyHandler implements MessageHandler
+ {
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ volatile int count;
+
+ public void onMessage(ClientMessage message)
+ {
+ if (count == numMessages)
+ {
+ fail("Too many messages");
+ }
+
+ assertEquals(count, message.getProperty(new SimpleString("count")));
+
+ count++;
+
+ if (count == numMessages)
+ {
+ latch.countDown();
+ }
+ }
+ }
+
+ Set<MyHandler> handlers = new HashSet<MyHandler>();
+
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler();
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+ }
+
+ handlers.clear();
+
+ // New handlers
+ for (ClientConsumer consumer : consumers)
+ {
+ MyHandler handler = new MyHandler();
+
+ consumer.setMessageHandler(handler);
+
+ handlers.add(handler);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ for (MyHandler handler : handlers)
+ {
+ boolean ok = handler.latch.await(10000, TimeUnit.MILLISECONDS);
+
+ assertTrue(ok);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ // Now with synchronous receive()
+
+ protected void doTestE(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, true, true);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(msg);
+
+ assertEquals(i, msg.getProperty(new SimpleString("count")));
+
+ msg.acknowledge();
+ }
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receiveImmediate();
+
+ assertNull(msg);
+ }
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestF(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, true, true);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, true, true);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
+
+ if (msg == null)
+ {
+ throw new IllegalStateException("Failed to receive message " + i);
+ }
+
+ assertNotNull(msg);
+
+ assertEquals(i, msg.getProperty(new SimpleString("count")));
+
+ msg.acknowledge();
+ }
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receiveImmediate();
+
+ assertNull(msg);
+ }
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ assertEquals(1, ((ClientSessionFactoryImpl)sf).numSessions());
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestG(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, false, false);
+
+ sessConsume.start();
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, false, false);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.rollback();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.commit();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(msg);
+
+ assertEquals(i, msg.getProperty(new SimpleString("count")));
+
+ msg.acknowledge();
+ }
+ }
+
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receiveImmediate();
+
+ assertNull(msg);
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(msg);
+
+ assertEquals(i, msg.getProperty(new SimpleString("count")));
+
+ msg.acknowledge();
+ }
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receiveImmediate();
+
+ assertNull(msg);
+ }
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestH(final ClientSessionFactory sf) throws Exception
+ {
+ long start = System.currentTimeMillis();
+
+ ClientSession s = sf.createSession(false, false, false);
+
+ final int numMessages = 100;
+
+ final int numSessions = 10;
+
+ Set<ClientConsumer> consumers = new HashSet<ClientConsumer>();
+ Set<ClientSession> sessions = new HashSet<ClientSession>();
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ ClientSession sessConsume = sf.createSession(false, false, false);
+
+ sessConsume.createQueue(ADDRESS, subName, null, false);
+
+ ClientConsumer consumer = sessConsume.createConsumer(subName);
+
+ consumers.add(consumer);
+
+ sessions.add(sessConsume);
+ }
+
+ ClientSession sessSend = sf.createSession(false, false, false);
+
+ ClientProducer producer = sessSend.createProducer(ADDRESS);
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.rollback();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = sessSend.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ producer.send(message);
+ }
+
+ sessSend.commit();
+
+ for (ClientSession session : sessions)
+ {
+ session.start();
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(msg);
+
+ assertEquals(i, msg.getProperty(new SimpleString("count")));
+
+ msg.acknowledge();
+ }
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receiveImmediate();
+
+ assertNull(msg);
+ }
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.rollback();
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(msg);
+
+ assertEquals(i, msg.getProperty(new SimpleString("count")));
+
+ msg.acknowledge();
+ }
+ }
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ for (ClientConsumer consumer : consumers)
+ {
+ ClientMessage msg = consumer.receiveImmediate();
+
+ assertNull(msg);
+ }
+ }
+
+ for (ClientSession session : sessions)
+ {
+ session.commit();
+ }
+
+ sessSend.close();
+ for (ClientSession session : sessions)
+ {
+ session.close();
+ }
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ SimpleString subName = new SimpleString("sub" + i);
+
+ s.deleteQueue(subName);
+ }
+
+ s.close();
+
+ long end = System.currentTimeMillis();
+
+ log.info("duration " + (end - start));
+ }
+
+ protected void doTestI(final ClientSessionFactory sf) throws Exception
+ {
+ ClientSession sessCreate = sf.createSession(false, true, true);
+
+ sessCreate.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientSession sess = sf.createSession(false, true, true);
+
+ sess.start();
+
+ ClientConsumer consumer = sess.createConsumer(ADDRESS);
+
+ ClientProducer producer = sess.createProducer(ADDRESS);
+
+ ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ producer.send(message);
+
+ ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(message2);
+
+ message2.acknowledge();
+
+ sess.close();
+
+ sessCreate.deleteQueue(ADDRESS);
+
+ sessCreate.close();
+ }
+
+ protected void doTestJ(final ClientSessionFactory sf) throws Exception
+ {
+ ClientSession sessCreate = sf.createSession(false, true, true);
+
+ sessCreate.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientSession sess = sf.createSession(false, true, true);
+
+ sess.start();
+
+ ClientConsumer consumer = sess.createConsumer(ADDRESS);
+
+ ClientProducer producer = sess.createProducer(ADDRESS);
+
+ ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ producer.send(message);
+
+ ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(message2);
+
+ message2.acknowledge();
+
+ sess.close();
+
+ sessCreate.deleteQueue(ADDRESS);
+
+ sessCreate.close();
+ }
+
+ protected void doTestK(final ClientSessionFactory sf) throws Exception
+ {
+ ClientSession s = sf.createSession(false, false, false);
+
+ s.createQueue(ADDRESS, ADDRESS, null, false);
+
+ final int numConsumers = 100;
+
+ for (int i = 0; i < numConsumers; i++)
+ {
+ ClientConsumer consumer = s.createConsumer(ADDRESS);
+
+ consumer.close();
+ }
+
+ s.deleteQueue(ADDRESS);
+
+ s.close();
+ }
+
+ protected void doTestL(final ClientSessionFactory sf) throws Exception
+ {
+ final int numSessions = 10;
+
+ for (int i = 0; i < numSessions; i++)
+ {
+ ClientSession session = sf.createSession(false, false, false);
+
+ session.close();
+ }
+ }
+
+ protected void doTestN(final ClientSessionFactory sf) throws Exception
+ {
+ ClientSession sessCreate = sf.createSession(false, true, true);
+
+ sessCreate.createQueue(ADDRESS, new SimpleString(ADDRESS.toString()), null, false);
+
+ ClientSession sess = sf.createSession(false, true, true);
+
+ sess.stop();
+
+ sess.start();
+
+ sess.stop();
+
+ ClientConsumer consumer = sess.createConsumer(new SimpleString(ADDRESS.toString()));
+
+ ClientProducer producer = sess.createProducer(ADDRESS);
+
+ ClientMessage message = sess.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ producer.send(message);
+
+ sess.start();
+
+ ClientMessage message2 = consumer.receive(RECEIVE_TIMEOUT);
+
+ assertNotNull(message2);
+
+ message2.acknowledge();
+
+ sess.stop();
+
+ sess.start();
+
+ sess.close();
+
+ sessCreate.deleteQueue(new SimpleString(ADDRESS.toString()));
+
+ sessCreate.close();
+ }
+
+ protected int getNumIterations()
+ {
+ return 2;
+ }
+
+ protected void setUp() throws Exception
+ {
+ super.setUp();
+
+ timer = new Timer(true);
+ }
+
+ protected void tearDown() throws Exception
+ {
+ timer.cancel();
+
+ InVMRegistry.instance.clear();
+
+ super.tearDown();
+ }
+
+ // Private -------------------------------------------------------
+
+ private Failer startFailer(final long time, final ClientSession session)
+ {
+ Failer failer = new Failer((ClientSessionInternal)session);
+
+ timer.schedule(failer, (long)(time * Math.random()), 100);
+
+ return failer;
+ }
+
+ private void start() throws Exception
+ {
+ Configuration liveConf = new ConfigurationImpl();
+ liveConf.setSecurityEnabled(false);
+ liveConf.getAcceptorConfigurations()
+ .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
+ Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
+ TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory");
+ connectors.put(backupTC.getName(), backupTC);
+ liveConf.setConnectorConfigurations(connectors);
+ liveConf.setBackupConnectorName(backupTC.getName());
+ liveService = HornetQ.newHornetQServer(liveConf, false);
+ liveService.start();
+ }
+
+ private void stop() throws Exception
+ {
+ liveService.stop();
+
+ assertEquals(0, InVMRegistry.instance.size());
+
+ liveService = null;
+ }
+
+ // Inner classes -------------------------------------------------
+
+ class Failer extends TimerTask
+ {
+ private final ClientSessionInternal session;
+
+ private boolean executed;
+
+ public Failer(final ClientSessionInternal session)
+ {
+ this.session = session;
+ }
+
+ public synchronized void run()
+ {
+ log.info("** Failing connection");
+
+ session.getConnection().fail(new HornetQException(HornetQException.NOT_CONNECTED, "oops"));
+
+ log.info("** Fail complete");
+
+ cancel();
+
+ executed = true;
+ }
+
+ public synchronized boolean isExecuted()
+ {
+ return executed;
+ }
+ }
+
+ public abstract class RunnableT
+ {
+ abstract void run(final ClientSessionFactory sf) throws Exception;
+ }
+}
Copied: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReattachTest.java (from rev 7946, branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectTest.java)
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReattachTest.java (rev 0)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReattachTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -0,0 +1,715 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.tests.integration.cluster.failover;
+
+import org.hornetq.core.client.ClientConsumer;
+import org.hornetq.core.client.ClientMessage;
+import org.hornetq.core.client.ClientProducer;
+import org.hornetq.core.client.ClientSession;
+import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
+import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
+import org.hornetq.core.client.impl.ClientSessionInternal;
+import org.hornetq.core.config.Configuration;
+import org.hornetq.core.config.TransportConfiguration;
+import org.hornetq.core.config.impl.ConfigurationImpl;
+import org.hornetq.core.exception.HornetQException;
+import org.hornetq.core.logging.Logger;
+import org.hornetq.core.remoting.FailureListener;
+import org.hornetq.core.remoting.RemotingConnection;
+import org.hornetq.core.remoting.impl.invm.InVMConnector;
+import org.hornetq.core.remoting.impl.invm.InVMRegistry;
+import org.hornetq.core.server.HornetQ;
+import org.hornetq.core.server.HornetQServer;
+import org.hornetq.jms.client.HornetQTextMessage;
+import org.hornetq.tests.util.UnitTestCase;
+import org.hornetq.utils.SimpleString;
+
+/**
+ *
+ * A ReattachTest
+ *
+ * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
+ *
+ * Created 4 Nov 2008 16:54:50
+ *
+ *
+ */
+public class ReattachTest extends UnitTestCase
+{
+ private static final Logger log = Logger.getLogger(ReattachTest.class);
+
+ // Constants -----------------------------------------------------
+
+ // Attributes ----------------------------------------------------
+
+ private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
+
+ private HornetQServer service;
+
+ // Static --------------------------------------------------------
+
+ // Constructors --------------------------------------------------
+
+ // Public --------------------------------------------------------
+
+ /*
+ * Test failure on connection, but server is still up so should immediately reconnect
+ */
+ public void testImmediateReattach() throws Exception
+ {
+ final long retryInterval = 500;
+
+ final double retryMultiplier = 1d;
+
+ final int reconnectAttempts = 1;
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setRetryInterval(retryInterval);
+ sf.setRetryIntervalMultiplier(retryMultiplier);
+ sf.setReconnectAttempts(reconnectAttempts);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ final int numIterations = 100;
+
+ for (int j = 0; j < numIterations; j++)
+ {
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(500);
+
+ assertNotNull(message);
+
+ assertEquals("aardvarks", message.getBody().readString());
+
+ assertEquals(i, message.getProperty(new SimpleString("count")));
+
+ message.acknowledge();
+ }
+
+ ClientMessage message = consumer.receiveImmediate();
+
+ assertNull(message);
+
+ producer.close();
+
+ consumer.close();
+ }
+
+ session.close();
+
+ sf.close();
+ }
+
+ /*
+ * Test failure on connection, simulate failure to create connection for a while, then
+ * allow connection to be recreated
+ */
+ public void testDelayedReattach() throws Exception
+ {
+ final long retryInterval = 500;
+
+ final double retryMultiplier = 1d;
+
+ final int reconnectAttempts = -1;
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setRetryInterval(retryInterval);
+ sf.setRetryIntervalMultiplier(retryMultiplier);
+ sf.setReconnectAttempts(reconnectAttempts);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ InVMConnector.failOnCreateConnection = true;
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ Thread t = new Thread()
+ {
+ public void run()
+ {
+ try
+ {
+ Thread.sleep(retryInterval * 3);
+ }
+ catch (InterruptedException ignore)
+ {
+ }
+
+ InVMConnector.failOnCreateConnection = false;
+ }
+ };
+
+ t.start();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(500);
+
+ assertNotNull(message);
+
+ assertEquals("aardvarks", message.getBody().readString());
+
+ assertEquals(i, message.getProperty(new SimpleString("count")));
+
+ message.acknowledge();
+ }
+
+ ClientMessage message = consumer.receiveImmediate();
+
+ assertNull(message);
+
+ session.close();
+
+ sf.close();
+
+ t.join();
+ }
+
+ // Test an async (e.g. pinger) failure coming in while a connection manager is already reconnecting
+ public void testAsyncFailureWhileReattaching() throws Exception
+ {
+ final long retryInterval = 500;
+
+ final double retryMultiplier = 1d;
+
+ final int reconnectAttempts = -1;
+
+ final long asyncFailDelay = 2000;
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setRetryInterval(retryInterval);
+ sf.setRetryIntervalMultiplier(retryMultiplier);
+ sf.setReconnectAttempts(reconnectAttempts);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ ClientSession session2 = sf.createSession(false, true, true);
+
+ class MyFailureListener implements FailureListener
+ {
+ volatile boolean failed;
+
+ public void connectionFailed(HornetQException me)
+ {
+ failed = true;
+ }
+ }
+
+ MyFailureListener listener = new MyFailureListener();
+
+ session2.addFailureListener(listener);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ InVMConnector.numberOfFailures = 10;
+ InVMConnector.failOnCreateConnection = true;
+
+ //We need to fail on different connections.
+
+ //We fail on one connection then the connection manager tries to reconnect all connections
+ //Then we fail the other, and the connection manager is then called while the reconnection is occurring
+ //We can't use the same connection since RemotingConnectionImpl only allows one fail to be in process
+ //at same time
+
+ final RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ final RemotingConnection conn2 = ((ClientSessionInternal)session2).getConnection();
+
+ assertTrue(conn != conn2);
+
+ Thread t = new Thread()
+ {
+ public void run()
+ {
+ try
+ {
+ Thread.sleep(asyncFailDelay);
+ }
+ catch (InterruptedException ignore)
+ {
+ }
+
+ log.info("calling fail async");
+
+ conn2.fail(new HornetQException(HornetQException.NOT_CONNECTED, "Did not receive pong from server"));
+ }
+ };
+
+ t.start();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ assertFalse(listener.failed);
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(500);
+
+ assertNotNull(message);
+
+ assertEquals("aardvarks", message.getBody().readString());
+
+ assertEquals(i, message.getProperty(new SimpleString("count")));
+
+ message.acknowledge();
+ }
+
+ ClientMessage message = consumer.receiveImmediate();
+
+ assertNull(message);
+
+ session.close();
+
+ session2.close();
+
+ sf.close();
+
+ t.join();
+ }
+
+ public void testReattachAttemptsFailsToReconnect() throws Exception
+ {
+ final long retryInterval = 500;
+
+ final double retryMultiplier = 1d;
+
+ final int reconnectAttempts = 3;
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setRetryInterval(retryInterval);
+ sf.setRetryIntervalMultiplier(retryMultiplier);
+ sf.setReconnectAttempts(reconnectAttempts);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ InVMConnector.failOnCreateConnection = true;
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ // Sleep for longer than max retries so should fail to reconnect
+
+ Thread t = new Thread()
+ {
+ public void run()
+ {
+ try
+ {
+ Thread.sleep(retryInterval * (reconnectAttempts + 1));
+ }
+ catch (InterruptedException ignore)
+ {
+ }
+
+ InVMConnector.failOnCreateConnection = false;
+ }
+ };
+
+ t.start();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ session.start();
+
+ // Should be null since failed to reconnect
+ ClientMessage message = consumer.receive(500);
+
+ assertNull(message);
+
+ session.close();
+
+ sf.close();
+
+ t.join();
+ }
+
+ public void testReattachAttemptsSucceedsInReconnecting() throws Exception
+ {
+ final long retryInterval = 500;
+
+ final double retryMultiplier = 1d;
+
+ final int reconnectAttempts = 10;
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setRetryInterval(retryInterval);
+ sf.setRetryIntervalMultiplier(retryMultiplier);
+ sf.setReconnectAttempts(reconnectAttempts);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ InVMConnector.failOnCreateConnection = true;
+ InVMConnector.numberOfFailures = reconnectAttempts - 1;
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(500);
+
+ assertNotNull(message);
+
+ assertEquals("aardvarks", message.getBody().readString());
+
+ assertEquals(i, message.getProperty(new SimpleString("count")));
+
+ message.acknowledge();
+ }
+
+ ClientMessage message = consumer.receiveImmediate();
+
+ assertNull(message);
+
+ session.close();
+
+ sf.close();
+ }
+
+ public void testRetryInterval() throws Exception
+ {
+ final long retryInterval = 500;
+
+ final double retryMultiplier = 1d;
+
+ final int reconnectAttempts = -1;
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setRetryInterval(retryInterval);
+ sf.setRetryIntervalMultiplier(retryMultiplier);
+ sf.setReconnectAttempts(reconnectAttempts);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ InVMConnector.failOnCreateConnection = true;
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ long start = System.currentTimeMillis();
+
+ Thread t = new Thread()
+ {
+ public void run()
+ {
+ try
+ {
+ Thread.sleep(retryInterval / 2);
+ }
+ catch (InterruptedException ignore)
+ {
+ }
+ InVMConnector.failOnCreateConnection = false;
+ }
+ };
+
+ t.start();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(500);
+
+ assertNotNull(message);
+
+ assertEquals("aardvarks", message.getBody().readString());
+
+ assertEquals(i, message.getProperty(new SimpleString("count")));
+
+ message.acknowledge();
+ }
+
+ ClientMessage message = consumer.receiveImmediate();
+
+ assertNull(message);
+
+ long end = System.currentTimeMillis();
+
+ assertTrue((end - start) >= retryInterval);
+
+ session.close();
+
+ sf.close();
+
+ t.join();
+ }
+
+ public void testExponentialBackoff() throws Exception
+ {
+ final long retryInterval = 500;
+
+ final double retryMultiplier = 4d;
+
+ final int reconnectAttempts = -1;
+
+ ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
+
+ sf.setRetryInterval(retryInterval);
+ sf.setRetryIntervalMultiplier(retryMultiplier);
+ sf.setReconnectAttempts(reconnectAttempts);
+
+ ClientSession session = sf.createSession(false, true, true);
+
+ session.createQueue(ADDRESS, ADDRESS, null, false);
+
+ ClientProducer producer = session.createProducer(ADDRESS);
+
+ final int numMessages = 1000;
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
+ false,
+ 0,
+ System.currentTimeMillis(),
+ (byte)1);
+ message.putIntProperty(new SimpleString("count"), i);
+ message.getBody().writeString("aardvarks");
+ producer.send(message);
+ }
+
+ ClientConsumer consumer = session.createConsumer(ADDRESS);
+
+ InVMConnector.failOnCreateConnection = true;
+
+ RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
+
+ long start = System.currentTimeMillis();
+
+ Thread t = new Thread()
+ {
+ public void run()
+ {
+ try
+ {
+ Thread.sleep(retryInterval * 2);
+ }
+ catch (InterruptedException ignore)
+ {
+ }
+
+ InVMConnector.failOnCreateConnection = false;
+ }
+ };
+
+ t.start();
+
+ conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
+
+ session.start();
+
+ for (int i = 0; i < numMessages; i++)
+ {
+ ClientMessage message = consumer.receive(500);
+
+ assertNotNull(message);
+
+ assertEquals("aardvarks", message.getBody().readString());
+
+ assertEquals(i, message.getProperty(new SimpleString("count")));
+
+ message.acknowledge();
+ }
+
+ ClientMessage message = consumer.receiveImmediate();
+
+ assertNull(message);
+
+ long end = System.currentTimeMillis();
+
+ assertTrue((end - start) >= retryInterval * (1 + retryMultiplier));
+
+ session.close();
+
+ sf.close();
+
+ t.join();
+ }
+
+ // Package protected ---------------------------------------------
+
+ // Protected -----------------------------------------------------
+
+ @Override
+ protected void setUp() throws Exception
+ {
+ super.setUp();
+
+ Configuration liveConf = new ConfigurationImpl();
+ liveConf.setSecurityEnabled(false);
+ liveConf.getAcceptorConfigurations()
+ .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
+ service = HornetQ.newHornetQServer(liveConf, false);
+ service.start();
+ }
+
+ @Override
+ protected void tearDown() throws Exception
+ {
+ InVMConnector.resetFailures();
+
+ service.stop();
+
+ assertEquals(0, InVMRegistry.instance.size());
+
+ service = null;
+
+ super.tearDown();
+ }
+
+ // Private -------------------------------------------------------
+
+ // Inner classes -------------------------------------------------
+}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,715 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.FailureListener;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMConnector;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A ReconnectTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 4 Nov 2008 16:54:50
- *
- *
- */
-public class ReconnectTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(ReconnectTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer service;
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- /*
- * Test failure on connection, but server is still up so should immediately reconnect
- */
- public void testImmediateReconnect() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = 1;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- final int numIterations = 100;
-
- for (int j = 0; j < numIterations; j++)
- {
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- producer.close();
-
- consumer.close();
- }
-
- session.close();
-
- sf.close();
- }
-
- /*
- * Test failure on connection, simulate failure to create connection for a while, then
- * allow connection to be recreated
- */
- public void testDelayedReconnect() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = -1;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- InVMConnector.failOnCreateConnection = true;
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- Thread.sleep(retryInterval * 3);
- }
- catch (InterruptedException ignore)
- {
- }
-
- InVMConnector.failOnCreateConnection = false;
- }
- };
-
- t.start();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- session.close();
-
- sf.close();
-
- t.join();
- }
-
- // Test an async (e.g. pinger) failure coming in while a connection manager is already reconnecting
- public void testAsyncFailureWhileReconnecting() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = -1;
-
- final long asyncFailDelay = 2000;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- ClientSession session2 = sf.createSession(false, true, true);
-
- class MyFailureListener implements FailureListener
- {
- volatile boolean failed;
-
- public void connectionFailed(HornetQException me)
- {
- failed = true;
- }
- }
-
- MyFailureListener listener = new MyFailureListener();
-
- session2.addFailureListener(listener);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- InVMConnector.numberOfFailures = 10;
- InVMConnector.failOnCreateConnection = true;
-
- //We need to fail on different connections.
-
- //We fail on one connection then the connection manager tries to reconnect all connections
- //Then we fail the other, and the connection manager is then called while the reconnection is occurring
- //We can't use the same connection since RemotingConnectionImpl only allows one fail to be in process
- //at same time
-
- final RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- final RemotingConnection conn2 = ((ClientSessionInternal)session2).getConnection();
-
- assertTrue(conn != conn2);
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- Thread.sleep(asyncFailDelay);
- }
- catch (InterruptedException ignore)
- {
- }
-
- log.info("calling fail async");
-
- conn2.fail(new HornetQException(HornetQException.NOT_CONNECTED, "Did not receive pong from server"));
- }
- };
-
- t.start();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- assertFalse(listener.failed);
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- session.close();
-
- session2.close();
-
- sf.close();
-
- t.join();
- }
-
- public void testReconnectAttemptsFailsToReconnect() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = 3;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- InVMConnector.failOnCreateConnection = true;
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- // Sleep for longer than max retries so should fail to reconnect
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- Thread.sleep(retryInterval * (reconnectAttempts + 1));
- }
- catch (InterruptedException ignore)
- {
- }
-
- InVMConnector.failOnCreateConnection = false;
- }
- };
-
- t.start();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- // Should be null since failed to reconnect
- ClientMessage message = consumer.receive(500);
-
- assertNull(message);
-
- session.close();
-
- sf.close();
-
- t.join();
- }
-
- public void testReconnectAttemptsSucceedsInReconnecting() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = 10;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- InVMConnector.failOnCreateConnection = true;
- InVMConnector.numberOfFailures = reconnectAttempts - 1;
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- session.close();
-
- sf.close();
- }
-
- public void testRetryInterval() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = -1;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- InVMConnector.failOnCreateConnection = true;
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- long start = System.currentTimeMillis();
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- Thread.sleep(retryInterval / 2);
- }
- catch (InterruptedException ignore)
- {
- }
- InVMConnector.failOnCreateConnection = false;
- }
- };
-
- t.start();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- long end = System.currentTimeMillis();
-
- assertTrue((end - start) >= retryInterval);
-
- session.close();
-
- sf.close();
-
- t.join();
- }
-
- public void testExponentialBackoff() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 4d;
-
- final int reconnectAttempts = -1;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- InVMConnector.failOnCreateConnection = true;
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- long start = System.currentTimeMillis();
-
- Thread t = new Thread()
- {
- public void run()
- {
- try
- {
- Thread.sleep(retryInterval * 2);
- }
- catch (InterruptedException ignore)
- {
- }
-
- InVMConnector.failOnCreateConnection = false;
- }
- };
-
- t.start();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- long end = System.currentTimeMillis();
-
- assertTrue((end - start) >= retryInterval * (1 + retryMultiplier));
-
- session.close();
-
- sf.close();
-
- t.join();
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- service = HornetQ.newHornetQServer(liveConf, false);
- service.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- InVMConnector.resetFailures();
-
- service.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- service = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectWithBackupTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectWithBackupTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReconnectWithBackupTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,314 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A ReconnectWithBackupTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 4 Nov 2008 16:54:50
- *
- *
- */
-public class ReconnectWithBackupTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(ReconnectWithBackupTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveService;
-
- private HornetQServer backupService;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- /*
- * Test retrying reconnect on current node before failover
- */
- public void testRetryBeforeFailoverSuccess() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = -1;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setFailoverOnServerShutdown(true);
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- final int numIterations = 10;
-
- // We reconnect in a loop a few times
- for (int j = 0; j < numIterations; j++)
- {
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- producer.close();
-
- consumer.close();
- }
-
- session.close();
-
- sf.close();
- }
-
- public void testFailoverThenFailAgainThenSuccessAfterRetry() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = -1;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setFailoverOnServerShutdown(true);
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 100;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- ClientMessage message = consumer.receiveImmediate();
-
- assertNull(message);
-
- session.stop();
-
- final int numIterations = 10;
-
- for (int j = 0; j < numIterations; j++)
- {
- // Send some more messages
-
- for (int i = numMessages; i < numMessages * 2; i++)
- {
- message = session.createClientMessage(HornetQTextMessage.TYPE, false, 0, System.currentTimeMillis(), (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- // Now fail again - should reconnect to the backup node
-
- conn = ((ClientSessionInternal)session).getConnection();
-
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- session.start();
-
- for (int i = numMessages; i < numMessages * 2; i++)
- {
- message = consumer.receive(500);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- assertEquals(i, message.getProperty(new SimpleString("count")));
-
- message.acknowledge();
- }
-
- message = consumer.receiveImmediate();
-
- assertNull(message);
- }
-
- session.close();
-
- sf.close();
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- backupService.stop();
-
- liveService.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupService = null;
-
- liveService = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReplicateConnectionFailureTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReplicateConnectionFailureTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/ReplicateConnectionFailureTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,179 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.client.impl.ConnectionManagerImpl;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.impl.RemotingConnectionImpl;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.tests.util.UnitTestCase;
-
-/**
- *
- * A ReplicateConnectionFailureTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 6 Nov 2008 08:42:36
- *
- * Test whether when a connection is failed on the server since server receives no ping, that close
- * is replicated to backup.
- *
- */
-public class ReplicateConnectionFailureTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(ReplicateConnectionFailureTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private HornetQServer liveServer;
-
- private HornetQServer backupServer;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testFailConnection() throws Exception
- {
- final long clientFailureCheckPeriod = 500;
-
- ClientSessionFactoryImpl sf1 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf1.setClientFailureCheckPeriod(clientFailureCheckPeriod);
- sf1.setConnectionTTL((long)(clientFailureCheckPeriod * 1.5));
- sf1.setProducerWindowSize(32 * 1024);
-
- assertEquals(0, liveServer.getRemotingService().getConnections().size());
-
- assertEquals(1, backupServer.getRemotingService().getConnections().size());
-
- ClientSession session1 = sf1.createSession(false, true, true);
-
- // One connection
- assertEquals(1, liveServer.getRemotingService().getConnections().size());
-
- // One replicating connection
- assertEquals(1, backupServer.getRemotingService().getConnections().size());
-
- session1.close();
-
- Thread.sleep(2000);
-
- assertEquals(0, liveServer.getRemotingService().getConnections().size());
-
- assertEquals(1, backupServer.getRemotingService().getConnections().size());
-
- session1 = sf1.createSession(false, true, true);
-
- final RemotingConnectionImpl conn1 = (RemotingConnectionImpl)((ClientSessionInternal)session1).getConnection();
-
- ((ConnectionManagerImpl)sf1.getConnectionManagers()[0]).stopPingingAfterOne();
-
- for (int i = 0; i < 1000; i++)
- {
- // a few tries to avoid a possible race caused by GCs or similar issues
- if (liveServer.getRemotingService().getConnections().isEmpty())
- {
- break;
- }
-
- Thread.sleep(10);
- }
-
- assertEquals(0, liveServer.getRemotingService().getConnections().size());
-
- assertEquals(1, backupServer.getRemotingService().getConnections().size());
-
- session1.close();
-
- assertEquals(0, liveServer.getRemotingService().getConnections().size());
-
- assertEquals(1, backupServer.getRemotingService().getConnections().size());
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupServer = HornetQ.newHornetQServer(backupConf, false);
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- liveServer.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- backupServer.stop();
-
- liveServer.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupServer = null;
-
- liveServer = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleAutomaticFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleAutomaticFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleAutomaticFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,981 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.FailureListener;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMConnector;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.remoting.spi.ConnectionLifeCycleListener;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A SimpleAutomaticFailoverTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- */
-public class SimpleAutomaticFailoverTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(SimpleAutomaticFailoverTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveService;
-
- private HornetQServer backupService;
-
- private final Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testReplication() throws Exception
- {
- ClientSessionFactory sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 100;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = consumer.receive(250);
-
- assertNull(message3);
-
- session.close();
- }
-
- public void testFailoverSameConnectionFactory() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn1 = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- session.close();
-
- session = sf.createSession(false, true, true);
-
- consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = numMessages / 2; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = consumer.receive(250);
-
- session.close();
-
- assertNull(message3);
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testFailoverChangeConnectionFactory() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- session.close();
-
- sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- session = sf.createSession(false, true, true);
-
- consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = numMessages / 2; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = consumer.receive(250);
-
- assertNull(message3);
-
- session.close();
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testNoMessagesLeftAfterFailoverNewSession() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- session.close();
-
- sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- session = sf.createSession(false, true, true);
-
- consumer = session.createConsumer(ADDRESS);
-
- ClientMessage message3 = consumer.receive(250);
-
- assertNull(message3);
-
- session.close();
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testCreateMoreSessionsAfterFailover() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- ClientSession session1 = sf.createSession(false, true, true);
-
- ClientSession session2 = sf.createSession(false, true, true);
-
- ClientSession session3 = sf.createSession(false, true, true);
-
- session.close();
-
- session1.close();
-
- session2.close();
-
- session3.close();
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testFailoverMultipleSessions() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- final int numSessions = ClientSessionFactoryImpl.DEFAULT_MAX_CONNECTIONS * 2;
-
- List<ClientSession> sessions = new ArrayList<ClientSession>();
-
- List<ClientConsumer> consumers = new ArrayList<ClientConsumer>();
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession sess = sf.createSession(false, true, true);
-
- SimpleString queueName = new SimpleString("subscription" + i);
-
- sess.createQueue(ADDRESS, queueName, null, false);
-
- ClientConsumer consumer = sess.createConsumer(queueName);
-
- sess.start();
-
- sessions.add(sess);
-
- consumers.add(consumer);
- }
-
- ClientSession session = sf.createSession(false, true, true);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 100;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientConsumer cons = consumers.get(i);
-
- for (int j = 0; j < numMessages; j++)
- {
- ClientMessage message2 = cons.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(j, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = cons.receive(250);
-
- assertNull(message3);
- }
-
- session.close();
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession sess = sessions.get(i);
-
- sess.close();
- }
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testAllConnectionsReturned() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- final int numSessions = ClientSessionFactoryImpl.DEFAULT_MAX_CONNECTIONS * 2;
-
- List<ClientSession> sessions = new ArrayList<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession sess = sf.createSession(false, true, true);
-
- sessions.add(sess);
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession sess = sessions.get(i);
-
- sess.close();
- }
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testAllConnectionsReturnedAfterFailover() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- final int numSessions = ClientSessionFactoryImpl.DEFAULT_MAX_CONNECTIONS * 2;
-
- List<ClientSession> sessions = new ArrayList<ClientSession>();
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession sess = sf.createSession(false, true, true);
-
- sessions.add(sess);
- }
-
- RemotingConnection conn = ((ClientSessionInternal)sessions.get(0)).getConnection();
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- for (int i = 0; i < numSessions; i++)
- {
- ClientSession sess = sessions.get(i);
-
- sess.close();
- }
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testFailureAfterFailover() throws Exception
- {
- final long retryInterval = 500;
-
- final double retryMultiplier = 1d;
-
- final int reconnectAttempts = 0;
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
-
- sf.setFailoverOnServerShutdown(true);
- sf.setRetryInterval(retryInterval);
- sf.setRetryIntervalMultiplier(retryMultiplier);
- sf.setReconnectAttempts(reconnectAttempts);
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- // Consume half of them
-
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- RemotingConnection conn2 = ((ClientSessionInternal)session).getConnection();
-
- final CountDownLatch latch = new CountDownLatch(1);
-
- class MyListener implements FailureListener
- {
- public void connectionFailed(final HornetQException me)
- {
- latch.countDown();
- }
- }
-
- conn2.addFailureListener(new MyListener());
-
- assertFalse(conn == conn2);
-
- InVMConnector.failOnCreateConnection = true;
- conn2.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
-
- try
- {
- session.createQueue(new SimpleString("blah"), new SimpleString("blah"), null, false);
-
- fail("Should throw exception");
- }
- catch (HornetQException me)
- {
- assertEquals(HornetQException.NOT_CONNECTED, me.getCode());
- }
-
- session.close();
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- public void testFailoverOnCreateSession() throws Exception
- {
- stopServers();
-
- for (int j = 0; j < 10; j++)
- {
- startServers();
-
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
-
- sf.setFailoverOnServerShutdown(true);
- sf.setRetryInterval(100);
- sf.setRetryIntervalMultiplier(1);
- sf.setReconnectAttempts(-1);
- sf.setProducerWindowSize(32 * 1024);
-
- for (int i = 0; i < 10; i++)
- {
- // We test failing on the 0th connection created, then the first, then the second etc, to make sure they are
- // all failed over ok
- if (i == j)
- {
- InVMConnector.numberOfFailures = 1;
- InVMConnector.failOnCreateConnection = true;
- }
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 10;
-
- for (int k = 0; k < numMessages; k++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), k);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int k = 0; k < numMessages; k++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(k, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- consumer.close();
-
- session.deleteQueue(ADDRESS);
-
- session.close();
- }
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
-
- sf.close();
-
- stopServers();
- }
- }
-
- public void testFailoverWithNotifications() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession sess = sf.createSession(false, true, true);
-
- sess.createQueue("hornetq.notifications", "notifqueue", false);
-
- ClientConsumer cons = sess.createConsumer("notifqueue");
-
- sess.start();
-
- sess.createQueue("blah", "blah", false);
- sess.createQueue("blah", "blah2", false);
-
- ClientMessage msg = cons.receive(1000);
- assertNotNull(msg);
- msg.acknowledge();
-
- msg = cons.receive(1000);
- assertNotNull(msg);
- msg.acknowledge();
-
- sess.stop();
-
- sess.createQueue("blah", "blah3", false);
- sess.createQueue("blah", "blah4", false);
-
- RemotingConnection conn = ((ClientSessionInternal)sess).getConnection();
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- sess.start();
-
- msg = cons.receive(1000);
- assertNotNull(msg);
- msg.acknowledge();
-
- msg = cons.receive(1000);
- assertNotNull(msg);
- msg.acknowledge();
-
- sess.close();
-
- }
-
- /*
- * When a real connection fails due to the server actually dying, the backup server will receive
- * a connection exception on the server side, since the live server has died taking the replicating
- * connection with it.
- * We cannot just fail the connection on the server side when this happens since this will cause the session
- * on the backup to be closed, so clients won't be able to re-attach.
- * This test verifies that server session is not closed on server side connection failure.
- */
- public void testFailoverFailBothOnClientAndServerSide() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
-
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- RemotingConnection conn1 = ((ClientSessionInternal)session).getConnection();
-
- // Simulate failure on connection
- // We fail on the replicating connection and the client connection
-
- HornetQException me = new HornetQException(HornetQException.NOT_CONNECTED);
-
- //Note we call the remoting service impl handler which is what would happen in event
- //of real connection failure
-
- RemotingConnection serverSideReplicatingConnection = backupService.getRemotingService()
- .getServerSideReplicatingConnection();
-
-
- ((ConnectionLifeCycleListener)backupService.getRemotingService()).connectionException(serverSideReplicatingConnection.getID(), me);
-
- conn1.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numMessages / 2; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- session.close();
-
- session = sf.createSession(false, true, true);
-
- consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = numMessages / 2; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = consumer.receive(250);
-
- session.close();
-
- assertNull(message3);
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected void startServers() throws Exception
- {
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- protected void stopServers() throws Exception
- {
- if (backupService.isStarted())
- {
- backupService.stop();
- }
-
- if (liveService.isStarted())
- {
- liveService.stop();
- }
-
- assertEquals(0, InVMRegistry.instance.size());
- }
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- startServers();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- stopServers();
-
- InVMConnector.resetFailures();
-
- liveService = null;
-
- backupService = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleManualFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleManualFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SimpleManualFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,205 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.FailureListener;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A SimpleAutomaticFailoverTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- */
-public class SimpleManualFailoverTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(SimpleManualFailoverTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer server0Service;
-
- private HornetQServer server1Service;
-
- private Map<String, Object> server1Params = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- private ClientSession sendAndConsume(final ClientSessionFactory sf) throws Exception
- {
- ClientSession session = sf.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 1000;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- ClientConsumer consumer = session.createConsumer(ADDRESS);
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message2 = consumer.receive();
-
- assertEquals("aardvarks", message2.getBody().readString());
-
- assertEquals(i, message2.getProperty(new SimpleString("count")));
-
- message2.acknowledge();
- }
-
- ClientMessage message3 = consumer.receive(250);
-
- assertNull(message3);
-
- return session;
- }
-
- public void testFailover() throws Exception
- {
- ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- ClientSession session = sendAndConsume(sf);
-
- final CountDownLatch latch = new CountDownLatch(1);
-
- class MyListener implements FailureListener
- {
- public void connectionFailed(HornetQException me)
- {
- latch.countDown();
- }
- }
-
- RemotingConnection conn = ((ClientSessionInternal)session).getConnection();
-
- conn.addFailureListener(new MyListener());
-
- // Simulate failure on connection
- conn.fail(new HornetQException(HornetQException.NOT_CONNECTED));
-
- // Wait to be informed of failure
-
- boolean ok = latch.await(1000, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
-
- session.close();
-
- sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- server1Params));
-
- session = sendAndConsume(sf);
-
- session.close();
-
- assertEquals(0, sf.numSessions());
-
- assertEquals(0, sf.numConnections());
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration server1Conf = new ConfigurationImpl();
- server1Conf.setSecurityEnabled(false);
- server1Params.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- server1Conf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- server1Params));
- server1Service = HornetQ.newHornetQServer(server1Conf, false);
- server1Service.start();
-
- Configuration server0Conf = new ConfigurationImpl();
- server0Conf.setSecurityEnabled(false);
- server0Conf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- server0Service = HornetQ.newHornetQServer(server0Conf, false);
- server0Service.start();
- }
-
- protected void tearDown() throws Exception
- {
- server1Service.stop();
-
- server0Service.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- server1Service = null;
-
- server0Service = null;
-
- server1Params = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SplitBrainTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SplitBrainTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/SplitBrainTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,251 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.client.impl.ClientSessionInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.RemotingConnection;
-import org.hornetq.core.remoting.impl.invm.InVMConnectorFactory;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.remoting.spi.Connection;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQTextMessage;
-import org.hornetq.tests.util.UnitTestCase;
-import org.hornetq.utils.SimpleString;
-
-/**
- *
- * A SplitBrainTest
- *
- * Verify that split brain can occur
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- *
- * Created 6 Nov 2008 11:27:17
- *
- *
- */
-public class SplitBrainTest extends UnitTestCase
-{
- private static final Logger log = Logger.getLogger(SplitBrainTest.class);
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- private static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private HornetQServer liveServer;
-
- private HornetQServer backupServer;
-
- private Map<String, Object> backupParams = new HashMap<String, Object>();
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- public void testDemonstrateSplitBrain() throws Exception
- {
- ClientSessionFactoryInternal sf1 = new ClientSessionFactoryImpl(new TransportConfiguration(InVMConnectorFactory.class.getName()),
- new TransportConfiguration(InVMConnectorFactory.class.getName(),
- backupParams));
-
- sf1.setBlockOnNonPersistentSend(true);
-
- ClientSession session = sf1.createSession(false, true, true);
-
- session.createQueue(ADDRESS, ADDRESS, null, false);
-
- ClientProducer producer = session.createProducer(ADDRESS);
-
- final int numMessages = 10;
-
- int sendCount = 0;
-
- int consumeCount = 0;
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), sendCount++);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- //Now fail the replicating connections
- Set<RemotingConnection> conns = liveServer.getRemotingService().getConnections();
- for (RemotingConnection conn : conns)
- {
- RemotingConnection replicatingConnection = liveServer.getReplicatingChannel().getConnection();
- Connection tcConn = replicatingConnection.getTransportConnection();
- tcConn.fail(new HornetQException(HornetQException.INTERNAL_ERROR, "blah"));
- }
-
- Thread.sleep(2000);
-
- //Fail client connection
- ((ClientSessionInternal)session).getConnection().fail(new HornetQException(HornetQException.NOT_CONNECTED, "simulated failure b/w client and live node"));
-
- ClientConsumer consumer1 = session.createConsumer(ADDRESS);
-
- session.start();
-
- Set<Integer> deliveredMessageIDs = new HashSet<Integer>();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer1.receive(1000);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- int count = (Integer)message.getProperty(new SimpleString("count"));
-
- assertEquals(consumeCount++, count);
-
- deliveredMessageIDs.add(count);
-
- message.acknowledge();
- }
-
- session.close();
-
- sf1.close();
-
- //Now try and connect to live node - even though we failed over
-
- ClientSessionFactoryInternal sf2 = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"));
-
- session = sf2.createSession(false, true, true);
-
- producer = session.createProducer(ADDRESS);
-
- consumer1 = session.createConsumer(ADDRESS);
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = session.createClientMessage(HornetQTextMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("count"), i + numMessages);
- message.getBody().writeString("aardvarks");
- producer.send(message);
- }
-
- session.start();
-
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = consumer1.receive(1000);
-
- assertNotNull(message);
-
- assertEquals("aardvarks", message.getBody().readString());
-
- int count = (Integer)message.getProperty(new SimpleString("count"));
-
- //Assert that this has been consumed before!!
- assertTrue(deliveredMessageIDs.contains(count));
-
- message.acknowledge();
- }
-
- session.close();
-
- sf2.close();
-
- }
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- @Override
- protected void setUp() throws Exception
- {
- super.setUp();
-
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupServer = HornetQ.newHornetQServer(backupConf, false);
- backupServer.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams, "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveServer = HornetQ.newHornetQServer(liveConf, false);
- liveServer.start();
- }
-
- @Override
- protected void tearDown() throws Exception
- {
- backupServer.stop();
-
- liveServer.stop();
-
- assertEquals(0, InVMRegistry.instance.size());
-
- backupServer = null;
-
- liveServer = null;
-
- backupParams = null;
-
- super.tearDown();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XALargeMessageMultiThreadFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XALargeMessageMultiThreadFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XALargeMessageMultiThreadFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,141 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.JournalType;
-
-/**
- * A LargeMessageMultiThreadFailoverTest
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- * Created Jan 18, 2009 4:52:09 PM
- *
- *
- */
-public class XALargeMessageMultiThreadFailoverTest extends XAMultiThreadRandomFailoverTest
-{
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
- private static final byte[] BODY = new byte[500];
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
- @Override
- protected ClientSessionFactoryInternal createSessionFactory()
- {
- ClientSessionFactoryInternal sf = super.createSessionFactory();
- sf.setMinLargeMessageSize(200);
- return sf;
-
- }
-
- @Override
- protected void start() throws Exception
- {
-
- deleteDirectory(new File(getTestDir()));
-
- Configuration backupConf = new ConfigurationImpl();
-
- backupConf.setJournalDirectory(getJournalDir(getTestDir() + "/backup"));
- backupConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/backup"));
- backupConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/backup"));
- backupConf.setPagingDirectory(getPageDir(getTestDir() + "/backup"));
- backupConf.setJournalFileSize(100 * 1024);
-
- backupConf.setJournalType(JournalType.ASYNCIO);
-
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
-
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName(), backupParams));
- backupConf.setBackup(true);
-
- backupService = HornetQ.newHornetQServer(backupConf);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
-
- liveConf.setJournalDirectory(getJournalDir(getTestDir() + "/live"));
- liveConf.setLargeMessagesDirectory(getLargeMessagesDir(getTestDir() + "/live"));
- liveConf.setBindingsDirectory(getBindingsDir(getTestDir() + "/live"));
- liveConf.setPagingDirectory(getPageDir(getTestDir() + "/live"));
-
- liveConf.setJournalFileSize(100 * 1024);
-
- liveConf.setJournalType(JournalType.ASYNCIO);
-
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration(InVMAcceptorFactory.class.getCanonicalName()));
-
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
-
- TransportConfiguration backupTC = new TransportConfiguration(INVM_CONNECTOR_FACTORY,
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf);
-
- liveService.start();
-
- }
-
- @Override
- protected void setBody(final ClientMessage message) throws Exception
- {
-
- message.getBody().writeBytes(BODY);
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTestBase#checkSize(org.hornetq.core.client.ClientMessage)
- */
- @Override
- protected boolean checkSize(final ClientMessage message)
- {
- return BODY.length == message.getBodySize();
- }
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-
-}
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XAMultiThreadRandomFailoverTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XAMultiThreadRandomFailoverTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/cluster/failover/XAMultiThreadRandomFailoverTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,924 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.integration.cluster.failover;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import javax.transaction.xa.XAException;
-import javax.transaction.xa.XAResource;
-import javax.transaction.xa.Xid;
-
-import org.hornetq.core.client.ClientConsumer;
-import org.hornetq.core.client.ClientMessage;
-import org.hornetq.core.client.ClientProducer;
-import org.hornetq.core.client.ClientSession;
-import org.hornetq.core.client.ClientSessionFactory;
-import org.hornetq.core.client.MessageHandler;
-import org.hornetq.core.client.impl.ClientSessionFactoryImpl;
-import org.hornetq.core.client.impl.ClientSessionFactoryInternal;
-import org.hornetq.core.config.Configuration;
-import org.hornetq.core.config.TransportConfiguration;
-import org.hornetq.core.config.impl.ConfigurationImpl;
-import org.hornetq.core.exception.HornetQException;
-import org.hornetq.core.logging.Logger;
-import org.hornetq.core.remoting.impl.invm.InVMRegistry;
-import org.hornetq.core.remoting.impl.invm.TransportConstants;
-import org.hornetq.core.server.HornetQ;
-import org.hornetq.core.server.HornetQServer;
-import org.hornetq.jms.client.HornetQBytesMessage;
-import org.hornetq.utils.SimpleString;
-
-/**
- * A MultiThreadRandomFailoverStressTest
- *
- * @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- */
-public class XAMultiThreadRandomFailoverTest extends MultiThreadFailoverSupport
-{
- protected static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- private static final int RECEIVE_TIMEOUT = 30000;
-
- private final Logger log = Logger.getLogger(getClass());
-
- protected HornetQServer liveService;
-
- protected HornetQServer backupService;
-
- protected Map<String, Object> backupParams = new HashMap<String, Object>();
-
- protected Map<ClientSession, Xid> xids;
-
- private int NUM_THREADS = getNumThreads();
-
- private final int LATCH_WAIT = getLatchWait();
-
- private final int NUM_SESSIONS = getNumSessions();
-
- protected void tearDown() throws Exception
- {
- try
- {
- if (backupService != null && backupService.isStarted())
- {
- backupService.stop();
- }
- }
- catch (Throwable ignored)
- {
- }
-
- try
- {
- if (liveService != null && liveService.isStarted())
- {
- liveService.stop();
- }
- }
- catch (Throwable ignored)
- {
- }
-
- liveService = null;
- backupService = null;
-
- super.tearDown();
- }
-
- protected int getNumSessions()
- {
- return 10;
- }
-
- protected int getLatchWait()
- {
- return 20000;
- }
-
- protected int getNumThreads()
- {
- return 10;
- }
-
- protected int getNumIterations()
- {
- return 2;
- }
-
- protected boolean shouldFail()
- {
- return true;
- }
-
- protected ClientSession createTransactionalSession(ClientSessionFactory sf) throws Exception
- {
- ClientSession sess = sf.createSession(true, false, false);
- return sess;
- }
-
- public void testC() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestC(sf, threadNum);
- }
- }, NUM_THREADS, false, 3000);
- }
-
- public void testD() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestD(sf, threadNum);
- }
- }, NUM_THREADS, false, 3000);
- }
-
- public void testG() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestG(sf, threadNum);
- }
- }, NUM_THREADS, false, 3000);
- }
-
- public void testH() throws Exception
- {
- runTestMultipleThreads(new RunnableT()
- {
- @Override
- public void run(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- doTestH(sf, threadNum);
- }
- }, NUM_THREADS, false, 3000);
- }
-
- protected void doTestC(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = NUM_SESSIONS;
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = createTransactionalSession(sf);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- MyHandler handler = new MyHandler(threadNum, numMessages, sessConsume, consumer);
-
- handler.setCommitOnComplete(false);
-
- handler.start();
-
- handlers.add(handler);
- }
-
- ClientSession sessSend = createTransactionalSession(sf);
-
- transactionallySendMessages(threadNum, numMessages, sessSend);
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
- }
-
- for (MyHandler handler : handlers)
- {
- handler.setCommitOnComplete(true);
- handler.start();
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- assertTrue(ok);
- }
-
- sessSend.close();
-
- for (MyHandler handler : handlers)
- {
- handler.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- for (MyHandler handler : handlers)
- {
- if (handler.failure != null)
- {
- fail(handler.failure);
- }
- }
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestD(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<MyHandler> handlers = new HashSet<MyHandler>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + " sub" + i);
-
- ClientSession sessConsume = createTransactionalSession(sf);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- MyHandler handler = new MyHandler(threadNum, numMessages, sessConsume, consumer);
-
- handlers.add(handler);
- }
-
- ClientSession sessSend = createTransactionalSession(sf);
-
- transactionallySendMessages(threadNum, numMessages, sessSend);
-
- for (MyHandler handler : handlers)
- {
- handler.session.start();
- }
-
- for (MyHandler handler : handlers)
- {
- handler.start();
- }
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed: " + handler.failure);
- }
- }
-
- Set<MyHandler> newhandlers = new HashSet<MyHandler>();
-
- for (MyHandler handler : handlers)
- {
- MyHandler newHandler = new MyHandler(threadNum, numMessages, handler.session, handler.consumer);
- newHandler.setCommitOnComplete(true);
- newHandler.start();
- newhandlers.add(newHandler);
- }
-
- handlers.clear();
-
- handlers = newhandlers;
-
- for (MyHandler handler : handlers)
- {
- boolean ok = handler.latch.await(LATCH_WAIT, TimeUnit.MILLISECONDS);
-
- if (!ok)
- {
- throw new Exception("Timed out waiting for messages on handler " + System.identityHashCode(handler) +
- " threadnum " +
- threadNum);
- }
-
- if (handler.failure != null)
- {
- throw new Exception("Handler failed on rollback: " + handler.failure);
- }
- }
-
- sessSend.close();
-
- for (MyHandler handler : handlers)
- {
- handler.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + " sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestG(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = getNumSessions();
-
- Set<MyInfo> myinfos = new HashSet<MyInfo>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = sf.createSession(true, false, false);
-
- sessConsume.start();
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- Xid xid = newXID();
-
- sessConsume.start(xid, XAResource.TMNOFLAGS);
-
- myinfos.add(new MyInfo(sessConsume, consumer, xid));
- }
-
- ClientSession sessSend = sf.createSession(true, false, false);
-
- transactionallySendMessages(threadNum, numMessages, sessSend);
- consumeMessages(myinfos, numMessages, threadNum);
-
- for (MyInfo info : myinfos)
- {
- info.session.end(info.xid, XAResource.TMSUCCESS);
- info.session.prepare(info.xid);
- info.session.rollback(info.xid);
- info.xid = newXID();
- info.session.start(info.xid, XAResource.TMNOFLAGS);
- }
-
- consumeMessages(myinfos, numMessages, threadNum);
-
- for (MyInfo info : myinfos)
- {
- info.session.end(info.xid, XAResource.TMSUCCESS);
- info.session.prepare(info.xid);
- info.session.commit(info.xid, false);
- info.xid = null;
- }
-
- sessSend.close();
- for (MyInfo info : myinfos)
- {
- info.session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- protected void doTestH(final ClientSessionFactory sf, final int threadNum) throws Exception
- {
- long start = System.currentTimeMillis();
-
- ClientSession s = sf.createSession(false, false, false);
-
- final int numMessages = 100;
-
- final int numSessions = 10;
-
- Set<MyInfo> myinfos = new HashSet<MyInfo>();
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- ClientSession sessConsume = sf.createSession(true, false, false);
-
- sessConsume.createQueue(ADDRESS, subName, null, false);
-
- ClientConsumer consumer = sessConsume.createConsumer(subName);
-
- Xid xid = newXID();
-
- sessConsume.start(xid, XAResource.TMNOFLAGS);
-
- myinfos.add(new MyInfo(sessConsume, consumer, xid));
- }
-
- ClientSession sessSend = sf.createSession(true, false, false);
-
- transactionallySendMessages(threadNum, numMessages, sessSend);
-
- for (MyInfo info : myinfos)
- {
- info.session.start();
- }
-
- consumeMessages(myinfos, numMessages, threadNum);
-
- for (MyInfo info : myinfos)
- {
- info.session.end(info.xid, XAResource.TMSUCCESS);
- info.session.prepare(info.xid);
- info.session.rollback(info.xid);
- info.xid = newXID();
- info.session.start(info.xid, XAResource.TMNOFLAGS);
- }
-
- consumeMessages(myinfos, numMessages, threadNum);
-
- for (MyInfo info : myinfos)
- {
- info.session.end(info.xid, XAResource.TMSUCCESS);
- info.session.prepare(info.xid);
- info.session.commit(info.xid, false);
- info.xid = null;
- }
-
- sessSend.close();
- for (MyInfo info : myinfos)
- {
- info.session.close();
- }
-
- for (int i = 0; i < numSessions; i++)
- {
- SimpleString subName = new SimpleString(threadNum + "sub" + i);
-
- s.deleteQueue(subName);
- }
-
- s.close();
-
- long end = System.currentTimeMillis();
-
- log.info("duration " + (end - start));
- }
-
- /**
- * @param threadNum
- * @param numMessages
- * @param sessSend
- * @throws XAException
- * @throws HornetQException
- * @throws Exception
- */
- private void transactionallySendMessages(final int threadNum, final int numMessages, ClientSession sessSend) throws XAException,
- HornetQException,
- Exception
- {
- Xid xid = newXID();
- sessSend.start(xid, XAResource.TMNOFLAGS);
-
- ClientProducer producer = sessSend.createProducer(ADDRESS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.end(xid, XAResource.TMSUCCESS);
- sessSend.rollback(xid);
-
- xid = newXID();
- sessSend.start(xid, XAResource.TMNOFLAGS);
-
- sendMessages(sessSend, producer, numMessages, threadNum);
-
- sessSend.end(xid, XAResource.TMSUSPEND);
-
- sessSend.start(xid, XAResource.TMRESUME);
-
- sessSend.end(xid, XAResource.TMSUCCESS);
-
- sessSend.commit(xid, true);
- }
-
- private void consumeMessages(final Set<MyInfo> myinfos, final int numMessages, final int threadNum) throws Exception
- {
- // We make sure the messages arrive in the order they were sent from a particular producer
- Map<ClientConsumer, Map<Integer, Integer>> counts = new HashMap<ClientConsumer, Map<Integer, Integer>>();
-
- for (int i = 0; i < numMessages; i++)
- {
- for (MyInfo myinfo : myinfos)
- {
- Map<Integer, Integer> consumerCounts = counts.get(myinfo);
-
- if (consumerCounts == null)
- {
- consumerCounts = new HashMap<Integer, Integer>();
- counts.put(myinfo.consumer, consumerCounts);
- }
-
- ClientMessage msg = myinfo.consumer.receive(RECEIVE_TIMEOUT);
-
- assertNotNull(msg);
-
- int tn = (Integer)msg.getProperty(new SimpleString("threadnum"));
- int cnt = (Integer)msg.getProperty(new SimpleString("count"));
-
- Integer c = consumerCounts.get(tn);
- if (c == null)
- {
- c = new Integer(cnt);
- }
-
- if (tn == threadNum && cnt != c.intValue())
- {
- throw new Exception("Invalid count, expected " + tn + ": " + c + " got " + cnt);
- }
-
- c++;
-
- // Wrap
- if (c == numMessages)
- {
- c = 0;
- }
-
- consumerCounts.put(tn, c);
-
- msg.acknowledge();
- }
- }
- }
-
- @Override
- protected void start() throws Exception
- {
- Configuration backupConf = new ConfigurationImpl();
- backupConf.setSecurityEnabled(false);
- backupParams.put(TransportConstants.SERVER_ID_PROP_NAME, 1);
- backupConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory",
- backupParams));
- backupConf.setBackup(true);
- backupService = HornetQ.newHornetQServer(backupConf, false);
- backupService.start();
-
- Configuration liveConf = new ConfigurationImpl();
- liveConf.setSecurityEnabled(false);
- liveConf.getAcceptorConfigurations()
- .add(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMAcceptorFactory"));
- Map<String, TransportConfiguration> connectors = new HashMap<String, TransportConfiguration>();
- TransportConfiguration backupTC = new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams,
- "backup-connector");
- connectors.put(backupTC.getName(), backupTC);
- liveConf.setConnectorConfigurations(connectors);
- liveConf.setBackupConnectorName(backupTC.getName());
- liveService = HornetQ.newHornetQServer(liveConf, false);
- liveService.start();
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadFailoverSupport#stop()
- */
- @Override
- protected void stop() throws Exception
- {
- backupService.stop();
-
- liveService.stop();
-
- backupService = null;
-
- liveService = null;
-
- assertEquals(0, InVMRegistry.instance.size());
- }
-
- @Override
- protected ClientSessionFactoryInternal createSessionFactory()
- {
- final ClientSessionFactoryInternal sf = new ClientSessionFactoryImpl(new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory"),
- new TransportConfiguration("org.hornetq.core.remoting.impl.invm.InVMConnectorFactory",
- backupParams));
-
- sf.setProducerWindowSize(32 * 1024);
- return sf;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTestBase#setBody(org.hornetq.core.client.ClientMessage)
- */
- protected void setBody(final ClientMessage message) throws Exception
- {
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTestBase#checkSize(org.hornetq.core.client.ClientMessage)
- */
- protected boolean checkSize(final ClientMessage message)
- {
- return 0 == message.getBody().writerIndex();
- }
-
- private void runTestMultipleThreads(final RunnableT runnable,
- final int numThreads,
- final boolean failOnCreateConnection,
- final long failDelay) throws Exception
- {
-
- runMultipleThreadsFailoverTest(runnable, numThreads, getNumIterations(), failOnCreateConnection, failDelay);
- }
-
- private void sendMessages(final ClientSession sessSend,
- final ClientProducer producer,
- final int numMessages,
- final int threadNum) throws Exception
- {
- for (int i = 0; i < numMessages; i++)
- {
- ClientMessage message = sessSend.createClientMessage(HornetQBytesMessage.TYPE,
- false,
- 0,
- System.currentTimeMillis(),
- (byte)1);
- message.putIntProperty(new SimpleString("threadnum"), threadNum);
- message.putIntProperty(new SimpleString("count"), i);
- setBody(message);
- producer.send(message);
- }
- }
-
- private class MyInfo
- {
- final ClientSession session;
-
- Xid xid;
-
- final ClientConsumer consumer;
-
- public MyInfo(final ClientSession session, final ClientConsumer consumer, final Xid xid)
- {
- this.session = session;
- this.consumer = consumer;
- this.xid = xid;
- }
- }
-
- private class MyHandler implements MessageHandler
- {
- CountDownLatch latch = new CountDownLatch(1);
-
- private final Map<Integer, Integer> counts = new HashMap<Integer, Integer>();
-
- volatile String failure;
-
- final int tn;
-
- final int numMessages;
-
- final ClientSession session;
-
- final ClientConsumer consumer;
-
- volatile Xid xid;
-
- volatile boolean done;
-
- volatile boolean started = false;
-
- volatile boolean commit = false;
-
- synchronized void start() throws Exception
- {
- counts.clear();
-
- done = false;
-
- failure = null;
-
- latch = new CountDownLatch(1);
-
- xid = newXID();
- session.start(xid, XAResource.TMNOFLAGS);
- started = true;
- consumer.setMessageHandler(this);
- session.start();
- }
-
- synchronized void stop() throws Exception
- {
- session.stop();
- // FIXME: Remove this line when https://jira.jboss.org/jira/browse/JBMESSAGING-1549 is done
- consumer.setMessageHandler(null);
- started = false;
- }
-
- synchronized void close() throws Exception
- {
- stop();
- session.close();
- }
-
- private synchronized void rollback()
- {
- try
- {
- stop();
- session.end(xid, XAResource.TMSUCCESS);
- session.prepare(xid);
- session.rollback(xid);
- }
- catch (Exception e)
- {
- this.failure = e.getLocalizedMessage();
- }
- }
-
- private synchronized void commit()
- {
- try
- {
- stop();
-
- // Suspend & resume... just exercising the API as part of the test
- session.end(xid, XAResource.TMSUSPEND);
- session.start(xid, XAResource.TMRESUME);
-
- session.end(xid, XAResource.TMSUCCESS);
- session.prepare(xid);
- session.commit(xid, false);
- }
- catch (Exception e)
- {
- this.failure = e.getLocalizedMessage();
- }
- }
-
- MyHandler(final int threadNum, final int numMessages, final ClientSession session, final ClientConsumer consumer) throws Exception
- {
- tn = threadNum;
-
- this.numMessages = numMessages;
-
- this.session = session;
-
- this.consumer = consumer;
-
- }
-
- public void setCommitOnComplete(boolean commit)
- {
- this.commit = commit;
- }
-
- public synchronized void onMessage(final ClientMessage message)
- {
-
- if (!started)
- {
- this.failure = "Received message with session stopped (thread = " + tn + ")";
- log.error(failure);
- return;
- }
-
- try
- {
- message.acknowledge();
- }
- catch (HornetQException me)
- {
- log.error("Failed to process", me);
- }
-
- if (done)
- {
- return;
- }
-
- int threadNum = (Integer)message.getProperty(new SimpleString("threadnum"));
- int cnt = (Integer)message.getProperty(new SimpleString("count"));
-
- Integer c = counts.get(threadNum);
- if (c == null)
- {
- c = new Integer(cnt);
- }
-
- if (tn == threadNum && cnt != c.intValue())
- {
- failure = "Invalid count, expected " + threadNum + ":" + c + " got " + cnt;
- log.error(failure);
-
- latch.countDown();
- }
-
- if (!checkSize(message))
- {
- failure = "Invalid size on message";
- log.error(failure);
- latch.countDown();
- }
-
- if (tn == threadNum && c == numMessages - 1)
- {
- done = true;
- if (commit)
- {
- commit();
- }
- else
- {
- rollback();
- }
- latch.countDown();
- }
-
- c++;
- // Wrap around at numMessages
- if (c == numMessages)
- {
- c = 0;
- }
-
- counts.put(threadNum, c);
-
- }
- }
-
-}
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -70,10 +70,15 @@
{
return new HornetQServerControl()
- {
+ {
private final CoreMessagingProxy proxy = new CoreMessagingProxy(session,
ResourceNames.CORE_SERVER);
+ public boolean isSharedStore()
+ {
+ return (Boolean)proxy.retrieveAttributeValue("sharedStore");
+ }
+
public boolean closeConnectionsForAddress(String ipAddress) throws Exception
{
return (Boolean)proxy.invokeOperation("closeConnectionsForAddress", ipAddress);
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/paging/PageCrashTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/paging/PageCrashTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/integration/paging/PageCrashTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -84,7 +84,11 @@
Configuration config = createDefaultConfig();
- HornetQServer messagingService = createServer(true, config, 10 * 1024, 100 * 1024, new HashMap<String, AddressSettings>());
+ HornetQServer messagingService = createServer(true,
+ config,
+ 10 * 1024,
+ 100 * 1024,
+ new HashMap<String, AddressSettings>());
messagingService.start();
@@ -241,8 +245,7 @@
return new PagingManagerImpl(new FailurePagingStoreFactoryNIO(super.getConfiguration().getPagingDirectory()),
super.getStorageManager(),
super.getAddressSettingsRepository(),
- super.getConfiguration().isJournalSyncNonTransactional(),
- false);
+ super.getConfiguration().isJournalSyncNonTransactional());
}
class FailurePagingStoreFactoryNIO extends PagingStoreFactoryNIO
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/soak/failover/RandomFailoverSoakTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/soak/failover/RandomFailoverSoakTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/soak/failover/RandomFailoverSoakTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -14,7 +14,7 @@
package org.hornetq.tests.soak.failover;
-import org.hornetq.tests.integration.cluster.failover.RandomFailoverTest;
+import org.hornetq.tests.integration.cluster.failover.RandomReattachTest;
/**
* A RandomFailoverSoakTest
@@ -25,7 +25,7 @@
*
*
*/
-public class RandomFailoverSoakTest extends RandomFailoverTest
+public class RandomFailoverSoakTest extends RandomReattachTest
{
protected int getNumIterations()
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/LargeMessageMultiThreadFailoverStressTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/LargeMessageMultiThreadFailoverStressTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/LargeMessageMultiThreadFailoverStressTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,56 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.stress.failover;
-
-import org.hornetq.tests.integration.cluster.failover.LargeMessageMultiThreadFailoverTest;
-
-/**
- * A LargeMessageMultiThreadFailoverStressTest
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- * Created Jan 21, 2009 8:38:28 PM
- *
- *
- */
-public class LargeMessageMultiThreadFailoverStressTest extends LargeMessageMultiThreadFailoverTest
-{
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- // Static --------------------------------------------------------
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
-
- protected int getNumIterations()
- {
- return 10;
- }
-
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-
-}
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/MultiThreadRandomFailoverStressTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/MultiThreadRandomFailoverStressTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/MultiThreadRandomFailoverStressTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -13,7 +13,7 @@
package org.hornetq.tests.stress.failover;
-import org.hornetq.tests.integration.cluster.failover.MultiThreadRandomFailoverTest;
+import org.hornetq.tests.integration.cluster.failover.MultiThreadRandomReattachTest;
/**
* A MultiThreadRandomFailoverStressTest
@@ -21,7 +21,7 @@
* @author <a href="mailto:tim.fox@jboss.com">Tim Fox</a>
* @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
*/
-public class MultiThreadRandomFailoverStressTest extends MultiThreadRandomFailoverTest
+public class MultiThreadRandomFailoverStressTest extends MultiThreadRandomReattachTest
{
protected int getNumIterations()
{
Deleted: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/PagingFailoverStressTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/PagingFailoverStressTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/PagingFailoverStressTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -1,71 +0,0 @@
-/*
- * Copyright 2009 Red Hat, Inc.
- * Red Hat 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.hornetq.tests.stress.failover;
-
-import org.hornetq.tests.integration.cluster.failover.PagingFailoverTest;
-import org.hornetq.utils.SimpleString;
-
-/**
- * A PagingFailoverTest
- *
- * @author <a href="mailto:clebert.suconic@jboss.org">Clebert Suconic</a>
- *
- * Created Dec 8, 2008 10:53:16 AM
- *
- *
- */
-public class PagingFailoverStressTest extends PagingFailoverTest
-{
-
- // Constants -----------------------------------------------------
-
- // Attributes ----------------------------------------------------
-
- // Static --------------------------------------------------------
-
- protected static final SimpleString ADDRESS = new SimpleString("FailoverTestAddress");
-
- // Constructors --------------------------------------------------
-
- // Public --------------------------------------------------------
-
- // Package protected ---------------------------------------------
-
- // Protected -----------------------------------------------------
- protected int getNumberOfMessages()
- {
- return 5000;
- }
- protected int getNumberOfThreads()
- {
- return 10;
- }
-
- protected int getMaxGlobal()
- {
- return 10 * 1024;
- }
-
- protected int getPageSize()
- {
- return 5 * 1024;
- }
-
-
-
- // Private -------------------------------------------------------
-
- // Inner classes -------------------------------------------------
-
-}
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/RandomFailoverStressTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/RandomFailoverStressTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/stress/failover/RandomFailoverStressTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -14,7 +14,7 @@
package org.hornetq.tests.stress.failover;
-import org.hornetq.tests.integration.cluster.failover.RandomFailoverTest;
+import org.hornetq.tests.integration.cluster.failover.RandomReattachTest;
/**
* A RandomFailoverStressTest
@@ -25,7 +25,7 @@
*
*
*/
-public class RandomFailoverStressTest extends RandomFailoverTest
+public class RandomFailoverStressTest extends RandomReattachTest
{
// Constants -----------------------------------------------------
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/client/impl/LargeMessageBufferTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/client/impl/LargeMessageBufferTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/client/impl/LargeMessageBufferTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -37,6 +37,7 @@
import org.hornetq.core.remoting.impl.wireformat.SessionReceiveMessage;
import org.hornetq.tests.util.RandomUtil;
import org.hornetq.tests.util.UnitTestCase;
+import org.hornetq.utils.SimpleString;
/**
* A LargeMessageBufferUnitTest
@@ -551,6 +552,11 @@
static class FakeConsumerInternal implements ClientConsumerInternal
{
+ public void clearAtFailover()
+ {
+ // TODO Auto-generated method stub
+
+ }
/* (non-Javadoc)
* @see org.hornetq.core.client.impl.ClientConsumerInternal#acknowledge(org.hornetq.core.client.ClientMessage)
@@ -752,7 +758,26 @@
public void setLargeMessageCacheDir(File largeMessageCacheDir)
{
}
+
+ public SimpleString getFilterString()
+ {
+ // TODO Auto-generated method stub
+ return null;
+ }
+ public SimpleString getQueueName()
+ {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ public boolean isBrowseOnly()
+ {
+ // TODO Auto-generated method stub
+ return false;
+ }
+
+
}
}
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -128,10 +128,13 @@
private class FakeServerControl implements HornetQServerControl
{
+ public boolean isSharedStore()
+ {
+ return false;
+ }
public int getThreadPoolMaxSize()
{
-
return 0;
}
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/paging/impl/PagingManagerImplTest.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/paging/impl/PagingManagerImplTest.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/unit/core/paging/impl/PagingManagerImplTest.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -63,8 +63,7 @@
PagingManagerImpl managerImpl = new PagingManagerImpl(new PagingStoreFactoryNIO(getPageDir(), new OrderedExecutorFactory(Executors.newCachedThreadPool())),
new NullStorageManager(),
addressSettings,
- true,
- false);
+ true);
managerImpl.start();
Modified: branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/util/ServiceTestBase.java
===================================================================
--- branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/util/ServiceTestBase.java 2009-09-16 12:31:31 UTC (rev 7961)
+++ branches/Branch_Replication_Changes/tests/src/org/hornetq/tests/util/ServiceTestBase.java 2009-09-16 16:46:12 UTC (rev 7962)
@@ -14,7 +14,6 @@
package org.hornetq.tests.util;
import java.lang.management.ManagementFactory;
-import java.lang.ref.WeakReference;
import java.util.HashMap;
import java.util.Map;
@@ -64,6 +63,7 @@
// Static --------------------------------------------------------
private final Logger log = Logger.getLogger(this.getClass());
+
// Constructors --------------------------------------------------
// Public --------------------------------------------------------
@@ -73,10 +73,10 @@
// Protected -----------------------------------------------------
protected HornetQServer createServer(final boolean realFiles,
- final Configuration configuration,
- int pageSize,
- int maxAddressSize,
- final Map<String, AddressSettings> settings)
+ final Configuration configuration,
+ int pageSize,
+ int maxAddressSize,
+ final Map<String, AddressSettings> settings)
{
HornetQServer server;
@@ -104,9 +104,9 @@
}
protected HornetQServer createServer(final boolean realFiles,
- final Configuration configuration,
- final MBeanServer mbeanServer,
- final Map<String, AddressSettings> settings)
+ final Configuration configuration,
+ final MBeanServer mbeanServer,
+ final Map<String, AddressSettings> settings)
{
HornetQServer server;
@@ -141,23 +141,21 @@
}
protected HornetQServer createServer(final boolean realFiles,
- final Configuration configuration,
- final HornetQSecurityManager securityManager)
+ final Configuration configuration,
+ final HornetQSecurityManager securityManager)
{
HornetQServer server;
if (realFiles)
{
- server = HornetQ.newHornetQServer(configuration,
- ManagementFactory.getPlatformMBeanServer(),
- securityManager);
+ server = HornetQ.newHornetQServer(configuration, ManagementFactory.getPlatformMBeanServer(), securityManager);
}
else
{
server = HornetQ.newHornetQServer(configuration,
- ManagementFactory.getPlatformMBeanServer(),
- securityManager,
- false);
+ ManagementFactory.getPlatformMBeanServer(),
+ securityManager,
+ false);
}
Map<String, AddressSettings> settings = new HashMap<String, AddressSettings>();
@@ -175,8 +173,8 @@
}
protected HornetQServer createClusteredServerWithParams(final int index,
- final boolean realFiles,
- final Map<String, Object> params)
+ final boolean realFiles,
+ final Map<String, Object> params)
{
return createServer(realFiles,
createClusteredDefaultConfig(index, params, INVM_ACCEPTOR_FACTORY),
16 years, 3 months
JBoss hornetq SVN: r7961 - trunk/src/main/org/hornetq/core/server/impl.
by do-not-reply@jboss.org
Author: jmesnil
Date: 2009-09-16 08:31:31 -0400 (Wed, 16 Sep 2009)
New Revision: 7961
Modified:
trunk/src/main/org/hornetq/core/server/impl/QueueImpl.java
Log:
HORNETQ-26: Why does QueueImpl::changeMessagePriority delete the ref?
* do not delete the ref. Instead, removed the ref from the messageReferences,
change its prioriity and add it back.
Modified: trunk/src/main/org/hornetq/core/server/impl/QueueImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/server/impl/QueueImpl.java 2009-09-16 03:05:20 UTC (rev 7960)
+++ trunk/src/main/org/hornetq/core/server/impl/QueueImpl.java 2009-09-16 12:31:31 UTC (rev 7961)
@@ -983,23 +983,20 @@
public synchronized boolean changeReferencePriority(final long messageID, final byte newPriority) throws Exception
{
- List<MessageReference> refs = list(null);
- for (MessageReference ref : refs)
+ Iterator<MessageReference> iter = messageReferences.iterator();
+
+ while (iter.hasNext())
{
- ServerMessage message = ref.getMessage();
- if (message.getMessageID() == messageID)
+ MessageReference ref = iter.next();
+ if (ref.getMessage().getMessageID() == messageID)
{
- message.setPriority(newPriority);
- // delete and add the reference so that it
- // goes to the right queues for the new priority
-
- // FIXME - why deleting the reference?? This will delete it from storage!!
-
- deleteReference(messageID);
+ iter.remove();
+ ref.getMessage().setPriority(newPriority);
addLast(ref);
return true;
}
}
+
return false;
}
16 years, 3 months
JBoss hornetq SVN: r7960 - trunk.
by do-not-reply@jboss.org
Author: gaohoward
Date: 2009-09-15 23:05:20 -0400 (Tue, 15 Sep 2009)
New Revision: 7960
Modified:
trunk/build-hornetq.xml
Log:
in 'clean' ant target, added 'quiet="true"' attribute to each delete task.
Modified: trunk/build-hornetq.xml
===================================================================
--- trunk/build-hornetq.xml 2009-09-15 16:24:21 UTC (rev 7959)
+++ trunk/build-hornetq.xml 2009-09-16 03:05:20 UTC (rev 7960)
@@ -305,13 +305,13 @@
</path>
<target name="clean">
- <delete dir="${build.dir}"/>
- <delete dir="${test.build.dir}"/>
- <delete dir="${test.output.dir}"/>
- <delete dir="${test.jms.build.dir}"/>
- <delete dir="${test.joram.build.dir}"/>
- <delete dir="${thirdparty.dir}"/>
- <delete>
+ <delete dir="${build.dir}" quiet="true"/>
+ <delete dir="${test.build.dir}" quiet="true"/>
+ <delete dir="${test.output.dir}" quiet="true"/>
+ <delete dir="${test.jms.build.dir}" quiet="true"/>
+ <delete dir="${test.joram.build.dir}" quiet="true"/>
+ <delete dir="${thirdparty.dir}" quiet="true"/>
+ <delete quiet="true">
<fileset dir="${logs.dir}" includes="*"/>
</delete>
<ant antfile="${examples.dir}/build.xml" target="clean" inheritall="false"/>
16 years, 3 months
JBoss hornetq SVN: r7959 - in trunk/src/main/org/hornetq/core: server/impl and 1 other directory.
by do-not-reply@jboss.org
Author: jmesnil
Date: 2009-09-15 12:24:21 -0400 (Tue, 15 Sep 2009)
New Revision: 7959
Modified:
trunk/src/main/org/hornetq/core/postoffice/impl/PostOfficeImpl.java
trunk/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java
Log:
Core queue & address management registration
* moved registration of queue & addresses to the management service
from PostOfficeImpl.addBinding() to the HornetQServerImpl class
Modified: trunk/src/main/org/hornetq/core/postoffice/impl/PostOfficeImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/postoffice/impl/PostOfficeImpl.java 2009-09-15 13:20:48 UTC (rev 7958)
+++ trunk/src/main/org/hornetq/core/postoffice/impl/PostOfficeImpl.java 2009-09-15 16:24:21 UTC (rev 7959)
@@ -455,27 +455,8 @@
{
binding.setID(generateTransientID());
- boolean existed = addressManager.addBinding(binding);
-
- // TODO - why is this code here?
- // Shouldn't it be in HornetQServerImpl::createQueue??
- if (binding.getType() == BindingType.LOCAL_QUEUE)
- {
- Queue queue = (Queue)binding.getBindable();
-
- if (backup)
- {
- queue.setBackup();
- }
-
- managementService.registerQueue(queue, binding.getAddress(), storageManager);
-
- if (!existed)
- {
- managementService.registerAddress(binding.getAddress());
- }
- }
-
+ addressManager.addBinding(binding);
+
TypedProperties props = new TypedProperties();
props.putIntProperty(ManagementHelper.HDR_BINDING_TYPE, binding.getType().toInt());
Modified: trunk/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java 2009-09-15 13:20:48 UTC (rev 7958)
+++ trunk/src/main/org/hornetq/core/server/impl/HornetQServerImpl.java 2009-09-15 16:24:21 UTC (rev 7959)
@@ -1204,11 +1204,19 @@
true,
false);
+ if (configuration.isBackup())
+ {
+ queue.setBackup();
+ }
+
Binding binding = new LocalQueueBinding(queueBindingInfo.getAddress(), queue, nodeID);
queues.put(queueBindingInfo.getPersistenceID(), queue);
postOffice.addBinding(binding);
+
+ managementService.registerAddress(queueBindingInfo.getAddress());
+ managementService.registerQueue(queue, queueBindingInfo.getAddress(), storageManager);
}
Map<SimpleString, List<Pair<byte[], Long>>> duplicateIDMap = new HashMap<SimpleString, List<Pair<byte[], Long>>>();
@@ -1294,6 +1302,11 @@
final Queue queue = queueFactory.createQueue(-1, address, queueName, filter, durable, temporary);
+ if (configuration.isBackup())
+ {
+ queue.setBackup();
+ }
+
binding = new LocalQueueBinding(address, queue, nodeID);
if (durable)
@@ -1302,6 +1315,9 @@
}
postOffice.addBinding(binding);
+
+ managementService.registerAddress(address);
+ managementService.registerQueue(queue, address, storageManager);
return queue;
}
16 years, 3 months
JBoss hornetq SVN: r7958 - trunk/src/main/org/hornetq/core/management/impl.
by do-not-reply@jboss.org
Author: jmesnil
Date: 2009-09-15 09:20:48 -0400 (Tue, 15 Sep 2009)
New Revision: 7958
Modified:
trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java
Log:
Message Counter configuration
* removed messageCounterEnable attribute from HornetQServerControlImpl and
delegates to the configuration's attribute instead
Modified: trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java 2009-09-15 12:17:22 UTC (rev 7957)
+++ trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java 2009-09-15 13:20:48 UTC (rev 7958)
@@ -82,8 +82,6 @@
private final NotificationBroadcasterSupport broadcaster;
- private boolean messageCounterEnabled;
-
// Static --------------------------------------------------------
// Constructors --------------------------------------------------
@@ -103,7 +101,6 @@
this.server = messagingServer;
this.messageCounterManager = messageCounterManager;
this.broadcaster = broadcaster;
- this.messageCounterEnabled = configuration.isMessageCounterEnabled();
}
// Public --------------------------------------------------------
@@ -314,7 +311,7 @@
public boolean isMessageCounterEnabled()
{
- return messageCounterEnabled;
+ return configuration.isMessageCounterEnabled();
}
public synchronized long getMessageCounterSamplePeriod()
@@ -564,16 +561,16 @@
{
if (isStarted())
{
- if (messageCounterEnabled && !enable)
+ if (configuration.isMessageCounterEnabled() && !enable)
{
stopMessageCounters();
}
- else if (!messageCounterEnabled && enable)
+ else if (!configuration.isMessageCounterEnabled() && enable)
{
startMessageCounters();
}
}
- messageCounterEnabled = enable;
+ configuration.setMessageCounterEnabled(enable);
}
private void startMessageCounters()
16 years, 3 months
JBoss hornetq SVN: r7957 - trunk/src/main/org/hornetq/core/server/impl.
by do-not-reply@jboss.org
Author: jmesnil
Date: 2009-09-15 08:17:22 -0400 (Tue, 15 Sep 2009)
New Revision: 7957
Modified:
trunk/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java
Log:
HORNETQ-132: Server side browser is still taking a snapshot of the queue
* removed queue snapshot. The browser iteration optimization has been
done in JBMESSAGING-1437.
Modified: trunk/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java 2009-09-15 09:23:07 UTC (rev 7956)
+++ trunk/src/main/org/hornetq/core/server/impl/ServerSessionImpl.java 2009-09-15 12:17:22 UTC (rev 7957)
@@ -42,7 +42,6 @@
import org.hornetq.core.postoffice.Bindings;
import org.hornetq.core.postoffice.PostOffice;
import org.hornetq.core.postoffice.QueueBinding;
-import org.hornetq.core.postoffice.impl.LocalQueueBinding;
import org.hornetq.core.remoting.Channel;
import org.hornetq.core.remoting.CloseListener;
import org.hornetq.core.remoting.FailureListener;
@@ -1277,30 +1276,6 @@
filter = new FilterImpl(filterString);
}
- Queue theQueue;
-
- if (browseOnly)
- {
- // We consume a copy of the queue - TODO - this is a temporary measure
- // and will disappear once we can provide a proper iterator on the queue
-
- theQueue = queueFactory.createQueue(-1, binding.getAddress(), name, filter, false, true);
-
- // There's no need for any special locking since the list method is synchronized
- List<MessageReference> refs = ((Queue)binding.getBindable()).list(filter);
-
- for (MessageReference ref : refs)
- {
- theQueue.addLast(ref);
- }
-
- binding = new LocalQueueBinding(binding.getAddress(), theQueue, nodeID);
- }
- else
- {
- theQueue = (Queue)binding.getBindable();
- }
-
ServerConsumer consumer = new ServerConsumerImpl(idGenerator.generateID(),
oppositeChannelID,
this,
@@ -1331,6 +1306,8 @@
props.putIntProperty(ManagementHelper.HDR_DISTANCE, binding.getDistance());
+ Queue theQueue = (Queue)binding.getBindable();
+
props.putIntProperty(ManagementHelper.HDR_CONSUMER_COUNT, theQueue.getConsumerCount());
if (filterString != null)
16 years, 3 months
JBoss hornetq SVN: r7956 - in trunk: src/main/org/hornetq/core/management and 11 other directories.
by do-not-reply@jboss.org
Author: jmesnil
Date: 2009-09-15 05:23:07 -0400 (Tue, 15 Sep 2009)
New Revision: 7956
Added:
trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/FakeQueue.java
Modified:
trunk/docs/user-manual/en/management.xml
trunk/src/main/org/hornetq/core/management/HornetQServerControl.java
trunk/src/main/org/hornetq/core/management/ManagementService.java
trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java
trunk/src/main/org/hornetq/core/management/impl/ManagementServiceImpl.java
trunk/src/main/org/hornetq/core/management/jmx/impl/ReplicationAwareHornetQServerControlWrapper.java
trunk/src/main/org/hornetq/jms/server/JMSServerManager.java
trunk/src/main/org/hornetq/jms/server/impl/JMSServerManagerImpl.java
trunk/src/main/org/hornetq/jms/server/management/JMSServerControl.java
trunk/src/main/org/hornetq/jms/server/management/impl/JMSServerControlImpl.java
trunk/src/main/org/hornetq/jms/server/management/jmx/impl/ReplicationAwareJMSServerControlWrapper.java
trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlTest.java
trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlUsingJMSTest.java
trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlTest.java
trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java
trunk/tests/src/org/hornetq/tests/integration/management/ManagementServiceImplTest.java
trunk/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java
trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/BindingImplTest.java
Log:
HORNETQ-131: List of the destinations (queues and topics) currently deployed
* added attributes to list queues/addresses from Core HornetQServerControl
* added attributes to list queues/topics/connection factories from JMSServerControl
* doc + tests
Modified: trunk/docs/user-manual/en/management.xml
===================================================================
--- trunk/docs/user-manual/en/management.xml 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/docs/user-manual/en/management.xml 2009-09-15 09:23:07 UTC (rev 7956)
@@ -75,7 +75,9 @@
<title>Core Server Management</title>
<itemizedlist>
<listitem>
- <para>Creating, deploying and destroying queues</para>
+ <para>Listing, creating, deploying and destroying queues</para>
+ <para>A list of deployed core queues can be retrieved using the
+ <literal>getQueueNames()</literal> method.</para>
<para>Core queues can be created or destroyed using the management operations
<literal>createQueue()</literal> or <literal>deployQueue()</literal> or
<literal>destroyQueue()</literal>)on the <literal
@@ -296,7 +298,9 @@
>jms.server</literal>).</para>
<itemizedlist>
<listitem>
- <para>Creating/destroying connection factories</para>
+ <para>Listing, creating, destroying connection factories</para>
+ <para>Names of the deployed connection factories can be retrieved by the
+ <literal>getConnectionFactoryNames()</literal> method.</para>
<para>JMS connection factories can be created or destroyed using the <literal
>createConnectionFactory()</literal> methods or <literal
>destroyConnectionFactory()</literal> methods. These connection factories
@@ -312,14 +316,18 @@
for a list of the transport parameters)</para>
</listitem>
<listitem>
- <para>Creating/destroying queues</para>
+ <para>Listing, creating, destroying queues</para>
+ <para>Names of the deployed JMS queues can be retrieved by the
+ <literal>getQueueNames()</literal> method.</para>
<para>JMS queues can be created or destroyed using the <literal
>createQueue()</literal> methods or <literal>destroyQueue()</literal>
methods. These queues are bound to JNDI so that JMS clients can look them
up</para>
</listitem>
<listitem>
- <para>Creating/destroying topics</para>
+ <para>Listing, creating/destroying topics</para>
+ <para>Names of the deployed topics can be retrieved by the
+ <literal>getTopicNames()</literal> method.</para>
<para>JMS topics can be created or destroyed using the <literal
>createTopic()</literal> or <literal>destroyTopic()</literal> methods. These
topics are bound to JNDI so that JMS clients can look them up</para>
Modified: trunk/src/main/org/hornetq/core/management/HornetQServerControl.java
===================================================================
--- trunk/src/main/org/hornetq/core/management/HornetQServerControl.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/core/management/HornetQServerControl.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -116,7 +116,11 @@
Object[] getConnectors() throws Exception;
String getConnectorsAsJSON() throws Exception;
+
+ String[] getAddressNames();
+ String[] getQueueNames();
+
// Operations ----------------------------------------------------
@Operation(desc = "Create a queue with the specified address", impact = ACTION)
Modified: trunk/src/main/org/hornetq/core/management/ManagementService.java
===================================================================
--- trunk/src/main/org/hornetq/core/management/ManagementService.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/core/management/ManagementService.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -127,6 +127,7 @@
Object getResource(String resourceName);
+ Object[] getResources(Class<?> resourceType);
+
ServerMessage handleMessage(ServerMessage message) throws Exception;
-
}
Modified: trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/core/management/impl/HornetQServerControlImpl.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -36,8 +36,10 @@
import org.hornetq.core.config.TransportConfiguration;
import org.hornetq.core.exception.HornetQException;
import org.hornetq.core.logging.Logger;
+import org.hornetq.core.management.AddressControl;
import org.hornetq.core.management.HornetQServerControl;
import org.hornetq.core.management.NotificationType;
+import org.hornetq.core.management.QueueControl;
import org.hornetq.core.messagecounter.MessageCounterManager;
import org.hornetq.core.messagecounter.impl.MessageCounterManagerImpl;
import org.hornetq.core.postoffice.PostOffice;
@@ -252,6 +254,32 @@
server.createQueue(new SimpleString(address), new SimpleString(name), filter, durable, false);
}
+ public String[] getQueueNames()
+ {
+ Object[] queues = server.getManagementService().getResources(QueueControl.class);
+ String[] names = new String[queues.length];
+ for (int i = 0; i < queues.length; i++)
+ {
+ QueueControl queue = (QueueControl)queues[i];
+ names[i] = queue.getName();
+ }
+
+ return names;
+ }
+
+ public String[] getAddressNames()
+ {
+ Object[] addresses = server.getManagementService().getResources(AddressControl.class);
+ String[] names = new String[addresses.length];
+ for (int i = 0; i < addresses.length; i++)
+ {
+ AddressControl address = (AddressControl)addresses[i];
+ names[i] = address.getAddress();
+ }
+
+ return names;
+ }
+
public void destroyQueue(final String name) throws Exception
{
SimpleString queueName = new SimpleString(name);
Modified: trunk/src/main/org/hornetq/core/management/impl/ManagementServiceImpl.java
===================================================================
--- trunk/src/main/org/hornetq/core/management/impl/ManagementServiceImpl.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/core/management/impl/ManagementServiceImpl.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -21,6 +21,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.Map.Entry;
import java.util.concurrent.ScheduledExecutorService;
import javax.management.MBeanServer;
@@ -50,6 +51,7 @@
import org.hornetq.core.management.Notification;
import org.hornetq.core.management.NotificationListener;
import org.hornetq.core.management.ObjectNames;
+import org.hornetq.core.management.QueueControl;
import org.hornetq.core.management.ReplicationOperationInvoker;
import org.hornetq.core.management.ResourceNames;
import org.hornetq.core.management.jmx.impl.ReplicationAwareAddressControlWrapper;
@@ -255,14 +257,17 @@
address.toString(),
postOffice,
addressSettingsRepository);
- MessageCounter counter = new MessageCounter(queue.getName().toString(),
- null,
- queueControl,
- false,
- queue.isDurable(),
- messageCounterManager.getMaxDayCount());
- queueControl.setMessageCounter(counter);
- messageCounterManager.registerMessageCounter(queue.getName().toString(), counter);
+ if (messageCounterManager != null)
+ {
+ MessageCounter counter = new MessageCounter(queue.getName().toString(),
+ null,
+ queueControl,
+ false,
+ queue.isDurable(),
+ messageCounterManager.getMaxDayCount());
+ queueControl.setMessageCounter(counter);
+ messageCounterManager.registerMessageCounter(queue.getName().toString(), counter);
+ }
ObjectName objectName = ObjectNames.getQueueObjectName(address, queue.getName());
registerInJMX(objectName, new ReplicationAwareQueueControlWrapper(queueControl, replicationInvoker));
registerInRegistry(ResourceNames.CORE_QUEUE + queue.getName(), queueControl);
@@ -489,6 +494,19 @@
{
return registry.get(resourceName);
}
+
+ public Object[] getResources(Class<?> resourceType)
+ {
+ List<Object> resources = new ArrayList<Object>();
+ for (Object entry : registry.values())
+ {
+ if (resourceType.isAssignableFrom(entry.getClass()))
+ {
+ resources.add(entry);
+ }
+ }
+ return (Object[])resources.toArray(new Object[resources.size()]);
+ }
private Set<ObjectName> registeredNames = new HashSet<ObjectName>();
Modified: trunk/src/main/org/hornetq/core/management/jmx/impl/ReplicationAwareHornetQServerControlWrapper.java
===================================================================
--- trunk/src/main/org/hornetq/core/management/jmx/impl/ReplicationAwareHornetQServerControlWrapper.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/core/management/jmx/impl/ReplicationAwareHornetQServerControlWrapper.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -226,6 +226,16 @@
{
return localControl.getConnectorsAsJSON();
}
+
+ public String[] getAddressNames()
+ {
+ return localControl.getAddressNames();
+ }
+
+ public String[] getQueueNames()
+ {
+ return localControl.getQueueNames();
+ }
public void sendQueueInfoToQueue(final String queueName, final String address) throws Exception
{
Modified: trunk/src/main/org/hornetq/jms/server/JMSServerManager.java
===================================================================
--- trunk/src/main/org/hornetq/jms/server/JMSServerManager.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/jms/server/JMSServerManager.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -19,6 +19,7 @@
import org.hornetq.core.config.TransportConfiguration;
import org.hornetq.core.server.HornetQComponent;
+import org.hornetq.core.server.HornetQServer;
import org.hornetq.utils.Pair;
/**
@@ -220,4 +221,9 @@
String[] listSessions(String connectionID) throws Exception;
void setContext(final Context context);
+
+ /**
+ * @return
+ */
+ HornetQServer getHornetQServer();
}
Modified: trunk/src/main/org/hornetq/jms/server/impl/JMSServerManagerImpl.java
===================================================================
--- trunk/src/main/org/hornetq/jms/server/impl/JMSServerManagerImpl.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/jms/server/impl/JMSServerManagerImpl.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -195,6 +195,11 @@
// JMSServerManager implementation -------------------------------
+ public HornetQServer getHornetQServer()
+ {
+ return server;
+ }
+
public synchronized void setContext(final Context context)
{
this.context = context;
Modified: trunk/src/main/org/hornetq/jms/server/management/JMSServerControl.java
===================================================================
--- trunk/src/main/org/hornetq/jms/server/management/JMSServerControl.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/jms/server/management/JMSServerControl.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -36,6 +36,12 @@
String getVersion();
+ String[] getTopicNames();
+
+ String[] getQueueNames();
+
+ String[] getConnectionFactoryNames();
+
// Operations ----------------------------------------------------
@Operation(desc = "Create a JMS Queue", impact = ACTION)
Modified: trunk/src/main/org/hornetq/jms/server/management/impl/JMSServerControlImpl.java
===================================================================
--- trunk/src/main/org/hornetq/jms/server/management/impl/JMSServerControlImpl.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/jms/server/management/impl/JMSServerControlImpl.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -29,7 +29,10 @@
import org.hornetq.core.client.management.impl.ManagementHelper;
import org.hornetq.core.config.TransportConfiguration;
import org.hornetq.jms.server.JMSServerManager;
+import org.hornetq.jms.server.management.ConnectionFactoryControl;
+import org.hornetq.jms.server.management.JMSQueueControl;
import org.hornetq.jms.server.management.JMSServerControl;
+import org.hornetq.jms.server.management.TopicControl;
import org.hornetq.utils.Pair;
/**
@@ -669,6 +672,42 @@
return server.getVersion();
}
+ public String[] getQueueNames()
+ {
+ Object[] queueControls = server.getHornetQServer().getManagementService().getResources(JMSQueueControl.class);
+ String[] names = new String[queueControls.length];
+ for (int i = 0; i < queueControls.length; i++)
+ {
+ JMSQueueControl queueControl = (JMSQueueControl)queueControls[i];
+ names[i] = queueControl.getName();
+ }
+ return names;
+ }
+
+ public String[] getTopicNames()
+ {
+ Object[] topicControls = server.getHornetQServer().getManagementService().getResources(TopicControl.class);
+ String[] names = new String[topicControls.length];
+ for (int i = 0; i < topicControls.length; i++)
+ {
+ TopicControl topicControl = (TopicControl)topicControls[i];
+ names[i] = topicControl.getName();
+ }
+ return names;
+ }
+
+ public String[] getConnectionFactoryNames()
+ {
+ Object[] cfControls = server.getHornetQServer().getManagementService().getResources(ConnectionFactoryControl.class);
+ String[] names = new String[cfControls.length];
+ for (int i = 0; i < cfControls.length; i++)
+ {
+ ConnectionFactoryControl cfControl = (ConnectionFactoryControl)cfControls[i];
+ names[i] = cfControl.getName();
+ }
+ return names;
+ }
+
// NotificationEmitter implementation ----------------------------
public void removeNotificationListener(final NotificationListener listener,
Modified: trunk/src/main/org/hornetq/jms/server/management/jmx/impl/ReplicationAwareJMSServerControlWrapper.java
===================================================================
--- trunk/src/main/org/hornetq/jms/server/management/jmx/impl/ReplicationAwareJMSServerControlWrapper.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/src/main/org/hornetq/jms/server/management/jmx/impl/ReplicationAwareJMSServerControlWrapper.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -532,7 +532,22 @@
{
return localControl.isStarted();
}
+
+ public String[] getQueueNames()
+ {
+ return localControl.getQueueNames();
+ }
+
+ public String[] getTopicNames()
+ {
+ return localControl.getTopicNames();
+ }
+ public String[] getConnectionFactoryNames()
+ {
+ return localControl.getConnectionFactoryNames();
+ }
+
public String[] listConnectionIDs() throws Exception
{
return localControl.listConnectionIDs();
Modified: trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlTest.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlTest.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -128,7 +128,26 @@
checkNoBinding(context, queueJNDIBinding);
checkNoResource(ObjectNames.getJMSQueueObjectName(queueName));
}
+
+ public void testGetQueueNames() throws Exception
+ {
+ String queueJNDIBinding = randomString();
+ String queueName = randomString();
+ JMSServerControl control = createManagementControl();
+ assertEquals(0, control.getQueueNames().length);
+
+ control.createQueue(queueName, queueJNDIBinding);
+
+ String[] names = control.getQueueNames();
+ assertEquals(1, names.length);
+ assertEquals(queueName, names[0]);
+
+ control.destroyQueue(queueName);
+
+ assertEquals(0, control.getQueueNames().length);
+ }
+
public void testCreateTopic() throws Exception
{
String topicJNDIBinding = randomString();
@@ -166,7 +185,26 @@
checkNoBinding(context, topicJNDIBinding);
checkNoResource(ObjectNames.getJMSTopicObjectName(topicName));
}
+
+ public void testGetTopicNames() throws Exception
+ {
+ String topicJNDIBinding = randomString();
+ String topicName = randomString();
+ JMSServerControl control = createManagementControl();
+ assertEquals(0, control.getTopicNames().length);
+
+ control.createTopic(topicName, topicJNDIBinding);
+
+ String[] names = control.getTopicNames();
+ assertEquals(1, names.length);
+ assertEquals(topicName, names[0]);
+
+ control.destroyTopic(topicName);
+
+ assertEquals(0, control.getTopicNames().length);
+ }
+
public void testCreateConnectionFactory_1() throws Exception
{
doCreateConnectionFactory(new ConnectionFactoryCreator()
@@ -528,7 +566,26 @@
}
}
+
+ public void testGetConnectionFactoryNames() throws Exception
+ {
+ String cfBinding = randomString();
+ String cfName = randomString();
+ JMSServerControl control = createManagementControl();
+ assertEquals(0, control.getConnectionFactoryNames().length);
+
+ TransportConfiguration tcLive = new TransportConfiguration(InVMConnectorFactory.class.getName());
+ control.createConnectionFactory(cfName, tcLive.getFactoryClassName(), tcLive.getParams(), new String[] {cfBinding});
+
+ String[] cfNames = control.getConnectionFactoryNames();
+ assertEquals(1, cfNames.length);
+ assertEquals(cfName, cfNames[0]);
+
+ control.destroyConnectionFactory(cfName);
+ assertEquals(0, control.getConnectionFactoryNames().length);
+ }
+
// Package protected ---------------------------------------------
// Protected -----------------------------------------------------
Modified: trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlUsingJMSTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlUsingJMSTest.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/tests/src/org/hornetq/tests/integration/jms/server/management/JMSServerControlUsingJMSTest.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -46,6 +46,16 @@
// Static --------------------------------------------------------
+ private static String[] toStringArray(Object[] res)
+ {
+ String[] names = new String[res.length];
+ for (int i = 0; i < res.length; i++)
+ {
+ names[i] = res[i].toString();
+ }
+ return names;
+ }
+
// Constructors --------------------------------------------------
// JMSServerControlTest overrides --------------------------------
@@ -575,7 +585,22 @@
{
return (Boolean)proxy.retrieveAttributeValue("started");
}
+
+ public String[] getQueueNames()
+ {
+ return toStringArray((Object[])proxy.retrieveAttributeValue("queueNames"));
+ }
+ public String[] getTopicNames()
+ {
+ return toStringArray((Object[])proxy.retrieveAttributeValue("topicNames"));
+ }
+
+ public String[] getConnectionFactoryNames()
+ {
+ return toStringArray((Object[])proxy.retrieveAttributeValue("connectionFactoryNames"));
+ }
+
public String[] listConnectionIDs() throws Exception
{
return (String[])proxy.invokeOperation("listConnectionIDs");
@@ -595,7 +620,7 @@
{
return (String[])proxy.invokeOperation("listSessions", connectionID);
}
-
+
};
}
// Public --------------------------------------------------------
Modified: trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlTest.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlTest.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -59,6 +59,20 @@
// Static --------------------------------------------------------
+ private static boolean contains(String name, String[] strings)
+ {
+ boolean found = false;
+ for (String str : strings)
+ {
+ if (name.equals(str))
+ {
+ found = true;
+ break;
+ }
+ }
+ return found;
+ }
+
// Constructors --------------------------------------------------
// Public --------------------------------------------------------
@@ -209,7 +223,45 @@
checkNoResource(ObjectNames.getQueueObjectName(address, name));
}
+
+ public void testGetQueueNames() throws Exception
+ {
+ SimpleString address = RandomUtil.randomSimpleString();
+ SimpleString name = RandomUtil.randomSimpleString();
+ HornetQServerControl serverControl = createManagementControl();
+
+ // due to replication, there can be another queue created for replicating
+ // management operations
+
+ assertFalse(contains(name.toString(), serverControl.getQueueNames()));
+
+ serverControl.createQueue(address.toString(), name.toString());
+ assertTrue(contains(name.toString(), serverControl.getQueueNames()));
+
+ serverControl.destroyQueue(name.toString());
+ assertFalse(contains(name.toString(), serverControl.getQueueNames()));
+ }
+
+ public void testGetAddressNames() throws Exception
+ {
+ SimpleString address = RandomUtil.randomSimpleString();
+ SimpleString name = RandomUtil.randomSimpleString();
+
+ HornetQServerControl serverControl = createManagementControl();
+
+ // due to replication, there can be another queue created for replicating
+ // management operations
+
+ assertFalse(contains(address.toString(), serverControl.getAddressNames()));
+
+ serverControl.createQueue(address.toString(), name.toString());
+ assertTrue(contains(address.toString(), serverControl.getAddressNames()));
+
+ serverControl.destroyQueue(name.toString());
+ assertFalse(contains(address.toString(), serverControl.getAddressNames()));
+ }
+
public void testMessageCounterMaxDayCount() throws Exception
{
HornetQServerControl serverControl = createManagementControl();
Modified: trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/tests/src/org/hornetq/tests/integration/management/HornetQServerControlUsingCoreTest.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -36,6 +36,16 @@
// Static --------------------------------------------------------
+ private static String[] toStringArray(Object[] res)
+ {
+ String[] names = new String[res.length];
+ for (int i = 0; i < res.length; i++)
+ {
+ names[i] = res[i].toString();
+ }
+ return names;
+ }
+
// Constructors --------------------------------------------------
// Public --------------------------------------------------------
@@ -68,7 +78,6 @@
@Override
protected HornetQServerControl createManagementControl() throws Exception
{
-
return new HornetQServerControl()
{
private final CoreMessagingProxy proxy = new CoreMessagingProxy(session,
@@ -153,6 +162,16 @@
{
return (String)proxy.retrieveAttributeValue("connectorsAsJSON");
}
+
+ public String[] getAddressNames()
+ {
+ return toStringArray((Object[])proxy.retrieveAttributeValue("addressNames"));
+ }
+
+ public String[] getQueueNames()
+ {
+ return toStringArray((Object[])proxy.retrieveAttributeValue("queueNames"));
+ }
public int getIDCacheSize()
{
@@ -161,13 +180,7 @@
public String[] getInterceptorClassNames()
{
- Object[] res = (Object[])proxy.retrieveAttributeValue("interceptorClassNames");
- String[] names = new String[res.length];
- for (int i = 0; i < res.length; i++)
- {
- names[i] = res[i].toString();
- }
- return names;
+ return toStringArray((Object[])proxy.retrieveAttributeValue("interceptorClassNames"));
}
public String getJournalDirectory()
Modified: trunk/tests/src/org/hornetq/tests/integration/management/ManagementServiceImplTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/integration/management/ManagementServiceImplTest.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/tests/src/org/hornetq/tests/integration/management/ManagementServiceImplTest.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -13,6 +13,7 @@
package org.hornetq.tests.integration.management;
+import static org.hornetq.tests.util.RandomUtil.randomSimpleString;
import static org.hornetq.tests.util.RandomUtil.randomString;
import org.hornetq.core.buffers.ChannelBuffers;
@@ -20,13 +21,20 @@
import org.hornetq.core.config.Configuration;
import org.hornetq.core.config.impl.ConfigurationImpl;
import org.hornetq.core.logging.Logger;
+import org.hornetq.core.management.AddressControl;
+import org.hornetq.core.management.QueueControl;
import org.hornetq.core.management.ResourceNames;
+import org.hornetq.core.management.impl.ManagementServiceImpl;
import org.hornetq.core.remoting.spi.HornetQBuffer;
import org.hornetq.core.server.HornetQ;
import org.hornetq.core.server.HornetQServer;
+import org.hornetq.core.server.Queue;
import org.hornetq.core.server.ServerMessage;
import org.hornetq.core.server.impl.ServerMessageImpl;
+import org.hornetq.tests.integration.server.FakeStorageManager;
+import org.hornetq.tests.unit.core.postoffice.impl.FakeQueue;
import org.hornetq.tests.util.UnitTestCase;
+import org.hornetq.utils.SimpleString;
/*
* @author <a href="mailto:jmesnil@redhat.com">Jeff Mesnil</a>
@@ -144,6 +152,31 @@
assertNotNull(ManagementHelper.getResult(reply));
server.stop();
}
+
+ public void testGetResources() throws Exception
+ {
+ Configuration conf = new ConfigurationImpl();
+ conf.setJMXManagementEnabled(false);
+ ManagementServiceImpl managementService = new ManagementServiceImpl(null, conf, -1);
+
+ SimpleString address = randomSimpleString();
+ managementService.registerAddress(address);
+ Queue queue = new FakeQueue(randomSimpleString());
+ managementService.registerQueue(queue, randomSimpleString(), new FakeStorageManager());
+
+ Object[] addresses = managementService.getResources(AddressControl.class);
+ assertEquals(1, addresses.length);
+ assertTrue(addresses[0] instanceof AddressControl);
+ AddressControl addressControl = (AddressControl)addresses[0];
+ assertEquals(address.toString(), addressControl.getAddress());
+
+ Object[] queues = managementService.getResources(QueueControl.class);
+ assertEquals(1, queues.length);
+ assertTrue(queues[0] instanceof QueueControl);
+ QueueControl queueControl = (QueueControl)queues[0];
+ assertEquals(queue.getName().toString(), queueControl.getName());
+ }
+
// Package protected ---------------------------------------------
// Protected -----------------------------------------------------
Modified: trunk/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/tests/src/org/hornetq/tests/unit/core/deployers/impl/QueueDeployerTest.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -157,6 +157,16 @@
}
+ public String[] getAddressNames()
+ {
+ return null;
+ }
+
+ public String[] getQueueNames()
+ {
+ return null;
+ }
+
List<QueueConfiguration> configs = new ArrayList<QueueConfiguration>();
public void deployQueue(String address, String name, String filter, boolean durable) throws Exception
Modified: trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/BindingImplTest.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/BindingImplTest.java 2009-09-11 14:22:36 UTC (rev 7955)
+++ trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/BindingImplTest.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -15,11 +15,8 @@
import java.io.InputStream;
import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.Executor;
import javax.transaction.xa.Xid;
@@ -28,11 +25,8 @@
import org.hornetq.core.postoffice.Binding;
import org.hornetq.core.postoffice.BindingType;
import org.hornetq.core.postoffice.impl.BindingsImpl;
-import org.hornetq.core.remoting.Channel;
import org.hornetq.core.remoting.spi.HornetQBuffer;
import org.hornetq.core.server.Bindable;
-import org.hornetq.core.server.Consumer;
-import org.hornetq.core.server.Distributor;
import org.hornetq.core.server.MessageReference;
import org.hornetq.core.server.Queue;
import org.hornetq.core.server.ServerMessage;
@@ -1014,497 +1008,6 @@
}
- class FakeQueue implements Queue
- {
- private SimpleString name;
-
- FakeQueue(SimpleString name)
- {
- this.name = name;
- }
-
- public void setExpiryAddress(SimpleString expiryAddress)
- {
- // TODO Auto-generated method stub
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#acknowledge(org.hornetq.core.server.MessageReference)
- */
- public void acknowledge(MessageReference ref) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#acknowledge(org.hornetq.core.transaction.Transaction, org.hornetq.core.server.MessageReference)
- */
- public void acknowledge(Transaction tx, MessageReference ref) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#activate()
- */
- public boolean activate()
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#activateNow(java.util.concurrent.Executor)
- */
- public void activateNow(Executor executor)
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#addConsumer(org.hornetq.core.server.Consumer)
- */
- public void addConsumer(Consumer consumer) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#addFirst(org.hornetq.core.server.MessageReference)
- */
- public void addFirst(MessageReference ref)
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#addLast(org.hornetq.core.server.MessageReference)
- */
- public void addLast(MessageReference ref)
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#addRedistributor(long, java.util.concurrent.Executor, org.hornetq.core.remoting.Channel)
- */
- public void addRedistributor(long delay, Executor executor, Channel replicatingChannel)
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#cancel(org.hornetq.core.transaction.Transaction, org.hornetq.core.server.MessageReference)
- */
- public void cancel(Transaction tx, MessageReference ref) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#cancel(org.hornetq.core.server.MessageReference)
- */
- public void cancel(MessageReference reference) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#cancelRedistributor()
- */
- public void cancelRedistributor() throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#changeReferencePriority(long, byte)
- */
- public boolean changeReferencePriority(long messageID, byte newPriority) throws Exception
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#checkDLQ(org.hornetq.core.server.MessageReference)
- */
- public boolean checkDLQ(MessageReference ref) throws Exception
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#consumerFailedOver()
- */
- public boolean consumerFailedOver()
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#deleteAllReferences()
- */
- public int deleteAllReferences() throws Exception
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#deleteMatchingReferences(org.hornetq.core.filter.Filter)
- */
- public int deleteMatchingReferences(Filter filter) throws Exception
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#deleteReference(long)
- */
- public boolean deleteReference(long messageID) throws Exception
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#deliverAsync(java.util.concurrent.Executor)
- */
- public void deliverAsync(Executor executor)
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#deliverNow()
- */
- public void deliverNow()
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#expire(org.hornetq.core.server.MessageReference)
- */
- public void expire(MessageReference ref) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#expireReference(long)
- */
- public boolean expireReference(long messageID) throws Exception
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#expireReferences(org.hornetq.core.filter.Filter)
- */
- public int expireReferences(Filter filter) throws Exception
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#expireReferences()
- */
- public void expireReferences() throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getConsumerCount()
- */
- public int getConsumerCount()
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getConsumers()
- */
- public Set<Consumer> getConsumers()
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getDeliveringCount()
- */
- public int getDeliveringCount()
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getDistributionPolicy()
- */
- public Distributor getDistributionPolicy()
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getFilter()
- */
- public Filter getFilter()
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getMessageCount()
- */
- public int getMessageCount()
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getMessagesAdded()
- */
- public int getMessagesAdded()
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getName()
- */
- public SimpleString getName()
- {
- return name;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getPersistenceID()
- */
- public long getPersistenceID()
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getReference(long)
- */
- public MessageReference getReference(long id)
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getScheduledCount()
- */
- public int getScheduledCount()
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#getScheduledMessages()
- */
- public List<MessageReference> getScheduledMessages()
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#isBackup()
- */
- public boolean isBackup()
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#isDurable()
- */
- public boolean isDurable()
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#isTemporary()
- */
- public boolean isTemporary()
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#list(org.hornetq.core.filter.Filter)
- */
- public List<MessageReference> list(Filter filter)
- {
-
- return null;
- }
-
- public Iterator<MessageReference> iterator()
- {
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#moveReference(long, org.hornetq.utils.SimpleString)
- */
- public boolean moveReference(long messageID, SimpleString toAddress) throws Exception
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#moveReferences(org.hornetq.core.filter.Filter, org.hornetq.utils.SimpleString)
- */
- public int moveReferences(Filter filter, SimpleString toAddress) throws Exception
- {
-
- return 0;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#reacknowledge(org.hornetq.core.transaction.Transaction, org.hornetq.core.server.MessageReference)
- */
- public void reacknowledge(Transaction tx, MessageReference ref) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#referenceHandled()
- */
- public void referenceHandled()
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#removeConsumer(org.hornetq.core.server.Consumer)
- */
- public boolean removeConsumer(Consumer consumer) throws Exception
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#removeFirstReference(long)
- */
- public MessageReference removeFirstReference(long id) throws Exception
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#removeReferenceWithID(long)
- */
- public MessageReference removeReferenceWithID(long id) throws Exception
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#reroute(org.hornetq.core.server.ServerMessage, org.hornetq.core.transaction.Transaction)
- */
- public MessageReference reroute(ServerMessage message, Transaction tx) throws Exception
- {
-
- return null;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#sendMessageToDeadLetterAddress(long)
- */
- public boolean sendMessageToDeadLetterAddress(long messageID) throws Exception
- {
-
- return false;
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#setBackup()
- */
- public void setBackup()
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#setDistributionPolicy(org.hornetq.core.server.Distributor)
- */
- public void setDistributionPolicy(Distributor policy)
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#setPersistenceID(long)
- */
- public void setPersistenceID(long id)
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Bindable#preroute(org.hornetq.core.server.ServerMessage, org.hornetq.core.transaction.Transaction)
- */
- public void preroute(ServerMessage message, Transaction tx) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Bindable#route(org.hornetq.core.server.ServerMessage, org.hornetq.core.transaction.Transaction)
- */
- public void route(ServerMessage message, Transaction tx) throws Exception
- {
-
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#lock()
- */
- public void lockDelivery()
- {
- }
-
- /* (non-Javadoc)
- * @see org.hornetq.core.server.Queue#unlock()
- */
- public void unlockDelivery()
- {
- }
-
- }
-
// Package protected ---------------------------------------------
// Protected -----------------------------------------------------
Added: trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/FakeQueue.java
===================================================================
--- trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/FakeQueue.java (rev 0)
+++ trunk/tests/src/org/hornetq/tests/unit/core/postoffice/impl/FakeQueue.java 2009-09-15 09:23:07 UTC (rev 7956)
@@ -0,0 +1,520 @@
+/*
+ * Copyright 2009 Red Hat, Inc.
+ * Red Hat 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.hornetq.tests.unit.core.postoffice.impl;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Executor;
+
+import org.hornetq.core.filter.Filter;
+import org.hornetq.core.remoting.Channel;
+import org.hornetq.core.server.Consumer;
+import org.hornetq.core.server.Distributor;
+import org.hornetq.core.server.MessageReference;
+import org.hornetq.core.server.Queue;
+import org.hornetq.core.server.ServerMessage;
+import org.hornetq.core.transaction.Transaction;
+import org.hornetq.utils.SimpleString;
+
+public class FakeQueue implements Queue
+{
+ private SimpleString name;
+
+ public FakeQueue(SimpleString name)
+ {
+ this.name = name;
+ }
+
+ public void setExpiryAddress(SimpleString expiryAddress)
+ {
+ // TODO Auto-generated method stub
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#acknowledge(org.hornetq.core.server.MessageReference)
+ */
+ public void acknowledge(MessageReference ref) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#acknowledge(org.hornetq.core.transaction.Transaction, org.hornetq.core.server.MessageReference)
+ */
+ public void acknowledge(Transaction tx, MessageReference ref) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#activate()
+ */
+ public boolean activate()
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#activateNow(java.util.concurrent.Executor)
+ */
+ public void activateNow(Executor executor)
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#addConsumer(org.hornetq.core.server.Consumer)
+ */
+ public void addConsumer(Consumer consumer) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#addFirst(org.hornetq.core.server.MessageReference)
+ */
+ public void addFirst(MessageReference ref)
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#addLast(org.hornetq.core.server.MessageReference)
+ */
+ public void addLast(MessageReference ref)
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#addRedistributor(long, java.util.concurrent.Executor, org.hornetq.core.remoting.Channel)
+ */
+ public void addRedistributor(long delay, Executor executor, Channel replicatingChannel)
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#cancel(org.hornetq.core.transaction.Transaction, org.hornetq.core.server.MessageReference)
+ */
+ public void cancel(Transaction tx, MessageReference ref) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#cancel(org.hornetq.core.server.MessageReference)
+ */
+ public void cancel(MessageReference reference) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#cancelRedistributor()
+ */
+ public void cancelRedistributor() throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#changeReferencePriority(long, byte)
+ */
+ public boolean changeReferencePriority(long messageID, byte newPriority) throws Exception
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#checkDLQ(org.hornetq.core.server.MessageReference)
+ */
+ public boolean checkDLQ(MessageReference ref) throws Exception
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#consumerFailedOver()
+ */
+ public boolean consumerFailedOver()
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#deleteAllReferences()
+ */
+ public int deleteAllReferences() throws Exception
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#deleteMatchingReferences(org.hornetq.core.filter.Filter)
+ */
+ public int deleteMatchingReferences(Filter filter) throws Exception
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#deleteReference(long)
+ */
+ public boolean deleteReference(long messageID) throws Exception
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#deliverAsync(java.util.concurrent.Executor)
+ */
+ public void deliverAsync(Executor executor)
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#deliverNow()
+ */
+ public void deliverNow()
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#expire(org.hornetq.core.server.MessageReference)
+ */
+ public void expire(MessageReference ref) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#expireReference(long)
+ */
+ public boolean expireReference(long messageID) throws Exception
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#expireReferences(org.hornetq.core.filter.Filter)
+ */
+ public int expireReferences(Filter filter) throws Exception
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#expireReferences()
+ */
+ public void expireReferences() throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getConsumerCount()
+ */
+ public int getConsumerCount()
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getConsumers()
+ */
+ public Set<Consumer> getConsumers()
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getDeliveringCount()
+ */
+ public int getDeliveringCount()
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getDistributionPolicy()
+ */
+ public Distributor getDistributionPolicy()
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getFilter()
+ */
+ public Filter getFilter()
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getMessageCount()
+ */
+ public int getMessageCount()
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getMessagesAdded()
+ */
+ public int getMessagesAdded()
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getName()
+ */
+ public SimpleString getName()
+ {
+ return name;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getPersistenceID()
+ */
+ public long getPersistenceID()
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getReference(long)
+ */
+ public MessageReference getReference(long id)
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getScheduledCount()
+ */
+ public int getScheduledCount()
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#getScheduledMessages()
+ */
+ public List<MessageReference> getScheduledMessages()
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#isBackup()
+ */
+ public boolean isBackup()
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#isDurable()
+ */
+ public boolean isDurable()
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#isTemporary()
+ */
+ public boolean isTemporary()
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#list(org.hornetq.core.filter.Filter)
+ */
+ public List<MessageReference> list(Filter filter)
+ {
+
+ return null;
+ }
+
+ public Iterator<MessageReference> iterator()
+ {
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#moveReference(long, org.hornetq.utils.SimpleString)
+ */
+ public boolean moveReference(long messageID, SimpleString toAddress) throws Exception
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#moveReferences(org.hornetq.core.filter.Filter, org.hornetq.utils.SimpleString)
+ */
+ public int moveReferences(Filter filter, SimpleString toAddress) throws Exception
+ {
+
+ return 0;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#reacknowledge(org.hornetq.core.transaction.Transaction, org.hornetq.core.server.MessageReference)
+ */
+ public void reacknowledge(Transaction tx, MessageReference ref) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#referenceHandled()
+ */
+ public void referenceHandled()
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#removeConsumer(org.hornetq.core.server.Consumer)
+ */
+ public boolean removeConsumer(Consumer consumer) throws Exception
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#removeFirstReference(long)
+ */
+ public MessageReference removeFirstReference(long id) throws Exception
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#removeReferenceWithID(long)
+ */
+ public MessageReference removeReferenceWithID(long id) throws Exception
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#reroute(org.hornetq.core.server.ServerMessage, org.hornetq.core.transaction.Transaction)
+ */
+ public MessageReference reroute(ServerMessage message, Transaction tx) throws Exception
+ {
+
+ return null;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#sendMessageToDeadLetterAddress(long)
+ */
+ public boolean sendMessageToDeadLetterAddress(long messageID) throws Exception
+ {
+
+ return false;
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#setBackup()
+ */
+ public void setBackup()
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#setDistributionPolicy(org.hornetq.core.server.Distributor)
+ */
+ public void setDistributionPolicy(Distributor policy)
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#setPersistenceID(long)
+ */
+ public void setPersistenceID(long id)
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Bindable#preroute(org.hornetq.core.server.ServerMessage, org.hornetq.core.transaction.Transaction)
+ */
+ public void preroute(ServerMessage message, Transaction tx) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Bindable#route(org.hornetq.core.server.ServerMessage, org.hornetq.core.transaction.Transaction)
+ */
+ public void route(ServerMessage message, Transaction tx) throws Exception
+ {
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#lock()
+ */
+ public void lockDelivery()
+ {
+ }
+
+ /* (non-Javadoc)
+ * @see org.hornetq.core.server.Queue#unlock()
+ */
+ public void unlockDelivery()
+ {
+ }
+
+}
\ No newline at end of file
16 years, 3 months