diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/DefaultEntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/DefaultEntryLogTest.java index 4b92df528fe..f3eb007386e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/DefaultEntryLogTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/DefaultEntryLogTest.java @@ -22,12 +22,14 @@ import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.assertEntryEquals; import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.makeEntry; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; @@ -41,7 +43,6 @@ import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.util.ReferenceCountUtil; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.RandomAccessFile; import java.nio.channels.FileChannel; @@ -72,20 +73,18 @@ import org.apache.bookkeeper.util.IOUtils; import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap; import org.apache.commons.io.FileUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.FixMethodOrder; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.runners.MethodSorters; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Tests for EntryLog. */ -@FixMethodOrder(MethodSorters.NAME_ASCENDING) +@TestMethodOrder(MethodOrderer.MethodName.class) public class DefaultEntryLogTest { private static final Logger LOG = LoggerFactory.getLogger(DefaultEntryLogTest.class); @@ -104,22 +103,22 @@ File createTempDir(String prefix, String suffix) throws IOException { private LedgerDirsManager dirsMgr; private DefaultEntryLogger entryLogger; - @Before + @BeforeEach public void setUp() throws Exception { this.rootDir = createTempDir("bkTest", ".dir"); this.curDir = BookieImpl.getCurrentDirectory(rootDir); BookieImpl.checkDirectoryStructure(curDir); this.conf = TestBKConfiguration.newServerConfiguration(); this.dirsMgr = new LedgerDirsManager( - conf, - new File[] { rootDir }, - new DiskChecker( - conf.getDiskUsageThreshold(), - conf.getDiskUsageWarnThreshold())); + conf, + new File[] { rootDir }, + new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold())); this.entryLogger = new DefaultEntryLogger(conf, dirsMgr); } - @After + @AfterEach public void tearDown() throws Exception { if (null != this.entryLogger) { entryLogger.close(); @@ -138,11 +137,11 @@ public void testDeferCreateNewLog() throws Exception { // mark `curDir` as filled this.conf.setMinUsableSizeForEntryLogCreation(1); this.dirsMgr = new LedgerDirsManager( - conf, - new File[] { rootDir }, - new DiskChecker( - conf.getDiskUsageThreshold(), - conf.getDiskUsageWarnThreshold())); + conf, + new File[] { rootDir }, + new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold())); this.dirsMgr.addToFilledDirs(curDir); entryLogger = new DefaultEntryLogger(conf, dirsMgr); @@ -209,11 +208,11 @@ public void testDeferCreateNewLogWithoutEnoughDiskSpaces() throws Exception { // mark `curDir` as filled this.conf.setMinUsableSizeForEntryLogCreation(Long.MAX_VALUE); this.dirsMgr = new LedgerDirsManager( - conf, - new File[] { rootDir }, - new DiskChecker( - conf.getDiskUsageThreshold(), - conf.getDiskUsageWarnThreshold())); + conf, + new File[] { rootDir }, + new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold())); this.dirsMgr.addToFilledDirs(curDir); entryLogger = new DefaultEntryLogger(conf, dirsMgr); @@ -332,28 +331,6 @@ public void testMissingLogId() throws Exception { } } - /** - * Test that EntryLogger Should fail with FNFE, if entry logger directories does not exist. - */ - @Ignore // no longer valid as LedgerDirsManager creates the directory as needed - public void testEntryLoggerShouldThrowFNFEIfDirectoriesDoesNotExist() - throws Exception { - File tmpDir = createTempDir("bkTest", ".dir"); - DefaultEntryLogger entryLogger = null; - try { - entryLogger = new DefaultEntryLogger(conf, new LedgerDirsManager(conf, new File[] { tmpDir }, - new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()))); - fail("Expecting FileNotFoundException"); - } catch (FileNotFoundException e) { - assertEquals("Entry log directory '" + tmpDir + "/current' does not exist", e - .getLocalizedMessage()); - } finally { - if (entryLogger != null) { - entryLogger.close(); - } - } - } - /** * Test to verify the DiskFull during addEntry. */ @@ -384,9 +361,9 @@ public void testAddEntryFailureOnDiskFull() throws Exception { .getCurrentLogForLedger(DefaultEntryLogger.UNASSIGNED_LEDGERID).getLogFile().getParentFile()); ledgerStorage.addEntry(generateEntry(3, 1)); // Verify written entries - Assert.assertTrue(0 == generateEntry(1, 1).compareTo(ledgerStorage.getEntry(1, 1))); - Assert.assertTrue(0 == generateEntry(2, 1).compareTo(ledgerStorage.getEntry(2, 1))); - Assert.assertTrue(0 == generateEntry(3, 1).compareTo(ledgerStorage.getEntry(3, 1))); + assertEquals(0, generateEntry(1, 1).compareTo(ledgerStorage.getEntry(1, 1))); + assertEquals(0, generateEntry(2, 1).compareTo(ledgerStorage.getEntry(2, 1))); + assertEquals(0, generateEntry(3, 1).compareTo(ledgerStorage.getEntry(3, 1))); } /** @@ -587,7 +564,7 @@ public void run() { flushThread.join(); createdNewLogThread.join(); - Assert.assertFalse("Exception happened in one of the operation", exceptionHappened.get()); + assertFalse(exceptionHappened.get(), "Exception happened in one of the operation"); if (conf.getFlushIntervalInBytes() > 0) { /* @@ -597,15 +574,14 @@ public void run() { * addEntry call, ledgers map will be appended at the end of entry * log) */ - Assert.assertTrue( + assertTrue(currentActiveChannel.getUnpersistedBytes() < currentActiveChannelUnpersistedBytes, "previous currentChannel unpersistedBytes should be less than " - + currentActiveChannelUnpersistedBytes - + ", but it is actually " + currentActiveChannel.getUnpersistedBytes(), - currentActiveChannel.getUnpersistedBytes() < currentActiveChannelUnpersistedBytes); + + currentActiveChannelUnpersistedBytes + + ", but it is actually " + currentActiveChannel.getUnpersistedBytes()); } for (BufferedLogChannel rotatedLogChannel : rotatedLogChannels) { - Assert.assertEquals("previous rotated entrylog should be flushandforcewritten", 0, - rotatedLogChannel.getUnpersistedBytes()); + assertEquals(0, rotatedLogChannel.getUnpersistedBytes(), + "previous rotated entrylog should be flushandforcewritten"); } } @@ -751,13 +727,12 @@ public void testConcurrentWriteAndReadCalls(String ledgerStorageClass, boolean e Random rand = new Random(0); if (ledgerStorageClass.equals(SortedLedgerStorage.class.getName())) { - Assert.assertEquals("LedgerStorage Class", SortedLedgerStorage.class, ledgerStorage.getClass()); + assertEquals(SortedLedgerStorage.class, ledgerStorage.getClass()); if (entryLogPerLedgerEnabled) { - Assert.assertEquals("MemTable Class", EntryMemTableWithParallelFlusher.class, + assertEquals(EntryMemTableWithParallelFlusher.class, ((SortedLedgerStorage) ledgerStorage).memTable.getClass()); } else { - Assert.assertEquals("MemTable Class", EntryMemTable.class, - ((SortedLedgerStorage) ledgerStorage).memTable.getClass()); + assertEquals(EntryMemTable.class, ((SortedLedgerStorage) ledgerStorage).memTable.getClass()); } } @@ -769,7 +744,7 @@ public void testConcurrentWriteAndReadCalls(String ledgerStorageClass, boolean e } ExecutorService executor = Executors.newFixedThreadPool(10); - List> writeAndFlushTasks = new ArrayList>(); + List> writeAndFlushTasks = new ArrayList<>(); for (int j = 0; j < numEntries; j++) { for (int i = 0; i < numOfLedgers; i++) { writeAndFlushTasks.add(new LedgerStorageWriteTask(i, j, ledgerStorage)); @@ -790,14 +765,14 @@ public void testConcurrentWriteAndReadCalls(String ledgerStorageClass, boolean e } catch (InterruptedException ie) { Thread.currentThread().interrupt(); LOG.error("Write/Flush task failed because of InterruptedException", ie); - Assert.fail("Write/Flush task interrupted"); + fail("Write/Flush task interrupted"); } catch (Exception ex) { LOG.error("Write/Flush task failed because of exception", ex); - Assert.fail("Write/Flush task failed " + ex.getMessage()); + fail("Write/Flush task failed " + ex.getMessage()); } }); - List> readAndFlushTasks = new ArrayList>(); + List> readAndFlushTasks = new ArrayList<>(); for (int j = 0; j < numEntries; j++) { for (int i = 0; i < numOfLedgers; i++) { readAndFlushTasks.add(new LedgerStorageReadTask(i, j, ledgerStorage)); @@ -818,10 +793,10 @@ public void testConcurrentWriteAndReadCalls(String ledgerStorageClass, boolean e } catch (InterruptedException ie) { Thread.currentThread().interrupt(); LOG.error("Read/Flush task failed because of InterruptedException", ie); - Assert.fail("Read/Flush task interrupted"); + fail("Read/Flush task interrupted"); } catch (Exception ex) { LOG.error("Read/Flush task failed because of exception", ex); - Assert.fail("Read/Flush task failed " + ex.getMessage()); + fail("Read/Flush task failed " + ex.getMessage()); } }); @@ -842,36 +817,36 @@ public void testEntryLoggersRecentEntryLogsStatus() throws Exception { DefaultEntryLogger.RecentEntryLogsStatus recentlyCreatedLogsStatus = entryLogger.recentlyCreatedEntryLogsStatus; recentlyCreatedLogsStatus.createdEntryLog(0L); - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 0L, entryLogger.getLeastUnflushedLogId()); + assertEquals(0L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.flushRotatedEntryLog(0L); // since we marked entrylog - 0 as rotated, LeastUnflushedLogId would be previous rotatedlog+1 - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 1L, entryLogger.getLeastUnflushedLogId()); + assertEquals(1L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.createdEntryLog(1L); - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 1L, entryLogger.getLeastUnflushedLogId()); + assertEquals(1L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.createdEntryLog(2L); recentlyCreatedLogsStatus.createdEntryLog(3L); recentlyCreatedLogsStatus.createdEntryLog(4L); - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 1L, entryLogger.getLeastUnflushedLogId()); + assertEquals(1L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.flushRotatedEntryLog(1L); - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 2L, entryLogger.getLeastUnflushedLogId()); + assertEquals(2L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.flushRotatedEntryLog(3L); // here though we rotated entrylog-3, entrylog-2 is not yet rotated so // LeastUnflushedLogId should be still 2 - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 2L, entryLogger.getLeastUnflushedLogId()); + assertEquals(2L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.flushRotatedEntryLog(2L); // entrylog-3 is already rotated, so leastUnflushedLogId should be 4 - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 4L, entryLogger.getLeastUnflushedLogId()); + assertEquals(4L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.flushRotatedEntryLog(4L); - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 5L, entryLogger.getLeastUnflushedLogId()); + assertEquals(5L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.createdEntryLog(5L); recentlyCreatedLogsStatus.createdEntryLog(7L); recentlyCreatedLogsStatus.createdEntryLog(9L); - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 5L, entryLogger.getLeastUnflushedLogId()); + assertEquals(5L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.flushRotatedEntryLog(5L); // since we marked entrylog-5 as rotated, LeastUnflushedLogId would be previous rotatedlog+1 - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 6L, entryLogger.getLeastUnflushedLogId()); + assertEquals(6L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); recentlyCreatedLogsStatus.flushRotatedEntryLog(7L); - Assert.assertEquals("entryLogger's leastUnflushedLogId ", 8L, entryLogger.getLeastUnflushedLogId()); + assertEquals(8L, entryLogger.getLeastUnflushedLogId(), "entryLogger's leastUnflushedLogId "); } String[] createAndGetLedgerDirs(int numOfLedgerDirs) throws IOException { @@ -911,16 +886,17 @@ public void testFlushIntervalInBytes() throws Exception { long entry0Position = entryLogger.addEntry(0L, generateEntry(ledgerId, 0L, firstEntrySize)); // entrylogger writes length of the entry (4 bytes) before writing entry long expectedUnpersistedBytes = DefaultEntryLogger.LOGFILE_HEADER_SIZE + firstEntrySize + 4; - Assert.assertEquals("Unpersisted Bytes of entrylog", expectedUnpersistedBytes, - entryLogManagerBase.getCurrentLogForLedger(ledgerId).getUnpersistedBytes()); + assertEquals(expectedUnpersistedBytes, + entryLogManagerBase.getCurrentLogForLedger(ledgerId).getUnpersistedBytes(), + "Unpersisted Bytes of entrylog"); /* * 'flushIntervalInBytes' number of bytes are flushed so BufferedChannel should be forcewritten */ int secondEntrySize = (int) (flushIntervalInBytes - expectedUnpersistedBytes); long entry1Position = entryLogger.addEntry(0L, generateEntry(ledgerId, 1L, secondEntrySize)); - Assert.assertEquals("Unpersisted Bytes of entrylog", 0, - entryLogManagerBase.getCurrentLogForLedger(ledgerId).getUnpersistedBytes()); + assertEquals(0, entryLogManagerBase.getCurrentLogForLedger(ledgerId).getUnpersistedBytes(), + "Unpersisted Bytes of entrylog"); /* * since entrylog/Bufferedchannel is persisted (forcewritten), we should be able to read the entrylog using @@ -929,20 +905,20 @@ public void testFlushIntervalInBytes() throws Exception { conf.setEntryLogPerLedgerEnabled(false); DefaultEntryLogger newEntryLogger = new DefaultEntryLogger(conf, ledgerDirsManager); EntryLogManager newEntryLogManager = newEntryLogger.getEntryLogManager(); - Assert.assertEquals("EntryLogManager class type", EntryLogManagerForSingleEntryLog.class, - newEntryLogManager.getClass()); + assertEquals(EntryLogManagerForSingleEntryLog.class, newEntryLogManager.getClass(), + "EntryLogManager class type"); ByteBuf buf = newEntryLogger.readEntry(ledgerId, 0L, entry0Position); long readLedgerId = buf.readLong(); long readEntryId = buf.readLong(); - Assert.assertEquals("LedgerId", ledgerId, readLedgerId); - Assert.assertEquals("EntryId", 0L, readEntryId); + assertEquals(ledgerId, readLedgerId, "LedgerId"); + assertEquals(0L, readEntryId, "EntryId"); buf = newEntryLogger.readEntry(ledgerId, 1L, entry1Position); readLedgerId = buf.readLong(); readEntryId = buf.readLong(); - Assert.assertEquals("LedgerId", ledgerId, readLedgerId); - Assert.assertEquals("EntryId", 1L, readEntryId); + assertEquals(ledgerId, readLedgerId, "LedgerId"); + assertEquals(1L, readEntryId, "EntryId"); } @Test @@ -961,8 +937,8 @@ public void testReadEntryWithoutLedgerID() throws Exception { assertEntryEquals(data, makeEntry(1L, i, 100)); long readLedgerId = data.readLong(); long readEntryId = data.readLong(); - Assert.assertEquals("LedgerId", 1L, readLedgerId); - Assert.assertEquals("EntryId", i, readEntryId); + assertEquals(1L, readLedgerId, "LedgerId"); + assertEquals(i, readEntryId, "EntryId"); ReferenceCountUtil.release(data); } } @@ -985,8 +961,8 @@ public void testEntryLogManagerInterfaceForEntryLogPerLedger() throws Exception EntryLogManagerForEntryLogPerLedger entryLogManager = (EntryLogManagerForEntryLogPerLedger) entryLogger .getEntryLogManager(); - Assert.assertEquals("Number of current active EntryLogs ", 0, entryLogManager.getCopyOfCurrentLogs().size()); - Assert.assertEquals("Number of Rotated Logs ", 0, entryLogManager.getRotatedLogChannels().size()); + assertEquals(0, entryLogManager.getCopyOfCurrentLogs().size(), "Number of current active EntryLogs "); + assertEquals(0, entryLogManager.getRotatedLogChannels().size(), "Number of Rotated Logs "); int numOfLedgers = 5; int numOfThreadsPerLedger = 10; @@ -994,72 +970,69 @@ public void testEntryLogManagerInterfaceForEntryLogPerLedger() throws Exception for (long i = 0; i < numOfLedgers; i++) { entryLogManager.setCurrentLogForLedgerAndAddToRotate(i, - createDummyBufferedLogChannel(entryLogger, i, conf)); + createDummyBufferedLogChannel(i, conf)); } for (long i = 0; i < numOfLedgers; i++) { - Assert.assertEquals("LogChannel for ledger: " + i, entryLogManager.getCurrentLogIfPresent(i), - entryLogManager.getCurrentLogForLedger(i)); + assertEquals(entryLogManager.getCurrentLogIfPresent(i), entryLogManager.getCurrentLogForLedger(i), + "LogChannel for ledger: " + i); } - Assert.assertEquals("Number of current active EntryLogs ", numOfLedgers, - entryLogManager.getCopyOfCurrentLogs().size()); - Assert.assertEquals("Number of Rotated Logs ", 0, entryLogManager.getRotatedLogChannels().size()); + assertEquals(numOfLedgers, entryLogManager.getCopyOfCurrentLogs().size(), + "Number of current active EntryLogs "); + assertEquals(0, entryLogManager.getRotatedLogChannels().size(), "Number of Rotated Logs "); for (long i = 0; i < numOfLedgers; i++) { entryLogManager.setCurrentLogForLedgerAndAddToRotate(i, - createDummyBufferedLogChannel(entryLogger, numOfLedgers + i, conf)); + createDummyBufferedLogChannel(numOfLedgers + i, conf)); } /* * since new entryLogs are set for all the ledgers, previous entrylogs would be added to rotatedLogChannels */ - Assert.assertEquals("Number of current active EntryLogs ", numOfLedgers, - entryLogManager.getCopyOfCurrentLogs().size()); - Assert.assertEquals("Number of Rotated Logs ", numOfLedgers, - entryLogManager.getRotatedLogChannels().size()); + assertEquals(numOfLedgers, entryLogManager.getCopyOfCurrentLogs().size(), + "Number of current active EntryLogs "); + assertEquals(numOfLedgers, entryLogManager.getRotatedLogChannels().size(), "Number of Rotated Logs "); for (long i = 0; i < numOfLedgers; i++) { entryLogManager.setCurrentLogForLedgerAndAddToRotate(i, - createDummyBufferedLogChannel(entryLogger, 2 * numOfLedgers + i, conf)); + createDummyBufferedLogChannel(2 * numOfLedgers + i, conf)); } /* * again since new entryLogs are set for all the ledgers, previous entrylogs would be added to * rotatedLogChannels */ - Assert.assertEquals("Number of current active EntryLogs ", numOfLedgers, - entryLogManager.getCopyOfCurrentLogs().size()); - Assert.assertEquals("Number of Rotated Logs ", 2 * numOfLedgers, - entryLogManager.getRotatedLogChannels().size()); + assertEquals(numOfLedgers, entryLogManager.getCopyOfCurrentLogs().size(), + "Number of current active EntryLogs "); + assertEquals(2 * numOfLedgers, entryLogManager.getRotatedLogChannels().size(), "Number of Rotated Logs "); for (BufferedLogChannel logChannel : entryLogManager.getRotatedLogChannels()) { entryLogManager.getRotatedLogChannels().remove(logChannel); } - Assert.assertEquals("Number of Rotated Logs ", 0, entryLogManager.getRotatedLogChannels().size()); + assertEquals(0, entryLogManager.getRotatedLogChannels().size(), "Number of Rotated Logs "); // entrylogid is sequential for (long i = 0; i < numOfLedgers; i++) { - assertEquals("EntryLogid for Ledger " + i, 2 * numOfLedgers + i, - entryLogManager.getCurrentLogForLedger(i).getLogId()); + assertEquals(2 * numOfLedgers + i, entryLogManager.getCurrentLogForLedger(i).getLogId(), + "EntryLogId for Ledger " + i); } for (long i = 2 * numOfLedgers; i < (3 * numOfLedgers); i++) { - assertTrue("EntryLog with logId: " + i + " should be present", - entryLogManager.getCurrentLogIfPresent(i) != null); + assertNotNull(entryLogManager.getCurrentLogIfPresent(i), + "EntryLog with logId: " + i + " should be present"); } } - private DefaultEntryLogger.BufferedLogChannel createDummyBufferedLogChannel(DefaultEntryLogger entryLogger, - long logid, + private DefaultEntryLogger.BufferedLogChannel createDummyBufferedLogChannel(long logid, ServerConfiguration servConf) - throws IOException { + throws IOException { File tmpFile = File.createTempFile("entrylog", logid + ""); tmpFile.deleteOnExit(); FileChannel fc = new RandomAccessFile(tmpFile, "rw").getChannel(); DefaultEntryLogger.BufferedLogChannel logChannel = - new BufferedLogChannel(UnpooledByteBufAllocator.DEFAULT, fc, 10, 10, - logid, tmpFile, servConf.getFlushIntervalInBytes()); + new BufferedLogChannel(UnpooledByteBufAllocator.DEFAULT, fc, 10, 10, + logid, tmpFile, servConf.getFlushIntervalInBytes()); return logChannel; } @@ -1070,13 +1043,13 @@ private DefaultEntryLogger.BufferedLogChannel createDummyBufferedLogChannel(Defa * of tasks are submitted to the Executor. In each task, lock of that ledger is acquired and then released. */ private void validateLockAcquireAndRelease(int numOfLedgers, int numOfThreadsPerLedger, - EntryLogManagerForEntryLogPerLedger entryLogManager) throws InterruptedException { + EntryLogManagerForEntryLogPerLedger entryLogManager) + throws InterruptedException { ExecutorService tpe = Executors.newFixedThreadPool(numOfLedgers * numOfThreadsPerLedger); CountDownLatch latchToStart = new CountDownLatch(1); CountDownLatch latchToWait = new CountDownLatch(1); AtomicInteger numberOfThreadsAcquiredLock = new AtomicInteger(0); AtomicBoolean irptExceptionHappened = new AtomicBoolean(false); - Random rand = new Random(); for (int i = 0; i < numOfLedgers * numOfThreadsPerLedger; i++) { long ledgerId = i % numOfLedgers; @@ -1094,7 +1067,7 @@ private void validateLockAcquireAndRelease(int numOfLedgers, int numOfThreadsPer }); } - assertEquals("Number Of Threads acquired Lock", 0, numberOfThreadsAcquiredLock.get()); + assertEquals(0, numberOfThreadsAcquiredLock.get(), "Number Of Threads acquired Lock"); latchToStart.countDown(); Thread.sleep(1000); /* @@ -1107,13 +1080,14 @@ private void validateLockAcquireAndRelease(int numOfLedgers, int numOfThreadsPer * After acquiring the lock there must be waiting on 'latchToWait' latch */ int currentNumberOfThreadsAcquiredLock = numberOfThreadsAcquiredLock.get(); - assertTrue("Number Of Threads acquired Lock " + currentNumberOfThreadsAcquiredLock, - (currentNumberOfThreadsAcquiredLock > 0) && (currentNumberOfThreadsAcquiredLock <= numOfLedgers)); + assertTrue((currentNumberOfThreadsAcquiredLock > 0) && (currentNumberOfThreadsAcquiredLock <= numOfLedgers), + "Number Of Threads acquired Lock " + currentNumberOfThreadsAcquiredLock); + latchToWait.countDown(); Thread.sleep(2000); - assertEquals("Number Of Threads acquired Lock", numOfLedgers * numOfThreadsPerLedger, - numberOfThreadsAcquiredLock.get()); - } + + assertEquals(numOfLedgers * numOfThreadsPerLedger, numberOfThreadsAcquiredLock.get(), + "Number Of Threads acquired Lock"); } /* * test EntryLogManager.EntryLogManagerForEntryLogPerLedger removes the @@ -1140,11 +1114,11 @@ public void testEntryLogManagerExpiryRemoval() throws Exception { long ledgerId = 0L; - BufferedLogChannel logChannel = createDummyBufferedLogChannel(entryLogger, 0, conf); + BufferedLogChannel logChannel = createDummyBufferedLogChannel(0, conf); entryLogManager.setCurrentLogForLedgerAndAddToRotate(ledgerId, logChannel); BufferedLogChannel currentLogForLedger = entryLogManager.getCurrentLogForLedger(ledgerId); - assertEquals("LogChannel for ledger " + ledgerId + " should match", logChannel, currentLogForLedger); + assertEquals(logChannel, currentLogForLedger, "LogChannel for ledger " + ledgerId + " should match"); Thread.sleep(evictionPeriod * 1000 + 100); entryLogManager.doEntryLogMapCleanup(); @@ -1154,15 +1128,15 @@ public void testEntryLogManagerExpiryRemoval() throws Exception { * ledger should not be available anymore */ currentLogForLedger = entryLogManager.getCurrentLogForLedger(ledgerId); - assertEquals("LogChannel for ledger " + ledgerId + " should be null", null, currentLogForLedger); - Assert.assertEquals("Number of current active EntryLogs ", 0, entryLogManager.getCopyOfCurrentLogs().size()); - Assert.assertEquals("Number of rotated EntryLogs ", 1, entryLogManager.getRotatedLogChannels().size()); - Assert.assertTrue("CopyOfRotatedLogChannels should contain the created LogChannel", - entryLogManager.getRotatedLogChannels().contains(logChannel)); - - Assert.assertTrue("since mapentry must have been evicted, it should be null", - (entryLogManager.getCacheAsMap().get(ledgerId) == null) - || (entryLogManager.getCacheAsMap().get(ledgerId).getEntryLogWithDirInfo() == null)); + assertNull(currentLogForLedger, "LogChannel for ledger " + ledgerId + " should be null"); + assertEquals(0, entryLogManager.getCopyOfCurrentLogs().size(), "Number of current active EntryLogs "); + assertEquals(1, entryLogManager.getRotatedLogChannels().size(), "Number of rotated EntryLogs "); + assertTrue(entryLogManager.getRotatedLogChannels().contains(logChannel), + "CopyOfRotatedLogChannels should contain the created LogChannel"); + + assertTrue((entryLogManager.getCacheAsMap().get(ledgerId) == null) + || (entryLogManager.getCacheAsMap().get(ledgerId).getEntryLogWithDirInfo() == null), + "since map entry must have been evicted, it should be null"); } /* @@ -1189,8 +1163,9 @@ public void testCacheMaximumSizeEvictionPolicy() throws Exception { for (int i = 0; i < cacheMaximumSize + 10; i++) { entryLogManager.createNewLog(i); int cacheSize = entryLogManager.getCacheAsMap().size(); - Assert.assertTrue("Cache maximum size is expected to be less than " + cacheMaximumSize - + " but current cacheSize is " + cacheSize, cacheSize <= cacheMaximumSize); + assertTrue(cacheSize <= cacheMaximumSize, + "Cache maximum size is expected to be less than " + cacheMaximumSize + + " but current cacheSize is " + cacheSize); } } @@ -1233,9 +1208,9 @@ public void testLongLedgerIdsWithEntryLogPerLedger() throws Exception { long readEntryId = buf.readLong(); byte[] readData = new byte[buf.readableBytes()]; buf.readBytes(readData); - assertEquals("LedgerId ", ledgerId, readLedgerId); - assertEquals("EntryId ", entryId, readEntryId); - assertEquals("Entry Data ", expectedValue, new String(readData)); + assertEquals(ledgerId, readLedgerId, "LedgerId "); + assertEquals(entryId, readEntryId, "EntryId "); + assertEquals(expectedValue, new String(readData), "Entry Data "); } } } @@ -1296,9 +1271,9 @@ public void testAppendLedgersMapOnCacheRemoval() throws Exception { EntryLogMetadata entryLogMetadata = entryLogger.extractEntryLogMetadataFromIndex(logIdOfLedger); ConcurrentLongLongHashMap ledgersMap = entryLogMetadata.getLedgersMap(); - Assert.assertEquals("There should be only one entry in entryLogMetadata", 1, ledgersMap.size()); - Assert.assertTrue("Usage should be 1", Double.compare(1.0, entryLogMetadata.getUsage()) == 0); - Assert.assertEquals("Total size of entries", (entrySize + 4) * numOfEntries, ledgersMap.get(ledgerId)); + assertEquals(1, ledgersMap.size(), "There should be only one entry in entryLogMetadata"); + assertEquals(0, Double.compare(1.0, entryLogMetadata.getUsage()), "Usage should be 1"); + assertEquals((entrySize + 4) * numOfEntries, ledgersMap.get(ledgerId), "Total size of entries"); } /** @@ -1325,7 +1300,7 @@ public void testExpiryRemovalByAccessingOnAnotherThread() throws Exception { long ledgerId = 0L; - BufferedLogChannel newLogChannel = createDummyBufferedLogChannel(entryLogger, 1, conf); + BufferedLogChannel newLogChannel = createDummyBufferedLogChannel(1, conf); entryLogManager.setCurrentLogForLedgerAndAddToRotate(ledgerId, newLogChannel); Thread t = new Thread() { @@ -1347,9 +1322,9 @@ public void run() { * eviction period time, so it should not be evicted. */ BufferedLogChannel currentLogForLedger = entryLogManager.getCurrentLogForLedger(ledgerId); - assertEquals("LogChannel for ledger " + ledgerId, newLogChannel, currentLogForLedger); - Assert.assertEquals("Number of current active EntryLogs ", 1, entryLogManager.getCopyOfCurrentLogs().size()); - Assert.assertEquals("Number of rotated EntryLogs ", 0, entryLogManager.getRotatedLogChannels().size()); + assertEquals(newLogChannel, currentLogForLedger, "LogChannel for ledger " + ledgerId); + assertEquals(1, entryLogManager.getCopyOfCurrentLogs().size(), "Number of current active EntryLogs "); + assertEquals(0, entryLogManager.getRotatedLogChannels().size(), "Number of rotated EntryLogs "); } /** @@ -1379,7 +1354,7 @@ public void testExpiryRemovalByAccessingNonCacheRelatedMethods() throws Exceptio long ledgerId = 0L; - BufferedLogChannel newLogChannel = createDummyBufferedLogChannel(entryLogger, 1, conf); + BufferedLogChannel newLogChannel = createDummyBufferedLogChannel(1, conf); entryLogManager.setCurrentLogForLedgerAndAddToRotate(ledgerId, newLogChannel); AtomicBoolean exceptionOccurred = new AtomicBoolean(false); @@ -1397,7 +1372,7 @@ public void run() { entryLogManager.getDirForNextEntryLog(ledgerDirsManager.getWritableLedgerDirs()); long newLedgerId = 100; BufferedLogChannel logChannelForNewLedger = - createDummyBufferedLogChannel(entryLogger, newLedgerId, conf); + createDummyBufferedLogChannel(newLedgerId, conf); entryLogManager.setCurrentLogForLedgerAndAddToRotate(newLedgerId, logChannelForNewLedger); entryLogManager.getCurrentLogIfPresent(newLedgerId); } catch (Exception e) { @@ -1410,23 +1385,23 @@ public void run() { t.start(); Thread.sleep(evictionPeriod * 1000 + 100); entryLogManager.doEntryLogMapCleanup(); - Assert.assertFalse("Exception occurred in thread, which is not expected", exceptionOccurred.get()); + assertFalse(exceptionOccurred.get(), "Exception occurred in thread, which is not expected"); /* * since for more than evictionPeriod, that ledger is not accessed and cache is cleaned up, mapping for that * ledger should not be available anymore */ BufferedLogChannel currentLogForLedger = entryLogManager.getCurrentLogForLedger(ledgerId); - assertEquals("LogChannel for ledger " + ledgerId + " should be null", null, currentLogForLedger); + assertNull(currentLogForLedger, "LogChannel for ledger " + ledgerId + " should be null"); // expected number of current active entryLogs is 1 since we created entrylog for 'newLedgerId' - Assert.assertEquals("Number of current active EntryLogs ", 1, entryLogManager.getCopyOfCurrentLogs().size()); - Assert.assertEquals("Number of rotated EntryLogs ", 1, entryLogManager.getRotatedLogChannels().size()); - Assert.assertTrue("CopyOfRotatedLogChannels should contain the created LogChannel", - entryLogManager.getRotatedLogChannels().contains(newLogChannel)); - - Assert.assertTrue("since mapentry must have been evicted, it should be null", - (entryLogManager.getCacheAsMap().get(ledgerId) == null) - || (entryLogManager.getCacheAsMap().get(ledgerId).getEntryLogWithDirInfo() == null)); + assertEquals(1, entryLogManager.getCopyOfCurrentLogs().size(), "Number of current active EntryLogs "); + assertEquals(1, entryLogManager.getRotatedLogChannels().size(), "Number of rotated EntryLogs "); + assertTrue(entryLogManager.getRotatedLogChannels().contains(newLogChannel), + "CopyOfRotatedLogChannels should contain the created LogChannel"); + + assertTrue((entryLogManager.getCacheAsMap().get(ledgerId) == null) + || (entryLogManager.getCacheAsMap().get(ledgerId).getEntryLogWithDirInfo() == null), + "since mapentry must have been evicted, it should be null"); } /* @@ -1443,8 +1418,8 @@ public void testEntryLogManagerForEntryLogPerLedger() throws Exception { new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); DefaultEntryLogger entryLogger = new DefaultEntryLogger(conf, ledgerDirsManager); EntryLogManagerBase entryLogManager = (EntryLogManagerBase) entryLogger.getEntryLogManager(); - Assert.assertEquals("EntryLogManager class type", EntryLogManagerForEntryLogPerLedger.class, - entryLogManager.getClass()); + assertEquals(EntryLogManagerForEntryLogPerLedger.class, entryLogManager.getClass(), + "EntryLogManager class type"); int numOfActiveLedgers = 20; int numEntries = 5; @@ -1457,8 +1432,8 @@ public void testEntryLogManagerForEntryLogPerLedger() throws Exception { for (long i = 0; i < numOfActiveLedgers; i++) { BufferedLogChannel logChannel = entryLogManager.getCurrentLogForLedger(i); - Assert.assertTrue("unpersistedBytes should be greater than LOGFILE_HEADER_SIZE", - logChannel.getUnpersistedBytes() > DefaultEntryLogger.LOGFILE_HEADER_SIZE); + assertTrue(logChannel.getUnpersistedBytes() > DefaultEntryLogger.LOGFILE_HEADER_SIZE, + "unpersistedBytes should be greater than LOGFILE_HEADER_SIZE"); } for (long i = 0; i < numOfActiveLedgers; i++) { @@ -1470,7 +1445,7 @@ public void testEntryLogManagerForEntryLogPerLedger() throws Exception { * should be rotated and hence the size of copyOfRotatedLogChannels should be numOfActiveLedgers */ List rotatedLogs = entryLogManager.getRotatedLogChannels(); - Assert.assertEquals("Number of rotated entrylogs", numOfActiveLedgers, rotatedLogs.size()); + assertEquals(numOfActiveLedgers, rotatedLogs.size(), "Number of rotated entrylogs"); /* * Since newlog is created for all slots, so they are moved to rotated logs and hence unpersistedBytes of all @@ -1479,8 +1454,8 @@ public void testEntryLogManagerForEntryLogPerLedger() throws Exception { */ for (long i = 0; i < numOfActiveLedgers; i++) { BufferedLogChannel logChannel = entryLogManager.getCurrentLogForLedger(i); - Assert.assertEquals("unpersistedBytes should be LOGFILE_HEADER_SIZE", - DefaultEntryLogger.LOGFILE_HEADER_SIZE, logChannel.getUnpersistedBytes()); + assertEquals(DefaultEntryLogger.LOGFILE_HEADER_SIZE, logChannel.getUnpersistedBytes(), + "unpersistedBytes should be LOGFILE_HEADER_SIZE"); } for (int j = numEntries; j < 2 * numEntries; j++) { @@ -1491,37 +1466,37 @@ public void testEntryLogManagerForEntryLogPerLedger() throws Exception { for (long i = 0; i < numOfActiveLedgers; i++) { BufferedLogChannel logChannel = entryLogManager.getCurrentLogForLedger(i); - Assert.assertTrue("unpersistedBytes should be greater than LOGFILE_HEADER_SIZE", - logChannel.getUnpersistedBytes() > DefaultEntryLogger.LOGFILE_HEADER_SIZE); + assertTrue(logChannel.getUnpersistedBytes() > DefaultEntryLogger.LOGFILE_HEADER_SIZE, + "unpersistedBytes should be greater than LOGFILE_HEADER_SIZE"); } - Assert.assertEquals("LeastUnflushedloggerID", 0, entryLogger.getLeastUnflushedLogId()); + assertEquals(0, entryLogger.getLeastUnflushedLogId(), "LeastUnflushedloggerID"); /* * here flush is called so all the rotatedLogChannels should be file closed and there shouldn't be any * rotatedlogchannel and also leastUnflushedLogId should be advanced to numOfActiveLedgers */ entryLogger.flush(); - Assert.assertEquals("Number of rotated entrylogs", 0, entryLogManager.getRotatedLogChannels().size()); - Assert.assertEquals("LeastUnflushedloggerID", numOfActiveLedgers, entryLogger.getLeastUnflushedLogId()); + assertEquals(0, entryLogManager.getRotatedLogChannels().size(), "Number of rotated entrylogs"); + assertEquals(numOfActiveLedgers, entryLogger.getLeastUnflushedLogId(), "LeastUnflushedloggerID"); /* * after flush (flushCurrentLogs) unpersistedBytes should be 0. */ for (long i = 0; i < numOfActiveLedgers; i++) { BufferedLogChannel logChannel = entryLogManager.getCurrentLogForLedger(i); - Assert.assertEquals("unpersistedBytes should be 0", 0L, logChannel.getUnpersistedBytes()); + assertEquals(0L, logChannel.getUnpersistedBytes(), "unpersistedBytes should be 0"); } } @Test public void testSingleEntryLogCreateNewLog() throws Exception { - Assert.assertTrue(entryLogger.getEntryLogManager() instanceof EntryLogManagerForSingleEntryLog); + assertInstanceOf(EntryLogManagerForSingleEntryLog.class, entryLogger.getEntryLogManager()); EntryLogManagerForSingleEntryLog singleEntryLog = (EntryLogManagerForSingleEntryLog) entryLogger.getEntryLogManager(); EntryLogManagerForSingleEntryLog mockSingleEntryLog = spy(singleEntryLog); BufferedLogChannel activeLogChannel = mockSingleEntryLog.getCurrentLogForLedgerForAddEntry(1, 1024, true); - Assert.assertTrue(activeLogChannel != null); + assertNotNull(activeLogChannel); verify(mockSingleEntryLog, times(1)).createNewLog(anyLong(), anyString()); // `readEntryLogHardLimit` and `reachEntryLogLimit` should not call if new create log @@ -1560,12 +1535,11 @@ public void testReadAddCallsOfMultipleEntryLogs() throws Exception { for (int i = 0; i < numOfActiveLedgers; i++) { positions[i][j] = entryLogger.addEntry((long) i, generateEntry(i, j)); long entryLogId = (positions[i][j] >> 32L); - /** - * + /* * Though EntryLogFilePreAllocation is enabled, Since things are not done concurrently here, * entryLogIds will be sequential. */ - Assert.assertEquals("EntryLogId for ledger: " + i, i, entryLogId); + assertEquals(i, entryLogId, "EntryLogId for ledger: " + i); } } @@ -1580,9 +1554,9 @@ public void testReadAddCallsOfMultipleEntryLogs() throws Exception { long entryId = buf.readLong(); byte[] data = new byte[buf.readableBytes()]; buf.readBytes(data); - assertEquals("LedgerId ", i, ledgerId); - assertEquals("EntryId ", j, entryId); - assertEquals("Entry Data ", expectedValue, new String(data)); + assertEquals(i, ledgerId, "LedgerId "); + assertEquals(j, entryId, "EntryId "); + assertEquals(expectedValue, new String(data), "Entry Data "); } } @@ -1601,14 +1575,14 @@ public void testReadAddCallsOfMultipleEntryLogs() throws Exception { long entryId = buf.readLong(); byte[] data = new byte[buf.readableBytes()]; buf.readBytes(data); - assertEquals("LedgerId ", i, ledgerId); - assertEquals("EntryId ", j, entryId); - assertEquals("Entry Data ", expectedValue, new String(data)); + assertEquals(i, ledgerId, "LedgerId "); + assertEquals(j, entryId, "EntryId "); + assertEquals(expectedValue, new String(data), "Entry Data "); } } } - class ReadTask implements Callable { + static class ReadTask implements Callable { long ledgerId; int entryId; long position; @@ -1669,7 +1643,7 @@ public void testConcurrentReadCallsAfterEntryLogsAreRotated() throws Exception { * Though EntryLogFilePreAllocation is enabled, Since things are not done concurrently here, entryLogIds * will be sequential. */ - Assert.assertEquals("EntryLogId for ledger: " + i, i, entryLogId); + assertEquals(i, entryLogId, "EntryLogId for ledger: " + i); } } @@ -1693,10 +1667,10 @@ public void testConcurrentReadCallsAfterEntryLogsAreRotated() throws Exception { } catch (InterruptedException ie) { Thread.currentThread().interrupt(); LOG.error("Read/Flush task failed because of InterruptedException", ie); - Assert.fail("Read/Flush task interrupted"); + fail("Read/Flush task interrupted"); } catch (Exception ex) { LOG.error("Read/Flush task failed because of exception", ex); - Assert.fail("Read/Flush task failed " + ex.getMessage()); + fail("Read/Flush task failed " + ex.getMessage()); } }); } @@ -1735,8 +1709,8 @@ public void testEntryLoggerAddEntryWhenLedgerDirsAreFull() throws Exception { DefaultEntryLogger entryLogger = new DefaultEntryLogger(conf, ledgerDirsManager); EntryLogManagerForEntryLogPerLedger entryLogManager = (EntryLogManagerForEntryLogPerLedger) entryLogger.getEntryLogManager(); - Assert.assertEquals("EntryLogManager class type", EntryLogManagerForEntryLogPerLedger.class, - entryLogManager.getClass()); + assertEquals(EntryLogManagerForEntryLogPerLedger.class, entryLogManager.getClass(), + "EntryLogManager class type"); entryLogger.addEntry(0L, generateEntry(0, 1)); entryLogger.addEntry(1L, generateEntry(1, 1)); @@ -1754,8 +1728,8 @@ public void testEntryLoggerAddEntryWhenLedgerDirsAreFull() throws Exception { /* * since there are 3 ledgerdirs, entrylogs for all the 3 ledgers should be in different ledgerdirs. */ - Assert.assertEquals("Current active LedgerDirs size", 3, ledgerDirs.size()); - Assert.assertEquals("Number of rotated logchannels", 0, entryLogManager.getRotatedLogChannels().size()); + assertEquals(3, ledgerDirs.size(), "Current active LedgerDirs size"); + assertEquals(0, entryLogManager.getRotatedLogChannels().size(), "Number of rotated logchannels"); /* * ledgerDirForLedger0 is added to filledDirs, for ledger0 new entrylog should not be created in @@ -1764,7 +1738,7 @@ public void testEntryLoggerAddEntryWhenLedgerDirsAreFull() throws Exception { ledgerDirsManager.addToFilledDirs(ledgerDirForLedger0); addEntryAndValidateFolders(entryLogger, entryLogManager, 2, ledgerDirForLedger0, false, ledgerDirForLedger1, ledgerDirForLedger2); - Assert.assertEquals("Number of rotated logchannels", 1, entryLogManager.getRotatedLogChannels().size()); + assertEquals(1, entryLogManager.getRotatedLogChannels().size(), "Number of rotated logchannels"); /* * ledgerDirForLedger1 is also added to filledDirs, so for all the ledgers new entryLogs should be in @@ -1773,8 +1747,8 @@ public void testEntryLoggerAddEntryWhenLedgerDirsAreFull() throws Exception { ledgerDirsManager.addToFilledDirs(ledgerDirForLedger1); addEntryAndValidateFolders(entryLogger, entryLogManager, 3, ledgerDirForLedger2, true, ledgerDirForLedger2, ledgerDirForLedger2); - Assert.assertTrue("Number of rotated logchannels", (2 <= entryLogManager.getRotatedLogChannels().size()) - && (entryLogManager.getRotatedLogChannels().size() <= 3)); + assertTrue((2 <= entryLogManager.getRotatedLogChannels().size()) + && (entryLogManager.getRotatedLogChannels().size() <= 3), "Number of rotated logchannels"); int numOfRotatedLogChannels = entryLogManager.getRotatedLogChannels().size(); /* @@ -1785,8 +1759,8 @@ public void testEntryLoggerAddEntryWhenLedgerDirsAreFull() throws Exception { ledgerDirsManager.addToFilledDirs(ledgerDirForLedger2); addEntryAndValidateFolders(entryLogger, entryLogManager, 4, ledgerDirForLedger2, true, ledgerDirForLedger2, ledgerDirForLedger2); - Assert.assertEquals("Number of rotated logchannels", numOfRotatedLogChannels, - entryLogManager.getRotatedLogChannels().size()); + assertEquals(numOfRotatedLogChannels, entryLogManager.getRotatedLogChannels().size(), + "Number of rotated logchannels"); /* * ledgerDirForLedger1 is added back to writableDirs, so new entrylog for all the ledgers should be created in @@ -1795,8 +1769,8 @@ public void testEntryLoggerAddEntryWhenLedgerDirsAreFull() throws Exception { ledgerDirsManager.addToWritableDirs(ledgerDirForLedger1, true); addEntryAndValidateFolders(entryLogger, entryLogManager, 4, ledgerDirForLedger1, true, ledgerDirForLedger1, ledgerDirForLedger1); - Assert.assertEquals("Number of rotated logchannels", numOfRotatedLogChannels + 3, - entryLogManager.getRotatedLogChannels().size()); + assertEquals(numOfRotatedLogChannels + 3, entryLogManager.getRotatedLogChannels().size(), + "Number of rotated logchannels"); } /* @@ -1811,16 +1785,18 @@ void addEntryAndValidateFolders(DefaultEntryLogger entryLogger, EntryLogManagerB entryLogger.addEntry(2L, generateEntry(2, entryId)); if (equalsForLedger0) { - Assert.assertEquals("LedgerDir for ledger 0 after adding entry " + entryId, expectedDirForLedger0, - entryLogManager.getCurrentLogForLedger(0L).getLogFile().getParentFile()); + assertEquals(expectedDirForLedger0, + entryLogManager.getCurrentLogForLedger(0L).getLogFile().getParentFile(), + "LedgerDir for ledger 0 after adding entry " + entryId); } else { - Assert.assertNotEquals("LedgerDir for ledger 0 after adding entry " + entryId, expectedDirForLedger0, - entryLogManager.getCurrentLogForLedger(0L).getLogFile().getParentFile()); - } - Assert.assertEquals("LedgerDir for ledger 1 after adding entry " + entryId, expectedDirForLedger1, - entryLogManager.getCurrentLogForLedger(1L).getLogFile().getParentFile()); - Assert.assertEquals("LedgerDir for ledger 2 after adding entry " + entryId, expectedDirForLedger2, - entryLogManager.getCurrentLogForLedger(2L).getLogFile().getParentFile()); + assertNotEquals(expectedDirForLedger0, + entryLogManager.getCurrentLogForLedger(0L).getLogFile().getParentFile(), + "LedgerDir for ledger 0 after adding entry " + entryId); + } + assertEquals(expectedDirForLedger1, entryLogManager.getCurrentLogForLedger(1L).getLogFile().getParentFile(), + "LedgerDir for ledger 1 after adding entry " + entryId); + assertEquals(expectedDirForLedger2, entryLogManager.getCurrentLogForLedger(2L).getLogFile().getParentFile(), + "LedgerDir for ledger 2 after adding entry " + entryId); } /* @@ -1842,7 +1818,7 @@ public void testSwappingEntryLogManagerFromSingleToEntryLogPerLedger() throws Ex } public void testSwappingEntryLogManager(boolean initialEntryLogPerLedgerEnabled, - boolean laterEntryLogPerLedgerEnabled) throws Exception { + boolean laterEntryLogPerLedgerEnabled) throws Exception { ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); conf.setEntryLogPerLedgerEnabled(initialEntryLogPerLedgerEnabled); conf.setLedgerDirNames(createAndGetLedgerDirs(2)); @@ -1853,10 +1829,9 @@ public void testSwappingEntryLogManager(boolean initialEntryLogPerLedgerEnabled, DefaultEntryLogger defaultEntryLogger = new DefaultEntryLogger(conf, ledgerDirsManager); EntryLogManagerBase entryLogManager = (EntryLogManagerBase) defaultEntryLogger.getEntryLogManager(); - Assert.assertEquals( - "EntryLogManager class type", initialEntryLogPerLedgerEnabled - ? EntryLogManagerForEntryLogPerLedger.class : EntryLogManagerForSingleEntryLog.class, - entryLogManager.getClass()); + assertEquals(initialEntryLogPerLedgerEnabled + ? EntryLogManagerForEntryLogPerLedger.class : EntryLogManagerForSingleEntryLog.class, + entryLogManager.getClass(), "EntryLogManager class type"); int numOfActiveLedgers = 10; int numEntries = 10; @@ -1873,9 +1848,9 @@ public void testSwappingEntryLogManager(boolean initialEntryLogPerLedgerEnabled, positions[i][j] = defaultEntryLogger.addEntry((long) i, generateEntry(i, j)); long entryLogId = (positions[i][j] >> 32L); if (initialEntryLogPerLedgerEnabled) { - Assert.assertEquals("EntryLogId for ledger: " + i, i, entryLogId); + assertEquals(i, entryLogId, "EntryLogId for ledger: " + i); } else { - Assert.assertEquals("EntryLogId for ledger: " + i, 0, entryLogId); + assertEquals(0, entryLogId, "EntryLogId for ledger: " + i); } } } @@ -1900,10 +1875,8 @@ public void testSwappingEntryLogManager(boolean initialEntryLogPerLedgerEnabled, new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold())); DefaultEntryLogger newEntryLogger = new DefaultEntryLogger(conf, newLedgerDirsManager); EntryLogManager newEntryLogManager = newEntryLogger.getEntryLogManager(); - Assert.assertEquals("EntryLogManager class type", - laterEntryLogPerLedgerEnabled ? EntryLogManagerForEntryLogPerLedger.class - : EntryLogManagerForSingleEntryLog.class, - newEntryLogManager.getClass()); + assertEquals(laterEntryLogPerLedgerEnabled ? EntryLogManagerForEntryLogPerLedger.class + : EntryLogManagerForSingleEntryLog.class, newEntryLogManager.getClass(), "EntryLogManager class type"); /* * read the entries (which are written with previous entrylogger) with @@ -1917,9 +1890,9 @@ public void testSwappingEntryLogManager(boolean initialEntryLogPerLedgerEnabled, long entryId = buf.readLong(); byte[] data = new byte[buf.readableBytes()]; buf.readBytes(data); - assertEquals("LedgerId ", i, ledgerId); - assertEquals("EntryId ", j, entryId); - assertEquals("Entry Data ", expectedValue, new String(data)); + assertEquals(i, ledgerId, "LedgerId "); + assertEquals(j, entryId, "EntryId "); + assertEquals(expectedValue, new String(data), "Entry Data "); } } }