|
@@ -157,6 +157,7 @@ public class TestLocalBlockCache {
|
|
|
.setFlusher(flusher)
|
|
|
.setCBlockTargetMetrics(metrics)
|
|
|
.build();
|
|
|
+ cache.start();
|
|
|
cache.put(blockID, data.getBytes(StandardCharsets.UTF_8));
|
|
|
Assert.assertEquals(1, metrics.getNumWriteOps());
|
|
|
// Please note that this read is from the local cache.
|
|
@@ -202,6 +203,7 @@ public class TestLocalBlockCache {
|
|
|
.setFlusher(flusher)
|
|
|
.setCBlockTargetMetrics(metrics)
|
|
|
.build();
|
|
|
+ cache.start();
|
|
|
cache.put(blockID, data.getBytes(StandardCharsets.UTF_8));
|
|
|
GenericTestUtils.waitFor(() -> !cache.isDirtyCache(), 100, 20 * 1000);
|
|
|
cache.close();
|
|
@@ -228,6 +230,7 @@ public class TestLocalBlockCache {
|
|
|
.setFlusher(flusher)
|
|
|
.setCBlockTargetMetrics(metrics)
|
|
|
.build();
|
|
|
+ cache.start();
|
|
|
long startTime = Time.monotonicNow();
|
|
|
for (long blockid = 0; blockid < totalBlocks; blockid++) {
|
|
|
cache.put(blockid, data.getBytes(StandardCharsets.UTF_8));
|
|
@@ -265,6 +268,7 @@ public class TestLocalBlockCache {
|
|
|
.setFlusher(flusher)
|
|
|
.setCBlockTargetMetrics(metrics)
|
|
|
.build();
|
|
|
+ cache.start();
|
|
|
// Read a non-existent block ID.
|
|
|
LogicalBlock block = cache.get(blockID);
|
|
|
Assert.assertNotNull(block);
|
|
@@ -298,6 +302,7 @@ public class TestLocalBlockCache {
|
|
|
.setFlusher(flusher)
|
|
|
.setCBlockTargetMetrics(metrics)
|
|
|
.build();
|
|
|
+ cache.start();
|
|
|
for (int x = 0; x < blockCount; x++) {
|
|
|
String data = RandomStringUtils.random(4 * 1024);
|
|
|
String dataHash = DigestUtils.sha256Hex(data);
|
|
@@ -342,7 +347,7 @@ public class TestLocalBlockCache {
|
|
|
.setFlusher(newflusher)
|
|
|
.setCBlockTargetMetrics(newMetrics)
|
|
|
.build();
|
|
|
-
|
|
|
+ newCache.start();
|
|
|
for (Map.Entry<Long, String> entry : blockShaMap.entrySet()) {
|
|
|
LogicalBlock block = newCache.get(entry.getKey());
|
|
|
String blockSha = DigestUtils.sha256Hex(block.getData().array());
|
|
@@ -471,6 +476,7 @@ public class TestLocalBlockCache {
|
|
|
.setFlusher(flusher)
|
|
|
.setCBlockTargetMetrics(metrics)
|
|
|
.build();
|
|
|
+ cache.start();
|
|
|
Assert.assertFalse(cache.isShortCircuitIOEnabled());
|
|
|
cache.put(blockID, data.getBytes(StandardCharsets.UTF_8));
|
|
|
Assert.assertEquals(1, metrics.getNumDirectBlockWrites());
|
|
@@ -498,4 +504,80 @@ public class TestLocalBlockCache {
|
|
|
GenericTestUtils.waitFor(() -> !cache.isDirtyCache(), 100, 20 * 1000);
|
|
|
cache.close();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This test writes some block to the cache and then shuts down the cache.
|
|
|
+ * The cache is then restarted to check that the
|
|
|
+ * correct number of blocks are read from Dirty Log
|
|
|
+ *
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testEmptyBlockBufferHandling() throws IOException,
|
|
|
+ InterruptedException, TimeoutException {
|
|
|
+ // Create a new config so that this tests write metafile to new location
|
|
|
+ OzoneConfiguration flushTestConfig = new OzoneConfiguration();
|
|
|
+ URL p = flushTestConfig.getClass().getResource("");
|
|
|
+ String path = p.getPath().concat(
|
|
|
+ TestOzoneContainer.class.getSimpleName()
|
|
|
+ + "/testEmptyBlockBufferHandling");
|
|
|
+ flushTestConfig.set(DFS_CBLOCK_DISK_CACHE_PATH_KEY, path);
|
|
|
+ flushTestConfig.setBoolean(DFS_CBLOCK_TRACE_IO, true);
|
|
|
+ flushTestConfig.setBoolean(DFS_CBLOCK_ENABLE_SHORT_CIRCUIT_IO, true);
|
|
|
+
|
|
|
+ String volumeName = "volume" + RandomStringUtils.randomNumeric(4);
|
|
|
+ String userName = "user" + RandomStringUtils.randomNumeric(4);
|
|
|
+ String data = RandomStringUtils.random(4 * KB);
|
|
|
+ CBlockTargetMetrics metrics = CBlockTargetMetrics.create();
|
|
|
+ ContainerCacheFlusher flusher = new ContainerCacheFlusher(flushTestConfig,
|
|
|
+ xceiverClientManager, metrics);
|
|
|
+ CBlockLocalCache cache = CBlockLocalCache.newBuilder()
|
|
|
+ .setConfiguration(flushTestConfig)
|
|
|
+ .setVolumeName(volumeName)
|
|
|
+ .setUserName(userName)
|
|
|
+ .setPipelines(getContainerPipeline(10))
|
|
|
+ .setClientManager(xceiverClientManager)
|
|
|
+ .setBlockSize(4 * KB)
|
|
|
+ .setVolumeSize(50 * GB)
|
|
|
+ .setFlusher(flusher)
|
|
|
+ .setCBlockTargetMetrics(metrics)
|
|
|
+ .build();
|
|
|
+ cache.start();
|
|
|
+ // Write data to the cache
|
|
|
+ cache.put(1, data.getBytes(StandardCharsets.UTF_8));
|
|
|
+ Assert.assertEquals(0, metrics.getNumDirectBlockWrites());
|
|
|
+ Assert.assertEquals(1, metrics.getNumWriteOps());
|
|
|
+ cache.put(2, data.getBytes(StandardCharsets.UTF_8));
|
|
|
+ Assert.assertEquals(0, metrics.getNumDirectBlockWrites());
|
|
|
+ Assert.assertEquals(2, metrics.getNumWriteOps());
|
|
|
+
|
|
|
+ // Store the previous block buffer position
|
|
|
+ Assert.assertEquals(2, metrics.getNumDirtyLogBlockUpdated());
|
|
|
+ // Simulate a shutdown by closing the cache
|
|
|
+ GenericTestUtils.waitFor(() -> !cache.isDirtyCache(), 100, 20 * 1000);
|
|
|
+ cache.close();
|
|
|
+ Assert.assertEquals(2 * (Long.SIZE/ Byte.SIZE),
|
|
|
+ metrics.getNumBytesDirtyLogWritten());
|
|
|
+ Assert.assertEquals(0, metrics.getNumFailedDirtyBlockFlushes());
|
|
|
+
|
|
|
+ // Restart cache and check that right number of entries are read
|
|
|
+ CBlockTargetMetrics newMetrics = CBlockTargetMetrics.create();
|
|
|
+ ContainerCacheFlusher newFlusher =
|
|
|
+ new ContainerCacheFlusher(flushTestConfig,
|
|
|
+ xceiverClientManager, newMetrics);
|
|
|
+ Thread fllushListenerThread = new Thread(newFlusher);
|
|
|
+ fllushListenerThread.setDaemon(true);
|
|
|
+ fllushListenerThread.start();
|
|
|
+
|
|
|
+ Thread.sleep(5000);
|
|
|
+ Assert.assertEquals(metrics.getNumDirtyLogBlockUpdated(),
|
|
|
+ newMetrics.getNumDirtyLogBlockRead());
|
|
|
+ Assert.assertEquals(newMetrics.getNumDirtyLogBlockRead()
|
|
|
+ * (Long.SIZE/ Byte.SIZE), newMetrics.getNumBytesDirtyLogReads());
|
|
|
+ // Now shutdown again, nothing should be flushed
|
|
|
+ newFlusher.shutdown();
|
|
|
+ Assert.assertEquals(0, newMetrics.getNumDirtyLogBlockUpdated());
|
|
|
+ Assert.assertEquals(0, newMetrics.getNumBytesDirtyLogWritten());
|
|
|
+ Assert.assertEquals(0, newMetrics.getNumFailedDirtyBlockFlushes());
|
|
|
+ }
|
|
|
}
|