[jboss-cvs] JBoss Messaging SVN: r5405 - in trunk: src/main/org/jboss/messaging/core/paging and 7 other directories.
jboss-cvs-commits at lists.jboss.org
jboss-cvs-commits at lists.jboss.org
Thu Nov 20 20:25:21 EST 2008
Author: clebert.suconic at jboss.com
Date: 2008-11-20 20:25:21 -0500 (Thu, 20 Nov 2008)
New Revision: 5405
Modified:
trunk/src/main/org/jboss/messaging/core/config/impl/ConfigurationImpl.java
trunk/src/main/org/jboss/messaging/core/paging/PageTransactionInfo.java
trunk/src/main/org/jboss/messaging/core/paging/PagingManager.java
trunk/src/main/org/jboss/messaging/core/paging/PagingStore.java
trunk/src/main/org/jboss/messaging/core/paging/impl/PageTransactionInfoImpl.java
trunk/src/main/org/jboss/messaging/core/paging/impl/PagingManagerImpl.java
trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreFactoryNIO.java
trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreImpl.java
trunk/src/main/org/jboss/messaging/core/persistence/impl/journal/JournalStorageManager.java
trunk/src/main/org/jboss/messaging/core/server/impl/MessagingServerImpl.java
trunk/src/main/org/jboss/messaging/core/server/impl/ServerSessionImpl.java
trunk/src/main/org/jboss/messaging/core/transaction/impl/TransactionImpl.java
trunk/tests/src/org/jboss/messaging/tests/integration/paging/PagingManagerIntegrationTest.java
trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreImplTest.java
trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreTestBase.java
trunk/tests/src/org/jboss/messaging/tests/util/ServiceTestBase.java
Log:
JBMESSAGING-1458 - tweaks on paging after code review with Tim
Modified: trunk/src/main/org/jboss/messaging/core/config/impl/ConfigurationImpl.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/config/impl/ConfigurationImpl.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/config/impl/ConfigurationImpl.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -61,9 +61,9 @@
public static final String DEFAULT_PAGING_DIR = "data/paging";
- public static final long DEFAULT_DEFAULT_PAGE_SIZE = 10 * 1024 * 1024;
+ public static final long DEFAULT_PAGE_SIZE = 10 * 1024 * 1024;
- public static final String DEFAULT_LARGEMESSAGES_DIR = "data/largemessages";
+ public static final String DEFAULT_LARGE_MESSAGES_DIR = "data/largemessages";
public static final boolean DEFAULT_CREATE_JOURNAL_DIR = true;
@@ -137,14 +137,14 @@
protected long pagingMaxGlobalSize = -1;
- protected long pagingDefaultSize = DEFAULT_DEFAULT_PAGE_SIZE;
+ protected long pagingDefaultSize = DEFAULT_PAGE_SIZE;
protected String pagingDirectory = DEFAULT_PAGING_DIR;
// File related attributes -----------------------------------------------------------
- protected String largeMessagesDirectory = DEFAULT_LARGEMESSAGES_DIR;
+ protected String largeMessagesDirectory = DEFAULT_LARGE_MESSAGES_DIR;
protected String bindingsDirectory = DEFAULT_BINDINGS_DIRECTORY;
Modified: trunk/src/main/org/jboss/messaging/core/paging/PageTransactionInfo.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/paging/PageTransactionInfo.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/paging/PageTransactionInfo.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -35,9 +35,9 @@
{
boolean waitCompletion() throws Exception;
- void complete();
+ void commit();
- void forget();
+ void rollback();
long getRecordID();
Modified: trunk/src/main/org/jboss/messaging/core/paging/PagingManager.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/paging/PagingManager.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/paging/PagingManager.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -113,7 +113,7 @@
*
* Duplication detection for paging processing
* */
- void setLastPage(LastPageRecord lastPage) throws Exception;
+ void setLastPageRecord(LastPageRecord lastPage) throws Exception;
/**
*
Modified: trunk/src/main/org/jboss/messaging/core/paging/PagingStore.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/paging/PagingStore.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/paging/PagingStore.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -46,9 +46,9 @@
/** Maximum number of bytes allowed in memory */
long getMaxSizeBytes();
- boolean isDroppedMessage();
+ boolean isPrintedDropMessagesWarning();
- void setDroppedMessage(boolean droppedMessages);
+ void setPrintedDropMessagesWarning(boolean droppedMessages);
boolean isDropWhenMaxSize();
@@ -85,7 +85,7 @@
*/
boolean startDepaging();
- LastPageRecord getLastRecord();
+ LastPageRecord getLastPageRecord();
- void setLastRecord(LastPageRecord record);
+ void setLastPageRecord(LastPageRecord record);
}
Modified: trunk/src/main/org/jboss/messaging/core/paging/impl/PageTransactionInfoImpl.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/paging/impl/PageTransactionInfoImpl.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/paging/impl/PageTransactionInfoImpl.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -128,7 +128,7 @@
return SIZE_LONG + SIZE_INT;
}
- public void complete()
+ public void commit()
{
complete = true;
/**
@@ -150,7 +150,7 @@
return complete;
}
- public void forget()
+ public void rollback()
{
complete = false;
Modified: trunk/src/main/org/jboss/messaging/core/paging/impl/PagingManagerImpl.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/paging/impl/PagingManagerImpl.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/paging/impl/PagingManagerImpl.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -194,6 +194,8 @@
*
* If persistent messages are also used, it will update eventual PageTransactions
*/
+
+ //TODO - this method should be moved to PagingStoreImpl
public boolean onDepage(final int pageId,
final SimpleString destination,
final PagingStore pagingStore,
@@ -205,12 +207,13 @@
// back to where it was
final long depageTransactionID = storageManager.generateUniqueID();
- LastPageRecord lastPage = pagingStore.getLastRecord();
+ LastPageRecord lastPage = pagingStore.getLastPageRecord();
if (lastPage == null)
{
lastPage = new LastPageRecordImpl(pageId, destination);
- pagingStore.setLastRecord(lastPage);
+
+ pagingStore.setLastPageRecord(lastPage);
}
else
{
@@ -321,11 +324,11 @@
return defaultPageSize;
}
- public void setLastPage(final LastPageRecord lastPage) throws Exception
+ public void setLastPageRecord(final LastPageRecord lastPage) throws Exception
{
trace("LastPage loaded was " + lastPage.getLastId() + " recordID = " + lastPage.getRecordId());
- getPageStore(lastPage.getDestination()).setLastRecord(lastPage);
+ getPageStore(lastPage.getDestination()).setLastPageRecord(lastPage);
}
public boolean isPaging(final SimpleString destination) throws Exception
@@ -381,6 +384,8 @@
return;
}
+ pagingSPI.setPagingManager(this);
+
started = true;
}
@@ -412,6 +417,7 @@
return pagingSPI.newStore(destinationName, queueSettingsRepository.getMatch(destinationName.toString()));
}
+ //TODO - this method should be moved to PagingStoreImpl
private long addSize(final SimpleString destination, final long size) throws Exception
{
final PagingStore store = getPageStore(destination);
@@ -426,9 +432,9 @@
// limit, we return -1 which means drop the message
if (store.getAddressSize() + size > maxSize || maxGlobalSize > 0 && globalSize.get() + size > maxGlobalSize)
{
- if (!store.isDroppedMessage())
+ if (!store.isPrintedDropMessagesWarning())
{
- store.setDroppedMessage(true);
+ store.setPrintedDropMessagesWarning(true);
log.warn("Messages are being dropped on adress " + store.getStoreName());
}
@@ -451,6 +457,7 @@
if (maxGlobalSize > 0 && currentGlobalSize > maxGlobalSize)
{
globalMode.set(true);
+
if (store.startPaging())
{
if (isTrace)
Modified: trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreFactoryNIO.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreFactoryNIO.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreFactoryNIO.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -62,7 +62,7 @@
public PagingStoreFactoryNIO(final String directory)
{
this.directory = directory;
-
+
executor = Executors.newCachedThreadPool(new JBMThreadFactory("JBM-depaging-threads"));
}
@@ -72,18 +72,22 @@
{
return executor;
}
-
+
public void stop() throws InterruptedException
{
executor.shutdown();
+
executor.awaitTermination(30, TimeUnit.SECONDS);
}
public PagingStore newStore(final SimpleString destinationName, final QueueSettings settings)
{
- // FIXME: This directory creation should be done inside PagingStoreImpl, or this method should be made synchornized
+ // FIXME: This directory creation should be done inside PagingStoreImpl::start, or this method should be made
+ // synchornized
final String destinationDirectory = directory + "/" + destinationName.toString();
+
File destinationFile = new File(destinationDirectory);
+
destinationFile.mkdirs();
return new PagingStoreImpl(pagingManager,
Modified: trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreImpl.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreImpl.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/paging/impl/PagingStoreImpl.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -32,7 +32,6 @@
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.jboss.messaging.core.config.impl.ConfigurationImpl;
import org.jboss.messaging.core.journal.SequentialFile;
import org.jboss.messaging.core.journal.SequentialFileFactory;
import org.jboss.messaging.core.logging.Logger;
@@ -70,9 +69,9 @@
private final long pageSize;
- private final boolean dropMessagesOnSize;
+ private final boolean dropMessagesWhenFull;
- private boolean droppedMessages;
+ private boolean printedDropMessagesWarning;
private final PagingManager pagingManager;
@@ -85,7 +84,7 @@
private volatile int numberOfPages;
- private volatile int firstPageId = Integer.MAX_VALUE;
+ private volatile int firstPageId;
private volatile int currentPageId;
@@ -111,23 +110,30 @@
final QueueSettings queueSettings,
final ExecutorService executor)
{
+ if (pagingManager == null)
+ {
+ throw new IllegalStateException("Paging Manager can't be null");
+ }
+
this.fileFactory = fileFactory;
+
this.storeName = storeName;
+
maxSize = queueSettings.getMaxSizeBytes();
+
if (queueSettings.getPageSizeBytes() != null)
{
this.pageSize = queueSettings.getPageSizeBytes();
- }
- else if (pagingManager != null)
+ }
+ else
{
this.pageSize = pagingManager.getDefaultPageSize();
}
- else
- {
- this.pageSize = ConfigurationImpl.DEFAULT_DEFAULT_PAGE_SIZE;
- }
- dropMessagesOnSize = queueSettings.isDropMessagesWhenFull();
+
+ dropMessagesWhenFull = queueSettings.isDropMessagesWhenFull();
+
this.executor = executor;
+
this.pagingManager = pagingManager;
}
@@ -135,14 +141,16 @@
// PagingStore implementation ------------------------------------
- public boolean isDroppedMessage()
+ //TODO - this methods shouldn't be necessary if move functionality from
+ //PagingManagerImpl to PagingStoreImpl
+ public boolean isPrintedDropMessagesWarning()
{
- return droppedMessages;
+ return printedDropMessagesWarning;
}
- public void setDroppedMessage(final boolean droppedMessages)
+ public void setPrintedDropMessagesWarning(final boolean droppedMessages)
{
- this.droppedMessages = droppedMessages;
+ this.printedDropMessagesWarning = droppedMessages;
}
public long getAddressSize()
@@ -163,7 +171,7 @@
public boolean isDropWhenMaxSize()
{
- return dropMessagesOnSize;
+ return dropMessagesWhenFull;
}
public long getPageSizeBytes()
@@ -199,21 +207,29 @@
* @return
* @throws Exception
*/
+ //FIXME - why is this public?
public boolean readPage() throws Exception
{
Page page = depage();
+
if (page == null)
{
if (lastPageRecord != null)
{
pagingManager.clearLastPageRecord(lastPageRecord);
}
+
lastPageRecord = null;
+
return false;
}
+
page.open();
+
PagedMessage messages[] = page.read();
+
boolean addressNotFull = pagingManager.onDepage(page.getPageId(), storeName, PagingStoreImpl.this, messages);
+
page.delete();
return addressNotFull;
@@ -224,6 +240,7 @@
* The method calling this method will remove the page and will start reading it outside of any locks.
*
* */
+ //FIXME - why is this public?
public Page depage() throws Exception
{
positioningGlobalLock.acquire(); // Can't change currentPage or any of ids
@@ -291,63 +308,82 @@
{
// Max-size is set, but reject is activated, what means.. never page on
// this address
- if (dropMessagesOnSize)
+ if (dropMessagesWhenFull)
{
- // TODO - why?
return false;
}
+
+ lock.readLock().lock();
+
+ try
+ {
+ // First done without a global lock, to avoid synchronization and increase throuput
+ if (currentPage == null)
+ {
+ return false;
+ }
+ }
+ finally
+ {
+ lock.readLock().unlock();
+ }
- // TODO why calc this before checking currentPage?
- int bytesToWrite = fileFactory.calculateBlockSize(message.getEncodeSize() + PageImpl.SIZE_RECORD);
// The only thing single-threaded done on paging is positioning and
// check-files (verifying if we need to open a new page file)
positioningGlobalLock.acquire();
+
+ boolean gotReadLock = false;
// After we have it locked we keep all the threads working until we need
// to move to a new file (in which case we demand a writeLock, to wait for
// the writes to finish)
try
{
- if (currentPage == null)
+ try
{
- return false;
- }
-
- if (pageUsedSize.addAndGet(bytesToWrite) > pageSize && currentPage.getNumberOfMessages() > 0)
- {
- // Wait any pending write on the current page to finish before we
- // can open another page.
- lock.writeLock().lock();
- try
+ if (currentPage == null)
{
- openNewPage();
- pageUsedSize.addAndGet(bytesToWrite);
+ return false;
}
- finally
+
+ int bytesToWrite = fileFactory.calculateBlockSize(message.getEncodeSize() + PageImpl.SIZE_RECORD);
+
+ if (pageUsedSize.addAndGet(bytesToWrite) > pageSize && currentPage.getNumberOfMessages() > 0)
{
- lock.writeLock().unlock();
+ // Wait any pending write on the current page to finish before we
+ // can open another page.
+ lock.writeLock().lock();
+ try
+ {
+ openNewPage();
+ // openNewPage will zero pageUsedSize, that's why this is done again
+ pageUsedSize.addAndGet(bytesToWrite);
+ }
+ finally
+ {
+ lock.writeLock().unlock();
+ }
}
+ // we must get the readLock before we release the synchronizedBlockLock
+ // or else we could end up with files records being added to the
+ // currentPage even if the max size was already achieved.
+ // (Condition tested by PagingStoreTestPage::testConcurrentPaging, The
+ // test would eventually fail, 1 in 100)
+ // This is because the checkSize and positioning has to be done
+ // protected. We only allow writing the file in multi-thread.
+ lock.readLock().lock();
+
+ gotReadLock = true;
+
}
- // we must get the readLock before we release the synchronizedBlockLock
- // or else we could end up with files records being added to the
- // currentPage even if the max size was already achieved.
- // (Condition tested by PagingStoreTestPage::testConcurrentPaging, The
- // test would eventually fail, 1 in 100)
- // This is because the checkSize and positioning has to be done
- // protected. We only allow writing the file in multi-thread.
- lock.readLock().lock();
+ finally
+ {
+ positioningGlobalLock.release();
+ }
- }
- finally
- {
- positioningGlobalLock.release();
- }
+ // End of a synchronized block..
- // End of a synchronized block..
-
- try
- {
if (currentPage != null)
{
currentPage.write(message);
@@ -360,7 +396,10 @@
}
finally
{
- lock.readLock().unlock();
+ if (gotReadLock)
+ {
+ lock.readLock().unlock();
+ }
}
}
@@ -413,12 +452,12 @@
}
}
- public LastPageRecord getLastRecord()
+ public LastPageRecord getLastPageRecord()
{
return lastPageRecord;
}
- public void setLastRecord(final LastPageRecord record)
+ public void setLastPageRecord(final LastPageRecord record)
{
lastPageRecord = record;
}
@@ -457,7 +496,6 @@
public synchronized void start() throws Exception
{
-
if (running)
{
// don't throw an exception.
@@ -476,7 +514,6 @@
try
{
-
List<String> files = fileFactory.listFiles("page");
numberOfPages = files.size();
@@ -535,6 +572,7 @@
if (currentPage == null)
{
openNewPage();
+
return true;
}
else
@@ -573,6 +611,7 @@
try
{
numberOfPages++;
+
currentPageId++;
if (currentPageId < firstPageId)
@@ -590,7 +629,6 @@
pageUsedSize.set(0);
currentPage.open();
-
}
finally
{
@@ -601,6 +639,7 @@
private Page createPage(final int page) throws Exception
{
String fileName = createFileName(page);
+
SequentialFile file = fileFactory.createSequentialFile(fileName, 1000);
file.open();
Modified: trunk/src/main/org/jboss/messaging/core/persistence/impl/journal/JournalStorageManager.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/persistence/impl/journal/JournalStorageManager.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/persistence/impl/journal/JournalStorageManager.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -504,7 +504,7 @@
PagingManager pagingManager = postOffice.getPagingManager();
- pagingManager.setLastPage(recordImpl);
+ pagingManager.setLastPageRecord(recordImpl);
break;
}
Modified: trunk/src/main/org/jboss/messaging/core/server/impl/MessagingServerImpl.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/server/impl/MessagingServerImpl.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/server/impl/MessagingServerImpl.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -209,16 +209,13 @@
new JBMThreadFactory("JBM-scheduled-threads"));
queueFactory = new QueueFactoryImpl(scheduledExecutor, queueSettingsRepository);
- PagingStoreFactory storeFactory = new PagingStoreFactoryNIO(configuration.getPagingDirectory());
-
- pagingManager = new PagingManagerImpl(storeFactory,
+ pagingManager = new PagingManagerImpl(new PagingStoreFactoryNIO(configuration.getPagingDirectory()),
storageManager,
queueSettingsRepository,
configuration.getPagingMaxGlobalSizeBytes(),
configuration.getPagingDefaultSize());
+ pagingManager.start();
- storeFactory.setPagingManager(pagingManager);
-
resourceManager = new ResourceManagerImpl((int)configuration.getTransactionTimeout() / 1000,
configuration.getTransactionTimeoutScanPeriod(),
storageManager,
Modified: trunk/src/main/org/jboss/messaging/core/server/impl/ServerSessionImpl.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/server/impl/ServerSessionImpl.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/server/impl/ServerSessionImpl.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -2151,7 +2151,6 @@
public void handleSendChunkMessage(final SessionSendChunkMessage packet)
{
-
if (packet.getMessageID() == 0)
{
packet.setMessageID(storageManager.generateUniqueID());
@@ -2490,6 +2489,9 @@
storageManager.storeMessage(msg);
}
+ //TODO - this code is also duplicated in transactionimpl and in depaging
+ //it should all be centralised
+
for (MessageReference ref : refs)
{
if (scheduledDeliveryTime != null)
Modified: trunk/src/main/org/jboss/messaging/core/transaction/impl/TransactionImpl.java
===================================================================
--- trunk/src/main/org/jboss/messaging/core/transaction/impl/TransactionImpl.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/src/main/org/jboss/messaging/core/transaction/impl/TransactionImpl.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -301,7 +301,7 @@
// or else we could deliver the messages out of order
if (pageTransaction != null)
{
- pageTransaction.complete();
+ pageTransaction.commit();
}
for (MessageReference reference : acknowledgements)
@@ -352,7 +352,7 @@
if (state == State.PREPARED && pageTransaction != null)
{
- pageTransaction.forget();
+ pageTransaction.rollback();
}
LinkedList<MessageReference> toCancel = new LinkedList<MessageReference>();
Modified: trunk/tests/src/org/jboss/messaging/tests/integration/paging/PagingManagerIntegrationTest.java
===================================================================
--- trunk/tests/src/org/jboss/messaging/tests/integration/paging/PagingManagerIntegrationTest.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/tests/src/org/jboss/messaging/tests/integration/paging/PagingManagerIntegrationTest.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -53,7 +53,7 @@
// Attributes ----------------------------------------------------
- protected String journalDir = System.getProperty("java.io.tmpdir", "/tmp") + "/journal-test";
+ protected String journalDir = System.getProperty("java.io.tmpdir", "/tmp") + "/journal-test2";
// Static --------------------------------------------------------
@@ -65,12 +65,13 @@
{
HierarchicalRepository<QueueSettings> queueSettings = new HierarchicalObjectRepository<QueueSettings>();
queueSettings.setDefault(new QueueSettings());
-
+
PagingManagerImpl managerImpl = new PagingManagerImpl(new PagingStoreFactoryNIO(journalDir),
null,
queueSettings,
-1,
1024 * 1024);
+
managerImpl.start();
PagingStore store = managerImpl.createPageStore(new SimpleString("simple-test"));
Modified: trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreImplTest.java
===================================================================
--- trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreImplTest.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreImplTest.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -61,7 +61,7 @@
{
SequentialFileFactory factory = new FakeSequentialFileFactory();
- PagingStore storeImpl = new PagingStoreImpl(null, factory, destinationTestName, new QueueSettings(), executor);
+ PagingStore storeImpl = new PagingStoreImpl(createMockManager(), factory, destinationTestName, new QueueSettings(), executor);
storeImpl.start();
@@ -78,7 +78,7 @@
{
SequentialFileFactory factory = new FakeSequentialFileFactory();
- PagingStore storeImpl = new PagingStoreImpl(null, factory, destinationTestName, new QueueSettings(), executor);
+ PagingStore storeImpl = new PagingStoreImpl(createMockManager(), factory, destinationTestName, new QueueSettings(), executor);
storeImpl.start();
@@ -105,7 +105,7 @@
storeImpl.sync();
- storeImpl = new PagingStoreImpl(null, factory, destinationTestName, new QueueSettings(), executor);
+ storeImpl = new PagingStoreImpl(createMockManager(), factory, destinationTestName, new QueueSettings(), executor);
storeImpl.start();
@@ -118,7 +118,7 @@
{
SequentialFileFactory factory = new FakeSequentialFileFactory();
- PagingStore storeImpl = new PagingStoreImpl(null, factory, destinationTestName, new QueueSettings(), executor);
+ PagingStore storeImpl = new PagingStoreImpl(createMockManager(), factory, destinationTestName, new QueueSettings(), executor);
storeImpl.start();
@@ -173,7 +173,7 @@
{
SequentialFileFactory factory = new FakeSequentialFileFactory();
- TestSupportPageStore storeImpl = new PagingStoreImpl(null,
+ TestSupportPageStore storeImpl = new PagingStoreImpl(createMockManager(),
factory,
destinationTestName,
new QueueSettings(),
@@ -292,6 +292,10 @@
testConcurrentPaging(factory, 10);
}
+
+ public void testFoo()
+ {
+ }
// Protected ----------------------------------------------------
Modified: trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreTestBase.java
===================================================================
--- trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreTestBase.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/tests/src/org/jboss/messaging/tests/unit/core/paging/impl/PagingStoreTestBase.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -32,10 +32,13 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import org.easymock.classextension.EasyMock;
+import org.jboss.messaging.core.config.impl.ConfigurationImpl;
import org.jboss.messaging.core.journal.SequentialFile;
import org.jboss.messaging.core.journal.SequentialFileFactory;
import org.jboss.messaging.core.paging.Page;
import org.jboss.messaging.core.paging.PagedMessage;
+import org.jboss.messaging.core.paging.PagingManager;
import org.jboss.messaging.core.paging.impl.PagedMessageImpl;
import org.jboss.messaging.core.paging.impl.PagingStoreImpl;
import org.jboss.messaging.core.paging.impl.TestSupportPageStore;
@@ -102,7 +105,7 @@
QueueSettings settings = new QueueSettings();
settings.setPageSizeBytes(MAX_SIZE);
- final TestSupportPageStore storeImpl = new PagingStoreImpl(null,
+ final TestSupportPageStore storeImpl = new PagingStoreImpl(createMockManager(),
factory,
new SimpleString("test"),
settings,
@@ -256,7 +259,7 @@
fileTmp.close();
}
- TestSupportPageStore storeImpl2 = new PagingStoreImpl(null, factory, new SimpleString("test"), settings, executor);
+ TestSupportPageStore storeImpl2 = new PagingStoreImpl(createMockManager(), factory, new SimpleString("test"), settings, executor);
storeImpl2.start();
int numberOfPages = storeImpl2.getNumberOfPages();
@@ -344,6 +347,17 @@
return buffer;
}
+ /**
+ * @return
+ */
+ protected PagingManager createMockManager()
+ {
+ PagingManager mockManager = EasyMock.createNiceMock(PagingManager.class);
+ EasyMock.expect(mockManager.getDefaultPageSize()).andStubReturn(ConfigurationImpl.DEFAULT_PAGE_SIZE);
+ EasyMock.replay(mockManager);
+ return mockManager;
+ }
+
// Private -------------------------------------------------------
// Inner classes -------------------------------------------------
Modified: trunk/tests/src/org/jboss/messaging/tests/util/ServiceTestBase.java
===================================================================
--- trunk/tests/src/org/jboss/messaging/tests/util/ServiceTestBase.java 2008-11-21 00:46:03 UTC (rev 5404)
+++ trunk/tests/src/org/jboss/messaging/tests/util/ServiceTestBase.java 2008-11-21 01:25:21 UTC (rev 5405)
@@ -84,7 +84,7 @@
protected MessagingService messagingService;
// Static --------------------------------------------------------
- private static final Logger log = Logger.getLogger(ServiceTestBase.class);
+ private final Logger log = Logger.getLogger(this.getClass());
// Constructors --------------------------------------------------
More information about the jboss-cvs-commits
mailing list