|
@@ -518,9 +518,7 @@ public class TestLocalBlockCache {
|
|
|
// 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");
|
|
|
+ String path = p.getPath().concat(TestOzoneContainer.class.getSimpleName());
|
|
|
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);
|
|
@@ -528,6 +526,8 @@ public class TestLocalBlockCache {
|
|
|
String volumeName = "volume" + RandomStringUtils.randomNumeric(4);
|
|
|
String userName = "user" + RandomStringUtils.randomNumeric(4);
|
|
|
String data = RandomStringUtils.random(4 * KB);
|
|
|
+ List<Pipeline> pipelines = getContainerPipeline(10);
|
|
|
+
|
|
|
CBlockTargetMetrics metrics = CBlockTargetMetrics.create();
|
|
|
ContainerCacheFlusher flusher = new ContainerCacheFlusher(flushTestConfig,
|
|
|
xceiverClientManager, metrics);
|
|
@@ -535,7 +535,7 @@ public class TestLocalBlockCache {
|
|
|
.setConfiguration(flushTestConfig)
|
|
|
.setVolumeName(volumeName)
|
|
|
.setUserName(userName)
|
|
|
- .setPipelines(getContainerPipeline(10))
|
|
|
+ .setPipelines(pipelines)
|
|
|
.setClientManager(xceiverClientManager)
|
|
|
.setBlockSize(4 * KB)
|
|
|
.setVolumeSize(50 * GB)
|
|
@@ -565,9 +565,21 @@ public class TestLocalBlockCache {
|
|
|
ContainerCacheFlusher newFlusher =
|
|
|
new ContainerCacheFlusher(flushTestConfig,
|
|
|
xceiverClientManager, newMetrics);
|
|
|
- Thread fllushListenerThread = new Thread(newFlusher);
|
|
|
- fllushListenerThread.setDaemon(true);
|
|
|
- fllushListenerThread.start();
|
|
|
+ CBlockLocalCache newCache = CBlockLocalCache.newBuilder()
|
|
|
+ .setConfiguration(flushTestConfig)
|
|
|
+ .setVolumeName(volumeName)
|
|
|
+ .setUserName(userName)
|
|
|
+ .setPipelines(pipelines)
|
|
|
+ .setClientManager(xceiverClientManager)
|
|
|
+ .setBlockSize(4 * KB)
|
|
|
+ .setVolumeSize(50 * GB)
|
|
|
+ .setFlusher(newFlusher)
|
|
|
+ .setCBlockTargetMetrics(newMetrics)
|
|
|
+ .build();
|
|
|
+ newCache.start();
|
|
|
+ Thread flushListenerThread = new Thread(newFlusher);
|
|
|
+ flushListenerThread.setDaemon(true);
|
|
|
+ flushListenerThread.start();
|
|
|
|
|
|
Thread.sleep(5000);
|
|
|
Assert.assertEquals(metrics.getNumDirtyLogBlockUpdated(),
|
|
@@ -575,9 +587,104 @@ public class TestLocalBlockCache {
|
|
|
Assert.assertEquals(newMetrics.getNumDirtyLogBlockRead()
|
|
|
* (Long.SIZE/ Byte.SIZE), newMetrics.getNumBytesDirtyLogReads());
|
|
|
// Now shutdown again, nothing should be flushed
|
|
|
+ newCache.close();
|
|
|
newFlusher.shutdown();
|
|
|
Assert.assertEquals(0, newMetrics.getNumDirtyLogBlockUpdated());
|
|
|
Assert.assertEquals(0, newMetrics.getNumBytesDirtyLogWritten());
|
|
|
Assert.assertEquals(0, newMetrics.getNumFailedDirtyBlockFlushes());
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This test writes some block to the cache and then shuts down the cache
|
|
|
+ * The cache is then restarted with "short.circuit.io" disable to check
|
|
|
+ * that the blocks are read correctly from the container.
|
|
|
+ *
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testContainerWrites() 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());
|
|
|
+ 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);
|
|
|
+
|
|
|
+ XceiverClientManager xcm = new XceiverClientManager(flushTestConfig);
|
|
|
+ String volumeName = "volume" + RandomStringUtils.randomNumeric(4);
|
|
|
+ String userName = "user" + RandomStringUtils.randomNumeric(4);
|
|
|
+
|
|
|
+ int numUniqueBlocks = 4;
|
|
|
+ String[] data = new String[numUniqueBlocks];
|
|
|
+ String[] dataHash = new String[numUniqueBlocks];
|
|
|
+ for (int i = 0; i < numUniqueBlocks; i++) {
|
|
|
+ data[i] = RandomStringUtils.random(4 * KB);
|
|
|
+ dataHash[i] = DigestUtils.sha256Hex(data[i]);
|
|
|
+ }
|
|
|
+
|
|
|
+ CBlockTargetMetrics metrics = CBlockTargetMetrics.create();
|
|
|
+ ContainerCacheFlusher flusher = new ContainerCacheFlusher(flushTestConfig,
|
|
|
+ xcm, metrics);
|
|
|
+ List<Pipeline> pipelines = getContainerPipeline(10);
|
|
|
+ CBlockLocalCache cache = CBlockLocalCache.newBuilder()
|
|
|
+ .setConfiguration(flushTestConfig)
|
|
|
+ .setVolumeName(volumeName)
|
|
|
+ .setUserName(userName)
|
|
|
+ .setPipelines(pipelines)
|
|
|
+ .setClientManager(xcm)
|
|
|
+ .setBlockSize(4 * KB)
|
|
|
+ .setVolumeSize(50 * GB)
|
|
|
+ .setFlusher(flusher)
|
|
|
+ .setCBlockTargetMetrics(metrics)
|
|
|
+ .build();
|
|
|
+ cache.start();
|
|
|
+ Thread fllushListenerThread = new Thread(flusher);
|
|
|
+ fllushListenerThread.setDaemon(true);
|
|
|
+ fllushListenerThread.start();
|
|
|
+ Assert.assertTrue(cache.isShortCircuitIOEnabled());
|
|
|
+ // Write data to the cache
|
|
|
+ for (int i = 0; i < 512; i++) {
|
|
|
+ cache.put(i, data[i % numUniqueBlocks].getBytes(StandardCharsets.UTF_8));
|
|
|
+ }
|
|
|
+ // Close the cache and flush the data to the containers
|
|
|
+ cache.close();
|
|
|
+ Assert.assertEquals(0, metrics.getNumDirectBlockWrites());
|
|
|
+ Assert.assertEquals(512, metrics.getNumWriteOps());
|
|
|
+ Thread.sleep(5000);
|
|
|
+ flusher.shutdown();
|
|
|
+ Assert.assertEquals(0, metrics.getNumWriteIOExceptionRetryBlocks());
|
|
|
+ Assert.assertEquals(0, metrics.getNumWriteGenericExceptionRetryBlocks());
|
|
|
+
|
|
|
+ // Now disable DFS_CBLOCK_ENABLE_SHORT_CIRCUIT_IO and restart cache
|
|
|
+ flushTestConfig.setBoolean(DFS_CBLOCK_ENABLE_SHORT_CIRCUIT_IO, false);
|
|
|
+ CBlockTargetMetrics newMetrics = CBlockTargetMetrics.create();
|
|
|
+ ContainerCacheFlusher newFlusher =
|
|
|
+ new ContainerCacheFlusher(flushTestConfig, xcm, newMetrics);
|
|
|
+ CBlockLocalCache newCache = CBlockLocalCache.newBuilder()
|
|
|
+ .setConfiguration(flushTestConfig)
|
|
|
+ .setVolumeName(volumeName)
|
|
|
+ .setUserName(userName)
|
|
|
+ .setPipelines(pipelines)
|
|
|
+ .setClientManager(xcm)
|
|
|
+ .setBlockSize(4 * KB)
|
|
|
+ .setVolumeSize(50 * GB)
|
|
|
+ .setFlusher(newFlusher)
|
|
|
+ .setCBlockTargetMetrics(newMetrics)
|
|
|
+ .build();
|
|
|
+ newCache.start();
|
|
|
+ Assert.assertFalse(newCache.isShortCircuitIOEnabled());
|
|
|
+ // this read will be from the container, also match the hash
|
|
|
+ for (int i = 0; i < 512; i++) {
|
|
|
+ LogicalBlock block = newCache.get(i);
|
|
|
+ String readHash = DigestUtils.sha256Hex(block.getData().array());
|
|
|
+ Assert.assertEquals("File content does not match, for index:"
|
|
|
+ + i, dataHash[i % numUniqueBlocks], readHash);
|
|
|
+ }
|
|
|
+ Assert.assertEquals(0, newMetrics.getNumReadLostBlocks());
|
|
|
+ Assert.assertEquals(0, newMetrics.getNumFailedReadBlocks());
|
|
|
+ newFlusher.shutdown();
|
|
|
+ newCache.close();
|
|
|
+ }
|
|
|
}
|